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 @@ -1655,27 +1655,17 @@ class Analyzer(
}
}.toSeq

// Third, for every Window Spec, we add a Window operator and set currentChild as the
// child of it.
var currentChild = child
var i = 0
while (i < groupedWindowExpressions.size) {
val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i)
// Set currentChild to the newly created Window operator.
currentChild =
Window(
windowExpressions,
partitionSpec,
orderSpec,
currentChild)

// Move to next Window Spec.
i += 1
}
// Third, we aggregate them by adding each Window operator for each Window Spec and then
// setting this to the child of the next Window operator.
val windowOps =
groupedWindowExpressions.foldLeft(child) {
case (last, ((partitionSpec, orderSpec), windowExpressions)) =>
Window(windowExpressions, partitionSpec, orderSpec, last)
}

// Finally, we create a Project to output currentChild's output
// Finally, we create a Project to output windowOps's output
// newExpressionsWithWindowFunctions.
Project(currentChild.output ++ newExpressionsWithWindowFunctions, currentChild)
Project(windowOps.output ++ newExpressionsWithWindowFunctions, windowOps)
} // end of addWindow

// We have to use transformDown at here to make sure the rule of
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,8 @@ abstract class CaseWhenBase(

override def eval(input: InternalRow): Any = {
var i = 0
while (i < branches.size) {
val size = branches.size
while (i < size) {
if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) {
return branches(i)._2.eval(input)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow

def compare(a: InternalRow, b: InternalRow): Int = {
var i = 0
while (i < ordering.size) {
val size = ordering.size
while (i < size) {
val order = ordering(i)
val left = order.child.eval(a)
val right = order.child.eval(b)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,18 +91,18 @@ class QuantileSummaries(
var sampleIdx = 0
// The index of the sample currently being inserted.
var opsIdx: Int = 0
while(opsIdx < sorted.length) {
while (opsIdx < sorted.length) {
val currentSample = sorted(opsIdx)
// Add all the samples before the next observation.
while(sampleIdx < sampled.size && sampled(sampleIdx).value <= currentSample) {
while (sampleIdx < sampled.length && sampled(sampleIdx).value <= currentSample) {
newSamples += sampled(sampleIdx)
sampleIdx += 1
}

// If it is the first one to insert, of if it is the last one
currentCount += 1
val delta =
if (newSamples.isEmpty || (sampleIdx == sampled.size && opsIdx == sorted.length - 1)) {
if (newSamples.isEmpty || (sampleIdx == sampled.length && opsIdx == sorted.length - 1)) {
0
} else {
math.floor(2 * relativeError * currentCount).toInt
Expand All @@ -114,7 +114,7 @@ class QuantileSummaries(
}

// Add all the remaining existing samples
while(sampleIdx < sampled.size) {
while (sampleIdx < sampled.length) {
newSamples += sampled(sampleIdx)
sampleIdx += 1
}
Expand Down Expand Up @@ -195,7 +195,7 @@ class QuantileSummaries(
// Minimum rank at current sample
var minRank = 0
var i = 1
while (i < sampled.size - 1) {
while (i < sampled.length - 1) {
val curSample = sampled(i)
minRank += curSample.g
val maxRank = minRank + curSample.delta
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -369,7 +369,7 @@ object JdbcUtils extends Logging {
val bytes = rs.getBytes(pos + 1)
var ans = 0L
var j = 0
while (j < bytes.size) {
while (j < bytes.length) {
ans = 256 * ans + (255 & bytes(j))
j = j + 1
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -703,7 +703,8 @@ private[hive] trait HiveInspectors {
// 1. create the pojo (most likely) object
val result = x.create()
var i = 0
while (i < fieldRefs.size) {
val size = fieldRefs.size
while (i < size) {
// 2. set the property for the pojo
val tpe = structType(i).dataType
x.setStructFieldData(
Expand All @@ -720,7 +721,8 @@ private[hive] trait HiveInspectors {
val row = a.asInstanceOf[InternalRow]
val result = new java.util.ArrayList[AnyRef](fieldRefs.size)
var i = 0
while (i < fieldRefs.size) {
val size = fieldRefs.size
while (i < size) {
val tpe = structType(i).dataType
result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe))
i += 1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,8 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
iterator.map { value =>
val raw = converter.convert(rawDeser.deserialize(value))
var i = 0
while (i < fieldRefs.length) {
val length = fieldRefs.length
while (i < length) {
val fieldValue = soi.getStructFieldData(raw, fieldRefs(i))
if (fieldValue == null) {
mutableRow.setNullAt(fieldOrdinals(i))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,8 @@ private[hive] case class HiveGenericUDF(
returnInspector // Make sure initialized.

var i = 0
while (i < children.length) {
val length = children.length
while (i < length) {
val idx = i
deferredObjects(i).asInstanceOf[DeferredObjectAdapter]
.set(() => children(idx).eval(input))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration)
row: InternalRow): Unit = {
val fieldRefs = oi.getAllStructFieldRefs
var i = 0
while (i < fieldRefs.size) {
val size = fieldRefs.size
while (i < size) {

oi.setStructFieldData(
struct,
Expand Down Expand Up @@ -289,7 +290,8 @@ private[orc] object OrcRelation extends HiveInspectors {
iterator.map { value =>
val raw = deserializer.deserialize(value)
var i = 0
while (i < fieldRefs.length) {
val length = fieldRefs.length
while (i < length) {
val fieldValue = oi.getStructFieldData(raw, fieldRefs(i))
if (fieldValue == null) {
mutableRow.setNullAt(fieldOrdinals(i))
Expand Down