Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
require -> validate
  • Loading branch information
hvanhovell committed Aug 16, 2016
commit d9b4bb1281ff0d4f956fa4861871e95f9c337dd6
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
// Build the insert clauses.
val inserts = ctx.multiInsertQueryBody.asScala.map {
body =>
require(body.querySpecification.fromClause == null,
validate(body.querySpecification.fromClause == null,
"Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
body)

Expand Down Expand Up @@ -596,7 +596,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
// function takes X PERCENT as the input and the range of X is [0, 100], we need to
// adjust the fraction.
val eps = RandomSampler.roundingEpsilon
require(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
validate(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
s"Sampling fraction ($fraction) must be on interval [0, 1]",
ctx)
Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
Expand Down Expand Up @@ -664,7 +664,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
// Get the backing expressions.
val expressions = ctx.expression.asScala.map { eCtx =>
val e = expression(eCtx)
require(e.foldable, "All expressions in an inline table must be constants.", eCtx)
validate(e.foldable, "All expressions in an inline table must be constants.", eCtx)
e
}

Expand All @@ -686,7 +686,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
val baseAttributes = structType.toAttributes.map(_.withNullability(true))
val attributes = if (ctx.identifierList != null) {
val aliases = visitIdentifierList(ctx.identifierList)
require(aliases.size == baseAttributes.size,
validate(aliases.size == baseAttributes.size,
"Number of aliases must match the number of fields in an inline table.", ctx)
baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
} else {
Expand Down Expand Up @@ -1094,7 +1094,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
// We currently only allow foldable integers.
def value: Int = {
val e = expression(ctx.expression)
require(e.resolved && e.foldable && e.dataType == IntegerType,
validate(e.resolved && e.foldable && e.dataType == IntegerType,
"Frame bound value must be a constant integer.",
ctx)
e.eval().asInstanceOf[Int]
Expand Down Expand Up @@ -1347,7 +1347,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
*/
override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) {
val intervals = ctx.intervalField.asScala.map(visitIntervalField)
require(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx)
validate(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx)
Literal(intervals.reduce(_.add(_)))
}

Expand All @@ -1374,7 +1374,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
case (from, Some(t)) =>
throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx)
}
require(interval != null, "No interval can be constructed", ctx)
validate(interval != null, "No interval can be constructed", ctx)
interval
} catch {
// Handle Exceptions thrown by CalendarInterval
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,8 +77,8 @@ object ParserUtils {
Origin(Option(token.getLine), Option(token.getCharPositionInLine))
}

/** Require that a condition holds. If it doesn't throw a parse exception. */
def require(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = {
/** Validate the condition. If it doesn't throw a parse exception. */
def validate(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = {
if (!f) {
throw new ParseException(message, ctx)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1181,7 +1181,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
entry("mapkey.delim", ctx.keysTerminatedBy) ++
Option(ctx.linesSeparatedBy).toSeq.map { token =>
val value = string(token)
require(
validate(
value == "\n",
s"LINES TERMINATED BY only supports newline '\\n' right now: $value",
ctx)
Expand Down