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 dj...@apache.org on 2006/12/14 20:10:46 UTC

svn commit: r487307 - in /db/derby/code/trunk/java: engine/org/apache/derby/catalog/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/impl/db/ engine/org/apache/derby/impl/sql/catalog/ engine/org/apache/derby/impl/sql/compile/ engin...

Author: djd
Date: Thu Dec 14 11:10:45 2006
New Revision: 487307

URL: http://svn.apache.org/viewvc?view=rev&rev=487307
Log:
DERBY-2138 Remove the DataDictionaryContext and instead use the fact there is a single DataDictionary per database.
Cleanup DependableFinder to take a passed in reference to a data dictionary rather than trying to find the data
dictionary from the context.

Removed:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/DataDictionaryContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryContextImpl.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/ConsistencyChecker.java
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/DependableFinder.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/db/BasicDatabase.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDColumnDependableFinder.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDdependableFinder.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/compile/TablePrivilegesNode.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateViewConstantAction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/T_ConsistencyChecker.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/DependableFinder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/catalog/DependableFinder.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/DependableFinder.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/DependableFinder.java Thu Dec 14 11:10:45 2006
@@ -23,6 +23,9 @@
 
 import java.sql.SQLException;
 
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+
 /**
 	
   A DependableFinder is an object that can find an in-memory
@@ -43,12 +46,14 @@
 	/**
 	  *	Get the in-memory object associated with the passed-in object ID.
 	  *
+      * @param dd DataDictionary to use for lookup.
 	  *	@param	dependableObjectID the ID of a Dependable. Used to locate that Dependable.
 	  *
 	  *	@return	the associated Dependable
-	  * @exception SQLException		thrown on error
+	  * @exception StandardException		thrown if the object cannot be found or on error o
 	  */
-	public	Dependable	getDependable(UUID dependableObjectID) throws SQLException;
+    public	Dependable	getDependable(DataDictionary dd,
+            UUID dependableObjectID) throws StandardException;
 
 	/**
 	  * The name of the class of Dependables as a "SQL Object" which this
@@ -61,17 +66,4 @@
 	  * @see Dependable
 	  */
 	public	String	getSQLObjectType();
-
-	/**
-	  * Get the name of the SQL Object that corresponds to the specified 
-	  * UUID String. For example, if getSQLObjectType() returns "Table", 
-	  * this will return the table name.
-	  *
-	  *	@param	idString the UUID String of a Dependable. Used to locate that Dependable.
-	  *
-	  *	@return	String		Name of the associated Dependable
-	  * @exception SQLException		thrown on error
-	  */
-	public	String	getSQLObjectName(String idString) throws SQLException;
-
 }

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?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- 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 Thu Dec 14 11:10:45 2006
@@ -177,14 +177,6 @@
 	public static final int DDL_MODE = 1;
 
 	/**
-	 * Push a data dictionary context onto the
-	 * current context manager.
-	 *
-	 * @param cm Context manager
-	 */
-	DataDictionaryContext pushDataDictionaryContext(ContextManager cm);
-
-	/**
 	 * Clear all of the DataDictionary caches.
 	 *
 	 * @exception StandardException Standard Cloudscape error policy

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java Thu Dec 14 11:10:45 2006
@@ -54,7 +54,6 @@
 import org.apache.derby.iapi.sql.conn.LanguageConnectionFactory;
 
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.DataDictionaryContext;
 import org.apache.derby.iapi.sql.dictionary.FileInfoDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.sql.dictionary.SPSDescriptor;
@@ -793,10 +792,6 @@
 	public StorageFile getJarFile(String schemaName, String sqlName)
 		throws StandardException {
 
-		DataDictionaryContext ddc =
-			(DataDictionaryContext) ContextService.getContext(DataDictionaryContext.CONTEXT_ID);
-		DataDictionary dd = getDataDictionary();
-
 		SchemaDescriptor sd = dd.getSchemaDescriptor(schemaName, null, true);
 		FileInfoDescriptor fid = dd.getFileInfoDescriptor(sd,sqlName);
 		if (fid == null)
@@ -804,7 +799,8 @@
 
 		long generationId = fid.getGenerationId();
 
-		FileResource fr = af.getTransaction(ddc.getContextManager()).getFileHandler();
+        ContextManager cm = ContextService.getFactory().getCurrentContextManager();
+		FileResource fr = af.getTransaction(cm).getFileHandler();
 
 		String externalName = org.apache.derby.impl.sql.execute.JarDDL.mkExternalName(schemaName, sqlName, fr.getSeparatorChar());
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDColumnDependableFinder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDColumnDependableFinder.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDColumnDependableFinder.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDColumnDependableFinder.java Thu Dec 14 11:10:45 2006
@@ -106,7 +106,7 @@
 	}
 
 	/**
-	 * Get a dependable object, which is essentially a table descriptor with
+	 * Find a dependable object, which is essentially a table descriptor with
 	 * referencedColumnMap field set.
 	 *
 	 * @param	dd data dictionary
@@ -114,7 +114,7 @@
 	 * @return	a dependable, a table descriptor with referencedColumnMap
 	 *			field set
 	 */
