You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2018/11/26 21:31:15 UTC

[2/4] kudu git commit: [compaction] KUDU-1400: Improve rowset compaction policy to consider merging small DRSs

[compaction] KUDU-1400: Improve rowset compaction policy to consider merging small DRSs

This implements the small rowset compaction scheme explained in the
KUDU-1400 design doc [1]. While the implementation is simple, the
reasoning behind it is nuanced, and I'll defer the explanations of them
to the design doc rather than repeating them here.

The three relevant constant factors:

--compaction_minimum_improvement_score
--compaction_small_rowset_tradeoff
kSupportAdjust

have been tuned to work well with each other. See the relevant tests for
reasoning on why, and validation of what the results should be.

Since compaction policy's performance is important, I ran the YCSB
benchmark before and after the change using 'perf stat' to compare the
performance:

Command:

Before:

 Performance counter stats for 'bin/compaction_policy-test --gtest_filter=*Ycsb*' (10 runs):

       1231.095442 task-clock                #    0.997 CPUs utilized            ( +-  2.00% )
               148 context-switches          #    0.120 K/sec                    ( +-  2.43% )
                 9 cpu-migrations            #    0.007 K/sec                    ( +- 16.14% )
             3,630 page-faults               #    0.003 M/sec                    ( +-  0.00% )
     3,251,530,478 cycles                    #    2.641 GHz                      ( +-  2.04% )
   <not supported> stalled-cycles-frontend
   <not supported> stalled-cycles-backend
     5,772,319,429 instructions              #    1.78  insns per cycle          ( +-  0.01% )
     1,070,627,520 branches                  #  869.654 M/sec                    ( +-  0.01% )
        13,583,368 branch-misses             #    1.27% of all branches          ( +-  0.10% )

       1.235037947 seconds time elapsed                                          ( +-  2.00% )

After:

Performance counter stats for 'bin/compaction_policy-test --gtest_filter=*Ycsb*' (10 runs):

       1297.749333 task-clock                #    0.994 CPUs utilized            ( +-  2.17% )
               158 context-switches          #    0.122 K/sec                    ( +-  3.01% )
                14 cpu-migrations            #    0.011 K/sec                    ( +- 17.48% )
             3,636 page-faults               #    0.003 M/sec
     3,509,480,140 cycles                    #    2.704 GHz                      ( +-  2.65% )
   <not supported> stalled-cycles-frontend
   <not supported> stalled-cycles-backend
     6,800,316,126 instructions              #    1.94  insns per cycle          ( +-  0.01% )
     1,073,111,416 branches                  #  826.902 M/sec                    ( +-  0.01% )
        13,574,780 branch-misses             #    1.26% of all branches          ( +-  0.15% )

       1.305058206 seconds time elapsed                                          ( +-  2.16% )

A follow up will integrate the below design doc with the existing one,
docs/design-docs/compaction-policy.md.

[1]: https://docs.google.com/document/d/1yTfxt0_2p5EfIjCnjJCt3o-nB9xk-Kl2O8yKTA1LQrQ/edit?usp=sharing

Change-Id: I7b421c6ed77d28ebab9b91a4d6fcb1e825997e6c
Reviewed-on: http://gerrit.cloudera.org:8080/11869
Tested-by: Kudu Jenkins
Reviewed-by: Andrew Wong <aw...@cloudera.com>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/f65211ba
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/f65211ba
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/f65211ba

Branch: refs/heads/master
Commit: f65211ba415d43dd6615bc28cfe2576bc0a43489
Parents: b37f264
Author: Will Berkeley <wd...@gmail.org>
Authored: Tue Oct 30 15:11:41 2018 -0700
Committer: Will Berkeley <wd...@gmail.com>
Committed: Mon Nov 26 16:52:58 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/compaction_policy-test.cc | 187 +++++++++++++++++++++++--
 src/kudu/tablet/compaction_policy.cc      |  67 +++++----
 src/kudu/tablet/rowset_info.cc            |  66 ++++++++-
 src/kudu/tablet/rowset_info.h             |  22 ++-
 4 files changed, 297 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f65211ba/src/kudu/tablet/compaction_policy-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/compaction_policy-test.cc b/src/kudu/tablet/compaction_policy-test.cc
