@@ -48,6 +48,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
4848 Batch (" Resolution" , fixedPoint,
4949 ResolveReferences ::
5050 ResolveRelations ::
51+ ResolveSortReferences ::
5152 NewRelationInstances ::
5253 ImplicitGenerate ::
5354 StarExpansion ::
@@ -120,6 +121,51 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
120121 }
121122 }
122123
124+ /**
125+ * In many dialects of SQL is it valid to sort by attributes that are not present in the SELECT
126+ * clause. This rule detects such queries and adds the required attributes to the original
127+ * projection, so that they will be available during sorting. Another projection is added to
128+ * remove these attributes after sorting.
129+ */
130+ object ResolveSortReferences extends Rule [LogicalPlan ] {
131+ def apply (plan : LogicalPlan ): LogicalPlan = plan transformUp {
132+ case s@ Sort (ordering, p@ Project (projectList, child)) if ! s.resolved && p.resolved =>
133+ val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute (name) => name})
134+ val resolved = unresolved.flatMap(child.resolveChildren)
135+ val requiredAttributes = resolved.collect { case a : Attribute => a }.toSet
136+
137+ val missingInProject = requiredAttributes -- p.output
138+ if (missingInProject.nonEmpty) {
139+ // Add missing attributes and then project them away after the sort.
140+ Project (projectList,
141+ Sort (ordering,
142+ Project (projectList ++ missingInProject, child)))
143+ } else {
144+ s // Nothing we can do here. Return original plan.
145+ }
146+ case s@ Sort (ordering, a@ Aggregate (grouping, aggs, child)) if ! s.resolved && a.resolved =>
147+ val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute (name) => name})
148+ // A small hack to create an object that will allow us to resolve any references that
149+ // refer to named expressions that are present in the grouping expressions.
150+ val groupingRelation = LocalRelation (
151+ grouping.collect { case ne : NamedExpression => ne.toAttribute}
152+ )
153+
154+ logWarning(s " Grouping expressions: $groupingRelation" )
155+ val resolved = unresolved.flatMap(groupingRelation.resolve).toSet
156+ val missingInAggs = resolved -- a.outputSet
157+ logWarning(s " Resolved: $resolved Missing in aggs: $missingInAggs" )
158+ if (missingInAggs.nonEmpty) {
159+ // Add missing grouping exprs and then project them away after the sort.
160+ Project (a.output,
161+ Sort (ordering,
162+ Aggregate (grouping, aggs ++ missingInAggs, child)))
163+ } else {
164+ s // Nothing we can do here. Return original plan.
165+ }
166+ }
167+ }
168+
123169 /**
124170 * Replaces [[UnresolvedFunction ]]s with concrete [[catalyst.expressions.Expression Expressions ]].
125171 */
0 commit comments