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 rh...@apache.org on 2014/05/21 14:05:26 UTC

svn commit: r1596548 - in /db/derby/code/trunk: java/engine/org/apache/derby/iapi/db/ java/engine/org/apache/derby/iapi/sql/ java/engine/org/apache/derby/iapi/sql/conn/ java/engine/org/apache/derby/iapi/sql/dictionary/ java/engine/org/apache/derby/impl...

Author: rhillegas
Date: Wed May 21 12:05:25 2014
New Revision: 1596548

URL: http://svn.apache.org/r1596548
Log:
DERBY-6542: Use sequence generators to create identity values; commit derby-6542-02-af-useNewSequenceGenerator.diff.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/BulkInsertCounter.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/db/ConnectionInfo.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceGenerator.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.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/execute/AlterTableConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.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/InsertConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/InsertResultSet.java
    db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/EmptyDictionary.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AlterTableTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AutoIncrementTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IdentitySequenceTest.java
    db/derby/code/trunk/tools/jar/extraDBMSclasses.properties

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/db/ConnectionInfo.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/db/ConnectionInfo.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/db/ConnectionInfo.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/db/ConnectionInfo.java Wed May 21 12:05:25 2014
@@ -1,72 +0,0 @@
-/*
-
-   Derby - Class org.apache.derby.iapi.db.ConnectionInfo
-
-   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.db;
-
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-import org.apache.derby.iapi.sql.conn.ConnectionUtil;
-import org.apache.derby.iapi.error.StandardException;
-import org.apache.derby.iapi.error.PublicAPI;
-import java.sql.SQLException;
-
-/**
- *
- * ConnectionInfo class provides static methods for getting information 
- * related to a JDBC connection.
- * 
- * When called from within the query language,
- * each method returns information about the connection from which it was called.
- * <p>
- * Use the methods of this class only within an SQL-J statement; do not call 
- * them directly. 
- */
- 
-public abstract class ConnectionInfo
-{
-
-	/** no requirement for a constructor */
-	private ConnectionInfo() {}
-
-	
-	/**
-	 * <B>INTERNAL USE ONLY</B>
-	 * (<B>THIS METHOD MAY BE REMOVED IN A FUTURE RELEASE</B>.)
-	 * @throws SQLException on error
-	 **/
-	public static long nextAutoincrementValue(String schemaName,
-											  String tableName,
-											  String columnName
-											  )
-	     throws SQLException
-	{
-		LanguageConnectionContext lcc = ConnectionUtil.getCurrentLCC();
-		try
-		{ 
-			return 
-				lcc.nextAutoincrementValue(schemaName, tableName, columnName);
-		}
-		catch (StandardException se)
-		{ 
-			throw PublicAPI.wrapStandardException(se);
-		}
-	}	
-}
-

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java Wed May 21 12:05:25 2014
@@ -41,6 +41,7 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.TransactionController;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.DataValueFactory;
+import org.apache.derby.iapi.types.NumberDataValue;
 import org.apache.derby.iapi.types.RowLocation;
 
 
@@ -629,4 +630,17 @@ public interface Activation extends Depe
 	 */
 	public Activation getParentActivation();
 
+	/**
+	 * Called by generated code to get the next number in an ANSI/ISO sequence
+     * and advance the sequence. Raises an exception if the sequence was declared
+     * NO CYCLE and its range is exhausted.
+	 *
+     * @param sequenceUUIDstring The string value of the sequence's UUID
+     * @param typeFormatID The format id of the data type to be returned. E.g., StoredFormatIds.SQL_INTEGER_ID.
+     *
+	 * @return The next number in the sequence
+	 */
+	public NumberDataValue getCurrentValueAndAdvance
+        ( String sequenceUUIDstring, int typeFormatID )
+        throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/conn/LanguageConnectionContext.java Wed May 21 12:05:25 2014
@@ -951,8 +951,6 @@ public interface LanguageConnectionConte
 	 * to an existing table.
 	 * 
 	 * @param flag 	the value for autoincrementUpdate (TRUE or FALSE)
-	 * @see org.apache.derby.impl.sql.execute.AlterTableConstantAction#updateNewAutoincrementColumn
-	 *
 	 */
 	public void setAutoincrementUpdate(boolean flag);
 

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/BulkInsertCounter.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/BulkInsertCounter.java?rev=1596548&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/BulkInsertCounter.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/BulkInsertCounter.java Wed May 21 12:05:25 2014
@@ -0,0 +1,53 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.sql.dictionary.BulkInsertCounter
+
+   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.dictionary;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.types.NumberDataValue;
+
+/**
+ * Thin wrapper around a sequence generator to support the bulk-insert
+ * optimization used by InsertResultSet.
+ */
+
+public interface BulkInsertCounter
+{
+    /**
+     * <p>
+     * Get the next sequence number for bulk-insert.
+     * </p>
+     *
+     * @param returnValue This value is stuffed with the new sequence number.
+     */
+    public void getCurrentValueAndAdvance
+        ( NumberDataValue returnValue ) throws StandardException;
+
+    /**
+     * <p>
+     * Get the current value of the sequence generator without advancing it.
+     * May return null if the generator is exhausted.
+     * </p>
+     */
+    public Long peekAtCurrentValue() throws StandardException;
+    
+}
+

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

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionary.java Wed May 21 12:05:25 2014
@@ -1841,6 +1841,42 @@ public interface DataDictionary
 		throws StandardException;
 	
 	/**
+	 * Computes the RowLocation in SYSSEQUENCES for a particular sequence. Also
+     * constructs the sequence descriptor.
+	 * 
+	 * @param tc			Transaction Controller to use.
+	 * @param sequenceIDstring UUID of the sequence as a string
+	 * @param rowLocation OUTPUT param for returing the row location
+	 * @param sequenceDescriptor OUTPUT param for return the sequence descriptor
+     *
+	 * @exception StandardException thrown on failure.
+	 */ 
+	public void computeSequenceRowLocation
+        ( TransactionController tc, String sequenceIDstring, RowLocation[] rowLocation, SequenceDescriptor[] sequenceDescriptor )
+		throws StandardException;
+    
+	/**
+	 * Set the current value of an ANSI/ISO sequence. This method does not perform
+     * any sanity checking but assumes that the caller knows what they are doing. If the
+     * old value on disk is not what we expect it to be, then we are in a race with another
+     * session. They won and we don't update the value on disk. However, if the old value
+     * is null, that is a signal to us that we should update the value on disk anyway.
+	 * 
+	 * @param tc			Transaction Controller to use.
+	 * @param rowLocation Row in SYSSEQUENCES to update.
+     * @param wait True if we should wait for locks
+     * @param oldValue What we expect to find in the CURRENTVALUE column.
+     * @param newValue What to stuff into the CURRENTVALUE column.
+	 * 
+	 * @return Returns true if the value was successfully updated, false if we lost a race with another session.
+     *
+	 * @exception StandardException thrown on failure.
+	 */
+    public  boolean updateCurrentSequenceValue
+        ( TransactionController tc, RowLocation rowLocation, boolean wait, Long oldValue, Long newValue )
+        throws StandardException;
+    
+	/**
 	 * Get the next number from an ANSI/ISO sequence generator
      * which was created with the CREATE SEQUENCE statement. May
      * raise an exception if the sequence was defined as NO CYCLE and
@@ -2386,4 +2422,26 @@ public interface DataDictionary
      */
     public DependableFinder getColumnDependableFinder(
             int formatId, byte[] columnBitMap);
+
+    /**
+     * Get the identity generator used to support the bulk-insert optimization
+     * in InsertResultSet.
+     *
+     * @param sequenceUUIDString UUID of the sequence which backs the identity column.
+     * @param restart   True if the counter should be re-initialized to its start position.
+     */
+    public  BulkInsertCounter   getBulkInsertCounter
+        ( String sequenceUUIDString, boolean restart )
+        throws StandardException;
+
+    /**
+     * Flush the updated values of the BulkInsertCounter to disk and to the original, cached
+     * SequenceUpdater. This is used for the bulk-insert optimization in InsertResultSet.
+     *
+     * @param sequenceUUIDString UUID of the sequence which backs the identity column.
+     * @param bic   the BulkInsertCounter which generates identities for bulk insert
+     */
+    public  void   flushBulkInsertCounter
+        ( String sequenceUUIDString, BulkInsertCounter bic )
+        throws StandardException;
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java Wed May 21 12:05:25 2014
@@ -52,6 +52,7 @@ import org.apache.derby.iapi.store.acces
 import org.apache.derby.iapi.store.access.TransactionController;
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.DataValueFactory;
+import org.apache.derby.iapi.types.NumberDataValue;
 import org.apache.derby.iapi.types.RowLocation;
 import org.apache.derby.impl.sql.execute.BaseActivation;
 