-	protected Dependable getDependable(DataDictionary dd, UUID dependableObjectID)
+	Dependable findDependable(DataDictionary dd, UUID dependableObjectID)
 		throws StandardException
 	{
 		TableDescriptor td = dd.getTableDescriptor(dependableObjectID);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDdependableFinder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDdependableFinder.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDdependableFinder.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DDdependableFinder.java Thu Dec 14 11:10:45 2006
@@ -21,42 +21,27 @@
 
 package	org.apache.derby.impl.sql.catalog;
 
-import org.apache.derby.iapi.services.context.ContextManager;
-import org.apache.derby.iapi.services.context.ContextService;
-
-import org.apache.derby.iapi.services.monitor.Monitor;
-
-import org.apache.derby.iapi.services.sanity.SanityManager;
-
-import org.apache.derby.iapi.services.io.StoredFormatIds;
-import org.apache.derby.iapi.services.io.Formatable;
-
-import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
-import org.apache.derby.iapi.sql.dictionary.DefaultDescriptor;
-import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.DataDictionaryContext;
-import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
-
-import org.apache.derby.iapi.error.StandardException;
-import org.apache.derby.iapi.error.PublicAPI;
-
-import org.apache.derby.iapi.services.i18n.MessageService;
-
-import org.apache.derby.iapi.services.uuid.UUIDFactory;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 
-import org.apache.derby.catalog.UUID;
 import org.apache.derby.catalog.Dependable;
 import org.apache.derby.catalog.DependableFinder;
-
+import org.apache.derby.catalog.UUID;
+import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.SQLState;
-
-import java.io.ObjectOutput;
-import java.io.ObjectInput;
-import java.io.IOException;
+import org.apache.derby.iapi.services.io.Formatable;
+import org.apache.derby.iapi.services.io.StoredFormatIds;
+import org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.sql.dictionary.ColumnDescriptor;
+import org.apache.derby.iapi.sql.dictionary.DataDictionary;
+import org.apache.derby.iapi.sql.dictionary.DefaultDescriptor;
 
 /**
- *	Class for all DependableFinders in the core DataDictionary
+ *	Class for most DependableFinders in the core DataDictionary.
+ * This class is stored in SYSDEPENDS for the finders for
+ * the provider and dependent. It stores no state, its functionality
+ * is driven off its format identifier.
  *
  *
  * @author Rick
@@ -70,9 +55,6 @@
 	//
 	////////////////////////////////////////////////////////////////////////
 
-	private transient DataDictionary			dataDictionary;
-	private transient UUIDFactory				uuidFactory;
-
 	private final int formatId;
 
 	////////////////////////////////////////////////////////////////////////
@@ -146,50 +128,6 @@
 	////////////////////////////////////////////////////////////////////////
 
 	/**
-	  *	Gets the AliasDescriptor associated with the passed-in object ID.
-	  *
-	  *	@param	dependableObjectID	the object ID of an Alias. Used to locate its AliasDescriptor
-	  *
-	  *	@return	the associated AliasDescriptor
-	  * @exception java.sql.SQLException		thrown on error
-	  */
-	public final Dependable	getDependable(UUID dependableObjectID)
-		 throws java.sql.SQLException
-	{
-		try 
-		{
-			return getDependable(getDataDictionary(),dependableObjectID);
-		} 
-		catch (StandardException se) 
-		{
-			throw PublicAPI.wrapStandardException(se);
-		}
-	}
-
-	/**
-	  * @see DependableFinder#getSQLObjectName
-	  * @exception java.sql.SQLException		thrown on error
-	  */
-	public final String	getSQLObjectName(String idString) throws java.sql.SQLException
-	{
-
-		try {
-
-			// This should really be getDependable(idString).getObjectName()
-			// and then the sub-classes would not have to provide a getSQLObjectName
-			// method. Currently getDependable(idString).getObjectName() would
-			// not always return the same result - fix in main.
-
-			return getSQLObjectName(getDataDictionary(), recreateUUID(idString));
-		} 
-		catch (StandardException se) 
-		{
-			throw PublicAPI.wrapStandardException( se );
-		}
-	}
-
-
-	/**
 	  * @see DependableFinder#getSQLObjectType
 	  */
 	public	String	getSQLObjectType()
@@ -249,93 +187,74 @@
 	}
 
 	/**
-	  *	Gets the datadictionary for this connection.
-	  *
-	  *	@return	the data dictionary for this connection
-	  *
-	  * @exception StandardException		Thrown on failure
-	  */
-	private	DataDictionary	getDataDictionary()
-						throws StandardException
-	{
-		if ( dataDictionary == null )
-	    {
-			ContextManager				cm  = ContextService.getFactory().getCurrentContextManager();
-			DataDictionaryContext		ddc = (DataDictionaryContext)
-			                              (cm.getContext(DataDictionaryContext.CONTEXT_ID));
-			dataDictionary = ddc.getDataDictionary();
-		}
-		return	dataDictionary;
-	}
-
-	/**
-	 * Get the UUID for the given string
-	 *
-	 * @param idString the string
-	 *
-	 * @return the UUID
-	 */
-	private UUID recreateUUID(String idString)
-	{
-		if (uuidFactory == null)
-		{
-			uuidFactory = Monitor.getMonitor().getUUIDFactory();
-		}
-		return uuidFactory.recreateUUID(idString);
-	}
-
-	/**
 		Get the dependable for the given UUID
 		@exception StandardException thrown on error
 	*/
