Skip to content

Commit

Permalink
fix more comments
Browse files Browse the repository at this point in the history
  • Loading branch information
wzhfy committed Dec 19, 2017
1 parent e1669ed commit 16797d2
Show file tree
Hide file tree
Showing 3 changed files with 53 additions and 76 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -236,28 +236,8 @@ object EstimationUtils {
// Only collect overlapped ranges.
if (left.lo <= right.hi && left.hi >= right.lo) {
// Collect overlapped ranges.
val range = if (left.lo == left.hi) {
// Case1: the left bin has only one value
OverlappedRange(
lo = left.lo,
hi = left.lo,
leftNdv = 1,
rightNdv = 1,
leftNumRows = leftHeight,
rightNumRows = rightHeight / right.ndv
)
} else if (right.lo == right.hi) {
// Case2: the right bin has only one value
OverlappedRange(
lo = right.lo,
hi = right.lo,
leftNdv = 1,
rightNdv = 1,
leftNumRows = leftHeight / left.ndv,
rightNumRows = rightHeight
)
} else if (right.lo >= left.lo && right.hi >= left.hi) {
// Case3: the left bin is "smaller" than the right bin
val range = if (right.lo >= left.lo && right.hi >= left.hi) {
// Case1: the left bin is "smaller" than the right bin
// left.lo right.lo left.hi right.hi
// --------+------------------+------------+----------------+------->
if (left.hi == right.lo) {
Expand All @@ -283,7 +263,7 @@ object EstimationUtils {
)
}
} else if (right.lo <= left.lo && right.hi <= left.hi) {
// Case4: the left bin is "larger" than the right bin
// Case2: the left bin is "larger" than the right bin
// right.lo left.lo right.hi left.hi
// --------+------------------+------------+----------------+------->
if (right.hi == left.lo) {
Expand All @@ -309,7 +289,7 @@ object EstimationUtils {
)
}
} else if (right.lo >= left.lo && right.hi <= left.hi) {
// Case5: the left bin contains the right bin
// Case3: the left bin contains the right bin
// left.lo right.lo right.hi left.hi
// --------+------------------+------------+----------------+------->
val leftRatio = (right.hi - right.lo) / (left.hi - left.lo)
Expand All @@ -323,7 +303,7 @@ object EstimationUtils {
)
} else {
assert(right.lo <= left.lo && right.hi >= left.hi)
// Case6: the right bin contains the left bin
// Case4: the right bin contains the left bin
// right.lo left.lo left.hi right.hi
// --------+------------------+------------+----------------+------->
val rightRatio = (left.hi - left.lo) / (right.hi - right.lo)
Expand All @@ -346,6 +326,11 @@ object EstimationUtils {
/**
* Given an original bin and a value range [lowerBound, upperBound], returns the trimmed part
* of the bin in that range and its number of rows.
* @param bin the input histogram bin.
* @param height the number of rows of the given histogram bin inside an equi-height histogram.
* @param lowerBound lower bound of the given range.
* @param upperBound upper bound of the given range.
* @return trimmed part of the given bin and its number of rows.
*/
def trimBin(bin: HistogramBin, height: Double, lowerBound: Double, upperBound: Double)
: (HistogramBin, Double) = {
Expand All @@ -364,14 +349,15 @@ object EstimationUtils {
} else {
// lowerBound bin.lo bin.hi upperBound
// --------+------------------+------------+-------------+------->
assert(bin.lo >= lowerBound && bin.hi <= upperBound)
(bin.lo, bin.hi)
}

if (bin.hi == bin.lo) {
(bin, height)
} else if (hi == lo) {
if (hi == lo) {
// Note that bin.hi == bin.lo also falls into this branch.
(HistogramBin(lo, hi, 1), height / bin.ndv)
} else {
assert(bin.hi != bin.lo)
val ratio = (hi - lo) / (bin.hi - bin.lo)
(HistogramBin(lo, hi, math.ceil(bin.ndv * ratio).toLong), height * ratio)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ case class JoinEstimation(join: Join) extends Logging {
val (newMin, newMax) = ValueInterval.intersect(lInterval, rInterval, leftKey.dataType)
val (card, joinStat) = (leftKeyStat.histogram, rightKeyStat.histogram) match {
case (Some(l: Histogram), Some(r: Histogram)) =>
computeByEquiHeightHistogram(leftKey, rightKey, l, r, newMin, newMax)
computeByHistogram(leftKey, rightKey, l, r, newMin, newMax)
case _ =>
computeByNdv(leftKey, rightKey, newMin, newMax)
}
Expand Down Expand Up @@ -237,7 +237,7 @@ case class JoinEstimation(join: Join) extends Logging {
}

/** Compute join cardinality using equi-height histograms. */
private def computeByEquiHeightHistogram(
private def computeByHistogram(
leftKey: AttributeReference,
rightKey: AttributeReference,
leftHistogram: Histogram,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,24 +136,22 @@ class JoinEstimationSuite extends StatsEstimationTestBase {

val expectedRanges = Seq(
// histogram1.bins(0) overlaps t0
OverlappedRange(10, 30, 10, 40*1/2, 300, 80*1/2),
OverlappedRange(10, 30, 10, 40 * 1 / 2, 300, 80 * 1 / 2),
// histogram1.bins(1) overlaps t0
OverlappedRange(30, 50, 30*2/3, 40*1/2, 300*2/3, 80*1/2),
OverlappedRange(30, 50, 30 * 2 / 3, 40 * 1 / 2, 300 * 2 / 3, 80 * 1 / 2),
// histogram1.bins(1) overlaps t1
OverlappedRange(50, 60, 30*1/3, 8, 300*1/3, 20)
OverlappedRange(50, 60, 30 * 1 / 3, 8, 300 * 1 / 3, 20)
)
assert(expectedRanges.equals(
getOverlappedRanges(histogram1, histogram2, lowerBound = 10D, upperBound = 60D)))
getOverlappedRanges(histogram1, histogram2, lowerBound = 10, upperBound = 60)))

estimateByHistogram(
leftHistogram = histogram1,
rightHistogram = histogram2,
expectedMin = 10D,
expectedMax = 60D,
// 10 + 20 + 8
expectedNdv = 38L,
// 300*40/20 + 200*40/20 + 100*20/10
expectedRows = 1200L)
expectedMin = 10,
expectedMax = 60,
expectedNdv = 10 + 20 + 8,
expectedRows = 300 * 40 / 20 + 200 * 40 / 20 + 100 * 20 / 10)
}

test("equi-height histograms: a bin has only one value after trimming") {
Expand All @@ -169,24 +167,22 @@ class JoinEstimationSuite extends StatsEstimationTestBase {

val expectedRanges = Seq(
// histogram1.bins(0) overlaps t0
OverlappedRange(50, 50, 1, 1, 300/10, 2),
OverlappedRange(50, 50, 1, 1, 300 / 10, 2),
// histogram1.bins(0) overlaps t1
OverlappedRange(50, 60, 10, 20*10/25, 300, 50*10/25),
OverlappedRange(50, 60, 10, 20 * 10 / 25, 300, 50 * 10 / 25),
// histogram1.bins(1) overlaps t1
OverlappedRange(60, 75, 3, 20*15/25, 300, 50*15/25)
OverlappedRange(60, 75, 3, 20 * 15 / 25, 300, 50 * 15 / 25)
)
assert(expectedRanges.equals(
getOverlappedRanges(histogram1, histogram2, lowerBound = 50D, upperBound = 75D)))
getOverlappedRanges(histogram1, histogram2, lowerBound = 50, upperBound = 75)))

estimateByHistogram(
leftHistogram = histogram1,
rightHistogram = histogram2,
expectedMin = 50D,
expectedMax = 75D,
// 1 + 8 + 3
expectedNdv = 12L,
// 30*2/1 + 300*20/10 + 300*30/12
expectedRows = 1410L)
expectedMin = 50,
expectedMax = 75,
expectedNdv = 1 + 8 + 3,
expectedRows = 30 * 2 / 1 + 300 * 20 / 10 + 300 * 30 / 12)
}

test("equi-height histograms: skew distribution (some bins have only one value)") {
Expand All @@ -203,23 +199,21 @@ class JoinEstimationSuite extends StatsEstimationTestBase {
assert(t1 ==HistogramBin(lo = 50, hi = 60, ndv = 8) && h1 == 20)

val expectedRanges = Seq(
OverlappedRange(30, 30, 1, 1, 300, 40/20),
OverlappedRange(30, 30, 1, 1, 300, 40/20),
OverlappedRange(30, 50, 30*2/3, 20, 300*2/3, 40),
OverlappedRange(50, 60, 30*1/3, 8, 300*1/3, 20)
OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
OverlappedRange(30, 50, 30 * 2 / 3, 20, 300 * 2 / 3, 40),
OverlappedRange(50, 60, 30 * 1 / 3, 8, 300 * 1 / 3, 20)
)
assert(expectedRanges.equals(
getOverlappedRanges(histogram1, histogram2, lowerBound = 30D, upperBound = 60D)))
getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 60)))

estimateByHistogram(
leftHistogram = histogram1,
rightHistogram = histogram2,
expectedMin = 30D,
expectedMax = 60D,
// 1 + 20 + 8
expectedNdv = 29L,
// 300*2/1 + 300*2/1 + 200*40/20 + 100*20/10
expectedRows = 1800L)
expectedMin = 30,
expectedMax = 60,
expectedNdv = 1 + 20 + 8,
expectedRows = 300 * 2 / 1 + 300 * 2 / 1 + 200 * 40 / 20 + 100 * 20 / 10)
}

test("equi-height histograms: skew distribution (histograms have different skewed values") {
Expand All @@ -234,22 +228,20 @@ class JoinEstimationSuite extends StatsEstimationTestBase {
assert(t1 == HistogramBin(lo = 30, hi = 50, ndv = 20) && h1 == 40)

val expectedRanges = Seq(
OverlappedRange(30, 30, 1, 1, 300, 40/20),
OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
OverlappedRange(30, 50, 20, 20, 200, 40),
OverlappedRange(50, 50, 1, 1, 200/20, 100)
OverlappedRange(50, 50, 1, 1, 200 / 20, 100)
)
assert(expectedRanges.equals(
getOverlappedRanges(histogram1, histogram2, lowerBound = 30D, upperBound = 50D)))
getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 50)))

estimateByHistogram(
leftHistogram = histogram1,
rightHistogram = histogram2,
expectedMin = 30D,
expectedMax = 50D,
// 1 + 20
expectedNdv = 21L,
// 300*2/1 + 200*40/20 + 10*100/1
expectedRows = 2000L)
expectedMin = 30,
expectedMax = 50,
expectedNdv = 1 + 20,
expectedRows = 300 * 2 / 1 + 200 * 40 / 20 + 10 * 100 / 1)
}

test("equi-height histograms: skew distribution (both histograms have the same skewed value") {
Expand All @@ -270,17 +262,16 @@ class JoinEstimationSuite extends StatsEstimationTestBase {
OverlappedRange(30, 30, 1, 1, 10, 150)
)
assert(expectedRanges.equals(
getOverlappedRanges(histogram1, histogram2, lowerBound = 30D, upperBound = 30D)))
getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 30)))

estimateByHistogram(
leftHistogram = histogram1,
rightHistogram = histogram2,
expectedMin = 30D,
expectedMax = 30D,
expectedMin = 30,
expectedMax = 30,
// only one value: 30
expectedNdv = 1L,
// 300*5/1 + 300*150/1 + 10*5/1 + 10*150/1
expectedRows = 48050L)
expectedNdv = 1,
expectedRows = 300 * 5 / 1 + 300 * 150 / 1 + 10 * 5 / 1 + 10 * 150 / 1)
}

test("cross join") {
Expand Down

0 comments on commit 16797d2

Please sign in to comment.