@@ -824,4 +825,12 @@ final public class GenericActivationHold
 	public int getMaxDynamicResults() {
 		return ac.getMaxDynamicResults();
 	}
+
+	public NumberDataValue getCurrentValueAndAdvance
+        ( String sequenceUUIDstring, int typeFormatID )
+        throws StandardException
+    {
+        return ac.getCurrentValueAndAdvance( sequenceUUIDstring, typeFormatID );
+    }
+
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Wed May 21 12:05:25 2014
@@ -84,6 +84,7 @@ import org.apache.derby.iapi.sql.conn.La
 import org.apache.derby.iapi.sql.conn.LanguageConnectionFactory;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
 import org.apache.derby.iapi.sql.dictionary.AliasDescriptor;
+import org.apache.derby.iapi.sql.dictionary.BulkInsertCounter;
 import org.apache.derby.iapi.sql.dictionary.CatalogRowFactory;
 import org.apache.derby.iapi.sql.dictionary.CheckConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ColPermsDescriptor;
@@ -10178,7 +10179,7 @@ public final class	DataDictionaryImpl
      *
 	 * @exception StandardException thrown on failure.
 	 */ 
-	void computeSequenceRowLocation
+	public void computeSequenceRowLocation
         ( TransactionController tc, String sequenceIDstring, RowLocation[] rowLocation, SequenceDescriptor[] sequenceDescriptor )
 		throws StandardException								  
 	{
@@ -10222,7 +10223,7 @@ public final class	DataDictionaryImpl
      *
 	 * @exception StandardException thrown on failure.
 	 */
-    boolean updateCurrentSequenceValue
+    public  boolean updateCurrentSequenceValue
         ( TransactionController tc, RowLocation rowLocation, boolean wait, Long oldValue, Long newValue )
         throws StandardException
     {
@@ -10363,6 +10364,45 @@ public final class	DataDictionaryImpl
         }
     }
     
+    public  BulkInsertCounter   getBulkInsertCounter
+        ( String sequenceUUIDString, boolean restart )
+        throws StandardException
+    {
+        SequenceUpdater sequenceUpdater = null;
+
+        try {
+            sequenceUpdater = (SequenceUpdater) sequenceGeneratorCache.find( sequenceUUIDString );
+            return sequenceUpdater.getBulkInsertUpdater( restart );
+        }
+        finally
+        {
+            if ( sequenceUpdater != null )
+            {
+                sequenceGeneratorCache.release( sequenceUpdater );
+            }
+        }
+    }
+
+    public  void   flushBulkInsertCounter
+        ( String sequenceUUIDString, BulkInsertCounter bic )
+        throws StandardException
+    {
+        SequenceUpdater sequenceUpdater = null;
+
+        try {
+            sequenceUpdater = (SequenceUpdater) sequenceGeneratorCache.find( sequenceUUIDString );
+            sequenceUpdater.reset( bic.peekAtCurrentValue() );
+        }
+        finally
+        {
+            if ( sequenceUpdater != null )
+            {
+                sequenceGeneratorCache.release( sequenceUpdater );
+            }
+        }
+    }
+    
+    
     public RowLocation getRowLocationTemplate(LanguageConnectionContext lcc,
                                               TableDescriptor td)
           throws StandardException

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceGenerator.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceGenerator.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceGenerator.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceGenerator.java Wed May 21 12:05:25 2014
@@ -225,6 +225,60 @@ public class SequenceGenerator
         //
         _remainingPreallocatedValues = 1L;
     }
+
+    /**
+     * <p>
+     * Clone this sequence generator. This method supports the special bulk-insert optimization in
+     * InsertResultSet.
+     * </p>
+     *
+     * @param restart   True if the clone should be reset to start at the beginning instead of at the current value.
+     */
+    public synchronized SequenceGenerator clone( boolean restart )
+    {
+        Long    startValue;
+
+        if ( restart ) { startValue = new Long( _RESTART_VALUE ); }
+        else if ( _isExhausted ) { startValue = null; }
+        else { startValue = new Long( _currentValue ); }
+
+        return new SequenceGenerator
+            (
+             startValue,
+             _CAN_CYCLE,
+             _INCREMENT,
+             _MAX_VALUE,
+             _MIN_VALUE,
+             _RESTART_VALUE,
+             _SCHEMA_NAME,
+             _SEQUENCE_NAME,
+             _PREALLOCATOR
+             );
+    }
+    
+    /**
+     * <p>
+     * Clone this sequence generator. This method supports the special bulk-insert optimization in
+     * InsertResultSet.
+     * </p>
+     *
+     * @param newStartValue New value to start with.
+     */
+    public synchronized SequenceGenerator clone( Long newStartValue )
+    {
+        return new SequenceGenerator
+            (
+             newStartValue,
+             _CAN_CYCLE,
+             _INCREMENT,
+             _MAX_VALUE,
+             _MIN_VALUE,
+             _RESTART_VALUE,
+             _SCHEMA_NAME,
+             _SEQUENCE_NAME,
+             _PREALLOCATOR
+             );
+    }
     
     ///////////////////////////////////////////////////////////////////////////////////
     //

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/SequenceUpdater.java Wed May 21 12:05:25 2014
@@ -36,6 +36,7 @@ import org.apache.derby.iapi.services.mo
 import org.apache.derby.iapi.services.property.PropertyUtil;
 import org.apache.derby.shared.common.sanity.SanityManager;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.sql.dictionary.BulkInsertCounter;
 import org.apache.derby.iapi.sql.dictionary.SequenceDescriptor;
 import org.apache.derby.iapi.store.access.AccessFactory;
 import org.apache.derby.iapi.store.access.TransactionController;
@@ -327,6 +328,35 @@ public abstract class SequenceUpdater im
 
     /**
      * <p>
+     * Reset the sequence generator to a new start value. This is used by the special
+     * bulk-insert optimization in InsertResultSet.
+     * </p>
+     */
+    public synchronized void reset( Long newValue )
+        throws StandardException
+    {
+        // first try to reset on disk
+        updateCurrentValueOnDisk( null, newValue );
+
+        // now reset the sequence generator
+        _sequenceGenerator = _sequenceGenerator.clone( newValue );
+    }
+    
+    /**
+     * <p>
+     * Get the SequenceUpdater used for the bulk-insert optimization in InsertResultSet.
+     * </p>
+     *
+     * @param restart   True if the counter should be re-initialized to its start position.
+     */
+    public synchronized BulkInsertUpdater   getBulkInsertUpdater( boolean restart )
+        throws StandardException
+    {
+        return new BulkInsertUpdater( this, restart );
+    }
+    
+    /**
+     * <p>
      * Get the next sequence number managed by this generator and advance the number. Could raise an
      * exception if the legal range is exhausted and wrap-around is not allowed.
      * Only one thread at a time is allowed through here. We do not want a race between the
@@ -611,5 +641,41 @@ public abstract class SequenceUpdater im
         }
     }
 
+    /**
+     * <p>
+     * Implementation of SequenceUpdater for use with the bulk-insert optimization
+     * used by InsertResultSet. This BulkInsertUpdater doesn't really write to disk. It is assumed
+     * that the BulkInsertUpdater will only be used by the bulk-insert code, where the
+     * user has exclusive write-access on the table whose identity column is backed by
+     * the original SequenceUpdater. At the end of bulk-insert, the current value of the
+     * BulkInsertUpdater is written to disk by other code.
+     * </p>
+     */
+    public static final class BulkInsertUpdater extends SequenceUpdater implements BulkInsertCounter
+    {
+        public BulkInsertUpdater() { super(); }
+        public BulkInsertUpdater( SequenceUpdater originalUpdater, boolean restart )
+        {
+            _sequenceGenerator = originalUpdater._sequenceGenerator.clone( restart );
+        }
+    
+        //
+        // SequenceUpdater BEHAVIOR
+        //
+
+        protected SequenceGenerator createSequenceGenerator( TransactionController readOnlyTC )
+            throws StandardException
+        {
+            return _sequenceGenerator;
+        }
+
+        protected boolean updateCurrentValueOnDisk( TransactionController tc, Long oldValue, Long newValue, boolean wait )
+            throws StandardException
+        {
+            // always succeeds
+            return true;
+        }
+    }
+
 }
 

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=1596548&r1=1596547&r2=1596548&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 Wed May 21 12:05:25 2014
@@ -41,6 +41,7 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.IndexLister;
+import org.apache.derby.iapi.sql.dictionary.SequenceDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.execute.ConstantAction;
 import org.apache.derby.iapi.sql.execute.ExecRowBuilder;
@@ -89,6 +90,9 @@ public final class InsertNode extends DM
     private     boolean           hasJDBClimitClause; // true if using JDBC limit/offset escape syntax
 
 	protected   RowLocation[] 		autoincRowLocation;
