Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,12 @@ class AnalysisException protected[sql] (
val startPosition: Option[Int] = None)
extends Exception with Serializable {

def withPosition(line: Option[Int], startPosition: Option[Int]) = {
val newException = new AnalysisException(message, line, startPosition)
newException.setStackTrace(getStackTrace)
newException
}

override def getMessage: String = {
val lineAnnotation = line.map(l => s" line $l").getOrElse("")
val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,7 @@ class Analyzer(catalog: Catalog,
catalog.lookupRelation(u.tableIdentifier, u.alias)
} catch {
case _: NoSuchTableException =>
u.failAnalysis(s"no such table ${u.tableIdentifier}")
u.failAnalysis(s"no such table ${u.tableName}")
}
}

Expand Down Expand Up @@ -275,7 +275,8 @@ class Analyzer(catalog: Catalog,
q.asInstanceOf[GroupingAnalytics].gid
case u @ UnresolvedAttribute(name) =>
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
val result = q.resolveChildren(name, resolver).getOrElse(u)
val result =
withPosition(u) { q.resolveChildren(name, resolver).getOrElse(u) }
logDebug(s"Resolving $u to $result")
result
case UnresolvedGetField(child, fieldName) if child.resolved =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,4 +42,12 @@ package object analysis {
throw new AnalysisException(msg, t.origin.line, t.origin.startPosition)
}
}

/** Catches any AnalysisExceptions thrown by `f` and attaches `t`'s position if any. */
def withPosition[A](t: TreeNode[_])(f: => A) = {
try f catch {
case a: AnalysisException =>
throw a.withPosition(t.origin.line, t.origin.startPosition)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,10 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str
case class UnresolvedRelation(
tableIdentifier: Seq[String],
alias: Option[String] = None) extends LeafNode {

/** Returns a `.` separated name for this relation. */
def tableName = tableIdentifier.mkString(".")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a TODO at here about using backticks to quote any part of the identifier having . in the string?


override def output = Nil
override lazy val resolved = false
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,13 @@ abstract class NamedExpression extends Expression {
def name: String
def exprId: ExprId

/**
* Returns a dot separated fully qualified name for this attribute. Given that there can be
* multiple qualifiers, it is possible that there are other possible way to refer to this
* attribute.
*/
def qualifiedName: String = (qualifiers.headOption.toSeq :+ name).mkString(".")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, add TODO at here?


/**
* All possible qualifiers for the expression.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -208,8 +208,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {

// More than one match.
case ambiguousReferences =>
val referenceNames = ambiguousReferences.map(_._1.qualifiedName).mkString(", ")
throw new AnalysisException(
s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}")
s"Reference '$name' is ambiguous, could be: $referenceNames.")
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,29 @@ package org.apache.spark.sql.hive

import java.io.{OutputStream, PrintStream}

import scala.util.Try

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.{AnalysisException, QueryTest}

import scala.util.Try

class ErrorPositionSuite extends QueryTest {
class ErrorPositionSuite extends QueryTest with BeforeAndAfter {

before {
Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes")
}

positionTest("ambiguous attribute reference 1",
"SELECT a from dupAttributes", "a")

positionTest("ambiguous attribute reference 2",
"SELECT a, b from dupAttributes", "a")

positionTest("ambiguous attribute reference 3",
"SELECT b, a from dupAttributes", "a")

positionTest("unresolved attribute 1",
"SELECT x FROM src", "x")
Expand Down Expand Up @@ -127,6 +144,10 @@ class ErrorPositionSuite extends QueryTest {
val error = intercept[AnalysisException] {
quietly(sql(query))
}

assert(!error.getMessage.contains("Seq("))
assert(!error.getMessage.contains("List("))

val (line, expectedLineNum) = query.split("\n").zipWithIndex.collect {
case (l, i) if l.contains(token) => (l, i + 1)
}.headOption.getOrElse(sys.error(s"Invalid test. Token $token not in $query"))
Expand Down