You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by mi...@apache.org on 2007/04/12 18:58:33 UTC

svn commit: r528033 [1/2] - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/services/io/ engine/org/apache/derby/iapi/sql/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/s...

Author: mikem
Date: Thu Apr 12 09:58:30 2007
New Revision: 528033

URL: http://svn.apache.org/viewvc?view=rev&rev=528033
Log:
DERBY-2537, 1st incremental checking for this issue.  This changes builds
the framework for storing/retrieving the collation metadata in store and for
passing that info down from language into store.  Some paths still have
default collation hard coded and have been marked TODO-COLLATION.


Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I_v10_2.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap_v10_2.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/ResultDescription.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/DiskHashtable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.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/conn/GenericLanguageConnectionContext.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/CreateIndexConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateTableConstantAction.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/MaterializedResultSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTree.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/conglomerate/ConglomerateUtil.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/Heap.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/heap/HeapConglomerateFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/LoggableAllocActions.java
    db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/store/TestDiskHashtable.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_QualifierTest.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_XA.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_b2i.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/RegisteredFormatIds.java Thu Apr 12 09:58:30 2007
@@ -143,7 +143,7 @@
         /* 88 */        "org.apache.derby.iapi.types.DTSClassInfo", //InstanceGetter
         /* 89 */        "org.apache.derby.iapi.types.SQLLongint",
         /* 90 */        "org.apache.derby.impl.store.access.heap.HeapClassInfo",
-        /* 91 */        "org.apache.derby.impl.store.access.heap.Heap",
+        /* 91 */        "org.apache.derby.impl.store.access.heap.Heap_v10_2",
         /* 92 */        null,
         /* 93 */        "org.apache.derby.impl.store.access.StorableFormatId",
         /* 94 */        null,
@@ -440,7 +440,7 @@
         /* 385 */   null,
         /* 386 */   null,
         /* 387 */       "org.apache.derby.catalog.types.IndexDescriptorImpl",
-        /* 388 */       "org.apache.derby.impl.store.access.btree.index.B2I",
+        /* 388 */       "org.apache.derby.impl.store.access.btree.index.B2I_v10_2",
         /* 389 */   null,
         /* 390 */   null,
         /* 391 */   null,
@@ -527,5 +527,7 @@
         /* 463 */   "org.apache.derby.impl.sql.catalog.CoreDDFinderClassInfo",
         /* 464 */   "org.apache.derby.iapi.types.SqlXmlUtil",        
 		/* 465 */   "org.apache.derby.impl.store.raw.data.CompressSpacePageOperation",
+        /* 466 */   "org.apache.derby.impl.store.access.btree.index.B2I",
+        /* 467 */   "org.apache.derby.impl.store.access.heap.Heap"
 };
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/io/StoredFormatIds.java Thu Apr 12 09:58:30 2007
@@ -1599,6 +1599,7 @@
     public static final int ACCESS_TREE_V1_ID =
             (MIN_ID_2 + 386);
 
+
     public static final int ACCESS_B2I_V3_ID =
             (MIN_ID_2 + 388);
 
@@ -1626,7 +1627,14 @@
     public static final int ACCESS_T_RECTANGLE_ID =
             (MIN_ID_4 + 34);
 
-    public static final int ACCESS_T_INTCOL_V1_ID =               MIN_ID_4 + 4;
+    public static final int ACCESS_T_INTCOL_V1_ID =      
+            (MIN_ID_4 + 4);
+
+    public static final int ACCESS_B2I_V4_ID =
+            (MIN_ID_2 + 466);
+
+    public static final int ACCESS_HEAP_V3_ID =
+            (MIN_ID_2 + 467);
 
     /******************************************************************
     **
@@ -1865,9 +1873,9 @@
      * Make sure this is updated when a new module is added
      */
     public static final int MAX_ID_2 =
-            (MIN_ID_2 + 465);
+            (MIN_ID_2 + 467);
 
     // DO NOT USE 4 BYTE IDS ANYMORE
     static public final int MAX_ID_4 =