+
+    private     String              identitySequenceUUIDString;
+    
 	/**
      * Constructor for an InsertNode.
 	 *
@@ -550,6 +554,21 @@ public final class InsertNode extends DM
                                                   null,
                                                   resultSet);
 		}
+
+        // if this is 10.11 or higher and the table has an identity column,
+        // get the uuid of the sequence generator backing the identity column
+        if (
+            targetTableDescriptor.tableHasAutoincrement() &&
+            dd.checkVersion( DataDictionary.DD_VERSION_DERBY_10_11, null )
+            )
+        {
+            SequenceDescriptor  seq = dd.getSequenceDescriptor
+                (
+                 dd.getSystemSchemaDescriptor(),
+                 TableDescriptor.makeSequenceName( targetTableDescriptor.getUUID() )
+                 );
+            identitySequenceUUIDString = seq.getUUID().toString();
+        }
         
         getCompilerContext().removePrivilegeFilter( ignorePermissions );
 		getCompilerContext().popCurrentPrivType();
@@ -811,7 +830,8 @@ public final class InsertNode extends DM
 				  null,
 				  resultSet.isOneRowResultSet(), 
 				  autoincRowLocation,
-				  inMatchingClause()
+				  inMatchingClause(),
+				  identitySequenceUUIDString
 				  );
 		}
 		else

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/AlterTableConstantAction.java Wed May 21 12:05:25 2014
@@ -60,6 +60,7 @@ import org.apache.derby.iapi.sql.diction
 import org.apache.derby.iapi.sql.dictionary.ReferencedKeyConstraintDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SPSDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
+import org.apache.derby.iapi.sql.dictionary.SequenceDescriptor;
 import org.apache.derby.iapi.sql.dictionary.StatisticsDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TriggerDescriptor;
@@ -1235,11 +1236,11 @@ class AlterTableConstantAction extends D
 		// now add the column to the tables column descriptor list.
 		td.getColumnDescriptorList().add(columnDescriptor);
 
-		if (columnDescriptor.isAutoincrement())
+        if (SanityManager.DEBUG)
 		{
-            updateNewAutoincrementColumn(columnInfo[ix].name,
-										 columnInfo[ix].autoincStart,
-										 columnInfo[ix].autoincInc);
+            // support for adding identity columns was removed before Derby
+            // was open-sourced
+			SanityManager.ASSERT( !columnDescriptor.isAutoincrement(), "unexpected attempt to add an identity column" );
 		}
 
 		// Update the new column to its default, if it has a non-null default
@@ -2200,11 +2201,37 @@ class AlterTableConstantAction extends D
             //
             if ( dd.checkVersion( DataDictionary.DD_VERSION_DERBY_10_11, null ) )
             {
+                Long    currentValue = null;
+                
+                // don't clobber the current value of the sequence generator if we
+                // are just changing the increment. see DERBY-6579.
+                if ( columnInfo[ix].action == ColumnInfo.MODIFY_COLUMN_DEFAULT_INCREMENT )
+                {
+                    currentValue = dd.peekAtIdentity( td.getSchemaName(), td.getName() );
+                }
+
                 DropTableConstantAction.dropIdentitySequence( dd, td, activation );
 
+                // recreate the sequence
+                String      sequenceName = TableDescriptor.makeSequenceName( td.getUUID() );
                 CreateSequenceConstantAction   csca = CreateTableConstantAction.makeCSCA
-                    ( columnInfo[ix], TableDescriptor.makeSequenceName( td.getUUID() ) );
+                    ( columnInfo[ix], sequenceName );
                 csca.executeConstantAction( activation );
+
+                // reset the current value of the sequence generator as necessary
+                if ( columnInfo[ix].action == ColumnInfo.MODIFY_COLUMN_DEFAULT_INCREMENT )
+                {
+                    SequenceDescriptor  sequence = dd.getSequenceDescriptor
+                        ( dd.getSystemSchemaDescriptor(), sequenceName );
+                    RowLocation[] rowLocation = new RowLocation[ 1 ];
+                    SequenceDescriptor[] sequenceDescriptor = new SequenceDescriptor[ 1 ];
+            
+                    dd.computeSequenceRowLocation
+                        ( tc, sequence.getUUID().toString(), rowLocation, sequenceDescriptor );
+                    dd.updateCurrentSequenceValue
+                        ( tc, rowLocation[ 0 ], true, null, currentValue );
+                }
+                
             }
         }
 	}
@@ -3472,72 +3499,6 @@ class AlterTableConstantAction extends D
 	}
 
 	/**
-	 * Update values in a new autoincrement column being added to a table.
-	 * This is similar to updateNewColumnToDefault whereby we issue an
-	 * update statement using a nested connection. The UPDATE statement 
-	 * uses a static method in ConnectionInfo (which is not documented) 
-	 * which returns the next value to be inserted into the autoincrement
-	 * column.
-	 *
-	 * @param columnName autoincrement column name that is being added.
-	 * @param initial    initial value of the autoincrement column.
-	 * @param increment  increment value of the autoincrement column.
-	 *
-	 * @see #updateNewColumnToDefault
-	 */
-    private void updateNewAutoincrementColumn(String columnName, long initial,
-											 long increment)
-		throws StandardException
-	{
-		// Don't throw an error in bind when we try to update the 
-		// autoincrement column.
-		lcc.setAutoincrementUpdate(true);
-
-		lcc.autoincrementCreateCounter(td.getSchemaName(),
-									   td.getName(),
-                                       columnName, Long.valueOf(initial),
-									   increment, 0);
-		// the sql query is.
-		// UPDATE table 
-		//  set ai_column = ConnectionInfo.nextAutoincrementValue(
-		//							schemaName, tableName, 
-		//							columnName)
-        String updateStmt = "UPDATE " +
-            IdUtil.mkQualifiedName(td.getSchemaName(), td.getName()) +
-            " SET " + IdUtil.normalToDelimited(columnName) + "=" +
-			"org.apache.derby.iapi.db.ConnectionInfo::" + 
-			"nextAutoincrementValue(" + 
-            StringUtil.quoteStringLiteral(td.getSchemaName()) + "," +
-            StringUtil.quoteStringLiteral(td.getName()) + "," +
-            StringUtil.quoteStringLiteral(columnName) + ")";
-
-
-
-		try
-		{
-			AlterTableConstantAction.executeUpdate(lcc, updateStmt);
-		}
-		catch (StandardException se)
-		{
-			if (se.getMessageId().equals(SQLState.LANG_OUTSIDE_RANGE_FOR_DATATYPE))
-			{
-				// If overflow, override with more meaningful message.
-				throw StandardException.newException(SQLState.LANG_AI_OVERFLOW,
-													 se,
-													 td.getName(),
-													 columnName);
-			}
-			throw se;
-		}
-		finally
-		{
-			// and now update the autoincrement value.
-			lcc.autoincrementFlushCache(td.getUUID());
-			lcc.setAutoincrementUpdate(false);		
-		}
-
-	} 
-	/**
 	 * Make sure that the columns are non null
 	 * If any column is nullable, check that the data is null.
 	 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java Wed May 21 12:05:25 2014
@@ -754,7 +754,7 @@ public abstract class BaseActivation imp
      *
 	 * @return The next number in the sequence
 	 */
-	protected NumberDataValue getCurrentValueAndAdvance
+	public NumberDataValue getCurrentValueAndAdvance
         ( String sequenceUUIDstring, int typeFormatID )
 	       throws StandardException
 	{

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=1596548&r1=1596547&r2=1596548&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 Wed May 21 12:05:25 2014
@@ -783,6 +783,7 @@ public class GenericConstantActionFactor
 	 *  @param autoincRowLocation array of row locations into syscolumns for
 	                              autoincrement columns
 	 *  @param underMerge   True if this is an INSERT action of a MERGE statement.
+	 *  @param identitySequenceUUIDString   For 10.11 and higher, the handle on the sequence for the identity column
 	 *
 	 * @exception StandardException		Thrown on failure
 	 */
@@ -811,7 +812,8 @@ public class GenericConstantActionFactor
 								Object[]			ddlList,
 								boolean				singleRowSource,
 								RowLocation[]		autoincRowLocation,
-								boolean		underMerge
+								boolean		underMerge,
+								String		identitySequenceUUIDString
 							)
 			throws StandardException
 	{
@@ -833,7 +835,8 @@ public class GenericConstantActionFactor
 										indexedCols,
 										singleRowSource,
 										autoincRowLocation,
-										underMerge
+										underMerge,
+										identitySequenceUUIDString
 										);
 	}
 

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=1596548&r1=1596547&r2=1596548&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 Wed May 21 12:05:25 2014
@@ -93,6 +93,8 @@ public class InsertConstantAction extend
     private transient   int firstAutoGenColumn = -1;
     final public boolean hasDeferrableChecks;
 