-	protected Dependable getDependable(DataDictionary dd, UUID dependableObjectID)
+	public final Dependable getDependable(DataDictionary dd, UUID dependableObjectID)
 		throws StandardException
 	{
+        Dependable dependable = findDependable(dd, dependableObjectID);
+        if (dependable == null)
+            throw StandardException.newException(SQLState.LANG_OBJECT_NOT_FOUND,
+                    getSQLObjectType(), dependableObjectID);
+        return dependable;
+    }
+        
+       
+    /**
+     * Find the dependable for getDependable.
+     * Can return a null references, in which case getDependable()
+     * will thrown an exception.
+     */
+    Dependable findDependable(DataDictionary dd, UUID dependableObjectID)
+        throws StandardException
+    {     
 		switch (formatId)
 		{
 			case StoredFormatIds.ALIAS_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getAliasDescriptor(dependableObjectID);
+                return dd.getAliasDescriptor(dependableObjectID);
 
 			case StoredFormatIds.CONGLOMERATE_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getConglomerateDescriptor(dependableObjectID);
+                return dd.getConglomerateDescriptor(dependableObjectID);
 
 			case StoredFormatIds.CONSTRAINT_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getConstraintDescriptor(dependableObjectID);
+                return dd.getConstraintDescriptor(dependableObjectID);
 
 			case StoredFormatIds.DEFAULT_DESCRIPTOR_FINDER_V01_ID:
 				ColumnDescriptor	cd = dd.getColumnDescriptorByDefaultId(dependableObjectID);
-				DefaultDescriptor ddi = new DefaultDescriptor(
+                if (cd != null)
+                    return new DefaultDescriptor(
 												dd, 
 												cd.getDefaultUUID(), cd.getReferencingUUID(), 
 												cd.getPosition());
-				return ddi;
+                return null;
 
 			case StoredFormatIds.FILE_INFO_FINDER_V01_ID:
-				return dd.getFileInfoDescriptor(dependableObjectID);
+                return dd.getFileInfoDescriptor(dependableObjectID);
 
 			case StoredFormatIds.SCHEMA_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getSchemaDescriptor(dependableObjectID, null);
+                return dd.getSchemaDescriptor(dependableObjectID, null);
 
 			case StoredFormatIds.SPS_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getSPSDescriptor(dependableObjectID);
+                return dd.getSPSDescriptor(dependableObjectID);
 
 			case StoredFormatIds.TABLE_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getTableDescriptor(dependableObjectID);
+                return dd.getTableDescriptor(dependableObjectID);
 
 			case StoredFormatIds.TRIGGER_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getTriggerDescriptor(dependableObjectID);
-
+                return dd.getTriggerDescriptor(dependableObjectID);
+ 
 			case StoredFormatIds.VIEW_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getViewDescriptor(dependableObjectID);
+                return dd.getViewDescriptor(dependableObjectID);
 
-			case StoredFormatIds.COLUMNS_PERMISSION_FINDER_V01_ID:
-				return dd.getColumnPermissions(dependableObjectID);
+            case StoredFormatIds.COLUMNS_PERMISSION_FINDER_V01_ID:
+                return dd.getColumnPermissions(dependableObjectID);
 
 			case StoredFormatIds.TABLE_PERMISSION_FINDER_V01_ID:
-				return dd.getTablePermissions(dependableObjectID);
+                return dd.getTablePermissions(dependableObjectID);
 
 			case StoredFormatIds.ROUTINE_PERMISSION_FINDER_V01_ID:
-				return dd.getRoutinePermissions(dependableObjectID);
+                return dd.getRoutinePermissions(dependableObjectID);
 
 			default:
 				if (SanityManager.DEBUG)
@@ -343,73 +262,7 @@
 					SanityManager.THROWASSERT(
 						"getDependable() called with unexpeced formatId = " + formatId);
 				}
-				return null;
-		}
-	}
-
-	/**
-		Get the SQL object name for the given UUID
-		@exception StandardException thrown on error
-	*/
-	protected String getSQLObjectName(DataDictionary dd, UUID dependableObjectID)
-		throws StandardException
-	{
-		switch (formatId)
-		{
-			case StoredFormatIds.ALIAS_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getAliasDescriptor(dependableObjectID).getDescriptorName();
-
-			case StoredFormatIds.CONGLOMERATE_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getConglomerateDescriptor(dependableObjectID).getConglomerateName();
-
-			case StoredFormatIds.CONSTRAINT_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getConstraintDescriptor(dependableObjectID).getConstraintName();
-
-			case StoredFormatIds.DEFAULT_DESCRIPTOR_FINDER_V01_ID:
-				ColumnDescriptor columnDescriptor = dd.getColumnDescriptorByDefaultId( dependableObjectID );
-				TableDescriptor tableDescriptor = dd.getTableDescriptor(
-										columnDescriptor.getReferencingUUID());
-
-				return	MessageService.getTextMessage(
-							SQLState.LANG_COLUMN_DEFAULT,
-							tableDescriptor.getQualifiedName() + "." +
-							columnDescriptor.getColumnName());
-
-			case StoredFormatIds.FILE_INFO_FINDER_V01_ID:
-				return dd.getFileInfoDescriptor(dependableObjectID).getName();
-
-			case StoredFormatIds.SCHEMA_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getSchemaDescriptor(dependableObjectID, null).getSchemaName();
-
-			case StoredFormatIds.SPS_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getSPSDescriptor(dependableObjectID).getName();
-
-			case StoredFormatIds.TABLE_DESCRIPTOR_FINDER_V01_ID:
-			case StoredFormatIds.COLUMN_DESCRIPTOR_FINDER_V01_ID:
-				return getDependable(dd, dependableObjectID).getObjectName();
-
-			case StoredFormatIds.TRIGGER_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getTriggerDescriptor(dependableObjectID).getName();
-
-			case StoredFormatIds.VIEW_DESCRIPTOR_FINDER_V01_ID:
-				return dd.getTableDescriptor(dependableObjectID).getName();
-
-			case StoredFormatIds.COLUMNS_PERMISSION_FINDER_V01_ID:
-				return dd.getColumnPermissions(dependableObjectID).getObjectName();
-
-			case StoredFormatIds.TABLE_PERMISSION_FINDER_V01_ID:
-				return dd.getTablePermissions(dependableObjectID).getObjectName();
-
-			case StoredFormatIds.ROUTINE_PERMISSION_FINDER_V01_ID:
-				return dd.getRoutinePermissions(dependableObjectID).getObjectName();
-
-			default:
-				if (SanityManager.DEBUG)
-				{
-					SanityManager.THROWASSERT(
-						"getSQLObjectName() called with unexpeced formatId = " + formatId);
-				}
-				return null;
+                return null;
 		}
-	}
+    }
 }

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=487307&r1=487306&r2=487307
==============================================================================
--- 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 Dec 14 11:10:45 2006
@@ -39,7 +39,6 @@
 import org.apache.derby.iapi.sql.dictionary.ConstraintDescriptorList;
 import org.apache.derby.iapi.sql.dictionary.DataDescriptorGenerator;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
