├── .DS_Store ├── LICENSE ├── README.md └── src ├── .DS_Store ├── benchmark ├── .DS_Store ├── AggregateFunction.java ├── ParallelizationTest.java ├── RunMoqoBenchmark.java ├── RunMoqoBenchmarkExact.java ├── RunParallelizationBenchmark.java ├── RunSoqoBenchmark.java ├── Statistics.java ├── StatisticsTest.java └── TestCase.java ├── common ├── Constants.java └── RandomNumbers.java ├── cost ├── .DS_Store ├── CostModel.java ├── MultiCostModel.java ├── SingleCostModel.java └── local │ ├── .DS_Store │ ├── BufferCostModel.java │ ├── DiscCostModel.java │ ├── LocalCostModelsTest.java │ └── TimeCostModel.java ├── optimizer ├── .DS_Store ├── Optimizer.java ├── ParallelOptimizer.java ├── approximate │ ├── BitSetIterator.java │ ├── BitSetIteratorTest.java │ ├── DPmoqo.java │ └── DPmoqoTest.java ├── greedy │ └── GreedyHeuristic.java ├── parallelized │ ├── .DS_Store │ ├── Master.java │ ├── Slave.java │ ├── SlaveResult.java │ ├── SlaveTask.java │ └── partitioning │ │ ├── ConstraintType.java │ │ ├── PartitioningMaster.java │ │ ├── PartitioningSlave.java │ │ ├── PartitioningSlaveResult.java │ │ ├── PartitioningSlaveTask.java │ │ └── PartitioningSlaveTest.java └── randomized │ ├── .DS_Store │ ├── RandomizedOptimizer.java │ ├── genetic │ ├── .DS_Store │ ├── Individual.java │ ├── IndividualCostComparator.java │ ├── IndividualCostComparatorTest.java │ ├── IndividualTest.java │ ├── JoinPair.java │ ├── JoinPairTest.java │ ├── NSGA2.java │ ├── NSGA2Test.java │ ├── SoqoGA.java │ └── SoqoGATest.java │ ├── moqo │ ├── .DS_Store │ ├── AnnealingPhaseSAH.java │ ├── AnnealingPhaseSAIO.java │ ├── ClimbingPhase.java │ ├── FastClimber.java │ ├── GreedyPhase.java │ ├── IterativeImprovement.java │ ├── OnePhase.java │ ├── Phase.java │ ├── TwoPhase.java │ └── X.java │ └── soqo │ ├── .DS_Store │ └── genetic │ ├── JGAPsoqoGA.java │ └── PlanFitness.java ├── plans ├── .DS_Store ├── JoinOrderSpace.java ├── JoinPlan.java ├── ParetoPlanSet.java ├── PathNode.java ├── Plan.java ├── ScanPlan.java ├── operators │ ├── JoinOperator.java │ ├── Operator.java │ ├── ScanOperator.java │ ├── cluster │ │ ├── ClusterScan.java │ │ └── ReduceSideJoin.java │ └── local │ │ ├── BNLjoin.java │ │ ├── HashJoin.java │ │ ├── LocalJoin.java │ │ ├── LocalScan.java │ │ └── SortMergeJoin.java └── spaces │ ├── .DS_Store │ ├── LocalPlanSpace.java │ ├── LocalPlanSpaceTest.java │ ├── LocalSpaceVariant.java │ └── PlanSpace.java ├── queries ├── JoinGraphType.java ├── JoinType.java ├── Query.java ├── QueryFactory.java ├── QueryFactoryTest.java └── QueryTest.java ├── relations ├── Relation.java ├── RelationFactory.java └── RelationTest.java └── util ├── GreedyCriterion.java ├── GreedyUtil.java ├── GreedyUtilTest.java ├── LocalSearchUtil.java ├── LocalSearchUtilTest.java ├── MathUtil.java ├── MathUtilTest.java ├── ParetoUtil.java ├── ParetoUtilTest.java ├── PruningUtil.java ├── PruningUtilTest.java ├── SamplingUtil.java └── TestUtil.java /.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/.DS_Store -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2016 itrummer 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # query-optimizer-lib 2 | Contains implementations of various algorithms for database query optimization. The algorithms cover multiple problem variants (e.g., traditional and multi-objective query optimization) and offer various tradeoffs between optimality guarantees and optimization time (e.g., various randomized algorithms, approximation schemes, exhaustive algorithms, parallel algorithms, ...). 3 | -------------------------------------------------------------------------------- /src/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/.DS_Store -------------------------------------------------------------------------------- /src/benchmark/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/benchmark/.DS_Store -------------------------------------------------------------------------------- /src/benchmark/AggregateFunction.java: -------------------------------------------------------------------------------- 1 | package benchmark; 2 | 3 | /** 4 | * Describes the way in which feature values for the same query size and time period are 5 | * aggregated: 6 | * MEAN arithmetic mean 7 | * MEDIAN the median 8 | * NR_WINS number of queries for which one algorithm has lower or same value as all others 9 | * 10 | * @author immanueltrummer 11 | * 12 | */ 13 | public enum AggregateFunction { 14 | MEAN, MEDIAN, NR_WINS 15 | } 16 | -------------------------------------------------------------------------------- /src/benchmark/ParallelizationTest.java: -------------------------------------------------------------------------------- 1 | package benchmark; 2 | 3 | import org.apache.spark.api.java.JavaRDD; 4 | import org.apache.spark.api.java.JavaSparkContext; 5 | import org.apache.spark.api.java.function.Function; 6 | 7 | import common.Constants; 8 | import optimizer.approximate.DPmoqo; 9 | import optimizer.parallelized.partitioning.PartitioningSlave; 10 | import optimizer.parallelized.partitioning.PartitioningSlaveResult; 11 | import optimizer.parallelized.partitioning.PartitioningSlaveTask; 12 | import plans.JoinOrderSpace; 13 | import plans.ParetoPlanSet; 14 | import plans.Plan; 15 | import queries.JoinGraphType; 16 | import queries.JoinType; 17 | import queries.Query; 18 | import queries.QueryFactory; 19 | import util.PruningUtil; 20 | import util.TestUtil; 21 | 22 | import java.util.LinkedList; 23 | import java.util.List; 24 | 25 | /** 26 | * Test version of benchmark for parallel query optimization. 27 | * 28 | * @author immanueltrummer 29 | * 30 | */ 31 | public class ParallelizationTest { 32 | 33 | public static void main(String[] args) { 34 | // TODO: this must iterate over different values of course 35 | final int degreeOfParallelism = 4; 36 | // Create Spark context 37 | //SparkConf sparkConf = new SparkConf().setAppName("JavaWordCount"); 38 | //JavaSparkContext ctx = new JavaSparkContext(sparkConf); 39 | JavaSparkContext ctx = new JavaSparkContext("local", "Simple App"); 40 | // Randomly generate query 41 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 6, JoinType.MN); 42 | // Create local collection containing task descriptions for the workers 43 | List tasksLocal = new LinkedList(); 44 | for (int partitionID = 0; partitionID tasks = ctx.parallelize(tasksLocal, degreeOfParallelism); 52 | @SuppressWarnings("serial") 53 | JavaRDD results = tasks.map( 54 | new Function() { 55 | public PartitioningSlaveResult call(PartitioningSlaveTask t) { 56 | return PartitioningSlave.optimize(t); } 57 | }); 58 | List resultsLocal = results.collect(); 59 | // Determine the overall best plan by comparing best plans from different plan space partitions 60 | List bestPlanList = new LinkedList(); 61 | for (PartitioningSlaveResult result : resultsLocal) { 62 | for (Plan partitionParetoPlan : result.paretoPlans) { 63 | PruningUtil.pruneCostBased(bestPlanList, partitionParetoPlan, new boolean[] {true}); 64 | } 65 | } 66 | // Print out best plan 67 | Plan bestPlanParallel = bestPlanList.iterator().next(); 68 | System.out.println("Cost of best plan found in parallel: " + bestPlanParallel.cost[0]); 69 | ctx.close(); 70 | // Verify that best plan calculated in parallel has same cost as best plan found by local optimizer 71 | Statistics.init(1, 1, 1, 1); 72 | DPmoqo dpMoqo = new DPmoqo(1); 73 | ParetoPlanSet bestPlansLocal = dpMoqo.approximateParetoSet(query, new boolean[] {true}, 74 | TestUtil.planSpace, TestUtil.timeCostModel, null, 0, 0, 0); 75 | Plan bestPlanLocal = bestPlansLocal.plans.iterator().next(); 76 | System.out.println("Cost of best plan found locally: " + bestPlanLocal.cost[0]); 77 | } 78 | 79 | } 80 | -------------------------------------------------------------------------------- /src/benchmark/RunParallelizationBenchmark.java: -------------------------------------------------------------------------------- 1 | package benchmark; 2 | 3 | import java.io.FileNotFoundException; 4 | import java.io.PrintWriter; 5 | import java.util.Arrays; 6 | 7 | import org.apache.spark.SparkConf; 8 | import org.apache.spark.api.java.JavaSparkContext; 9 | 10 | import common.Constants; 11 | import cost.MultiCostModel; 12 | import cost.SingleCostModel; 13 | import cost.local.BufferCostModel; 14 | import cost.local.TimeCostModel; 15 | import optimizer.ParallelOptimizer; 16 | import optimizer.parallelized.partitioning.PartitioningMaster; 17 | import plans.JoinOrderSpace; 18 | import plans.spaces.LocalPlanSpace; 19 | import plans.spaces.LocalSpaceVariant; 20 | import plans.spaces.PlanSpace; 21 | import queries.JoinGraphType; 22 | import queries.JoinType; 23 | import queries.Query; 24 | import queries.QueryFactory; 25 | 26 | /** 27 | * Benchmark for comparing different parallelized optimization algorithms. 28 | * 29 | * @author immanueltrummer 30 | * 31 | */ 32 | public class RunParallelizationBenchmark { 33 | /** 34 | * Outputs performance statistics about one specific configuration (join graph structure 35 | * and join type). 36 | * 37 | * @param joinGraph the structure of the join graph 38 | * @param joinType the type of the joins (determines the selectivity of join predicates) 39 | * @param nrQuerySizes the number of considered query sizes (i.e., number of tables) 40 | * @param querySizes associates each query size index to the number of joined tables 41 | * @param folder directory into which to put result files 42 | * @throws FileNotFoundException 43 | */ 44 | static void writeStatistics(JoinGraphType joinGraph, JoinType joinType, int nrQuerySizes, 45 | int[] querySizes, String folder) throws FileNotFoundException { 46 | // Select features for output and aggregation functions 47 | String[] featureNames = new String[] {"millis", "network", "mainMemory", 48 | "maxSlaveMillis", "minSlaveMillis", "timeouts", "memoryouts", 49 | "nrParetoPlans"}; 50 | AggregateFunction[] aggregateFunctions = new AggregateFunction[] { 51 | AggregateFunction.MEDIAN, AggregateFunction.MEDIAN, AggregateFunction.MEDIAN, 52 | AggregateFunction.MEDIAN, AggregateFunction.MEDIAN, 53 | AggregateFunction.MEAN, AggregateFunction.MEAN, 54 | AggregateFunction.MEDIAN 55 | }; 56 | int nrFeaturesToOutput = featureNames.length; 57 | // Write results into file 58 | for (int querySizeIndex=0; querySizeIndex models; 33 | 34 | public MultiCostModel(List models) { 35 | this.nrMetrics = models.size(); 36 | this.models = models; 37 | assert(nrMetrics>0); 38 | // Make sure that metric indices are consistent 39 | Set metricIndices = new TreeSet(); 40 | for (SingleCostModel model : models) { 41 | int metricIndex = model.metricIndex; 42 | assert(metricIndex=0); 39 | } 40 | 41 | @Override 42 | protected void updateJoinRoot(JoinPlan plan) { 43 | plan.setCostValue(metricIndex, -1); 44 | JoinOperator genericJoin = plan.getJoinOperator(); 45 | Plan leftPlan = plan.getLeftPlan(); 46 | Plan rightPlan = plan.getRightPlan(); 47 | if (genericJoin instanceof LocalJoin) { 48 | LocalJoin join = (LocalJoin)genericJoin; 49 | // We conservatively assume that buffer space consumption is added between the left and right 50 | // sub-plan and the top-level join operator. After materialization the buffer consumption 51 | // reduces however to zero. In those cases we take into account the buffer consumption before 52 | // and after materialization and take the maximum (-> we represent the maximal buffer consumption 53 | // that occured over the whole execution of the query plan). 54 | boolean leftMaterializes = leftPlan.materializes; 55 | boolean rightMaterializes = rightPlan.materializes; 56 | double leftBuffer = leftPlan.getCostValue(metricIndex); 57 | double rightBuffer = rightPlan.getCostValue(metricIndex); 58 | double joinBuffer = join.buffer; 59 | if (!leftMaterializes && !rightMaterializes) { 60 | plan.setCostValue(metricIndex, leftBuffer + rightBuffer + joinBuffer); 61 | } else if (leftMaterializes && !rightMaterializes) { 62 | plan.setCostValue(metricIndex, Math.max(leftBuffer, rightBuffer + joinBuffer)); 63 | } else if (!leftMaterializes && rightMaterializes) { 64 | plan.setCostValue(metricIndex, Math.max(rightBuffer, leftBuffer + joinBuffer)); 65 | } else { 66 | assert(leftMaterializes && rightMaterializes); 67 | plan.setCostValue(metricIndex, Math.max(leftBuffer, Math.max(rightBuffer, joinBuffer))); 68 | } 69 | } 70 | assert(plan.getCostValue(metricIndex)>=0); 71 | } 72 | 73 | @Override 74 | public String toString() { 75 | return "Buffer"; 76 | } 77 | 78 | } 79 | -------------------------------------------------------------------------------- /src/cost/local/DiscCostModel.java: -------------------------------------------------------------------------------- 1 | package cost.local; 2 | 3 | import cost.SingleCostModel; 4 | import plans.JoinPlan; 5 | import plans.Plan; 6 | import plans.ScanPlan; 7 | import plans.operators.JoinOperator; 8 | import plans.operators.ScanOperator; 9 | import plans.operators.local.LocalJoin; 10 | import plans.operators.local.LocalScan; 11 | 12 | /** 13 | * Calculates the amount of disc space (e.g., for materializing intermediate results) 14 | * that is consumed by a query plan. 15 | * 16 | * @author immanueltrummer 17 | * 18 | */ 19 | public class DiscCostModel extends SingleCostModel { 20 | /** 21 | * Used to verify the class version. 22 | */ 23 | private static final long serialVersionUID = 1L; 24 | 25 | public DiscCostModel(int metricIndex) { 26 | super(metricIndex); 27 | } 28 | 29 | @Override 30 | protected void updateScanRoot(ScanPlan plan) { 31 | plan.setCostValue(metricIndex, -1); 32 | ScanOperator scanOperator = plan.scanOperator; 33 | if (scanOperator instanceof LocalScan) { 34 | plan.setCostValue(metricIndex, plan.outputPages); 35 | } 36 | assert(plan.getCostValue(metricIndex)>=0); 37 | } 38 | 39 | @Override 40 | protected void updateJoinRoot(JoinPlan plan) { 41 | plan.setCostValue(metricIndex, -1); 42 | JoinOperator genericJoin = plan.getJoinOperator(); 43 | Plan leftPlan = plan.getLeftPlan(); 44 | Plan rightPlan = plan.getRightPlan(); 45 | if (genericJoin instanceof LocalJoin) { 46 | LocalJoin join = (LocalJoin)genericJoin; 47 | // We assume that disc space is not re-used during the execution of the same query plan. 48 | // Therefore the total disc space consumption is the sum over the disc consumptions of 49 | // individual operations. 50 | double leftDisc = leftPlan.getCostValue(metricIndex); 51 | double rightDisc = rightPlan.getCostValue(metricIndex); 52 | double addedDisc = join.materializeResult ? plan.outputPages : 0; 53 | plan.setCostValue(metricIndex, leftDisc + rightDisc + addedDisc); 54 | } 55 | assert(plan.getCostValue(metricIndex)>=0); 56 | } 57 | 58 | @Override 59 | public String toString() { 60 | return "Disc"; 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /src/cost/local/TimeCostModel.java: -------------------------------------------------------------------------------- 1 | package cost.local; 2 | 3 | import cost.SingleCostModel; 4 | import plans.JoinPlan; 5 | import plans.Plan; 6 | import plans.ScanPlan; 7 | import plans.operators.JoinOperator; 8 | import plans.operators.ScanOperator; 9 | import plans.operators.cluster.ClusterScan; 10 | import plans.operators.local.BNLjoin; 11 | import plans.operators.local.HashJoin; 12 | import plans.operators.local.LocalScan; 13 | import plans.operators.local.SortMergeJoin; 14 | import util.MathUtil; 15 | 16 | /** 17 | * Estimates the execution time of a query plan. The cost formulas are based on the VLDB'97 18 | * paper by Steinbrunn et al. "Heuristic and randomized optimization for the join ordering 19 | * problem". 20 | * 21 | * @author immanueltrummer 22 | * 23 | */ 24 | public class TimeCostModel extends SingleCostModel { 25 | /** 26 | * Used to verify the class version. 27 | */ 28 | private static final long serialVersionUID = 1L; 29 | /** 30 | * This constructor allows to specify the index at which the corresponding cost values 31 | * are stored within the cost vectors. 32 | * 33 | * @param metricIndex index of this cost metric within the cost vectors 34 | */ 35 | public TimeCostModel(int metricIndex) { 36 | super(metricIndex); 37 | } 38 | 39 | @Override 40 | protected void updateScanRoot(ScanPlan plan) { 41 | ScanOperator scanOperator = plan.scanOperator; 42 | if (scanOperator instanceof LocalScan) { 43 | // Attention: this benchmark does not consider single-table predicates: 44 | // only therefore input and output size of the scan is identical. 45 | // plan.cost[metricIndex] = plan.outputPages; 46 | 47 | // Scan cost will be counted at the first join 48 | plan.setCostValue(metricIndex, 0); 49 | } else { 50 | assert(scanOperator instanceof ClusterScan); 51 | assert(false); 52 | } 53 | } 54 | /** 55 | * Calculates the fraction of a relation whose hash table fits into memory. 56 | * 57 | * @param leftPages the number of buffer pages consumed by the left oeprand 58 | * @param buffer the amount of buffer space reserved for this join operator 59 | * @return the fraction of the hash table of the left relation that fits into main memory 60 | */ 61 | double calculateTableFraction(double leftPages, double buffer) { 62 | assert(buffer > 0); 63 | if (Double.isInfinite(leftPages)) { 64 | return 0; 65 | } else { 66 | double q = (buffer - Math.ceil((1.4*leftPages-buffer)/(buffer-1))) / leftPages; 67 | q = Math.max(q, 0); 68 | q = Math.min(q, 1); 69 | return q; 70 | } 71 | } 72 | @Override 73 | protected void updateJoinRoot(JoinPlan plan) { 74 | // store important fields in local variables for quick access 75 | JoinOperator genericJoin = plan.getJoinOperator(); 76 | Plan leftPlan = plan.getLeftPlan(); 77 | Plan rightPlan = plan.getRightPlan(); 78 | double leftPages = leftPlan.outputPages; 79 | double rightPages = rightPlan.outputPages; 80 | double leftGenerationCost = leftPlan.getCostValue(metricIndex); 81 | double rightGenerationCost = rightPlan.getCostValue(metricIndex); 82 | // calculate cost for reading and generating both inputs 83 | double inputCost = -1; 84 | if (genericJoin instanceof BNLjoin) { 85 | BNLjoin join = (BNLjoin)genericJoin; 86 | // In case of pipelining, the left operand is not materialized hence no reading cost 87 | double leftReadCost = leftPlan.materializes ? leftPages : 0; 88 | double leftCost = leftGenerationCost + leftReadCost; 89 | // Either right input is materialized or we have to re-execute it for each outer loop iteration 90 | double nrOuterIterations = Math.ceil(leftPages / join.buffer); 91 | double rightCost = rightPlan.materializes ? 92 | nrOuterIterations * rightPages + rightGenerationCost : 93 | nrOuterIterations * rightGenerationCost; 94 | inputCost = leftCost + rightCost; 95 | 96 | } else if (genericJoin instanceof HashJoin) { 97 | assert(leftPlan.materializes); 98 | assert(rightPlan.materializes); 99 | HashJoin join = (HashJoin)genericJoin; 100 | // q is ratio of left table whose hash table fits into memory 101 | double buffer = join.buffer; 102 | double q = calculateTableFraction(leftPages, buffer); 103 | // Must treat the special case that input size or generation cost have 104 | // become too large to be represented accurately. In that case, the cost 105 | // is infinite. 106 | inputCost = Double.isInfinite(leftPages) || Double.isInfinite(rightPages) || 107 | Double.isInfinite(leftGenerationCost) || 108 | Double.isInfinite(rightGenerationCost) ? 109 | Double.POSITIVE_INFINITY : 110 | leftPages + rightPages + 2 * (leftPages + rightPages) * (1-q) 111 | + leftGenerationCost + rightGenerationCost; 112 | assert(q>=0 && q<=1 && inputCost >= 0) : 113 | "inputCost: " + inputCost + 114 | "; leftPages: " + leftPages + "; buffer: " + buffer + "; q: " + q + 115 | "; leftPlan: " + leftPlan + "; leftRel: " + leftPlan.resultRel + 116 | "; rightPages: " + rightPages + "; leftGenerationCost: " + leftGenerationCost + 117 | "; rightGenerationCost: " + rightGenerationCost; 118 | 119 | } else if (genericJoin instanceof SortMergeJoin){ 120 | assert(leftPlan.materializes); 121 | assert(rightPlan.materializes); 122 | SortMergeJoin join = (SortMergeJoin)genericJoin; 123 | double buffer = join.buffer; 124 | // We assume that sorting is always required 125 | double leftCost = leftPages + leftPages * MathUtil.logOfBase(buffer, leftPages); 126 | double rightCost = rightPages + rightPages * MathUtil.logOfBase(buffer, rightPages); 127 | inputCost = leftCost + rightCost + leftGenerationCost + rightGenerationCost; 128 | 129 | } else { 130 | assert (false); 131 | } 132 | assert(inputCost >= 0) : "inputCost: " + inputCost + "; plan with old cost: " + plan; 133 | // calculate cost for writing output 134 | double outputCost = plan.materializes ? plan.outputPages : 0; 135 | plan.setCostValue(metricIndex, inputCost + outputCost); 136 | } 137 | 138 | @Override 139 | public String toString() { 140 | return "Time"; 141 | } 142 | } 143 | -------------------------------------------------------------------------------- /src/optimizer/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/optimizer/.DS_Store -------------------------------------------------------------------------------- /src/optimizer/Optimizer.java: -------------------------------------------------------------------------------- 1 | package optimizer; 2 | 3 | import java.io.Serializable; 4 | 5 | import cost.MultiCostModel; 6 | import plans.ParetoPlanSet; 7 | import plans.spaces.PlanSpace; 8 | import queries.Query; 9 | 10 | /** 11 | * Generic multi-objective query optimizer class - sub-classes implement 12 | * randomized and deterministic algorithms. 13 | * 14 | * @author immanueltrummer 15 | * 16 | */ 17 | public abstract class Optimizer implements Serializable { 18 | /** 19 | * Used to verify the class version. 20 | */ 21 | private static final long serialVersionUID = 1L; 22 | /** 23 | * Approximate Pareto plan set for query and store corresponding statistics 24 | * 25 | * @param query the query being optimized 26 | * @param consideredMetrics Boolean flags indicating for each cost metric whether we consider it 27 | * @param planSpace the plan space determines the set of admissible scan and join operators 28 | * @param costModel cost model estimating plan cost according to different metrics 29 | * @param refPlanSet set of Pareto-optimal reference plans against which the optimizers compare 30 | * @param algIndex the algorithm index under which statistics should be stored 31 | * @param sizeIndex the query size index under which statistics should be stored 32 | * @param queryIndex the test case index under which statistics should be stored 33 | * @return the set of Pareto-optimal query plans generated in this optimizer run 34 | */ 35 | public abstract ParetoPlanSet approximateParetoSet(Query query, boolean[] consideredMetrics, 36 | PlanSpace planSpace, MultiCostModel costModel, ParetoPlanSet refPlanSet, 37 | int algIndex, int sizeIndex, int queryIndex); 38 | } 39 | -------------------------------------------------------------------------------- /src/optimizer/ParallelOptimizer.java: -------------------------------------------------------------------------------- 1 | package optimizer; 2 | 3 | import java.io.Serializable; 4 | 5 | import org.apache.spark.api.java.JavaSparkContext; 6 | 7 | import cost.MultiCostModel; 8 | import plans.ParetoPlanSet; 9 | import plans.spaces.PlanSpace; 10 | import queries.Query; 11 | 12 | public abstract class ParallelOptimizer implements Serializable { 13 | /** 14 | * Used to verify class version. 15 | */ 16 | private static final long serialVersionUID = 1L; 17 | 18 | /** 19 | * Approximate Pareto plan set for query and store corresponding statistics 20 | * 21 | * @param query the query being optimized 22 | * @param consideredMetrics Boolean flags indicating for each cost metric whether we consider it 23 | * @param planSpace the plan space determines the set of admissible scan and join operators 24 | * @param costModel cost model estimating plan cost according to different metrics 25 | * @param refPlanSet set of Pareto-optimal reference plans against which the optimizers compare 26 | * @param algIndex the algorithm index under which statistics should be stored 27 | * @param sizeIndex the query size index under which statistics should be stored 28 | * @param queryIndex the test case index under which statistics should be stored 29 | * @param sparkContext Spark context to use for parallelization 30 | * @return the set of Pareto-optimal query plans generated in this optimizer run 31 | */ 32 | public abstract ParetoPlanSet approximateParetoSet(Query query, boolean[] consideredMetrics, 33 | PlanSpace planSpace, MultiCostModel costModel, ParetoPlanSet refPlanSet, 34 | int algIndex, int sizeIndex, int queryIndex, JavaSparkContext sparkContext); 35 | } 36 | -------------------------------------------------------------------------------- /src/optimizer/approximate/BitSetIterator.java: -------------------------------------------------------------------------------- 1 | package optimizer.approximate; 2 | 3 | import java.util.ArrayList; 4 | import java.util.BitSet; 5 | import java.util.Iterator; 6 | import java.util.List; 7 | 8 | public class BitSetIterator implements Iterator { 9 | 10 | private int k; 11 | private int[] indices; 12 | private List elements; 13 | private boolean hasNext = true; 14 | 15 | public BitSetIterator(BitSet set, int k) throws IllegalArgumentException { 16 | this.k = k; 17 | if (k>0) { 18 | this.indices = new int[k]; 19 | for(int i=0; i(); 22 | for (int i = set.nextSetBit(0); i >= 0; i = set.nextSetBit(i+1)) { 23 | elements.add(i); 24 | } 25 | } 26 | } 27 | 28 | public boolean hasNext() { 29 | return hasNext; 30 | } 31 | 32 | private int inc(int[] indices, int maxIndex, int depth) throws IllegalStateException { 33 | if(depth == indices.length) { 34 | throw new IllegalStateException("The End"); 35 | } 36 | if(indices[depth] < maxIndex) { 37 | indices[depth] = indices[depth]+1; 38 | } else { 39 | indices[depth] = inc(indices, maxIndex-1, depth+1)+1; 40 | } 41 | return indices[depth]; 42 | } 43 | 44 | private boolean inc() { 45 | try { 46 | inc(indices, elements.size() - 1, 0); 47 | return true; 48 | } catch (IllegalStateException e) { 49 | return false; 50 | } 51 | } 52 | 53 | public BitSet next() { 54 | BitSet result = new BitSet(); 55 | if (k > 0) { 56 | for(int i=indices.length-1; i>=0; i--) { 57 | result.set(elements.get(indices[i])); 58 | } 59 | hasNext = inc(); 60 | } else { 61 | hasNext = false; 62 | } 63 | return result; 64 | } 65 | 66 | public void remove() { 67 | throw new UnsupportedOperationException(); 68 | } 69 | } 70 | 71 | -------------------------------------------------------------------------------- /src/optimizer/approximate/BitSetIteratorTest.java: -------------------------------------------------------------------------------- 1 | package optimizer.approximate; 2 | 3 | import static org.junit.Assert.*; 4 | 5 | import java.util.ArrayList; 6 | import java.util.BitSet; 7 | import java.util.Iterator; 8 | import java.util.List; 9 | 10 | import org.junit.Test; 11 | 12 | public class BitSetIteratorTest { 13 | 14 | @Test 15 | public void test() { 16 | // test with small set numbers 17 | { 18 | BitSet set = new BitSet(); 19 | set.set(1); 20 | set.set(3); 21 | set.set(4); 22 | BitSetIterator iter = new BitSetIterator(set, 1); 23 | // generate list of result BitSets 24 | List setsToCover = new ArrayList(); 25 | for (int i : new int[] {1, 3, 4}) { 26 | BitSet resultSet = new BitSet(); 27 | resultSet.set(i); 28 | setsToCover.add(resultSet); 29 | } 30 | // test whether iterator covers all sets 31 | int iterationCtr = 0; 32 | while (iter.hasNext()) { 33 | ++iterationCtr; 34 | BitSet next = iter.next(); 35 | Iterator coverIter = setsToCover.iterator(); 36 | while (coverIter.hasNext()) { 37 | BitSet xorSet = (BitSet)next.clone(); 38 | BitSet setToCover = coverIter.next(); 39 | xorSet.xor(setToCover); 40 | if (xorSet.isEmpty()) { 41 | coverIter.remove(); 42 | } 43 | } 44 | } 45 | assertEquals(iterationCtr, 3); 46 | assertTrue(setsToCover.isEmpty()); 47 | } 48 | // test with large number of sets 49 | { 50 | BitSet set = new BitSet(); 51 | set.set(1); 52 | set.set(3); 53 | set.set(4); 54 | BitSetIterator iter = new BitSetIterator(set, 2); 55 | // generate list of result BitSets 56 | List setsToCover = new ArrayList(); 57 | for (int i : new int[] {1, 3, 4}) { 58 | for (int j : new int[] {1, 3, 4}) { 59 | if (i != j) { 60 | BitSet resultSet = new BitSet(); 61 | resultSet.set(i); 62 | resultSet.set(j); 63 | setsToCover.add(resultSet); 64 | } 65 | } 66 | } 67 | // test whether iterator covers all sets 68 | int iterationCtr = 0; 69 | while (iter.hasNext()) { 70 | ++iterationCtr; 71 | BitSet next = iter.next(); 72 | Iterator coverIter = setsToCover.iterator(); 73 | while (coverIter.hasNext()) { 74 | BitSet xorSet = (BitSet)next.clone(); 75 | BitSet setToCover = coverIter.next(); 76 | xorSet.xor(setToCover); 77 | if (xorSet.isEmpty()) { 78 | coverIter.remove(); 79 | } 80 | } 81 | } 82 | assertEquals(iterationCtr, 3); 83 | assertTrue(setsToCover.isEmpty()); 84 | } 85 | } 86 | 87 | } 88 | -------------------------------------------------------------------------------- /src/optimizer/approximate/DPmoqoTest.java: -------------------------------------------------------------------------------- 1 | package optimizer.approximate; 2 | 3 | import static org.junit.Assert.*; 4 | import static common.RandomNumbers.*; 5 | 6 | import java.util.Arrays; 7 | import java.util.LinkedList; 8 | import java.util.List; 9 | import java.util.Set; 10 | import java.util.TreeSet; 11 | 12 | import cost.MultiCostModel; 13 | import cost.SingleCostModel; 14 | import cost.local.BufferCostModel; 15 | import cost.local.DiscCostModel; 16 | import cost.local.TimeCostModel; 17 | import plans.spaces.LocalPlanSpace; 18 | import plans.spaces.PlanSpace; 19 | import queries.JoinGraphType; 20 | import queries.JoinType; 21 | import queries.Query; 22 | import queries.QueryFactory; 23 | import util.LocalSearchUtil; 24 | import util.ParetoUtil; 25 | import plans.ParetoPlanSet; 26 | import plans.Plan; 27 | 28 | import org.junit.Test; 29 | 30 | public class DPmoqoTest { 31 | 32 | @Test 33 | public void test() { 34 | { 35 | PlanSpace planSpace = new LocalPlanSpace(); 36 | List costModels = Arrays.asList(new SingleCostModel[] { 37 | new TimeCostModel(0), new BufferCostModel(1), new DiscCostModel(2) 38 | }); 39 | MultiCostModel multiModel = new MultiCostModel(costModels); 40 | DPmoqo dpAlg = new DPmoqo(2); 41 | boolean[] consideredMetrics = new boolean[] {true, true, true}; 42 | // Compare DP optimizer with hill climbing 43 | int nrJoinGraphTypes = JoinGraphType.values().length; 44 | for (int queryCtr=0; queryCtr<50; ++queryCtr) { 45 | // Generate random query 46 | int joinGraphIndex = random.nextInt(nrJoinGraphTypes); 47 | JoinGraphType joinGraph = JoinGraphType.values()[joinGraphIndex]; 48 | int nrTables = 4 + random.nextInt(2); 49 | Query query = QueryFactory.produceSteinbrunn(joinGraph, nrTables, JoinType.RANDOM); 50 | // Generate approximation with near-optimality guarantees 51 | ParetoPlanSet dpResult = dpAlg.approximateParetoSet( 52 | query, consideredMetrics, planSpace, multiModel, null, 0, 0, 0); 53 | assertTrue(dpResult.plans.size()>0); 54 | // Make sure that near-optimality guarantees hold when comparing to random plan sample 55 | List hillClimbingResults = new LinkedList(); 56 | Set tableIndices = new TreeSet(); 57 | for (int tableIndex=0; tableIndex resultPlans = new LinkedList(); 54 | resultPlans.add(resultPlan); 55 | // Update statistics 56 | if (refPlanSet != null){ 57 | long timePeriodMillis = TIMEOUT_MILLIS/NR_TIME_PERIODS; 58 | long millisPassed = System.currentTimeMillis() - startMillis; 59 | int curTimePeriod = (int)(millisPassed/timePeriodMillis); 60 | double curEpsilon = ParetoUtil.epsilonError( 61 | resultPlans, refPlanSet.plans, consideredMetrics); 62 | for (int periodCtr=0; periodCtr= curTimePeriod ? curEpsilon : Double.POSITIVE_INFINITY; 65 | Statistics.addToDoubleFeature(featureName, 66 | algIndex, sizeIndex, periodCtr, queryIndex, epsilon); 67 | } 68 | } 69 | // Return result plan as Pareto frontier 70 | return new ParetoPlanSet(resultPlans); 71 | } 72 | @Override 73 | public String toString() { 74 | return "Greedy(" + greedyCriterion + ")"; 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /src/optimizer/parallelized/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/optimizer/parallelized/.DS_Store -------------------------------------------------------------------------------- /src/optimizer/parallelized/Master.java: -------------------------------------------------------------------------------- 1 | package optimizer.parallelized; 2 | 3 | import static common.Constants.*; 4 | 5 | import java.io.ByteArrayOutputStream; 6 | import java.io.IOException; 7 | import java.io.ObjectOutputStream; 8 | import java.util.List; 9 | 10 | import org.apache.spark.api.java.JavaSparkContext; 11 | 12 | import benchmark.Statistics; 13 | import common.Constants; 14 | import cost.MultiCostModel; 15 | import optimizer.ParallelOptimizer; 16 | import plans.JoinOrderSpace; 17 | import plans.ParetoPlanSet; 18 | import plans.Plan; 19 | import plans.spaces.PlanSpace; 20 | import queries.Query; 21 | 22 | /** 23 | * Superclass of several optimization algorithms that can be parallelized on a large cluster. 24 | * 25 | * @author immanueltrummer 26 | * 27 | */ 28 | public abstract class Master extends ParallelOptimizer { 29 | /** 30 | * Used to verify the class version. 31 | */ 32 | private static final long serialVersionUID = 1L; 33 | /** 34 | * Whether only left deep or also bushy query plans should be considered by the optimizer. 35 | */ 36 | protected final JoinOrderSpace joinOrderSpace; 37 | /** 38 | * Approximation factor to use during pruning. 39 | */ 40 | protected final double alpha; 41 | /** 42 | * The number of milliseconds consumed for the last optimizer invocation. 43 | */ 44 | protected long lastRunMillis; 45 | /** 46 | * The number of bytes sent over the network during the last optimizer invocation. 47 | */ 48 | protected long lastRunBytesSent; 49 | /** 50 | * The maximal amount of main memory consumed by any node during the last optimizer invocation. 51 | */ 52 | protected long lastRunMainMemory; 53 | /** 54 | * Maximal number of milliseconds used for processing any of the slave tasks during last invocation. 55 | */ 56 | protected long lastRunMaxSlaveMillis; 57 | /** 58 | * Minimal number of milliseconds used for processing any of the slave tasks during last invocation. 59 | */ 60 | protected long lastRunMinSlaveMillis; 61 | /** 62 | * Whether at least one of the slaves had a timeout during the last invocation. 63 | */ 64 | protected boolean lastRunTimeouts; 65 | /** 66 | * Whether at least one of the slaves had a memoryout during the last invocation. 67 | */ 68 | protected boolean lastRunMemoryouts; 69 | /** 70 | * The number of Pareto plans collected from the workers in the last run. 71 | */ 72 | protected int lastRunNrParetoPlans; 73 | 74 | /** 75 | * Contains the best (=Pareto-optimal) plans generated in the last run. 76 | */ 77 | protected List lastRunBestPlanList; 78 | 79 | public Master(JoinOrderSpace joinOrderSpace, double alpha) { 80 | this.joinOrderSpace = joinOrderSpace; 81 | this.alpha = alpha; 82 | } 83 | /** 84 | * Calculates the size of an object in bytes by serializing it. 85 | * 86 | * @param obj the object whose size we want to calculate 87 | * @return long value representing the number of bytes consumed by the object 88 | * @throws IOException 89 | */ 90 | public static long sizeof(Object obj) { 91 | try { 92 | ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream(); 93 | ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteOutputStream); 94 | objectOutputStream.writeObject(obj); 95 | objectOutputStream.flush(); 96 | objectOutputStream.close(); 97 | return byteOutputStream.toByteArray().length; 98 | } catch (Exception e) { 99 | assert(false); 100 | return 0; 101 | } 102 | } 103 | /** 104 | * Solve the given test case by a sub-class specific parallelized optimization algorithm 105 | * and store performance statistics about the run. 106 | * 107 | * @param query the query being optimized 108 | * @param consideredMetrics Boolean flags indicating whether specific cost metrics are considered 109 | * @param planSpace determines the set of applicable scan and join operators 110 | * @param costModel estimates the execution cost of query plans according to multiple metrics 111 | * @param alpha approximation factor to use during pruning 112 | * @param degreeOfParallelism the degree of parallelism to use for the optimization 113 | * @param sparkContext Java spark context 114 | * @param timeoutMillis number of milliseconds after which a timeout is registered 115 | */ 116 | protected abstract void solveTestcase(Query query, boolean[] consideredMetrics, 117 | PlanSpace planSpace, MultiCostModel costModel, double alpha, 118 | int degreeOfParallelism, JavaSparkContext sparkContext, long timeoutMillis); 119 | /** 120 | * Optimizes one query multiple times with different degrees of parallelism 121 | * and logs the performance results. 122 | */ 123 | @Override 124 | public ParetoPlanSet approximateParetoSet(Query query, 125 | boolean[] consideredMetrics, PlanSpace planSpace, 126 | MultiCostModel costModel, ParetoPlanSet refPlanSet, int algIndex, 127 | int sizeIndex, int queryIndex, JavaSparkContext sparkContext) { 128 | // So many levels of degree of parallelism are tried per query 129 | int nrDegrees = DEGREES_OF_PARALLELISM.length; 130 | // Iterate over the degree of parallelism 131 | for (int parallelismIndex=0; parallelismIndex parallelizedTasks = 70 | sparkContext.parallelize(tasksLocal, degreeOfParallelism); 71 | System.out.println("Parallelized tasks"); 72 | @SuppressWarnings("serial") 73 | JavaRDD results = parallelizedTasks.map( 74 | new Function() { 75 | public PartitioningSlaveResult call(PartitioningSlaveTask t) { 76 | return PartitioningSlave.optimize(t);} 77 | }); 78 | System.out.println("Found best plans in each plan space partition"); 79 | // Collect result plans from all workers 80 | List resultsLocal = results.collect(); 81 | System.out.println("Collected best plans in each plan space partition"); 82 | // Determine the overall best plan by comparing best plans from different plan space partitions 83 | lastRunBestPlanList = new LinkedList(); 84 | for (PartitioningSlaveResult result : resultsLocal) { 85 | // Update statistics based on results 86 | lastRunMainMemory = Math.max(lastRunMainMemory, result.mainMemoryConsumption); 87 | lastRunMaxSlaveMillis = Math.max(lastRunMaxSlaveMillis, result.slaveTaskMillis); 88 | lastRunMinSlaveMillis = Math.min(lastRunMinSlaveMillis, result.slaveTaskMillis); 89 | lastRunTimeouts = lastRunTimeouts || result.timeout; 90 | lastRunMemoryouts = lastRunMemoryouts || result.memoryOut; 91 | lastRunNrParetoPlans += result.paretoPlans == null ? 0 : result.paretoPlans.size(); 92 | // Output statistics of current slave 93 | System.out.println("Slave memory: " + result.mainMemoryConsumption); 94 | System.out.println("Slave millis: " + result.mainMemoryConsumption); 95 | System.out.println("Slave timeout: " + result.timeout); 96 | System.out.println("Slave memory out: " + result.memoryOut); 97 | System.out.println("Checking for errors"); 98 | String errors = result.errors == null ? "none" : result.errors; 99 | System.out.println("Any errors: " + errors); 100 | // check for timeout indicated by a null pointer 101 | if (result.paretoPlans == null) { 102 | System.out.println("Timeout occurred - collecting statistics"); 103 | System.out.println("ATTENTION: we do not integrate statistics from other slaves!"); 104 | lastRunMainMemory = Math.max(lastRunMainMemory, result.mainMemoryConsumption); 105 | lastRunMillis = System.currentTimeMillis() - startMillis; 106 | lastRunBytesSent += sizeof(resultsLocal); 107 | lastRunBestPlanList = null; 108 | System.out.println("Finished solving test case"); 109 | return; 110 | } 111 | for (Plan partitionParetoPlan : result.paretoPlans) { 112 | System.out.println("Optimal plan in partition: " + partitionParetoPlan); 113 | PruningUtil.pruneCostBased(lastRunBestPlanList, partitionParetoPlan, consideredMetrics); 114 | } 115 | } 116 | // Update performance statistics 117 | System.out.println("Collecting statistics"); 118 | lastRunMillis = System.currentTimeMillis() - startMillis; 119 | lastRunBytesSent += sizeof(resultsLocal); 120 | System.out.println("Finished solving test case"); 121 | } 122 | /** 123 | * Class identifier for the master. 124 | * 125 | * @return short String identifier for the master 126 | */ 127 | @Override 128 | public String masterID() { 129 | return "PartitioningMaster(" + joinOrderSpace.toString() + ")"; 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /src/optimizer/parallelized/partitioning/PartitioningSlaveResult.java: -------------------------------------------------------------------------------- 1 | package optimizer.parallelized.partitioning; 2 | 3 | import java.util.List; 4 | 5 | import optimizer.parallelized.SlaveResult; 6 | import plans.Plan; 7 | 8 | public class PartitioningSlaveResult extends SlaveResult { 9 | /** 10 | * Used to verify the class version. 11 | */ 12 | private static final long serialVersionUID = 1L; 13 | /** 14 | * Contains the Pareto-optimal plans found in the plan space partition dedicate 15 | * to the corresponding worker node. 16 | */ 17 | public final List paretoPlans; 18 | 19 | public PartitioningSlaveResult(List paretoPlans, long mainMemoryConsumption, 20 | long elapsedMillis, boolean timeout, boolean memoryOut, String errors) { 21 | super(mainMemoryConsumption, elapsedMillis, timeout, memoryOut, errors); 22 | this.paretoPlans = paretoPlans; 23 | } 24 | 25 | } 26 | -------------------------------------------------------------------------------- /src/optimizer/parallelized/partitioning/PartitioningSlaveTask.java: -------------------------------------------------------------------------------- 1 | package optimizer.parallelized.partitioning; 2 | 3 | import cost.CostModel; 4 | import optimizer.parallelized.SlaveTask; 5 | import plans.JoinOrderSpace; 6 | import plans.spaces.PlanSpace; 7 | import queries.Query; 8 | 9 | /** 10 | * Describes a task that must be executed by a slave. 11 | * 12 | * @author immanueltrummer 13 | * 14 | */ 15 | public class PartitioningSlaveTask extends SlaveTask { 16 | /** 17 | * Used to verify the class version. 18 | */ 19 | private static final long serialVersionUID = 1L; 20 | /** 21 | * identifier of current search space partition 22 | */ 23 | final int partitionID; 24 | /** 25 | * total number of search space partitions 26 | */ 27 | final int nrPartitions; 28 | 29 | public PartitioningSlaveTask(Query query, JoinOrderSpace joinOrderSpace, 30 | PlanSpace planSpace, CostModel costModel, boolean[] consideredMetrics, 31 | double alpha, int partitionID, int nrPartitions, long timeoutMillis) { 32 | super(query, joinOrderSpace, planSpace, costModel, consideredMetrics, alpha, timeoutMillis); 33 | this.partitionID = partitionID; 34 | this.nrPartitions = nrPartitions; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/optimizer/randomized/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/optimizer/randomized/.DS_Store -------------------------------------------------------------------------------- /src/optimizer/randomized/genetic/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/optimizer/randomized/genetic/.DS_Store -------------------------------------------------------------------------------- /src/optimizer/randomized/genetic/IndividualCostComparator.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.genetic; 2 | 3 | import java.util.Comparator; 4 | 5 | /** 6 | * This class is used to compare individuals according to a specific cost metric 7 | * whose index is passed as constructor argument. 8 | * 9 | * @author immanueltrummer 10 | * 11 | */ 12 | class IndividualCostComparator implements Comparator { 13 | /** 14 | * The index of the cost metric according to which we compare individuals. 15 | */ 16 | final int metricIndex; 17 | 18 | public IndividualCostComparator(int metricIndex) { 19 | this.metricIndex = metricIndex; 20 | } 21 | @Override 22 | public int compare(Individual o1, Individual o2) { 23 | return Double.compare(o1.cost[metricIndex], o2.cost[metricIndex]); 24 | } 25 | } -------------------------------------------------------------------------------- /src/optimizer/randomized/genetic/IndividualCostComparatorTest.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.genetic; 2 | 3 | import static util.TestUtil.planSpace; 4 | import static org.junit.Assert.*; 5 | 6 | import java.util.Arrays; 7 | import java.util.Collections; 8 | 9 | import cost.MultiCostModel; 10 | import cost.SingleCostModel; 11 | import cost.local.TimeCostModel; 12 | import queries.JoinGraphType; 13 | import queries.JoinType; 14 | import queries.Query; 15 | import queries.QueryFactory; 16 | 17 | import org.junit.Test; 18 | 19 | public class IndividualCostComparatorTest { 20 | 21 | @Test 22 | public void test() { 23 | MultiCostModel costModel = new MultiCostModel( 24 | Arrays.asList(new SingleCostModel[] {new TimeCostModel(0)})); 25 | boolean[] consideredMetrics = new boolean[] {true}; 26 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 50, JoinType.MIN); 27 | SoqoGA ga = new SoqoGA(); 28 | ga.init(query, consideredMetrics, planSpace, costModel); 29 | // Sort population by cost value using tested comparator 30 | Collections.sort(ga.population, new IndividualCostComparator(0)); 31 | // Make sure that individuals are indeed ordered in ascending order of cost 32 | for (int i=0; i<127; ++i) { 33 | assertTrue(ga.population.get(i).cost[0] <= ga.population.get(i+1).cost[0]); 34 | } 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/optimizer/randomized/genetic/IndividualTest.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.genetic; 2 | 3 | import static org.junit.Assert.*; 4 | import static util.TestUtil.*; 5 | import static common.RandomNumbers.*; 6 | 7 | import java.util.List; 8 | 9 | import queries.JoinGraphType; 10 | import queries.JoinType; 11 | import queries.Query; 12 | import queries.QueryFactory; 13 | import util.TestUtil; 14 | 15 | import org.junit.Test; 16 | 17 | public class IndividualTest { 18 | 19 | // Checks whether two join pairs are equivalent 20 | boolean equivalentJoinPairs(JoinPair joinPair1, JoinPair joinPair2) { 21 | if (joinPair1.leftOperand == joinPair2.leftOperand && 22 | joinPair1.rightOperand == joinPair2.rightOperand && 23 | joinPair1.preferredOperator == joinPair2.preferredOperator) { 24 | return true; 25 | } else { 26 | return false; 27 | } 28 | } 29 | 30 | @Test 31 | public void test() { 32 | // Generating random join pairs 33 | { 34 | for (int i=0; i<50; ++i) { 35 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 4, JoinType.MN); 36 | Individual individual = new Individual(query, planSpace, costModel); 37 | assertEquals(3, individual.genes.length); 38 | for (int joinCtr=0; joinCtr<3; ++joinCtr) { 39 | JoinPair joinPair = individual.genes[joinCtr]; 40 | assertFalse(joinPair.leftOperand == joinPair.rightOperand); 41 | assertTrue(joinPair.leftOperand >= 0); 42 | assertTrue(joinPair.rightOperand >= 0); 43 | assertTrue(joinPair.preferredOperator < planSpace.consideredJoinOps.size()); 44 | } 45 | assertTrue(individual.genes[0].leftOperand < 4); 46 | assertTrue(individual.genes[0].rightOperand < 4); 47 | assertTrue(individual.genes[1].leftOperand < 3); 48 | assertTrue(individual.genes[1].rightOperand < 3); 49 | assertTrue(individual.genes[2].leftOperand < 2); 50 | assertTrue(individual.genes[2].rightOperand < 2); 51 | } 52 | } 53 | // Crossover 54 | { 55 | for (int i=0; i<50; ++i) { 56 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 7, JoinType.MN); 57 | Individual individual1 = new Individual(query, planSpace, costModel); 58 | Individual individual2 = new Individual(query, planSpace, costModel); 59 | List offspring = individual1.crossover(individual2); 60 | assertEquals(2, offspring.size()); 61 | Individual offspring1 = offspring.get(0); 62 | Individual offspring2 = offspring.get(1); 63 | for (int geneCtr=0; geneCtr<6; ++geneCtr) { 64 | boolean admissibleCopy = false; 65 | JoinPair individual1Join = individual1.genes[geneCtr]; 66 | JoinPair individual2Join = individual2.genes[geneCtr]; 67 | JoinPair offspring1Join = offspring1.genes[geneCtr]; 68 | JoinPair offspring2Join = offspring2.genes[geneCtr]; 69 | if (individual1Join.leftOperand == offspring1Join.leftOperand && 70 | individual1Join.rightOperand == offspring1Join.rightOperand && 71 | individual1Join.preferredOperator == offspring1Join.preferredOperator && 72 | individual2Join.leftOperand == offspring2Join.leftOperand && 73 | individual2Join.rightOperand == offspring2Join.rightOperand && 74 | individual2Join.preferredOperator == offspring2Join.preferredOperator) { 75 | admissibleCopy = true; 76 | } else if (individual1Join.leftOperand == offspring2Join.leftOperand && 77 | individual1Join.rightOperand == offspring2Join.rightOperand && 78 | individual1Join.preferredOperator == offspring2Join.preferredOperator && 79 | individual2Join.leftOperand == offspring1Join.leftOperand && 80 | individual2Join.rightOperand == offspring1Join.rightOperand && 81 | individual2Join.preferredOperator == offspring1Join.preferredOperator) { 82 | admissibleCopy = true; 83 | } 84 | assertTrue(admissibleCopy); 85 | } 86 | } 87 | } 88 | // Mutation 89 | { 90 | // Make sure: if mutation probability is zero then the original individual is returned 91 | for (int i=0; i<50; ++i) { 92 | int nrTables = 1 + random.nextInt(10); 93 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.STAR, nrTables, JoinType.MN); 94 | Individual original = new Individual(query, planSpace, costModel); 95 | Individual mutation = original.mutated(0); 96 | for (int joinCtr=0; joinCtr population; 28 | /** 29 | * The number of individuals in the population (we use the number proposed by Steinbrunn et al.). 30 | */ 31 | final static int POPULATION_SIZE = 128; 32 | /** 33 | * The probability that one gene gets mutated (we use the number proposed by Steinbrunn et al.). 34 | */ 35 | final static double MUTATION_PROBABILITY = 0.05; 36 | /** 37 | * The probability that we perform a crossover between two individuals instead of just 38 | * copying them into the next generation (we use the number proposed by Steinbrunn et al.). 39 | */ 40 | final static double CROSSOVER_RATE = 0.65; 41 | /** 42 | * Select one individual from the population by ranking-based wheel selection. 43 | * 44 | * @return a randomly selected individual (the higher its fitness, 45 | * the higher the selection probability). 46 | */ 47 | Individual selectIndividual() { 48 | // Each individual is associated with an interval in the domain of the random 49 | // selection whose interval size is proportional to the individual's rank (the 50 | // better the rank, the larger the interval). 51 | int max = POPULATION_SIZE * (POPULATION_SIZE/2); 52 | int rand = RandomNumbers.random.nextInt(max); 53 | // Find out whose individual's interval we have hit 54 | int sum = POPULATION_SIZE; 55 | int intervalWidth = POPULATION_SIZE; 56 | int index = 0; 57 | while (rand >= sum) { 58 | intervalWidth -= 1; 59 | sum += intervalWidth; 60 | index += 1; 61 | } 62 | assert(index < POPULATION_SIZE) : "max: " + max + "; rand: " + rand + 63 | "; sum: " + sum + "; intervalWidth: " + intervalWidth; 64 | return population.get(index); 65 | } 66 | /** 67 | * Randomly selects two individuals from the current population and uses them 68 | * to create two individuals for the next generation - either by a crossover 69 | * or by simply copying them. 70 | * 71 | * @return a list of two individuals to insert into the next generation 72 | */ 73 | List newIndividuals() { 74 | List newIndividuals = new LinkedList(); 75 | // Select parents randomly 76 | Individual parent1 = selectIndividual(); 77 | Individual parent2 = selectIndividual(); 78 | // Decide whether to copy or to use a crossover 79 | if (RandomNumbers.random.nextDouble() <= CROSSOVER_RATE) { 80 | // Add mutated offspring 81 | List offspring = parent1.crossover(parent2); 82 | for (Individual singleOffspring : offspring) { 83 | newIndividuals.add(singleOffspring.mutated(MUTATION_PROBABILITY)); 84 | } 85 | } else { 86 | // Add mutated copies 87 | newIndividuals.add(parent1.mutated(MUTATION_PROBABILITY)); 88 | newIndividuals.add(parent2.mutated(MUTATION_PROBABILITY)); 89 | } 90 | return newIndividuals; 91 | } 92 | /** 93 | * Fill up the new generation by repeating the following steps: 94 | * - select two individuals by ranking based selection 95 | * - either do a crossover or copy the two individuals into the next generation 96 | * - mutate the two new (or old) individuals 97 | */ 98 | @Override 99 | protected void refineApproximation(Query query, 100 | boolean[] consideredMetrics, PlanSpace planSpace, 101 | MultiCostModel costModel, int algIndex, int sizeIndex, 102 | int queryIndex) { 103 | // Order individuals in current generation by cost 104 | Collections.sort(population, new IndividualCostComparator(0)); 105 | // Start building new generation 106 | List newGeneration = new ArrayList(); 107 | // Iterate until the new generation is complete 108 | while (newGeneration.size() < POPULATION_SIZE) { 109 | newGeneration.addAll(newIndividuals()); 110 | } 111 | // grow individuals in new generation 112 | for (Individual individual : newGeneration) { 113 | individual.grow(); 114 | } 115 | population = newGeneration; 116 | // Determine best plan in current population 117 | Plan bestPlan = population.iterator().next().plan; 118 | for (Individual individual : population) { 119 | Plan newPlan = individual.plan; 120 | if (newPlan.cost[0] < bestPlan.cost[0]) { 121 | bestPlan = newPlan; 122 | } 123 | } 124 | // Insert best plan in current population into "frontier" approximation 125 | currentApproximation.clear(); 126 | currentApproximation.add(bestPlan); 127 | } 128 | /** 129 | * Initializes the population by randomly generated individuals. 130 | */ 131 | @Override 132 | protected void init(Query query, boolean[] consideredMetrics, 133 | PlanSpace planSpace, MultiCostModel costModel) { 134 | // We use an array list implementation as we have many random accesses 135 | population = new ArrayList(); 136 | // Generate the required number of individuals 137 | for (int individualCtr=0; individualCtr nrSelected[127]); 53 | assertTrue(nrSelected[1] > nrSelected[126]); 54 | } 55 | // Generation of individuals for next iteration 56 | { 57 | // If the population consists of identical individuals then the new individuals do, too 58 | for (int i=0; i<10; ++i) { 59 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 50, JoinType.MIN); 60 | SoqoGA ga = new SoqoGA(); 61 | ga.population = new ArrayList(); 62 | Individual templateIndividual = new Individual(query, planSpace, costModel); 63 | for (int j=0; j<128; ++j) { 64 | Individual individual = new Individual(query, planSpace, costModel, templateIndividual.genes); 65 | ga.population.add(individual); 66 | } 67 | List newIndividuals = ga.newIndividuals(); 68 | for (Individual newIndividual : newIndividuals) { 69 | assertArrayEquals(templateIndividual.genes, newIndividual.genes); 70 | assertArrayEquals(templateIndividual.genes, newIndividual.genes); 71 | } 72 | } 73 | } 74 | // Refining the approximation 75 | { 76 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CHAIN, 50, JoinType.MIN); 77 | SoqoGA ga = new SoqoGA(); 78 | ga.init(query, consideredMetrics, planSpace, costModel); 79 | ga.refineApproximation(query, consideredMetrics, planSpace, costModel, 0, 0, 0); 80 | // Make sure that new population has required number of individuals 81 | assertEquals(128, ga.population.size()); 82 | } 83 | } 84 | 85 | } 86 | -------------------------------------------------------------------------------- /src/optimizer/randomized/moqo/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/optimizer/randomized/moqo/.DS_Store -------------------------------------------------------------------------------- /src/optimizer/randomized/moqo/AnnealingPhaseSAH.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.moqo; 2 | 3 | import cost.CostModel; 4 | import cost.MultiCostModel; 5 | import plans.Plan; 6 | import plans.spaces.PlanSpace; 7 | import queries.Query; 8 | import util.LocalSearchUtil; 9 | 10 | /** 11 | * Implements a multi-objective generalization of the SAH variant of simulated annealing 12 | * that was described in the SIGMOD '88 paper "Optimization of large join queries" by Swami. 13 | * The calculation of the number of moves to try before reducing temperature differs however. 14 | * 15 | * @author immanueltrummer 16 | * 17 | */ 18 | public class AnnealingPhaseSAH extends Phase { 19 | /** 20 | * The default number of moves to try before giving up on improving a plan is scaled 21 | * by that factor. 22 | */ 23 | final int nrTriesScale; 24 | public AnnealingPhaseSAH(int nrTriesScale) { 25 | this.nrTriesScale = nrTriesScale; 26 | } 27 | /** 28 | * Influences the likelihood of moving towards dominated plans. 29 | */ 30 | double temperature = -1; 31 | /** 32 | * Cost distribution standard deviation - will be used to initialize and update temperature 33 | */ 34 | double costStDev = -1; 35 | /** 36 | * Determine initial temperature for simulated annealing based on the estimated 37 | * standard deviation of the cost distribution. 38 | * 39 | * @param query the query to optimize 40 | * @param consideredMetrics Boolean flags indicating for each metric if it is considered 41 | * @param planSpace determines the set of scan and join operators to consider 42 | * @param costModel estimates query plan cost according to multiple metrics 43 | */ 44 | void initializeTemperature( 45 | Query query, boolean[] consideredMetrics, PlanSpace planSpace, CostModel costModel) { 46 | costStDev = LocalSearchUtil.estimateCostStDev(query, consideredMetrics, planSpace, costModel); 47 | temperature = 20 * costStDev; 48 | } 49 | /** 50 | * Reduce the temperature to balance the tradeoff between exploration and exploitation. 51 | */ 52 | void updateTemperature() { 53 | assert(costStDev >= 0); 54 | assert(temperature >= 0.5); 55 | temperature = Math.max(0.5, Math.exp(-0.7*temperature/costStDev)); 56 | } 57 | /** 58 | * Initializes the current plan by a random plan and otherwise 59 | */ 60 | @Override 61 | public Plan nextPlan(Plan currentPlan) { 62 | // Generate random plan at first invocation 63 | if (currentPlan == null) { 64 | currentPlan = LocalSearchUtil.randomBushyPlan(query, planSpace); 65 | costModel.updateAll(currentPlan); 66 | } else { 67 | // Try a certain number of times to move to a different acceptable plan 68 | boolean currentPlanReplaced = false; 69 | // The number of tries corresponds by default to the number of predicates 70 | // which is linear in the number of tables for chain and star graphs. 71 | int nrTries = (query.nrTables - 1) * nrTriesScale; 72 | for (int tryCtr=0; tryCtr 0); 84 | boolean planInitialized = false; 85 | // Initialize current plan if necessary 86 | if (currentPlan == null) { 87 | // Plan needs to be initialized 88 | currentPlan = LocalSearchUtil.randomBushyPlan(query, planSpace); 89 | costModel.updateAll(currentPlan); 90 | planInitialized = true; 91 | } 92 | // Initialize temperature if necessary 93 | if (temperature < 1) { 94 | double avgCost = MathUtil.aggMean(currentPlan.getCostValuesCopy()); 95 | temperature = Math.max(initTemperatureScale * avgCost, 1); 96 | nrMovesWithCurrentTemperature = 0; 97 | } 98 | assert(temperature >= 1); 99 | // Return new plan if freshly initialized 100 | if (planInitialized) { 101 | return currentPlan; 102 | } 103 | // Try more random moves until either the temperature needs to be reduced 104 | // or an improvement was reached. 105 | while (nrMovesWithCurrentTemperature < chainLength) { 106 | Plan randomPlan = LocalSearchUtil.randomMove(query, currentPlan, planSpace, costModel); 107 | ++nrMovesWithCurrentTemperature; 108 | double[] curCost = currentPlan.getCostValuesCopy(); 109 | double[] newCost = randomPlan.getCostValuesCopy(); 110 | if (LocalSearchUtil.acceptMove( 111 | curCost, newCost, consideredMetrics, true, temperature)) { 112 | return randomPlan; 113 | } 114 | } 115 | // If we arrive here then the maximal number of moves for the current temperature 116 | // has been executed. 117 | temperature = 0.95 * temperature; 118 | nrMovesWithCurrentTemperature = 0; 119 | // Check if system is frozen and force re-initialization in that case 120 | if (temperature < 1) { 121 | return null; 122 | } 123 | return currentPlan; 124 | } 125 | @Override 126 | public String toString() { 127 | return "SAIO"; 128 | } 129 | } 130 | -------------------------------------------------------------------------------- /src/optimizer/randomized/moqo/ClimbingPhase.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.moqo; 2 | 3 | import plans.Plan; 4 | import util.LocalSearchUtil; 5 | 6 | /** 7 | * Implements hill climbing to improve query plans. Either tries all possible moves to find 8 | * a dominating plan or tries a fixed number of random moves before giving up. 9 | * 10 | * @author immanueltrummer 11 | * 12 | */ 13 | public class ClimbingPhase extends Phase { 14 | /** 15 | * Configures whether all possible moves are considered from each search node or only 16 | * a randomly selected subset. 17 | */ 18 | final boolean tryAllMoves; 19 | /** 20 | * The default number of moves to try before giving up on improving a plan is scaled by 21 | * this factor. 22 | */ 23 | final int nrTriesScale; 24 | 25 | public ClimbingPhase(boolean tryAllMoves, int nrTriesScale) { 26 | this.tryAllMoves = tryAllMoves; 27 | this.nrTriesScale = nrTriesScale; 28 | } 29 | /** 30 | * If no current plan is given then a random plan is generated, otherwise either all moves 31 | * or a fixed number of random moves are tried out from the current plan. We return the new 32 | * plan if an improvement was reached or a null pointer if no improvement was possible. 33 | */ 34 | @Override 35 | public Plan nextPlan(Plan currentPlan) { 36 | if (currentPlan == null) { 37 | // If no current plan then regenerate one 38 | Plan randomPlan = LocalSearchUtil.randomBushyPlan(query, planSpace); 39 | costModel.updateAll(randomPlan); 40 | return randomPlan; 41 | } else { 42 | // If we have current plan then try to improve it 43 | if (tryAllMoves) { 44 | // Exhaustively try all ways of mutating the plan 45 | return LocalSearchUtil.ParetoClimb( 46 | query, currentPlan, planSpace, costModel, consideredMetrics); 47 | /* 48 | return LocalSearchUtil.improvedPlan( 49 | query, currentPlan, consideredMetrics, 50 | planSpace, costModel, null); 51 | */ 52 | } else { 53 | // Try a certain number of random moves to improve the plan 54 | double[] curCost = currentPlan.getCostValuesCopy(); 55 | int nrTries = (query.nrTables - 1) * nrTriesScale; 56 | for (int tryCtr=0; tryCtr 14 | * Attention: need to pass different phase objects into each OnePhase object 15 | * since the phases might maintain internal state. 16 | * 17 | * @author immanueltrummer 18 | * 19 | */ 20 | public class OnePhase extends RandomizedOptimizer { 21 | final Phase phase; 22 | Plan currentPlan; 23 | 24 | public OnePhase(Phase phase) { 25 | this.phase = phase; 26 | } 27 | 28 | @Override 29 | protected void refineApproximation(Query query, 30 | boolean[] consideredMetrics, PlanSpace planSpace, 31 | MultiCostModel costModel, 32 | int algIndex, int sizeIndex, int queryIndex) { 33 | // Generate next plan and add to frontier if not null 34 | currentPlan = phase.nextPlan(currentPlan); 35 | if (currentPlan != null) { 36 | addToFrontier(query, currentPlan, consideredMetrics); 37 | } 38 | } 39 | 40 | @Override 41 | protected void init(Query query, boolean[] consideredMetrics, 42 | PlanSpace planSpace, MultiCostModel costModel) { 43 | currentPlan = null; 44 | phase.init(query, consideredMetrics, planSpace, costModel); 45 | } 46 | 47 | @Override 48 | public void cleanUp() { 49 | } 50 | 51 | @Override 52 | public String toString() { 53 | return "1P:" + phase.toString(); 54 | } 55 | /** 56 | * No algorithm specific features. 57 | */ 58 | @Override 59 | protected void storeSpecificStatistics(int algIndex, int sizeIndex, 60 | int queryIndex) { 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/optimizer/randomized/moqo/Phase.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.moqo; 2 | 3 | import cost.MultiCostModel; 4 | import plans.Plan; 5 | import plans.spaces.PlanSpace; 6 | import queries.Query; 7 | 8 | /** 9 | * Represents a phase within a randomized search algorithm. A randomized search algorithm 10 | * might have several phases that are executed consecutively. Each phases is initialized once 11 | * per query and invoked until a public flag indicates that the phase is terminated. While 12 | * executing, each phase generates a series of query plans and obtains the last generated 13 | * plan as input parameter. 14 | * 15 | * @author immanueltrummer 16 | * 17 | */ 18 | public abstract class Phase { 19 | protected Query query; 20 | protected boolean[] consideredMetrics; 21 | protected PlanSpace planSpace; 22 | protected MultiCostModel costModel; 23 | 24 | public void init(Query query, boolean[] consideredMetrics, 25 | PlanSpace planSpace, MultiCostModel costModel) { 26 | this.query = query; 27 | this.consideredMetrics = consideredMetrics; 28 | this.planSpace = planSpace; 29 | this.costModel = costModel; 30 | } 31 | public abstract Plan nextPlan(Plan currentPlan); 32 | } 33 | -------------------------------------------------------------------------------- /src/optimizer/randomized/moqo/TwoPhase.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.moqo; 2 | 3 | import cost.MultiCostModel; 4 | import optimizer.randomized.RandomizedOptimizer; 5 | import plans.Plan; 6 | import plans.spaces.PlanSpace; 7 | import queries.Query; 8 | 9 | /** 10 | * Generic local search algorithm with two search phases. The two search phases are specified 11 | * as parameters and might for instance include local search and simulated annealing. We switch 12 | * from the first to the second phase after a certain number of null pointers (semantic: no 13 | * suitable plans found in the neighborhood of current plan) has been returned by the first 14 | * phase. The number of null pointers before the switch is specified in the constructor. 15 | *