+    String  identitySequenceUUIDString;
+
 	// CONSTRUCTORS
 
 	/**
@@ -130,6 +132,7 @@ public class InsertConstantAction extend
 	 *  @param autoincRowLocation Array of rowlocations of autoincrement values
 	 * 							  in SYSCOLUMNS for each ai column.
 	 *  @param underMerge   True if this is an action of a MERGE statement.
+	 *  @param identitySequenceUUIDString   For 10.11 and higher, the handle on the sequence for the identity column
 	 */
     InsertConstantAction(TableDescriptor tableDescriptor,
 								long				conglomId,
@@ -149,7 +152,8 @@ public class InsertConstantAction extend
 								boolean[]			indexedCols,
 								boolean				singleRowSource,
                                 RowLocation[]		autoincRowLocation,
-								boolean		underMerge)
+								boolean		underMerge,
+								String		identitySequenceUUIDString)
 	{
 		super(conglomId, 
 			  heapSCOCI,
@@ -177,6 +181,7 @@ public class InsertConstantAction extend
 		this.autoincIncrement = tableDescriptor.getAutoincIncrementArray();
 		this.indexNames = indexNames;
         this.hasDeferrableChecks = hasDeferrableChecks;
+        this.identitySequenceUUIDString = identitySequenceUUIDString;
 	}
 
 	// INTERFACE METHODS
@@ -212,6 +217,7 @@ public class InsertConstantAction extend
 		}
 		
 		autoincIncrement = ArrayUtil.readLongArray(in);
+        identitySequenceUUIDString = (String) in.readObject();
 	}
 
 
@@ -233,6 +239,7 @@ public class InsertConstantAction extend
 		out.writeObject(tableName);
 		ArrayUtil.writeArray(out, columnNames);
 		ArrayUtil.writeLongArray(out, autoincIncrement);
+        out.writeObject( identitySequenceUUIDString );
 	}
 
     /** Get the 0-based position of the autogenerated column */

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=1596548&r1=1596547&r2=1596548&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 Wed May 21 12:05:25 2014
@@ -46,6 +46,7 @@ import org.apache.derby.iapi.sql.ResultS
 import org.apache.derby.iapi.sql.StatementUtil;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.sql.dictionary.BulkInsertCounter;
 import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptor;
@@ -164,7 +165,8 @@ class InsertResultSet extends DMLWriteRe
 	 * keeps track of autoincrement values that are generated by 
 	 * getSetAutoincrementValues.
 	 */
-	private NumberDataValue				aiCache[];
+	private DataValueDescriptor				aiCache[];
+    private BulkInsertCounter[]                 bulkInsertCounters;
     private BackingStoreHashtable   deferredChecks; // cached ref.
     private List<UUID>              violatingCheckConstraints;
 	/**
@@ -177,6 +179,8 @@ class InsertResultSet extends DMLWriteRe
 	private long					identityVal;  //support of IDENTITY_LOCAL_VAL function
 	private boolean					setIdentity;
 	
+    private String              identitySequenceUUIDString;
+
 	// TargetResultSet interface
 
 	/**
@@ -354,6 +358,7 @@ class InsertResultSet extends DMLWriteRe
         this.schemaName = schemaName;
         this.tableName = tableName;
 		heapConglom = constants.conglomId;
+        identitySequenceUUIDString = constants.identitySequenceUUIDString;
 
         tc = activation.getTransactionController();
 		fkInfoArray = constants.getFKInfo();
@@ -372,15 +377,15 @@ class InsertResultSet extends DMLWriteRe
 
 		if ((rla = constants.getAutoincRowLocation()) != null)
 		{
-			aiCache = 
-				new NumberDataValue[rla.length];
+			aiCache = new DataValueDescriptor[ rla.length ];
+			bulkInsertCounters = new BulkInsertCounter[ rla.length ];
 			for (int i = 0; i < resultDescription.getColumnCount(); i++)
 			{
 				if (rla[i] == null)
 					continue;
 				ResultColumnDescriptor rcd = 
 					resultDescription.getColumnDescriptor(i + 1);
-				aiCache[i] = (NumberDataValue)rcd.getType().getNull();
+				aiCache[i] = rcd.getType().getNull();
 			}
 		}
 
@@ -725,7 +730,6 @@ class InsertResultSet extends DMLWriteRe
 		getSetAutoincrementValue(int columnPosition, long increment)
 		throws StandardException
 	{
-        NumberDataValue dvd;
 		int index = columnPosition - 1;	// all our indices are 0 based.
 
 		/* As in DB2, only for single row insert: insert into t1(c1) values (..) do
@@ -741,140 +745,191 @@ class InsertResultSet extends DMLWriteRe
 
   		if (bulkInsert)
   		{
-			ColumnDescriptor cd = td.getColumnDescriptor(columnPosition);
-			long ret;
-
-			// for bulk insert we have the table descriptor
-			//			System.out.println("in bulk insert");
-			if (aiCache[index].isNull())
-			{
-                long startValue;
-
-				if (bulkInsertReplace)
-				{
-					startValue = cd.getAutoincStart();
-				}
-				else
-				{
-					dvd = dd.getSetAutoincrementValue(
-						    constants.autoincRowLocation[index],
-							tc, false, aiCache[index], true);
-					startValue = dvd.getLong();
-				}
-				lcc.autoincrementCreateCounter(td.getSchemaName(),
-											   td.getName(),
-											   cd.getColumnName(),
-                                               Long.valueOf(startValue),
-											   increment,
-											   columnPosition);
-			
-			}  		
-			ret = lcc.nextAutoincrementValue(td.getSchemaName(),
-											 td.getName(),
-											 cd.getColumnName());
-			aiCache[columnPosition - 1].setValue(ret);
+            if ( identitySequenceUUIDString == null )
+            {
+                getOldStyleBulkInsertValue( index, increment );
+            }
+            else
+            {
+                if ( bulkInsertCounters[ index ] == null )
+                {
+                    bulkInsertCounters[ index ] = dd.getBulkInsertCounter( identitySequenceUUIDString, bulkInsertReplace );
+                }
+                bulkInsertCounters[ index ].getCurrentValueAndAdvance( (NumberDataValue) aiCache[ index ] );
+            }
 		}	
 		else
 		{
 			NumberDataValue newValue;
-            TransactionController nestedTC = null;
-            TransactionController tcToUse;
-
-			try
-			{
-                // DERBY-5780, defaulting log syncing to false, which improves
-                // performance of identity value generation.  If system 
-                // crashes may reuse an identity value because commit did not
-                // sync, but only if no subsequent user transaction has 
-                // committed or aborted and thus no row can exist that used
-                // the previous value.  Without this identity values pay
-                // a synchronous I/O to the log file for each new value no
-                // matter how many are inserted in a single transaction.
-				nestedTC = tc.startNestedUserTransaction(false, false);
-				tcToUse = nestedTC;
-			}
-			catch (StandardException se)
-			{
-				// If I cannot start a Nested User Transaction use the parent
-				// transaction to do all the work.
-				tcToUse = tc;
-			}
-
-			try 
-			{
-				/* If tcToUse == tc, then we are using parent xaction-- this
-				   can happen if for some reason we couldn't start a nested
-				   transaction
-				*/
-				newValue = dd.getSetAutoincrementValue(
-						   constants.autoincRowLocation[index],
-						   tcToUse, true, aiCache[index], (tcToUse == tc));
-			}
-			catch (StandardException se)
-			{
-				if (tcToUse == tc)
-				{
-					/* we've using the parent xaction and we've timed out; just
-					   throw an error and exit.
-					*/
-					throw se;
-				}
 
-				if (
-                    se.getMessageId().equals(SQLState.LOCK_TIMEOUT) ||
-                    se.getMessageId().equals(SQLState.SELF_DEADLOCK)
-                    )
-				{
-					// if we couldn't do this with a nested xaction, retry with
-					// parent-- we need to wait this time!
-					newValue = dd.getSetAutoincrementValue(
-									constants.autoincRowLocation[index],
-									tc, true, aiCache[index], true);
-				}
-				else if (se.getMessageId().equals(SQLState.LANG_OUTSIDE_RANGE_FOR_DATATYPE))
-				{
-					// if we got an overflow error, throw a more meaningful
-					// error message
-					throw StandardException.newException(
-												 SQLState.LANG_AI_OVERFLOW,
-												 se,
-												 constants.getTableName(),
-												 constants.getColumnName(index));
-				}
-				else throw se;
-			}
-			finally 
-			{
-				// no matter what, commit the nested transaction; if something
-				// bad happened in the child xaction lets not abort the parent
-				// here.
-                
-				if (nestedTC != null)
-				{
-                    // DERBY-5493 - prior to fix all nested user update 
-                    // transactions did a nosync commit when commit() was 
-                    // called, this default has been changed to do synced 
-                    // commit.  Changed this commit to be commitNoSync to
-                    // not introduce performce degredation for autoincrement
-                    // keys.  As before, if server crashes the changes 
-                    // made in the nested transaction may be lost.  If any
-                    // subsequent user transaction is commited, including any
-                    // inserts that would depend on the autoincrement value
-                    // change then the nested tranaction is guaranteed on
-                    // system crash.
-					nestedTC.commitNoSync(TransactionController.RELEASE_LOCKS);
-					nestedTC.destroy();
-				}
-			}
+            //
+            // If there is a sequence generator uuid, then the database is at level
+            // 10.11 or higher and we use the sequence generator to get the next
+            // identity value. Otherwise, we use old-style logic.
+            //
+            if ( identitySequenceUUIDString == null )
+            {
+                newValue = getOldStyleIdentityValue( index );
+            }
+            else
+            {
+                newValue = activation.getCurrentValueAndAdvance
+                    ( identitySequenceUUIDString, aiCache[ index ].getTypeFormatId() );
+            }
+            
 			aiCache[index] = newValue;
 			if (setIdentity)