-import org.apache.derby.iapi.sql.dictionary.DataDictionaryContext;
 import org.apache.derby.iapi.sql.dictionary.DefaultDescriptor;
 import org.apache.derby.iapi.sql.dictionary.DependencyDescriptor;
 import org.apache.derby.iapi.sql.dictionary.ForeignKeyConstraintDescriptor;
@@ -642,9 +641,6 @@
 		if (SanityManager.DEBUG)
 			SanityManager.ASSERT((cm != null), "Failed to get current ContextManager");
 
-		/* push a datadictionary context onto this stack */
-		pushDataDictionaryContext(cm);
-
 		// RESOLVE other non-StandardException errors.
 		bootingTC = null;
 		try
@@ -723,7 +719,6 @@
 				bootingTC.destroy();  // gets rid of the transaction context
 				bootingTC = null;
 			}
-			cm.popContext(); // the data dictionary context; check that it is?
 		}
 	
 		setDependencyManager();
@@ -1221,18 +1216,6 @@
 	{
 		return exFactory;
 	}
-
-	/**
-	 * @see DataDictionary#pushDataDictionaryContext
-	 */
-	public DataDictionaryContext pushDataDictionaryContext(ContextManager contextManager)
-	{
-		DataDictionaryContextImpl dataDictionaryContextImpl =
-			new DataDictionaryContextImpl(contextManager, this);
-
-		return dataDictionaryContextImpl;
-	}
-
 
     /* We defer getting the builtin schemas (system and default) past boot time so that
      * the language connection context will be available.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TablePrivilegesNode.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TablePrivilegesNode.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TablePrivilegesNode.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/compile/TablePrivilegesNode.java Thu Dec 14 11:10:45 2006
@@ -150,28 +150,14 @@
 		int siz = pis.length;
 		for (int i=0; i < siz; i++) 
 		{
-			try 
-			{
-				Provider provider = (Provider) pis[i].getDependableFinder().getDependable(pis[i].getObjectId());
-				if (provider == null)  
-				{
-					throw StandardException.newException(
-							SQLState.LANG_OBJECT_NOT_FOUND, 
-							"OBJECT", 
-							pis[i].getObjectId());
-				}
+				Provider provider = (Provider) pis[i].getDependableFinder().getDependable(dd, pis[i].getObjectId());
 							
 				if (provider instanceof TableDescriptor || 
 					provider instanceof ViewDescriptor ||
 					provider instanceof AliasDescriptor)
 				{
 					descriptorList.add(provider);
-				}
-			}
-			catch(java.sql.SQLException ex)
-			{
-				throw StandardException.plainWrapException(ex);
-			}		   
+				}	   
 		}
 	}
 	

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionFactory.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/conn/GenericLanguageConnectionFactory.java Thu Dec 14 11:10:45 2006
@@ -169,8 +169,6 @@
 		String drdaID,
 		String dbname) throws StandardException {
 		
-		db.getDataDictionary().pushDataDictionaryContext(cm);
-
 		return new GenericLanguageConnectionContext(cm,
 													tc,
 													lf,

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java Thu Dec 14 11:10:45 2006
@@ -321,20 +321,10 @@
 							for (int i = 0; i < providerInfos.length; i++)
 							{
 								Provider provider = null;
-								try
-								{
 									provider = (Provider) providerInfos[i].
 													getDependableFinder().
-													getDependable(
+													getDependable(dd,
 													providerInfos[i].getObjectId());
-								}
-								catch(java.sql.SQLException te)
-								{
-									if (SanityManager.DEBUG)
-									{
-										SanityManager.THROWASSERT("unexpected java.sql.SQLException - " + te);
-									}
-								}
 								if (provider instanceof TableDescriptor)
 								{
 									TableDescriptor tab = (TableDescriptor)provider;
@@ -1044,25 +1034,26 @@
 				for (ListIterator depsIterator = storedDeps.listIterator();
 					 depsIterator.hasNext(); )
 				{
-					DependencyDescriptor dd = (DependencyDescriptor)depsIterator.next();
+					DependencyDescriptor dependDescr =
+                        (DependencyDescriptor) depsIterator.next();
 
 					if (! foundStored)
 					{
 						debugBuf.append("Stored Dependencies:\n");
 						foundStored = true;
 					}
-
-					dependStr[i++] = new String(
-									dd.getProviderFinder().getSQLObjectName(
-										dd.getProviderID().toString()) +
-									", type " +
-									dd.getProviderFinder().getSQLObjectType() +
-									", provides for " +
-									dd.getDependentFinder().getSQLObjectName(
-										dd.getUUID().toString()) +
-									", type " +
-									dd.getDependentFinder().getSQLObjectType() +
-									"\n");
+                    
+                    DependableFinder providerFinder = dependDescr.getProviderFinder();
+                    DependableFinder dependentFinder = dependDescr.getDependentFinder();
+                    
+
+					dependStr[i++] =
+                        providerFinder.getDependable(dd, dependDescr.getProviderID()).getObjectName() +
+                        ", type " + providerFinder.getSQLObjectType() +
+                        ", provides for " +
+                        dependentFinder.getDependable(dd, dependDescr.getUUID()).getObjectName() +
+                        ", type " + dependentFinder.getSQLObjectType() +
+                        "\n";
 				}
 
 				// sort stored dependencies; dependStr
@@ -1198,9 +1189,8 @@
 
 				DependencyDescriptor depDesc = (DependencyDescriptor) depsIterator.next();
 
-				try {
 					finder = depDesc.getDependentFinder();
-					tempD = (Dependent) finder.getDependable( depDesc.getUUID() );
+					tempD = (Dependent) finder.getDependable(dd, depDesc.getUUID() );
 
 					if (providerForList != null)
 					{
@@ -1218,14 +1208,9 @@
 					else
 					{
 						finder = depDesc.getProviderFinder();
-						tempP = (Provider) finder.getDependable( depDesc.getProviderID() );
+						tempP = (Provider) finder.getDependable(dd, depDesc.getProviderID() );
 						
 					}
-
-				} catch (java.sql.SQLException te) {
-					throw StandardException.newException(SQLState.DEP_UNABLE_TO_RESTORE, finder.getClass().getName(), te.getMessage());
-
-				}
 
 				depsIterator.set(new BasicDependency(tempD, tempP));
 			}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateConstraintConstantAction.java Thu Dec 14 11:10:45 2006
@@ -374,20 +374,11 @@
 				Provider provider = null;
 	
 				/* We should always be able to find the Provider */