16 | * Attention: need to pass different phase objects into each TwoPhase object 17 | * since the phases might maintain internal state. 18 | * 19 | * @author immanueltrummer 20 | * 21 | */ 22 | public class TwoPhase extends RandomizedOptimizer { 23 | /** 24 | * How often the first phase can return a null pointer before switching to the second phase. 25 | */ 26 | final int nrRunsPhase1; 27 | final Phase phase1; 28 | final Phase phase2; 29 | /** 30 | * How often the first phase already returned a null pointer for the current query. 31 | */ 32 | int phase1RunCtr; 33 | Plan currentPlan; 34 | 35 | public TwoPhase(int nrRunsPhase1, Phase phase1, Phase phase2) { 36 | this.nrRunsPhase1 = nrRunsPhase1; 37 | this.phase1 = phase1; 38 | this.phase2 = phase2; 39 | } 40 | /** 41 | * Executes the first phase until the first phase generated a null pointer 42 | * a pre-specified number of times. Then it selects the first plan of the 43 | * frontier remaining after the first phase and uses it to initialize the 44 | * second phase. The second phase is executed until the timeout. 45 | * 46 | */ 47 | @Override 48 | protected void refineApproximation(Query query, 49 | boolean[] consideredMetrics, PlanSpace planSpace, 50 | MultiCostModel costModel, 51 | int algIndex, int sizeIndex, int queryIndex) { 52 | // Execute either phase 1 or phase 2 depending on run counter 53 | if (phase1RunCtr planList = Arrays.asList(new Plan[] {bestPlan}); 88 | return new ParetoPlanSet(planList); 89 | } catch (Exception e) { 90 | e.printStackTrace(System.out); 91 | return null; 92 | } 93 | } 94 | 95 | } 96 | -------------------------------------------------------------------------------- /src/optimizer/randomized/soqo/genetic/PlanFitness.java: -------------------------------------------------------------------------------- 1 | package optimizer.randomized.soqo.genetic; 2 | 3 | import cost.CostModel; 4 | import plans.JoinPlan; 5 | import plans.Plan; 6 | import plans.ScanPlan; 7 | import plans.operators.JoinOperator; 8 | import plans.operators.ScanOperator; 9 | import plans.spaces.PlanSpace; 10 | import queries.Query; 11 | 12 | import java.util.LinkedList; 13 | import java.util.List; 14 | 15 | import org.jgap.FitnessFunction; 16 | import org.jgap.IChromosome; 17 | 18 | /** 19 | * Calculates a fitness value based on the cost value in the first 20 | * cost metric realized by a query plan encoded as chromosome. 21 | * 22 | * @author immanueltrummer 23 | * 24 | */ 25 | public class PlanFitness extends FitnessFunction { 26 | /** 27 | * Used to verify class version. 28 | */ 29 | private static final long serialVersionUID = 1L; 30 | /** 31 | * The query that is currently being optimized. 32 | */ 33 | final Query query; 34 | /** 35 | * The number of tables in the query that is currently optimized. 36 | */ 37 | final int nrTables; 38 | /** 39 | * Left-deep plans answering the current query must perform that many joins. 40 | */ 41 | final int nrJoins; 42 | /** 43 | * We use default scan and join operators from that plan space to build the plan. 44 | */ 45 | final PlanSpace planSpace; 46 | /** 47 | * Default scan operator to use. 48 | */ 49 | final ScanOperator scanOperator; 50 | /** 51 | * Default join operator to use. 52 | */ 53 | final JoinOperator joinOperator; 54 | /** 55 | * Cost model used to calculate the cost of query plans. 56 | */ 57 | final CostModel costModel; 58 | 59 | public PlanFitness(Query query, PlanSpace planSpace, CostModel costModel) { 60 | this.query = query; 61 | this.nrTables = query.nrTables; 62 | this.nrJoins = nrTables - 1; 63 | this.planSpace = planSpace; 64 | this.scanOperator = planSpace.defaultScanOperator; 65 | this.joinOperator = planSpace.defaultJoinOperator; 66 | this.costModel = costModel; 67 | } 68 | /** 69 | * Extract the query plan that is encoded by the given chromosome and 70 | * calculate its cost. 71 | * 72 | * @param planChromosome a chromosome encoding a left-deep query plan 73 | * @return the query plan represented by the chromosome 74 | */ 75 | public Plan extractPlan(IChromosome planChromosome) { 76 | // Prepare list of tables that have not been used as join operands yet 77 | List remainingTables = new LinkedList(); 78 | for (int table=0; table plans; 21 | 22 | public ParetoPlanSet(List plans) { 23 | this.plans = plans; 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /src/plans/PathNode.java: -------------------------------------------------------------------------------- 1 | package plans; 2 | 3 | /** 4 | * Used to describes path from original plan root to some partial plan. 5 | * 6 | * @author immanueltrummer 7 | * 8 | */ 9 | public class PathNode { 10 | /** 11 | * The plan represented by the path node. 12 | */ 13 | public Plan plan; 14 | /** 15 | * The parent node on the path. 16 | */ 17 | public PathNode parent; 18 | /** 19 | * Whether this path node was the left child of its parent. 20 | */ 21 | public boolean isLeftChild; 22 | 23 | public PathNode(Plan plan, PathNode parent, boolean isLeftChild) { 24 | this.plan = plan; 25 | this.parent = parent; 26 | this.isLeftChild = isLeftChild; 27 | } 28 | /** 29 | * Checks whether this node represents the root (if parent is null) 30 | * 31 | * @return Boolean indicating whether this node is associated with the plan root 32 | */ 33 | public boolean isRoot() { 34 | return parent == null; 35 | } 36 | } -------------------------------------------------------------------------------- /src/plans/Plan.java: -------------------------------------------------------------------------------- 1 | package plans; 2 | 3 | import static common.Constants.*; 4 | 5 | import java.io.Serializable; 6 | import java.util.Arrays; 7 | 8 | import common.Constants; 9 | import relations.Relation; 10 | 11 | /** 12 | * Represents a query plan with associated cost vector. 13 | * 14 | * @author immanueltrummer 15 | * 16 | */ 17 | public abstract class Plan implements Serializable { 18 | /** 19 | * Used to check the class version. 20 | */ 21 | private static final long serialVersionUID = 1L; 22 | /** 23 | * Keeps track of the number of partial plans 24 | */ 25 | public static long nrPlansCreated = 0; 26 | /** 27 | * Whether cost, sub-plans, etc. can be changed 28 | */ 29 | protected boolean immutable = false; 30 | /** 31 | * Execution cost of plan - we often consider multiple plan cost metrics hence 32 | * plan cost is represented as a vector where different components represent 33 | * cost according to different metrics. 34 | */ 35 | public double[] cost = new double[NR_COST_METRICS]; 36 | /** 37 | * Cardinality of plan output 38 | */ 39 | public final double outputRows; 40 | /** 41 | * Number of disc pages consumed by the result generated by this plan 42 | */ 43 | public final double outputPages; 44 | /** 45 | * Whether the output of the plan is materialized (i.e., written to disc) 46 | */ 47 | public final boolean materializes; 48 | /** 49 | * The relation (i.e., a set of joined tables) produced by this plan - 50 | * this field is not used by all optimizers and may be set to null. 51 | */ 52 | public final Relation resultRel; 53 | /** 54 | * The tree height of the query plan. 55 | */ 56 | public final int height; 57 | /** 58 | * This constructor is used if no result relations are used. 59 | * 60 | * @param outputRows cardinality of relation produced by this plan 61 | * @param outputPages the number of disc pages consumed by the plan result 62 | * @param materializes whether the result of the plan is written to disc 63 | * @param height the height of the plan tree 64 | */ 65 | public Plan(double outputRows, double outputPages, boolean materializes, int height) { 66 | this.outputRows = outputRows; 67 | this.outputPages = outputPages; 68 | this.materializes = materializes; 69 | this.height = height; 70 | this.resultRel = null; 71 | ++nrPlansCreated; 72 | } 73 | /** 74 | * This constructor automatically sets the result relation and extracts required fields. 75 | * 76 | * @param resultRel the result relation created by this plan 77 | * @param materializes whether the result relation is written to disc 78 | * @param height the height of the query plan tree 79 | */ 80 | public Plan(Relation resultRel, boolean materializes, int height) { 81 | this.outputRows = resultRel.cardinality; 82 | this.outputPages = resultRel.pages; 83 | this.materializes = materializes; 84 | this.height = height; 85 | this.resultRel = resultRel; 86 | ++nrPlansCreated; 87 | } 88 | /** 89 | * Makes the plan (its cost, operators, sub-plans) immutable 90 | */ 91 | public abstract void makeImmutable(); 92 | /** 93 | * Sets a cost value for one specific cost metric - this is only possible if the plan is mutable. 94 | * 95 | * @param metric index of the cost metric whose value we want to change 96 | * @param costValue the new cost estimate 97 | */ 98 | public void setCostValue(int metric, double costValue) { 99 | assert(!immutable); 100 | cost[metric] = costValue; 101 | } 102 | /** 103 | * Sets cost values for all plan cost metrics at once - this is only possible if the plan is mutable. 104 | * 105 | * @param newCost the new cost value 106 | */ 107 | public void setCostValues(double[] newCost) { 108 | assert(!immutable); 109 | for (int metricCtr=0; metricCtr"; 26 | } 27 | @Override 28 | public BNLjoin deepCopy() { 29 | return new BNLjoin(buffer, materializeResult); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/plans/operators/local/HashJoin.java: -------------------------------------------------------------------------------- 1 | package plans.operators.local; 2 | 3 | /** 4 | * Represents the hash join operator implementation: this implementation builds a hash 5 | * table on one of the join input operands and uses it to quickly identify matching 6 | * partners for the tuples from the other join operand. 7 | * 8 | * @author immanueltrummer 9 | * 10 | */ 11 | public class HashJoin extends LocalJoin { 12 | /** 13 | * Used to verify the class version. 14 | */ 15 | private static final long serialVersionUID = 1L; 16 | 17 | public HashJoin(double buffer, boolean materializeResult) { 18 | super(buffer, materializeResult); 19 | } 20 | @Override 21 | public String toString() { 22 | return "HSJ<" + buffer + "," + materializeResult + ">"; 23 | } 24 | @Override 25 | public HashJoin deepCopy() { 26 | return new HashJoin(buffer, materializeResult); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/plans/operators/local/LocalJoin.java: -------------------------------------------------------------------------------- 1 | package plans.operators.local; 2 | 3 | import plans.operators.JoinOperator; 4 | 5 | /** 6 | * Common super class of all local (i.e., those joins operators are meant for execution 7 | * on a single node) join operator implementations. 8 | * 9 | * @author immanueltrummer 10 | * 11 | */ 12 | public abstract class LocalJoin extends JoinOperator { 13 | /** 14 | * Used to verify the class version. 15 | */ 16 | private static final long serialVersionUID = 1L; 17 | /** 18 | * The amount of buffer space reserved for the operator, measured in bytes. 19 | */ 20 | public final double buffer; 21 | 22 | public LocalJoin(double buffer, boolean materializeResult) { 23 | super(materializeResult); 24 | assert(buffer>=0); 25 | this.buffer = buffer; 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/plans/operators/local/LocalScan.java: -------------------------------------------------------------------------------- 1 | package plans.operators.local; 2 | 3 | import plans.operators.ScanOperator; 4 | 5 | /** 6 | * Common super class of all local (i.e., executed on a single node) scan operators. 7 | * 8 | * @author immanueltrummer 9 | * 10 | */ 11 | public class LocalScan extends ScanOperator{ 12 | /** 13 | * Used to verify the class version. 14 | */ 15 | private static final long serialVersionUID = 1L; 16 | 17 | @Override 18 | public String toString() { 19 | return "SCAN"; 20 | } 21 | 22 | @Override 23 | public LocalScan deepCopy() { 24 | return new LocalScan(); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/plans/operators/local/SortMergeJoin.java: -------------------------------------------------------------------------------- 1 | package plans.operators.local; 2 | 3 | /** 4 | * Represents sort-merge join operator implementation: the sort-merge join operator sorts 5 | * its inputs on the join column (we assume an equality join) and merges the sorted inputs. 6 | * 7 | * @author immanueltrummer 8 | * 9 | */ 10 | public class SortMergeJoin extends LocalJoin { 11 | /** 12 | * Used to verify the class version. 13 | */ 14 | private static final long serialVersionUID = 1L; 15 | 16 | public SortMergeJoin(double buffer, boolean materializeResult) { 17 | super(buffer, materializeResult); 18 | } 19 | @Override 20 | public String toString() { 21 | return "SMJ<" + buffer + "," + materializeResult + ">"; 22 | } 23 | @Override 24 | public SortMergeJoin deepCopy() { 25 | return new SortMergeJoin(buffer, materializeResult); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/plans/spaces/.DS_Store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itrummer/query-optimizer-lib/f560e517d41e6b6de1541e748fe6ea281c483d31/src/plans/spaces/.DS_Store -------------------------------------------------------------------------------- /src/plans/spaces/LocalPlanSpace.java: -------------------------------------------------------------------------------- 1 | package plans.spaces; 2 | 3 | import java.util.LinkedList; 4 | 5 | import plans.Plan; 6 | import plans.operators.JoinOperator; 7 | import plans.operators.ScanOperator; 8 | import plans.operators.local.BNLjoin; 9 | import plans.operators.local.HashJoin; 10 | import plans.operators.local.LocalScan; 11 | import plans.operators.local.SortMergeJoin; 12 | import relations.Relation; 13 | 14 | /** 15 | * This plan space describes a single-node execution scenario in which different configurations 16 | * of the standard join operators (block-nested loop, hash join, sort-merge join) are applicable. 17 | * 18 | * @author immanueltrummer 19 | * 20 | */ 21 | public class LocalPlanSpace extends PlanSpace { 22 | /** 23 | * Used to verify the class version. 24 | */ 25 | private static final long serialVersionUID = 1L; 26 | /** 27 | * The non-parameterized constructor generates the standard variant of this 28 | * plan space which is appropriate for multi-objective query optimization 29 | * scenarios. 30 | */ 31 | public LocalPlanSpace() { 32 | this(LocalSpaceVariant.MOQO); 33 | } 34 | /** 35 | * The parameterized variant of the constructor allows to explicitly choose 36 | * the plan space variant. 37 | * 38 | * @param variant the plan space variant determining the available join operators 39 | */ 40 | public LocalPlanSpace(LocalSpaceVariant variant) { 41 | // initialize list of scan and join operators 42 | consideredScanOps = new LinkedList(); 43 | consideredJoinOps = new LinkedList(); 44 | // scan operators 45 | consideredScanOps.add(new LocalScan()); 46 | // join operators 47 | switch (variant) { 48 | case MOQO: 49 | { 50 | for (Boolean materializes : new Boolean[]{false, true}) { 51 | for (Integer buffer : new Integer[]{10, 100, 1000, 10000, 100000}) { 52 | consideredJoinOps.add(new BNLjoin(buffer, materializes)); 53 | consideredJoinOps.add(new HashJoin(buffer, materializes)); 54 | consideredJoinOps.add(new SortMergeJoin(buffer, materializes)); 55 | } 56 | } 57 | } 58 | break; 59 | case SOQO: 60 | { 61 | int buffer = 10000; 62 | boolean materializes = true; 63 | consideredJoinOps.add(new BNLjoin(buffer, materializes)); 64 | consideredJoinOps.add(new HashJoin(buffer, materializes)); 65 | consideredJoinOps.add(new SortMergeJoin(buffer, materializes)); 66 | } 67 | break; 68 | case SIMPLE: 69 | { 70 | consideredJoinOps.add(new BNLjoin(10000, true)); 71 | } 72 | break; 73 | default: 74 | assert(false) : "Unknown local plan space variant!"; 75 | } 76 | // default scan and join operators 77 | defaultScanOperator = new LocalScan(); 78 | //defaultJoinOperator = new BNLjoin(100, true); 79 | defaultJoinOperator = new BNLjoin(10000, true); 80 | } 81 | /** 82 | * The local scan operator is applicable to each relation. 83 | */ 84 | @Override 85 | public boolean scanOperatorApplicable(ScanOperator scanOperator, Relation relation) { 86 | return true; // changes once we consider indices 87 | } 88 | /** 89 | * Only the block-nested loop operator allows pipelining while the others require 90 | * materialized input (we use the same assumptions as Steinbrunn et al., VLDB 1997). 91 | */ 92 | @Override 93 | public boolean joinOperatorApplicable(JoinOperator joinOperator, Plan leftPlan, Plan rightPlan) { 94 | if (joinOperator instanceof BNLjoin) { 95 | return true; 96 | } else if (joinOperator instanceof HashJoin) { 97 | return leftPlan.materializes && rightPlan.materializes; 98 | } else { 99 | assert(joinOperator instanceof SortMergeJoin); 100 | return leftPlan.materializes && rightPlan.materializes; 101 | } 102 | } 103 | /** 104 | * The output of the local scan operator is suitable as input to any join operator. 105 | */ 106 | @Override 107 | public boolean scanOutputCompatible(ScanOperator scanOperator, 108 | JoinOperator nextJoinOperator) { 109 | return true; 110 | } 111 | /** 112 | * Hash join and merge join require materialized relations as output while the 113 | * block-nested-loop join can be used for pipelining. 114 | */ 115 | @Override 116 | public boolean joinOutputComptatible(JoinOperator joinOperator, 117 | JoinOperator nextJoinOperator) { 118 | if (nextJoinOperator instanceof HashJoin || nextJoinOperator instanceof SortMergeJoin) { 119 | return joinOperator.materializeResult; 120 | } else { 121 | return true; 122 | } 123 | } 124 | @Override 125 | public String toString() { 126 | return "Local Processing Plan Space"; 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /src/plans/spaces/LocalPlanSpaceTest.java: -------------------------------------------------------------------------------- 1 | package plans.spaces; 2 | 3 | import static org.junit.Assert.*; 4 | import plans.JoinPlan; 5 | import plans.Plan; 6 | import plans.ScanPlan; 7 | import plans.operators.JoinOperator; 8 | import plans.operators.ScanOperator; 9 | import plans.operators.local.BNLjoin; 10 | import plans.operators.local.HashJoin; 11 | import plans.operators.local.LocalScan; 12 | import plans.operators.local.SortMergeJoin; 13 | import queries.JoinGraphType; 14 | import queries.JoinType; 15 | import queries.Query; 16 | import queries.QueryFactory; 17 | import relations.Relation; 18 | import relations.RelationFactory; 19 | 20 | import org.junit.Test; 21 | 22 | public class LocalPlanSpaceTest { 23 | 24 | @Test 25 | public void test() { 26 | PlanSpace localPlanSpace = new LocalPlanSpace(); 27 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.STAR, 3, JoinType.MN); 28 | Relation rel0 = RelationFactory.createSingleTableRel(query, 0); 29 | ScanOperator scanOp = new LocalScan(); 30 | JoinOperator materializingBNL = new BNLjoin(100, true); 31 | JoinOperator pipeliningBNL = new BNLjoin(100, false); 32 | JoinOperator sortMerge = new SortMergeJoin(100, true); 33 | JoinOperator hashJoin = new HashJoin(50, false); 34 | Plan scan0 = new ScanPlan(query, 0, scanOp); 35 | Plan scan1 = new ScanPlan(query, 1, scanOp); 36 | Plan scan2 = new ScanPlan(query, 2, scanOp); 37 | Plan join01pipelined = new JoinPlan(query, scan0, scan1, pipeliningBNL); 38 | Plan join01materialized = new JoinPlan(query, scan0, scan1, materializingBNL); 39 | // Test operator applicability (depending on input properties) 40 | { 41 | assertTrue(localPlanSpace.scanOperatorApplicable(scanOp, rel0)); 42 | assertTrue(localPlanSpace.joinOperatorApplicable(materializingBNL, join01pipelined, scan2)); 43 | assertTrue(localPlanSpace.joinOperatorApplicable(pipeliningBNL, join01pipelined, scan2)); 44 | assertFalse(localPlanSpace.joinOperatorApplicable(sortMerge, join01pipelined, scan2)); 45 | assertFalse(localPlanSpace.joinOperatorApplicable(hashJoin, join01pipelined, scan2)); 46 | assertTrue(localPlanSpace.joinOperatorApplicable(sortMerge, join01materialized, scan2)); 47 | assertTrue(localPlanSpace.joinOperatorApplicable(hashJoin, join01materialized, scan2)); 48 | } 49 | // Test operator compatibility (with required output properties) 50 | { 51 | assertTrue(localPlanSpace.scanOutputCompatible(scanOp, hashJoin)); 52 | // All pairs of BNL operators are compatible 53 | assertTrue(localPlanSpace.joinOutputComptatible(pipeliningBNL, pipeliningBNL)); 54 | assertTrue(localPlanSpace.joinOutputComptatible(materializingBNL, pipeliningBNL)); 55 | assertTrue(localPlanSpace.joinOutputComptatible(materializingBNL, materializingBNL)); 56 | assertTrue(localPlanSpace.joinOutputComptatible(pipeliningBNL, materializingBNL)); 57 | // Hash and sort merge join require materialized input 58 | assertFalse(localPlanSpace.joinOutputComptatible(pipeliningBNL, hashJoin)); 59 | assertFalse(localPlanSpace.joinOutputComptatible(pipeliningBNL, sortMerge)); 60 | assertTrue(localPlanSpace.joinOutputComptatible(materializingBNL, hashJoin)); 61 | assertTrue(localPlanSpace.joinOutputComptatible(materializingBNL, sortMerge)); 62 | } 63 | } 64 | 65 | } 66 | -------------------------------------------------------------------------------- /src/plans/spaces/LocalSpaceVariant.java: -------------------------------------------------------------------------------- 1 | package plans.spaces; 2 | 3 | /** 4 | * Different variants of the local plan space are available that differ by the 5 | * set of considered operators: 6 | * - MOQO features different versions of each join operator that use different amount of buffer space 7 | * - SOQO features the three standard join operators but with a fixed amount of buffer space 8 | * - SIMPLE features only one standard join operator 9 | * The default variant is MOQO. 10 | * 11 | * @author immanueltrummer 12 | * 13 | */ 14 | public enum LocalSpaceVariant { 15 | MOQO, SOQO, SIMPLE 16 | } 17 | -------------------------------------------------------------------------------- /src/queries/JoinGraphType.java: -------------------------------------------------------------------------------- 1 | package queries; 2 | 3 | /** 4 | * Join graph structure to generate by query factory. The names refer to the ones 5 | * presented in "Heuristic and randomized optimization for the join ordering problem" 6 | * by Steinbrunn et al. 7 | * CHAIN: all tables are connected by a chain of join predicates; 8 | * each table except the two ends of the chain are connected to two other tables 9 | * CYCLE: all tables are connected by a series of join predicates forming a cycle; 10 | * each table is connected to two other tables 11 | * STAR: one table is connected via one join predicate to each of the other tables 12 | * 13 | * @author immanueltrummer 14 | * 15 | */ 16 | public enum JoinGraphType { 17 | CHAIN, CYCLE, STAR 18 | } 19 | -------------------------------------------------------------------------------- /src/queries/JoinType.java: -------------------------------------------------------------------------------- 1 | package queries; 2 | 3 | /** 4 | * The join type decides how the selectivity is randomly chosen. 5 | * RANDOM selectivity is chosen randomly with uniform distribution 6 | * MN selectivity is chosen assuming an equality join 7 | * MIN selectivity is chosen assuming foreign key constraint from smaller to larger table 8 | * MAX selectivity is chosen assuming foreign key constraint from larger to smaller table 9 | * MINMAX selectivity is chosen somewhere in between the MIN and the MAX selectivity 10 | * 11 | * @author itrummer 12 | * 13 | */ 14 | public enum JoinType { 15 | RANDOM, MN, MIN, MAX, MINMAX 16 | } 17 | -------------------------------------------------------------------------------- /src/queries/Query.java: -------------------------------------------------------------------------------- 1 | package queries; 2 | 3 | import cost.CostModel; 4 | import plans.Plan; 5 | import plans.ScanPlan; 6 | import plans.operators.ScanOperator; 7 | import plans.spaces.PlanSpace; 8 | 9 | import java.io.Serializable; 10 | import java.util.Arrays; 11 | import java.util.LinkedList; 12 | import java.util.List; 13 | 14 | /** 15 | * Represents a join query where the goal is to join all tables while applying join predicates 16 | * as early as possible. A query is characterized by the table cardinalities, the join graph 17 | * structure, and by the selectivity values of the predicates. 18 | * 19 | * @author immanueltrummer 20 | * 21 | */ 22 | public class Query implements Serializable { 23 | /** 24 | * Used to verify the class version. 25 | */ 26 | private static final long serialVersionUID = 1L; 27 | /** 28 | * The number of tables that need to be joined. 29 | */ 30 | public final int nrTables; 31 | /** 32 | * The cardinality (number of rows) for each base table. 33 | */ 34 | public final double[] tableCardinalities; 35 | /** 36 | * The selectivity between table pairs, if no join predicate is defined between 37 | * two specific tables then the selectivity value must be one. Note that the 38 | * selectivity matrix must be symmetric. 39 | */ 40 | public final double[][] selectivities; 41 | 42 | public Query(int nrTables, double[] tableCardinalities, double[][] selectivities) { 43 | assert(tableCardinalities.length == nrTables); 44 | assert(selectivities.length == nrTables); 45 | assert(selectivities[0].length == nrTables); 46 | this.nrTables = nrTables; 47 | this.tableCardinalities = tableCardinalities; 48 | this.selectivities = selectivities; 49 | } 50 | @Override 51 | public String toString() { 52 | String output = "Cardinalities:"; 53 | for (int i=0; i allScanPlans(PlanSpace planSpace, CostModel costModel) { 72 | List allScanPlans = new LinkedList(); 73 | ScanOperator scanOperator = planSpace.defaultScanOperator; 74 | for (int tableIndex=0; tableIndex= 1 && cardinality <= 100000); 23 | } 24 | // Verify that selectivities are lower than one only for admissible index pairs 25 | for (int table1=0; table1= 1 && cardinality <= 100000); 44 | } 45 | // Verify that selectivities are lower than one only for admissible index pairs 46 | for (int table1=0; table1 table1 || table2 == 0 && table1 > table2)) { 49 | assertEquals(1, query.selectivities[table1][table2], EPSILON); 50 | } 51 | } 52 | } 53 | } 54 | } 55 | // Generation of cycle queries 56 | { 57 | for (int i=0; i<50; ++i) { 58 | int nrTables = random.nextInt(10) + 2; 59 | Query query = QueryFactory.produceSteinbrunn(JoinGraphType.CYCLE, nrTables, JoinType.MINMAX); 60 | assertEquals(nrTables, query.nrTables); 61 | // Verify that cardinality is within admissible range 62 | for (int table=0; table= 1 && cardinality <= 100000); 65 | } 66 | // Verify that selectivities are lower than one only for admissible index pairs 67 | for (int table1=0; table1 allScanPlans = query.allScanPlans(planSpace, timeCostModel); 20 | assertEquals(10, allScanPlans.size()); 21 | // Create expected set of scanned table indices 22 | BitSet expectedScannedTables = new BitSet(); 23 | for (int tableIndex=0; tableIndex<10; ++tableIndex) { 24 | expectedScannedTables.set(tableIndex); 25 | } 26 | // Create set of actually scanned table indices 27 | BitSet scannedTables = new BitSet(); 28 | for (Plan plan : allScanPlans) { 29 | ScanPlan scanPlan = (ScanPlan)plan; 30 | scannedTables.set(scanPlan.tableIndex); 31 | } 32 | // Compare both sets 33 | assertEquals(expectedScannedTables, scannedTables); 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /src/relations/Relation.java: -------------------------------------------------------------------------------- 1 | package relations; 2 | 3 | import java.io.Serializable; 4 | import java.util.BitSet; 5 | import java.util.List; 6 | 7 | import common.Constants; 8 | import plans.Plan; 9 | 10 | /** 11 | * Represents a base table or a join between several tables. 12 | * 13 | * @author immanueltrummer 14 | * 15 | */ 16 | public class Relation implements Serializable { 17 | /** 18 | * Used to check the class version. 19 | */ 20 | private static final long serialVersionUID = 1L; 21 | /** 22 | * Indices of the tables that are joined in this relation. 23 | */ 24 | public final BitSet tableSet; 25 | /** 26 | * The number of rows of this relation. 27 | */ 28 | public final double cardinality; 29 | /** 30 | * The number of disc pages consumed by this relation. 31 | */ 32 | public final double pages; 33 | /** 34 | * The Pareto-optimal plans for creating this relation - this field 35 | * is not used by all optimization algorithms. 36 | */ 37 | public List ParetoPlans; 38 | /** 39 | * How often was this relation considered? This field is used by algorithms that 40 | * associate relations with arms in a multi-armed bandit scenario. 41 | */ 42 | public long nrRounds = 1; 43 | /** 44 | * How often was this relation selected? This field is used by algorithms that 45 | * associate relations with arms in a multi-armed bandit scenario. 46 | */ 47 | public long nrPlayed = 1; 48 | /** 49 | * What was the accumulated reward acquired by selecting this relation? This field is used by 50 | * algorithms that associate relations with arms in a multi-armed bandit scenario. 51 | */ 52 | public double accumulatedReward; 53 | /** 54 | * What is the UCB value of this relation derived from the number of times it was considered and 55 | * played and from the accumulated reward achieved by "playing" this relation. 56 | */ 57 | public double UCBvalue; 58 | 59 | public Relation(BitSet tableSet, double cardinality) { 60 | this.tableSet = tableSet; 61 | this.cardinality = cardinality; 62 | this.pages = Math.ceil(cardinality * Constants.BYTES_PER_TUPLE/Constants.BYTES_PER_PAGE); 63 | } 64 | /** 65 | * Obtains index of first table joined in this relation. 66 | * 67 | * @return the index of a table that is joined by this relation 68 | */ 69 | public int firstTableIndex() { 70 | return tableSet.nextSetBit(0); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/relations/RelationFactory.java: -------------------------------------------------------------------------------- 1 | package relations; 2 | 3 | import java.util.BitSet; 4 | import queries.Query; 5 | 6 | /** 7 | * Produces relation objects (i.e., calculates among others the cardinality, byte size etc. 8 | * of the new relations). 9 | * 10 | * @author immanueltrummer 11 | * 12 | */ 13 | public class RelationFactory { 14 | /** 15 | * Creates a relation representing only a single table. 16 | * 17 | * @param query the query to which the table belongs 18 | * @param tableIndex the index of the table that shall be represented 19 | * @return a relation representing the specified table 20 | */ 21 | public static Relation createSingleTableRel(Query query, int tableIndex) { 22 | // create table set 23 | BitSet tableSet = new BitSet(); 24 | tableSet.set(tableIndex); 25 | // get cardinality 26 | double cardinality = query.tableCardinalities[tableIndex]; 27 | // create new relation 28 | return new Relation(tableSet, cardinality); 29 | } 30 | /** 31 | * Creates a new relation by joining two existing relations. 32 | * 33 | * @param query the query to which all joined tables belong 34 | * @param rel1 the first relation to be joined 35 | * @param rel2 the second relation to be joined 36 | * @return a new relation representing the join of the two given relations 37 | */ 38 | public static Relation createJoinRel(Query query, Relation rel1, Relation rel2) { 39 | // verify that the joined relations have no tables in common 40 | assert(!rel1.tableSet.intersects(rel2.tableSet)) : "rel1: " + rel1.tableSet + "; rel2: " + rel2.tableSet; 41 | // create result table set 42 | BitSet resultSet = new BitSet(); 43 | resultSet.or(rel1.tableSet); 44 | resultSet.or(rel2.tableSet); 45 | // calculate result relation cardinality 46 | double resultCardinality = 1.0; 47 | resultCardinality *= rel1.cardinality; 48 | resultCardinality *= rel2.cardinality; 49 | // take into account applicable join predicates 50 | BitSet tableSet1 = rel1.tableSet; 51 | BitSet tableSet2 = rel2.tableSet; 52 | for (int tableIndex1 = tableSet1.nextSetBit(0); tableIndex1 >= 0; tableIndex1 = tableSet1.nextSetBit(tableIndex1+1)) { 53 | for (int tableIndex2 = tableSet2.nextSetBit(0); tableIndex2 >= 0; tableIndex2 = tableSet2.nextSetBit(tableIndex2+1)) { 54 | resultCardinality *= query.selectivities[tableIndex1][tableIndex2]; 55 | } 56 | } 57 | // create and return new relation 58 | return new Relation(resultSet, resultCardinality); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/relations/RelationTest.java: -------------------------------------------------------------------------------- 1 | package relations; 2 | 3 | import static util.TestUtil.*; 4 | import static org.junit.Assert.*; 5 | import queries.Query; 6 | import util.TestUtil; 7 | 8 | import org.junit.Test; 9 | 10 | 11 | public class RelationTest { 12 | 13 | @Test 14 | public void test() { 15 | // create query 16 | double[] cardinalitites = new double[] {10, 100, 500}; 17 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 18 | TestUtil.setSelectivity(selectivities, 0, 1, 0.5); 19 | TestUtil.setSelectivity(selectivities, 1, 2, 0.1); 20 | Query query = new Query(3, cardinalitites, selectivities); 21 | // create relations 22 | Relation rel100 = RelationFactory.createSingleTableRel(query, 0); 23 | Relation rel010 = RelationFactory.createSingleTableRel(query, 1); 24 | Relation rel001 = RelationFactory.createSingleTableRel(query, 2); 25 | Relation rel110 = RelationFactory.createJoinRel(query, rel100, rel010); 26 | Relation rel111 = RelationFactory.createJoinRel(query, rel110, rel001); 27 | // test relations 28 | assertEquals(rel100.cardinality, 10, EPSILON); 29 | assertEquals(rel010.cardinality, 100, EPSILON); 30 | assertEquals(rel110.cardinality, 500, EPSILON); 31 | assertTrue(rel110.tableSet.get(0)); 32 | assertTrue(rel110.tableSet.get(1)); 33 | assertFalse(rel110.tableSet.get(2)); 34 | assertEquals(rel111.cardinality, 25000, EPSILON); 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/util/GreedyCriterion.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | /** 4 | * Criteria for greedily selecting the next join to perform. Alternatives are 5 | * - MIN_SIZE: always select the join that leads to the intermediate result with minimal size 6 | * - MIN_SELECTIVITY: always select the join with the most selective join predicate 7 | * 8 | * @author immanueltrummer 9 | * 10 | */ 11 | public enum GreedyCriterion { 12 | MIN_SIZE, MIN_SELECTIVITY 13 | } 14 | -------------------------------------------------------------------------------- /src/util/GreedyUtil.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import java.util.Collections; 4 | import java.util.LinkedList; 5 | import java.util.List; 6 | 7 | import cost.MultiCostModel; 8 | import plans.JoinPlan; 9 | import plans.Plan; 10 | import plans.ScanPlan; 11 | import plans.operators.JoinOperator; 12 | import plans.spaces.PlanSpace; 13 | import queries.Query; 14 | 15 | /** 16 | * Contains several methods that are helpful for greedy approaches that build a 17 | * query plan greedily according to diverse criteria. 18 | * 19 | * @author immanueltrummer 20 | * 21 | */ 22 | public class GreedyUtil { 23 | /** 24 | * Checks whether one join plan is better than another one according to some simple 25 | * greedy criterion that is used to select the next join. Assumes that both plans 26 | * are no null pointers and does not check for that case. 27 | * 28 | * @param better the plan that is presumably better 29 | * @param worse the plan that is presumably worse 30 | * @param criterion the criterion according to which the plans are judged 31 | * @return true if the better plan is indeed better according to the given criterion 32 | */ 33 | static boolean better(JoinPlan better, JoinPlan worse, GreedyCriterion criterion) { 34 | switch (criterion) { 35 | case MIN_SIZE: 36 | return better.resultRel.pages * 10 < worse.resultRel.pages; 37 | case MIN_SELECTIVITY: 38 | double betterSelectivity = better.resultRel.cardinality/ 39 | (better.getLeftPlan().resultRel.cardinality * better.getRightPlan().resultRel.cardinality); 40 | double worseSelectivity = worse.resultRel.cardinality/ 41 | (worse.getLeftPlan().resultRel.cardinality * worse.getRightPlan().resultRel.cardinality); 42 | return betterSelectivity < worseSelectivity; 43 | default: 44 | assert(false); 45 | return false; 46 | } 47 | } 48 | /** 49 | * Determines the join between two partial plans optimizing a simple criterion such 50 | * as minimizing the intermediate result size. Adds the corresponding plan to the 51 | * partial plans and removes its sub-plans. 52 | * 53 | * @param query the query being optimized 54 | * @param partialPlans a set of partial query plans 55 | * @param planSpace the plan space determining the applicable operators 56 | * @param costModel used to estimate the cost of query plans according to multiple metrics 57 | * @param criterion the criterion according to which the join is selected 58 | */ 59 | static void performGreedyJoin(Query query, List partialPlans, 60 | PlanSpace planSpace, MultiCostModel costModel, GreedyCriterion criterion) { 61 | Collections.shuffle(partialPlans); 62 | // Determine join leading to an intermediate result of minimal size. 63 | // This is the best possible join according to the minSize heuristic. 64 | JoinPlan bestJoin = null; 65 | for (Plan leftPlan : partialPlans) { 66 | for (Plan rightPlan : partialPlans) { 67 | if (leftPlan != rightPlan) { 68 | JoinOperator joinOperator = planSpace.randomJoinOperator(leftPlan, rightPlan); 69 | JoinPlan joinPlan = new JoinPlan(query, leftPlan, rightPlan, joinOperator); 70 | if (bestJoin == null) { 71 | bestJoin = joinPlan; 72 | } else if (better(joinPlan, bestJoin, criterion)) { 73 | bestJoin = joinPlan; 74 | } 75 | } 76 | } 77 | } 78 | // Calculate the cost of best plan 79 | costModel.updateRoot(bestJoin); 80 | // Insert best join plan and remove its sub-plans from partial plans list 81 | partialPlans.remove(bestJoin.getLeftPlan()); 82 | partialPlans.remove(bestJoin.getRightPlan()); 83 | partialPlans.add(bestJoin); 84 | } 85 | /** 86 | * Creates a query plan by selecting joins greedily according to some simple criterion. 87 | * 88 | * @param query the query for which a plan is created 89 | * @param planSpace determines the set of applicable scan and join operators 90 | * @param costModel used to calculate cost of query plans according to different metrics 91 | * @param criterion the criterion according to which joins are selected 92 | * @return a greedily constructed query plan with its cost calculated 93 | */ 94 | public static Plan greedyPlan(Query query, PlanSpace planSpace, 95 | MultiCostModel costModel, GreedyCriterion criterion) { 96 | int nrTables = query.nrTables; 97 | // Generate scan plans for all query tables 98 | List partialPlans = new LinkedList(); 99 | for (int tableIndex=0; tableIndex partialPlans, GreedyCriterion criterion) { 122 | // Combine partial plans until we obtain a completed plan 123 | while (partialPlans.size() > 1) { 124 | performGreedyJoin(query, partialPlans, planSpace, costModel, criterion); 125 | } 126 | // Return completed plan 127 | return partialPlans.iterator().next(); 128 | } 129 | } 130 | -------------------------------------------------------------------------------- /src/util/GreedyUtilTest.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static org.junit.Assert.*; 4 | import static util.TestUtil.*; 5 | 6 | import java.util.BitSet; 7 | import java.util.List; 8 | 9 | import plans.JoinPlan; 10 | import plans.Plan; 11 | import plans.ScanPlan; 12 | import queries.Query; 13 | 14 | import org.junit.Test; 15 | 16 | public class GreedyUtilTest { 17 | 18 | @Test 19 | public void test() { 20 | // Deciding which of two plans is better 21 | { 22 | // Create query 23 | double[] tableCardinalities = new double[] {100, 10000, 100000}; 24 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 25 | TestUtil.setSelectivity(selectivities, 1, 2, 0.1); 26 | Query query = new Query(3, tableCardinalities, selectivities); 27 | // Create scan plans 28 | ScanPlan scan0 = new ScanPlan(query, 0, planSpace.defaultScanOperator); 29 | ScanPlan scan1 = new ScanPlan(query, 1, planSpace.defaultScanOperator); 30 | ScanPlan scan2 = new ScanPlan(query, 2, planSpace.defaultScanOperator); 31 | // Create join plans 32 | JoinPlan join01 = new JoinPlan(query, scan0, scan1, planSpace.defaultJoinOperator); 33 | JoinPlan join10 = new JoinPlan(query, scan1, scan0, planSpace.defaultJoinOperator); 34 | JoinPlan join12 = new JoinPlan(query, scan1, scan2, planSpace.defaultJoinOperator); 35 | JoinPlan join21 = new JoinPlan(query, scan2, scan1, planSpace.defaultJoinOperator); 36 | // Minimum selectivity criterion 37 | { 38 | // The join between the second and third table leads to minimal selectivity 39 | assertTrue(GreedyUtil.better(join12, join01, GreedyCriterion.MIN_SELECTIVITY)); 40 | assertTrue(GreedyUtil.better(join12, join10, GreedyCriterion.MIN_SELECTIVITY)); 41 | assertTrue(GreedyUtil.better(join21, join01, GreedyCriterion.MIN_SELECTIVITY)); 42 | assertTrue(GreedyUtil.better(join21, join10, GreedyCriterion.MIN_SELECTIVITY)); 43 | 44 | assertFalse(GreedyUtil.better(join01, join12, GreedyCriterion.MIN_SELECTIVITY)); 45 | assertFalse(GreedyUtil.better(join10, join12, GreedyCriterion.MIN_SELECTIVITY)); 46 | assertFalse(GreedyUtil.better(join01, join21, GreedyCriterion.MIN_SELECTIVITY)); 47 | assertFalse(GreedyUtil.better(join10, join21, GreedyCriterion.MIN_SELECTIVITY)); 48 | 49 | assertFalse(GreedyUtil.better(join10, join10, GreedyCriterion.MIN_SELECTIVITY)); 50 | assertFalse(GreedyUtil.better(join01, join10, GreedyCriterion.MIN_SELECTIVITY)); 51 | assertFalse(GreedyUtil.better(join12, join12, GreedyCriterion.MIN_SELECTIVITY)); 52 | assertFalse(GreedyUtil.better(join21, join12, GreedyCriterion.MIN_SELECTIVITY)); 53 | } 54 | // Minimum size criterion 55 | { 56 | // The join between the first and the second table leads to minimal result size 57 | assertFalse(GreedyUtil.better(join12, join01, GreedyCriterion.MIN_SIZE)); 58 | assertFalse(GreedyUtil.better(join12, join10, GreedyCriterion.MIN_SIZE)); 59 | assertFalse(GreedyUtil.better(join21, join01, GreedyCriterion.MIN_SIZE)); 60 | assertFalse(GreedyUtil.better(join21, join10, GreedyCriterion.MIN_SIZE)); 61 | 62 | assertTrue(GreedyUtil.better(join01, join12, GreedyCriterion.MIN_SIZE)); 63 | assertTrue(GreedyUtil.better(join10, join12, GreedyCriterion.MIN_SIZE)); 64 | assertTrue(GreedyUtil.better(join01, join21, GreedyCriterion.MIN_SIZE)); 65 | assertTrue(GreedyUtil.better(join10, join21, GreedyCriterion.MIN_SIZE)); 66 | 67 | assertFalse(GreedyUtil.better(join10, join10, GreedyCriterion.MIN_SIZE)); 68 | assertFalse(GreedyUtil.better(join01, join10, GreedyCriterion.MIN_SIZE)); 69 | assertFalse(GreedyUtil.better(join12, join12, GreedyCriterion.MIN_SIZE)); 70 | assertFalse(GreedyUtil.better(join21, join12, GreedyCriterion.MIN_SIZE)); 71 | 72 | } 73 | } 74 | // Selecting the minSize join 75 | { 76 | { 77 | // Create query 78 | double[] tableCardinalities = new double[] {100, 10000, 100000}; 79 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 80 | Query query = new Query(3, tableCardinalities, selectivities); 81 | // Create all scan plans for the query 82 | List partialPlans = query.allScanPlans(planSpace, timeCostModel); 83 | // Insert minSize join 84 | GreedyUtil.performGreedyJoin(query, partialPlans, planSpace, 85 | timeCostModel, GreedyCriterion.MIN_SIZE); 86 | // After inserting a new join plan and removing its sub-plans the list size must be two 87 | assertEquals(2, partialPlans.size()); 88 | // The list needs to contain a join of the first two tables 89 | boolean performedRightJoin = false; 90 | BitSet joinResultIndices = new BitSet(); 91 | joinResultIndices.set(0); 92 | joinResultIndices.set(1); 93 | for (Plan plan : partialPlans) { 94 | if (plan.resultRel.tableSet.equals(joinResultIndices)) { 95 | performedRightJoin = true; 96 | } 97 | } 98 | assertTrue(performedRightJoin); 99 | } 100 | { 101 | // Create query 102 | double[] tableCardinalities = new double[] {1000000, 10000, 100000}; 103 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 104 | Query query = new Query(3, tableCardinalities, selectivities); 105 | // Create all scan plans for the query 106 | List partialPlans = query.allScanPlans(planSpace, timeCostModel); 107 | // Insert minSize join 108 | GreedyUtil.performGreedyJoin(query, partialPlans, 109 | planSpace, timeCostModel, GreedyCriterion.MIN_SIZE); 110 | // After inserting a new join plan and removing its sub-plans the list size must be two 111 | assertEquals(2, partialPlans.size()); 112 | // The list needs to contain a join of the first two tables 113 | boolean performedRightJoin = false; 114 | BitSet joinResultIndices = new BitSet(); 115 | joinResultIndices.set(1); 116 | joinResultIndices.set(2); 117 | for (Plan plan : partialPlans) { 118 | if (plan.resultRel.tableSet.equals(joinResultIndices)) { 119 | performedRightJoin = true; 120 | } 121 | } 122 | assertTrue(performedRightJoin); 123 | } 124 | } 125 | // Creating a minSize plan 126 | { 127 | // Create query 128 | double[] tableCardinalities = new double[] {100, 10000, 100000}; 129 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 130 | Query query = new Query(3, tableCardinalities, selectivities); 131 | Plan plan = GreedyUtil.greedyPlan(query, planSpace, 132 | timeCostModel, GreedyCriterion.MIN_SIZE); 133 | assertTrue(plan.orderToString().contains("((0)(1))") || 134 | plan.orderToString().contains("((1)(0))")); 135 | TestUtil.validatePlan(plan, planSpace, timeCostModel, true); 136 | } 137 | { 138 | // Create query 139 | double[] tableCardinalities = new double[] {100000000, 10000, 100000}; 140 | double[][] selectivities = TestUtil.defaultSelectivityMatrix(3); 141 | Query query = new Query(3, tableCardinalities, selectivities); 142 | Plan plan = GreedyUtil.greedyPlan(query, planSpace, 143 | timeCostModel, GreedyCriterion.MIN_SIZE); 144 | assertTrue(plan.orderToString().contains("((1)(2))") || 145 | plan.orderToString().contains("((2)(1))")); 146 | TestUtil.validatePlan(plan, planSpace, timeCostModel, true); 147 | } 148 | } 149 | 150 | } 151 | -------------------------------------------------------------------------------- /src/util/MathUtilTest.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static org.junit.Assert.*; 4 | 5 | import java.util.Arrays; 6 | import java.util.BitSet; 7 | import java.util.HashSet; 8 | import java.util.Set; 9 | 10 | import org.junit.Test; 11 | 12 | import static util.TestUtil.*; 13 | 14 | public class MathUtilTest { 15 | 16 | @Test 17 | public void test() { 18 | // Test transformation to bit vector 19 | { 20 | boolean[] e = new boolean[] {true, false, true}; 21 | assertEquals(Arrays.toString(e), Arrays.toString(MathUtil.toBitVector(5, 3))); 22 | } 23 | { 24 | boolean[] e = new boolean[] {true, false, true, true}; 25 | assertEquals(Arrays.toString(e), Arrays.toString(MathUtil.toBitVector(13, 4))); 26 | } 27 | // Generation of BitSet from bit string 28 | { 29 | String bitstring = "1001"; 30 | BitSet expected = new BitSet(); 31 | expected.set(0); 32 | expected.set(3); 33 | assertEquals(expected, MathUtil.getBitSet(bitstring)); 34 | } 35 | { 36 | String bitstring = "0100"; 37 | BitSet expected = new BitSet(); 38 | expected.set(2); 39 | assertEquals(expected, MathUtil.getBitSet(bitstring)); 40 | } 41 | // Generation of Power set 42 | { 43 | Set expectedSets = new HashSet(); 44 | expectedSets.addAll(Arrays.asList(new BitSet[] { 45 | MathUtil.getBitSet("0"), 46 | MathUtil.getBitSet("1"), 47 | })); 48 | assertEquals(expectedSets, MathUtil.powerSet(0)); 49 | } 50 | { 51 | Set expectedSets = new HashSet(); 52 | expectedSets.addAll(Arrays.asList(new BitSet[] { 53 | MathUtil.getBitSet("00"), 54 | MathUtil.getBitSet("01"), 55 | MathUtil.getBitSet("10"), 56 | MathUtil.getBitSet("11"), 57 | })); 58 | assertEquals(expectedSets, MathUtil.powerSet(1)); 59 | } 60 | { 61 | Set expectedSets = new HashSet(); 62 | expectedSets.addAll(Arrays.asList(new BitSet[] { 63 | MathUtil.getBitSet("000"), 64 | MathUtil.getBitSet("001"), 65 | MathUtil.getBitSet("010"), 66 | MathUtil.getBitSet("100"), 67 | MathUtil.getBitSet("101"), 68 | MathUtil.getBitSet("110"), 69 | MathUtil.getBitSet("101"), 70 | MathUtil.getBitSet("011"), 71 | MathUtil.getBitSet("111"), 72 | })); 73 | assertEquals(expectedSets, MathUtil.powerSet(2)); 74 | } 75 | // test counting true entries 76 | { 77 | boolean[] v = new boolean[] {true, false, false, true, true}; 78 | assertEquals(3, MathUtil.nrTrueValues(v)); 79 | } 80 | // Test summing up vectors 81 | { 82 | double[] v = new double[] {1, 2.3, 4}; 83 | assertEquals(7.3, MathUtil.aggSum(v), EPSILON); 84 | } 85 | // Test averaging vectors 86 | { 87 | double[] v = new double[] {1, 2.3, 3}; 88 | assertEquals(2.1, MathUtil.aggMean(v), EPSILON); 89 | } 90 | // Test calculating the median 91 | { 92 | // For vectors with even number of entries 93 | { 94 | double[] v = new double[] {5, 1, 2, 3}; 95 | assertEquals(2.5, MathUtil.aggMedian(v), EPSILON); 96 | } 97 | // For vectors with odd number of entries 98 | { 99 | double[] v = new double[] {5, 1, 2}; 100 | assertEquals(2, MathUtil.aggMedian(v), EPSILON); 101 | } 102 | 103 | } 104 | // Test calculating variance 105 | { 106 | double[] v = new double[] {1, 2, 6}; 107 | assertEquals((4+1+9)/3.0, MathUtil.aggVariance(v), EPSILON); 108 | } 109 | // Test calculating standard deviation 110 | { 111 | double[] v = new double[] {1, 2, 6}; 112 | assertEquals(Math.sqrt((4+1+9)/3.0), MathUtil.aggStDev(v), EPSILON); 113 | } 114 | // test logarithm 115 | { 116 | assertEquals(MathUtil.logOfBase(4, 16), 2, EPSILON); 117 | assertEquals(MathUtil.logOfBase(10, 0.01), -2, EPSILON); 118 | assertEquals(Double.POSITIVE_INFINITY, 119 | MathUtil.logOfBase(4, Double.POSITIVE_INFINITY), 120 | EPSILON); 121 | } 122 | // aggregate minimum 123 | { 124 | int[] i = new int[] {7, 2, 5, 7}; 125 | int er = 2; 126 | int r = MathUtil.aggVectorMin(i); 127 | assertEquals(er, r); 128 | } 129 | // aggregate minimum 130 | { 131 | int[] i = new int[] {4, 7, 2, 5, 7}; 132 | int er = 7; 133 | int r = MathUtil.aggVectorMax(i); 134 | assertEquals(er, r); 135 | } 136 | // vector ceil 137 | { 138 | double[] i = new double[] {2.2, 3}; 139 | int[] er = new int[] {3, 3}; 140 | int[] r = MathUtil.vectorCeil(i); 141 | assertArrayEquals(er, r); 142 | } 143 | // vector floor 144 | { 145 | double[] i = new double[] {2.2, 3}; 146 | int[] er = new int[] {2, 3}; 147 | int[] r = MathUtil.vectorFloor(i); 148 | assertArrayEquals(er, r); 149 | } 150 | // vector multiplication 151 | { 152 | int[] v1 = new int[] {2, 3}; 153 | int[] v2 = MathUtil.vectorMult(v1, 2); 154 | assertEquals(v2[0], 4); 155 | } 156 | // vector division 157 | { 158 | int[] i1 = new int[] {2, 3}; 159 | int[] r = MathUtil.vectorDiv(i1, 3); 160 | int[] er = new int[] {0, 1}; 161 | assertArrayEquals(er, r); 162 | } 163 | // vector mod 164 | { 165 | int[] i1 = new int[] {4, 8, 4, 3}; 166 | int[] er = new int[] {1, 2, 1, 0}; 167 | int[] r = MathUtil.vectorMod(i1, 3); 168 | assertArrayEquals(er, r); 169 | } 170 | // power vectors 171 | { 172 | int[] input = new int[] {2, 3}; 173 | double[] result = MathUtil.powerVector(10, input); 174 | double[] expectedResult = new double[] {100, 1000}; 175 | assertArrayEquals(expectedResult, result, EPSILON); 176 | } 177 | // logarithm vectors 178 | { 179 | double[] input = new double[] {4, 16}; 180 | double[] result = MathUtil.logVector(2, input); 181 | double[] expectedResult = new double[] {2, 4}; 182 | assertArrayEquals(expectedResult, result, EPSILON); 183 | } 184 | // scalar product 185 | { 186 | long[] i1 = new long[] {3, 5}; 187 | int[] i2 = new int[] {2, 1}; 188 | long r = MathUtil.scalarProduct(i1, i2); 189 | long er = 11; 190 | assertEquals(er, r); 191 | } 192 | // vector addition 193 | { 194 | int[] v1 = new int[] {3, 5}; 195 | int[] v2 = new int[] {2, 1}; 196 | int[] v3 = MathUtil.vectorAdd(v1, v2); 197 | assertEquals(v3[0], 5); 198 | assertEquals(v3[1], 6); 199 | } 200 | // vector subtraction 201 | { 202 | int[] v1 = new int[] {3, 5}; 203 | int[] v2 = new int[] {2, 1}; 204 | int[] v3 = MathUtil.vectorSubtract(v1, v2); 205 | assertEquals(v3[0], 1); 206 | assertEquals(v3[1], 4); 207 | } 208 | // vector minimum 209 | { 210 | int[] i1 = new int[] {1, 3, 2}; 211 | int[] i2 = new int[] {4, 2, 0}; 212 | int[] er = new int[] {1, 2, 0}; 213 | int[] r = MathUtil.vectorMin(i1, i2); 214 | assertArrayEquals(er, r); 215 | } 216 | // vectors concatenation 217 | { 218 | int[] v1 = new int[] {2, 3}; 219 | int[] v2 = new int[] {4, 6, 7}; 220 | int[] v3 = MathUtil.vectorConcatenate(v1, v2); 221 | assertEquals(v3[0], 2); 222 | assertEquals(v3[2], 4); 223 | assertEquals(v3[4], 7); 224 | assertEquals(v3.length, 5); 225 | } 226 | } 227 | 228 | } 229 | -------------------------------------------------------------------------------- /src/util/ParetoUtil.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import plans.Plan; 4 | 5 | import java.util.List; 6 | 7 | /** 8 | * Utility functions for comparing Pareto plan frontier approximations. 9 | * 10 | * @author immanueltrummer 11 | * 12 | */ 13 | public class ParetoUtil { 14 | /** 15 | * Calculates the epsilon error when trying to approximate the reference vector by the 16 | * tested vector. The epsilon error is the minimal value such that scaling the reference 17 | * cost vector up by (1 + epsilon) makes the tested vector dominate the reference vector. 18 | * 19 | * @param testedVector epsilon error captures how well this vector approximates reference 20 | * @param referenceVector the reference vector that must be approximated 21 | * @param consideredMetrics Boolean flags indicating for each cost metric if it is considered 22 | * @return the relative error when approximating reference by test vector 23 | */ 24 | public static double epsilonError(double[] testedVector, 25 | double[] referenceVector, boolean[] consideredMetrics) { 26 | assert(testedVector.length == referenceVector.length); 27 | int nrMetrics = testedVector.length; 28 | // Calculate error as maximum over all relevant cost metrics 29 | double maxError = 0; 30 | for (int metricCtr=0; metricCtr testedFrontier, 66 | List referenceFrontier, boolean[] consideredMetrics) { 67 | // Calculate total error as maximum over all plans in the reference set 68 | double setError = 0; 69 | for (Plan referencePlan : referenceFrontier) { 70 | // Calculate reference plan error as minimum over all plans in the tested set 71 | double planError = Double.POSITIVE_INFINITY; 72 | for (Plan testPlan : testedFrontier) { 73 | double planPairError = epsilonError(testPlan.getCostValuesCopy(), 74 | referencePlan.getCostValuesCopy(), consideredMetrics); 75 | planError = Math.min(planError, planPairError); 76 | } 77 | setError = Math.max(setError, planError); 78 | } 79 | return setError; 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/util/ParetoUtilTest.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static org.junit.Assert.*; 4 | import static util.TestUtil.*; 5 | import plans.Plan; 6 | import plans.ScanPlan; 7 | import plans.operators.ScanOperator; 8 | import plans.operators.local.LocalScan; 9 | import queries.Query; 10 | 11 | import java.util.LinkedList; 12 | import java.util.List; 13 | 14 | import org.junit.Test; 15 | 16 | public class ParetoUtilTest { 17 | 18 | @Test 19 | public void test() { 20 | boolean[] allMetrics = new boolean[] {true, true, true}; 21 | // Calculating epsilon error between two vectors 22 | { 23 | // Worst approximation for second dimension - therefore epsilon=0.5 24 | double[] testedVector = new double[] {2, 3, 5}; 25 | double[] referenceVector = new double[] {3, 2, 6}; 26 | assertEquals(0.5, ParetoUtil.epsilonError(testedVector, 27 | referenceVector, allMetrics), EPSILON); 28 | } 29 | { 30 | // Tested vector has lower cost than reference in all dimension; therefore the 31 | // the error is zero. 32 | double[] testedVector = new double[] {2, 2, 5}; 33 | double[] referenceVector = new double[] {3, 2, 6}; 34 | assertEquals(0, ParetoUtil.epsilonError(testedVector, 35 | referenceVector, allMetrics), EPSILON); 36 | } 37 | // Calculating epsilon error between two vector sets 38 | { 39 | Query dummyQuery = new Query(1, new double[] {1}, new double[][]{{1}}); 40 | ScanOperator scanOperator = new LocalScan(); 41 | Plan testedPlan1 = new ScanPlan(dummyQuery, 0, scanOperator); 42 | Plan testedPlan2 = new ScanPlan(dummyQuery, 0, scanOperator); 43 | Plan referencePlan1 = new ScanPlan(dummyQuery, 0, scanOperator); 44 | Plan referencePlan2 = new ScanPlan(dummyQuery, 0, scanOperator); 45 | testedPlan1.setCostValues(new double[] {3, 5, 3}); 46 | testedPlan2.setCostValues(new double[] {2, 3, 6}); 47 | referencePlan1.setCostValues(new double[] {1, 4, 4}); 48 | referencePlan2.setCostValues(new double[] {2, 1, 6}); 49 | List testedFrontier = new LinkedList(); 50 | List referenceFrontier = new LinkedList(); 51 | referenceFrontier.add(referencePlan1); 52 | testedFrontier.add(testedPlan1); 53 | assertEquals(2, ParetoUtil.epsilonError(testedFrontier, 54 | referenceFrontier, allMetrics), EPSILON); 55 | referenceFrontier.add(referencePlan2); 56 | assertEquals(4, ParetoUtil.epsilonError(testedFrontier, 57 | referenceFrontier, allMetrics), EPSILON); 58 | testedFrontier.add(testedPlan2); 59 | assertEquals(2, ParetoUtil.epsilonError(testedFrontier, 60 | referenceFrontier, allMetrics), EPSILON); 61 | } 62 | } 63 | 64 | } 65 | -------------------------------------------------------------------------------- /src/util/PruningUtil.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static common.Constants.*; 4 | 5 | import java.util.Iterator; 6 | import java.util.LinkedList; 7 | import java.util.List; 8 | 9 | import plans.Plan; 10 | import queries.Query; 11 | import relations.Relation; 12 | 13 | /** 14 | * Contains several utility functions related to cost vector and plan comparisons. 15 | * 16 | * @author immanueltrummer 17 | * 18 | */ 19 | public class PruningUtil { 20 | /** 21 | * Check if the fist cost value approximates the second for given approximation factor. 22 | * 23 | * @param c1 first cost value 24 | * @param c2 second cost value 25 | * @param alpha approximation factor 26 | * @return Boolean value indicating whether first cost value approximates second 27 | */ 28 | public static boolean approximates(double c1, double c2, double alpha) { 29 | assert(alpha >= 1); 30 | if (c1 <= c2 * alpha) { 31 | return true; 32 | } else { 33 | return false; 34 | } 35 | } 36 | /** 37 | * Checks if the first cost vector approximately dominates the second 38 | * for given approximation factor and set of cost metrics to consider. 39 | * 40 | * @param v1 first cost vector 41 | * @param v2 second cost vector 42 | * @param alpha approximation factor 43 | * @param consideredMetric Boolean flag for each metric indicating whether it is used 44 | * @return Boolean indicating whether first vector approximates second 45 | */ 46 | public static boolean approximatelyDominates( 47 | double[] v1, double[] v2, double alpha, boolean[] consideredMetric) { 48 | assert(v1.length == NR_COST_METRICS); 49 | assert(v2.length == NR_COST_METRICS); 50 | assert(consideredMetric.length == NR_COST_METRICS); 51 | for (int metricCtr=0; metricCtr v2[metricCtr]) { 84 | worseInOne = true; 85 | } 86 | } 87 | } 88 | return betterInOne && !worseInOne; 89 | } 90 | /** 91 | * Check if two plans producing the same intermediate result 92 | * generate the output in the same form. 93 | * 94 | * @param plan1 first query plan 95 | * @param plan2 second query plan 96 | * @return Boolean indicating if both plans produce data in the same format 97 | */ 98 | public static boolean sameOutputProperties(Plan plan1, Plan plan2) { 99 | assert(TestUtil.joinSameTables(plan1, plan2)); 100 | // If one plan materializes its output then this might speed up future operations 101 | if (plan1.materializes != plan2.materializes) { 102 | return false; 103 | } 104 | // If all prior checks were passed then the plans are comparable 105 | return true; 106 | } 107 | /** 108 | * Prune plans based on cost values alone (not considering output properties). 109 | * This is appropriate when comparing complete plans since a higher cost cannot 110 | * be made up for by producing data in a format speeding up the next operations. 111 | * A new plan is inserted into the old plans and dominated plans are pruned out. 112 | * 113 | * @param oldPlans set of Pareto-optimal query plans 114 | * @param newPlan one new plan not contained in the old plans 115 | * @param consideredMetric Boolean flags indicating which metrics to consider 116 | */ 117 | public static void pruneCostBased( 118 | List oldPlans, Plan newPlan, boolean[] consideredMetric) { 119 | // Check if new plan dominated 120 | for (Plan oldPlan : oldPlans) { 121 | if (approximatelyDominates(oldPlan.getCostValuesCopy(), newPlan.getCostValuesCopy(), 1, consideredMetric)) { 122 | //return oldPlans; 123 | return; 124 | } 125 | } 126 | // If we arrive here then the new plan will definitely be inserted. 127 | // Prune old plans dominated by new one 128 | Iterator oldPlansIter = oldPlans.iterator(); 129 | while (oldPlansIter.hasNext()) { 130 | Plan oldPlan = oldPlansIter.next(); 131 | if (approximatelyDominates(newPlan.getCostValuesCopy(), oldPlan.getCostValuesCopy(), 1, consideredMetric)) { 132 | oldPlansIter.remove(); 133 | } 134 | } 135 | oldPlans.add(newPlan); 136 | } 137 | /** 138 | * Prune plans producing the same relation using their cost and output properties. 139 | * This function changes the list of Pareto plans that is associated with the given relation. 140 | * One new plan is inserted if it is not approximately dominated by another plan whose 141 | * output has the same properties. 142 | * 143 | * @param query we compare partial plans for that query 144 | * @param rel a relation 145 | * @param newPlan a new plan producing the given relation 146 | * @param alpha approximation factor; less plans are kept with a higher alpha 147 | * @param consideredMetric Boolean flags indicating which metrics to consider 148 | * @param insertCopy whether to insert the given plan as Pareto plan or a copy of it 149 | */ 150 | public static void prune(Query query, Relation rel, Plan newPlan, double alpha, 151 | boolean[] consideredMetric, boolean insertCopy) { 152 | // Make sure that Pareto plan list is initialized 153 | if (rel.ParetoPlans == null) { 154 | rel.ParetoPlans = new LinkedList(); 155 | } 156 | // Check if there are similar plans to the new plan and return in that case 157 | double[] newCost = newPlan.cost; 158 | for (Plan oldPlan : rel.ParetoPlans) { 159 | if (PruningUtil.sameOutputProperties(newPlan, oldPlan) && 160 | PruningUtil.approximatelyDominates( 161 | oldPlan.cost, newCost, alpha, consideredMetric)) { 162 | return; 163 | } 164 | } 165 | // New plan will be inserted - prune prior plans with precise comparisons 166 | Iterator planIter = rel.ParetoPlans.iterator(); 167 | while (planIter.hasNext()) { 168 | Plan oldPlan = planIter.next(); 169 | if (PruningUtil.sameOutputProperties(newPlan, oldPlan) && 170 | PruningUtil.approximatelyDominates( 171 | newCost, oldPlan.cost, 1, consideredMetric)) { 172 | planIter.remove(); 173 | } 174 | } 175 | if (insertCopy) { 176 | newPlan = newPlan.deepMutableCopy(); 177 | } 178 | if (SAFE_MODE) { 179 | newPlan.makeImmutable(); 180 | } 181 | rel.ParetoPlans.add(newPlan); 182 | } 183 | } 184 | -------------------------------------------------------------------------------- /src/util/PruningUtilTest.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static org.junit.Assert.*; 4 | 5 | import java.util.LinkedList; 6 | import java.util.List; 7 | 8 | import plans.JoinPlan; 9 | import plans.Plan; 10 | import plans.ScanPlan; 11 | import plans.operators.JoinOperator; 12 | import plans.operators.ScanOperator; 13 | import plans.operators.local.BNLjoin; 14 | import plans.operators.local.LocalScan; 15 | import queries.JoinGraphType; 16 | import queries.JoinType; 17 | import queries.Query; 18 | import queries.QueryFactory; 19 | 20 | import org.junit.Test; 21 | 22 | public class PruningUtilTest { 23 | 24 | @Test 25 | public void test() { 26 | // Approximation for single cost metric 27 | { 28 | assertTrue(PruningUtil.approximates(10, 11, 1.1)); 29 | assertTrue(PruningUtil.approximates(11, 10, 1.1)); 30 | assertFalse(PruningUtil.approximates(12, 10, 1.1)); 31 | } 32 | // Approximation for multiple cost metrics 33 | { 34 | { 35 | double[] v1 = new double[] {11, 11, 11}; 36 | double[] v2 = new double[] {10, 10, 10}; 37 | boolean[] consideredMetric = new boolean[] {true, true, true}; 38 | assertTrue(PruningUtil.approximatelyDominates(v1, v2, 1.1, consideredMetric)); 39 | assertFalse(PruningUtil.approximatelyDominates(v1, v2, 1, consideredMetric)); 40 | } 41 | { 42 | double[] v1 = new double[] {10, 11, 10}; 43 | double[] v2 = new double[] {10, 10, 10}; 44 | boolean[] consideredMetric = new boolean[] {true, false, true}; 45 | assertTrue(PruningUtil.approximatelyDominates(v1, v2, 1.1, consideredMetric)); 46 | assertTrue(PruningUtil.approximatelyDominates(v1, v2, 1, consideredMetric)); 47 | } 48 | } 49 | // Check for Pareto dominance 50 | { 51 | { 52 | double[] v1 = new double[] {10, 10, 10}; 53 | double[] v2 = new double[] {10, 10, 10}; 54 | boolean[] consideredMetric = new boolean[] {true, true, true}; 55 | assertFalse(PruningUtil.ParetoDominates(v1, v2, consideredMetric)); 56 | } 57 | { 58 | double[] v1 = new double[] {10, 9, 10}; 59 | double[] v2 = new double[] {10, 10, 10}; 60 | boolean[] consideredMetric = new boolean[] {true, true, true}; 61 | assertTrue(PruningUtil.ParetoDominates(v1, v2, consideredMetric)); 62 | } 63 | { 64 | double[] v1 = new double[] {10, 9, 10}; 65 | double[] v2 = new double[] {10, 10, 10}; 66 | boolean[] consideredMetric = new boolean[] {true, false, true}; 67 | assertFalse(PruningUtil.ParetoDominates(v1, v2, consideredMetric)); 68 | } 69 | { 70 | double[] v1 = new double[] {8, 9, 12}; 71 | double[] v2 = new double[] {10, 10, 10}; 72 | boolean[] consideredMetric = new boolean[] {true, true, true}; 73 | assertFalse(PruningUtil.ParetoDominates(v1, v2, consideredMetric)); 74 | } 75 | { 76 | double[] v1 = new double[] {10, 10, 10}; 77 | double[] v2 = new double[] {10, 10, 10}; 78 | boolean[] consideredMetric = new boolean[] {true, true, true}; 79 | assertFalse(PruningUtil.ParetoDominates(v1, v2, consideredMetric)); 80 | } 81 | } 82 | // Comparison of output properties 83 | { 84 | { 85 | Query query = QueryFactory.produce(JoinGraphType.CHAIN, 10, 100000, JoinType.MN); 86 | ScanOperator scanOp = new LocalScan(); 87 | Plan scan0 = new ScanPlan(query, 0, scanOp); 88 | Plan scan1 = new ScanPlan(query, 1, scanOp); 89 | JoinOperator materializedJoinOp = new BNLjoin(10, true); 90 | JoinOperator nonMaterializedJoinOp = new BNLjoin(10, false); 91 | Plan joinMaterialized = new JoinPlan(query, scan0, scan1, materializedJoinOp); 92 | Plan joinNonMaterialized = new JoinPlan(query, scan0, scan1, nonMaterializedJoinOp); 93 | assertTrue(PruningUtil.sameOutputProperties(joinMaterialized, joinMaterialized)); 94 | assertFalse(PruningUtil.sameOutputProperties(joinMaterialized, joinNonMaterialized)); 95 | } 96 | } 97 | // Cost-based pruning 98 | { 99 | { 100 | Query query = QueryFactory.produce(JoinGraphType.CHAIN, 10, 100000, JoinType.MN); 101 | ScanOperator scanOp = new LocalScan(); 102 | Plan plan1 = new ScanPlan(query, 0, scanOp); 103 | Plan plan2 = new ScanPlan(query, 0, scanOp); 104 | Plan plan3 = new ScanPlan(query, 0, scanOp); 105 | Plan plan4 = new ScanPlan(query, 0, scanOp); 106 | plan1.setCostValues(new double[] {1, 2, 0}); 107 | plan2.setCostValues(new double[] {3, 2, 0}); 108 | plan3.setCostValues(new double[] {1, 1, 0}); 109 | plan4.setCostValues(new double[] {3, 0.5, 0}); 110 | boolean[] consideredMetric = new boolean[] {true, true, true}; 111 | List plans = new LinkedList(); 112 | plans.add(plan1); 113 | assertEquals(1, plans.size()); 114 | PruningUtil.pruneCostBased(plans, plan2, consideredMetric); 115 | assertEquals(1, plans.size()); // new plan was dominated 116 | PruningUtil.pruneCostBased(plans, plan3, consideredMetric); 117 | assertEquals(1, plans.size()); // new plan dominated old one 118 | PruningUtil.pruneCostBased(plans, plan4, consideredMetric); 119 | assertEquals(2, plans.size()); // new plan and old plan both Pareto-optimal 120 | } 121 | } 122 | } 123 | 124 | } 125 | -------------------------------------------------------------------------------- /src/util/SamplingUtil.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import java.util.Collections; 4 | import java.util.List; 5 | 6 | import cost.MultiCostModel; 7 | import plans.JoinPlan; 8 | import plans.Plan; 9 | import plans.operators.JoinOperator; 10 | import plans.spaces.PlanSpace; 11 | import queries.Query; 12 | 13 | public class SamplingUtil { 14 | 15 | public static void performJoin(Query query, PlanSpace planSpace, 16 | MultiCostModel costModel, List partialPlans, double maxHeightCutoff) { 17 | Collections.shuffle(partialPlans); 18 | // Find out what is the highest plan among the partial plans 19 | int maxHeight = 0; 20 | for (Plan partialPlan : partialPlans) { 21 | maxHeight = Math.max(maxHeight, partialPlan.height); 22 | } 23 | // Select and remove left join operand 24 | Plan leftOperand = null; 25 | for (Plan partialPlan : partialPlans) { 26 | if (partialPlan.height >= maxHeight * maxHeightCutoff) { 27 | leftOperand = partialPlan; 28 | break; 29 | } 30 | } 31 | partialPlans.remove(leftOperand); 32 | // Select and remove right operand 33 | Plan rightOperand = partialPlans.get(0); 34 | partialPlans.remove(0); 35 | // Prepare and add join plan 36 | JoinOperator joinOperator = planSpace.randomJoinOperator(leftOperand, rightOperand); 37 | JoinPlan joinPlan = new JoinPlan(query, leftOperand, rightOperand, joinOperator); 38 | partialPlans.add(joinPlan); 39 | costModel.updateRoot(joinPlan); 40 | } 41 | 42 | public static Plan samplePlan(Query query, PlanSpace planSpace, MultiCostModel costModel, 43 | List partialPlans, double maxHeightCutoff) { 44 | while (partialPlans.size() > 1) { 45 | performJoin(query, planSpace, costModel, partialPlans, maxHeightCutoff); 46 | } 47 | return partialPlans.iterator().next(); 48 | } 49 | 50 | } 51 | -------------------------------------------------------------------------------- /src/util/TestUtil.java: -------------------------------------------------------------------------------- 1 | package util; 2 | 3 | import static common.Constants.*; 4 | import cost.MultiCostModel; 5 | import cost.SingleCostModel; 6 | import cost.local.BufferCostModel; 7 | import cost.local.DiscCostModel; 8 | import cost.local.TimeCostModel; 9 | import plans.JoinPlan; 10 | import plans.Plan; 11 | import plans.ScanPlan; 12 | import plans.operators.JoinOperator; 13 | import plans.spaces.LocalPlanSpace; 14 | import plans.spaces.PlanSpace; 15 | 16 | import java.util.Arrays; 17 | import java.util.BitSet; 18 | import java.util.List; 19 | 20 | // Utility functions for performing JUnit tests. 21 | public class TestUtil { 22 | public final static double EPSILON = 1E-10; 23 | public final static double LARGE_EPSILON = 1E-4; 24 | // Standard local cost model 25 | public final static PlanSpace planSpace = new LocalPlanSpace(); 26 | public final static List costModels = 27 | Arrays.asList(new SingleCostModel[] { 28 | new TimeCostModel(0), new BufferCostModel(1), new DiscCostModel(2)}); 29 | public final static MultiCostModel costModel = new MultiCostModel(costModels); 30 | public final static MultiCostModel timeCostModel = new MultiCostModel( 31 | Arrays.asList(new SingleCostModel[] {new TimeCostModel(0)})); 32 | 33 | // Obtain a selectivity matrix of specified dimensions that is initialized with 1. 34 | public static double[][] defaultSelectivityMatrix(int nrTables) { 35 | double[][] matrix = new double[nrTables][nrTables]; 36 | for (int table1=0; table1 testedFrontier, 52 | List referenceFrontier, boolean[] consideredMetric) { 53 | for (Plan refPlan : referenceFrontier) { 54 | System.out.println("Reference plan with " + Arrays.toString(refPlan.getCostValuesCopy()) + ":\t" + refPlan); 55 | // How well is this cost vector approximated? 56 | double epsilon = Double.POSITIVE_INFINITY; 57 | Plan bestPlan = null; 58 | for (Plan testPlan : testedFrontier) { 59 | double curEpsilon = ParetoUtil.epsilonError(testPlan.getCostValuesCopy(), 60 | refPlan.getCostValuesCopy(), consideredMetric); 61 | if (curEpsilon recalculatedCost[metricCtr] + recalculatedCost[metricCtr] * LARGE_EPSILON || 80 | originalCost[metricCtr] < recalculatedCost[metricCtr] - recalculatedCost[metricCtr] * LARGE_EPSILON) { 81 | consistentCost = false; 82 | } 83 | } 84 | assert consistentCost : 85 | "Original: " + Arrays.toString(originalCost) + 86 | "; Recalculated: " + Arrays.toString(recalculatedCost) 87 | + "; Plan: " + plan.toString() + "; Order: " + plan.orderToString(); 88 | } 89 | // Validate that each operator in the plan is applicable. 90 | public static void validateOperators(Plan plan, PlanSpace planSpace) { 91 | if (plan instanceof ScanPlan) { 92 | assert(planSpace.scanOperatorApplicable(((ScanPlan)plan).scanOperator, plan.resultRel)); 93 | } else { 94 | assert(plan instanceof JoinPlan); 95 | JoinPlan joinPlan = (JoinPlan)plan; 96 | JoinOperator operator = joinPlan.getJoinOperator(); 97 | assert(planSpace.joinOperatorApplicable( 98 | operator, joinPlan.getLeftPlan(), joinPlan.getRightPlan())); 99 | validateOperators(joinPlan.getLeftPlan(), planSpace); 100 | validateOperators(joinPlan.getRightPlan(), planSpace); 101 | } 102 | } 103 | // Validates that the plan joins precisely the tables of its result relation. 104 | public static BitSet validateJoinedTables(Plan plan) { 105 | BitSet actualTablesRead = new BitSet(); 106 | if (plan instanceof ScanPlan) { 107 | ScanPlan scanPlan = (ScanPlan)plan; 108 | actualTablesRead.set(scanPlan.tableIndex); 109 | } else { 110 | assert(plan instanceof JoinPlan); 111 | JoinPlan joinPlan = (JoinPlan)plan; 112 | BitSet actualTablesReadLeft = validateJoinedTables(joinPlan.getLeftPlan()); 113 | BitSet actualTablesReadRight = validateJoinedTables(joinPlan.getRightPlan()); 114 | actualTablesRead.or(actualTablesReadLeft); 115 | actualTablesRead.or(actualTablesReadRight); 116 | } 117 | BitSet claimedTablesRead = plan.resultRel.tableSet; 118 | if (claimedTablesRead != null) { 119 | assert(claimedTablesRead.equals(actualTablesRead)); 120 | } 121 | return actualTablesRead; 122 | } 123 | // Validates one plan with all available tests 124 | public static void validatePlan(Plan plan, PlanSpace planSpace, 125 | MultiCostModel costModel, boolean checkJoinedTables) { 126 | validateCost(plan, costModel); 127 | validateOperators(plan, planSpace); 128 | if (checkJoinedTables) { 129 | validateJoinedTables(plan); 130 | } 131 | } 132 | // Validates all plans in the test with all available tests 133 | public static void validatePlans(List plans, PlanSpace planSpace, 134 | MultiCostModel costModel, boolean checkJoinedTables) { 135 | for (Plan plan : plans) { 136 | validatePlan(plan, planSpace, costModel, checkJoinedTables); 137 | } 138 | } 139 | // Returns true if both plans join the same tables or both have a null pointer as result relation. 140 | public static boolean joinSameTables(Plan plan1, Plan plan2) { 141 | return (plan1.resultRel == null && plan2.resultRel == null || 142 | plan1.resultRel.tableSet.equals(plan2.resultRel.tableSet)); 143 | } 144 | } 145 | --------------------------------------------------------------------------------