+            {
 				identityVal = newValue.getLong();
+            }
 		}
 
-		return aiCache[index];
-		
+		return (NumberDataValue) aiCache[index];
 	}
 
+     /**
+      * Identity generation logic for bulk-insert used in pre-10.11 databases.
+      *
+      * @param index   0-based index into aiCache
+      */
+     private void    getOldStyleBulkInsertValue( int index, long increment )
+         throws StandardException
+     {
+         NumberDataValue dvd;
+         int columnPosition = index + 1;
+         ColumnDescriptor cd = td.getColumnDescriptor(columnPosition);
+         long ret;
+ 
+         // for bulk insert we have the table descriptor
+         //			System.out.println("in bulk insert");
+         if (aiCache[index].isNull())
+         {
+             long startValue;
+ 
+             if (bulkInsertReplace)
+             {
+                 startValue = cd.getAutoincStart();
+             }
+             else
+             {
+                 dvd = dd.getSetAutoincrementValue(
+                                                   constants.autoincRowLocation[index],
+                                                   tc, false, (NumberDataValue) aiCache[index], true);
+                 startValue = dvd.getLong();
+             }
+             lcc.autoincrementCreateCounter(td.getSchemaName(),
+                                            td.getName(),
+                                            cd.getColumnName(),
+                                            Long.valueOf(startValue),
+                                            increment,
+                                            columnPosition);
+ 			
+         }  		
+         ret = lcc.nextAutoincrementValue(td.getSchemaName(),
+                                          td.getName(),
+                                          cd.getColumnName());
+         aiCache[columnPosition - 1].setValue(ret);
+     }
+ 
+     /**
+      * Identity generation logic used in pre-10.11 databases.
+      *
+      * @param index   0-based index into aiCache
+      */
+     private NumberDataValue getOldStyleIdentityValue( int index )
+         throws StandardException
+     {
+         NumberDataValue newValue;
+         TransactionController nestedTC = null;
+         TransactionController tcToUse;
+ 
+         try
+         {
+             // DERBY-5780, defaulting log syncing to false, which improves
+             // performance of identity value generation.  If system 
+             // crashes may reuse an identity value because commit did not
+             // sync, but only if no subsequent user transaction has 
+             // committed or aborted and thus no row can exist that used
+             // the previous value.  Without this identity values pay
+             // a synchronous I/O to the log file for each new value no
+             // matter how many are inserted in a single transaction.
+             nestedTC = tc.startNestedUserTransaction(false, false);
+             tcToUse = nestedTC;
+         }
+         catch (StandardException se)
+         {
+             // If I cannot start a Nested User Transaction use the parent
+             // transaction to do all the work.
+             tcToUse = tc;
+         }
+ 
+         try 
+         {
+             /* If tcToUse == tc, then we are using parent xaction-- this
+                can happen if for some reason we couldn't start a nested
+                transaction
+             */
+             newValue = dd.getSetAutoincrementValue(
+                                                    constants.autoincRowLocation[index],
+                                                    tcToUse, true, (NumberDataValue) aiCache[index], (tcToUse == tc));
+         }
+         catch (StandardException se)
+         {
+             if (tcToUse == tc)
+             {
+                 /* we've using the parent xaction and we've timed out; just
+                    throw an error and exit.
+                 */
+                 throw se;
+             }
+ 
+             if (se.getMessageId().equals(SQLState.LOCK_TIMEOUT))
+             {
+                 // if we couldn't do this with a nested xaction, retry with
+                 // parent-- we need to wait this time!
+                 newValue = dd.getSetAutoincrementValue(
+                                                        constants.autoincRowLocation[index],
+                                                        tc, true, (NumberDataValue) aiCache[index], true);
+             }
+             else if (se.getMessageId().equals(SQLState.LANG_OUTSIDE_RANGE_FOR_DATATYPE))
+             {
+                 // if we got an overflow error, throw a more meaningful
+                 // error message
+                 throw StandardException.newException(
+                                                      SQLState.LANG_AI_OVERFLOW,
+                                                      se,
+                                                      constants.getTableName(),
+                                                      constants.getColumnName(index));
+             }
+             else throw se;
+         }
+         finally 
+         {
+             // no matter what, commit the nested transaction; if something
+             // bad happened in the child xaction lets not abort the parent
+             // here.
+                 
+             if (nestedTC != null)
+             {
+                 // DERBY-5493 - prior to fix all nested user update 
+                 // transactions did a nosync commit when commit() was 
+                 // called, this default has been changed to do synced 
+                 // commit.  Changed this commit to be commitNoSync to
+                 // not introduce performce degredation for autoincrement
+                 // keys.  As before, if server crashes the changes 
+                 // made in the nested transaction may be lost.  If any
+                 // subsequent user transaction is commited, including any
+                 // inserts that would depend on the autoincrement value
+                 // change then the nested tranaction is guaranteed on
+                 // system crash.
+                 nestedTC.commitNoSync(TransactionController.RELEASE_LOCKS);
+                 nestedTC.destroy();
+             }
+         }
+ 
+         return newValue;
+     }
+     
 	// Is sourceResultSet a RowResultSet (values clause)?
 	private boolean isSourceRowResultSet ()
 	{
@@ -1464,7 +1519,24 @@ class InsertResultSet extends DMLWriteRe
 		*/
 		dd.startWriting(lcc);
 
-		lcc.autoincrementFlushCache(constants.targetUUID);
+        //
+        // If we were doing bulkInsert, then we need to flush the last
+        // identity value to disk.
+        //
+        if ( identitySequenceUUIDString == null )
+        {
+            lcc.autoincrementFlushCache(constants.targetUUID);
+        }
+        else
+        {
+            for ( BulkInsertCounter bic : bulkInsertCounters )
+            {
+                if ( bic != null )
+                {
+                    dd.flushBulkInsertCounter( identitySequenceUUIDString, bic );
+                }
+            }
+        }
 
 		// invalidate any prepared statements that
 		// depended on this table (including this one)

Modified: db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/EmptyDictionary.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/EmptyDictionary.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/EmptyDictionary.java (original)
+++ db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/EmptyDictionary.java Wed May 21 12:05:25 2014
@@ -38,6 +38,7 @@ import org.apache.derby.iapi.sql.compile
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
 import org.apache.derby.iapi.sql.dictionary.AliasDescriptor;
+import org.apache.derby.iapi.sql.dictionary.BulkInsertCounter;
 import org.apache.derby.iapi.sql.dictionary.ColPermsDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptor;
@@ -752,6 +753,21 @@ public class EmptyDictionary implements 
 		return null;
 	}
 
+	public void computeSequenceRowLocation
+        ( TransactionController tc, String sequenceIDstring, RowLocation[] rowLocation, SequenceDescriptor[] sequenceDescriptor )
+		throws StandardException
+    {
+		// Auto-generated method stub
+    }
+
+    public  boolean updateCurrentSequenceValue
+        ( TransactionController tc, RowLocation rowLocation, boolean wait, Long oldValue, Long newValue )
+        throws StandardException
+    {
+		// Auto-generated method stub
+        return true;
+    }
+               
     public void getCurrentValueAndAdvance
         ( String sequenceUUIDstring, NumberDataValue returnValue )
         throws StandardException
@@ -975,4 +991,13 @@ public class EmptyDictionary implements 
             int formatId, byte[] columnBitMap) {
         return null;
     }