-				try 
-				{
 					provider = (Provider) providerInfo[ix].
 											getDependableFinder().
-												getDependable(
+												getDependable(dd, 
 													providerInfo[ix].getObjectId());
-				}
-				catch(java.sql.SQLException te)
-				{
-					if (SanityManager.DEBUG)
-					{
-						SanityManager.THROWASSERT("unexpected java.sql.SQLException - " + te);
-					}
-				}
+
 				dm.addDependency(conDesc, provider, lcc.getContextManager());
 			}
 		}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateViewConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateViewConstantAction.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateViewConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/CreateViewConstantAction.java Thu Dec 14 11:10:45 2006
@@ -199,23 +199,11 @@
 		for (int ix = 0; ix < providerInfo.length; ix++)
 		{
 			/* We should always be able to find the Provider */
-			try 
-			{
 				Provider provider = (Provider) providerInfo[ix].
 										getDependableFinder().
-											getDependable(
+											getDependable(dd,
 												providerInfo[ix].getObjectId());
-				if (provider == null)  //see beetle 4444
-				{
-					throw StandardException.newException(SQLState.LANG_OBJECT_NOT_FOUND, "OBJECT", providerInfo[ix].getObjectId());
-				}
 				dm.addDependency(vd, provider, lcc.getContextManager());
-			}
-			catch(java.sql.SQLException te)
-			{
-				// we should allow timeout to be thrown
-				throw StandardException.plainWrapException(te);
-			}
 		}
 		//store view's dependency on various privileges in the dependeny system
 		storeViewTriggerDependenciesOnPrivileges(activation, vd);

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/T_ConsistencyChecker.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/T_ConsistencyChecker.java?view=diff&rev=487307&r1=487306&r2=487307
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/T_ConsistencyChecker.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/util/T_ConsistencyChecker.java Thu Dec 14 11:10:45 2006
@@ -24,7 +24,6 @@
 
 import org.apache.derby.iapi.error.StandardException;
 
-import org.apache.derby.iapi.sql.dictionary.DataDictionaryContext;
 import org.apache.derby.iapi.sql.dictionary.DataDictionary;
 import org.apache.derby.iapi.sql.dictionary.SchemaDescriptor;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
@@ -447,103 +446,93 @@
 	{
 		return id;
 	}
+    /**
+     * Run all of the consistency checkers which do not take parameters.
+     * Actually, just run the ones that "make sense" to run.  Today,
+     * that is:
+     *      countOpens()
+     *
+     * @return String       If an inconsistency is found, and if DEBUG is on, 
+     *                      then a string will be returned with more info.  
+     *                      If DEBUG is off, then a simple string will be 
+     *                      returned stating whether or not there are open scans.
+     *
+     * @exception StandardException     Thrown on error
+     * @exception java.sql.SQLException     Thrown on error
+     */
+    public static String runConsistencyChecker() throws StandardException, java.sql.SQLException
+    {
+        return countOpens() + countDependencies();
+    }
+
+    /**
+     * Check to make sure that there are no open conglomerates, scans or sorts.
+     *
+     * @return String       If an inconsistency is found, and if DEBUG is on, 
+     *                      then a string will be returned with more info.  
+     *                      If DEBUG is off, then a simple string will be 
+     *                      returned stating whether or not there are open scans.
+     *
+     * @exception StandardException     Thrown on error
+     */
+    public static String countOpens() throws StandardException
+    {
+        int                     numOpens = 0;
+        LanguageConnectionContext lcc;
+        String                  output = "No open scans, etc.\n";
+        TransactionController   tc;
+
+        lcc = (LanguageConnectionContext)
+            ContextService.getContext(LanguageConnectionContext.CONTEXT_ID);
+        tc = lcc.getTransactionExecute();
 
+        numOpens = tc.countOpens(TransactionController.OPEN_TOTAL);
 
-	// following methods are originally from a different class - used in the test store/backupRestore1
-	// original comment for that class:
-	/**
-	 * This class provides static methods for checking the consistency of database
-	 * objects like tables.
-	 */
-
-	/**
-	 * Run all of the consistency checkers which do not take parameters.
-	 * Actually, just run the ones that "make sense" to run.  Today,
-	 * that is:
-	 *		countOpens()
-	 *
-	 * @return String		If an inconsistency is found, and if DEBUG is on, 
-	 *						then a string will be returned with more info.  
-	 *						If DEBUG is off, then a simple string will be 
-	 *						returned stating whether or not there are open scans.
-	 *
-	 * @exception StandardException		Thrown on error
-	 * @exception java.sql.SQLException		Thrown on error
-	 */
-	public static String runConsistencyChecker() throws StandardException, java.sql.SQLException
-	{
-		return countOpens() + countDependencies();
-	}
-
-	/**
-	 * Check to make sure that there are no open conglomerates, scans or sorts.
-	 *
-	 * @return String		If an inconsistency is found, and if DEBUG is on, 
-	 *						then a string will be returned with more info.  
-	 *						If DEBUG is off, then a simple string will be 
-	 *						returned stating whether or not there are open scans.
-	 *
-	 * @exception StandardException		Thrown on error
-	 */
-	public static String countOpens() throws StandardException
-	{
-		int						numOpens = 0;
-		LanguageConnectionContext lcc;
-		String					output = "No open scans, etc.\n";
-		TransactionController	tc;
-
-		lcc = (LanguageConnectionContext)
-			ContextService.getContext(LanguageConnectionContext.CONTEXT_ID);
-		tc = lcc.getTransactionExecute();
-
-		numOpens = tc.countOpens(TransactionController.OPEN_TOTAL);
-
-		if (numOpens > 0)
-		{
+        if (numOpens > 0)
+        {
             output = numOpens + " conglomerates/scans/sorts found open\n";
 
-		}
-
-		return output;
-	}
-
-	/**
-	 * Check to make sure that there are no active dependencies (stored or
-	 * in memory).
-	 *
-	 * @return String		If an inconsistency is found, and if DEBUG is on, 
-	 *						then a string will be returned with more info.  
-	 *						If DEBUG is off, then a simple string will be 
-	 *						returned stating whether or not there are open scans.
-	 *
-	 * @exception StandardException		Thrown on error
-	 * @exception java.sql.SQLException		Thrown on error
-	 */
-	public static String countDependencies() throws StandardException, java.sql.SQLException
-	{
-		int						numDependencies = 0;
-		DataDictionary			dd;
-		DataDictionaryContext	ddc;
-		DependencyManager		dm;
-		StringBuffer			debugBuf = new StringBuffer();
-
-		ddc = (DataDictionaryContext)
-				(ContextService.getContext(DataDictionaryContext.CONTEXT_ID));
-
-		dd = ddc.getDataDictionary();
-		dm = dd.getDependencyManager();
+        }
 
-		numDependencies = dm.countDependencies();
+        return output;
+    }
 
-		if (numDependencies > 0)
-		{
-			debugBuf.append(numDependencies + " dependencies found");
-		}
-		else
-		{
-			debugBuf.append("No outstanding dependencies.\n");
-		}
+    /**
+     * Check to make sure that there are no active dependencies (stored or
+     * in memory).
+     *
+     * @return String       If an inconsistency is found, and if DEBUG is on, 
+     *                      then a string will be returned with more info.  
+     *                      If DEBUG is off, then a simple string will be 
+     *                      returned stating whether or not there are open scans.
+     *
+     * @exception StandardException     Thrown on error
+     * @exception java.sql.SQLException     Thrown on error
+     */
+    public static String countDependencies() throws StandardException, java.sql.SQLException
+    {
+        int                     numDependencies = 0;
+        DataDictionary          dd;
+        DependencyManager       dm;
+        StringBuffer            debugBuf = new StringBuffer();
+
+        LanguageConnectionContext lcc = (LanguageConnectionContext)
+             ContextService.getContext(LanguageConnectionContext.CONTEXT_ID);
+
+        dd = lcc.getDataDictionary();
+        dm = dd.getDependencyManager();
+
+        numDependencies = dm.countDependencies();
+
+        if (numDependencies > 0)
+        {
+            debugBuf.append(numDependencies + " dependencies found");
+        }
+        else
+        {
+            debugBuf.append("No outstanding dependencies.\n");
+        }
 
-		return debugBuf.toString();
-	}
+        return debugBuf.toString();
+    }
 }