-        MIN_ID_4 + 34;
+            (MIN_ID_4 + 34);
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/ResultDescription.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/ResultDescription.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/ResultDescription.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/ResultDescription.java Thu Apr 12 09:58:30 2007
@@ -55,7 +55,7 @@
 	public ResultColumnDescriptor[] getColumnInfo();
 
 	/**
-	 * Returns a ResultColumnDescriptor for the column, given the oridinal
+	 * Returns a ResultColumnDescriptor for the column, given the ordiinal
 	 * position of the column.
 	 * NOTE - position is 1-based.
 	 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/TableDescriptor.java Thu Apr 12 09:58:30 2007
@@ -502,6 +502,32 @@
 		return result;
 	}
 
+    /**
+     * Return an array of collation ids.
+     * <p>
+     * Return an array of collation ids, one for each column in the
+     * columnDescriptorList.  This is useful for passing collation id info
+     * down to store, for instance in createConglomerate().
+     *
+	 * @exception  StandardException  Standard exception policy.
+     **/
+    public int[] getColumnCollationIds()
+		throws StandardException
+    {
+        int[] collation_ids = new int[getNumberOfColumns()]; 
+
+		for (int index = 0; index < collation_ids.length; index++)
+		{
+			ColumnDescriptor cd = 
+                (ColumnDescriptor) columnDescriptorList.elementAt(index);
+
+            collation_ids[index] = cd.getType().getCollationType();
+
+		}
+		return(collation_ids);
+
+    }
+
 	/**
 	 * Gets the conglomerate descriptor list
 	 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/BackingStoreHashtable.java Thu Apr 12 09:58:30 2007
@@ -498,11 +498,14 @@
                 // Do not know how to put it on disk
                 return false;
             }
-            diskHashtable = new DiskHashtable( tc,
-                                               (DataValueDescriptor[]) row,
-                                               key_column_numbers,
-                                               remove_duplicates,
-                                               keepAfterCommit);
+            diskHashtable = 
+                new DiskHashtable(
+                       tc,
+                       (DataValueDescriptor[]) row,
+                       (int[]) null, //TODO-COLLATION, set non default collation if necessary.
+                       key_column_numbers,
+                       remove_duplicates,
+                       keepAfterCommit);
         }
         Object key = KeyHasher.buildHashKey(row, key_column_numbers);
         Object duplicateValue = hash_table.get( key);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/DiskHashtable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/DiskHashtable.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/DiskHashtable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/DiskHashtable.java Thu Apr 12 09:58:30 2007
@@ -25,14 +25,17 @@
 import java.util.Properties;
 import java.util.Vector;
 import org.apache.derby.iapi.error.StandardException;
-import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.types.DataValueDescriptor;
 import org.apache.derby.iapi.types.SQLInteger;
 import org.apache.derby.iapi.types.RowLocation;
+import org.apache.derby.iapi.types.StringDataValue;
+
 import org.apache.derby.iapi.services.context.ContextService;
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 
+import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
+
 /**
  * This class is used by BackingStoreHashtable when the BackingStoreHashtable 
  * must spill to disk.  It implements the methods of a hash table: put, get, 
@@ -76,6 +79,7 @@
     public DiskHashtable( 
     TransactionController   tc,
     DataValueDescriptor[]   template,
+    int[]                   collation_ids,
     int[]                   key_column_numbers,
     boolean                 remove_duplicates,
     boolean                 keepAfterCommit)
@@ -118,6 +122,7 @@
                 "heap",
                 template,
                 (ColumnOrdering[]) null,
+                collation_ids,
                 (Properties) null,
                 tempFlags);
 
@@ -151,11 +156,18 @@
                 "2"); // Include the row location column
         btreeProps.put("maintainParentLinks", 
                 "false");
+
+        // default collation is used for hash code and row location
+        int[] index_collation_ids = 
+            {StringDataValue.COLLATION_TYPE_UCS_BASIC,
+             StringDataValue.COLLATION_TYPE_UCS_BASIC};
+
         btreeConglomerateId = 
             tc.createConglomerate( 
                 "BTREE",
                 btreeRow,
                 (ColumnOrdering[]) null,
+                index_collation_ids,
                 btreeProps,
                 tempFlags);
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Thu Apr 12 09:58:30 2007
@@ -301,6 +301,8 @@
 		long conglomId = tc.createConglomerate(
 			"heap", // we're requesting a heap conglomerate
 			template, // a populated template is required for heap and btree.
+            null, // no column order
+            null, // default collation order for all columns
 			null, // default properties
 			0); // not temporary
 	</blockquote></pre>
@@ -379,6 +381,13 @@
 	value is 'null', which means all columns needs to be sorted in 
 	Ascending order.
 
+    @param collationIds Specifies the collation id of each of the columns
+    in the new conglomerate.  Collation id along with format id may be used
+    to create DataValueDescriptor's which may subsequently be used for
+    comparisons.  For instance the correct collation specific order and
+    searching is maintained by correctly specifying the collation id of 
+    the columns in the index when the index is created.
+
 
 	@param properties Implementation-specific properties of the
 	conglomerate.  
@@ -417,6 +426,7 @@
     String                  implementation,
     DataValueDescriptor[]   template,
     ColumnOrdering[]        columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int                     temporaryFlag)
 		throws StandardException;
@@ -439,6 +449,33 @@
 	THE WAY THAT THE IMPLEMENTATION IS CHOSEN STILL NEEDS SOME WORK.
     For now, use "BTREE" or "heap" for a local access manager.
 
+    @param template A row which describes the prototypical
+	row that the conglomerate will be holding.
+	Typically this row gives the conglomerate
+	information about the number and type of
+	columns it will be holding.  The implementation
+	may require a specific subclass of row type.
+    Note that the createConglomerate call reads the template and makes a copy
+    of any necessary information from the template, no reference to the
+    template is kept (and thus this template can be re-used in subsequent
+    calls - such as openScan()).  This field is required when creating either
+    a heap or btree conglomerate.
+
+	@param columnOrder Specifies the colummns sort order.
+	Useful only when the conglomerate is of type BTREE, default
+	value is 'null', which means all columns needs to be sorted in 
+	Ascending order.
+
+    @param collationIds Specifies the collation id of each of the columns
+    in the new conglomerate.  Collation id along with format id may be used
+    to create DataValueDescriptor's which may subsequently be used for
+    comparisons.  For instance the correct collation specific order and
+    searching is maintained by correctly specifying the collation id of 
+    the columns in the index when the index is created.
+
+	@param properties Implementation-specific properties of the
+	conglomerate.  
+
 	@param rowSource the interface to recieve rows to load into the
 	conglomerate. 
 
@@ -456,10 +493,11 @@
     String                  implementation,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]		columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int                     temporaryFlag,
     RowLocationRetRowSource rowSource,
-	long[] rowCount)
+	long[]                  rowCount)
     throws StandardException;
 
 	/**
@@ -506,6 +544,13 @@
 	value is 'null', which means all columns needs to be sorted in 
 	Ascending order.
 
+    @param collationIds Specifies the collation id of each of the columns
+    in the new conglomerate.  Collation id along with format id may be used
+    to create DataValueDescriptor's which may subsequently be used for
+    comparisons.  For instance the correct collation specific order and
+    searching is maintained by correctly specifying the collation id of 
+    the columns in the index when the index is created.
+
 	@param properties Implementation-specific properties of the conglomerate.  
 
     @param  temporaryFlag  If true, the conglomerate is temporary.
@@ -536,6 +581,7 @@
     boolean                 recreate_ifempty,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]		columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int			            temporaryFlag,
     long                    orig_conglomId,
@@ -566,6 +612,7 @@
 	@param conglomId        The identifier of the conglomerate to alter.
 	@param column_id        The column number to add this column at.
 	@param template_column  An instance of the column to be added to table.
+	@param collation_id     Collation id of the added column.
 
 	@exception StandardException Only some types of conglomerates can support
         adding a column, for instance "heap" conglomerates support adding a 
@@ -575,7 +622,8 @@
     public void addColumnToConglomerate(
     long        conglomId, 
     int         column_id, 
-    Storable    template_column)
+    Storable    template_column,
+    int         collation_id)
 		throws StandardException;
 
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/Conglomerate.java Thu Apr 12 09:58:30 2007
@@ -77,13 +77,15 @@
      *                         operation takes place.
      * @param column_id        The column number to add this column at.
      * @param template_column  An instance of the column to be added to table.
+     * @param collation_id     Collation id of the added column.
      *
 	 * @exception  StandardException  Standard exception policy.
      **/
 	public void addColumn(
     TransactionManager  xact_manager,
     int                 column_id,
-    Storable            template_column)
+    Storable            template_column,
+    int                 collation_id)
         throws StandardException;
 
     /**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/ConglomerateFactory.java Thu Apr 12 09:58:30 2007
@@ -75,6 +75,7 @@
                                 raw store to assign an id.
     @param template             Template of row in the conglomerate.
 	@param columnOrder          columns sort order for Index creation
+    @param collationIds         collation ids of columns in the conglomerate.
     @param properties           Properties associated with the conglomerate.
 
  	@exception StandardException if the conglomerate could not be
@@ -87,6 +88,7 @@
     long                    input_containerid,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]		columnOrder,
+    int[]                   collationIds,
     Properties              properties,
 	int						temporaryFlag)
             throws StandardException;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/conglomerate/TransactionManager.java Thu Apr 12 09:58:30 2007
@@ -81,6 +81,29 @@
 	public void addPostCommitWork(Serviceable work);
 
     /**
+     *  Check to see if a database has been upgraded to the required
+     *  level in order to use a store feature.
+     *
+     * @param requiredMajorVersion  required database Engine major version
+     * @param requiredMinorVersion  required database Engine minor version
+     * @param feature               Non-null to throw an exception, null to 
+     *                              return the state of the version match.
+     *
+     * @return <code> true </code> if the database has been upgraded to 
+     *         the required level, <code> false </code> otherwise.
+     *
+     * @exception  StandardException 
+     *             if the database is not at the require version 
+     *             when <code>feature</code> feature is 
+     *             not <code> null </code>. 
+     */
+	public boolean checkVersion(
+    int     requiredMajorVersion, 
+    int     requiredMinorVersion, 
+    String  feature) 
+        throws StandardException;
+
+    /**
      * The ScanManager.close() method has been called on "scan".
      * <p>
      * Take whatever cleanup action is appropriate to a closed scan.  It is
@@ -149,6 +172,7 @@
      **/
     public Transaction getRawStoreXact()
         throws StandardException;
+
 
     /**
      * Do work necessary to maintain the current position in all the scans.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Thu Apr 12 09:58:30 2007
@@ -908,4 +908,27 @@
 		Get the Transaction Factory to use with this store.
 	*/
 	public TransactionFactory getXactFactory();
+
+    /**
+     *  Check to see if a database has been upgraded to the required
+     *  level in order to use a store feature.
+     *
+     * @param requiredMajorVersion  required database Engine major version
+     * @param requiredMinorVersion  required database Engine minor version
+     * @param feature               Non-null to throw an exception, null to 
+     *                              return the state of the version match.
+     *
+     * @return <code> true </code> if the database has been upgraded to 
+     *         the required level, <code> false </code> otherwise.
+     *
+     * @exception  StandardException 
+     *             if the database is not at the require version 
+     *             when <code>feature</code> feature is 
+     *             not <code> null </code>. 
+     */
+	public boolean checkVersion(
+    int     requiredMajorVersion, 
+    int     requiredMinorVersion, 
+    String  feature) 
+        throws StandardException;
 }

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?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- 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 Thu Apr 12 09:58:30 2007
@@ -6785,9 +6785,13 @@
 			int columnID = newColumnIDs[ix];
 			int storablePosition = columnID - 1;			// from 1 to 0 based
 
-			tc.addColumnToConglomerate( conglomID,
-										storablePosition,
-										templateRow.getColumn( columnID) );
+            // system catalog columns always have UCS_BASIC collation.
+
+			tc.addColumnToConglomerate( 
+                conglomID,
+                storablePosition,
+                templateRow.getColumn( columnID),
+                StringDataValue.COLLATION_TYPE_UCS_BASIC);
 		}
 
 	}
@@ -7219,6 +7223,7 @@
 			"BTREE", // we're requesting an index conglomerate
 			indexableRow.getRowArray(),
 			null, //default sort order
+            null, //default collation id's for collumns in all system congloms
 			indexProperties, // default properties
 			TransactionController.IS_DEFAULT); // not temporary
 
@@ -7418,6 +7423,7 @@
 			"heap", // we're requesting a heap conglomerate
 			rowTemplate.getRowArray(), // row template
 			null, // default sort order
+            null, // default collation ids
 			properties, // default properties
 			TransactionController.IS_DEFAULT); // not temporary
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionContext.java Thu Apr 12 09:58:30 2007
@@ -51,7 +51,7 @@
 import org.apache.derby.iapi.sql.dictionary.ConglomerateDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
-import	org.apache.derby.iapi.sql.dictionary.TableDescriptor;
+import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
 import org.apache.derby.iapi.types.DataValueFactory;
 import org.apache.derby.iapi.sql.compile.TypeCompilerFactory;
 import org.apache.derby.iapi.sql.depend.DependencyManager;