index 97714f3..ce99b5b 100644
--- a/src/kudu/tablet/compaction_policy-test.cc
+++ b/src/kudu/tablet/compaction_policy-test.cc
@@ -18,12 +18,14 @@
 #include "kudu/tablet/compaction_policy.h"
 
 #include <algorithm>
+#include <limits>
 #include <memory>
 #include <ostream>
 #include <string>
 #include <utility>
 #include <vector>
 
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <glog/stl_logging.h>
 #include <gtest/gtest.h>
@@ -47,6 +49,10 @@
 using std::string;
 using std::vector;
 
+DECLARE_double(compaction_minimum_improvement);
+DECLARE_double(compaction_small_rowset_tradeoff);
+DECLARE_int64(budgeted_compaction_target_rowset_size);
+
 namespace kudu {
 namespace tablet {
 
@@ -65,10 +71,13 @@ class TestCompactionPolicy : public KuduTest {
   }
 };
 
-
 // Simple test for budgeted compaction: with three rowsets which
 // mostly overlap, and a high budget, they should all be selected.
 TEST_F(TestCompactionPolicy, TestBudgetedSelection) {
+  // This tests the overlap-based part of compaction policy and not the
+  // rowset-size based policy.
+  FLAGS_compaction_small_rowset_tradeoff = 0.0;
+
   /*
    *   [C ------ c]
    *  [B ----- a]
@@ -96,6 +105,10 @@ TEST_F(TestCompactionPolicy, TestBudgetedSelection) {
 // overlap at all. This is likely to occur in workloads where the
 // primary key is always increasing (such as a timestamp).
 TEST_F(TestCompactionPolicy, TestNonOverlappingRowSets) {
+  // This tests the overlap-based part of compaction policy and not the
+  // rowset-size based policy.
+  FLAGS_compaction_small_rowset_tradeoff = 0.0;
+
   /* NB: Zero-padding of string keys omitted to save space.
    *
    * [0 - 1] [2 - 3] ... [198 - 199]
@@ -111,13 +124,17 @@ TEST_F(TestCompactionPolicy, TestNonOverlappingRowSets) {
   CompactionSelection picked;
   double quality = 0.0;
   NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
-  ASSERT_EQ(quality, 0.0);
+  ASSERT_EQ(0.0, quality);
   ASSERT_TRUE(picked.empty());
 }
 
 // Similar to the above, but with some small overlap between adjacent
 // rowsets.
 TEST_F(TestCompactionPolicy, TestTinyOverlapRowSets) {
+  // This tests the overlap-based part of compaction policy and not the
+  // rowset-size based policy.
+  FLAGS_compaction_small_rowset_tradeoff = 0.0;
+
   /* NB: Zero-padding of string keys omitted to save space.
    *
    * [0 - 11000]
@@ -137,13 +154,17 @@ TEST_F(TestCompactionPolicy, TestTinyOverlapRowSets) {
   double quality = 0;
   NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
   // With such small overlaps, no compaction will be considered worthwhile.
-  ASSERT_EQ(quality, 0.0);
+  ASSERT_EQ(0.0, quality);
   ASSERT_TRUE(picked.empty());
 }
 
 // Test case with 100 rowsets, each of which overlaps with its two
 // neighbors to the right.
 TEST_F(TestCompactionPolicy, TestSignificantOverlap) {
+  // This tests the overlap-based part of compaction policy and not the
+  // rowset-size based policy.
+  FLAGS_compaction_small_rowset_tradeoff = 0.0;
+
   /* NB: Zero-padding of string keys omitted to save space.
    *
    * [0 ------ 20000]
@@ -165,7 +186,7 @@ TEST_F(TestCompactionPolicy, TestSignificantOverlap) {
   NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
   // Each rowset is 1MB so the number of rowsets picked should be the number of
   // MB in the budget.
-  ASSERT_EQ(picked.size(), kBudgetMb);
+  ASSERT_EQ(kBudgetMb, picked.size());
 
   // In picking 64 of 100 equally-sized and -spaced rowsets, the union width is
   // about 0.64. Each rowset intersects the next in half its width, which makes
@@ -183,23 +204,39 @@ TEST_F(TestCompactionPolicy, TestSignificantOverlap) {
 //  [ -- B -- ]
 //            [ -- C -- ]
 //
-// compacting {A, B, C} results in the same quality score as {A, B}, 0.67, but
-// uses more budget. By penalizing the wider solution {A, B, C}, we ensure we
-// don't waste I/O.
+// compacting {A, B, C} results in the same reduction in average tablet height
+// as compacting {A, B}, but uses more I/O. We'd like to choose {A, B} if
+// A, B, and C are large, but {A, B, C} if A, B, and C are small.
 TEST_F(TestCompactionPolicy, TestSupportAdjust) {
-  const RowSetVector rowsets = {
-    std::make_shared<MockDiskRowSet>("A", "B"),
-    std::make_shared<MockDiskRowSet>("A", "B"),
-    std::make_shared<MockDiskRowSet>("B", "C"),
-  };
   constexpr auto kBudgetMb = 1000; // Enough to select all rowsets.
   CompactionSelection picked;
   double quality = 0.0;
-  NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
+
+  // For big rowsets, compaction should favor just compacting the two that
+  // overlap.
+  const auto big_rowset_size = FLAGS_budgeted_compaction_target_rowset_size * 0.9;
+  const RowSetVector big_rowsets = {
+    std::make_shared<MockDiskRowSet>("A", "B", big_rowset_size),
+    std::make_shared<MockDiskRowSet>("A", "B", big_rowset_size),
+    std::make_shared<MockDiskRowSet>("B", "C", big_rowset_size),
+  };
+  NO_FATALS(RunTestCase(big_rowsets, kBudgetMb, &picked, &quality));
   ASSERT_EQ(2, picked.size());
-  // The weights of A and B are both 0.67, so with the adjustment the quality
+  // The widths of A and B are both 0.67, so with the adjustment the quality
   // score should be a little less than 0.67.
   ASSERT_GE(quality, 0.6);
+
+  // For small rowsets, compaction should favor compacting all three.
+  const auto small_rowset_size = FLAGS_budgeted_compaction_target_rowset_size * 0.3;
+  const RowSetVector small_rowsets = {
+    std::make_shared<MockDiskRowSet>("A", "B", small_rowset_size),
+    std::make_shared<MockDiskRowSet>("A", "B", small_rowset_size),
+    std::make_shared<MockDiskRowSet>("B", "C", small_rowset_size),
+  };
+  picked.clear();
+  NO_FATALS(RunTestCase(small_rowsets, kBudgetMb, &picked, &quality));
+  ASSERT_EQ(3, picked.size());
+  ASSERT_GE(quality, 0.6);
 }
 
 static RowSetVector LoadFile(const string& name) {
@@ -261,6 +298,10 @@ TEST_F(TestCompactionPolicy, TestYcsbCompaction) {
 // Regression test for KUDU-2251 which ensures that large (> 2GiB) rowsets don't
 // cause integer overflow in compaction planning.
 TEST_F(TestCompactionPolicy, KUDU2251) {
+  // Raise the target size to keep the quality score in the same range as for
+  // "normal size" rowsets.
+  FLAGS_budgeted_compaction_target_rowset_size = 1L << 34;
+
   // Same arrangement as in TestBudgetedSelection.
   const RowSetVector rowsets = {
     std::make_shared<MockDiskRowSet>("C", "c", 1L << 31),
@@ -272,11 +313,127 @@ TEST_F(TestCompactionPolicy, KUDU2251) {
   CompactionSelection picked;
   double quality = 0.0;
   NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
-  ASSERT_EQ(3, picked.size());
+  ASSERT_EQ(rowsets.size(), picked.size());
   ASSERT_GT(quality, 1.0);
   ASSERT_LT(quality, 2.0);
 }
 
+// Test that we don't compact together non-overlapping rowsets if doing so
+// doesn't actually reduce the number of rowsets.
+TEST_F(TestCompactionPolicy, TestSmallRowsetCompactionReducesRowsetCount) {
+  constexpr auto kBudgetMb = 1000000; // Enough to select all rowsets in all cases.
+  const auto num_rowsets = 100;
+  // Use rowsets close enough to the target size that 'num_rowsets' count of
+  // them compacted together does not reduce the count of rowsets.
+  const auto rowset_size =
+    FLAGS_budgeted_compaction_target_rowset_size * (1 - 1.0 / (num_rowsets - 1));
+  RowSetVector rowsets;
+  CompactionSelection picked;
+  double quality = 0.0;
+  for (int i = 0; i < num_rowsets; i++) {
+    picked.clear();
+
+    // [0 - 1][1 - 2] ... [98 - 99][99 - 100] built up over time.
+    rowsets.push_back(std::make_shared<MockDiskRowSet>(
+        StringPrintf("%010d", i),
+        StringPrintf("%010d", i + 1),
+        rowset_size));
+    NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
+
+    // There should never be a worthwhile compaction.
+    ASSERT_TRUE(picked.empty());
+    ASSERT_EQ(0.0, quality);
+  }
+}
+
+// Test that the score of compacting two rowsets decreases monotonically as the
+// size of the two rowsets increases, and that the tradeoff factor is set
+// correctly so that a compaction actually reduce the count of rowsets.
+TEST_F(TestCompactionPolicy, TestSmallRowsetTradeoffFactor) {
+  constexpr auto kBudgetMb = 1000; // Enough to select all rowsets in all cases.
+  CompactionSelection picked;
+  double quality = 0.0;
+  double prev_quality = std::numeric_limits<double>::max();
+
+  // Compact two equal-sized rowsets that are various fractions of the target
+  // rowset size.
+  const double target_size_bytes =
+      FLAGS_budgeted_compaction_target_rowset_size;
+  for (const auto divisor : { 32, 16, 8, 4, 2, 1 }) {
+    SCOPED_TRACE(strings::Substitute("divisor = $0", divisor));
+    picked.clear();
+    const auto size_bytes = target_size_bytes / divisor;
+    /*
+     * [A -- B][B -- C]
+     */
+    const RowSetVector rowsets = {
+      std::make_shared<MockDiskRowSet>("A", "B", size_bytes),
+      std::make_shared<MockDiskRowSet>("B", "C", size_bytes),
+    };
+    NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
+
+    // The quality score should monotonically decrease.
+    ASSERT_LE(quality, prev_quality);
+    prev_quality = quality;
+
+    // As a spot check of the prioritization, we should stop wanting to compact
+    // when the divisor is two or one.
+    if (divisor > 2) {
+      ASSERT_EQ(rowsets.size(), picked.size());
+    } else {
+      ASSERT_LT(quality, FLAGS_compaction_minimum_improvement);
+      ASSERT_TRUE(picked.empty());
+    }
+  }
+
+  // However, compacting a target rowset size's worth of data with it split
+  // between more than two rowsets should result in a compaction.
+  const auto size_bytes = target_size_bytes / 3;
+  /*
+   * [A -- B][B -- C][C -- D]
+   */
+  const RowSetVector rowsets = {
+    std::make_shared<MockDiskRowSet>("A", "B", size_bytes),
+    std::make_shared<MockDiskRowSet>("B", "C", size_bytes),
+    std::make_shared<MockDiskRowSet>("C", "D", size_bytes),
+  };
+  NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
+  ASSERT_EQ(rowsets.size(), picked.size());
+  ASSERT_GT(quality, FLAGS_compaction_minimum_improvement);
+}
+
+// Compaction policy is designed so that height-reducing compactions should
+// generally take priority over size-based compactions. So, in particular,
+// given a layout like
+//
+// [ -- 32MiB -- ]    [ 8MiB ][ 8 MiB ][8 MiB ][ 8MiB ]
+// [ -- 32MiB -- ]
+//
+// and a budget of 64MiB, the policy should choose to compact the two
+// overlapping rowsets despite the fact that that selection has a size-score
+// of 0, while compacting one or more of the 8MiB rowsets has a big size score.
+TEST_F(TestCompactionPolicy, TestHeightBasedDominatesSizeBased) {
+  constexpr auto kBigRowSetSizeBytes = 32 * 1024 * 1024;
+  constexpr auto kSmallRowSetSizeBytes = 8 * 1024 * 1024;
+  constexpr auto kBudgetMb = 64;
+  const RowSetVector rowsets = {
+    std::make_shared<MockDiskRowSet>("A", "B", kBigRowSetSizeBytes),
+    std::make_shared<MockDiskRowSet>("A", "B", kBigRowSetSizeBytes),
+    std::make_shared<MockDiskRowSet>("C", "D", kSmallRowSetSizeBytes),
+    std::make_shared<MockDiskRowSet>("D", "E", kSmallRowSetSizeBytes),
+    std::make_shared<MockDiskRowSet>("E", "F", kSmallRowSetSizeBytes),
+    std::make_shared<MockDiskRowSet>("F", "G", kSmallRowSetSizeBytes),
+  };
+
+  CompactionSelection picked;
+  double quality = 0.0;
+  NO_FATALS(RunTestCase(rowsets, kBudgetMb, &picked, &quality));
+  ASSERT_EQ(2, picked.size());
+  for (const auto* rowset : picked) {
+    ASSERT_EQ(kBigRowSetSizeBytes, rowset->OnDiskSize());
+  }
+}
+
 namespace {
 double ComputeAverageRowsetHeight(
     const vector<std::pair<string, string>>& intervals) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/f65211ba/src/kudu/tablet/compaction_policy.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/compaction_policy.cc b/src/kudu/tablet/compaction_policy.cc
index 6dee57c..77af79b 100644
--- a/src/kudu/tablet/compaction_policy.cc
+++ b/src/kudu/tablet/compaction_policy.cc
@@ -40,7 +40,7 @@
 using std::vector;
 using strings::Substitute;
 
-DEFINE_int32(budgeted_compaction_target_rowset_size, 32*1024*1024,
+DEFINE_int64(budgeted_compaction_target_rowset_size, 32 * 1024 * 1024,
              "The target size in bytes for DiskRowSets produced by flushes or "
              "compactions when the budgeted compaction policy is used.");
 TAG_FLAG(budgeted_compaction_target_rowset_size, advanced);
@@ -62,24 +62,31 @@ TAG_FLAG(compaction_minimum_improvement, advanced);
 namespace kudu {
 namespace tablet {
 
-// Adjust the result downward slightly for wider solutions.
+// A constant factor used to penalize wider solutions when there is rowset
+// overlap.
 // Consider this input:
 //
 //  |-----A----||----C----|
 //  |-----B----|
 //
-// where A, B, and C are all 1MB, and the budget is 10MB.
+// where A, B, and C are all 30MiB, the budget is 100MiB, and the target rowset
+// size is 32MiB. Without this tweak, the decrease in tablet height from the
+// compaction selection {A, B, C} is the exact same as from {A, B}, since both
+// compactions would yield a tablet with average height 1. Taking the rowset
+// sizes into account, {A, B, C} is favored over {A, B}. However, including
+// C in the compaction is not greatly beneficial: there's 60MiB of extra I/O
+// in exchange for no benefit in average height over {A, B} and no decrease in
+// the number of rowsets, just a reorganization of the data into 2 rowsets of
+// size 32MiB and one of size 26MiB. On the other hand, if A, B, and C were all
+// 8MiB rowsets, then {A, B, C} is clearly a superior selection to {A, B}.
+// Furthermore, the arrangement
 //
-// Without this tweak, the solution {A, B, C} has the exact same
-// solution value as {A, B}, since both compactions would yield a
-// tablet with average height 1. Since both solutions fit within
-// the budget, either would be a valid pick, and it would be up
-// to chance which solution would be selected.
-// Intuitively, though, there's no benefit to including "C" in the
-// compaction -- it just uses up some extra IO. If we slightly
-// penalize wider solutions as a tie-breaker, then we'll pick {A, B}
-// here.
-static const double kSupportAdjust = 1.01;
+// |- A -| |- B -| |- C -|
+//
+// shouldn't be penalized based on total width at all. So this penalty is
+// applied only when there is overlap, and it is small so that a significant
+// benefit in reducing rowset count can overwhelm it.
+static const double kSupportAdjust = 1.003;
 
 ////////////////////////////////////////////////////////////
 // BudgetedCompactionPolicy
@@ -116,14 +123,15 @@ namespace {
 struct KnapsackTraits {
   typedef const RowSetInfo* item_type;
   typedef double value_type;
-  static int get_weight(const RowSetInfo* item) {
+  static int get_weight(item_type item) {
     return item->size_mb();
   }
-  static value_type get_value(const RowSetInfo* item) {
-    return item->width();
+  static value_type get_value(item_type item) {
+    return item->value();
   }
 };
 
+
 // Incremental calculator for lower and upper bounds on a knapsack solution,
 // given a set of items. The bounds are computed by solving the
 // simpler "fractional knapsack problem" -- i.e the related problem
@@ -172,11 +180,11 @@ class BoundCalculator {
                    fractional_solution_.end(),
                    compareByDescendingDensity);
     current_weight_ += candidate.size_mb();
-    current_value_ += candidate.width();
+    current_value_ += candidate.value();
     const RowSetInfo* top = fractional_solution_.front();
     while (current_weight_ - top->size_mb() > max_weight_) {
       current_weight_ -= top->size_mb();
-      current_value_ -= top->width();
+      current_value_ -= top->value();
       std::pop_heap(fractional_solution_.begin(),
                     fractional_solution_.end(),
                     compareByDescendingDensity);
@@ -203,7 +211,7 @@ class BoundCalculator {
     // - the value of the (N+1)th item alone, if it fits
     // This is a 2-approximation (i.e. no worse than 1/2 of the best solution).
     // See https://courses.engr.illinois.edu/cs598csc/sp2009/lectures/lecture_4.pdf
-    double lower_bound = std::max(current_value_ - top.width(), top.width());
+    double lower_bound = std::max(current_value_ - top.value(), top.value());
 
     // An upper bound for the integer problem is the solution to the fractional
     // problem since in the fractional problem we can add the fraction of the
@@ -233,7 +241,7 @@ class BoundCalculator {
     // See above: there are two choices for the lower-bound estimate,
     // and we need to return the one matching the bound we computed.
     const RowSetInfo* top = fractional_solution_.front();
-    if (current_value_ - top->width() > top->width()) {
+    if (current_value_ - top->value() > top->value()) {
       // The current solution less the top (minimum density) element.
       solution->assign(fractional_solution_.begin() + 1,
                        fractional_solution_.end());
@@ -257,6 +265,16 @@ class BoundCalculator {
   double top_density_;
 };
 
+// If 'sum_width' is no bigger than 'union_width', there is little or no overlap
+// between the rowsets relative to their total width. In this case, we don't
+// want to penalize the solution value for being wide, so that small rowset
+// compaction will work. If there is significant overlap, then we do want
+// to penalize wider solutions. See the comment about 'kSupportAdjust' above.
+double MaybePenalizeWideSolution(double sum_width, double union_width) {
+  return sum_width <= union_width ? sum_width - union_width :
+                                    sum_width - kSupportAdjust * union_width;
+}
+
 } // anonymous namespace
 
 void BudgetedCompactionPolicy::RunApproximation(
@@ -286,8 +304,8 @@ void BudgetedCompactionPolicy::RunApproximation(
 
       bound_calc.Add(rowset_b);
       auto bounds = bound_calc.ComputeLowerAndUpperBound();
-      double lower = bounds.first - union_width * kSupportAdjust;
-      double upper = bounds.second - union_width * kSupportAdjust;
+      double lower = MaybePenalizeWideSolution(bounds.first, union_width);
+      double upper = MaybePenalizeWideSolution(bounds.second, union_width);
       best_upper = std::max(upper, best_upper);
       if (lower > best_solution->value) {
         vector<const RowSetInfo*> approx_solution;
@@ -353,7 +371,8 @@ void BudgetedCompactionPolicy::RunExact(
 
       union_max = std::max(item->cdf_max_key(), union_max);
       DCHECK_GE(union_max, union_min);
-      double solution = best_value - (union_max - union_min) * kSupportAdjust;
+      double solution = MaybePenalizeWideSolution(best_value,
+                                                  union_max - union_min);
       if (solution > best_solution->value) {
         solver.TracePath(best_with_this_item, &chosen_indexes);
         best_solution->value = solution;
@@ -452,7 +471,7 @@ Status BudgetedCompactionPolicy::PickRowSets(
       *std::max_element(best_upper_bounds.begin(), best_upper_bounds.end()) <=
       FLAGS_compaction_minimum_improvement) {
     VLOG(1) << "Approximation algorithm short-circuited exact compaction calculation";
-    *quality = 0;
+    *quality = 0.0;
     return Status::OK();
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/f65211ba/src/kudu/tablet/rowset_info.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/rowset_info.cc b/src/kudu/tablet/rowset_info.cc
index 171a0e4..7adb953 100644
--- a/src/kudu/tablet/rowset_info.cc
+++ b/src/kudu/tablet/rowset_info.cc
@@ -26,24 +26,52 @@
 #include <unordered_map>
 #include <utility>
 
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
+#include "kudu/common/key_range.h"
 #include "kudu/gutil/casts.h"
 #include "kudu/gutil/endian.h"
+#include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/rowset.h"
 #include "kudu/tablet/rowset_tree.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/flag_validators.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
-#include "kudu/common/key_range.h"
 
 using std::shared_ptr;
 using std::string;
 using std::unordered_map;
 using std::vector;
 
+DECLARE_double(compaction_minimum_improvement);
+DECLARE_int64(budgeted_compaction_target_rowset_size);
+
+DEFINE_bool(compaction_force_small_rowset_tradeoff, false,
+            "Whether to allow the compaction small rowset tradeoff factor to "
+            "be larger than the compaction minimum improvement score. Doing so "
+            "will have harmful effects on the performance of the tablet "
+            "server. Do not set this unless you know what you are doing.");
+TAG_FLAG(compaction_force_small_rowset_tradeoff, advanced);
+TAG_FLAG(compaction_force_small_rowset_tradeoff, experimental);
+TAG_FLAG(compaction_force_small_rowset_tradeoff, runtime);
+TAG_FLAG(compaction_force_small_rowset_tradeoff, unsafe);
+
+DEFINE_double(compaction_small_rowset_tradeoff, 0.009,
+              "The weight of small rowset compaction compared to "
+              "height-based compaction. This value must be less than "
+              "-compaction_minimum_improvement to prevent compaction loops. "
+              "Only change this if you know what you are doing.");
+TAG_FLAG(compaction_small_rowset_tradeoff, advanced);
+TAG_FLAG(compaction_small_rowset_tradeoff, experimental);
+TAG_FLAG(compaction_small_rowset_tradeoff, runtime);
+
 // Enforce a minimum size of 1MB, since otherwise the knapsack algorithm
 // will always pick up small rowsets no matter what.
 static const int kMinSizeMb = 1;
@@ -53,6 +81,26 @@ namespace tablet {
 
 namespace {
 
+bool ValidateSmallRowSetTradeoffVsMinScore() {
+  if (FLAGS_compaction_force_small_rowset_tradeoff) {
+    return true;
+  }
+  const auto tradeoff = FLAGS_compaction_small_rowset_tradeoff;
+  const auto min_score = FLAGS_compaction_minimum_improvement;
+  if (tradeoff >= min_score) {
+    LOG(ERROR) << strings::Substitute(
+        "-compaction_small_rowset_tradeoff=$0 must be less than "
+        "-compaction_minimum_improvement=$1 in order to prevent pointless "
+        "compactions; if you know what you are doing, pass "
+        "-compaction_force_small_rowset_tradeoff to permit this",
+        tradeoff, min_score);
+    return false;
+  }
+  return true;
+}
+GROUP_FLAG_VALIDATOR(compaction_small_rowset_tradeoff_and_min_score,
+                     &ValidateSmallRowSetTradeoffVsMinScore);
+
 // Less-than comparison by minimum key (both by actual encoded key and cdf)
 bool LessCDFAndRSMin(const RowSetInfo& a, const RowSetInfo& b) {
   return a.cdf_min_key() < b.cdf_min_key() && a.min_key().compare(b.min_key()) < 0;
@@ -65,7 +113,7 @@ bool LessCDFAndRSMax(const RowSetInfo& a, const RowSetInfo& b) {
 
 // Debug-checks that min <= imin <= imax <= max
 void DCheckInside(const Slice& min, const Slice& max,
-                 const Slice& imin, const Slice& imax) {
+                  const Slice& imin, const Slice& imax) {
   DCHECK_LE(min.compare(max), 0);
   DCHECK_LE(imin.compare(imax), 0);
   DCHECK_LE(min.compare(imin), 0);
@@ -178,6 +226,17 @@ void CheckCollectOrderedCorrectness(const vector<RowSetInfo>& min_key,
   DCHECK(std::is_sorted(max_key.begin(), max_key.end(), LessCDFAndRSMax));
 }
 
+double ComputeRowsetValue(double width, uint64_t size_bytes) {
+  const auto gamma = FLAGS_compaction_small_rowset_tradeoff;
+  const auto target_size_bytes = FLAGS_budgeted_compaction_target_rowset_size;
+
+  // This is an approximation to the expected reduction in rowset count per
+  // input rowset. See the compaction policy design doc for more details.
+  const auto size_score =
+      1 - static_cast<double>(size_bytes) / target_size_bytes;
+  return width + gamma * size_score;
+}
+
 } // anonymous namespace
 
 // RowSetInfo class ---------------------------------------------------
@@ -408,7 +467,8 @@ void RowSetInfo::FinalizeCDFVector(double quot, vector<RowSetInfo>* vec) {
                                  << " bytes.";
     cdf_rs.cdf_min_key_ /= quot;
     cdf_rs.cdf_max_key_ /= quot;
-    cdf_rs.density_ = (cdf_rs.cdf_max_key() - cdf_rs.cdf_min_key()) / cdf_rs.size_mb_;
+    cdf_rs.value_ = ComputeRowsetValue(cdf_rs.width(), cdf_rs.extra_->size_bytes);
+    cdf_rs.density_ = cdf_rs.value_ / cdf_rs.size_mb_;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/f65211ba/src/kudu/tablet/rowset_info.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/rowset_info.h b/src/kudu/tablet/rowset_info.h
index 437e357..a8290d3 100644
--- a/src/kudu/tablet/rowset_info.h
+++ b/src/kudu/tablet/rowset_info.h
@@ -83,13 +83,18 @@ class RowSetInfo {
 
   // Return the "width" of the candidate rowset.
   //
-  // This is an estimate of the percentage of the tablet data which
-  // is spanned by this RowSet, calculated by integrating the
-  // probability distribution function across this rowset's keyrange.
+  // This is an estimate of the percentage of the tablet data which lies between
+  // the min and max key of this RowSet (including data not in this rowset but
+  // in overlapping ones), calculated by integrating the probability
+  // distribution function across this rowset's keyrange.
   double width() const {
     return cdf_max_key_ - cdf_min_key_;
   }
 
+  double value() const {
+    return value_;
+  }
+
   double density() const { return density_; }
 
   RowSet* rowset() const { return extra_->rowset; }
@@ -113,6 +118,17 @@ class RowSetInfo {
   int size_mb_;
 
   double cdf_min_key_, cdf_max_key_;
+
+  // The value of the rowset is its value as an item in the compaction knapsack
+  // problem:
+  // value = width + tradeoff * (1 - size in bytes / target size in bytes).
+  double value_;
+
+  // The density is the density of the rowset as an item in compaction knapsack
+  // problems:
+  // density = value / size in MiB
+  // It doesn't matter if the size is in bytes or MiB since densities are only
+  // used to compare rowsets.
   double density_;
 
   // We move these out of the RowSetInfo object because the std::strings are relatively