+
+    public  BulkInsertCounter   getBulkInsertCounter
+        ( String sequenceUUIDString, boolean restart )
+    { return null; }
+    
+    public  void   flushBulkInsertCounter
+        ( String sequenceUUIDString, BulkInsertCounter bic )
+        throws StandardException
+    {}
 }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AlterTableTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AlterTableTest.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AlterTableTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AlterTableTest.java Wed May 21 12:05:25 2014
@@ -1145,33 +1145,16 @@ public final class AlterTableTest extend
     {
 	// First, the repro from the Jira issue originally logged:
 	Statement st = createStatement();
+    st.executeUpdate( "set schema app" );
 	st.executeUpdate("create table d4693" +
 		"(a int generated always as identity, b int)");
-        JDBC.assertFullResultSet(st.executeQuery(
-                "select columnname,columnnumber,columndatatype," +
-		"       autoincrementvalue," +
-		"       autoincrementstart," +
-		"       autoincrementinc" +
-		" from sys.syscolumns where " +
-		"      columnname = 'A' and " +
-		"      referenceid in (select tableid " +
-                "             from sys.systables where tablename = 'D4693')"),
-                new String[][]{ {"A","1","INTEGER NOT NULL","1","1","1"} });
+    vetSequenceState( "D4693", "INTEGER NOT NULL", 1, 1, 1 );
 	st.executeUpdate("insert into d4693 (b) values (1)");
 	st.executeUpdate("rename column d4693.a to a2");
-        JDBC.assertFullResultSet(st.executeQuery(
-                "select columnname,columnnumber,columndatatype," +
-		"       autoincrementvalue," +
-		"       autoincrementstart," +
-		"       autoincrementinc" +
-		" from sys.syscolumns where " +
-		"      columnname = 'A2' and " +
-		"      referenceid in (select tableid " +
-                "             from sys.systables where tablename = 'D4693')"),
-                new String[][]{ {"A2","1","INTEGER NOT NULL","2","1","1"} });
+    vetSequenceState( "D4693", "INTEGER NOT NULL", 2, 1, 1 );
 	st.executeUpdate("insert into d4693 (b) values (2)");
         JDBC.assertFullResultSet(st.executeQuery(
-                "select a2, b from d4693"),
+                "select a2, b from d4693 order by a2"),
                 new String[][]{ {"1", "1"}, {"2", "2"} });
         st.executeUpdate("drop table d4693");
 
@@ -1202,6 +1185,37 @@ public final class AlterTableTest extend
 	    st.executeUpdate("drop table d4693");
 	}
     }
+    private void    vetSequenceState
+        ( String tableName,
+          String columnDatatype,
+          long currentValue,
+          long startValue,
+          long stepValue
+          )
+        throws Exception
+    {
+        Connection  conn = getConnection();
+        String  sequenceName = IdentitySequenceTest.getIdentitySequenceName( conn, tableName );
+        ResultSet   rs = conn.prepareStatement
+            (
+             "select s.sequenceDatatype, s.startValue, s.increment\n" +
+             "from sys.syssequences s\n" +
+             "where sequenceName = '" + sequenceName + "'"
+             ).executeQuery();
+        String[][]  expectedRows = new String[][]
+        {
+            { columnDatatype, Long.toString( startValue ), Long.toString( stepValue ) }
+        };
+        JDBC.assertFullResultSet( rs,expectedRows );
+
+        rs = conn.prepareStatement
+            (
+             "values syscs_util.syscs_peek_at_identity( 'APP', '" + tableName.toUpperCase() + "' )"
+             ).executeQuery();
+        expectedRows = new String[][] { { Long.toString( currentValue ) } };
+        JDBC.assertFullResultSet( rs,expectedRows );
+    }
+    
     private void checkValStartInc(Statement st, String nm, String v,
 					String s, String inc)
 	throws Exception