@@ -1177,23 +1177,33 @@
 	}
 
 	/**
-	 * If dropAndRedeclare is true, that means we have come here for temp tables with on commit delete rows
-	 * and no held curosr open on them. We will drop the existing conglomerate and redeclare a new conglomerate
-	 * similar to old conglomerate. This is a more efficient way of deleting all rows from the table.
+	 * If dropAndRedeclare is true, that means we have come here for temp 
+     * tables with on commit delete rows and no held curosr open on them. We 
+     * will drop the existing conglomerate and redeclare a new conglomerate
+	 * similar to old conglomerate. This is a more efficient way of deleting 
+     * all rows from the table.
 	 *
-	 * If dropAndRedeclare is false, that means we have come here for the rollback cleanup work. We are trying
-	 * to restore old definition of the temp table (because the drop on it is being rolled back).
-	 */
-	private TableDescriptor cleanupTempTableOnCommitOrRollback(TableDescriptor td, boolean dropAndRedeclare)
+	 * If dropAndRedeclare is false, that means we have come here for the 
+     * rollback cleanup work. We are trying to restore old definition of the 
+     * temp table (because the drop on it is being rolled back).
+	 */
+	private TableDescriptor cleanupTempTableOnCommitOrRollback(
+    TableDescriptor td, 
+    boolean         dropAndRedeclare)
 		 throws StandardException
 	{
-		//create new conglomerate with same properties as the old conglomerate and same row template as the old conglomerate
-		long conglomId = tran.createConglomerate(
-			"heap", // we're requesting a heap conglomerate
-			td.getEmptyExecRow(getContextManager()).getRowArray(), // row template
-			null, //column sort order - not required for heap
-			null, // properties
-			(TransactionController.IS_TEMPORARY | TransactionController.IS_KEPT));
+		//create new conglomerate with same properties as the old conglomerate 
+        //and same row template as the old conglomerate
+		long conglomId = 
+            tran.createConglomerate(
+                "heap", // we're requesting a heap conglomerate
+                td.getEmptyExecRow(
+                    getContextManager()).getRowArray(), // row template
+                null, //column sort order - not required for heap
+                td.getColumnCollationIds(),  // same ids as old conglomerate
+                null, // properties
+                (TransactionController.IS_TEMPORARY | 
+                 TransactionController.IS_KEPT));
 
 		long cid = td.getHeapConglomerateId();
 

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?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- 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 Thu Apr 12 09:58:30 2007
@@ -77,6 +77,7 @@
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.types.DataValueDescriptor;
 import org.apache.derby.iapi.types.RowLocation;
+import org.apache.derby.iapi.types.StringDataValue;
 import org.apache.derby.impl.sql.catalog.DDColumnDependableFinder;
 
 /**
@@ -124,6 +125,7 @@
 	private SortController[]		sorters;
 	private int						columnPosition;
 	private ColumnOrdering[][]		ordering;
+	private int[][]		            collation;
 
 	private	TableDescriptor 		td;
 
@@ -572,8 +574,11 @@
 			storableDV = columnInfo[ix].dataType.getNull();
 
 		// Add the column to the conglomerate.(Column ids in store are 0-based)
-		tc.addColumnToConglomerate(td.getHeapConglomerateId(), colNumber, 
-								   storableDV);
+		tc.addColumnToConglomerate(
+            td.getHeapConglomerateId(), 
+            colNumber, 
+            storableDV, 
+            columnInfo[ix].dataType.getCollationType());
 
 		UUID defaultUUID = columnInfo[ix].newDefaultUUID;
 
@@ -685,7 +690,7 @@
 	 *  - if that column was the only column in the index, the
 	 *    entire index is dropped. 
 	 *
-         * @param   activation          the current activation
+     * @param   activation  the current activation
 	 * @param   ix 			the index of the column specfication in the ALTER 
 	 *						statement-- currently we allow only one.
 	 * @exception StandardException 	thrown on failure.
@@ -1113,7 +1118,8 @@
 	/* NOTE: compressTable can also be called for 
 	 * ALTER TABLE <t> DROP COLUMN <c>;
 	 */
-	private void compressTable(Activation activation)
+	private void compressTable(
+    Activation activation)
 		throws StandardException
 	{
 		ExecRow					emptyHeapRow;
@@ -1131,7 +1137,8 @@
 			if (lockGranularity != '\0')
 			{
 				SanityManager.THROWASSERT(
-					"lockGranularity expected to be '\0', not " + lockGranularity);
+					"lockGranularity expected to be '\0', not " + 
+                    lockGranularity);
 			}
 			SanityManager.ASSERT(! compressTable || columnInfo == null,
 				"columnInfo expected to be null");
@@ -1146,12 +1153,11 @@
                                 TransactionController.MODE_TABLE,
                                 TransactionController.ISOLATION_SERIALIZABLE);
 
-		// invalidate any prepared statements that
-		// depended on this table (including this one)
-		// bug 3653 has threads that start up and block on our lock, but do
-		// not see they have to recompile their plan.    We now invalidate earlier
-		// however they still might recompile using the old conglomerate id before we
-		// commit our DD changes.
+		// invalidate any prepared statements that depended on this table 
+        // (including this one), this fixes problem with threads that start up 
+        // and block on our lock, but do not see they have to recompile their 
+        // plan.  We now invalidate earlier however they still might recompile
+        // using the old conglomerate id before we commit our DD changes.
 		//
 		dm.invalidateFor(td, DependencyManager.COMPRESS_TABLE, lcc);
 
@@ -1175,7 +1181,10 @@
 		indexRows = new ExecIndexRow[numIndexes];
 		if (! compressTable)
 		{
-			ExecRow newRow = activation.getExecutionFactory().getValueRow(emptyHeapRow.nColumns() - 1);
+			ExecRow newRow = 
+                activation.getExecutionFactory().getValueRow(
+                    emptyHeapRow.nColumns() - 1);
+
 			for (int i = 0; i < newRow.nColumns(); i++)
 			{
 				newRow.setColumn(i + 1, i < columnPosition - 1 ?
@@ -1202,14 +1211,22 @@
 		}
 
 
-		newHeapConglom = tc.createAndLoadConglomerate(
-									"heap",
-									emptyHeapRow.getRowArray(),
-									null, //column sort order - not required for heap
-									properties,
-									TransactionController.IS_DEFAULT,
-									this,
-									(long[]) null);
+		newHeapConglom = 
+            tc.createAndLoadConglomerate(
+                "heap",
+                emptyHeapRow.getRowArray(),
+                null, //column sort order - not required for heap
+                (int[]) null, // TODO-COLLATION - implement correct setting of
+                              // collation ids.  This is made more tricky as
+                              // this routine could be called by either compress
+                              // table where the heap ids are same as old heap
+                              // or drop table where the collation ids need
+                              // to be shuffled about.  Setting default 
+                              // collation for now.
+                properties,
+                TransactionController.IS_DEFAULT,
+                this,
+                (long[]) null);
 
 		closeBulkFetchScan();
 
@@ -1366,19 +1383,22 @@
 		compressHeapCC = null;
 
 		//create new conglomerate
-		newHeapConglom = tc.createConglomerate(
-									"heap",
-									emptyHeapRow.getRowArray(),
-									null, //column sort order - not required for heap
-									properties,
-									TransactionController.IS_DEFAULT);
+		newHeapConglom = 
+            tc.createConglomerate(
+                "heap",
+                emptyHeapRow.getRowArray(),
+                null, //column sort order - not required for heap
+                td.getColumnCollationIds(),
+                properties,
+                TransactionController.IS_DEFAULT);
 		
 		/* Set up index info to perform truncate on them*/
 		getAffectedIndexes(activation);
 		if(numIndexes > 0)
 		{
 			indexRows = new ExecIndexRow[numIndexes];
-			ordering = new ColumnOrdering[numIndexes][];
+			ordering  = new ColumnOrdering[numIndexes][];
+
 			for (int index = 0; index < numIndexes; index++)
 			{
 				// create a single index row template for each index
@@ -1392,16 +1412,23 @@
 				 * No need to try to enforce uniqueness here as
 				 * index should be valid.
 				 */
-				int[] baseColumnPositions = compressIRGs[index].baseColumnPositions();
+				int[] baseColumnPositions = 
+                    compressIRGs[index].baseColumnPositions();
+
 				boolean[] isAscending = compressIRGs[index].isAscending();
+
 				int numColumnOrderings;
 				numColumnOrderings = baseColumnPositions.length + 1;
-				ordering[index] = new ColumnOrdering[numColumnOrderings];
+				ordering[index]    = new ColumnOrdering[numColumnOrderings];
+                collation[index]   = new int[baseColumnPositions.length + 1];
+
 				for (int ii =0; ii < numColumnOrderings - 1; ii++) 
 				{
-					ordering[index][ii] = new IndexColumnOrder(ii, isAscending[ii]);
+					ordering[index][ii] = 
+                        new IndexColumnOrder(ii, isAscending[ii]);
 				}
-				ordering[index][numColumnOrderings - 1] = new IndexColumnOrder(numColumnOrderings - 1);
+				ordering[index][numColumnOrderings - 1] = 
+                    new IndexColumnOrder(numColumnOrderings - 1);
 			}
 		}
 
@@ -1546,6 +1573,11 @@
 								   "BTREE",
 								   indexRows[index].getRowArray(),
 								   ordering[index],
+                                   (int[]) null, // TODO-COLLATION, implement 
+                                                 // collation support for alter
+                                                 // table.  Currently only
+                                                 // supports default collation
+                                                 // columns.
 								   properties,
 								   TransactionController.IS_DEFAULT,
 								   cCount,
@@ -1577,14 +1609,19 @@
 							true, tc);	// no error on duplicate.
 				}
 			}
