You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cayenne.apache.org by aa...@apache.org on 2008/05/06 15:31:06 UTC

svn commit: r653768 - in /cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src: main/java/org/apache/cayenne/access/trans/ test/java/org/apache/cayenne/access/ test/java/org/apache/cayenne/access/trans/

Author: aadamchik
Date: Tue May  6 06:31:03 2008
New Revision: 653768

URL: http://svn.apache.org/viewvc?rev=653768&view=rev
Log:
CAY-802  Split Expressions
(refactoring of QueryAssembler to support split joins. NOte that it makes it incompatible with 2.0 version)

Added:
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinStack.java
      - copied, changed from r653758, cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinProcessor.java
Removed:
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinProcessor.java
Modified:
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/DeleteTranslator.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinTreeNode.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QualifierTranslator.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssembler.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssemblerHelper.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/SelectTranslator.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/TrimmingQualifierTranslator.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/UpdateTranslator.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/DataContextJoinAliasesTest.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/SelectTranslatorTest.java
    cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/TstQueryAssembler.java

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/DeleteTranslator.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/DeleteTranslator.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/DeleteTranslator.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/DeleteTranslator.java Tue May  6 06:31:03 2008
@@ -34,18 +34,18 @@
 public class DeleteTranslator extends QueryAssembler {
 
     @Override
-    public String aliasForTable(DbEntity dbEnt) {
-        throw new RuntimeException("aliases not supported");
+    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
+        throw new UnsupportedOperationException("db relationships not supported");
     }
 
     @Override
-    public void dbRelationshipAdded(DbRelationship dbRel) {
-        throw new RuntimeException("db relationships not supported");
+    public String getCurrentAlias() {
+        throw new UnsupportedOperationException("aliases not supported");
     }
 
     @Override
-    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
-        throw new RuntimeException("db relationships not supported");
+    public void resetJoinStack() {
+        // noop - joins are not supported
     }
 
     /**

Copied: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinStack.java (from r653758, cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinProcessor.java)
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinStack.java?p2=cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinStack.java&p1=cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinProcessor.java&r1=653758&r2=653768&rev=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinProcessor.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinStack.java Tue May  6 06:31:03 2008
@@ -34,25 +34,34 @@
  * @since 3.0
  * @author Andrus Adamchik
  */
-class JoinProcessor {
+class JoinStack {
 
-    private JoinTreeNode root;
+    private JoinTreeNode rootNode;
     private JoinTreeNode topNode;
 
-    JoinProcessor(SelectTranslator tableAliasSource) {
-        String rootAlias = tableAliasSource.aliasForTable(tableAliasSource
-                .getRootDbEntity());
-        this.root = new JoinTreeNode(tableAliasSource);
-        this.root.setTargetTableAlias(rootAlias);
+    private int aliasCounter;
+
+    JoinStack() {
+        this.rootNode = new JoinTreeNode(this);
+        this.rootNode.setTargetTableAlias(newAlias());
         resetStack();
     }
+    
+    String getCurrentAlias() {
+        return topNode.getTargetTableAlias();
+    }
 
     /**
      * Returns the number of configured joins.
      */
     int size() {
         // do not count root as a join
-        return root.size() - 1;
+        return rootNode.size() - 1;
+    }
+
+    void appendRoot(Appendable out, DbEntity rootEntity) throws IOException {
+        out.append(rootEntity.getFullyQualifiedName());
+        out.append(' ').append(rootNode.getTargetTableAlias());
     }
 
     /**
@@ -61,7 +70,7 @@
     void appendJoins(Appendable out) throws IOException {
 
         // skip root, recursively append its children
-        for (JoinTreeNode child : root.getChildren()) {
+        for (JoinTreeNode child : rootNode.getChildren()) {
             appendJoinSubtree(out, child);
         }
     }
@@ -119,7 +128,7 @@
      * Pops the stack all the way to the root node.
      */
     void resetStack() {
-        topNode = root;
+        topNode = rootNode;
     }
 
     /**
@@ -129,4 +138,8 @@
     void pushJoin(DbRelationship relationship, JoinType joinType, String alias) {
         topNode = topNode.findOrCreateChild(relationship, joinType, alias);
     }
+
+    protected String newAlias() {
+        return "t" + aliasCounter++;
+    }
 }

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinTreeNode.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinTreeNode.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinTreeNode.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/JoinTreeNode.java Tue May  6 06:31:03 2008
@@ -22,7 +22,6 @@
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.cayenne.map.DbEntity;
 import org.apache.cayenne.map.DbRelationship;
 import org.apache.cayenne.map.JoinType;
 import org.apache.cayenne.util.Util;
@@ -39,15 +38,15 @@
     private String alias;
     private JoinType joinType;
     private Collection<JoinTreeNode> children;
-    private SelectTranslator tableAliasSource;
+    private JoinStack joinProcessor;
 
-    JoinTreeNode(SelectTranslator tableAliasSource) {
-        this.tableAliasSource = tableAliasSource;
+    JoinTreeNode(JoinStack joinProcessor) {
+        this.joinProcessor = joinProcessor;
     }
 
-    JoinTreeNode(SelectTranslator tableAliasSource, DbRelationship relationship,
+    JoinTreeNode(JoinStack joinProcessor, DbRelationship relationship,
             JoinType joinType, String alias) {
-        this(tableAliasSource);
+        this(joinProcessor);
         this.relationship = relationship;
         this.alias = alias;
         this.joinType = joinType;
@@ -87,13 +86,12 @@
         }
 
         JoinTreeNode child = new JoinTreeNode(
-                tableAliasSource,
+                joinProcessor,
                 relationship,
                 joinType,
                 alias);
         child.setSourceTableAlias(this.targetTableAlias);
-        child.setTargetTableAlias(tableAliasSource
-                .newAliasForTable((DbEntity) relationship.getTargetEntity()));
+        child.setTargetTableAlias(joinProcessor.newAlias());
         children.add(child);
         return child;
     }

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QualifierTranslator.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QualifierTranslator.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QualifierTranslator.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QualifierTranslator.java Tue May  6 06:31:03 2008
@@ -142,8 +142,8 @@
         matchingObject = false;
 
         boolean first = true;
-        DbRelationship relationship = objectMatchTranslator.getRelationship();
 
+        DbRelationship relationship = objectMatchTranslator.getRelationship();
         if (!relationship.isToMany() && !relationship.isToPK()) {
             queryAssembler.dbRelationshipAdded(relationship, JoinType.INNER);
         }
@@ -161,7 +161,7 @@
             DbAttribute attr = objectMatchTranslator.getAttribute(key);
             Object val = objectMatchTranslator.getValue(key);
 
-            processColumn(qualBuf, attr, relationship);
+            processColumn(qualBuf, attr);
             qualBuf.append(objectMatchTranslator.getOperation());
             appendLiteral(qualBuf, val, attr, objectMatchTranslator.getExpression());
         }

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssembler.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssembler.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssembler.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssembler.java Tue May  6 06:31:03 2008
@@ -28,7 +28,6 @@
 import org.apache.cayenne.access.QueryLogger;
 import org.apache.cayenne.access.QueryTranslator;
 import org.apache.cayenne.map.DbAttribute;
-import org.apache.cayenne.map.DbEntity;
 import org.apache.cayenne.map.DbRelationship;
 import org.apache.cayenne.map.JoinType;
 import org.apache.cayenne.query.QualifiedQuery;
@@ -49,18 +48,11 @@
     protected List<DbAttribute> attributes = new ArrayList<DbAttribute>();
 
     /**
-     * Processes a join being added.
-     * 
-     * @deprecated since 3.0 use {@link #dbRelationshipAdded(DbRelationship, JoinType)}.
-     */
-    public abstract void dbRelationshipAdded(DbRelationship dbRel);
-
-    /**
-     * Returns aliases for the joins defined in the query.
+     * Returns aliases for the path splits defined in the query.
      * 
      * @since 3.0
      */
-    protected Map<String, String> getJoinAliases() {
+    protected Map<String, String> getPathAliases() {
         if (query instanceof QualifiedQuery) {
             return ((QualifiedQuery) query).getJoinAliases();
         }
@@ -68,6 +60,21 @@
     }
 
     /**
+     * A callback invoked by a child qualifier or ordering processor allowing query
+     * assembler to reset its join stack.
+     * 
+     * @since 3.0
+     */
+    public abstract void resetJoinStack();
+
+    /**
+     * Returns an alias of the table which is currently at the top of the join stack.
+     * 
+     * @since 3.0
+     */
+    public abstract String getCurrentAlias();
+
+    /**
      * Appends a join with given semantics to the query.
      * 
      * @since 3.0
@@ -83,20 +90,6 @@
      */
     public abstract String createSqlString() throws Exception;
 
-    public String aliasForTable(DbEntity ent, DbRelationship rel) {
-        return aliasForTable(ent); // Default implementation
-    }
-
-    /**
-     * Returns a name that can be used as column alias. This can be one of the following:
-     * <ul>
-     * <li>an alias for this table, if it uses aliases</li>
-     * <li>a fully qualified table name, if not.</li>
-     * </ul>
-     * CayenneRuntimeException is thrown if a table alias can not be created.
-     */
-    public abstract String aliasForTable(DbEntity dbEnt);
-
     /**
      * Returns <code>true</code> if table aliases are supported. Default implementation
      * returns false.

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssemblerHelper.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssemblerHelper.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssemblerHelper.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/QueryAssemblerHelper.java Tue May  6 06:31:03 2008
@@ -38,6 +38,7 @@
 import org.apache.cayenne.map.ObjEntity;
 import org.apache.cayenne.map.ObjRelationship;
 import org.apache.cayenne.map.PathComponent;
+import org.apache.cayenne.util.CayenneMapEntry;
 
 /**
  * Translates parts of the query to SQL. Always works in the context of parent Translator.
@@ -88,10 +89,10 @@
      */
     protected void appendObjPath(StringBuffer buf, Expression pathExp) {
 
-        ObjRelationship lastRelationship = null;
+        queryAssembler.resetJoinStack();
 
         for (PathComponent<ObjAttribute, ObjRelationship> component : getObjEntity()
-                .resolvePath(pathExp, queryAssembler.getJoinAliases())) {
+                .resolvePath(pathExp, queryAssembler.getPathAliases())) {
 
             ObjRelationship relationship = component.getRelationship();
             ObjAttribute attribute = component.getAttribute();
@@ -110,27 +111,36 @@
                                 .dbRelationshipAdded(dbRel, component.getJoinType());
                     }
                 }
-                lastRelationship = relationship;
             }
             else {
-                if (lastRelationship != null) {
-                    List<DbRelationship> lastDbRelList = lastRelationship
-                            .getDbRelationships();
-                    DbRelationship lastDbRel = lastDbRelList
-                            .get(lastDbRelList.size() - 1);
-                    processColumn(buf, attribute.getDbAttribute(), lastDbRel);
-                }
-                else {
-                    processColumn(buf, attribute.getDbAttribute());
+                Iterator<CayenneMapEntry> dbPathIterator = attribute.getDbPathIterator();
+                while (dbPathIterator.hasNext()) {
+                    Object pathPart = dbPathIterator.next();
+
+                    if (pathPart == null) {
+                        throw new CayenneRuntimeException(
+                                "ObjAttribute has no component: " + attribute.getName());
+                    }
+                    else if (pathPart instanceof DbRelationship) {
+                        queryAssembler.dbRelationshipAdded(
+                                (DbRelationship) pathPart,
+                                JoinType.INNER);
+                    }
+                    else if (pathPart instanceof DbAttribute) {
+                        processColumn(buf, (DbAttribute) pathPart);
+                    }
                 }
+
             }
         }
     }
 
     protected void appendDbPath(StringBuffer buf, Expression pathExp) {
 
+        queryAssembler.resetJoinStack();
+
         for (PathComponent<DbAttribute, DbRelationship> component : getDbEntity()
-                .resolvePath(pathExp, queryAssembler.getJoinAliases())) {
+                .resolvePath(pathExp, queryAssembler.getPathAliases())) {
 
             DbRelationship relationship = component.getRelationship();
 
@@ -153,49 +163,9 @@
         }
     }
 
-    /** 
-     * Appends column name of a column in a root entity.
-     * 
-     *  @deprecated since 3.0 - unused
-     */
-    protected void processColumn(StringBuffer buf, Expression nameExp) {
-        if (queryAssembler.supportsTableAliases()) {
-            String alias = queryAssembler.aliasForTable(getDbEntity());
-            buf.append(alias).append('.');
-        }
-
-        buf.append(nameExp.getOperand(0));
-    }
-
-    protected void processColumn(
-            StringBuffer buf,
-            DbAttribute dbAttr,
-            DbRelationship relationship) {
-        String alias = null;
-
-        if (queryAssembler.supportsTableAliases()) {
-
-            if (relationship != null) {
-                alias = queryAssembler.aliasForTable(
-                        (DbEntity) dbAttr.getEntity(),
-                        relationship);
-            }
-
-            // sometimes lookup for relationship fails (any specific case other than
-            // relationship being null?), so lookup by entity. Note that as CAY-194
-            // shows, lookup by DbEntity may produce incorrect results for
-            // reflexive relationships.
-            if (alias == null) {
-                alias = queryAssembler.aliasForTable((DbEntity) dbAttr.getEntity());
-            }
-        }
-
-        buf.append(dbAttr.getAliasedName(alias));
-    }
-
     protected void processColumn(StringBuffer buf, DbAttribute dbAttr) {
         String alias = (queryAssembler.supportsTableAliases()) ? queryAssembler
-                .aliasForTable((DbEntity) dbAttr.getEntity()) : null;
+                .getCurrentAlias() : null;
 
         buf.append(dbAttr.getAliasedName(alias));
     }
@@ -325,7 +295,7 @@
                     PathComponent<ObjAttribute, ObjRelationship> last = getObjEntity()
                             .lastPathComponent(
                                     expression,
-                                    queryAssembler.getJoinAliases());
+                                    queryAssembler.getPathAliases());
 
                     // TODO: handle EmbeddableAttribute
                     // if (last instanceof EmbeddableAttribute)
@@ -349,7 +319,7 @@
                     PathComponent<DbAttribute, DbRelationship> last = getDbEntity()
                             .lastPathComponent(
                                     expression,
-                                    queryAssembler.getJoinAliases());
+                                    queryAssembler.getPathAliases());
                     if (last.getAttribute() != null) {
                         attribute = last.getAttribute();
                         break;
@@ -383,20 +353,6 @@
      * primary key. If this is a "to one" relationship, column expression for the source
      * foreign key is added.
      * 
-     * @deprecated since 3.0 use
-     *             {@link #processRelTermination(StringBuffer, ObjRelationship, JoinType)}.
-     */
-    protected void processRelTermination(StringBuffer buf, ObjRelationship rel) {
-        processRelTermination(buf, rel, JoinType.INNER);
-
-    }
-
-    /**
-     * Processes case when an OBJ_PATH expression ends with relationship. If this is a "to
-     * many" relationship, a join is added and a column expression for the target entity
-     * primary key. If this is a "to one" relationship, column expression for the source
-     * foreign key is added.
-     * 
      * @since 3.0
      */
     protected void processRelTermination(
@@ -427,19 +383,6 @@
      * primary key. If this is a "to one" relationship, column expression for the source
      * foreign key is added.
      * 
-     * @deprecated since 3.0 use
-     *             {@link #processRelTermination(StringBuffer, DbRelationship, JoinType)}.
-     */
-    protected void processRelTermination(StringBuffer buf, DbRelationship rel) {
-        processRelTermination(buf, rel, JoinType.INNER);
-    }
-
-    /**
-     * Handles case when a DB_NAME expression ends with relationship. If this is a "to
-     * many" relationship, a join is added and a column expression for the target entity
-     * primary key. If this is a "to one" relationship, column expression for the source
-     * foreign key is added.
-     * 
      * @since 3.0
      */
     protected void processRelTermination(

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/SelectTranslator.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/SelectTranslator.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/SelectTranslator.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/SelectTranslator.java Tue May  6 06:31:03 2008
@@ -76,19 +76,12 @@
         return false;
     }
 
-    final Map<DbRelationship, String> aliasLookup = new HashMap<DbRelationship, String>();
-
-    final List<DbEntity> tableList = new ArrayList<DbEntity>();
-    final List<String> aliasList = new ArrayList<String>();
-    final List<DbRelationship> dbRelList = new ArrayList<DbRelationship>();
-    final List<JoinType> dbRelationshipSemantics = new ArrayList<JoinType>();
+    JoinStack joinStack = new JoinStack();
 
     List<ColumnDescriptor> resultColumns;
     Map attributeOverrides;
     Map<ColumnDescriptor, ObjAttribute> defaultAttributesByColumn;
 
-    int aliasCounter;
-
     boolean suppressingDistinct;
 
     /**
@@ -171,11 +164,8 @@
         queryBuf.append(" FROM ");
 
         // append tables and joins
-        appendRootTable(queryBuf);
-        int joinCount = dbRelList.size();
-        for (int i = 0; i < joinCount; i++) {
-            appendJoins(queryBuf, i);
-        }
+        joinStack.appendRoot(queryBuf, getRootDbEntity());
+        joinStack.appendJoins(queryBuf);
 
         // append qualifier
         if (qualifierStr != null) {
@@ -191,6 +181,11 @@
         return queryBuf.toString();
     }
 
+    @Override
+    public String getCurrentAlias() {
+        return joinStack.getCurrentAlias();
+    }
+
     /**
      * Returns a list of ColumnDescriptors for the query columns.
      * 
@@ -234,9 +229,6 @@
 
         this.defaultAttributesByColumn = new HashMap<ColumnDescriptor, ObjAttribute>();
 
-        // create alias for root table
-        newAliasForTable(getRootDbEntity());
-
         List<ColumnDescriptor> columns = new ArrayList<ColumnDescriptor>();
         SelectQuery query = getSelectQuery();
 
@@ -278,6 +270,8 @@
 
             public boolean visitAttribute(AttributeProperty property) {
                 ObjAttribute oa = property.getAttribute();
+
+                resetJoinStack();
                 Iterator<CayenneMapEntry> dbPathIterator = oa.getDbPathIterator();
                 while (dbPathIterator.hasNext()) {
                     Object pathPart = dbPathIterator.next();
@@ -286,7 +280,6 @@
                         throw new CayenneRuntimeException(
                                 "ObjAttribute has no component: " + oa.getName());
                     }
-
                     else if (pathPart instanceof DbRelationship) {
                         DbRelationship rel = (DbRelationship) pathPart;
                         dbRelationshipAdded(rel, JoinType.INNER);
@@ -352,9 +345,10 @@
 
                 // add joins and find terminating element
 
+                resetJoinStack();
                 PathComponent<DbAttribute, DbRelationship> lastComponent = null;
                 for (PathComponent<DbAttribute, DbRelationship> component : table
-                        .resolvePath(pathExp, getJoinAliases())) {
+                        .resolvePath(pathExp, getPathAliases())) {
 
                     // do not add join for the last DB Rel
                     if (component.getRelationship() != null && !component.isLast()) {
@@ -419,9 +413,10 @@
                 Expression prefetchExp = Expression.fromString(prefetch.getPath());
                 Expression dbPrefetch = oe.translateToDbPath(prefetchExp);
 
+                resetJoinStack();
                 DbRelationship r = null;
                 for (PathComponent<DbAttribute, DbRelationship> component : table
-                        .resolvePath(dbPrefetch, getJoinAliases())) {
+                        .resolvePath(dbPrefetch, getPathAliases())) {
                     r = component.getRelationship();
                     dbRelationshipAdded(r, JoinType.INNER);
                 }
@@ -522,8 +517,7 @@
                         + customAttributes.get(i));
             }
 
-            String alias = aliasForTable((DbEntity) attribute.getEntity());
-            columns.add(new ColumnDescriptor(attribute, alias));
+            columns.add(new ColumnDescriptor(attribute, getCurrentAlias()));
         }
 
         return columns;
@@ -537,7 +531,7 @@
             String label) {
 
         if (skipSet.add(attribute)) {
-            String alias = aliasForTable((DbEntity) attribute.getEntity());
+            String alias = getCurrentAlias();
             ColumnDescriptor column = (objAttribute != null) ? new ColumnDescriptor(
                     objAttribute,
                     attribute,
@@ -577,67 +571,12 @@
         }
     }
 
-    private void appendRootTable(StringBuilder queryBuf) {
-        DbEntity ent = tableList.get(0);
-        queryBuf.append(ent.getFullyQualifiedName());
-
-        // The alias should be the alias from the same index in aliasList, not that
-        // returned by aliasForTable.
-        queryBuf.append(' ').append(aliasList.get(0));
-    }
-
-    private void appendJoins(StringBuilder queryBuf, int index) {
-
-        DbRelationship relationship = dbRelList.get(index);
-        JoinType joinType = dbRelationshipSemantics.get(index);
-
-        DbEntity targetEntity = (DbEntity) relationship.getTargetEntity();
-        String srcAlias = aliasForTable((DbEntity) relationship.getSourceEntity());
-        String targetAlias = aliasLookup.get(relationship);
-
-        switch (joinType) {
-            case INNER:
-                queryBuf.append(" JOIN");
-                break;
-            case LEFT_OUTER:
-                queryBuf.append(" LEFT JOIN");
-                break;
-            default:
-                throw new IllegalArgumentException("Unsupported join type: " + joinType);
-        }
-
-        queryBuf
-                .append(' ')
-                .append(targetEntity.getFullyQualifiedName())
-                .append(' ')
-                .append(targetAlias)
-                .append(" ON (");
-
-        List<DbJoin> joins = relationship.getJoins();
-        int len = joins.size();
-        for (int i = 0; i < len; i++) {
-            DbJoin join = joins.get(i);
-
-            if (i > 0) {
-                queryBuf.append(" AND ");
-            }
-
-            queryBuf.append(srcAlias).append('.').append(join.getSourceName()).append(
-                    " = ").append(targetAlias).append('.').append(join.getTargetName());
-        }
-
-        queryBuf.append(')');
-    }
-
     /**
-     * Stores a new relationship in an internal list. Later it will be used to create
-     * joins to relationship destination table.
-     * 
-     * @deprecated since 3.0 as super is deprecated.
+     * @since 3.0
      */
     @Override
-    public void dbRelationshipAdded(DbRelationship rel) {
-        dbRelationshipAdded(rel, JoinType.INNER);
+    public void resetJoinStack() {
+        joinStack.resetStack();
     }
 
     /**
@@ -652,61 +591,7 @@
             forcingDistinct = true;
         }
 
-        String existingAlias = aliasLookup.get(relationship);
-
-        if (existingAlias == null) {
-            dbRelList.add(relationship);
-            dbRelationshipSemantics.add(joinType);
-
-            // add alias for the destination table of the relationship
-            String newAlias = newAliasForTable((DbEntity) relationship.getTargetEntity());
-            aliasLookup.put(relationship, newAlias);
-        }
-    }
-
-    /**
-     * Sets up and returns a new alias for a specified table.
-     */
-    protected String newAliasForTable(DbEntity ent) {
-        String newAlias = "t" + aliasCounter++;
-        tableList.add(ent);
-        aliasList.add(newAlias);
-        return newAlias;
-    }
-
-    @Override
-    public String aliasForTable(DbEntity ent, DbRelationship rel) {
-        return aliasLookup.get(rel);
-    }
-
-    /**
-     * Overrides superclass implementation. Will return an alias that should be used for a
-     * specified DbEntity in the query (or null if this DbEntity is not included in the
-     * FROM clause).
-     */
-    @Override
-    public String aliasForTable(DbEntity ent) {
-
-        int entIndex = tableList.indexOf(ent);
-        if (entIndex >= 0) {
-            return aliasList.get(entIndex);
-        }
-        else {
-            StringBuilder msg = new StringBuilder();
-            msg.append("Alias not found, DbEntity: '").append(
-                    ent != null ? ent.getName() : "<null entity>").append(
-                    "'\nExisting aliases:");
-
-            int len = aliasList.size();
-            for (int i = 0; i < len; i++) {
-                String dbeName = (tableList.get(i) != null)
-                        ? tableList.get(i).getName()
-                        : "<null entity>";
-                msg.append("\n").append(aliasList.get(i)).append(" => ").append(dbeName);
-            }
-
-            throw new CayenneRuntimeException(msg.toString());
-        }
+        joinStack.pushJoin(relationship, joinType, null);
     }
 
     /**

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/TrimmingQualifierTranslator.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/TrimmingQualifierTranslator.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/TrimmingQualifierTranslator.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/TrimmingQualifierTranslator.java Tue May  6 06:31:03 2008
@@ -17,18 +17,15 @@
  *  under the License.
  ****************************************************************/
 
-
 package org.apache.cayenne.access.trans;
 
 import java.sql.Types;
 
 import org.apache.cayenne.map.DbAttribute;
-import org.apache.cayenne.map.DbRelationship;
 
-/** 
- * QualifierTranslator that allows translation of qualifiers that perform
- * comparison with CHAR columns. Some databases require trimming the values for
- * this to work.
+/**
+ * QualifierTranslator that allows translation of qualifiers that perform comparison with
+ * CHAR columns. Some databases require trimming the values for this to work.
  * 
  * @author Andrus Adamchik
  */
@@ -46,9 +43,7 @@
     /**
      * Constructor for TrimmingQualifierTranslator.
      */
-    public TrimmingQualifierTranslator(
-        QueryAssembler queryAssembler,
-        String trimFunction) {
+    public TrimmingQualifierTranslator(QueryAssembler queryAssembler, String trimFunction) {
         super(queryAssembler);
         this.trimFunction = trimFunction;
     }
@@ -62,30 +57,15 @@
             buf.append(trimFunction).append("(");
             super.processColumn(buf, dbAttr);
             buf.append(')');
-        } else {
-            super.processColumn(buf, dbAttr);
         }
-    }
-
-    /**
-     * Adds special handling of CHAR columns.
-     */
-    @Override
-    protected void processColumn(
-        StringBuffer buf,
-        DbAttribute dbAttr,
-        DbRelationship rel) {
-        if (dbAttr.getType() == Types.CHAR) {
-            buf.append(trimFunction).append("(");
-            super.processColumn(buf, dbAttr, rel);
-            buf.append(')');
-        } else {
-            super.processColumn(buf, dbAttr, rel);
+        else {
+            super.processColumn(buf, dbAttr);
         }
     }
 
     /**
      * Returns the trimFunction.
+     * 
      * @return String
      */
     public String getTrimFunction() {
@@ -94,6 +74,7 @@
 
     /**
      * Sets the trimFunction.
+     * 
      * @param trimFunction The trimFunction to set
      */
     public void setTrimFunction(String trimFunction) {

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/UpdateTranslator.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/UpdateTranslator.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/UpdateTranslator.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/main/java/org/apache/cayenne/access/trans/UpdateTranslator.java Tue May  6 06:31:03 2008
@@ -39,18 +39,18 @@
 public class UpdateTranslator extends QueryAssembler {
 
     @Override
-    public String aliasForTable(DbEntity dbEnt) {
-        throw new RuntimeException("aliases not supported");
+    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
+        throw new UnsupportedOperationException("db relationships not supported");
     }
-
+    
     @Override
-    public void dbRelationshipAdded(DbRelationship dbRel) {
-        throw new RuntimeException("db relationships not supported");
+    public String getCurrentAlias() {
+        throw new UnsupportedOperationException("aliases not supported");
     }
 
     @Override
-    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
-        throw new RuntimeException("db relationships not supported");
+    public void resetJoinStack() {
+        // noop - path processing is not supported.
     }
 
     /** Method that converts an update query into SQL string */

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/DataContextJoinAliasesTest.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/DataContextJoinAliasesTest.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/DataContextJoinAliasesTest.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/DataContextJoinAliasesTest.java Tue May  6 06:31:03 2008
@@ -20,7 +20,6 @@
 
 import java.util.Date;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.art.Artist;
@@ -61,10 +60,10 @@
         query.andQualifier(ExpressionFactory.matchExp("p.toArtist", picasso));
         query.andQualifier(ExpressionFactory.matchExp("d.toArtist", dali));
 
-//        List<Gallery> galleries = context.performQuery(query);
-//
-//        assertEquals(1, galleries.size());
-//        assertEquals("G1", galleries.get(0).getGalleryName());
+        // List<Gallery> galleries = context.performQuery(query);
+        //
+        // assertEquals(1, galleries.size());
+        // assertEquals("G1", galleries.get(0).getGalleryName());
     }
 
 }

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/SelectTranslatorTest.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/SelectTranslatorTest.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/SelectTranslatorTest.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/SelectTranslatorTest.java Tue May  6 06:31:03 2008
@@ -22,7 +22,6 @@
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.Date;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.art.Artist;
@@ -172,7 +171,7 @@
                 assertTrue(ind1 > 0);
 
                 int ind2 = generatedSql.indexOf("ARTIST t", ind1 + 1);
-                assertTrue(ind2 < 0);
+                assertTrue(generatedSql, ind2 < 0);
             }
         };
 
@@ -342,7 +341,7 @@
                 assertTrue(sql, sql.indexOf("PAINTING_ID") > 0);
 
                 // assert we have one join
-                assertEquals(1, transl.dbRelList.size());
+                assertEquals(1, transl.joinStack.size());
             }
         };
 
@@ -364,7 +363,7 @@
                 transl.createSqlString();
 
                 // assert we only have one join
-                assertEquals(1, transl.dbRelList.size());
+                assertEquals(1, transl.joinStack.size());
             }
         };
 
@@ -399,7 +398,7 @@
                 assertTrue(sql, sql.indexOf("PAINTING_ID") > 0);
 
                 // assert we have one join
-                assertEquals(1, transl.dbRelList.size());
+                assertEquals(1, transl.joinStack.size());
             }
         };
 

Modified: cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/TstQueryAssembler.java
URL: http://svn.apache.org/viewvc/cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/TstQueryAssembler.java?rev=653768&r1=653767&r2=653768&view=diff
==============================================================================
--- cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/TstQueryAssembler.java (original)
+++ cayenne/main/trunk/framework/cayenne-jdk1.5-unpublished/src/test/java/org/apache/cayenne/access/trans/TstQueryAssembler.java Tue May  6 06:31:03 2008
@@ -17,7 +17,6 @@
  *  under the License.
  ****************************************************************/
 
-
 package org.apache.cayenne.access.trans;
 
 import java.sql.SQLException;
@@ -26,7 +25,6 @@
 
 import org.apache.cayenne.CayenneRuntimeException;
 import org.apache.cayenne.access.DataNode;
-import org.apache.cayenne.map.DbEntity;
 import org.apache.cayenne.map.DbRelationship;
 import org.apache.cayenne.map.JoinType;
 import org.apache.cayenne.query.Query;
@@ -53,22 +51,19 @@
         super.getConnection().close();
     }
 
-    /**
-     * @deprecated since 3.0
-     */
     @Override
-    public void dbRelationshipAdded(DbRelationship dbRel) {
-        dbRels.add(dbRel);
+    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
+        dbRels.add(relationship);
     }
     
     @Override
-    public void dbRelationshipAdded(DbRelationship relationship, JoinType joinType) {
-        dbRels.add(relationship);
+    public String getCurrentAlias() {
+        return "ta";
     }
 
     @Override
-    public String aliasForTable(DbEntity dbEnt) {
-        return "ta";
+    public void resetJoinStack() {
+        // noop
     }
 
     @Override