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 @@ -215,19 +215,19 @@ case class EndsWith(left: Expression, right: Expression)
case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression {

type EvaluatedType = Any
def nullable: Boolean = true

def nullable: Boolean = str.nullable || pos.nullable || len.nullable
def dataType: DataType = {
if (!resolved) {
throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved")
}
if (str.dataType == BinaryType) str.dataType else StringType
}

def references = children.flatMap(_.references).toSet

override def children = str :: pos :: len :: Nil

@inline
def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int)
(implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = {
Expand All @@ -237,40 +237,40 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends
// refers to element i-1 in the sequence. If a start index i is less than 0, it refers
// to the -ith element before the end of the sequence. If a start index i is 0, it
// refers to the first element.

val start = startPos match {
case pos if pos > 0 => pos - 1
case neg if neg < 0 => len + neg
case _ => 0
}

val end = sliceLen match {
case max if max == Integer.MAX_VALUE => max
case x => start + x
}

str.slice(start, end)
}

override def eval(input: Row): Any = {
val string = str.eval(input)

val po = pos.eval(input)
val ln = len.eval(input)

if ((string == null) || (po == null) || (ln == null)) {
null
} else {
val start = po.asInstanceOf[Int]
val length = ln.asInstanceOf[Int]

string match {
case ba: Array[Byte] => slice(ba, start, length)
case other => slice(other.toString, start, length)
}
}
}

override def toString = len match {
case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)"
case _ => s"SUBSTR($str, $pos, $len)"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -469,9 +469,9 @@ class ExpressionEvaluationSuite extends FunSuite {

test("Substring") {
val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte)))

val s = 'a.string.at(0)

// substring from zero position with less-than-full length
checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row)
checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row)
Expand Down Expand Up @@ -501,7 +501,7 @@ class ExpressionEvaluationSuite extends FunSuite {

// substring(null, _, _) -> null
checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null)))

// substring(_, null, _) -> null
checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row)

Expand All @@ -514,6 +514,12 @@ class ExpressionEvaluationSuite extends FunSuite {

// 2-arg substring from nonzero position
checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row)

val s_notNull = 'a.string.notNull.at(0)

assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true)
assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false)
assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true)
assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true)
}
}