-		}else
+		}
+        else
 		{
-			newIndexCongloms[index] = tc.createConglomerate(
-								   "BTREE",
-									indexRows[index].getRowArray(),
-									ordering[index],
-									properties,
-								   TransactionController.IS_DEFAULT);
+            newIndexCongloms[index] = 
+                tc.createConglomerate(
+                    "BTREE",
+                    indexRows[index].getRowArray(),
+                    ordering[index],
+                    (int[]) null, // TODO-COLLATION, implement alter table 
+                                  // collation passing, 
+                                  // currently only supports default collation.
+                    properties,
+                    TransactionController.IS_DEFAULT);
 
 
 			//on truncate drop the statistics because we know for sure 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateIndexConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateIndexConstantAction.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateIndexConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateIndexConstantAction.java Thu Apr 12 09:58:30 2007
@@ -565,11 +565,15 @@
 													 indexTemplateRow,
 													 true);
 			}
+
 			ColumnOrdering[]	order = new ColumnOrdering[numColumnOrderings];
 			for (int i=0; i < numColumnOrderings; i++) 
 			{
-				order[i] = new IndexColumnOrder(i, unique || i < numColumnOrderings - 1 
-													? isAscending[i] : true);
+				order[i] = 
+                    new IndexColumnOrder(
+                        i, 
+                        unique || i < numColumnOrderings - 1 ? 
+                            isAscending[i] : true);
 			}
 
 			// create the sorter
@@ -590,10 +594,12 @@
 			rowSource = loadSorter(baseRows, indexRows, tc,
 								   scan, sortId, rl);
 