@@ -3168,6 +3182,8 @@ public final class AlterTableTest extend
             throws Exception {
         Statement st = createStatement();
 
+        st.executeUpdate( "set schema app" );
+
         st.executeUpdate(
                 "create table d3175 (x varchar(12), y varchar(12), " +
                 "id int primary key generated by default as identity)");
@@ -3189,17 +3205,7 @@ public final class AlterTableTest extend
         st.executeUpdate("alter table d3175 drop column y");
         st.executeUpdate("insert into d3175(x) values 'a'");
 
-        rs =
-                st.executeQuery(
-                " select COLUMNNAME, COLUMNNUMBER, COLUMNDATATYPE, " +
-                " COLUMNDEFAULT, AUTOINCREMENTVALUE, AUTOINCREMENTSTART, " +
-                " AUTOINCREMENTINC,  TABLENAME, TABLETYPE, LOCKGRANULARITY " +
-                " from sys.syscolumns c,sys.systables t " +
-                "where c.referenceid = t.tableid and t.tablename='D3175'");
-        JDBC.assertUnorderedResultSet(rs, new String[][]{
-                    {"X", "1", "VARCHAR(12)", null, null, null, null, "D3175", "T", "R"},
-                    {"ID", "2", "INTEGER NOT NULL", "GENERATED_BY_DEFAULT", "3", "1", "1", "D3175", "T", "R"}
-                });
+        vetSequenceState( "D3175", "INTEGER NOT NULL", 3, 1, 1 );
     }
 
 // JIRA 3177 appears to be aduplicate of JIRA 3175, but 

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AutoIncrementTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AutoIncrementTest.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AutoIncrementTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/AutoIncrementTest.java Wed May 21 12:05:25 2014
@@ -223,11 +223,32 @@ public class AutoIncrementTest extends B
 		rs = s.executeQuery("select * from ai_short order by i");
 		String[][]expectedRows=new String[][]{{"0","0"},{"1","2"},{"2","4"},{"33","6"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
-		rs = s.executeQuery("select COLUMNNAME, AUTOINCREMENTVALUE, AUTOINCREMENTSTART, AUTOINCREMENTINC from sys.syscolumns where COLUMNNAME = 'AIS'");
-		expectedRows=new String[][]{{"AIS","8","0","2"}};
-		JDBC.assertFullResultSet(rs,expectedRows);
-
+        vetSequenceState( "ai_short", 8, 0, 2 );
 	}
+    private void    vetSequenceState( String tableName, long currentValue, long startValue, long stepValue )
+        throws Exception
+    {
+        Connection  conn = getConnection();
+        String  sequenceName = IdentitySequenceTest.getIdentitySequenceName( conn, tableName );
+        ResultSet   rs = conn.prepareStatement
+            (
+             "select s.startValue, s.increment\n" +
+             "from sys.syssequences s\n" +
+             "where sequenceName = '" + sequenceName + "'"
+             ).executeQuery();
+        String[][]  expectedRows = new String[][]
+        {
+            { Long.toString( startValue ), Long.toString( stepValue ) }
+        };
+        JDBC.assertFullResultSet( rs,expectedRows );
+
+        rs = conn.prepareStatement
+            (
+             "values syscs_util.syscs_peek_at_identity( 'APP', '" + tableName.toUpperCase() + "' )"
+             ).executeQuery();
+        expectedRows = new String[][] { { Long.toString( currentValue ) } };
+        JDBC.assertFullResultSet( rs,expectedRows );
+    }
 	public void testonegeneratedcolumn() throws Exception
 	{
 		//-- table with one generated column spec should succeed
@@ -380,18 +401,14 @@ public class AutoIncrementTest extends B
 		rs=s.executeQuery("select * from tab1 order by s1");
 		expectedRows=new String[][]{{"1","1"},{"2","2"},{"3","3"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
-		rs=s.executeQuery("select b.tablename, a.autoincrementvalue, a.autoincrementstart, a.autoincrementinc from sys.syscolumns a, sys.systables b where a.referenceid=b.tableid and a.columnname ='S1' and b.tablename = 'TAB1'");
-		expectedRows=new String[][]{{"TAB1","4","1","1"}};
-		JDBC.assertFullResultSet(rs,expectedRows);
+        vetSequenceState( "TAB1", 4, 1, 1 );
 		s.executeUpdate("create table tab2 (lvl int, s1  bigint generated always as identity)");
 		s.executeUpdate("create trigger tab1_after2 after insert on tab3 referencing new as newrow for each row insert into tab2 (lvl) values 1,2,3");
 		s.executeUpdate("insert into tab3 values null");
 		rs=s.executeQuery("select * from tab2 order by lvl");
 		expectedRows=new String[][]{{"1","1"},{"2","2"},{"3","3"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
-		rs=s.executeQuery("select b.tablename, a.autoincrementvalue, a.autoincrementstart, a.autoincrementinc from sys.syscolumns a, sys.systables b where a.referenceid=b.tableid and a.columnname ='S1' and b.tablename = 'TAB2'");
-		expectedRows=new String[][]{{"TAB2","4","1","1"}};
-		JDBC.assertFullResultSet(rs,expectedRows);
+        vetSequenceState( "TAB2", 4, 1, 1 );
 	}
 	public void testadditionalSysCol() throws Exception
 	{
@@ -492,13 +509,24 @@ public class AutoIncrementTest extends B
 		JDBC.assertFullResultSet(rs,expectedRows);
 		s.execute("set isolation serializable");
 		rs=s.executeQuery("select columnname, autoincrementvalue from sys.syscolumns where columnname = 'YYYY'");
-		expectedRows=new String[][]{{"APP     ","UserTran","TABLE   ","1   ","S   ","SYSCOLUMNS  ","GRANT","ACTIVE"}};
+		expectedRows=new String[][]
+            {
+                {"APP     ","UserTran","TABLE   ","1   ","S   ","SYSCOLUMNS  ","GRANT","ACTIVE"}
+            };
 		rs=s.executeQuery("select * from lock_table order by tabname, type desc, mode, cnt");
-		expectedRows=new String[][]{{"APP     ","UserTran","TABLE   ","1  ","S","SYSCOLUMNS  ","GRANT","ACTIVE"}};
+		expectedRows=new String[][]
+            {
+                {"APP     ","UserTran","TABLE   ","1  ","S","SYSCOLUMNS  ","GRANT","ACTIVE"}
+            };
 		JDBC.assertFullResultSet(rs,expectedRows);
 		s.execute("insert into lockt1 (x) values (3)");
 		rs=s.executeQuery("select * from lock_table order by tabname, type desc, mode, cnt");
-		expectedRows=new String[][]{{"APP     ","UserTran","TABLE   ","1  ","IX","LOCKT1      ","GRANT","ACTIVE"},{"APP     ","UserTran","ROW     ","1  ","X","LOCKT1      ","GRANT","ACTIVE"},{"APP     ","UserTran","TABLE   ","1  ","IX","SYSCOLUMNS  ","GRANT","ACTIVE"},{"APP     ","UserTran","TABLE   ","1  ","S","SYSCOLUMNS  ","GRANT","ACTIVE"},{"APP     ","UserTran","ROW     ","2  ","X","SYSCOLUMNS  ","GRANT","ACTIVE"}};
+		expectedRows=new String[][]
+            {
+                {"APP     ","UserTran","TABLE   ","1  ","IX","LOCKT1      ","GRANT","ACTIVE"},
+                {"APP     ","UserTran","ROW     ","1  ","X","LOCKT1      ","GRANT","ACTIVE"},
+                {"APP     ","UserTran","TABLE   ","1  ","S","SYSCOLUMNS  ","GRANT","ACTIVE"},
+            };
 		JDBC.assertFullResultSet(rs,expectedRows);
 		commit();
 		
@@ -604,10 +632,10 @@ public class AutoIncrementTest extends B
 		ResultSet rs;
 		Statement pst=createStatement();
 		Statement s=createStatement();
-		assertStatementError("22003", pst,"insert into ai_over1 (x) values (1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11),(12),(13),(14),(15),(16),(17),(18),(19)");
-		assertStatementError("22003", pst,"insert into ai_over1 (x) values (1)");		
+		assertStatementError("2200H", pst,"insert into ai_over1 (x) values (1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11),(12),(13),(14),(15),(16),(17),(18),(19)");
+		assertStatementError("2200H", pst,"insert into ai_over1 (x) values (1)");		
 		s.executeUpdate("insert into ai_over2 (x) values (1),(2),(3),(4),(5),(6),(7),(8)");
-		assertStatementError("22003", pst,"insert into ai_over2 (x) values (9),(10)");
+		assertStatementError("2200H", pst,"insert into ai_over2 (x) values (9),(10)");
 		String[][]expectedRows=new String[][]{{"1","-32760"},{"2","-32761"},{"3","-32762"},{"4","-32763"},{"5","-32764"},{"6","-32765"},{"7","-32766"},{"8","-32767"}};
 		rs=s.executeQuery("select * from ai_over2 order by x");
 		JDBC.assertFullResultSet(rs,expectedRows);		
@@ -616,12 +644,17 @@ public class AutoIncrementTest extends B
 		rs=s.executeQuery("select * from ai_over3 order by x");
 		expectedRows=new String[][]{{"1","2147483646"},{"2","2147483647"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
-		assertStatementError("22003", pst,"insert into ai_over3 (x) select x from ai_over3");			
+		assertStatementError("2200H", pst,"insert into ai_over3 (x) select x from ai_over3");			
 		//bigint overflow check		
-		s.executeUpdate("insert into ai_over4 (x) values (1),(2)");
-		assertStatementError("22003", pst,"insert into ai_over4 (x) values (3)");
+		s.executeUpdate("insert into ai_over4 (x) values (1),(2),(3)");
+		assertStatementError("2200H", pst,"insert into ai_over4 (x) values (4)");
 		rs=s.executeQuery("select * from ai_over4 order by x");
-		expectedRows=new String[][]{{"1","9223372036854775805"},{"2","9223372036854775806"}};
+		expectedRows=new String[][]
+            {
+                {"1","9223372036854775805"},
+                {"2","9223372036854775806"},
+                {"3","9223372036854775807"},
+            };
 		JDBC.assertFullResultSet(rs,expectedRows);
 
 	}
@@ -965,9 +998,7 @@ public class AutoIncrementTest extends B
                 {"2","9999"}
             };
 		JDBC.assertFullResultSet(rs,expectedRows);
-		rs=s.executeQuery("select COLUMNNAME, AUTOINCREMENTVALUE, AUTOINCREMENTSTART, AUTOINCREMENTINC from sys.syscolumns where COLUMNNAME = 'REC11'");
-		expectedRows=new String[][]{{"REC11","4","2","2"}};
-		JDBC.assertFullResultSet(rs,expectedRows);
+        vetSequenceState( "RESTARTT1", 4, 2, 2 );
 		assertStatementError("42837",s,"alter table restartt1 alter column c12 RESTART WITH 2");
 		assertStatementError("42X49",s,"alter table restartt1 alter column rec11 RESTART WITH 2.20");
 		s.execute("alter table restartt1 alter column rec11 RESTART WITH 2");
@@ -999,10 +1030,7 @@ public class AutoIncrementTest extends B
 		expectedRows=new String[][]{{"LOCKC11","1","1","1"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
 		assertStatementError("23505",s,"insert into t1lock(c12) values(3)");
-		rs=s.executeQuery("select COLUMNNAME, AUTOINCREMENTVALUE, AUTOINCREMENTSTART, AUTOINCREMENTINC from sys.syscolumns where COLUMNNAME = 'LOCKC11'");
-		//Utilities.showResultSet(rs);
-		expectedRows=new String[][]{{"LOCKC11","2","1","1"}};
-		JDBC.assertFullResultSet(rs,expectedRows);
+        vetSequenceState( "T1LOCK", 2, 1, 1 );
 
 		rs=s.executeQuery("select * from t1lock");
 		expectedRows=new String[][]{{"1","1"}};
@@ -1036,9 +1064,9 @@ public class AutoIncrementTest extends B
 		rs=s.executeQuery("SELECT col.columndefault,col.autoincrementvalue, col.autoincrementstart,col.autoincrementinc FROM sys.syscolumns col INNER JOIN sys.systables tab ON col.referenceId = tab.tableid WHERE tab.tableName = 'DERBY_1645' AND ColumnName = 'TESTTABLEID'");
 		expectedRows=new String[][]{{"GENERATED_BY_DEFAULT","1","1","1"}};
 		JDBC.assertFullResultSet(rs,expectedRows);
-		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (1, 'test1')");
-		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (2, 'test2')");
-		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (3, 'test3')");
+		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (-1, 'test1')");
+		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (-2, 'test2')");
+		s.execute("INSERT INTO derby_1645 (TESTTableId, TESTStringValue) VALUES (-3, 'test3')");
 		s.execute("ALTER TABLE derby_1645 ALTER TESTTableId SET INCREMENT BY 50");
 		rs=s.executeQuery("SELECT col.columndefault,col.autoincrementvalue, col.autoincrementstart,col.autoincrementinc FROM sys.syscolumns col INNER JOIN sys.systables tab ON col.referenceId = tab.tableid WHERE tab.tableName = 'DERBY_1645' AND ColumnName = 'TESTTABLEID'");
 		expectedRows=new String[][]{{"GENERATED_BY_DEFAULT","53","1","50"}};

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IdentitySequenceTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IdentitySequenceTest.java?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IdentitySequenceTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/IdentitySequenceTest.java Wed May 21 12:05:25 2014
@@ -116,27 +116,26 @@ public class IdentitySequenceTest extend
         goodStatement
             (
              conn,
-             "create table t1_01\n" +
+             "create table T1_01_IST\n" +
              "(\n" +
              "    a int generated always as identity ( start with 10, increment by 20 ),\n" +
              "    b int\n" +
              ")\n"
              );
-        String  sequenceName = getIdentitySequenceName( conn, "t1_01" );
+        String  sequenceName = getIdentitySequenceName( conn, "T1_01_IST" );
 
         // sequence should be in SYS, its name should be based on the table id,
         // and its start/stop/max/min/cycle values should be correct.
-        String  sequenceStats =
-            "select\n" +
-            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
-            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
-            "from sys.syssequences s, sys.sysschemas c\n" +
-            "where s.schemaID = c.schemaID\n";
 
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][]
              {
                  { "SYS", sequenceName, "10", "10", "-2147483648", "2147483647", "20", "N" },
@@ -147,7 +146,7 @@ public class IdentitySequenceTest extend
         assertResults
             (
              conn,
-             "values syscs_util.syscs_peek_at_identity( 'APP', 'T1_01' )",
+             "values syscs_util.syscs_peek_at_identity( 'APP', 'T1_01_IST' )",
              new String[][]
              {
                  { "10" },
@@ -160,22 +159,32 @@ public class IdentitySequenceTest extend
             ( conn, BAD_NEXT_VALUE, "values ( next value for sys.\"" + sequenceName + "\" )" );
 
         // alter the identity column and observe that the sequence generator changes
-        goodStatement( conn, "alter table t1_01 alter column a set increment by 15" );
+        goodStatement( conn, "alter table T1_01_IST alter column a set increment by 15" );
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][]
              {
                  { "SYS", sequenceName, "10", "10", "-2147483648", "2147483647", "15", "N" },
              },
              false
              );
-        goodStatement( conn, "alter table t1_01 alter column a restart with 500" );
+        goodStatement( conn, "alter table T1_01_IST alter column a restart with 500" );
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][]
              {
                  { "SYS", sequenceName, "500", "500", "-2147483648", "2147483647", "15", "N" },
@@ -184,36 +193,46 @@ public class IdentitySequenceTest extend
              );
         
         // system sequence should disappear when the table is dropped
-        goodStatement( conn, "drop table t1_01" );
+        goodStatement( conn, "drop table T1_01_IST" );
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][] {},
              false
              );
 
         // can't add an identity column to a table
-        goodStatement( conn, "create table t2_01( b int )" );
+        goodStatement( conn, "create table T2_01_IST( b int )" );
         expectCompilationError
             ( conn, CANT_ADD_IDENTITY,
-              "alter table t2_01 add column a int generated always as identity ( start with 10, increment by 20 )" );
+              "alter table T2_01_IST add column a int generated always as identity ( start with 10, increment by 20 )" );
 
         // dropping an identity column should drop the sequence generator too
         goodStatement
             (
              conn,
-             "create table t1_03\n" +
+             "create table T3_03_IST\n" +
              "(\n" +
              "    a int generated always as identity ( start with 10, increment by 20 ),\n" +
              "    b int\n" +
              ")\n"
              );
-        sequenceName = getIdentitySequenceName( conn, "t1_03" );
+        sequenceName = getIdentitySequenceName( conn, "T3_03_IST" );
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][]
              {
                  { "SYS", sequenceName, "10", "10", "-2147483648", "2147483647", "20", "N" },
@@ -223,38 +242,112 @@ public class IdentitySequenceTest extend
         assertResults
             (
              conn,
-             "values syscs_util.syscs_peek_at_identity( 'APP', 'T1_03' )",
+             "values syscs_util.syscs_peek_at_identity( 'APP', 'T3_03_IST' )",
              new String[][]
              {
                  { "10" },
              },
              false
              );
-        goodStatement( conn, "alter table t1_03 drop column a" );
+        goodStatement( conn, "alter table T3_03_IST drop column a" );
         assertResults
             (
              conn,
-             sequenceStats,
+            "select\n" +
+            "    c.schemaName, s.sequenceName, s.currentValue, s.startValue,\n" +
+            "    s.minimumValue, s.maximumValue, s.increment, s.cycleoption\n" +
+            "from sys.syssequences s, sys.sysschemas c\n" +
+            "where s.schemaID = c.schemaID\n" +
+             "and s.sequenceName = '" + sequenceName + "'",
              new String[][] {},
              false
              );
         expectExecutionError
             ( conn, TABLE_DOESNT_HAVE_IDENTITY,
-             "values syscs_util.syscs_peek_at_identity( 'APP', 'T1_03' )"
+             "values syscs_util.syscs_peek_at_identity( 'APP', 'T3_03_IST' )"
               );
     }
     
+    /**
+     * <p>
+     * Test ALTER TABLE behavior.
+     * </p>
+     */
+    public  void    test_002_alterTable()
+        throws Exception
+    {
+        Connection  conn = getConnection();
+
+        //
+        // Test that changing the increment value for an identity
+        // column does not affect its current value. See DERBY-6579.
+        //
+        goodStatement( conn, "create table t1_002( a int, b int generated always as identity )" );
+        goodStatement( conn, "insert into t1_002( a ) values ( 100 ), ( 200 )" );
+        goodStatement( conn, "alter table t1_002 alter b set increment by 10" );
+        goodStatement( conn, "insert into t1_002( a ) values ( 300 ), ( 400 )" );
+        assertResults
+            (
+             conn,
+             "select * from t1_002 order by a",
+             new String[][]
+             {
+                 { "100", "1" },
+                 { "200", "2" },
+                 { "300", "3" },
+                 { "400", "13" },
+             },
+             false
+             );
+
+        goodStatement( conn, "drop table t1_002" );
+        goodStatement( conn, "create table t1_002( a int, b int generated always as identity )" );
+        goodStatement( conn, "insert into t1_002( a ) values ( 100 ), ( 200 )" );
+        goodStatement( conn, "delete from t1_002 where a = 200" );
+        goodStatement( conn, "alter table t1_002 alter b set increment by 10" );
+        goodStatement( conn, "insert into t1_002( a ) values ( 300 ), ( 400 )" );
+        assertResults
+            (
+             conn,
+             "select * from t1_002 order by a",
+             new String[][]
+             {
+                 { "100", "1" },
+                 { "300", "3" },
+                 { "400", "13" },
+             },
+             false
+             );
+
+        // now restart the identity column at a later number
+        goodStatement( conn, "alter table t1_002 alter b restart with 1000" );
+        goodStatement( conn, "insert into t1_002( a ) values ( 500 ), ( 600 )" );
+        assertResults
+            (
+             conn,
+             "select * from t1_002 order by a",
+             new String[][]
+             {
+                 { "100", "1" },
+                 { "300", "3" },
+                 { "400", "13" },
+                 { "500", "1000" },
+                 { "600", "1010" },
+             },
+             false
+             );
+    }
+
     ///////////////////////////////////////////////////////////////////////////////////
     //
     // MINIONS
     //
     ///////////////////////////////////////////////////////////////////////////////////
 
-    private String  getIdentitySequenceName( Connection conn, String tableName )
+    public  static String  getIdentitySequenceName( Connection conn, String tableName )
         throws Exception
     {
-        PreparedStatement   ps = chattyPrepare
-            ( conn, "select tableID from sys.systables where tablename = ?" );
+        PreparedStatement   ps = conn.prepareStatement( "select tableID from sys.systables where tablename = ?" );
         ps.setString( 1, tableName.toUpperCase() );
         ResultSet   rs = ps.executeQuery();
         rs.next();

Modified: db/derby/code/trunk/tools/jar/extraDBMSclasses.properties
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/tools/jar/extraDBMSclasses.properties?rev=1596548&r1=1596547&r2=1596548&view=diff
==============================================================================
--- db/derby/code/trunk/tools/jar/extraDBMSclasses.properties (original)
+++ db/derby/code/trunk/tools/jar/extraDBMSclasses.properties Wed May 21 12:05:25 2014
@@ -23,7 +23,6 @@ derby.module.uuidtype=org.apache.derby.c
 derby.module.sysinfo=org.apache.derby.tools.sysinfo
 derby.module.database.PropInfo=org.apache.derby.iapi.db.PropertyInfo
 derby.module.database.Factory=org.apache.derby.iapi.db.Factory
-derby.module.database.ConnectionInfo=org.apache.derby.iapi.db.ConnectionInfo
 derby.module.aggs.MaxMinAggDef=org.apache.derby.impl.sql.compile.MaxMinAggregateDefinition
 derby.module.aggs.SumAvgAggDef=org.apache.derby.impl.sql.compile.SumAvgAggregateDefinition
 derby.module.aggs.SumAvgAggDef=org.apache.derby.impl.sql.execute.UserDefinedAggregator