-			conglomId = tc.createAndLoadConglomerate(
+			conglomId = 
+                tc.createAndLoadConglomerate(
 					indexType,
 					indexTemplateRow.getRowArray(),	// index row template
 					order, //colums sort order
+                    null,  // TODO-COLLATION, implement non-default collation
 					indexProperties,
 					TransactionController.IS_DEFAULT, // not temporary
 					rowSource,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateTableConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateTableConstantAction.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateTableConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateTableConstantAction.java Thu Apr 12 09:58:30 2007
@@ -163,24 +163,36 @@
 		/* Mark the activation as being for create table */
 		activation.setForCreateTable();
 
-		/*
-		** Create a row template to tell the store what type of rows this table
-		** holds.
-		*/
-		template = RowUtil.getEmptyValueRow(columnInfo.length, lcc);
+        // setup for create conglomerate call:
+        //   o create row template to tell the store what type of rows this
+        //     table holds.
+        //   o create array of collation id's to tell collation id of each
+        //     column in table.
+		template            = RowUtil.getEmptyValueRow(columnInfo.length, lcc);
+        int[] collation_ids = new int[columnInfo.length];
 
-		/* Get a template value for each column */
 		for (int ix = 0; ix < columnInfo.length; ix++)
 		{
-			/* If there is a default value, use it, otherwise use null */
-			if (columnInfo[ix].defaultValue != null)
-				template.setColumn(ix + 1, columnInfo[ix].defaultValue);
+            ColumnInfo  col_info = columnInfo[ix];
+
+            // Get a template value for each column
+
+			if (col_info.defaultValue != null)
+            {
+                /* If there is a default value, use it, otherwise use null */
+				template.setColumn(ix + 1, col_info.defaultValue);
+            }
 			else
-				template.setColumn(ix + 1,
-									columnInfo[ix].dataType.getNull()
-								);
+            {
+				template.setColumn(ix + 1, col_info.dataType.getNull());
+            }
+
+            // get collation info for each column.
+
+            collation_ids[ix] = col_info.dataType.getCollationType();
 		}
 
+
 		/* create the conglomerate to hold the table's rows
 		 * RESOLVE - If we ever have a conglomerate creator
 		 * that lets us specify the conglomerate number then
@@ -190,9 +202,12 @@
 				"heap", // we're requesting a heap conglomerate
 				template.getRowArray(), // row template
 				null, //column sort order - not required for heap
+                collation_ids,
 				properties, // properties
 				tableType == TableDescriptor.GLOBAL_TEMPORARY_TABLE_TYPE ?
-				(TransactionController.IS_TEMPORARY | TransactionController.IS_KEPT) : TransactionController.IS_DEFAULT);
+                    (TransactionController.IS_TEMPORARY | 
+                     TransactionController.IS_KEPT) : 
+                        TransactionController.IS_DEFAULT);
 
 		/*
 		** Inform the data dictionary that we are about to write to it.

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?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- 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 Thu Apr 12 09:58:30 2007
@@ -1250,27 +1250,31 @@
 		long[] loadedRowCount = new long[1];
 		if (bulkInsertReplace)
 		{
-			newHeapConglom = tc.createAndLoadConglomerate(
-										"heap",
-										fullTemplate.getRowArray(),
-										null, //column sort order - not required for heap
-										properties,
-										TransactionController.IS_DEFAULT,
-										sourceResultSet,
-										loadedRowCount);
+			newHeapConglom = 
+                tc.createAndLoadConglomerate(
+                    "heap",
+                    fullTemplate.getRowArray(),
+                    null, //column sort order - not required for heap
+                    td.getColumnCollationIds(),
+                    properties,
+                    TransactionController.IS_DEFAULT,
+                    sourceResultSet,
+                    loadedRowCount);
 		}
 		else
 		{
-			newHeapConglom = tc.recreateAndLoadConglomerate(
-										"heap",
-										false,
-										fullTemplate.getRowArray(),
-										null, //column sort order - not required for heap
-										properties,
-										TransactionController.IS_DEFAULT,
-										oldHeapConglom,
-										sourceResultSet,
-										loadedRowCount);
+			newHeapConglom = 
+                tc.recreateAndLoadConglomerate(
+                    "heap",
+                    false,
+                    fullTemplate.getRowArray(),
+                    null, //column sort order - not required for heap
+                    td.getColumnCollationIds(),
+                    properties,
+                    TransactionController.IS_DEFAULT,
+                    oldHeapConglom,
+                    sourceResultSet,
+                    loadedRowCount);
 		}
 
 		/* Nothing else to do if we get back the same conglomerate number.
@@ -1808,14 +1812,16 @@
 			sorters[index].completedInserts();
 			sorters[index] = null;
 			rowSources[index] = new CardinalityCounter(tc.openSortRowSource(sortIds[index]));
-			newIndexCongloms[index] = tc.createAndLoadConglomerate(
-										"BTREE",
-										indexRows[index].getRowArray(),
-										ordering[index],
-										properties,
-										TransactionController.IS_DEFAULT,
-										rowSources[index],
-										(long[]) null);
+			newIndexCongloms[index] = 
+                tc.createAndLoadConglomerate(
+                    "BTREE",
+                    indexRows[index].getRowArray(),
+                    ordering[index],
+                    (int[]) null, // TODO-COLLATION, set non default collation if necessary
+                    properties,
+                    TransactionController.IS_DEFAULT,
+                    rowSources[index],
+                    (long[]) null);
 
 			CardinalityCounter cCount = (CardinalityCounter)rowSources[index];
 			long numRows;
@@ -2265,14 +2271,16 @@
 			// We can finally drain the sorter and rebuild the index
 			// RESOLVE - all indexes are btrees right now
 			// Populate the index.
-			newIndexCongloms[index] = tc.createAndLoadConglomerate(
-										"BTREE",
-										indexRows[index].getRowArray(),
-										null, //default column sort order 
-										properties,
-										TransactionController.IS_DEFAULT,
-										rowSources[index],
-										(long[]) null);
+			newIndexCongloms[index] = 
+                tc.createAndLoadConglomerate(
+                    "BTREE",
+                    indexRows[index].getRowArray(),
+                    null, //default column sort order 
+                    (int[]) null, // TODO-COLLATION - set collation based on collation of original index.
+                    properties,
+                    TransactionController.IS_DEFAULT,
+                    rowSources[index],
+                    (long[]) null);
 
 			/* Update the DataDictionary
 			 *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MaterializedResultSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MaterializedResultSet.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MaterializedResultSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/MaterializedResultSet.java Thu Apr 12 09:58:30 2007
@@ -249,10 +249,16 @@
 
 				tc = activation.getTransactionController();
 
-				materializedCID = tc.createConglomerate("heap",	materializedRowBuffer.getRowArray(),
-														null, null,
-														TransactionController.IS_TEMPORARY |
-														TransactionController.IS_KEPT);
+				materializedCID = 
+                    tc.createConglomerate(
+                        "heap",	
+                        materializedRowBuffer.getRowArray(),
+                        null, 
+                        (int[]) null, // TODO-COLLATION, implement collation in materialized result sets if necessary
+                        null, 
+                        (TransactionController.IS_TEMPORARY |
+                         TransactionController.IS_KEPT));
+
 				materializedCreated = true;
 				materializedCC = 
                     tc.openConglomerate(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/TemporaryRowHolderImpl.java Thu Apr 12 09:58:30 2007
@@ -256,15 +256,54 @@
 		{
 			TransactionController tc = activation.getTransactionController();
 
+            // TODO-COLLATE, I think collation needs to get set always correctly
+            // but did see what to get collate id when there was no result
+            // description.  The problem comes if row holder is used to stream
+            // row to temp disk, then row is read from disk using an interface
+            // where store creates the DataValueDescriptor template itself, 
+            // and subsquently the returned column is used for some sort of
+            // comparison.  Also could be a problem is reader of tempoary 
+            // table uses qualifiers, that would result in comparisons internal
+            // to store.  I believe the below impl is incomplete - either
+            // it should always be default, or real collate_ids should be 
+            // passed in.
+
+            // null collate_ids in createConglomerate call indicates to use all
+            // default collate ids.
+            int collation_ids[] = null;
+
+            /*
+            TODO-COLLATE - if we could count on resultDescription I think the
+            following would work.
+
+            if (resultDescription != null)
+            {
+                // init collation id info from resultDescription for create call
+                collation_ids = new int[resultDescription.getColumnCount()];
+
+                for (int i = 0; i < collation_ids.length; i++)
+                {
+                    collation_ids[i] = 
+                        resultDescription.getColumnDescriptor(
+                            i + 1).getType().getCollationType();
+                }
+            }
+            */
+
+
 			/*
 			** Create the conglomerate with the template row.
 			*/
-			CID = tc.createConglomerate("heap",
-										inputRow.getRowArray(),
-										null, //column sort order - not required for heap
-										properties,
-										TransactionController.IS_TEMPORARY | 
-										TransactionController.IS_KEPT);
+			CID = 
+                tc.createConglomerate(
+                    "heap",
+                    inputRow.getRowArray(),
+                    null, //column sort order - not required for heap
+                    collation_ids,
+                    properties,
+                    TransactionController.IS_TEMPORARY | 
+                    TransactionController.IS_KEPT);
+
 			conglomCreated = true;
 
 			cc = tc.openConglomerate(CID, 
@@ -334,9 +373,15 @@
 				uniqueIndexRow[1] = baseRowLocation;
 				Properties props = makeIndexProperties(uniqueIndexRow, CID);
 				uniqueIndexConglomId =
-					tc.createConglomerate("BTREE",uniqueIndexRow , null,  props, 
-										  TransactionController.IS_TEMPORARY | 
-										  TransactionController.IS_KEPT);
+					tc.createConglomerate(
+                        "BTREE",
+                        uniqueIndexRow, 
+                        null,  
+                        null, // no collation needed for index on row locations.
+                        props, 
+                        (TransactionController.IS_TEMPORARY | 
+                         TransactionController.IS_KEPT));
+
 				uniqueIndex_cc = tc.openConglomerate(
 								uniqueIndexConglomId, 
 								false,
@@ -394,15 +439,23 @@
 			positionIndexRow[1] = rl;				
 			Properties props = makeIndexProperties(positionIndexRow, CID);
 			positionIndexConglomId =
-				tc.createConglomerate("BTREE", positionIndexRow, null,  props, 
-									  TransactionController.IS_TEMPORARY |
-									  TransactionController.IS_KEPT);
-			positionIndex_cc = tc.openConglomerate(
-													positionIndexConglomId, 
-													false,
-													TransactionController.OPENMODE_FORUPDATE,
-													TransactionController.MODE_TABLE,
-													TransactionController.ISOLATION_SERIALIZABLE);
+                tc.createConglomerate(
+                    "BTREE",
+                    positionIndexRow, 
+                    null,  
+                    null, // no collation needed for index on row locations.
+                    props, 
+                    (TransactionController.IS_TEMPORARY | 
+                     TransactionController.IS_KEPT));
+
+			positionIndex_cc = 
+                tc.openConglomerate(
+                    positionIndexConglomId, 
+                    false,
+                    TransactionController.OPENMODE_FORUPDATE,
+                    TransactionController.MODE_TABLE,
+                    TransactionController.ISOLATION_SERIALIZABLE);
+
 			positionIndexCreated = true;
 		}
 		

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java Thu Apr 12 09:58:30 2007
@@ -143,6 +143,7 @@
                     AccessFactoryGlobals.HEAP,
                     template, 
                     null, 
+                    (int[]) null, // use default collation for property conglom.
                     conglomProperties, 
                     TransactionController.IS_DEFAULT);
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Thu Apr 12 09:58:30 2007
@@ -601,6 +601,7 @@
 	@param conglomId        The identifier of the conglomerate to drop.
 	@param column_id        The column number to add this column at.
 	@param template_column  An instance of the column to be added to table.
+	@param collation_id     collation id of the added column.
 
 	@exception StandardException Only some types of conglomerates can support
         adding a column, for instance "heap" conglomerates support adding a
@@ -610,7 +611,8 @@
     public void addColumnToConglomerate(
     long        conglomId,
     int         column_id,
-    Storable    template_column)
+    Storable    template_column,
+    int         collation_id)
 		throws StandardException
     {
         boolean is_temporary = (conglomId < 0);
@@ -632,7 +634,7 @@
                 (StaticCompiledOpenConglomInfo) null,
                 (DynamicCompiledOpenConglomInfo) null);
 
-		conglom.addColumn(this, column_id, template_column);
+		conglom.addColumn(this, column_id, template_column, collation_id);
 
         // remove the old entry in the Conglomerate directory, and add the
         // new one.
@@ -802,6 +804,7 @@
     String                  implementation,
     DataValueDescriptor[]   template,
     ColumnOrdering[]        columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int                     temporaryFlag)
 		throws StandardException
@@ -839,7 +842,7 @@
         Conglomerate conglom =
             cfactory.createConglomerate(
                 this, segment, conglomid, template, 
-                columnOrder, properties, temporaryFlag);
+                columnOrder, collationIds, properties, temporaryFlag);
 
 		long conglomId;
 		if ((temporaryFlag & TransactionController.IS_TEMPORARY)
@@ -870,6 +873,7 @@
     String                  implementation,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]		columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int                     temporaryFlag,
     RowLocationRetRowSource rowSource,
@@ -882,6 +886,7 @@
                 true,
                 template,
 				columnOrder,
+                collationIds,
                 properties,
                 temporaryFlag,
                 0 /* unused if recreate_ifempty is true */,
@@ -900,6 +905,7 @@
     boolean                 recreate_ifempty,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]		columnOrder,
+    int[]                   collationIds,
     Properties              properties,
     int			            temporaryFlag,
     long                    orig_conglomId,
@@ -912,7 +918,8 @@
 		// necessary although still correct.
 		long conglomId = 
 			createConglomerate(
-                implementation, template, columnOrder, properties, temporaryFlag);
+                implementation, template, columnOrder, collationIds, 
+                properties, temporaryFlag);
 
         long rows_loaded = 
             loadConglomerate(
@@ -2148,6 +2155,34 @@
         rawtran.addPostCommitWork(work);
 
         return;
+    }
+
+    /**
+     *  Check to see if a database has been upgraded to the required
+     *  level in order to use a store feature.
+     *
+     * @param requiredMajorVersion  required database Engine major version
+     * @param requiredMinorVersion  required database Engine minor version
+     * @param feature               Non-null to throw an exception, null to 
+     *                              return the state of the version match.
+     *
+     * @return <code> true </code> if the database has been upgraded to 
+     *         the required level, <code> false </code> otherwise.
+     *
+     * @exception  StandardException 
+     *             if the database is not at the require version 
+     *             when <code>feature</code> feature is 
+     *             not <code> null </code>. 
+     */
+	public boolean checkVersion(
+    int     requiredMajorVersion, 
+    int     requiredMinorVersion, 
+    String  feature) 
+        throws StandardException
+    {
+        return(
+            accessmanager.getRawStore().checkVersion(
+                requiredMajorVersion, requiredMinorVersion, feature));
     }
 
     /**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTree.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTree.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTree.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/BTree.java Thu Apr 12 09:58:30 2007
@@ -165,12 +165,19 @@
     /**
     The array of format id's, one for each column in the template.
     **/
-    int[]    format_ids;
+    protected int[]    format_ids;
+
 
 	//columns sorting order information
 	// true - Ascending Order ; false -Descending Order
 	protected boolean[]	ascDescInfo;
 
+    /**
+    The array of collation id's for each column in the template.
+    **/
+    protected int[]   collation_ids;
+
+
 	/*
 	** Private Methods of BTree.
 	*/
@@ -308,7 +315,8 @@
 	public void addColumn(
     TransactionManager  xact_manager,
     int                 column_id,
-    Storable            template_column)
+    Storable            template_column,
+    int                 collation_id)
         throws StandardException
     {
         throw StandardException.newException(

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I.java Thu Apr 12 09:58:30 2007
@@ -33,76 +33,120 @@
 
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.error.StandardException;
+
+import org.apache.derby.impl.store.access.conglomerate.ConglomerateUtil;
 import org.apache.derby.iapi.store.access.conglomerate.LogicalUndo;
 import org.apache.derby.iapi.store.access.conglomerate.ScanManager;
 import org.apache.derby.iapi.store.access.conglomerate.TransactionManager;
+
+import org.apache.derby.iapi.store.access.ColumnOrdering;
 import org.apache.derby.iapi.store.access.ConglomerateController;
 import org.apache.derby.iapi.store.access.DynamicCompiledOpenConglomInfo;
 import org.apache.derby.iapi.store.access.Qualifier;
-import org.apache.derby.iapi.types.RowLocation;
 import org.apache.derby.iapi.store.access.RowLocationRetRowSource;
 import org.apache.derby.iapi.store.access.StaticCompiledOpenConglomInfo;
 import org.apache.derby.iapi.store.access.StoreCostController;
 import org.apache.derby.iapi.store.access.TransactionController;
-import org.apache.derby.iapi.store.access.ColumnOrdering;
 
-import org.apache.derby.iapi.services.io.StoredFormatIds;
 
 import org.apache.derby.iapi.store.raw.ContainerHandle;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
 import org.apache.derby.iapi.store.raw.Transaction;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
+import org.apache.derby.iapi.types.RowLocation;
+import org.apache.derby.iapi.types.StringDataValue;
 
 import org.apache.derby.impl.store.access.btree.BTree;
 import org.apache.derby.impl.store.access.btree.BTreeLockingPolicy;
 import org.apache.derby.impl.store.access.btree.LeafControlRow;
 import org.apache.derby.impl.store.access.btree.OpenBTree;
 
-import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.cache.ClassSize;
 
+import org.apache.derby.iapi.services.io.CompressedNumber;
+import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.services.io.StoredFormatIds;
+
+
 // For JavaDoc references (i.e. @see)
 import org.apache.derby.iapi.store.access.conglomerate.Conglomerate;
 
+
 /*
- * @format_id ACCESS_B2I_V1_ID
+ * @format_id ACCESS_B2I_V3_ID
  *
  * @purpose   The tag that describes the on disk representation of the B2I
- *            conglomerate object.  The B2I conglomerate object is stored in
- *            a field of a row in the Conglomerate directory.  
+ *            conglomerate object.  Access contains no "directory" of 
+ *            conglomerate information.  In order to bootstrap opening a file
+ *            it encodes the factory that can open the conglomerate in the 
+ *            conglomerate id itself.  There exists a single B2IFactory which
+ *            must be able to read all btree format id's.  
+ *
+ *            This format was used for all Derby database B2I's in version
+ *            10.2 and previous versions.
+ *
+ * @upgrade   The format id of this object is currently always read from disk
+ *            as the first field of the conglomerate itself.  A bootstrap
+ *            problem exists as we don't know the format id of the B2I 
+ *            until we are in the "middle" of reading the B2I.  Thus the
+ *            base B2I implementation must be able to read and write 
+ *            all formats based on the reading the 
+ *            "format_of_this_conglomerate". 
  *
- * @upgrade   This format was made obsolete in the kimono release.
+ *            soft upgrade to ACCESS_B2I_V4_ID:
+ *                read:
+ *                    old format is readable by current B2I implementation,
+ *                    with automatic in memory creation of default collation
+ *                    id needed by new format.  No code other than
+ *                    readExternal and writeExternal need know about old format.
+ *                write:
+ *                    will never write out new format id in soft upgrade mode.
+ *                    Code in readExternal and writeExternal handles writing
+ *                    correct version.  Code in the factory handles making
+ *                    sure new conglomerates use the B2I_v10_2 class to 
+ *                    that will write out old format info.
+ *
+ *            hard upgrade to ACCESS_B2I_V4_ID:
+ *                read:
+ *                    old format is readable by current B2I implementation,
+ *                    with automatic in memory creation of default collation
+ *                    id needed by new format.
+ *                write:
+ *                    Only "lazy" upgrade will happen.  New format will only
+ *                    get written for new conglomerate created after the 
+ *                    upgrade.  Old conglomerates continue to be handled the
+ *                    same as soft upgrade.
  *
  * @disk_layout 
+ *     format_of_this_conlgomerate(byte[])
  *     containerid(long)
  *     segmentid(int)
  *     number_of_key_fields(int)
  *     number_of_unique_columns(int)
  *     allow_duplicates(boolean)
  *     maintain_parent_links(boolean)
- *     format_of_this_conlgomerate(byte[])
  *     array_of_format_ids(byte[][])
  *     baseConglomerateId(long)
  *     rowLocationColumn(int)
+ *     ascend_column_info(FormatableBitSet)
+ *
  */
 
 /*
- * @format_id ACCESS_B2I_V2_ID
+ * @format_id ACCESS_B2I_V4_ID
  *
  * @purpose   The tag that describes the on disk representation of the B2I
- *            conglomerate object.  The B2I conglomerate object is stored in
- *            a field of a row in the Conglomerate directory.  
+ *            conglomerate object.  Access contains no "directory" of 
+ *            conglomerate information.  In order to bootstrap opening a file
+ *            it encodes the factory that can open the conglomerate in the 
+ *            conglomerate id itself.  There exists a single B2IFactory which
+ *            must be able to read all btree format id's.  
  *
- * @upgrade   The format id of this object is currently always read from disk
- *            as a separate column in the conglomerate directory.  To read
- *            A conglomerate object from disk and upgrade it to the current
- *            version do the following:
- *
- *                format_id = get format id from a separate column
- *                Upgradable conglom_obj = instantiate empty obj(format_id)
- *                read in conglom_obj from disk
- *                conglom = conglom_obj.upgradeToCurrent();
+ *            This format is the current version id of B2I and has been used 
+ *            in versions of Derby after the 10.2 release.
+ *
+ * @upgrade   This is the current version, no upgrade necessary.
  *
  * @disk_layout 
  *     format_of_this_conlgomerate(byte[])
@@ -115,6 +159,9 @@
  *     array_of_format_ids(byte[][])
  *     baseConglomerateId(long)
  *     rowLocationColumn(int)
+ *     ascend_column_info(FormatableBitSet)
+ *     collation_ids(compressed array of ints)
+ *
  */
 
 /**
@@ -140,7 +187,7 @@
     private static final String PROPERTY_BASECONGLOMID = "baseConglomerateId";
     private static final String PROPERTY_ROWLOCCOLUMN  = "rowLocationColumn";
 
-    static final int FORMAT_NUMBER = StoredFormatIds.ACCESS_B2I_V3_ID;
+    static final int FORMAT_NUMBER = StoredFormatIds.ACCESS_B2I_V4_ID;
 
 	/*
 	** Fields of B2I.
@@ -164,7 +211,8 @@
 	**/
 	int rowLocationColumn;
 
-    private static final int BASE_MEMORY_USAGE = ClassSize.estimateBaseFromCatalog( B2I.class);
+    private static final int BASE_MEMORY_USAGE = 
+        ClassSize.estimateBaseFromCatalog( B2I.class);
 
     public int estimateMemoryUsage()
     {
@@ -392,6 +440,7 @@
     long                    input_conglomid, 
     DataValueDescriptor[]	template, 
 	ColumnOrdering[]	    columnOrder,
+    int[]                   collationIds,
     Properties              properties,
 	int                     temporaryFlag)
 		throws StandardException
@@ -460,13 +509,14 @@
         }
 
 
-		/* covert the sorting order information into a boolean array map.
+		/* convert the sorting order information into a boolean array map.
 		 * If the sorting order for the columns is not provided, we
 		 * assign the default as Ascending Order.
-		 * array length is equla to template length , because column order
-		 * length changes wther it is unique is non unique. store assumes
-		 * template length arrays. So , we make  template length array and make
-		 * the last column as ascending instead of having lot of execeptions code.
+		 * array length is equal to template length, because column order
+		 * length changes whether it is unique or is non unique. store assumes
+		 * template length arrays. So, we make template length array and make
+		 * the last column as ascending instead of having lot of execeptions 
+         * code.
 		 */
 		
 		ascDescInfo = new boolean[template.length];
@@ -478,8 +528,15 @@
 				ascDescInfo[i] = true;  // default values - ascending order
 		}
 
+        // get collation ids from input collation ids, store it in the 
+        // conglom state.
+        collation_ids = 
+            ConglomerateUtil.createCollationIds(template.length, collationIds);
+
 		// Do the generic part of creating the b-tree.
-		super.create(rawtran, segmentId, input_conglomid, template, properties, getTypeFormatId(), temporaryFlag);
+		super.create(
+            rawtran, segmentId, input_conglomid, template, 
+            properties, getTypeFormatId(), temporaryFlag);
 
         // open the base conglomerate - to get the lock
         ConglomerateController base_cc = 
@@ -498,7 +555,8 @@
                 TransactionController.MODE_TABLE,
                 rawtran.newLockingPolicy(
                     LockingPolicy.MODE_CONTAINER,
-                    TransactionController.ISOLATION_SERIALIZABLE, true), base_cc, open_btree);
+                    TransactionController.ISOLATION_SERIALIZABLE, true), 
+                base_cc, open_btree);
 
 
         // The following call will "open" the new btree.  Create is
@@ -562,7 +620,8 @@
      * @param lock_level      One of (MODE_TABLE, MODE_RECORD, or MODE_NONE).
      *
      * @param isolation_level The isolation level to lock the conglomerate at.
-     *                        One of (ISOLATION_READ_COMMITTED or ISOLATION_SERIALIZABLE).
+     *                        One of (ISOLATION_READ_COMMITTED or 
+     *                        ISOLATION_SERIALIZABLE).
      *
 	 * @param scanColumnList  A description of which columns to return from 
      *                        every fetch in the scan.  template, 
@@ -940,103 +999,118 @@
 		return StoredFormatIds.ACCESS_B2I_V3_ID;
 	}
 
-	/**
-	Store the stored representation of the column value in the stream.
-	It might be easier to simply store the properties - which would certainly
-	make upgrading easier.*/
-	public void writeExternal_v36(ObjectOutput out) throws IOException {
-		super.writeExternal(out);
-		out.writeLong(baseConglomerateId);
-		out.writeInt(rowLocationColumn);
-	}
-
-	/**
-	Restore the in-memory representation from the stream.
-
-	@exception ClassNotFoundException Thrown if the stored representation is
-	serialized and a class named in the stream could not be found.
-
-	@see java.io.Externalizable#readExternal
-	*/
-	public void readExternal_v36(ObjectInput in)
-		throws IOException, ClassNotFoundException
-	{
-		super.readExternal(in);
-		
-		// XXX (nat) need to improve error handling
-		baseConglomerateId = in.readLong();
-		rowLocationColumn = in.readInt();
-		//set the default (Ascending) sort order
-		ascDescInfo = new boolean[nKeyFields];
-		for (int i=0 ; i < ascDescInfo.length; i++)
-			ascDescInfo[i] = true;
-	}
-
-
-	/**
-	Store the stored representation of the column value in the stream.
-	It might be easier to simply store the properties - which would certainly
-	make upgrading easier.
 
-    */
-	public void writeExternal(ObjectOutput out) throws IOException {
+    /**
+     * Store the stored representation of the column value in the
+     * stream.
+     * <p>
+     * For more detailed description of the ACCESS_B2I_V3_ID format see 
+     * documentation at top of file.
+     *
+     * @see java.io.Externalizable#writeExternal
+     **/
+	public void writeExternal_v10_2(ObjectOutput out) 
+        throws IOException 
+    {
 		super.writeExternal(out);
 		out.writeLong(baseConglomerateId);
 		out.writeInt(rowLocationColumn);
 
-		// if the conglomerate type is not not the version2
-		// sorting information is stored from version V3(release 3.7)
-		if (conglom_format_id != StoredFormatIds.ACCESS_B2I_V2_ID)
-		{
-			//write the coulmsn sort information as bits
-			FormatableBitSet ascDescBits = new FormatableBitSet(ascDescInfo.length);
-			for (int i = 0; i < ascDescInfo.length; i++)
-			{	
-				if (ascDescInfo[i])
-					ascDescBits.set(i);
-			}
-			ascDescBits.writeExternal(out);
-		}
-
+        //write the columns ascend/descend information as bits
+        FormatableBitSet ascDescBits = 
+            new FormatableBitSet(ascDescInfo.length);
+
+        for (int i = 0; i < ascDescInfo.length; i++)
+        {	
+            if (ascDescInfo[i])
+                ascDescBits.set(i);
+        }
+        ascDescBits.writeExternal(out);
 	}
 
-	/**
-	Restore the in-memory representation from the stream.
+    /**
+     * Store the stored representation of the column value in the
+     * stream.
+     * <p>
+     * For more detailed description of the ACCESS_B2I_V3_ID and 
+     * ACCESS_B2I_V4_ID formats see documentation at top of file.
+     *
+     * @see java.io.Externalizable#writeExternal
+     **/
+	public void writeExternal(ObjectOutput out) 
+        throws IOException 
+    {
+        // First part of ACCESS_B2I_V4_ID format is the ACCESS_B2I_V3_ID format.
+        writeExternal_v10_2(out);
 
-	@exception ClassNotFoundException Thrown if the stored representation is
-	serialized and a class named in the stream could not be found.
+		if (conglom_format_id == StoredFormatIds.ACCESS_B2I_V4_ID)
+        {
+            // Now append sparse array of collation ids
+            ConglomerateUtil.writeCollationIdArray(collation_ids, out);
+        }
+	}
 
-	@see java.io.Externalizable#readExternal
-	*/
+    /**
+     * Restore the in-memory representation from the stream.
+     * <p>
+     *
+     * @exception ClassNotFoundException Thrown if the stored representation 
+     *                                   is serialized and a class named in 
+     *                                   the stream could not be found.
+     *
+     * @see java.io.Externalizable#readExternal
+     **/
 	private final void localReadExternal(ObjectInput in)
 		throws IOException, ClassNotFoundException
 	{
 		super.readExternal(in);
 		
-		// XXX (nat) need to improve error handling
 		baseConglomerateId = in.readLong();
-		rowLocationColumn = in.readInt();
+		rowLocationColumn  = in.readInt();
 
-		// if the conglomerate type is  not the version2
-		// sorting info is avaialable  from version v3(release 3.7)
-		if (conglom_format_id != StoredFormatIds.ACCESS_B2I_V2_ID)
-		{
-			// read the column sort order info
-			FormatableBitSet ascDescBits = new FormatableBitSet();
-			ascDescBits.readExternal(in);
-			ascDescInfo = new boolean[ascDescBits.getLength()];
-			for(int i =0 ; i < ascDescBits.getLength(); i++)
-				ascDescInfo[i] = ascDescBits.isSet(i);
-		}
-		else
-		{
-			//set the default (Ascending) sort order
-			ascDescInfo = new boolean[nKeyFields];
-			for (int i=0 ; i < ascDescInfo.length; i++)
-				ascDescInfo[i] = true;
+        // read the column sort order info
+        FormatableBitSet ascDescBits = new FormatableBitSet();
+        ascDescBits.readExternal(in);
+        ascDescInfo = new boolean[ascDescBits.getLength()];
+        for(int i =0 ; i < ascDescBits.getLength(); i++)
+            ascDescInfo[i] = ascDescBits.isSet(i);
+
+        // In memory maintain a collation id per column in the template.
+        collation_ids = new int[format_ids.length];
+
+        // initialize all the entries to COLLATION_TYPE_UCS_BASIC, 
+        // and then reset as necessary.  For version ACCESS_B2I_V3_ID,
+        // this is the default and no resetting is necessary.
+        for (int i = 0; i < format_ids.length; i++)
+            collation_ids[i] = StringDataValue.COLLATION_TYPE_UCS_BASIC;
 
-		}
+		if (conglom_format_id == StoredFormatIds.ACCESS_B2I_V4_ID)
+        {
+            // current format id, read collation info from disk
+
+            if (SanityManager.DEBUG)
+            {
+                // length must include row location column and at least
+                // one other field.
+                SanityManager.ASSERT(
+                    collation_ids.length >= 2, 
+                    "length = " + collation_ids.length);
+            }
+
+            ConglomerateUtil.readCollationIdArray(collation_ids, in);
+        }
+        else if (conglom_format_id != StoredFormatIds.ACCESS_B2I_V3_ID)
+        {
+            // Currently only V3 and V4 should be possible in a Derby DB.
+            // Actual work for V3 is handled by default code above, so no
+            // special work is necessary.
 
+            if (SanityManager.DEBUG)
+            {
+                SanityManager.THROWASSERT(
+                    "Unexpected format id: " + conglom_format_id);
+            }
+        }
 	}
 
 	public void readExternal(ObjectInput in)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java?view=diff&rev=528033&r1=528032&r2=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2IFactory.java Thu Apr 12 09:58:30 2007
@@ -40,6 +40,7 @@
 import org.apache.derby.iapi.store.raw.ContainerKey;
 import org.apache.derby.iapi.store.raw.ContainerHandle;
 import org.apache.derby.iapi.store.raw.LockingPolicy;
+import org.apache.derby.iapi.store.raw.RawStoreFactory;
 
 import org.apache.derby.iapi.types.DataValueDescriptor;
 
@@ -165,14 +166,36 @@
     long                    input_containerid,
     DataValueDescriptor[]   template,
 	ColumnOrdering[]        columnOrder,
+    int[]                   collationIds,
     Properties              properties,
 	int                     temporaryFlag)
             throws StandardException
 	{
-		B2I btree = new B2I();
+        B2I btree = null;
+
+        if (xact_mgr.checkVersion(
+                RawStoreFactory.DERBY_STORE_MAJOR_VERSION_10,
+                RawStoreFactory.DERBY_STORE_MINOR_VERSION_3,
+                null))
+        {
+            // on disk databases with version higher than 10.2 should use
+            // current disk format B2I.  This includes new databases or
+            // hard upgraded databases.
+            btree = new B2I();
+        }
+        else
+        {
+            // Old databases that are running in new versions of the software,
+            // but are running in soft upgrade mode at release level 10.2
+            // and before should use the old B2I version.  This version will
+            // continue to write metadata that can be read by 10.2 and previous
+            // versions.
+            btree = new B2I_v10_2();
+        }
+
 		btree.create(
             xact_mgr, segment, input_containerid, template, columnOrder, 
-            properties, temporaryFlag);
+            collationIds, properties, temporaryFlag);
 
 		return(btree);
 	}

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I_v10_2.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I_v10_2.java?view=auto&rev=528033
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I_v10_2.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/btree/index/B2I_v10_2.java Thu Apr 12 09:58:30 2007
@@ -0,0 +1,162 @@
+/*
+
+   Derby - Class org.apache.derby.impl.store.access.btree.index.B2I
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+ */
+
+package org.apache.derby.impl.store.access.btree.index;
+
+
+import org.apache.derby.iapi.services.io.StoredFormatIds; 
+
+import java.io.IOException; 
+import java.io.ObjectOutput;
+import java.io.ObjectInput;
+
+import java.lang.ClassNotFoundException;
+
+
+/*
+ * @format_id ACCESS_B2I_V3_ID
+ *
+ * @purpose   The tag that describes the on disk representation of the B2I
+ *            conglomerate object.  Access contains no "directory" of 
+ *            conglomerate information.  In order to bootstrap opening a file
+ *            it encodes the factory that can open the conglomerate in the 
+ *            conglomerate id itself.  There exists a single B2IFactory which
+ *            must be able to read all btree format id's.  
+ *
+ *            This format was used for all Derby database B2I's in version
+ *            10.2 and previous versions.
+ *
+ * @upgrade   The format id of this object is currently always read from disk
+ *            as the first field of the conglomerate itself.  A bootstrap
+ *            problem exists as we don't know the format id of the B2I 
+ *            until we are in the "middle" of reading the B2I.  Thus the
+ *            base B2I implementation must be able to read and write 
+ *            all formats based on the reading the 
+ *            "format_of_this_conglomerate". 
+ *
+ *            soft upgrade to ACCESS_B2I_V4_ID:
+ *                read:
+ *                    old format is readable by current B2I implementation,
+ *                    with automatic in memory creation of default collation
+ *                    id needed by new format.  No code other than
+ *                    readExternal and writeExternal need know about old format.
+ *                write:
+ *                    will never write out new format id in soft upgrade mode.
+ *                    Code in readExternal and writeExternal handles writing
+ *                    correct version.  Code in the factory handles making
+ *                    sure new conglomerates use the B2I_v10_2 class to 
+ *                    that will write out old format info.
+ *
+ *            hard upgrade to ACCESS_B2I_V4_ID:
+ *                read:
+ *                    old format is readable by current B2I implementation,
+ *                    with automatic in memory creation of default collation
+ *                    id needed by new format.
+ *                write:
+ *                    Only "lazy" upgrade will happen.  New format will only
+ *                    get written for new conglomerate created after the 
+ *                    upgrade.  Old conglomerates continue to be handled the
+ *                    same as soft upgrade.
+ *
+ * @disk_layout 
+ *     format_of_this_conlgomerate(byte[])
+ *     containerid(long)
+ *     segmentid(int)
+ *     number_of_key_fields(int)
+ *     number_of_unique_columns(int)
+ *     allow_duplicates(boolean)
+ *     maintain_parent_links(boolean)
+ *     array_of_format_ids(byte[][])
+ *     baseConglomerateId(long)
+ *     rowLocationColumn(int)
+ *     ascend_column_info(FormatableBitSet)
+ *
+ */
+
+/**
+ * Class used to instantiate 10.2 version of the B2I object.
+ *
+ * This class implements the format of the B2I object as existed in 
+ * the 10.2 and previous releases of Derby.  In subsequent releases
+ * the format was enhanced to store the Collation Id of the columns
+ * in the index.  
+ *
+ * Collation can be configured on a per column basis to allow for
+ * alter sort ordering of each column.  One use of this is to allow
+ * a column to be sorted according to language based rules rather
+ * than the default numerical ordering of the binary value.
+ *
+ * For upgrade purpose all columns stored with ACCESS_B2I_V3_ID format
+ * are assumed to be USC_BASIC collation id (ie. the default numerical
+ * ordering, rather than any alternate collation).  
+ *
+ * This class reads and writes the V3 version to/from disk and reads/writes
+ * current in-memory version of the data structure.
+ *
+ */
+public class B2I_v10_2 extends B2I
+{
+
+    /**
+     * No arg constructor, required by Formatable.
+     **/
+    public B2I_v10_2()
+    {
+        super();
+    }
+
+
+    /**************************************************************************
+     * Public Methods required by Storable interface, implies 
+     *     Externalizable, TypedFormat:
+     **************************************************************************
+     */
+
+    /**
+     * Return my format identifier.
+     * <p>
+     * This identifier was used for B2I in all Derby versions prior to and
+     * including 10.2.  Databases hard upgraded to a version subsequent
+     * to 10.2 will write the new format, see B2I.  Databases created in
+     * a version subsequent to 10.2 will also write the new formate, see
+     * B2I.
+     *
+     * @see org.apache.derby.iapi.services.io.TypedFormat#getTypeFormatId
+     **/
+	public int getTypeFormatId() 
+    {
+		return StoredFormatIds.ACCESS_B2I_V3_ID;
+	}
+
+    /**
+     * Store the stored representation of the column value in the
+     * stream.
+     * <p>
+     * For more detailed description of the format see documentation
+     * at top of file.
+     *
+     * @see java.io.Externalizable#writeExternal
+     **/
+	public void writeExternal(ObjectOutput out) throws IOException 
+    {
+		super.writeExternal_v10_2(out);
+	}
+}