diff --git a/src/brain/index_selection.cpp b/src/brain/index_selection.cpp new file mode 100644 index 00000000000..cbaf0c516e8 --- /dev/null +++ b/src/brain/index_selection.cpp @@ -0,0 +1,488 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection.cpp +// +// Identification: src/brain/index_selection.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include +#include + +#include "brain/index_selection.h" +#include "brain/what_if_index.h" + +namespace peloton { +namespace brain { + +IndexSelection::IndexSelection(Workload &query_set, IndexSelectionKnobs knobs, + concurrency::TransactionContext *txn) + : query_set_(query_set), context_(knobs), txn_(txn) {} + +void IndexSelection::GetBestIndexes(IndexConfiguration &final_indexes) { + // http://www.vldb.org/conf/1997/P146.PDF + // Figure 4 of the "Index Selection Tool" paper. + // Split the workload 'W' into small workloads 'Wi', with each + // containing one query, and find out the candidate indexes + // for these 'Wi' + // Finally, combine all the candidate indexes 'Ci' into a larger + // set to form a candidate set 'C' for the provided workload 'W'. + + // The best indexes after every iteration + IndexConfiguration candidate_indexes; + // Single column indexes that are useful for at least one query + IndexConfiguration admissible_indexes; + + // Start the index selection. + for (unsigned long i = 0; i < context_.knobs_.num_iterations_; i++) { + LOG_DEBUG("******* Iteration %ld **********", i); + LOG_DEBUG("Candidate Indexes Before: %s", + candidate_indexes.ToString().c_str()); + GenerateCandidateIndexes(candidate_indexes, admissible_indexes, query_set_); + LOG_DEBUG("Admissible Indexes: %s", admissible_indexes.ToString().c_str()); + LOG_DEBUG("Candidate Indexes After: %s", + candidate_indexes.ToString().c_str()); + + // Configuration Enumeration + IndexConfiguration top_candidate_indexes; + Enumerate(candidate_indexes, top_candidate_indexes, query_set_, + context_.knobs_.num_indexes_); + LOG_DEBUG("Top Candidate Indexes: %s", + candidate_indexes.ToString().c_str()); + + candidate_indexes = top_candidate_indexes; + + // Generate multi-column indexes before starting the next iteration. + // Only do this if there is next iteration. + if (i < (context_.knobs_.num_iterations_ - 1)) { + GenerateMultiColumnIndexes(top_candidate_indexes, admissible_indexes, + candidate_indexes); + } + } + + final_indexes = candidate_indexes; +} + +void IndexSelection::GenerateCandidateIndexes( + IndexConfiguration &candidate_config, IndexConfiguration &admissible_config, + Workload &workload) { + // If there are no admissible indexes, then this is the first iteration. + // Candidate indexes will be a union of admissible index set of each query. + if (admissible_config.IsEmpty() && candidate_config.IsEmpty()) { + for (auto query : workload.GetQueries()) { + Workload wi(query, workload.GetDatabaseName()); + + IndexConfiguration ai; + GetAdmissibleIndexes(query.first, ai); + admissible_config.Merge(ai); + + IndexConfiguration pruned_ai; + PruneUselessIndexes(ai, wi, pruned_ai); + // Candidate config for the single-column indexes is the union of + // candidates for each query. + candidate_config.Merge(pruned_ai); + } + LOG_TRACE("Single column candidate indexes: %lu", + candidate_config.GetIndexCount()); + } else { + LOG_TRACE("Pruning multi-column indexes"); + IndexConfiguration pruned_ai; + PruneUselessIndexes(candidate_config, workload, pruned_ai); + candidate_config.Set(pruned_ai); + } +} + +void IndexSelection::PruneUselessIndexes(IndexConfiguration &config, + Workload &workload, + IndexConfiguration &pruned_config) { + IndexConfiguration empty_config; + auto indexes = config.GetIndexes(); + + for (auto it = indexes.begin(); it != indexes.end(); it++) { + bool is_useful = false; + + for (auto query : workload.GetQueries()) { + IndexConfiguration c; + c.AddIndexObject(*it); + + Workload w(query, workload.GetDatabaseName()); + + auto c1 = ComputeCost(c, w); + auto c2 = ComputeCost(empty_config, w); + LOG_TRACE("Cost with index %s is %lf", c.ToString().c_str(), c1); + LOG_TRACE("Cost without is %lf", c2); + + if (c1 < c2) { + is_useful = true; + break; + } + } + // Index is useful if it benefits any query. + if (is_useful) { + pruned_config.AddIndexObject(*it); + } + } +} + +void IndexSelection::Enumerate(IndexConfiguration &indexes, + IndexConfiguration &top_indexes, + Workload &workload, size_t num_indexes) { + // Get the cheapest indexes through exhaustive search upto a threshold + ExhaustiveEnumeration(indexes, top_indexes, workload); + + // Get all the remaining indexes which can be part of our optimal set + auto remaining_indexes = indexes - top_indexes; + + // Greedily add the remaining indexes until there is no improvement in the + // cost or our required size is reached + GreedySearch(top_indexes, remaining_indexes, workload, num_indexes); +} + +void IndexSelection::GreedySearch(IndexConfiguration &indexes, + IndexConfiguration &remaining_indexes, + Workload &workload, size_t k) { + // Algorithm: + // 1. Let S = the best m index configuration using the naive enumeration + // algorithm. If m = k then exit. + // 2. Pick a new index I such that Cost (S U {I}, W) <= Cost(S U {I'}, W) for + // any choice of I' != I + // 3. If Cost (S U {I}) >= Cost(S) then exit + // Else S = S U {I} + // 4. If |S| = k then exit + LOG_TRACE("GREEDY: Starting with the following index: %s", + indexes.ToString().c_str()); + size_t current_index_count = indexes.GetIndexCount(); + + LOG_TRACE("GREEDY: At start: #indexes chosen : %zu, #num_indexes: %zu", + current_index_count, k); + + if (current_index_count >= k) return; + + double global_min_cost = ComputeCost(indexes, workload); + double cur_min_cost = global_min_cost; + double cur_cost; + std::shared_ptr best_index; + + // go through till you get top k indexes + while (current_index_count < k) { + // this is the set S so far + auto new_indexes = indexes; + for (auto const &index : remaining_indexes.GetIndexes()) { + new_indexes = indexes; + new_indexes.AddIndexObject(index); + cur_cost = ComputeCost(new_indexes, workload); + LOG_TRACE("GREEDY: Considering this index: %s \n with cost: %lf", + index->ToString().c_str(), cur_cost); + if (cur_cost < cur_min_cost || + (best_index != nullptr && cur_cost == cur_min_cost && + new_indexes.ToString() < best_index->ToString())) { + cur_min_cost = cur_cost; + best_index = index; + } + } + + // if we found a better configuration + if (cur_min_cost < global_min_cost) { + LOG_TRACE("GREEDY: Adding the following index: %s", + best_index->ToString().c_str()); + indexes.AddIndexObject(best_index); + remaining_indexes.RemoveIndexObject(best_index); + current_index_count++; + global_min_cost = cur_min_cost; + + // we are done with all remaining indexes + if (remaining_indexes.GetIndexCount() == 0) { + LOG_TRACE("GREEDY: Breaking because nothing more"); + break; + } + } else { // we did not find any better index to add to our current + // configuration + LOG_TRACE("GREEDY: Breaking because nothing better found"); + break; + } + } +} + +void IndexSelection::ExhaustiveEnumeration(IndexConfiguration &indexes, + IndexConfiguration &top_indexes, + Workload &workload) { + // Get the best m index configurations using the naive enumeration algorithm + // The naive algorithm gets all the possible subsets of size <= m and then + // returns the cheapest m indexes + + auto max_num_indexes = std::min(context_.knobs_.naive_enumeration_threshold_, + context_.knobs_.num_indexes_); + + // Define a set ordering of (index config, cost) and define the ordering in + // the set + std::set, IndexConfigComparator> + running_index_config(workload), temp_index_config(workload), + result_index_config(workload); + + IndexConfiguration new_element; + + // Add an empty configuration as initialization + IndexConfiguration empty; + // The running index configuration contains the possible subsets generated so + // far. It is updated after every iteration + auto cost_empty = ComputeCost(empty, workload); + running_index_config.emplace(empty, cost_empty); + + for (auto const &index : indexes.GetIndexes()) { + // Make a copy of the running index configuration and add each element to it + temp_index_config = running_index_config; + + for (auto t : temp_index_config) { + new_element = t.first; + new_element.AddIndexObject(index); + + // If the size of the subset reaches our threshold, add to result set + // instead of adding to the running list + if (new_element.GetIndexCount() >= max_num_indexes) { + result_index_config.emplace(new_element, + ComputeCost(new_element, workload)); + } else { + running_index_config.emplace(new_element, + ComputeCost(new_element, workload)); + } + } + } + + // Put all the subsets in the result set + result_index_config.insert(running_index_config.begin(), + running_index_config.end()); + // Remove the starting empty set that we added + result_index_config.erase({empty, cost_empty}); + + for (auto index : result_index_config) { + LOG_TRACE("EXHAUSTIVE: Index: %s, Cost: %lf", + index.first.ToString().c_str(), index.second); + } + + // Since the insertion into the sets ensures the order of cost, get the first + // m configurations + if (result_index_config.empty()) return; + + // if having no indexes is better (for eg. for insert heavy workload), + // then don't choose anything + if (cost_empty < result_index_config.begin()->second) return; + + auto best_m_index = result_index_config.begin()->first; + top_indexes.Merge(best_m_index); +} + +void IndexSelection::GetAdmissibleIndexes( + std::shared_ptr query, IndexConfiguration &indexes) { + // Find out the indexable columns of the given workload. + // The following rules define what indexable columns are: + // 1. A column that appears in the WHERE clause with format + // ==> Column OP Expr <== + // OP such as {=, <, >, <=, >=, LIKE, etc.} + // Column is a table column name. + // 2. GROUP BY (if present) + // 3. ORDER BY (if present) + // 4. all updated columns for UPDATE query. + switch (query->GetType()) { + case StatementType::INSERT: { + auto insert_stmt = dynamic_cast(query.get()); + // If the insert is along with a select statement, i.e another table's + // select output is fed into this table. + if (insert_stmt->select != nullptr) { + IndexColsParseWhereHelper(insert_stmt->select->where_clause.get(), + indexes); + } + break; + } + + case StatementType::DELETE: { + auto delete_stmt = dynamic_cast(query.get()); + IndexColsParseWhereHelper(delete_stmt->expr.get(), indexes); + break; + } + + case StatementType::UPDATE: { + auto update_stmt = dynamic_cast(query.get()); + IndexColsParseWhereHelper(update_stmt->where.get(), indexes); + break; + } + + case StatementType::SELECT: { + auto select_stmt = dynamic_cast(query.get()); + IndexColsParseWhereHelper(select_stmt->where_clause.get(), indexes); + IndexColsParseOrderByHelper(select_stmt->order, indexes); + IndexColsParseGroupByHelper(select_stmt->group_by, indexes); + break; + } + + default: { LOG_DEBUG("DDL Statement encountered, Ignoring.."); } + } +} + +void IndexSelection::IndexColsParseWhereHelper( + const expression::AbstractExpression *where_expr, + IndexConfiguration &config) { + if (where_expr == nullptr) { + LOG_DEBUG("No Where Clause Found"); + return; + } + auto expr_type = where_expr->GetExpressionType(); + const expression::AbstractExpression *left_child; + const expression::AbstractExpression *right_child; + const expression::TupleValueExpression *tuple_child; + + switch (expr_type) { + case ExpressionType::COMPARE_EQUAL: + case ExpressionType::COMPARE_NOTEQUAL: + case ExpressionType::COMPARE_GREATERTHAN: + case ExpressionType::COMPARE_GREATERTHANOREQUALTO: + case ExpressionType::COMPARE_LESSTHAN: + case ExpressionType::COMPARE_LESSTHANOREQUALTO: + case ExpressionType::COMPARE_LIKE: + case ExpressionType::COMPARE_NOTLIKE: + case ExpressionType::COMPARE_IN: + // Get left and right child and extract the column name. + left_child = where_expr->GetChild(0); + right_child = where_expr->GetChild(1); + + // if where clause is something like a = b, we don't benefit from index + if (left_child->GetExpressionType() == ExpressionType::VALUE_TUPLE && + right_child->GetExpressionType() == ExpressionType::VALUE_TUPLE) { + return; + } + + // if where clause is something like 1 = 2, we don't benefit from index + if (left_child->GetExpressionType() == ExpressionType::VALUE_CONSTANT && + right_child->GetExpressionType() == ExpressionType::VALUE_CONSTANT) { + return; + } + + if (left_child->GetExpressionType() == ExpressionType::VALUE_TUPLE) { + PELOTON_ASSERT(right_child->GetExpressionType() != + ExpressionType::VALUE_TUPLE); + tuple_child = + dynamic_cast(left_child); + } else { + PELOTON_ASSERT(right_child->GetExpressionType() == + ExpressionType::VALUE_TUPLE); + tuple_child = + dynamic_cast(right_child); + } + + if (!tuple_child->GetIsBound()) { + LOG_ERROR("Query is not bound"); + PELOTON_ASSERT(false); + } + IndexObjectPoolInsertHelper(tuple_child->GetBoundOid(), config); + + break; + case ExpressionType::CONJUNCTION_AND: + case ExpressionType::CONJUNCTION_OR: + left_child = where_expr->GetChild(0); + right_child = where_expr->GetChild(1); + IndexColsParseWhereHelper(left_child, config); + IndexColsParseWhereHelper(right_child, config); + break; + default: + LOG_ERROR("Index selection doesn't allow %s in where clause", + where_expr->GetInfo().c_str()); + PELOTON_ASSERT(false); + } +} + +void IndexSelection::IndexColsParseGroupByHelper( + std::unique_ptr &group_expr, + IndexConfiguration &config) { + if ((group_expr == nullptr) || (group_expr->columns.size() == 0)) { + LOG_DEBUG("Group by expression not present"); + return; + } + auto &columns = group_expr->columns; + for (auto it = columns.begin(); it != columns.end(); it++) { + PELOTON_ASSERT((*it)->GetExpressionType() == ExpressionType::VALUE_TUPLE); + auto tuple_value = (expression::TupleValueExpression *)((*it).get()); + IndexObjectPoolInsertHelper(tuple_value->GetBoundOid(), config); + } +} + +void IndexSelection::IndexColsParseOrderByHelper( + std::unique_ptr &order_expr, + IndexConfiguration &config) { + if ((order_expr == nullptr) || (order_expr->exprs.size() == 0)) { + LOG_DEBUG("Order by expression not present"); + return; + } + auto &exprs = order_expr->exprs; + for (auto it = exprs.begin(); it != exprs.end(); it++) { + PELOTON_ASSERT((*it)->GetExpressionType() == ExpressionType::VALUE_TUPLE); + auto tuple_value = (expression::TupleValueExpression *)((*it).get()); + IndexObjectPoolInsertHelper(tuple_value->GetBoundOid(), config); + } +} + +void IndexSelection::IndexObjectPoolInsertHelper( + const std::tuple &tuple_oid, + IndexConfiguration &config) { + auto db_oid = std::get<0>(tuple_oid); + auto table_oid = std::get<1>(tuple_oid); + auto col_oid = std::get<2>(tuple_oid); + + // Add the object to the pool. + HypotheticalIndexObject iobj(db_oid, table_oid, col_oid); + auto pool_index_obj = context_.pool_.GetIndexObject(iobj); + if (!pool_index_obj) { + pool_index_obj = context_.pool_.PutIndexObject(iobj); + } + config.AddIndexObject(pool_index_obj); +} + +double IndexSelection::ComputeCost(IndexConfiguration &config, + Workload &workload) { + double cost = 0.0; + auto queries = workload.GetQueries(); + for (auto query : queries) { + std::pair state = { + config, query.first.get()}; + if (context_.memo_.find(state) != context_.memo_.end()) { + cost += context_.memo_[state]; + } else { + auto result = WhatIfIndex::GetCostAndBestPlanTree( + query, config, workload.GetDatabaseName(), txn_); + context_.memo_[state] = result->cost; + cost += result->cost; + } + } + return cost; +} + +void IndexSelection::CrossProduct( + const IndexConfiguration &config, + const IndexConfiguration &single_column_indexes, + IndexConfiguration &result) { + auto indexes = config.GetIndexes(); + auto columns = single_column_indexes.GetIndexes(); + for (auto index : indexes) { + for (auto column : columns) { + if (!index->IsCompatible(column)) continue; + auto merged_index = (index->Merge(column)); + result.AddIndexObject(context_.pool_.PutIndexObject(merged_index)); + } + } +} + +void IndexSelection::GenerateMultiColumnIndexes( + IndexConfiguration &config, IndexConfiguration &single_column_indexes, + IndexConfiguration &result) { + CrossProduct(config, single_column_indexes, result); +} + +std::shared_ptr IndexSelection::AddConfigurationToPool( + HypotheticalIndexObject object) { + return context_.pool_.PutIndexObject(object); +} + +} // namespace brain +} // namespace peloton diff --git a/src/brain/index_selection_context.cpp b/src/brain/index_selection_context.cpp new file mode 100644 index 00000000000..3933b72c844 --- /dev/null +++ b/src/brain/index_selection_context.cpp @@ -0,0 +1,23 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_context.cpp +// +// Identification: src/brain/index_selection_context.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/index_selection_context.h" +#include "common/logger.h" + +namespace peloton { +namespace brain { + +IndexSelectionContext::IndexSelectionContext(IndexSelectionKnobs knobs) + : knobs_(knobs) {} + +} // namespace brain +} // namespace peloton diff --git a/src/brain/index_selection_job.cpp b/src/brain/index_selection_job.cpp new file mode 100644 index 00000000000..a23a2b21ace --- /dev/null +++ b/src/brain/index_selection_job.cpp @@ -0,0 +1,189 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_job.cpp +// +// Identification: src/brain/index_selection_job.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/index_selection_util.h" +#include "brain/index_selection_job.h" +#include "brain/index_selection.h" +#include "catalog/query_history_catalog.h" +#include "catalog/system_catalogs.h" +#include "optimizer/stats/stats_storage.h" + +namespace peloton { +namespace brain { + +void IndexSelectionJob::OnJobInvocation(BrainEnvironment *env) { + LOG_INFO("Started Index Suggestion Task"); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + // Analyze stats for all the tables. + // TODO: AnalyzeStatsForAllTables crashes sometimes. +// optimizer::StatsStorage *stats_storage = +// optimizer::StatsStorage::GetInstance(); +// ResultType stats_result = stats_storage->AnalyzeStatsForAllTables(txn); +// if (stats_result != ResultType::SUCCESS) { +// LOG_ERROR( +// "Cannot generate stats for table columns. Not performing index " +// "suggestion..."); +// txn_manager.AbortTransaction(txn); +// return; +// } + + // Query the catalog for new SQL queries. + // New SQL queries are the queries that were added to the system + // after the last_timestamp_ + auto &query_catalog = catalog::QueryHistoryCatalog::GetInstance(txn); + auto query_history = + query_catalog.GetQueryStringsAfterTimestamp(last_timestamp_, txn); + if (query_history->size() > num_queries_threshold_) { + LOG_INFO("Tuning threshold has crossed. Time to tune the DB!"); + + // Run the index selection. + std::vector queries; + for (auto query_pair : *query_history) { + queries.push_back(query_pair.second); + } + + // TODO: Handle multiple databases + brain::Workload workload(queries, DEFAULT_DB_NAME, txn); + LOG_INFO("Knob: Num Indexes: %zu", + env->GetIndexSelectionKnobs().num_indexes_); + LOG_INFO("Knob: Naive: %zu", + env->GetIndexSelectionKnobs().naive_enumeration_threshold_); + LOG_INFO("Knob: Num Iterations: %zu", + env->GetIndexSelectionKnobs().num_iterations_); + brain::IndexSelection is = {workload, env->GetIndexSelectionKnobs(), txn}; + brain::IndexConfiguration best_config; + is.GetBestIndexes(best_config); + + if (best_config.IsEmpty()) { + LOG_INFO("Best config is empty. No new indexes this time..."); + } + + // Get the index objects from database. + auto database_object = catalog::Catalog::GetInstance()->GetDatabaseObject( + DEFAULT_DB_NAME, txn); + auto pg_index = catalog::Catalog::GetInstance() + ->GetSystemCatalogs(database_object->GetDatabaseOid()) + ->GetIndexCatalog(); + auto cur_indexes = pg_index->GetIndexObjects(txn); + auto drop_indexes = GetIndexesToDrop(cur_indexes, best_config); + + // Drop useless indexes. + for (auto index : drop_indexes) { + LOG_DEBUG("Dropping Index: %s", index->GetIndexName().c_str()); + DropIndexRPC(database_object->GetDatabaseOid(), index.get()); + } + + // Create new indexes. + for (auto index : best_config.GetIndexes()) { + CreateIndexRPC(index.get()); + } + + last_timestamp_ = GetLatestQueryTimestamp(query_history.get()); + } else { + LOG_INFO("Index Suggestion - not performing this time"); + } + txn_manager.CommitTransaction(txn); +} + +std::vector> +IndexSelectionJob::GetIndexesToDrop( + std::unordered_map> + &index_objects, + brain::IndexConfiguration best_config) { + std::vector> ret_indexes; + // Get the existing indexes and drop them. + for (auto index : index_objects) { + auto index_name = index.second->GetIndexName(); + // TODO [vamshi]: REMOVE THIS IN THE FINAL CODE + // This is a hack for now. Add a boolean to the index catalog to + // find out if an index is a brain suggested index/user created index. + if (index_name.find(brain_suggested_index_prefix_str) != + std::string::npos) { + bool found = false; + for (auto installed_index : best_config.GetIndexes()) { + if ((index.second.get()->GetTableOid() == + installed_index.get()->table_oid) && + (index.second.get()->GetKeyAttrs() == + installed_index.get()->column_oids)) { + found = true; + } + } + // Drop only indexes which are not suggested this time. + if (!found) { + ret_indexes.push_back(index.second); + } + } + } + return ret_indexes; +} + +void IndexSelectionJob::CreateIndexRPC(brain::HypotheticalIndexObject *index) { + // TODO: Remove hardcoded database name and server end point. + capnp::EzRpcClient client("localhost:15445"); + PelotonService::Client peloton_service = client.getMain(); + + // Create the index name: concat - db_id, table_id, col_ids + std::stringstream sstream; + sstream << brain_suggested_index_prefix_str << "_" << index->db_oid << "_" + << index->table_oid << "_"; + std::vector col_oid_vector; + for (auto col : index->column_oids) { + col_oid_vector.push_back(col); + sstream << col << "_"; + } + auto index_name = sstream.str(); + + auto request = peloton_service.createIndexRequest(); + request.getRequest().setDatabaseOid(index->db_oid); + request.getRequest().setTableOid(index->table_oid); + request.getRequest().setIndexName(index_name); + request.getRequest().setUniqueKeys(false); + + auto col_list = + request.getRequest().initKeyAttrOids(index->column_oids.size()); + for (auto i = 0UL; i < index->column_oids.size(); i++) { + col_list.set(i, index->column_oids[i]); + } + + PELOTON_ASSERT(index->column_oids.size() > 0); + auto response = request.send().wait(client.getWaitScope()); +} + +void IndexSelectionJob::DropIndexRPC(oid_t database_oid, + catalog::IndexCatalogObject *index) { + // TODO: Remove hardcoded database name and server end point. + // TODO: Have to be removed when merged with tli's code. + capnp::EzRpcClient client("localhost:15445"); + PelotonService::Client peloton_service = client.getMain(); + + auto request = peloton_service.dropIndexRequest(); + request.getRequest().setDatabaseOid(database_oid); + request.getRequest().setIndexOid(index->GetIndexOid()); + + auto response = request.send().wait(client.getWaitScope()); +} + +uint64_t IndexSelectionJob::GetLatestQueryTimestamp( + std::vector> *queries) { + uint64_t latest_time = 0; + for (auto query : *queries) { + if (query.first > latest_time) { + latest_time = query.first; + } + } + return latest_time; +} +} +} diff --git a/src/brain/index_selection_job_lspi.cpp b/src/brain/index_selection_job_lspi.cpp new file mode 100644 index 00000000000..09bf179458c --- /dev/null +++ b/src/brain/index_selection_job_lspi.cpp @@ -0,0 +1,167 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_job.cpp +// +// Identification: src/brain/index_selection_job.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/lspi/lspi_tuner.h" +#include "brain/index_selection_job_lspi.h" +#include "catalog/query_history_catalog.h" +#include "catalog/system_catalogs.h" +#include "optimizer/stats/stats_storage.h" + +namespace peloton { +namespace brain { + +bool IndexSelectionJobLSPI::enable_ = false; + +IndexSelectionJobLSPI::IndexSelectionJobLSPI(BrainEnvironment *env, uint64_t num_queries_threshold) +: BrainJob(env), +last_timestamp_(0), +num_queries_threshold_(num_queries_threshold) {} + +void IndexSelectionJobLSPI::OnJobInvocation(UNUSED_ATTRIBUTE BrainEnvironment *env) { + LOG_INFO("Started Index Suggestion Task"); + if (!enable_) { + LOG_INFO("Index Suggestion - not performing this time..Yet to be enabled"); + return; + } + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + // Analyze stats for all the tables. + // TODO: AnalyzeStatsForAllTables crashes sometimes. +// optimizer::StatsStorage *stats_storage = +// optimizer::StatsStorage::GetInstance(); +// ResultType stats_result = stats_storage->AnalyzeStatsForAllTables(txn); +// if (stats_result != ResultType::SUCCESS) { +// LOG_ERROR( +// "Cannot generate stats for table columns. Not performing index " +// "suggestion..."); +// txn_manager.AbortTransaction(txn); +// return; +// } + + + + // Query the catalog for new SQL queries. + // New SQL queries are the queries that were added to the system + // after the last_timestamp_ + auto &query_catalog = catalog::QueryHistoryCatalog::GetInstance(txn); + auto query_history = + query_catalog.GetQueryStringsAfterTimestamp(last_timestamp_, txn); + if (query_history->size() > num_queries_threshold_) { + LOG_INFO("Tuning threshold has crossed. Time to tune the DB!"); + + // Run the index selection. + std::vector queries; + std::vector query_latencies; + for (auto query_pair : *query_history) { + queries.push_back(query_pair.second); + } + + if(!tuner_initialized_ && queries.size() > 0) { + tuner_initialized_ = true; + std::set ignore_table_oids; + CompressedIndexConfigUtil::GetIgnoreTables(DEFAULT_DB_NAME, + ignore_table_oids); + tuner_ = std::unique_ptr(new LSPIIndexTuner(DEFAULT_DB_NAME, + ignore_table_oids, + CandidateSelectionType::Simple, + 3)); + } + + if(tuner_initialized_) { + auto container = CompressedIndexConfigUtil::ToIndexConfiguration(*tuner_->GetConfigContainer()); + for(auto query: queries) { + auto query_latency = brain::CompressedIndexConfigUtil::WhatIfIndexCost(query, + container, + DEFAULT_DB_NAME); + query_latencies.push_back(query_latency); + LOG_DEBUG("Query: %s, What-If cost: %.5f", query.c_str(), query_latency); + } + // Run the tuner + std::set> add_set, drop_set; + tuner_->Tune(queries, query_latencies, add_set, drop_set); + for(auto &index: add_set) { + LOG_DEBUG("Adding Index: %s", index->ToString().c_str()); + CreateIndexRPC(index.get()); + } + // Skip dropping for now +// for(auto &drop_index: drop_set) { +// LOG_DEBUG("Adding Index: %s", index->ToString().c_str()); +// DropIndexRPC(drop_index.get()); +// } + } + last_timestamp_ = GetLatestQueryTimestamp(query_history.get()); + } else { + LOG_INFO("Index Suggestion - not performing this time"); + } + txn_manager.CommitTransaction(txn); +} + +void IndexSelectionJobLSPI::CreateIndexRPC(brain::HypotheticalIndexObject *index) { + // TODO: Remove hardcoded database name and server end point. + capnp::EzRpcClient client("localhost:15445"); + PelotonService::Client peloton_service = client.getMain(); + + // Create the index name: concat - db_id, table_id, col_ids + std::stringstream sstream; + sstream << brain_suggested_index_prefix_str << "_" << index->db_oid << "_" + << index->table_oid << "_"; + std::vector col_oid_vector; + for (auto col : index->column_oids) { + col_oid_vector.push_back(col); + sstream << col << "_"; + } + auto index_name = sstream.str(); + + auto request = peloton_service.createIndexRequest(); + request.getRequest().setDatabaseOid(index->db_oid); + request.getRequest().setTableOid(index->table_oid); + request.getRequest().setIndexName(index_name); + request.getRequest().setUniqueKeys(false); + + auto col_list = + request.getRequest().initKeyAttrOids(index->column_oids.size()); + for (auto i = 0UL; i < index->column_oids.size(); i++) { + col_list.set(i, index->column_oids[i]); + } + + PELOTON_ASSERT(index->column_oids.size() > 0); + auto response = request.send().wait(client.getWaitScope()); +} + +void IndexSelectionJobLSPI::DropIndexRPC(oid_t database_oid, + catalog::IndexCatalogObject *index) { + // TODO: Remove hardcoded database name and server end point. + // TODO: Have to be removed when merged with tli's code. + capnp::EzRpcClient client("localhost:15445"); + PelotonService::Client peloton_service = client.getMain(); + + auto request = peloton_service.dropIndexRequest(); + request.getRequest().setDatabaseOid(database_oid); + request.getRequest().setIndexOid(index->GetIndexOid()); + + auto response = request.send().wait(client.getWaitScope()); +} + +uint64_t IndexSelectionJobLSPI::GetLatestQueryTimestamp( + std::vector> *queries) { + uint64_t latest_time = 0; + for (auto query : *queries) { + if (query.first > latest_time) { + latest_time = query.first; + } + } + return latest_time; +} +} +} diff --git a/src/brain/index_selection_util.cpp b/src/brain/index_selection_util.cpp new file mode 100644 index 00000000000..83453d9532c --- /dev/null +++ b/src/brain/index_selection_util.cpp @@ -0,0 +1,296 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_util.cpp +// +// Identification: src/brain/index_selection_util.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/index_selection_util.h" +#include "common/logger.h" + +namespace peloton { +namespace brain { + +//===--------------------------------------------------------------------===// +// IndexObject +//===--------------------------------------------------------------------===// + +const std::string HypotheticalIndexObject::ToString() const { + std::stringstream str_stream; + str_stream << "Database: " << db_oid << "\n"; + str_stream << "Table: " << table_oid << "\n"; + str_stream << "Columns: "; + for (auto col : column_oids) { + str_stream << col << ", "; + } + str_stream << "\n"; + return str_stream.str(); +} + +bool HypotheticalIndexObject::operator==( + const HypotheticalIndexObject &obj) const { + return (db_oid == obj.db_oid && table_oid == obj.table_oid && + column_oids == obj.column_oids); +} + +bool HypotheticalIndexObject::IsCompatible( + std::shared_ptr index) const { + return (db_oid == index->db_oid) && (table_oid == index->table_oid); +} + +HypotheticalIndexObject HypotheticalIndexObject::Merge( + std::shared_ptr index) { + HypotheticalIndexObject result; + result.db_oid = db_oid; + result.table_oid = table_oid; + result.column_oids = column_oids; + for (auto column : index->column_oids) { + if (std::find(column_oids.begin(), column_oids.end(), column) == + column_oids.end()) + result.column_oids.push_back(column); + } + return result; +} + +//===--------------------------------------------------------------------===// +// IndexConfiguration +//===--------------------------------------------------------------------===// + +void IndexConfiguration::Merge(IndexConfiguration &config) { + auto indexes = config.GetIndexes(); + for (auto it = indexes.begin(); it != indexes.end(); it++) { + indexes_.insert(*it); + } +} + +void IndexConfiguration::Set(IndexConfiguration &config) { + indexes_.clear(); + auto indexes = config.GetIndexes(); + for (auto it = indexes.begin(); it != indexes.end(); it++) { + indexes_.insert(*it); + } +} + +void IndexConfiguration::RemoveIndexObject( + const std::shared_ptr &index_info) { + indexes_.erase(index_info); +} + +void IndexConfiguration::AddIndexObject( + const std::shared_ptr &index_info) { + indexes_.insert(index_info); +} + +size_t IndexConfiguration::GetIndexCount() const { return indexes_.size(); } + +bool IndexConfiguration::IsEmpty() const { return indexes_.empty(); } + +const std::set> + &IndexConfiguration::GetIndexes() const { + return indexes_; +} + +const std::string IndexConfiguration::ToString() const { + std::stringstream str_stream; + str_stream << "Num of indexes: " << GetIndexCount() << "\n"; + for (auto index : indexes_) { + str_stream << index->ToString() << " "; + } + return str_stream.str(); +} + +bool IndexConfiguration::operator==(const IndexConfiguration &config) const { + auto config_indexes = config.GetIndexes(); + return indexes_ == config_indexes; +} + +IndexConfiguration IndexConfiguration::operator-( + const IndexConfiguration &config) { + auto config_indexes = config.GetIndexes(); + + std::set> result; + std::set_difference(indexes_.begin(), indexes_.end(), config_indexes.begin(), + config_indexes.end(), + std::inserter(result, result.end())); + return IndexConfiguration(result); +} + +void IndexConfiguration::Clear() { indexes_.clear(); } + +//===--------------------------------------------------------------------===// +// IndexObjectPool +//===--------------------------------------------------------------------===// + +std::shared_ptr IndexObjectPool::GetIndexObject( + HypotheticalIndexObject &obj) { + auto ret = map_.find(obj); + if (ret != map_.end()) { + return ret->second; + } + return nullptr; +} + +std::shared_ptr IndexObjectPool::PutIndexObject( + HypotheticalIndexObject &obj) { + auto index_s_ptr = GetIndexObject(obj); + if (index_s_ptr != nullptr) return index_s_ptr; + HypotheticalIndexObject *index_copy = new HypotheticalIndexObject(); + *index_copy = obj; + index_s_ptr = std::shared_ptr(index_copy); + map_[*index_copy] = index_s_ptr; + return index_s_ptr; +} + +//===--------------------------------------------------------------------===// +// Workload +//===--------------------------------------------------------------------===// + +Workload::Workload(std::vector &queries, std::string database_name, + concurrency::TransactionContext *txn) + : database_name(database_name) { + LOG_TRACE("Initializing workload with %ld queries", queries.size()); + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, database_name)); + + // Parse and bind every query. Store the results in the workload vector. + for (auto query : queries) { + LOG_DEBUG("Query: %s", query.c_str()); + + // Create a unique_ptr to free this pointer at the end of this loop + // iteration. + auto stmt_list = std::unique_ptr( + parser::PostgresParser::ParseSQLString(query)); + PELOTON_ASSERT(stmt_list->is_valid); + // TODO[vamshi]: Only one query for now. + PELOTON_ASSERT(stmt_list->GetNumStatements() == 1); + + // Create a new shared ptr from the unique ptr because + // these queries will be referenced by multiple objects later. + // Release the unique ptr from the stmt list to avoid freeing at the end + // of this loop iteration. + auto stmt = stmt_list->PassOutStatement(0); + auto stmt_shared = std::shared_ptr(stmt.release()); + PELOTON_ASSERT(stmt_shared->GetType() != StatementType::INVALID); + + try { + // Bind the query + binder->BindNameToNode(stmt_shared.get()); + } catch (Exception e) { + LOG_DEBUG("Cannot bind this query"); + continue; + } + + // Only take the DML queries from the workload + switch (stmt_shared->GetType()) { + case StatementType::INSERT: + case StatementType::DELETE: + case StatementType::UPDATE: + case StatementType::SELECT: { + // Get all the table names referenced in the query. + std::unordered_set tables_used; + bool placeholder; + Workload::GetTableNamesReferenced(stmt_shared, tables_used, placeholder); + AddQuery(stmt_shared, tables_used); + } + default: + // Ignore other queries. + LOG_TRACE("Ignoring query: %s", stmt->GetInfo().c_str()); + } + } +} + +void Workload::GetTableNamesReferenced( + std::shared_ptr query, + std::unordered_set &table_names, bool &illegal_query) { + // populated if this query has a cross-product table references. + std::vector> *table_cp_list; + + switch (query->GetType()) { + case StatementType::INSERT: { + auto sql_statement = dynamic_cast(query.get()); + table_names.insert(sql_statement->table_ref_->GetTableName()); + break; + } + + case StatementType::DELETE: { + auto sql_statement = dynamic_cast(query.get()); + table_names.insert(sql_statement->table_ref->GetTableName()); + break; + } + + case StatementType::UPDATE: { + auto sql_statement = dynamic_cast(query.get()); + table_names.insert(sql_statement->table->GetTableName()); + break; + } + + case StatementType::SELECT: { + auto sql_statement = dynamic_cast(query.get()); + // Select can operate on more than 1 table. + switch (sql_statement->from_table->type) { + case TableReferenceType::NAME: { + // Single table. + LOG_DEBUG("Table name is %s", + sql_statement->from_table.get()->GetTableName().c_str()); + table_names.insert(sql_statement->from_table.get()->GetTableName()); + break; + } + case TableReferenceType::JOIN: { + // Get all table names in the join. + std::deque queue; + queue.push_back(sql_statement->from_table->join->left.get()); + queue.push_back(sql_statement->from_table->join->right.get()); + while (queue.size() != 0) { + auto front = queue.front(); + queue.pop_front(); + if (front == nullptr) { + continue; + } + if (front->type == TableReferenceType::JOIN) { + queue.push_back(front->join->left.get()); + queue.push_back(front->join->right.get()); + } else if (front->type == TableReferenceType::NAME) { + table_names.insert(front->GetTableName()); + } else { + illegal_query = true; + PELOTON_ASSERT(false); + } + } + break; + } + case TableReferenceType::SELECT: { + Workload::GetTableNamesReferenced( + std::shared_ptr( + sql_statement->from_table->select), + table_names, illegal_query); + break; + } + case TableReferenceType::CROSS_PRODUCT: { + // Cross product table list. + table_cp_list = &(sql_statement->from_table->list); + for (auto &table : *table_cp_list) { + table_names.insert(table->GetTableName()); + } + break; + } + case TableReferenceType::INVALID: { + LOG_ERROR("Invalid table reference"); + return; + } + } + break; + } + default: { + LOG_ERROR("Cannot handle DDL statements"); + illegal_query = true; + } + } +} + +} // namespace brain +} // namespace peloton diff --git a/src/brain/indextune/compressed_index_config.cpp b/src/brain/indextune/compressed_index_config.cpp new file mode 100644 index 00000000000..e3e586d1775 --- /dev/null +++ b/src/brain/indextune/compressed_index_config.cpp @@ -0,0 +1,359 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// compressed_index_config.cpp +// +// Identification: src/brain/indextune/compressed_index_config.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/compressed_index_config.h" + +namespace peloton { +namespace brain { + +CompressedIndexConfigContainer::CompressedIndexConfigContainer( + const std::string &database_name, const std::set &ignore_table_oids, + size_t max_index_size, catalog::Catalog *catalog, + concurrency::TransactionManager *txn_manager) + : database_name_{database_name}, + catalog_{catalog}, + txn_manager_{txn_manager}, + next_table_offset_{0}, + cur_index_config_{nullptr} { + if (catalog_ == nullptr) { + catalog_ = catalog::Catalog::GetInstance(); + catalog_->Bootstrap(); + } + + if (nullptr == txn_manager_) { + txn_manager_ = &concurrency::TransactionManagerFactory::GetInstance(); + } + + auto txn = txn_manager_->BeginTransaction(); + + const auto db_obj = catalog_->GetDatabaseObject(database_name_, txn); + database_oid_ = db_obj->GetDatabaseOid(); + LOG_DEBUG("IndexConfigContainerInit: DB OID: %d", database_oid_); + const auto table_objs = db_obj->GetTableObjects(); + LOG_DEBUG("IndexConfigContainerInit: Num Tables: %lu", table_objs.size()); + LOG_DEBUG("Ignore Tables: %lu", ignore_table_oids.size()); + + // Uniq identifier per index config + size_t next_index_id = 0; + // Scan tables to populate the internal maps + for (const auto &table_obj : table_objs) { + const auto table_oid = table_obj.first; + + if (ignore_table_oids.find(table_oid) != ignore_table_oids.end()) { + continue; + } + LOG_DEBUG("Building datastructure info for OID: %d/(of %lu tables)", table_oid, table_objs.size()); + // Enumerate configurations and prepare data structures for future usage + table_indexid_map_[table_oid] = {}; + indexid_table_map_[table_oid] = {}; + auto &indexconf_id_map = table_indexid_map_[table_oid]; + auto &id_indexconf_map = indexid_table_map_[table_oid]; + const auto col_objs = table_obj.second->GetColumnObjects(); + std::vector null_conf; + std::vector cols; + for (const auto &col_obj : col_objs) { + cols.push_back(col_obj.first); + } + LOG_DEBUG("Beginning Config Enumeration for OID: %d", table_oid); + EnumerateConfigurations(cols, max_index_size, indexconf_id_map, + id_indexconf_map, null_conf, next_index_id); + LOG_DEBUG("Completed Config Enumeration for OID: %d", table_oid); + table_offset_map_[table_oid] = next_table_offset_; + table_offset_reverse_map_[next_table_offset_] = table_oid; + next_table_offset_ += indexconf_id_map.size(); + } + + cur_index_config_ = std::unique_ptr>( + new boost::dynamic_bitset<>(next_table_offset_)); + + // Scan tables to populate current config + for (const auto &table_obj : table_objs) { + const auto table_oid = table_obj.first; + + if (ignore_table_oids.find(table_oid) != ignore_table_oids.end()) { + continue; + } + + const auto index_objs = table_obj.second->GetIndexObjects(); + if (index_objs.empty()) { + SetBit(table_offset_map_.at(table_oid)); + } else { + for (const auto &index_obj : index_objs) { + const auto &indexed_cols = index_obj.second->GetKeyAttrs(); + + std::vector col_oids(indexed_cols); + auto idx_obj = std::make_shared( + database_oid_, table_oid, col_oids); + + const auto global_index_offset = GetGlobalOffset(idx_obj); + + SetBit(global_index_offset); + } + } + } + + txn_manager_->CommitTransaction(txn); +} + +void CompressedIndexConfigContainer::EnumerateConfigurations( + const std::vector &cols, size_t max_index_size, + std::map, size_t> &indexconf_id_map, + std::map> &id_indexconf_map, + std::vector &index_conf, size_t &next_id) { + indexconf_id_map[index_conf] = next_id; + id_indexconf_map[next_id] = index_conf; + next_id++; + if (index_conf.size() == std::min(max_index_size, cols.size())) return; + for (auto col : cols) { + if (std::find(index_conf.begin(), index_conf.end(), col) == + index_conf.end()) { + index_conf.push_back(col); + EnumerateConfigurations(cols, max_index_size, indexconf_id_map, + id_indexconf_map, index_conf, next_id); + index_conf.pop_back(); + } + } +} + + +// TODO: Add HypotheticalIndexObject set to Add/Drop index RPC call here +void CompressedIndexConfigContainer::AdjustIndexes( + const boost::dynamic_bitset<> &new_bitset, + std::set>& add_set, + std::set>& drop_set) { + + boost::dynamic_bitset<> &ori_bitset = *cur_index_config_; + + const auto drop_bitset = ori_bitset - new_bitset; + + for (size_t current_bit = drop_bitset.find_first(); + current_bit != boost::dynamic_bitset<>::npos; + current_bit = drop_bitset.find_next(current_bit)) { + // 1. unset current bit + UnsetBit(current_bit); + + // 2. add to the drop_set + drop_set.insert(GetIndex(current_bit)); + + } + + const auto add_bitset = new_bitset - ori_bitset; + + for (size_t current_bit = add_bitset.find_first(); + current_bit != boost::dynamic_bitset<>::npos; + current_bit = add_bitset.find_next(current_bit)) { + // 1. set current bit + SetBit(current_bit); + + // 2. add to add_set + add_set.insert(GetIndex(current_bit)); + } +} + +//**Setter fns**/ +void CompressedIndexConfigContainer::SetBit( + const std::shared_ptr &idx_object) { + size_t offset = GetGlobalOffset(idx_object); + cur_index_config_->set(offset); +} + +void CompressedIndexConfigContainer::SetBit(size_t offset) { + cur_index_config_->set(offset); +} + +void CompressedIndexConfigContainer::UnsetBit( + const std::shared_ptr &idx_object) { + size_t offset = GetGlobalOffset(idx_object); + cur_index_config_->set(offset, false); +} + +void CompressedIndexConfigContainer::UnsetBit(size_t offset) { + cur_index_config_->set(offset, false); +} + +//**Getter fns**/ + +size_t CompressedIndexConfigContainer::GetGlobalOffset( + const std::shared_ptr &index_obj) const { + oid_t table_oid = index_obj->table_oid; + if(index_obj->column_oids.empty()) { + return table_offset_map_.at(table_oid); + } else { + return table_indexid_map_.at(table_oid).at(index_obj->column_oids); + } + +} + +bool CompressedIndexConfigContainer::IsSet( + const std::shared_ptr &index_obj) const { + size_t offset = GetGlobalOffset(index_obj); + return cur_index_config_->test(offset); +} + +bool CompressedIndexConfigContainer::IsSet(const size_t offset) const { + return cur_index_config_->test(offset); +} + +std::shared_ptr +CompressedIndexConfigContainer::GetIndex(size_t global_offset) const { + const oid_t table_oid = GetCurrentTableOID(global_offset); + std::vector col_oids = + indexid_table_map_.at(table_oid).at(global_offset); + + return std::make_shared(database_oid_, table_oid, + col_oids); +} + +size_t CompressedIndexConfigContainer::GetConfigurationCount() const { + return next_table_offset_; +} + +const boost::dynamic_bitset<> + *CompressedIndexConfigContainer::GetCurrentIndexConfig() const { + return cur_index_config_.get(); +} + +concurrency::TransactionManager * +CompressedIndexConfigContainer::GetTransactionManager() { + return txn_manager_; +} + +catalog::Catalog *CompressedIndexConfigContainer::GetCatalog() { + return catalog_; +} + +std::string CompressedIndexConfigContainer::GetDatabaseName() const { + return database_name_; +} + +size_t CompressedIndexConfigContainer::GetTableOffsetStart( + oid_t table_oid) const { + return table_offset_map_.at(table_oid); +} + +size_t CompressedIndexConfigContainer::GetTableOffsetEnd( + oid_t table_oid) const { + size_t start_idx = GetTableOffsetStart(table_oid); + return GetNextTableIdx(start_idx); +} + +oid_t CompressedIndexConfigContainer::GetCurrentTableOID(size_t idx) const { + auto gteq_iter = table_offset_reverse_map_.lower_bound(idx); + if(gteq_iter->first == idx) { + // Idx = Offset corresponding to table OID + return gteq_iter->second; + } else { + // Idx = Offset corresponding to table OID one after the one we want + gteq_iter--; + return gteq_iter->second; + } +} + +size_t CompressedIndexConfigContainer::GetNextTableIdx(size_t start_idx) const { + auto next_tbl_offset_iter = table_offset_reverse_map_.upper_bound(start_idx); + if (next_tbl_offset_iter == table_offset_reverse_map_.end()) { + return GetConfigurationCount(); + } else { + return next_tbl_offset_iter->first; + } +} + +std::string CompressedIndexConfigContainer::ToString() const { + // First get the entire bitset + std::stringstream str_stream; + std::string bitset_str; + boost::to_string(*GetCurrentIndexConfig(), bitset_str); + // since bitset follows MSB <---- LSB + std::reverse(bitset_str.begin(), bitset_str.end()); + str_stream << "Database: " << database_name_ << std::endl; + str_stream << "Compressed Index Representation: " << bitset_str << std::endl; + for (auto tbl_offset_iter = table_offset_reverse_map_.begin(); + tbl_offset_iter != table_offset_reverse_map_.end(); ++tbl_offset_iter) { + size_t start_idx = tbl_offset_iter->first; + size_t end_idx = GetNextTableIdx(start_idx); + oid_t table_oid = tbl_offset_iter->second; + str_stream << "Table OID: " << table_oid << " Compressed Section: " + << bitset_str.substr(start_idx, end_idx - start_idx) + << std::endl; + size_t set_idx = start_idx; + while (set_idx != boost::dynamic_bitset<>::npos && set_idx < end_idx) { + str_stream << "("; + for (auto col_oid : indexid_table_map_.at(table_oid).at(set_idx)) { + str_stream << col_oid << ","; + } + str_stream << "):" << set_idx << std::endl; + set_idx = GetNextSetIndexConfig(set_idx); + } + } + return str_stream.str(); +} + +std::string CompressedIndexConfigContainer::ToString( + const boost::dynamic_bitset<> &bs) const { + // First get the entire bitset + std::stringstream str_stream; + std::string bitset_str; + boost::to_string(bs, bitset_str); + // since bitset follows MSB <---- LSB + std::reverse(bitset_str.begin(), bitset_str.end()); + str_stream << "Database: " << database_name_ << std::endl; + str_stream << "Compressed Index Representation: " << bitset_str << std::endl; + for (auto tbl_offset_iter = table_offset_reverse_map_.begin(); + tbl_offset_iter != table_offset_reverse_map_.end(); ++tbl_offset_iter) { + size_t start_idx = tbl_offset_iter->first; + size_t end_idx = GetNextTableIdx(start_idx); + oid_t table_oid = tbl_offset_iter->second; + str_stream << "Table OID: " << table_oid << " Compressed Section: " + << bitset_str.substr(start_idx, end_idx - start_idx) + << std::endl; + size_t set_idx = start_idx; + while (set_idx != boost::dynamic_bitset<>::npos && set_idx < end_idx) { + str_stream << "("; + for (auto col_oid : indexid_table_map_.at(table_oid).at(set_idx)) { + str_stream << col_oid << ","; + } + str_stream << "):" << set_idx << std::endl; + set_idx = bs.find_next(set_idx); + } + } + return str_stream.str(); +} + +size_t CompressedIndexConfigContainer::GetNumIndexes(oid_t table_oid) const { + size_t start_idx = GetTableOffsetStart(table_oid); + size_t end_idx = GetNextTableIdx(start_idx); + if (IsSet(start_idx)) { + return 0; + } else { + size_t idx = GetNextSetIndexConfig(start_idx); + size_t count = 0; + while (idx != boost::dynamic_bitset<>::npos && idx < end_idx) { + count += 1; + idx = GetNextSetIndexConfig(idx); + } + return count; + } +} + +size_t CompressedIndexConfigContainer::GetNextSetIndexConfig( + size_t from_idx) const { + return cur_index_config_->find_next(from_idx); +} + +bool CompressedIndexConfigContainer::EmptyConfig( + peloton::oid_t table_oid) const { + size_t table_offset = table_offset_map_.at(table_oid); + return IsSet(table_offset); +} + +} // namespace brain +} // namespace peloton \ No newline at end of file diff --git a/src/brain/indextune/compressed_index_config_util.cpp b/src/brain/indextune/compressed_index_config_util.cpp new file mode 100644 index 00000000000..3bb6edebedc --- /dev/null +++ b/src/brain/indextune/compressed_index_config_util.cpp @@ -0,0 +1,351 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// compressed_index_config_util.cpp +// +// Identification: src/brain/indextune/compressed_index_config_util.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/compressed_index_config_util.h" +#include "brain/index_selection.h" +#include "brain/what_if_index.h" + +namespace peloton { +namespace brain { + +void CompressedIndexConfigUtil::AddCandidates( + CompressedIndexConfigContainer &container, const std::string &query, + boost::dynamic_bitset<> &add_candidates, CandidateSelectionType cand_sel_type, + size_t max_index_size, IndexSelectionKnobs knobs) { + add_candidates = boost::dynamic_bitset<>(container.GetConfigurationCount()); + // First add all {} empty index bits + for (const auto it : container.table_offset_map_) { + const auto table_offset = it.second; + add_candidates.set(table_offset); + } + if(cand_sel_type == CandidateSelectionType::AutoAdmin) { + // Generate autoadmin candidates + IndexConfiguration best_config; + auto txn = container.GetTransactionManager()->BeginTransaction(); + std::vector queries = {query}; + brain::Workload w = {queries, container.GetDatabaseName(), txn}; + brain::IndexSelection is = {w, knobs, txn}; + is.GetBestIndexes(best_config); + container.GetTransactionManager()->CommitTransaction(txn); + for(const auto& hypot_index_obj: best_config.GetIndexes()) { + MarkPrefixClosure(container, add_candidates, hypot_index_obj); + } + } else if (cand_sel_type == CandidateSelectionType::Simple || cand_sel_type == CandidateSelectionType::Exhaustive) { + auto sql_stmt_list = ToBindedSqlStmtList(container, query); + auto txn = container.GetTransactionManager()->BeginTransaction(); + container.GetCatalog()->GetDatabaseObject(container.GetDatabaseName(), txn); + + std::vector indexable_cols_vector = + planner::PlanUtil::GetIndexableColumns(txn->catalog_cache, + std::move(sql_stmt_list), + container.GetDatabaseName()); + container.GetTransactionManager()->CommitTransaction(txn); + + if (cand_sel_type == CandidateSelectionType::Simple) { + for (const auto &each_triplet : indexable_cols_vector) { + const auto db_oid = std::get<0>(each_triplet); + const auto table_oid = std::get<1>(each_triplet); + const auto col_oid = std::get<2>(each_triplet); + + std::vector col_oids = {col_oid}; + auto idx_new = std::make_shared( + db_oid, table_oid, col_oids); + + SetBit(container, add_candidates, idx_new); + } + } else if (cand_sel_type == CandidateSelectionType::Exhaustive) { + // Aggregate all columns in the same table + std::unordered_map aggregate_map; + for (const auto &each_triplet : indexable_cols_vector) { + const auto db_oid = std::get<0>(each_triplet); + const auto table_oid = std::get<1>(each_triplet); + const auto col_oid = std::get<2>(each_triplet); + + if (aggregate_map.find(table_oid) == aggregate_map.end()) { + aggregate_map[table_oid] = brain::HypotheticalIndexObject(); + aggregate_map.at(table_oid).db_oid = db_oid; + aggregate_map.at(table_oid).table_oid = table_oid; + } + + aggregate_map.at(table_oid).column_oids.push_back(col_oid); + } + + const auto db_oid = container.GetDatabaseOID(); + + for (const auto it : aggregate_map) { + const auto table_oid = it.first; + const auto &column_oids = it.second.column_oids; + + // Insert empty index + add_candidates.set(container.GetTableOffsetStart(table_oid)); + + std::vector index_conf; + + // Insert index consisting of up to max_index_size columns + PermuateConfigurations(container, column_oids, max_index_size, index_conf, + add_candidates, db_oid, table_oid); + } + } + } +} + +void CompressedIndexConfigUtil::DropCandidates( + CompressedIndexConfigContainer &container, const std::string &query, + boost::dynamic_bitset<> &drop_candidates) { + drop_candidates = boost::dynamic_bitset<>(container.GetConfigurationCount()); + + auto sql_stmt_list = ToBindedSqlStmtList(container, query); + auto sql_stmt = sql_stmt_list->GetStatement(0); + + auto txn = container.GetTransactionManager()->BeginTransaction(); + container.GetCatalog()->GetDatabaseObject(container.GetDatabaseName(), txn); + std::vector affected_indexes = + planner::PlanUtil::GetAffectedIndexes(txn->catalog_cache, *sql_stmt, + true); + for (const auto &col_triplet : affected_indexes) { + auto idx_obj = ConvertIndexTriplet(container, col_triplet); + SetBit(container, drop_candidates, idx_obj); + } + container.GetTransactionManager()->CommitTransaction(txn); +} + +std::shared_ptr +CompressedIndexConfigUtil::ConvertIndexTriplet( + CompressedIndexConfigContainer &container, + const planner::col_triplet &idx_triplet) { + const auto db_oid = std::get<0>(idx_triplet); + const auto table_oid = std::get<1>(idx_triplet); + const auto idx_oid = std::get<2>(idx_triplet); + + auto txn = container.GetTransactionManager()->BeginTransaction(); + const auto db_obj = container.GetCatalog()->GetDatabaseObject(db_oid, txn); + const auto table_obj = db_obj->GetTableObject(table_oid); + const auto idx_obj = table_obj->GetIndexObject(idx_oid); + const auto col_oids = idx_obj->GetKeyAttrs(); + std::vector input_oids(col_oids); + + container.GetTransactionManager()->CommitTransaction(txn); + + return std::make_shared(db_oid, table_oid, + input_oids); +} + +std::unique_ptr +CompressedIndexConfigUtil::ToBindedSqlStmtList( + CompressedIndexConfigContainer &container, + const std::string &query_string) { + auto txn = container.GetTransactionManager()->BeginTransaction(); + auto &peloton_parser = parser::PostgresParser::GetInstance(); + auto sql_stmt_list = peloton_parser.BuildParseTree(query_string); + auto sql_stmt = sql_stmt_list->GetStatement(0); + auto bind_node_visitor = + binder::BindNodeVisitor(txn, container.GetDatabaseName()); + bind_node_visitor.BindNameToNode(sql_stmt); + container.GetTransactionManager()->CommitTransaction(txn); + + return sql_stmt_list; +} + +std::unique_ptr> +CompressedIndexConfigUtil::GenerateBitSet( + const CompressedIndexConfigContainer &container, + const std::vector> + &idx_objs) { + auto result = std::unique_ptr>( + new boost::dynamic_bitset<>(container.GetConfigurationCount())); + + for (const auto &idx_obj : idx_objs) { + SetBit(container, *result, idx_obj); + } + + return result; +} + +void CompressedIndexConfigUtil::SetBit( + const CompressedIndexConfigContainer &container, + boost::dynamic_bitset<> &bitmap, + const std::shared_ptr &idx_object) { + size_t offset = container.GetGlobalOffset(idx_object); + bitmap.set(offset); +} + +void CompressedIndexConfigUtil::ConstructQueryConfigFeature( + const boost::dynamic_bitset<> &curr_config_set, + const boost::dynamic_bitset<> &add_candidate_set, + const boost::dynamic_bitset<> &drop_candidate_set, + vector_eig &query_config_vec) { + size_t num_configs = curr_config_set.size(); + query_config_vec = vector_eig::Zero(2 * num_configs); + + // Featurization mechanism: Add candidates + // 1 if idx belongs to add cand set + current state config + // -1 if idx belongs to add cand set + not in curr state config + // 0 otherwise + size_t offset_rec = 0; + // TODO(saatviks): Disabling this for now +// query_config_vec[offset_rec] = 1.0; + size_t config_id_rec = add_candidate_set.find_first(); + while (config_id_rec != boost::dynamic_bitset<>::npos) { + if (curr_config_set.test(config_id_rec)) { + query_config_vec[offset_rec + config_id_rec] = 1.0f; + } else { + query_config_vec[offset_rec + config_id_rec] = -1.0f; + } + config_id_rec = add_candidate_set.find_next(config_id_rec); + } + + // Featurization mechanism: Drop candidates + // 1 if idx belongs to drop cand set + current state config + // 0 otherwise + size_t offset_drop = num_configs; + size_t config_id_drop = drop_candidate_set.find_first(); + // TODO(saatviks): Disabling this for now +// query_config_vec[offset_drop] = 1.0; + while (config_id_drop != boost::dynamic_bitset<>::npos) { + if (curr_config_set.test(config_id_drop)) { + query_config_vec[offset_drop + config_id_drop] = 1.0f; + } + config_id_drop = drop_candidate_set.find_next(config_id_drop); + } +} + +void CompressedIndexConfigUtil::GetIgnoreTables( + const std::string &db_name, std::set &ori_table_oids) { + peloton::concurrency::TransactionManager *txn_manager = + &concurrency::TransactionManagerFactory::GetInstance(); + + auto txn = txn_manager->BeginTransaction(); + const auto table_objs = catalog::Catalog::GetInstance() + ->GetDatabaseObject(db_name, txn) + ->GetTableObjects(); + + for (const auto &it : table_objs) { + auto table_name = it.second->GetTableName(); + if(table_name.find("pg_") != 0) continue; + LOG_DEBUG("Ignoring table %s", it.second->GetTableName().c_str()); + ori_table_oids.insert(it.first); + } + + txn_manager->CommitTransaction(txn); +} + +void CompressedIndexConfigUtil::ConstructStateConfigFeature( + const boost::dynamic_bitset<> &config_set, vector_eig &config_vec) { + // Note that the representation is reversed - but this should not affect + // anything + config_vec = -vector_eig::Ones(config_set.size()); + size_t config_id = config_set.find_first(); + while (config_id != boost::dynamic_bitset<>::npos) { + config_vec[config_id] = 1.0; + config_id = config_set.find_next(config_id); + } +} + +IndexConfiguration CompressedIndexConfigUtil::ToIndexConfiguration( + const CompressedIndexConfigContainer &container) { + brain::IndexConfiguration index_config; + + for (const auto it : container.table_offset_map_) { + const auto start_idx = it.second; + size_t end_idx = container.GetNextTableIdx(start_idx); + auto idx = start_idx; + while (idx != boost::dynamic_bitset<>::npos && idx < end_idx) { + auto hypo_index_obj = container.GetIndex(idx); + index_config.AddIndexObject(hypo_index_obj); + idx = container.GetNextSetIndexConfig(idx); + } + } + + return index_config; +} + +void CompressedIndexConfigUtil::PermuateConfigurations( + const CompressedIndexConfigContainer &container, + const std::vector &cols, size_t max_index_size, + std::vector &index_conf, boost::dynamic_bitset<> &bitset, + oid_t db_oid, oid_t table_oid) { + if (index_conf.size() <= std::min(max_index_size, cols.size())) { + auto idx_new = std::make_shared( + db_oid, table_oid, index_conf); + SetBit(container, bitset, idx_new); + } + for (auto col : cols) { + if (std::find(index_conf.begin(), index_conf.end(), col) == + index_conf.end()) { + index_conf.push_back(col); + PermuateConfigurations(container, cols, max_index_size, index_conf, + bitset, db_oid, table_oid); + index_conf.pop_back(); + } + } +} + +void CompressedIndexConfigUtil::MarkPrefixClosure(const CompressedIndexConfigContainer &container, + boost::dynamic_bitset<> &bitset, + const std::shared_ptr &hypot_index_obj) { + auto &col_oids = hypot_index_obj->column_oids; + for(size_t i = 1; i <= hypot_index_obj->column_oids.size(); i++) { + auto index_conf = std::vector(col_oids.begin(), col_oids.begin() + i); + auto idx_new = std::make_shared( + hypot_index_obj->db_oid, hypot_index_obj->table_oid, index_conf); + SetBit(container, bitset, idx_new); + } +} + +std::string CompressedIndexConfigUtil::ToString( + std::vector config_vector) { + std::stringstream str_stream; + str_stream << "("; + for (auto idx : config_vector) { + str_stream << idx << ","; + } + str_stream << ")" << std::endl; + return str_stream.str(); +} + +std::string CompressedIndexConfigUtil::ToString(peloton::vector_eig v) { + std::stringstream str_stream; + str_stream << v.transpose() << std::endl; + return str_stream.str(); +} + +double CompressedIndexConfigUtil::WhatIfIndexCost(std::string query, + brain::IndexConfiguration &config, + std::string database_name) { + std::unique_ptr stmt_list( + parser::PostgresParser::ParseSQLString(query)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, database_name)); + + // Get the first statement. + auto sql_statement = std::shared_ptr( + stmt_list->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement.get()); + auto tree = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + database_name, txn); + double cost; + if(tree == nullptr) { + cost = 0.; + } else { + cost = tree->cost; + } + txn_manager.CommitTransaction(txn); + return cost; +} + +} // namespace brain +} // namespace peloton \ No newline at end of file diff --git a/src/brain/indextune/lspi/lspi_tuner.cpp b/src/brain/indextune/lspi/lspi_tuner.cpp new file mode 100644 index 00000000000..9f63569ea27 --- /dev/null +++ b/src/brain/indextune/lspi/lspi_tuner.cpp @@ -0,0 +1,173 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// lspi_tuner.cpp +// +// Identification: src/brain/indextune/lspi/lspi_tuner.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/lspi/lspi_tuner.h" + +namespace peloton { +namespace brain { +LSPIIndexTuner::LSPIIndexTuner( + const std::string &db_name, const std::set &ignore_table_oids, + CandidateSelectionType cand_sel_type, size_t max_index_size, + double variance_init, double reg_coeff, + peloton::catalog::Catalog *catalog, + peloton::concurrency::TransactionManager *txn_manager) + : db_name_{db_name}, + cand_sel_type_{cand_sel_type}, + max_index_size_{max_index_size}{ + index_config_ = std::unique_ptr( + new CompressedIndexConfigContainer(db_name, ignore_table_oids, + max_index_size, catalog, + txn_manager)); + size_t feat_len = index_config_->GetConfigurationCount(); + LOG_DEBUG("feat_len: %zu", feat_len); + rlse_model_ = std::unique_ptr(new RLSEModel(2 * feat_len, variance_init, reg_coeff)); + lstdq_model_ = std::unique_ptr(new LSTDQModel(feat_len)); + prev_config_vec = vector_eig::Zero(feat_len); + // Empty config + prev_config_vec[0] = 1.0; +} + +const CompressedIndexConfigContainer *LSPIIndexTuner::GetConfigContainer() + const { + return index_config_.get(); +} + +void LSPIIndexTuner::Tune(const std::vector &queries, + const std::vector &query_costs, + std::set>& add_set, + std::set>& drop_set) { + size_t num_queries = queries.size(); + std::vector> add_candidate_sets; + std::vector> drop_candidate_sets; + double cost_avg = 0.0; + const boost::dynamic_bitset<> &curr_config_set = + *index_config_->GetCurrentIndexConfig(); + // Be careful about not duplicating bitsets anywhere since they can + // be potentially huge + // Step 1: Populate the add and drop candidates per query + boost::dynamic_bitset<> add_candidate_set, drop_candidate_set; + for (size_t i = 0; i < num_queries; i++) { + CompressedIndexConfigUtil::AddCandidates(*index_config_, queries[i], + add_candidate_set, cand_sel_type_, + max_index_size_); + add_candidate_sets.push_back(std::move(add_candidate_set)); + CompressedIndexConfigUtil::DropCandidates(*index_config_, queries[i], + drop_candidate_set); + drop_candidate_sets.push_back(std::move(drop_candidate_set)); + cost_avg += query_costs[i]; + } + cost_avg /= num_queries; + // Step 2: Update the RLSE model with the new samples + for (size_t i = 0; i < num_queries; i++) { + vector_eig query_config_feat; + CompressedIndexConfigUtil::ConstructQueryConfigFeature( + curr_config_set, add_candidate_sets[i], drop_candidate_sets[i], + query_config_feat); + rlse_model_->Update(query_config_feat, query_costs[i]); + } + // Step 3: Iterate through the queries/latencies and obtain a new optimal + // config + auto optimal_config_set = curr_config_set; + for (size_t i = 0; i < num_queries; i++) { + FindOptimalConfig(curr_config_set, add_candidate_sets[i], + drop_candidate_sets[i], optimal_config_set); + } + + vector_eig new_config_vec; + CompressedIndexConfigUtil::ConstructStateConfigFeature(optimal_config_set, + new_config_vec); + // Step 4: Update the LSPI model based on current most optimal query config + lstdq_model_->Update(prev_config_vec, new_config_vec, cost_avg); + + // Step 5: Adjust to the most optimal query config + index_config_->AdjustIndexes(optimal_config_set, add_set, drop_set); + // TODO(saatviks, weichenl): Is this a heavy op? + PELOTON_ASSERT(optimal_config_set == *index_config_->GetCurrentIndexConfig()); +} + +void LSPIIndexTuner::FindOptimalConfig( + const boost::dynamic_bitset<> &curr_config_set, + const boost::dynamic_bitset<> &add_candidate_set, + const boost::dynamic_bitset<> &drop_candidate_set, + boost::dynamic_bitset<> &optimal_config_set) { + // Iterate through add candidates + size_t index_id_rec = add_candidate_set.find_first(); + vector_eig query_config_vec, config_vec; + // Find current cost + CompressedIndexConfigUtil::ConstructQueryConfigFeature( + curr_config_set, add_candidate_set, drop_candidate_set, query_config_vec); + CompressedIndexConfigUtil::ConstructStateConfigFeature( + *index_config_->GetCurrentIndexConfig(), config_vec); + double max_exec_cost = rlse_model_->Predict(query_config_vec); + double max_config_cost = lstdq_model_->Predict(config_vec); + double max_cost = max_exec_cost + max_config_cost; + while (index_id_rec != boost::dynamic_bitset<>::npos) { + if (!optimal_config_set.test(index_id_rec)) { + // Make a copy of the current config + auto hypothetical_config = boost::dynamic_bitset<>(curr_config_set); + // Set the corresponding bit for candidate + hypothetical_config.set(index_id_rec); + LOG_DEBUG("Prev: %s", index_config_->ToString(curr_config_set).c_str()); + LOG_DEBUG("Trying Add Cand: %s", + index_config_->ToString(hypothetical_config).c_str()); + LOG_DEBUG("QueryConfig Vector: %s", + CompressedIndexConfigUtil::ToString(query_config_vec).c_str()); + LOG_DEBUG("RLSE Wts: %s", + CompressedIndexConfigUtil::ToString(rlse_model_->GetWeights()).c_str()); + // Construct the query-state and state feature + CompressedIndexConfigUtil::ConstructQueryConfigFeature( + hypothetical_config, add_candidate_set, drop_candidate_set, + query_config_vec); + CompressedIndexConfigUtil::ConstructStateConfigFeature( + hypothetical_config, config_vec); + // Get the new hypothetical configs overall cost + double hypothetical_exec_cost = rlse_model_->Predict(query_config_vec); + double hypothetical_config_cost = lstdq_model_->Predict(config_vec); + double cost = hypothetical_config_cost + hypothetical_exec_cost; + + LOG_DEBUG("Candidate Cost: %f, Max Cost: %f", cost, max_cost); + if (cost < max_cost) { + optimal_config_set.set(index_id_rec); + } + } + // We are done go to next + index_id_rec = add_candidate_set.find_next(index_id_rec); + } + // Iterate through drop candidates + size_t index_id_drop = drop_candidate_set.find_first(); + while (index_id_drop != boost::dynamic_bitset<>::npos) { + if (optimal_config_set.test(index_id_drop)) { + // Make a copy of the current config + auto hypothetical_config = curr_config_set; + hypothetical_config.reset(index_id_drop); + CompressedIndexConfigUtil::ConstructQueryConfigFeature( + hypothetical_config, add_candidate_set, drop_candidate_set, + query_config_vec); + CompressedIndexConfigUtil::ConstructStateConfigFeature( + hypothetical_config, config_vec); + double hypothetical_exec_cost = rlse_model_->Predict(query_config_vec); + double hypothetical_config_cost = lstdq_model_->Predict(config_vec); + double cost = hypothetical_config_cost + hypothetical_exec_cost; + LOG_DEBUG("Prev: %s", index_config_->ToString(curr_config_set).c_str()); + LOG_DEBUG("Trying Drop Cand: %s", + index_config_->ToString(hypothetical_config).c_str()); + LOG_DEBUG("Candidate Cost: %f, Max Cost: %f", cost, max_cost); + if (cost < max_cost) { + optimal_config_set.reset(index_id_drop); + } + } + // We are done go to next + index_id_drop = drop_candidate_set.find_next(index_id_drop); + } +} +} // namespace brain +} // namespace peloton \ No newline at end of file diff --git a/src/brain/indextune/lspi/lstdq.cpp b/src/brain/indextune/lspi/lstdq.cpp new file mode 100644 index 00000000000..d54c539c16e --- /dev/null +++ b/src/brain/indextune/lspi/lstdq.cpp @@ -0,0 +1,42 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// lstd.cpp +// +// Identification: src/brain/indextune/lspi/lstd.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/lspi/lstdq.h" + +namespace peloton { +namespace brain { +LSTDQModel::LSTDQModel(size_t feat_len, double variance_init, double gamma) + : feat_len_(feat_len), gamma_(gamma) { + model_variance_ = matrix_eig::Zero(feat_len, feat_len); + model_variance_.diagonal().array() += variance_init; + weights_ = vector_eig::Zero(feat_len); +} + +// TODO(saatvik): Recheck and better variable naming +void LSTDQModel::Update(const vector_eig &state_feat_curr, + const vector_eig &state_feat_next, double true_cost) { + vector_eig var1 = state_feat_curr - state_feat_next * gamma_; + double var2 = 1 + (var1.transpose() * model_variance_).dot(state_feat_curr); + matrix_eig var3 = + model_variance_ * (state_feat_curr)*var1.transpose() * model_variance_; + double epsilon = true_cost - var1.dot(weights_); + vector_eig error = model_variance_ * state_feat_curr * (epsilon / var2); + weights_ += error; + model_variance_ -= var3 / var2; + // TODO(saatvik): Log error here? +} + +double LSTDQModel::Predict(const vector_eig &state_feat) const { + return gamma_ * weights_.dot(state_feat); +} +} // namespace brain +} // namespace peloton diff --git a/src/brain/indextune/lspi/rlse.cpp b/src/brain/indextune/lspi/rlse.cpp new file mode 100644 index 00000000000..f1bb9f46805 --- /dev/null +++ b/src/brain/indextune/lspi/rlse.cpp @@ -0,0 +1,48 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// rlse.cpp +// +// Identification: src/brain/indextune/lspi/rlse.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/lspi/rlse.h" +#include + +namespace peloton { +namespace brain { +RLSEModel::RLSEModel(size_t feat_len, double variance_init, double reg_coeff, bool random_weights) + : feat_len_(feat_len), + reg_coeff_(reg_coeff) { + model_variance_ = matrix_eig::Zero(feat_len, feat_len); + model_variance_.diagonal().array() += variance_init; + if (random_weights) { + weights_ = vector_eig::Random(feat_len); + float min_weight = weights_.minCoeff(); + float max_weight = weights_.maxCoeff(); + weights_ = 2*(weights_.array() - min_weight)/(max_weight - min_weight) - 1; + } else { + weights_ = vector_eig::Zero(feat_len); + } + +} + +void RLSEModel::Update(const vector_eig &feat_vector, double true_val) { + double err = Predict(feat_vector) - true_val; + double gamma = + reg_coeff_ + (feat_vector.transpose() * model_variance_).dot(feat_vector); + matrix_eig H = model_variance_ * (1 / gamma); + model_variance_ -= model_variance_ * feat_vector * (feat_vector.transpose()) * + model_variance_; + weights_ -= (H * feat_vector) * err; +} + +double RLSEModel::Predict(const vector_eig &feat_vector) const { + return weights_.dot(feat_vector); +} +} // namespace brain +} // namespace peloton diff --git a/src/brain/what_if_index.cpp b/src/brain/what_if_index.cpp new file mode 100644 index 00000000000..727b203a02d --- /dev/null +++ b/src/brain/what_if_index.cpp @@ -0,0 +1,116 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// what_if_index.cpp +// +// Identification: src/brain/what_if_index.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/what_if_index.h" +#include "optimizer/operators.h" +#include "traffic_cop/traffic_cop.h" + +namespace peloton { +namespace brain { + +unsigned long WhatIfIndex::index_seq_no = 0; + +std::unique_ptr +WhatIfIndex::GetCostAndBestPlanTree(std::shared_ptr query, + IndexConfiguration &config, + std::string database_name, + concurrency::TransactionContext *txn) { + // Find all the tables that are referenced in the parsed query. + std::unordered_set tables_used; + bool illegal_query = false; + Workload::GetTableNamesReferenced(query, tables_used, illegal_query); + if(illegal_query) return nullptr; + else { + return GetCostAndBestPlanTree(std::make_pair(query, tables_used), config, + database_name, txn); + } +} + +std::unique_ptr +WhatIfIndex::GetCostAndBestPlanTree( + std::pair, + std::unordered_set> query, + IndexConfiguration &config, std::string database_name, + concurrency::TransactionContext *txn) { + LOG_TRACE("***** GetCostAndBestPlanTree **** \n"); + // Load the indexes into the cache for each table so that the optimizer uses + // the indexes that we provide. + for (auto table_name : query.second) { + // Load the tables into cache. + + // TODO: Hard coding the schema name for build to pass. + auto table_object = catalog::Catalog::GetInstance()->GetTableObject( + database_name, "public", table_name, txn); + + // Evict all the existing real indexes and + // insert the what-if indexes into the cache. + table_object->EvictAllIndexObjects(); + + // Upon evict index objects, the index set becomes + // invalid. Set it to valid so that we don't query + // the catalog again while doing query optimization later. + table_object->SetValidIndexObjects(true); + + auto index_set = config.GetIndexes(); + for (auto it = index_set.begin(); it != index_set.end(); it++) { + auto index = *it; + if (index->table_oid == table_object->GetTableOid()) { + auto index_catalog_obj = CreateIndexCatalogObject(index.get()); + table_object->InsertIndexObject(index_catalog_obj); + LOG_TRACE("Created a new hypothetical index %d on table: %d", + index_catalog_obj->GetIndexOid(), + index_catalog_obj->GetTableOid()); + for (auto col : index_catalog_obj->GetKeyAttrs()) { + (void)col; // for debug mode. + LOG_TRACE("Cols: %d", col); + } + } + } + } + + // Perform query optimization with the hypothetical indexes + optimizer::Optimizer optimizer; + auto opt_info_obj = optimizer.GetOptimizedPlanInfo(query.first, txn); + + LOG_TRACE("Query: %s", query.first->GetInfo().c_str()); + LOG_TRACE("Hypothetical config: %s", config.ToString().c_str()); + LOG_TRACE("Got cost %lf", opt_info_obj->cost); + LOG_TRACE("Plan type: %s", opt_info_obj->plan->GetInfo().c_str()); + return opt_info_obj; +} + +std::shared_ptr +WhatIfIndex::CreateIndexCatalogObject(HypotheticalIndexObject *index_obj) { + // Create an index name: + // index_____... + std::ostringstream index_name_oss; + index_name_oss << "index_" << index_obj->db_oid << "_" + << index_obj->table_oid; + for (auto it = index_obj->column_oids.begin(); + it != index_obj->column_oids.end(); it++) { + index_name_oss << (*it) << "_"; + } + // TODO: For now, we assume BW-TREE and DEFAULT index constraint type for the + // hypothetical indexes + // TODO: Support unique keys. + // Create a dummy catalog object. + auto col_oids = std::vector(index_obj->column_oids.begin(), + index_obj->column_oids.end()); + auto index_cat_obj = std::shared_ptr( + new catalog::IndexCatalogObject( + index_seq_no++, index_name_oss.str(), index_obj->table_oid, + IndexType::BWTREE, IndexConstraintType::DEFAULT, false, col_oids)); + return index_cat_obj; +} + +} // namespace brain +} // namespace peloton diff --git a/src/catalog/abstract_catalog.cpp b/src/catalog/abstract_catalog.cpp index 9d9934a7c61..512213926e4 100644 --- a/src/catalog/abstract_catalog.cpp +++ b/src/catalog/abstract_catalog.cpp @@ -6,7 +6,7 @@ // // Identification: src/catalog/abstract_catalog.cpp // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -120,8 +120,8 @@ bool AbstractCatalog::InsertTuple(std::unique_ptr tuple, executor::ExecutionResult this_p_status; auto on_complete = [&this_p_status]( - executor::ExecutionResult p_status, - std::vector &&values UNUSED_ATTRIBUTE) { + executor::ExecutionResult p_status, + std::vector &&values UNUSED_ATTRIBUTE) { this_p_status = p_status; }; @@ -192,6 +192,26 @@ AbstractCatalog::GetResultWithIndexScan( std::vector column_offsets, oid_t index_offset, std::vector values, concurrency::TransactionContext *txn) const { + std::vector expr_types(values.size(), + ExpressionType::COMPARE_EQUAL); + return GetResultWithIndexScan(column_offsets, index_offset, values, + expr_types, txn); +} + +/*@brief Index scan helper function + * @param column_offsets Column ids for search (projection) + * @param index_offset Offset of index for scan + * @param values Values for search + * @param expr_types comparision expressions for the values + * @param txn TransactionContext + * @return Unique pointer of vector of logical tiles + */ +std::unique_ptr>> +AbstractCatalog::GetResultWithIndexScan( + const std::vector &column_offsets, const oid_t &index_offset, + const std::vector &values, + const std::vector &expr_types, + concurrency::TransactionContext *txn) const { if (txn == nullptr) throw CatalogException("Scan table requires transaction"); // Index scan @@ -202,8 +222,7 @@ AbstractCatalog::GetResultWithIndexScan( std::vector key_column_offsets = index->GetMetadata()->GetKeySchema()->GetIndexedColumns(); PELOTON_ASSERT(values.size() == key_column_offsets.size()); - std::vector expr_types(values.size(), - ExpressionType::COMPARE_EQUAL); + PELOTON_ASSERT(values.size() == expr_types.size()); std::vector runtime_keys; planner::IndexScanPlan::IndexScanDesc index_scan_desc( diff --git a/src/catalog/column_stats_catalog.cpp b/src/catalog/column_stats_catalog.cpp index bbe94340cdb..8d603483fa7 100644 --- a/src/catalog/column_stats_catalog.cpp +++ b/src/catalog/column_stats_catalog.cpp @@ -1,235 +1,237 @@ -//===----------------------------------------------------------------------===// -// -// Peloton -// -// column_stats_catalog.cpp -// -// Identification: src/catalog/column_stats_catalog.cpp -// -// Copyright (c) 2015-17, Carnegie Mellon University Database Group -// -//===----------------------------------------------------------------------===// - -#include "catalog/column_stats_catalog.h" - -#include "catalog/catalog.h" -#include "executor/logical_tile.h" -#include "optimizer/stats/column_stats_collector.h" -#include "storage/data_table.h" -#include "storage/tuple.h" - -namespace peloton { -namespace catalog { - -ColumnStatsCatalog *ColumnStatsCatalog::GetInstance( - concurrency::TransactionContext *txn) { - static ColumnStatsCatalog column_stats_catalog{txn}; - return &column_stats_catalog; -} - -ColumnStatsCatalog::ColumnStatsCatalog(concurrency::TransactionContext *txn) - : AbstractCatalog("CREATE TABLE " CATALOG_DATABASE_NAME - "." CATALOG_SCHEMA_NAME "." COLUMN_STATS_CATALOG_NAME - " (" - "database_id INT NOT NULL, " - "table_id INT NOT NULL, " - "column_id INT NOT NULL, " - "num_rows INT NOT NULL, " - "cardinality DECIMAL NOT NULL, " - "frac_null DECIMAL NOT NULL, " - "most_common_vals VARCHAR, " - "most_common_freqs VARCHAR, " - "histogram_bounds VARCHAR, " - "column_name VARCHAR, " - "has_index BOOLEAN);", - txn) { - // unique key: (database_id, table_id, column_id) - Catalog::GetInstance()->CreateIndex( - CATALOG_DATABASE_NAME, CATALOG_SCHEMA_NAME, COLUMN_STATS_CATALOG_NAME, - {0, 1, 2}, COLUMN_STATS_CATALOG_NAME "_skey0", true, IndexType::BWTREE, - txn); - // non-unique key: (database_id, table_id) - Catalog::GetInstance()->CreateIndex( - CATALOG_DATABASE_NAME, CATALOG_SCHEMA_NAME, COLUMN_STATS_CATALOG_NAME, - {0, 1}, COLUMN_STATS_CATALOG_NAME "_skey1", false, IndexType::BWTREE, - txn); -} - -ColumnStatsCatalog::~ColumnStatsCatalog() {} - -bool ColumnStatsCatalog::InsertColumnStats( - oid_t database_id, oid_t table_id, oid_t column_id, int num_rows, - double cardinality, double frac_null, std::string most_common_vals, - std::string most_common_freqs, std::string histogram_bounds, - std::string column_name, bool has_index, type::AbstractPool *pool, - concurrency::TransactionContext *txn) { - std::unique_ptr tuple( - new storage::Tuple(catalog_table_->GetSchema(), true)); - - auto val_db_id = type::ValueFactory::GetIntegerValue(database_id); - auto val_table_id = type::ValueFactory::GetIntegerValue(table_id); - auto val_column_id = type::ValueFactory::GetIntegerValue(column_id); - auto val_num_row = type::ValueFactory::GetIntegerValue(num_rows); - auto val_cardinality = type::ValueFactory::GetDecimalValue(cardinality); - auto val_frac_null = type::ValueFactory::GetDecimalValue(frac_null); - - type::Value val_common_val, val_common_freq; - if (!most_common_vals.empty()) { - val_common_val = type::ValueFactory::GetVarcharValue(most_common_vals); - val_common_freq = type::ValueFactory::GetVarcharValue(most_common_freqs); - } else { - val_common_val = - type::ValueFactory::GetNullValueByType(type::TypeId::VARCHAR); - val_common_freq = - type::ValueFactory::GetNullValueByType(type::TypeId::DECIMAL); - } - - type::Value val_hist_bounds; - if (!histogram_bounds.empty()) { - val_hist_bounds = type::ValueFactory::GetVarcharValue(histogram_bounds); - } else { - val_hist_bounds = - type::ValueFactory::GetNullValueByType(type::TypeId::VARCHAR); - } - - type::Value val_column_name = - type::ValueFactory::GetVarcharValue(column_name); - type::Value val_has_index = type::ValueFactory::GetBooleanValue(has_index); - - tuple->SetValue(ColumnId::DATABASE_ID, val_db_id, nullptr); - tuple->SetValue(ColumnId::TABLE_ID, val_table_id, nullptr); - tuple->SetValue(ColumnId::COLUMN_ID, val_column_id, nullptr); - tuple->SetValue(ColumnId::NUM_ROWS, val_num_row, nullptr); - tuple->SetValue(ColumnId::CARDINALITY, val_cardinality, nullptr); - tuple->SetValue(ColumnId::FRAC_NULL, val_frac_null, nullptr); - tuple->SetValue(ColumnId::MOST_COMMON_VALS, val_common_val, pool); - tuple->SetValue(ColumnId::MOST_COMMON_FREQS, val_common_freq, pool); - tuple->SetValue(ColumnId::HISTOGRAM_BOUNDS, val_hist_bounds, pool); - tuple->SetValue(ColumnId::COLUMN_NAME, val_column_name, pool); - tuple->SetValue(ColumnId::HAS_INDEX, val_has_index, nullptr); - - // Insert the tuple into catalog table - return InsertTuple(std::move(tuple), txn); -} - -bool ColumnStatsCatalog::DeleteColumnStats( - oid_t database_id, oid_t table_id, oid_t column_id, - concurrency::TransactionContext *txn) { - oid_t index_offset = IndexId::SECONDARY_KEY_0; // Secondary key index - - std::vector values; - values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); - values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); - values.push_back(type::ValueFactory::GetIntegerValue(column_id).Copy()); - - return DeleteWithIndexScan(index_offset, values, txn); -} - -std::unique_ptr> ColumnStatsCatalog::GetColumnStats( - oid_t database_id, oid_t table_id, oid_t column_id, - concurrency::TransactionContext *txn) { - std::vector column_ids( - {ColumnId::NUM_ROWS, ColumnId::CARDINALITY, ColumnId::FRAC_NULL, - ColumnId::MOST_COMMON_VALS, ColumnId::MOST_COMMON_FREQS, - ColumnId::HISTOGRAM_BOUNDS, ColumnId::COLUMN_NAME, ColumnId::HAS_INDEX}); - oid_t index_offset = IndexId::SECONDARY_KEY_0; // Secondary key index - - std::vector values; - values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); - values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); - values.push_back(type::ValueFactory::GetIntegerValue(column_id).Copy()); - - auto result_tiles = - GetResultWithIndexScan(column_ids, index_offset, values, txn); - - PELOTON_ASSERT(result_tiles->size() <= 1); // unique - if (result_tiles->size() == 0) { - return nullptr; - } - - auto tile = (*result_tiles)[0].get(); - PELOTON_ASSERT(tile->GetTupleCount() <= 1); - if (tile->GetTupleCount() == 0) { - return nullptr; - } - - type::Value num_rows, cardinality, frac_null, most_common_vals, - most_common_freqs, hist_bounds, column_name, has_index; - - num_rows = tile->GetValue(0, ColumnStatsOffset::NUM_ROWS_OFF); - cardinality = tile->GetValue(0, ColumnStatsOffset::CARDINALITY_OFF); - frac_null = tile->GetValue(0, ColumnStatsOffset::FRAC_NULL_OFF); - most_common_vals = tile->GetValue(0, ColumnStatsOffset::COMMON_VALS_OFF); - most_common_freqs = tile->GetValue(0, ColumnStatsOffset::COMMON_FREQS_OFF); - hist_bounds = tile->GetValue(0, ColumnStatsOffset::HIST_BOUNDS_OFF); - column_name = tile->GetValue(0, ColumnStatsOffset::COLUMN_NAME_OFF); - has_index = tile->GetValue(0, ColumnStatsOffset::HAS_INDEX_OFF); - - std::unique_ptr> column_stats( - new std::vector({num_rows, cardinality, frac_null, - most_common_vals, most_common_freqs, - hist_bounds, column_name, has_index})); - - return column_stats; -} - -// Return value: number of column stats -size_t ColumnStatsCatalog::GetTableStats( - oid_t database_id, oid_t table_id, concurrency::TransactionContext *txn, - std::map>> - &column_stats_map) { - std::vector column_ids( - {ColumnId::COLUMN_ID, ColumnId::NUM_ROWS, ColumnId::CARDINALITY, - ColumnId::FRAC_NULL, ColumnId::MOST_COMMON_VALS, - ColumnId::MOST_COMMON_FREQS, ColumnId::HISTOGRAM_BOUNDS, - ColumnId::COLUMN_NAME, ColumnId::HAS_INDEX}); - oid_t index_offset = IndexId::SECONDARY_KEY_1; // Secondary key index - - std::vector values; - values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); - values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); - - auto result_tiles = - GetResultWithIndexScan(column_ids, index_offset, values, txn); - - PELOTON_ASSERT(result_tiles->size() <= 1); // unique - if (result_tiles->size() == 0) { - return 0; - } - auto tile = (*result_tiles)[0].get(); - size_t tuple_count = tile->GetTupleCount(); - LOG_DEBUG("Tuple count: %lu", tuple_count); - if (tuple_count == 0) { - return 0; - } - - type::Value num_rows, cardinality, frac_null, most_common_vals, - most_common_freqs, hist_bounds, column_name, has_index; - for (size_t tuple_id = 0; tuple_id < tuple_count; ++tuple_id) { - num_rows = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::NUM_ROWS_OFF); - cardinality = - tile->GetValue(tuple_id, 1 + ColumnStatsOffset::CARDINALITY_OFF); - frac_null = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::FRAC_NULL_OFF); - most_common_vals = - tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COMMON_VALS_OFF); - most_common_freqs = - tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COMMON_FREQS_OFF); - hist_bounds = - tile->GetValue(tuple_id, 1 + ColumnStatsOffset::HIST_BOUNDS_OFF); - column_name = - tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COLUMN_NAME_OFF); - has_index = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::HAS_INDEX_OFF); - - std::unique_ptr> column_stats( - new std::vector({num_rows, cardinality, frac_null, - most_common_vals, most_common_freqs, - hist_bounds, column_name, has_index})); - - oid_t column_id = tile->GetValue(tuple_id, 0).GetAs(); - column_stats_map[column_id] = std::move(column_stats); - } - return tuple_count; -} - -} // namespace catalog -} // namespace peloton +//===----------------------------------------------------------------------===// +// +// Peloton +// +// column_stats_catalog.cpp +// +// Identification: src/catalog/column_stats_catalog.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "catalog/column_stats_catalog.h" + +#include "catalog/catalog.h" +#include "executor/logical_tile.h" +#include "optimizer/stats/column_stats_collector.h" +#include "storage/data_table.h" +#include "storage/tuple.h" + +namespace peloton { +namespace catalog { + +ColumnStatsCatalog *ColumnStatsCatalog::GetInstance( + concurrency::TransactionContext *txn) { + static ColumnStatsCatalog column_stats_catalog{txn}; + return &column_stats_catalog; +} + +// TODO [VAMSHI]: Removing the NOT NULL contraints for benchmark results. +// Enable it later +ColumnStatsCatalog::ColumnStatsCatalog(concurrency::TransactionContext *txn) + : AbstractCatalog("CREATE TABLE " CATALOG_DATABASE_NAME + "." CATALOG_SCHEMA_NAME "." COLUMN_STATS_CATALOG_NAME + " (" + "database_id INT, " + "table_id INT, " + "column_id INT, " + "num_rows INT, " + "cardinality DECIMAL, " + "frac_null DECIMAL, " + "most_common_vals VARCHAR, " + "most_common_freqs VARCHAR, " + "histogram_bounds VARCHAR, " + "column_name VARCHAR, " + "has_index BOOLEAN);", + txn) { + // unique key: (database_id, table_id, column_id) + Catalog::GetInstance()->CreateIndex( + CATALOG_DATABASE_NAME, CATALOG_SCHEMA_NAME, COLUMN_STATS_CATALOG_NAME, + {0, 1, 2}, COLUMN_STATS_CATALOG_NAME "_skey0", true, IndexType::BWTREE, + txn); + // non-unique key: (database_id, table_id) + Catalog::GetInstance()->CreateIndex( + CATALOG_DATABASE_NAME, CATALOG_SCHEMA_NAME, COLUMN_STATS_CATALOG_NAME, + {0, 1}, COLUMN_STATS_CATALOG_NAME "_skey1", false, IndexType::BWTREE, + txn); +} + +ColumnStatsCatalog::~ColumnStatsCatalog() {} + +bool ColumnStatsCatalog::InsertColumnStats( + oid_t database_id, oid_t table_id, oid_t column_id, int num_rows, + double cardinality, double frac_null, std::string most_common_vals, + std::string most_common_freqs, std::string histogram_bounds, + std::string column_name, bool has_index, type::AbstractPool *pool, + concurrency::TransactionContext *txn) { + std::unique_ptr tuple( + new storage::Tuple(catalog_table_->GetSchema(), true)); + + auto val_db_id = type::ValueFactory::GetIntegerValue(database_id); + auto val_table_id = type::ValueFactory::GetIntegerValue(table_id); + auto val_column_id = type::ValueFactory::GetIntegerValue(column_id); + auto val_num_row = type::ValueFactory::GetIntegerValue(num_rows); + auto val_cardinality = type::ValueFactory::GetDecimalValue(cardinality); + auto val_frac_null = type::ValueFactory::GetDecimalValue(frac_null); + + type::Value val_common_val, val_common_freq; + if (!most_common_vals.empty()) { + val_common_val = type::ValueFactory::GetVarcharValue(most_common_vals); + val_common_freq = type::ValueFactory::GetVarcharValue(most_common_freqs); + } else { + val_common_val = + type::ValueFactory::GetNullValueByType(type::TypeId::VARCHAR); + val_common_freq = + type::ValueFactory::GetNullValueByType(type::TypeId::DECIMAL); + } + + type::Value val_hist_bounds; + if (!histogram_bounds.empty()) { + val_hist_bounds = type::ValueFactory::GetVarcharValue(histogram_bounds); + } else { + val_hist_bounds = + type::ValueFactory::GetNullValueByType(type::TypeId::VARCHAR); + } + + type::Value val_column_name = + type::ValueFactory::GetVarcharValue(column_name); + type::Value val_has_index = type::ValueFactory::GetBooleanValue(has_index); + + tuple->SetValue(ColumnId::DATABASE_ID, val_db_id, nullptr); + tuple->SetValue(ColumnId::TABLE_ID, val_table_id, nullptr); + tuple->SetValue(ColumnId::COLUMN_ID, val_column_id, nullptr); + tuple->SetValue(ColumnId::NUM_ROWS, val_num_row, nullptr); + tuple->SetValue(ColumnId::CARDINALITY, val_cardinality, nullptr); + tuple->SetValue(ColumnId::FRAC_NULL, val_frac_null, nullptr); + tuple->SetValue(ColumnId::MOST_COMMON_VALS, val_common_val, pool); + tuple->SetValue(ColumnId::MOST_COMMON_FREQS, val_common_freq, pool); + tuple->SetValue(ColumnId::HISTOGRAM_BOUNDS, val_hist_bounds, pool); + tuple->SetValue(ColumnId::COLUMN_NAME, val_column_name, pool); + tuple->SetValue(ColumnId::HAS_INDEX, val_has_index, nullptr); + + // Insert the tuple into catalog table + return InsertTuple(std::move(tuple), txn); +} + +bool ColumnStatsCatalog::DeleteColumnStats( + oid_t database_id, oid_t table_id, oid_t column_id, + concurrency::TransactionContext *txn) { + oid_t index_offset = IndexId::SECONDARY_KEY_0; // Secondary key index + + std::vector values; + values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); + values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); + values.push_back(type::ValueFactory::GetIntegerValue(column_id).Copy()); + + return DeleteWithIndexScan(index_offset, values, txn); +} + +std::unique_ptr> ColumnStatsCatalog::GetColumnStats( + oid_t database_id, oid_t table_id, oid_t column_id, + concurrency::TransactionContext *txn) { + std::vector column_ids( + {ColumnId::NUM_ROWS, ColumnId::CARDINALITY, ColumnId::FRAC_NULL, + ColumnId::MOST_COMMON_VALS, ColumnId::MOST_COMMON_FREQS, + ColumnId::HISTOGRAM_BOUNDS, ColumnId::COLUMN_NAME, ColumnId::HAS_INDEX}); + oid_t index_offset = IndexId::SECONDARY_KEY_0; // Secondary key index + + std::vector values; + values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); + values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); + values.push_back(type::ValueFactory::GetIntegerValue(column_id).Copy()); + + auto result_tiles = + GetResultWithIndexScan(column_ids, index_offset, values, txn); + + PELOTON_ASSERT(result_tiles->size() <= 1); // unique + if (result_tiles->size() == 0) { + return nullptr; + } + + auto tile = (*result_tiles)[0].get(); + PELOTON_ASSERT(tile->GetTupleCount() <= 1); + if (tile->GetTupleCount() == 0) { + return nullptr; + } + + type::Value num_rows, cardinality, frac_null, most_common_vals, + most_common_freqs, hist_bounds, column_name, has_index; + + num_rows = tile->GetValue(0, ColumnStatsOffset::NUM_ROWS_OFF); + cardinality = tile->GetValue(0, ColumnStatsOffset::CARDINALITY_OFF); + frac_null = tile->GetValue(0, ColumnStatsOffset::FRAC_NULL_OFF); + most_common_vals = tile->GetValue(0, ColumnStatsOffset::COMMON_VALS_OFF); + most_common_freqs = tile->GetValue(0, ColumnStatsOffset::COMMON_FREQS_OFF); + hist_bounds = tile->GetValue(0, ColumnStatsOffset::HIST_BOUNDS_OFF); + column_name = tile->GetValue(0, ColumnStatsOffset::COLUMN_NAME_OFF); + has_index = tile->GetValue(0, ColumnStatsOffset::HAS_INDEX_OFF); + + std::unique_ptr> column_stats( + new std::vector({num_rows, cardinality, frac_null, + most_common_vals, most_common_freqs, + hist_bounds, column_name, has_index})); + + return column_stats; +} + +// Return value: number of column stats +size_t ColumnStatsCatalog::GetTableStats( + oid_t database_id, oid_t table_id, concurrency::TransactionContext *txn, + std::map>> + &column_stats_map) { + std::vector column_ids( + {ColumnId::COLUMN_ID, ColumnId::NUM_ROWS, ColumnId::CARDINALITY, + ColumnId::FRAC_NULL, ColumnId::MOST_COMMON_VALS, + ColumnId::MOST_COMMON_FREQS, ColumnId::HISTOGRAM_BOUNDS, + ColumnId::COLUMN_NAME, ColumnId::HAS_INDEX}); + oid_t index_offset = IndexId::SECONDARY_KEY_1; // Secondary key index + + std::vector values; + values.push_back(type::ValueFactory::GetIntegerValue(database_id).Copy()); + values.push_back(type::ValueFactory::GetIntegerValue(table_id).Copy()); + + auto result_tiles = + GetResultWithIndexScan(column_ids, index_offset, values, txn); + + PELOTON_ASSERT(result_tiles->size() <= 1); // unique + if (result_tiles->size() == 0) { + return 0; + } + auto tile = (*result_tiles)[0].get(); + size_t tuple_count = tile->GetTupleCount(); + LOG_TRACE("Tuple count: %lu", tuple_count); + if (tuple_count == 0) { + return 0; + } + + type::Value num_rows, cardinality, frac_null, most_common_vals, + most_common_freqs, hist_bounds, column_name, has_index; + for (size_t tuple_id = 0; tuple_id < tuple_count; ++tuple_id) { + num_rows = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::NUM_ROWS_OFF); + cardinality = + tile->GetValue(tuple_id, 1 + ColumnStatsOffset::CARDINALITY_OFF); + frac_null = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::FRAC_NULL_OFF); + most_common_vals = + tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COMMON_VALS_OFF); + most_common_freqs = + tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COMMON_FREQS_OFF); + hist_bounds = + tile->GetValue(tuple_id, 1 + ColumnStatsOffset::HIST_BOUNDS_OFF); + column_name = + tile->GetValue(tuple_id, 1 + ColumnStatsOffset::COLUMN_NAME_OFF); + has_index = tile->GetValue(tuple_id, 1 + ColumnStatsOffset::HAS_INDEX_OFF); + + std::unique_ptr> column_stats( + new std::vector({num_rows, cardinality, frac_null, + most_common_vals, most_common_freqs, + hist_bounds, column_name, has_index})); + + oid_t column_id = tile->GetValue(tuple_id, 0).GetAs(); + column_stats_map[column_id] = std::move(column_stats); + } + return tuple_count; +} + +} // namespace catalog +} // namespace peloton diff --git a/src/catalog/index_catalog.cpp b/src/catalog/index_catalog.cpp index da666f36f60..50273bce07f 100644 --- a/src/catalog/index_catalog.cpp +++ b/src/catalog/index_catalog.cpp @@ -6,7 +6,7 @@ // // Identification: src/catalog/index_catalog.cpp // -// Copyright (c) 2015-17, Carnegie Mellon University Index Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -54,6 +54,19 @@ IndexCatalogObject::IndexCatalogObject(executor::LogicalTile *tile, int tupleId) LOG_TRACE("the size for indexed key is %lu", key_attrs.size()); } +IndexCatalogObject::IndexCatalogObject(oid_t index_oid, std::string index_name, + oid_t table_oid, IndexType index_type, + IndexConstraintType index_constraint, + bool unique_keys, + std::vector &key_attrs) + : index_oid(index_oid), + index_name(index_name), + table_oid(table_oid), + index_type(index_type), + index_constraint(index_constraint), + unique_keys(unique_keys), + key_attrs(std::vector(key_attrs.begin(), key_attrs.end())) {} + IndexCatalog::IndexCatalog( storage::Database *pg_catalog, UNUSED_ATTRIBUTE type::AbstractPool *pool, UNUSED_ATTRIBUTE concurrency::TransactionContext *txn) @@ -213,7 +226,7 @@ std::shared_ptr IndexCatalog::GetIndexObject( auto table_object = pg_table->GetTableObject(index_object->GetTableOid(), txn); PELOTON_ASSERT(table_object && - table_object->GetTableOid() == index_object->GetTableOid()); + table_object->GetTableOid() == index_object->GetTableOid()); return table_object->GetIndexObject(index_oid); } else { LOG_DEBUG("Found %lu index with oid %u", result_tiles->size(), index_oid); @@ -259,7 +272,7 @@ std::shared_ptr IndexCatalog::GetIndexObject( auto table_object = pg_table->GetTableObject(index_object->GetTableOid(), txn); PELOTON_ASSERT(table_object && - table_object->GetTableOid() == index_object->GetTableOid()); + table_object->GetTableOid() == index_object->GetTableOid()); return table_object->GetIndexObject(index_name); } else { LOG_DEBUG("Found %lu index with name %s", result_tiles->size(), @@ -270,6 +283,28 @@ std::shared_ptr IndexCatalog::GetIndexObject( return nullptr; } +std::unordered_map> +IndexCatalog::GetIndexObjects(concurrency::TransactionContext *txn) { + std::unordered_map> result_indexes; + if (txn == nullptr) { + throw CatalogException("Transaction is invalid!"); + } + // try get from cache + auto pg_table = Catalog::GetInstance() + ->GetSystemCatalogs(database_oid) + ->GetTableCatalog(); + auto table_objects = pg_table->GetTableObjects(txn); + if (!table_objects.empty()) { + for (auto table_obj : table_objects) { + auto index_objects = GetIndexObjects(table_obj.first, txn); + for (auto index_obj : index_objects) { + result_indexes[index_obj.first] = index_obj.second; + } + } + } + return result_indexes; +} + /*@brief get all index records from the same table * this function may be useful when calling DropTable * @param table_oid diff --git a/src/catalog/query_history_catalog.cpp b/src/catalog/query_history_catalog.cpp index 4433197ba28..a8435093ba8 100644 --- a/src/catalog/query_history_catalog.cpp +++ b/src/catalog/query_history_catalog.cpp @@ -10,11 +10,12 @@ // //===----------------------------------------------------------------------===// +#include "type/value_factory.h" #include "catalog/query_history_catalog.h" #include "catalog/catalog.h" #include "storage/data_table.h" -#include "type/value_factory.h" +#include "executor/logical_tile.h" namespace peloton { namespace catalog { @@ -32,7 +33,12 @@ QueryHistoryCatalog::QueryHistoryCatalog(concurrency::TransactionContext *txn) "query_string VARCHAR NOT NULL, " "fingerprint VARCHAR NOT NULL, " "timestamp TIMESTAMP NOT NULL);", - txn) {} + txn) { + // Secondary index on timestamp + Catalog::GetInstance()->CreateIndex( + CATALOG_DATABASE_NAME, CATALOG_SCHEMA_NAME, QUERY_HISTORY_CATALOG_NAME, + {2}, QUERY_HISTORY_CATALOG_NAME "_skey0", false, IndexType::BWTREE, txn); +} QueryHistoryCatalog::~QueryHistoryCatalog() = default; @@ -56,5 +62,40 @@ bool QueryHistoryCatalog::InsertQueryHistory( return InsertTuple(std::move(tuple), txn); } +std::unique_ptr>> +QueryHistoryCatalog::GetQueryStringsAfterTimestamp( + const uint64_t start_timestamp, concurrency::TransactionContext *txn) { + LOG_INFO("Start querying.... %" PRId64, start_timestamp); + // Get both timestamp and query string in the result. + std::vector column_ids({ColumnId::TIMESTAMP, ColumnId::QUERY_STRING}); + oid_t index_offset = IndexId::SECONDARY_KEY_0; // Secondary key index + + std::vector values; + values.push_back(type::ValueFactory::GetTimestampValue( + static_cast(start_timestamp))); + + std::vector expr_types(values.size(), + ExpressionType::COMPARE_GREATERTHAN); + + auto result_tiles = + GetResultWithIndexScan(column_ids, index_offset, values, expr_types, txn); + + std::unique_ptr>> queries( + new std::vector>()); + if (result_tiles->size() > 0) { + for (auto &tile : *result_tiles.get()) { + PELOTON_ASSERT(tile->GetColumnCount() == column_ids.size()); + for (auto i = 0UL; i < tile->GetTupleCount(); i++) { + auto timestamp = tile->GetValue(i, 0).GetAs(); + auto query_string = tile->GetValue(i, 1).GetAs(); + auto pair = std::make_pair(timestamp, query_string); + LOG_INFO("Query: %" PRId64 ": %s", pair.first, pair.second); + queries->emplace_back(pair); + } + } + } + return queries; +} + } // namespace catalog } // namespace peloton diff --git a/src/catalog/table_catalog.cpp b/src/catalog/table_catalog.cpp index 9f181d4c0dc..8bbf23ce881 100644 --- a/src/catalog/table_catalog.cpp +++ b/src/catalog/table_catalog.cpp @@ -6,7 +6,7 @@ // // Identification: src/catalog/table_catalog.cpp // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -127,6 +127,16 @@ void TableCatalogObject::EvictAllIndexObjects() { valid_index_objects = false; } +/* + * @brief Sets the index objects to be invalid. + * This is useful in what-if API to avoid querying + * the catalog again by setting is_valid to true. + * @param is_valid + */ +void TableCatalogObject::SetValidIndexObjects(bool is_valid) { + valid_index_objects = is_valid; +} + /* @brief get all index objects of this table into cache * @return map from index oid to cached index object */ diff --git a/src/executor/analyze_executor.cpp b/src/executor/analyze_executor.cpp index a3544cab54f..eb0a84014b2 100644 --- a/src/executor/analyze_executor.cpp +++ b/src/executor/analyze_executor.cpp @@ -18,6 +18,7 @@ #include "common/logger.h" #include "catalog/catalog.h" #include "optimizer/stats/stats_storage.h" +#include "brain/index_selection_job_lspi.h" namespace peloton { namespace executor { @@ -36,6 +37,7 @@ bool AnalyzeExecutor::DInit() { bool AnalyzeExecutor::DExecute() { LOG_TRACE("Executing Analyze..."); + const planner::AnalyzePlan &node = GetPlanNode(); storage::DataTable* target_table = node.GetTable(); @@ -57,6 +59,16 @@ bool AnalyzeExecutor::DExecute() { LOG_TRACE("Failed to analyze table %s", node.GetTableName().c_str()); } } else { + optimizer::StatsStorage *stats_storage = + optimizer::StatsStorage::GetInstance(); + + ResultType stats_result = stats_storage->AnalyzeStatsForAllTables(current_txn); + if (stats_result != ResultType::SUCCESS) { + LOG_ERROR( + "Cannot generate stats for table columns. Not performing index " + "suggestion..."); + } + brain::IndexSelectionJobLSPI::enable_ = true; // other operations unsupported for now current_txn->SetResult(peloton::ResultType::SUCCESS); } diff --git a/src/executor/create_function_executor.cpp b/src/executor/create_function_executor.cpp index 11ceafb0c17..ce5be323aae 100644 --- a/src/executor/create_function_executor.cpp +++ b/src/executor/create_function_executor.cpp @@ -10,6 +10,7 @@ // //===----------------------------------------------------------------------===// +#include #include "executor/create_function_executor.h" #include "catalog/catalog.h" @@ -21,6 +22,7 @@ #include "planner/create_function_plan.h" #include "udf/udf_handler.h" + namespace peloton { namespace executor { @@ -34,10 +36,24 @@ bool CreateFunctionExecutor::DInit() { } bool CreateFunctionExecutor::DExecute() { - LOG_TRACE("Executing Create..."); + LOG_DEBUG("Executing Create Function..."); const auto &node = GetPlanNode(); auto *current_txn = executor_context_->GetTransaction(); +// // TODO: HACK: Remove: Analyze table column stats +// optimizer::StatsStorage *stats_storage = +// optimizer::StatsStorage::GetInstance(); +// +// ResultType stats_result = stats_storage->AnalyzeStatsForAllTables(current_txn); +// if (stats_result != ResultType::SUCCESS) { +// LOG_ERROR( +// "Cannot generate stats for table columns. Not performing index " +// "suggestion..."); +// } + + // TODO: HACK: Now run the brain job. +// brain::IndexSelectionJobLSPI::enable_ = true; + auto proname = node.GetFunctionName(); oid_t prolang = catalog::LanguageCatalog::GetInstance() .GetLanguageByName("plpgsql", current_txn) diff --git a/src/include/brain/brain.h b/src/include/brain/brain.h index 6614767423b..59b43e1fddf 100644 --- a/src/include/brain/brain.h +++ b/src/include/brain/brain.h @@ -19,6 +19,7 @@ #include "capnp/ez-rpc.h" #include "peloton/capnp/peloton_service.capnp.h" #include "common/notifiable_task.h" +#include "brain/index_selection_util.h" namespace peloton { namespace brain { @@ -28,7 +29,15 @@ namespace brain { * the brain, such as RPC and Catalog. */ class BrainEnvironment { - // TODO(tianyu): fill in as needed + public: + BrainEnvironment() { index_selection_knobs = {3, 3, 10}; } + IndexSelectionKnobs GetIndexSelectionKnobs() { return index_selection_knobs; } + void SetIndexSelectionKnobs(IndexSelectionKnobs knobs) { + index_selection_knobs = knobs; + } + + private: + IndexSelectionKnobs index_selection_knobs; }; /** @@ -55,6 +64,7 @@ class BrainJob { * provided BrainEnvironment for interaction with Brain's resources. */ virtual void OnJobInvocation(BrainEnvironment *) = 0; + private: BrainEnvironment *env_; }; @@ -68,6 +78,7 @@ class SimpleBrainJob : public BrainJob { std::function task) : BrainJob(env), task_(std::move(task)) {} inline void OnJobInvocation(BrainEnvironment *env) override { task_(env); } + private: std::function task_; }; @@ -83,13 +94,12 @@ class Brain { Brain() : scheduler_(0) {} ~Brain() { - for (auto entry : jobs_) - delete entry.second; + for (auto entry : jobs_) delete entry.second; } template - inline void RegisterJob(const struct timeval *period, - std::string name, Args... args) { + inline void RegisterJob(const struct timeval *period, std::string name, + Args... args) { auto *job = new BrainJob(&env_, args...); jobs_[name] = job; auto callback = [](int, short, void *arg) { @@ -99,13 +109,9 @@ class Brain { scheduler_.RegisterPeriodicEvent(period, callback, job); } - inline void Run() { - scheduler_.EventLoop(); - } + inline void Run() { scheduler_.EventLoop(); } - inline void Terminate() { - scheduler_.ExitLoop(); - } + inline void Terminate() { scheduler_.ExitLoop(); } private: NotifiableTask scheduler_; @@ -113,5 +119,5 @@ class Brain { std::unordered_map job_handles_; BrainEnvironment env_; }; -} // namespace brain -} // namespace peloton +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/index_selection.h b/src/include/brain/index_selection.h new file mode 100644 index 00000000000..822b5e1385f --- /dev/null +++ b/src/include/brain/index_selection.h @@ -0,0 +1,228 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection.h +// +// Identification: src/include/brain/index_selection.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "brain/index_selection_context.h" +#include "brain/index_selection_util.h" +#include "catalog/index_catalog.h" +#include "expression/tuple_value_expression.h" +#include "parser/sql_statement.h" + +namespace peloton { +namespace brain { + +/** + * @brief Comparator for set of (Index Configuration, Cost) + */ +struct IndexConfigComparator { + IndexConfigComparator(Workload &workload) { this->w = &workload; } + bool operator()(const std::pair &s1, + const std::pair &s2) const { + // Order by cost. If cost is same, then by the number of indexes + // Unless the configuration is exactly the same, get some ordering + + if (s1.second < s2.second) { + return true; + } else if (s1.second > s2.second) { + return false; + } else { + if (s1.first.GetIndexCount() > s2.first.GetIndexCount()) { + return true; + } else if (s1.first.GetIndexCount() < s2.first.GetIndexCount()) { + return false; + } else { + // TODO[Siva]: Change this to a better one, choose the one with bigger/ + // smaller indexes + return (s1.first.ToString() < s2.first.ToString()); + } + } + } + + Workload *w; +}; + +//===--------------------------------------------------------------------===// +// IndexSelection +//===--------------------------------------------------------------------===// + +class IndexSelection { + public: + /** + * IndexSelection + * + * @param query_set set of queries as a workload + * @param knobs the tunable parameters of the algorithm that includes + * number of indexes to be chosen, threshold for naive enumeration, + * maximum number of columns in each index. + */ + IndexSelection(Workload &query_set, IndexSelectionKnobs knobs, + concurrency::TransactionContext *txn); + + /** + * @brief The main external API for the Index Prediction Tool + * @returns The best possible Index Congurations for the workload + */ + void GetBestIndexes(IndexConfiguration &final_indexes); + + /** + * @brief Gets the indexable columns of a given query + */ + void GetAdmissibleIndexes(std::shared_ptr query, + IndexConfiguration &indexes); + + /** + * @brief GenerateCandidateIndexes. + * If the admissible config set is empty, generate + * the single-column (admissible) indexes for each query from the provided + * queries and prune the useless ones. This becomes candidate index set. If + * not empty, prune the useless indexes from the candidate set for the given + * workload. + * + * @param candidate_config - new candidate index to be pruned. + * @param admissible_config - admissible index set of the queries + * @param workload - queries + */ + void GenerateCandidateIndexes(IndexConfiguration &candidate_config, + IndexConfiguration &admissible_config, + Workload &workload); + + /** + * @brief gets the top k indexes for the workload which would reduce the cost + * of executing them + * + * @param indexes - the indexes in the workload + * @param top_indexes - the top k cheapest indexes in the workload are + * returned through this parameter + * @param workload - the given workload + * @param k - the number of indexes to return + */ + void Enumerate(IndexConfiguration &indexes, IndexConfiguration &top_indexes, + Workload &workload, size_t k); + + /** + * @brief generate multi-column indexes from the single column indexes by + * doing a cross product and adds it into the result. + * + * @param config - the set of candidate indexes chosen after the enumeration + * @param single_column_indexes - the set of admissible single column indexes + * @param result - return the set of multi column indexes + */ + void GenerateMultiColumnIndexes(IndexConfiguration &config, + IndexConfiguration &single_column_indexes, + IndexConfiguration &result); + + /** + * @brief Add a given configuration to the IndexObject pool + * return the corresponding shared pointer if the object already exists in + * the pool. Otherwise create one and return. + * Currently, this is used only for unit testing + */ + std::shared_ptr AddConfigurationToPool( + HypotheticalIndexObject object); + + private: + /** + * @brief PruneUselessIndexes + * Delete the indexes from the configuration which do not help at least one of + * the queries in the workload + * + * @param config - index set + * @param workload - queries + * @param pruned_config - result configuration + */ + void PruneUselessIndexes(IndexConfiguration &config, Workload &workload, + IndexConfiguration &pruned_config); + + /** + * @brief Gets the cost of an index configuration for a given workload. It + * would call the What-If API appropriately and stores the results in the memo + * table + */ + double ComputeCost(IndexConfiguration &config, Workload &workload); + + // Configuration Enumeration related + /** + * @brief Gets the cheapest indexes through naive exhaustive enumeration by + * generating all possible subsets of size <= m where m is a tunable parameter + */ + void ExhaustiveEnumeration(IndexConfiguration &indexes, + IndexConfiguration &top_indexes, + Workload &workload); + + /** + * @brief Gets the remaining cheapest indexes through greedy search + */ + void GreedySearch(IndexConfiguration &indexes, + IndexConfiguration &remaining_indexes, Workload &workload, + size_t num_indexes); + + // Admissible index selection related + /** + * @brief Helper to parse the order where in the SQL statements such as + * select, delete, update. + */ + void IndexColsParseWhereHelper( + const expression::AbstractExpression *where_expr, + IndexConfiguration &config); + + /** + * @brief Helper to parse the group by clause in the SQL statements such as + * select, delete, update. + */ + void IndexColsParseGroupByHelper( + std::unique_ptr &where_expr, + IndexConfiguration &config); + + /** + * @brief Helper to parse the order by clause in the SQL statements such as + * select, delete, update. + */ + void IndexColsParseOrderByHelper( + std::unique_ptr &order_by, + IndexConfiguration &config); + + /** + * @brief Helper function to convert a tuple of + * to an IndexObject and store into the IndexObject shared pool. + * + * @param - tuple_col: representation of a column + * @param - config: returns a new index object here + */ + void IndexObjectPoolInsertHelper( + const std::tuple &tuple_col, + IndexConfiguration &config); + + /** + * @brief Create a new index configuration which is a cross product of the + * given configurations and merge it into the result. + * result = result union (configuration1 * configuration2) + * Ex: {I1} * {I23, I45} = {I123, I145} + * + * @param - configuration1: config1 + * @param - configuration2: config2 + * @param - result: cross product + */ + void CrossProduct(const IndexConfiguration &configuration1, + const IndexConfiguration &configuration2, + IndexConfiguration &result); + + // Set of parsed and bound queries + Workload query_set_; + // Common context of index selection object. + IndexSelectionContext context_; + // Transaction. + concurrency::TransactionContext *txn_; +}; + +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/index_selection_context.h b/src/include/brain/index_selection_context.h new file mode 100644 index 00000000000..2f11f6ff3ea --- /dev/null +++ b/src/include/brain/index_selection_context.h @@ -0,0 +1,68 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_context.h +// +// Identification: src/include/brain/index_selection_context.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include + +#include "brain/index_selection_util.h" + +namespace parser { +class SQLStatement; +} + +namespace peloton { +namespace brain { + +// Hasher for the KeyType of the memo used for cost evalutation +struct KeyHasher { + std::size_t operator()( + const std::pair &key) const { + auto indexes = key.first.GetIndexes(); + // TODO[Siva]: Can we do better? + auto result = std::hash()(key.second->GetInfo()); + for (auto index : indexes) { + // TODO[Siva]: Use IndexObjectHasher to hash this + result ^= std::hash()(index->ToString()); + } + return result; + } +}; + +//===--------------------------------------------------------------------===// +// IndexSelectionContext +//===--------------------------------------------------------------------===// + +class IndexSelectionContext { + public: + /** + * @brief Constructor + * + */ + IndexSelectionContext(IndexSelectionKnobs knobs); + + private: + friend class IndexSelection; + + // memoization of the cost of a query for a given configuration + std::unordered_map, + double, KeyHasher> memo_; + // map from index configuration to the sharedpointer of the + // IndexConfiguration object + IndexObjectPool pool_; + + // The knobs for this run of the algorithm + IndexSelectionKnobs knobs_; +}; + +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/index_selection_job.h b/src/include/brain/index_selection_job.h new file mode 100644 index 00000000000..374c978b234 --- /dev/null +++ b/src/include/brain/index_selection_job.h @@ -0,0 +1,79 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_job.h +// +// Identification: src/include/brain/index_selection_job.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once +#include "brain.h" +#include "brain/index_selection_util.h" + +namespace peloton { + +namespace brain { +class IndexSelectionJob : public BrainJob { + public: + IndexSelectionJob(BrainEnvironment *env, uint64_t num_queries_threshold) + : BrainJob(env), + last_timestamp_(0), + num_queries_threshold_(num_queries_threshold) {} + const std::string brain_suggested_index_prefix_str = "brain_suggested_index"; + + /** + * Task function. + * @param env + */ + void OnJobInvocation(BrainEnvironment *env); + + private: + /** + * Go through the queries and return the timestamp of the latest query. + * @return latest timestamp + */ + static uint64_t GetLatestQueryTimestamp( + std::vector> *); + /** + * Sends an RPC message to server for creating indexes. + * @param table_name + * @param keys + */ + void CreateIndexRPC(brain::HypotheticalIndexObject *index); + + /** + * Finds current indexes - suggested indexes. + * @param cur_indexes + * @param best_config + * @return indexes that are not useful and to be dropped. + */ + std::vector> GetIndexesToDrop( + std::unordered_map> + &cur_indexes, + brain::IndexConfiguration best_config); + + /** + * Sends an RPC message to server for drop indexes. + * @param index + */ + void DropIndexRPC(oid_t database_oid, catalog::IndexCatalogObject *index); + + /** + * Timestamp of the latest query of the recently processed + * query workload. + */ + uint64_t last_timestamp_; + /** + * Tuning threshold in terms of queries + * Run the index suggestion only if the number of new queries + * in the workload exceeds this number + */ + uint64_t num_queries_threshold_; +}; +} // peloton brain + +} // namespace peloton diff --git a/src/include/brain/index_selection_job_lspi.h b/src/include/brain/index_selection_job_lspi.h new file mode 100644 index 00000000000..dc683ef83b9 --- /dev/null +++ b/src/include/brain/index_selection_job_lspi.h @@ -0,0 +1,83 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_job.h +// +// Identification: src/include/brain/index_selection_job.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once +#include "brain.h" +#include "brain/indextune/lspi/lspi_tuner.h" + + +namespace peloton { + +namespace brain { + + +class IndexSelectionJobLSPI : public BrainJob { + public: + explicit IndexSelectionJobLSPI(BrainEnvironment *env, uint64_t num_queries_threshold); + const std::string brain_suggested_index_prefix_str = "brain_suggested_index"; + + /** + * Task function. + * @param env + */ + void OnJobInvocation(BrainEnvironment *env); + static bool enable_; + + private: + /** + * Go through the queries and return the timestamp of the latest query. + * @return latest timestamp + */ + static uint64_t GetLatestQueryTimestamp( + std::vector> *); + /** + * Sends an RPC message to server for creating indexes. + * @param table_name + * @param keys + */ + void CreateIndexRPC(brain::HypotheticalIndexObject *index); + + /** + * Finds current indexes - suggested indexes. + * @param cur_indexes + * @param best_config + * @return indexes that are not useful and to be dropped. + */ +// std::vector> GetIndexesToDrop( +// std::unordered_map> +// &cur_indexes, +// brain::IndexConfiguration best_config); + + /** + * Sends an RPC message to server for drop indexes. + * @param index + */ + void DropIndexRPC(oid_t database_oid, catalog::IndexCatalogObject *index); + + /** + * Timestamp of the latest query of the recently processed + * query workload. + */ + uint64_t last_timestamp_; + /** + * Tuning threshold in terms of queries + * Run the index suggestion only if the number of new queries + * in the workload exceeds this number + */ + uint64_t num_queries_threshold_; + std::unique_ptr tuner_; + bool tuner_initialized_ = false; + +}; +} // peloton brain + +} // namespace peloton diff --git a/src/include/brain/index_selection_util.h b/src/include/brain/index_selection_util.h new file mode 100644 index 00000000000..9aacdf083b9 --- /dev/null +++ b/src/include/brain/index_selection_util.h @@ -0,0 +1,298 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_util.h +// +// Identification: src/include/brain/index_selection_util.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include +#include +#include + +#include "binder/bind_node_visitor.h" +#include "catalog/index_catalog.h" +#include "concurrency/transaction_manager_factory.h" +#include "parser/sql_statement.h" +#include "parser/postgresparser.h" +#include "concurrency/transaction_context.h" + +namespace peloton { +namespace brain { + +//===--------------------------------------------------------------------===// +// IndexSuggestionKnobs +//===--------------------------------------------------------------------===// + +// Tunable knobs of the index selection algorithm +struct IndexSelectionKnobs { + // The number of iterations of the main algorithm which is also the maximum + // number of columns in a single index as in ith iteration we consider indexes + // with i or lesser columns + size_t num_iterations_; + // The number of indexes up to which we will do exhaustive enumeration + size_t naive_enumeration_threshold_; + // The number of indexes in the final configuration returned by the + // IndexSelection algorithm + size_t num_indexes_; +}; + +//===--------------------------------------------------------------------===// +// IndexObject +//===--------------------------------------------------------------------===// + +// Class to represent a (hypothetical) index +struct HypotheticalIndexObject { + // the OID of the database + oid_t db_oid; + // the OID of the table + oid_t table_oid; + // OIDs of each column in the index + std::vector column_oids; + + /** + * @brief - Constructor + */ + HypotheticalIndexObject(){}; + + /** + * @brief - Constructor + */ + HypotheticalIndexObject(oid_t db_oid, oid_t table_oid, oid_t col_oid) + : db_oid(db_oid), table_oid(table_oid) { + column_oids.push_back(col_oid); + } + + /** + * @brief - Constructor + */ + HypotheticalIndexObject(oid_t db_oid, oid_t table_oid, + std::vector &col_oids) + : db_oid(db_oid), table_oid(table_oid), column_oids(col_oids) {} + + /** + * @brief - Equality operator of the index object + */ + bool operator==(const HypotheticalIndexObject &obj) const; + + /** + * @brief - Checks whether the 2 indexes can be merged to make a multi column + * index. Return true if they are in the same database and table, else false + */ + bool IsCompatible(std::shared_ptr index) const; + + /** + * @brief - Merges the 2 index objects to make a multi column index + */ + HypotheticalIndexObject Merge(std::shared_ptr index); + + const std::string ToString() const; +}; + +//===--------------------------------------------------------------------===// +// IndexConfiguration +//===--------------------------------------------------------------------===// + +// Hasher for the IndexObject +struct IndexObjectHasher { + size_t operator()(const HypotheticalIndexObject &obj) const { + return std::hash()(obj.ToString()); + } +}; + +// Call to represent a configuration - a set of hypothetical indexes +class IndexConfiguration { + public: + /** + * @brief - Constructor + */ + IndexConfiguration() {} + + /** + * @brief - Constructor + */ + IndexConfiguration( + std::set> &index_obj_set) + : indexes_(index_obj_set) {} + + /** + * @brief - Merges with the argument configuration + */ + void Merge(IndexConfiguration &config); + + /** + * @brief replace config + */ + void Set(IndexConfiguration &config); + + /** + * @brief - Adds an index into the configuration + */ + void AddIndexObject( + const std::shared_ptr &index_info); + + /** + * @brief - Removes an index from the configuration + */ + void RemoveIndexObject( + const std::shared_ptr &index_info); + + /** + * @brief - Returns the number of indexes in the configuration + */ + size_t GetIndexCount() const; + + /** + * @brief is empty + * @return bool + */ + bool IsEmpty() const; + + /** + * @brief - Returns the indexes in the configuration + */ + const std::set> &GetIndexes() const; + + /** + * @brief - Equality operator of the index configurations + */ + bool operator==(const IndexConfiguration &obj) const; + + /** + * @brief - Set difference of the two configurations + */ + IndexConfiguration operator-(const IndexConfiguration &obj); + + const std::string ToString() const; + + void Clear(); + + private: + // The set of hypothetical indexes in the configuration + std::set> indexes_; +}; + +//===--------------------------------------------------------------------===// +// IndexObjectPool +//===--------------------------------------------------------------------===// + +// This class is a wrapper around a map from the IndexConfiguration to the +// shared pointer of the object. This shared pointer is used else where in the +// the algorithm to identify a configuration - memoization, enumeration, +// equality while sorting etc. +class IndexObjectPool { + public: + /** + * @brief - Constructor + */ + IndexObjectPool() {} + + /** + * @brief - Return the shared pointer of the object from the global + */ + std::shared_ptr GetIndexObject( + HypotheticalIndexObject &obj); + + /** + * @brief - Add the object to the pool of index objects + * if the object already exists, return the shared pointer + * else create the object, add it to the pool and return the shared pointer + */ + std::shared_ptr PutIndexObject( + HypotheticalIndexObject &obj); + + private: + // The mapping from the object to the shared pointer + std::unordered_map, + IndexObjectHasher> map_; +}; + +//===--------------------------------------------------------------------===// +// Workload +//===--------------------------------------------------------------------===// + +// Represents a workload of SQL queries +class Workload { + public: + /** + * @brief - Constructor + */ + Workload(std::string database_name) : database_name(database_name) {} + + /** + * @brief - Initialize a workload with the given query strings. Parse, bind + * and + * add SQLStatements. + */ + Workload(std::vector &queries, std::string database_name, + concurrency::TransactionContext *txn); + + /** + * @brief - Constructor + */ + Workload(std::pair, + std::unordered_set> query, + std::string database_name) + : sql_queries_({query}), database_name(database_name) {} + + /** + * @brief - Add a query into the workload + */ + inline void AddQuery(std::shared_ptr query, + std::unordered_set tables) { + sql_queries_.push_back(std::make_pair(query, tables)); + } + + /** + * @brief - Return the queries + */ + inline const std::vector, + std::unordered_set>> + &GetQueries() { + return sql_queries_; + } + + /** + * @brief - Return the parsed SQLstatements + */ + inline size_t Size() { return sql_queries_.size(); } + + /** + * @brief Return the database name + */ + inline std::string GetDatabaseName() { + PELOTON_ASSERT(database_name != ""); + return database_name; + }; + + /** + * * @brief GetTableNamesReferenced + * Given a parsed & bound query, this function returns all the tables + * referenced. + * @param query - a parsed and bound SQL statement + * @param table_names - where the table names will be stored. + */ + static void GetTableNamesReferenced( + std::shared_ptr query, + std::unordered_set &table_names, + bool &illegal_query); + + private: + /** + * Parsed SQL queries along with the referenced table names. + */ + std::vector, + std::unordered_set>> sql_queries_; + std::string database_name; +}; + +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/indextune/compressed_index_config.h b/src/include/brain/indextune/compressed_index_config.h new file mode 100644 index 00000000000..066e66e7681 --- /dev/null +++ b/src/include/brain/indextune/compressed_index_config.h @@ -0,0 +1,229 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// compressed_index_config.h +// +// Identification: src/include/brain/indextune/compressed_index_config.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include "brain/index_selection.h" +#include "brain/util/eigen_util.h" +#include "catalog/catalog.h" +#include "catalog/database_catalog.h" +#include "catalog/index_catalog.h" +#include "catalog/table_catalog.h" +#include "concurrency/transaction_manager_factory.h" +#include "planner/plan_util.h" +#include "brain/indextune/lspi/lspi_common.h" + +namespace peloton { +namespace brain { + +class CompressedIndexConfigContainer { + friend class CompressedIndexConfigUtil; + + public: + /** + * Constructor for CompressedIndexConfigContainer: Initialize + * (1) catalog pointer + * (2) txn_manager pointer + * One such configuration is for only one database. + * + * Then scan all the tables in the database to populate the internal maps + * Finally, scan all tables again to generate current index configuration (a + * bitset) + */ + explicit CompressedIndexConfigContainer( + const std::string &database_name, + const std::set &ignore_table_oids, size_t max_index_size = 3, + catalog::Catalog *catalog = nullptr, + concurrency::TransactionManager *txn_manager = nullptr); + + /** + * @brief Given a new bitset, add/drop corresponding indexes and update + * current bitset + */ + void AdjustIndexes(const boost::dynamic_bitset<> &new_bitset, + std::set>& add_set, + std::set>& drop_set); + + // **Useful getter fns** + + /** + * Get the global offset of an index in a table + * @param index_obj: the index + * @return the global offset of the index in the bitset + */ + size_t GetGlobalOffset( + const std::shared_ptr &index_obj) const; + + /** + * Check whether an index is in current configuration or not + * @param index_obj: the index to be checked + * @return the bit for that index is set or not + */ + bool IsSet( + const std::shared_ptr &index_obj) const; + + /** + * Check whether an index is in current configuration or not + * @param offset: the global offset of the index + * @return the bit for that index is set or not + */ + bool IsSet(const size_t offset) const; + + /** + * @brief Get the total number of possible indexes in current database + */ + size_t GetConfigurationCount() const; + + /** + * Given a global offset, get the corresponding hypothetical index object + * @param global_offset: the global offset + * @return the index object at "global_offset" of current configuration + */ + std::shared_ptr GetIndex( + size_t global_offset) const; + + /** + * @brief Get the current index configuration as a bitset(read-only) + */ + const boost::dynamic_bitset<> *GetCurrentIndexConfig() const; + + /** + * @brief Get instance of the txn manager + */ + concurrency::TransactionManager *GetTransactionManager(); + /** + * @brief Get instance of the catalog + */ + catalog::Catalog *GetCatalog(); + + std::string GetDatabaseName() const; + + oid_t GetDatabaseOID() const { return database_oid_; }; + /** + * @brief Given a table oid get the bitset offset where it lies + */ + size_t GetTableOffsetStart(oid_t table_oid) const; + /** + * @brief Given a table oid get the bitset offset where it ends + */ + size_t GetTableOffsetEnd(oid_t table_oid) const; + /** + * @brief Given a bitset offset, get the current table_oid. + */ + oid_t GetCurrentTableOID(size_t idx) const; + /** + * @brief Given a bitset offset, get the bitset offset where the next table_oid lies. + */ + size_t GetNextTableIdx(size_t start_idx) const; + /** + * @brief Get the total number of indexes on a given table + */ + size_t GetNumIndexes(oid_t table_oid) const; + /** + * @brief Get the next index configuration offset + */ + size_t GetNextSetIndexConfig(size_t from_idx) const; + /** + * @brief Check if a table has any index config + */ + bool EmptyConfig(oid_t table_oid) const; + /** + * @brief Extremely verbose representation + */ + std::string ToString() const; + std::string ToString(const boost::dynamic_bitset<> &bs) const; + + private: + std::string database_name_; + catalog::Catalog *catalog_; + concurrency::TransactionManager *txn_manager_; + oid_t database_oid_; + + /** + * Add an index to current configuration + * @param idx_object: the index to be added + */ + void SetBit(const std::shared_ptr &idx_object); + + /** + * Add an index to current configuration + * @param offset: the global offset of the index to be added + */ + void SetBit(size_t offset); + + /** + * Remove an index from current configuration + * @param idx_object: the index to be removed + */ + void UnsetBit(const std::shared_ptr &idx_object); + + /** + * Remove and index from current configuration + * @param offset: the global offset of the index to be removed + */ + void UnsetBit(size_t offset); + + void EnumerateConfigurations( + const std::vector &cols, size_t max_index_size, + std::map, size_t> &indexconf_id_map, + std::map> &id_indexconf_map, + std::vector &index_conf, size_t &next_id); + + /** + * Outer mapping: table_oid -> inner mapping + * Inner mapping: column_oid -> internal mapping ID + * + * For example, table T (table_oid = 12345) has three columns: A (column_oid = + * 5), B (column_oid = 3), C (column_oid = 14). Then we will have: + * table_id_map_[12345] ==> inner mapping + * inner mapping ==> {Nothing->0, {5}->1, {3}->2, {14}-> 3, {5, 3} -> 4.... + * Basically every possible single and multicol index ordering gets a unique + * identifier. + * Identifiers continue when we go from one table to the next - i.e. if table + * T1 ends at id 15 + * Table T2 starts at 16 and goes on from there. + * TODO(saatviks): Come up with an even more compressed rep.(like eg. a->0, + * b->1, c->2 + * and Nothing = 000, {a} = 001, {ab} = 011, etc. Problem is this doesnt work + * for + * permutations - only for combinations). + */ + std::unordered_map, size_t>> + table_indexid_map_; + + /** + * Outer mapping: table_oid -> inner reverse mapping + * Inner reverse mapping is the reverse of `inner mapping` + * explained above + */ + std::unordered_map>> + indexid_table_map_; + + /** + * In order to enable faster table->col lookups we also store table offsets + * separately. + * This also allows for other functionality. + */ + std::map table_offset_map_; + + // This map is just the reverse mapping of table_offset_map_ + std::map table_offset_reverse_map_; + + // the next offset of a new table(during construction) + // the end pointer - post construction + size_t next_table_offset_; + + std::unique_ptr> cur_index_config_; +}; +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/indextune/compressed_index_config_util.h b/src/include/brain/indextune/compressed_index_config_util.h new file mode 100644 index 00000000000..2763a96dac0 --- /dev/null +++ b/src/include/brain/indextune/compressed_index_config_util.h @@ -0,0 +1,144 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// compressed_index_config_util.h +// +// Identification: src/include/brain/indextune/compressed_index_config_util.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include "brain/indextune/lspi/lspi_common.h" +#include "brain/indextune/compressed_index_config.h" + +namespace peloton { +namespace brain { + +class CompressedIndexConfigUtil { + public: + /** + * Given a SQLStatementList, generates a suitable set of add candidates + * @param container: input container + * @param query: query in question + * @param add_candidates: the resulting add_candidates + * @param cand_sel_type: candidate index selection mechanism to follow + * @param max_index_size: max number of columns to use to build index + * permutations(useful only when doing an exhaustive search) + * @param knobs: Knobs if using Autoadmin candidate selection + * @return the permuation as a bitset + */ + static void AddCandidates(CompressedIndexConfigContainer &container, + const std::string &query, + boost::dynamic_bitset<> &add_candidates, + CandidateSelectionType cand_sel_type, + size_t max_index_size = 0, + IndexSelectionKnobs knobs = {}); + /** + * Given a SQLStatement, generate drop candidates + * @param container: input container + * @param sql_stmt: the SQLStatement + * @return the drop candidates + */ + static void DropCandidates(CompressedIndexConfigContainer &container, + const std::string &query, + boost::dynamic_bitset<> &drop_candidates); + + /** + * @brief Return a bitset initialized using a list of indexes + */ + static std::unique_ptr> GenerateBitSet( + const CompressedIndexConfigContainer &container, + const std::vector> + &idx_objs); + + static void SetBit( + const CompressedIndexConfigContainer &container, + boost::dynamic_bitset<> &bitmap, + const std::shared_ptr &idx_object); + + // Feature constructors + /** + * Constructs the feature vector representing the SQL query running on the + * current + * index configuration. This is done by using the following feature vector: + * = 0.0 if not in f(query) + * = 1.0 if in f(query) and belongs to current config + * = -1 if in f(query) but not in current config + * where f(query) is first recommended_index(query)(0->n), then + * drop_index(query)(n->2*n) + * @param add_candidates: add candidate suggestions + * @param drop_candidates: drop candidate suggestions + * @param query_config_vec: query configuration vector to construct + * // TODO: not in f(query) should split into: (i)!f(query) && + * belongs(config) (ii) !(f(query) && belongs(config))? + */ + static void ConstructQueryConfigFeature( + const boost::dynamic_bitset<> &curr_config_set, + const boost::dynamic_bitset<> &add_candidate_set, + const boost::dynamic_bitset<> &drop_candidate_set, + vector_eig &query_config_vec); + + /** + * Generate an IndexConfiguration object using a + * CompressedIndexConfigContainer + * @param index_config + */ + static IndexConfiguration ToIndexConfiguration( + const CompressedIndexConfigContainer &container); + + static void GetIgnoreTables(const std::string &db_name, + std::set &ori_table_oids); + + /** + * @brief Get the Eigen vector/feature representation from the + * provided config set: 1 if Index config present, else -1 + */ + static void ConstructStateConfigFeature( + const boost::dynamic_bitset<> &config_set, vector_eig &config_vec); + + /** + * @brief: A general util to print a vector + */ + static std::string ToString(std::vector config_vector); + + /** + * @brief: A general util to print an Eigen vector + */ + static std::string ToString(vector_eig v); + + static double WhatIfIndexCost(std::string query, + brain::IndexConfiguration &config, + std::string database_name); + + private: + /** + * @brief: converts query string to a binded sql-statement list + */ + static std::unique_ptr ToBindedSqlStmtList( + CompressedIndexConfigContainer &container, + const std::string &query_string); + + /** + * @brief Convert an index triplet to an index object + */ + static std::shared_ptr ConvertIndexTriplet( + CompressedIndexConfigContainer &container, + const planner::col_triplet &idx_triplet); + + static void PermuateConfigurations( + const CompressedIndexConfigContainer &container, + const std::vector &cols, size_t max_index_size, + std::vector &index_conf, boost::dynamic_bitset<> &bitset, + oid_t db_oid, oid_t table_oid); + + static void MarkPrefixClosure(const CompressedIndexConfigContainer &container, + boost::dynamic_bitset<> &bitset, + const std::shared_ptr& hypot_index_obj); +}; +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/indextune/lspi/lspi_common.h b/src/include/brain/indextune/lspi/lspi_common.h new file mode 100644 index 00000000000..057a5eaa7ee --- /dev/null +++ b/src/include/brain/indextune/lspi/lspi_common.h @@ -0,0 +1,7 @@ +#pragma once + +namespace peloton{ +namespace brain{ +enum class CandidateSelectionType{ Simple, AutoAdmin, Exhaustive}; +} +} \ No newline at end of file diff --git a/src/include/brain/indextune/lspi/lspi_tuner.h b/src/include/brain/indextune/lspi/lspi_tuner.h new file mode 100644 index 00000000000..ff36ed3a48a --- /dev/null +++ b/src/include/brain/indextune/lspi/lspi_tuner.h @@ -0,0 +1,74 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// lspi_tuner.h +// +// Identification: src/include/brain/indextune/lspi/lspi_tuner.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include +#include "brain/indextune/lspi/lspi_common.h" +#include "brain/indextune/compressed_index_config.h" +#include "brain/indextune/compressed_index_config_util.h" +#include "brain/indextune/lspi/lstdq.h" +#include "brain/indextune/lspi/rlse.h" +#include "brain/util/eigen_util.h" + +/** + * Least-Squares Policy Iteration based Index tuning + * (Derived from Cost Model Oblivious DB Tuning by Basu et. al.) + * This can be extended to any configuration knob tuning problem. + * For now, we assume one instance of the tuner per database. + */ +namespace peloton { +namespace brain { +class LSPIIndexTuner { + public: + explicit LSPIIndexTuner( + const std::string &db_name, const std::set &ignore_table_oids, + CandidateSelectionType cand_sel_type, size_t max_index_size, + double variance_init = 1e-3, double reg_coeff = 1, + catalog::Catalog *catalog = nullptr, + concurrency::TransactionManager *txn_manager = nullptr); + /** + * Given a recent set of queries and their latency on the current + * configuration this function will automatically tune the database for future + * workloads. + * Currently it only supports IndexTuning but should be relatively simple to + * support more utility functions. + * @param query_latency_pairs: vector of pairs + */ + void Tune(const std::vector &queries, + const std::vector &query_latencies, + std::set>& add_set, + std::set>& drop_set); + void FindOptimalConfig(const boost::dynamic_bitset<> &curr_config_set, + const boost::dynamic_bitset<> &add_candidate_set, + const boost::dynamic_bitset<> &drop_candidate_set, + boost::dynamic_bitset<> &optimal_config_set); + const CompressedIndexConfigContainer *GetConfigContainer() const; + + private: + // Database to tune + std::string db_name_; + CandidateSelectionType cand_sel_type_; + size_t max_index_size_; + // Index configuration object - Represents current set of indexes compactly + // and exposes APIs for generating a search space for our RL algorithm + std::unique_ptr index_config_; + // RLSE model for computing immediate cost of an action + std::unique_ptr rlse_model_; + // LSTD model for computing + std::unique_ptr lstdq_model_; + // Previous config feature vector + vector_eig prev_config_vec; +}; +} // namespace brain +} // namespace peloton \ No newline at end of file diff --git a/src/include/brain/indextune/lspi/lstdq.h b/src/include/brain/indextune/lspi/lstdq.h new file mode 100644 index 00000000000..9a37011e980 --- /dev/null +++ b/src/include/brain/indextune/lspi/lstdq.h @@ -0,0 +1,55 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// lstd.h +// +// Identification: src/include/brain/indextune/lspi/lstd.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "brain/util/eigen_util.h" + +/** + * LSTDQ Estimator + * References: + * [1] Cost Model Oblivious DB Tuning by Basu et. al. + * [2] Least Squares Policy Iteration by M. Lagoudakis et. al.(Pg. 18) of JMLR article + * Good Resources: https://www2.cs.duke.edu/research/AI/LSPI/jmlr03.pdf, + * https://www.cs.utexas.edu/~pstone/Courses/394Rspring11/resources/week14a-lspi.pdf + * provides the LSTDQ-Opt formula which the authors in [1] seem to have used. + * LSTDQ provides a way of determining the Q value for a parametrized state-action pair given + * such a state-action for the current and previous timesteps along with associated "reward"(or cost as + * we see it here). + * TODO(saatvik): The formula used below is a reproduction from the code of + *[1]. Some parts of the formulation don't match whats present in the + *literature. Might be worth revisiting. + * TODO(saatvik): Figure out a good way to test this. + **/ + +namespace peloton { +namespace brain { +class LSTDQModel { + public: + explicit LSTDQModel(size_t feat_len, double variance_init = 1e-3, + double gamma = 0.9999); + void Update(const vector_eig &state_feat_curr, + const vector_eig &state_feat_next, double true_cost); + double Predict(const vector_eig &state_feat) const; + + private: + // feature length + size_t feat_len_; + // discounting-factor + double gamma_; + // model variance + matrix_eig model_variance_; + // parameters of model + vector_eig weights_; +}; +} // namespace brain +} // namespace peloton \ No newline at end of file diff --git a/src/include/brain/indextune/lspi/rlse.h b/src/include/brain/indextune/lspi/rlse.h new file mode 100644 index 00000000000..30b330470c4 --- /dev/null +++ b/src/include/brain/indextune/lspi/rlse.h @@ -0,0 +1,76 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// rlse.h +// +// Identification: src/include/brain/indextune/lspi/rlse.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "brain/util/eigen_util.h" + +/** + * Recursive Least Squares Estimator: + * References: + * [1] https://www.otexts.org/1582 + * [2] Cost Model Oblivious DB Tuning by Basu et. al. + * Used for efficiently estimating the immediate cost of executing + * a query on a given configuration. + * TODO(saatvik): The formula used below is a reproduction from the code of [2]. + * Some parts of the formulation don't match whats present in the literature. + * Might be worth revisiting. + */ + +namespace peloton { +namespace brain { +class RLSEModel { + public: + /** + * Constructor for RLSE model: Initializes the + * (1) Variance matrix + * (2) Zero weight model params + * Note that feature length must stay constant + * Any changes to feature length will need model reinitialization + * explicitly by the user + */ + explicit RLSEModel(size_t feat_len, double variance_init = 1e-3, + double reg_coeff = 1, bool random_weights = false); + /** + * Update model weights + * @param feat_vector: Feature vector(X) - Independent variables + * For example in Index tuning this should represent the workload + * and current Index config + * @param true_val: Labels(y) - Dependent variable + * For example in Index tuning this should represent the cost of + * running the workload with the current Index config + */ + void Update(const vector_eig &feat_vector, double true_val); + + /** + * Predicts the dependent variable(y) given the independent variable(X) + * @param feat_vector: X + * @return: y + */ + double Predict(const vector_eig &feat_vector) const; + + vector_eig GetWeights() const { + return weights_; + }; + + private: + // feature length + size_t feat_len_; + // model variance + matrix_eig model_variance_; + // Regularization coefficient + double reg_coeff_; + // parameters of model + vector_eig weights_; +}; +} // namespace brain +} // namespace peloton diff --git a/src/include/brain/what_if_index.h b/src/include/brain/what_if_index.h new file mode 100644 index 00000000000..99e1417eb1b --- /dev/null +++ b/src/include/brain/what_if_index.h @@ -0,0 +1,90 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// what_if_index.h +// +// Identification: src/include/brain/what_if_index.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include +#include + +#include "brain/index_selection_util.h" +#include "catalog/catalog.h" +#include "catalog/column_catalog.h" +#include "catalog/database_catalog.h" +#include "catalog/index_catalog.h" +#include "catalog/table_catalog.h" +#include "common/internal_types.h" +#include "optimizer/optimizer.h" +#include "parser/postgresparser.h" + +namespace peloton { +namespace brain { + +/** + * @brief Static class to query what-if cost of an index set. + */ +class WhatIfIndex { + public: + /** + * @brief GetCostAndBestPlanTree + * Perform optimization on the given parsed & bound SQL statement and + * return the best physical plan tree and the cost associated with it. + * + * @param query - parsed and bound query + * @param config - a hypothetical index configuration + * @param database_name - database name string + * @param transaction - already created transaction object. + * @return physical plan info + */ + static std::unique_ptr GetCostAndBestPlanTree( + std::shared_ptr query, IndexConfiguration &config, + std::string database_name, concurrency::TransactionContext *txn); + + /** + * @brief GetCostAndBestPlanTree + * Perform optimization on the given parsed & bound SQL statement and + * return the best physical plan tree and the cost associated with it. + * + * Use this when the referenced table names are already known. + * + * @param query + * @param tables_used + * @param config + * @param database_name + * @param txn + * @return + */ + static std::unique_ptr GetCostAndBestPlanTree( + std::pair, + std::unordered_set> query, + IndexConfiguration &config, std::string database_name, + concurrency::TransactionContext *txn); + + private: + /** + * @brief Creates a hypothetical index catalog object, that would be used + * to fill the catalog cache. + * + * @param obj - Index object + * @return index catalog object + */ + static std::shared_ptr CreateIndexCatalogObject( + HypotheticalIndexObject *obj); + /** + * @brief a monotonically increasing sequence number for creating dummy oids + * for the given hypothetical indexes. + */ + static unsigned long index_seq_no; +}; + +} // namespace brain +} // namespace peloton diff --git a/src/include/capnp/peloton_service.capnp b/src/include/capnp/peloton_service.capnp index 80f8c38a171..2e44fa39d6e 100644 --- a/src/include/capnp/peloton_service.capnp +++ b/src/include/capnp/peloton_service.capnp @@ -1,20 +1,28 @@ @0xf3d342883f3f0344; struct CreateIndexRequest { - databaseName @0 :Text; - tableName @1 :Text; + databaseOid @0 :Int32; + tableOid @1 :Int32; - keyAttrs @2 :List(Int32); + keyAttrOids @2 :List(Int32); indexName @3 :Text; uniqueKeys @4 :Bool; - - indexKeys @5 :Int32; } struct CreateIndexResponse { message @0 :Text; } +struct DropIndexRequest { + databaseOid @0 :Int32; + indexOid @1 :Int32; +} + +struct DropIndexResponse { + message @0 :Text; +} + interface PelotonService { createIndex @0 (request :CreateIndexRequest) -> (response :CreateIndexResponse); + dropIndex @1 (request :DropIndexRequest) -> (response :DropIndexResponse); } diff --git a/src/include/catalog/abstract_catalog.h b/src/include/catalog/abstract_catalog.h index e0c8d81df53..15a66b15a99 100644 --- a/src/include/catalog/abstract_catalog.h +++ b/src/include/catalog/abstract_catalog.h @@ -6,7 +6,7 @@ // // Identification: src/include/catalog/abstract_catalog.h // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -67,6 +67,13 @@ class AbstractCatalog { std::vector values, concurrency::TransactionContext *txn) const; + std::unique_ptr>> + GetResultWithIndexScan(const std::vector &column_offsets, + const oid_t &index_offset, + const std::vector &values, + const std::vector &expr_types, + concurrency::TransactionContext *txn) const; + std::unique_ptr>> GetResultWithSeqScan(std::vector column_offsets, expression::AbstractExpression *predicate, diff --git a/src/include/catalog/index_catalog.h b/src/include/catalog/index_catalog.h index 3ece01952b9..6c80b35377d 100644 --- a/src/include/catalog/index_catalog.h +++ b/src/include/catalog/index_catalog.h @@ -6,29 +6,7 @@ // // Identification: src/include/catalog/index_catalog.h // -// Copyright (c) 2015-17, Carnegie Mellon University Index Group -// -//===----------------------------------------------------------------------===// - -//===----------------------------------------------------------------------===// -// pg_index -// -// Schema: (column: column_name) -// 0: index_oid (pkey) -// 1: index_name -// 2: table_oid (which table this index belongs to) -// 3: schema_name (which namespace this index belongs to) -// 4: index_type (default value is BWTREE) -// 5: index_constraint -// 6: unique_keys (is this index supports duplicate keys) -// 7: indexed_attributes (indicate which table columns this index indexes. For -// example a value of 0 2 would mean that the first and the third table columns -// make up the index.) -// -// Indexes: (index offset: indexed columns) -// 0: index_oid (unique & primary key) -// 1: index_name & schema_name (unique) -// 2: table_oid (non-unique) +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -36,6 +14,7 @@ #include "catalog/abstract_catalog.h" #include "executor/logical_tile.h" +#include namespace peloton { namespace catalog { @@ -46,6 +25,11 @@ class IndexCatalogObject { public: IndexCatalogObject(executor::LogicalTile *tile, int tupleId = 0); + // This constructor should only be used for what-if index API. + IndexCatalogObject(oid_t index_oid, std::string index_name, oid_t table_oid, + IndexType index_type, IndexConstraintType index_constraint, + bool unique_keys, std::vector &key_attrs); + inline oid_t GetIndexOid() { return index_oid; } inline const std::string &GetIndexName() { return index_name; } inline oid_t GetTableOid() { return table_oid; } @@ -94,6 +78,14 @@ class IndexCatalog : public AbstractCatalog { const std::string &index_name, const std::string &schema_name, concurrency::TransactionContext *txn); + /** + * Get all the indexes present in the catalog. + * @param txn + * @return Returns vector of index catalog objects. + */ + std::unordered_map> + GetIndexObjects(concurrency::TransactionContext *txn); + private: std::shared_ptr GetIndexObject( oid_t index_oid, concurrency::TransactionContext *txn); diff --git a/src/include/catalog/query_history_catalog.h b/src/include/catalog/query_history_catalog.h index 3f004508d02..8bd7e6608f4 100644 --- a/src/include/catalog/query_history_catalog.h +++ b/src/include/catalog/query_history_catalog.h @@ -10,16 +10,6 @@ // //===----------------------------------------------------------------------===// -//===----------------------------------------------------------------------===// -// pg_query -// -// Schema: (column offset: column_name) -// 0: query_string -// 1: fingerprint -// 2: timestamp -// -//===----------------------------------------------------------------------===// - #pragma once #include "catalog/abstract_catalog.h" @@ -46,6 +36,10 @@ class QueryHistoryCatalog : public AbstractCatalog { type::AbstractPool *pool, concurrency::TransactionContext *txn); + std::unique_ptr>> + GetQueryStringsAfterTimestamp(const uint64_t start_timestamp, + concurrency::TransactionContext *txn); + enum ColumnId { QUERY_STRING = 0, FINGERPRINT = 1, @@ -57,6 +51,11 @@ class QueryHistoryCatalog : public AbstractCatalog { // Pool to use for variable length strings type::EphemeralPool pool_; + + enum IndexId { + SECONDARY_KEY_0 = 0, + // Add new indexes here in creation order + }; }; } // namespace catalog diff --git a/src/include/catalog/table_catalog.h b/src/include/catalog/table_catalog.h index 9a01ee6e07f..08cbc396696 100644 --- a/src/include/catalog/table_catalog.h +++ b/src/include/catalog/table_catalog.h @@ -6,24 +6,7 @@ // // Identification: src/include/catalog/table_catalog.h // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group -// -//===----------------------------------------------------------------------===// - -//===----------------------------------------------------------------------===// -// pg_table -// -// Schema: (column position: column_name) -// 0: table_oid (pkey) -// 1: table_name, -// 2: schema_name (the namespace name that this table belongs to) -// 3: database_oid -// 4: version_id: for fast ddl(alter table) -// -// Indexes: (index offset: indexed columns) -// 0: table_oid (unique & primary key) -// 1: table_name & schema_name(unique) -// 2: database_oid (non-unique) +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -68,6 +51,11 @@ class TableCatalogObject { std::shared_ptr GetIndexObject( const std::string &index_name, bool cached_only = false); + // Get index objects + bool InsertIndexObject(std::shared_ptr index_object); + bool EvictIndexObject(oid_t index_oid); + bool EvictIndexObject(const std::string &index_name); + // Get columns void EvictAllColumnObjects(); std::unordered_map> @@ -94,6 +82,9 @@ class TableCatalogObject { inline oid_t GetDatabaseOid() { return database_oid; } inline uint32_t GetVersionId() { return version_id; } + // NOTE: should be only used by What-if API. + void SetValidIndexObjects(bool is_valid); + private: // member variables oid_t table_oid; @@ -102,11 +93,6 @@ class TableCatalogObject { oid_t database_oid; uint32_t version_id; - // Get index objects - bool InsertIndexObject(std::shared_ptr index_object); - bool EvictIndexObject(oid_t index_oid); - bool EvictIndexObject(const std::string &index_name); - // Get column objects bool InsertColumnObject(std::shared_ptr column_object); bool EvictColumnObject(oid_t column_id); diff --git a/src/include/common/internal_types.h b/src/include/common/internal_types.h index 22598226407..246bbfc3f37 100644 --- a/src/include/common/internal_types.h +++ b/src/include/common/internal_types.h @@ -1454,5 +1454,7 @@ enum class SSLLevel { typedef std::vector> matrix_t; typedef Eigen::Matrix matrix_eig; +typedef Eigen::Matrix + vector_eig; } // namespace peloton diff --git a/src/include/network/peloton_rpc_handler_task.h b/src/include/network/peloton_rpc_handler_task.h index 8abfa510af4..9334252b0c1 100644 --- a/src/include/network/peloton_rpc_handler_task.h +++ b/src/include/network/peloton_rpc_handler_task.h @@ -11,24 +11,224 @@ //===----------------------------------------------------------------------===// #pragma once +#include #include "capnp/ez-rpc.h" #include "capnp/message.h" +#include "catalog/catalog.h" #include "common/dedicated_thread_task.h" #include "common/logger.h" +#include "common/internal_types.h" #include "kj/debug.h" #include "peloton/capnp/peloton_service.capnp.h" +#include "codegen/buffering_consumer.h" +#include "executor/executor_context.h" +#include "planner/populate_index_plan.h" +#include "storage/storage_manager.h" +#include "planner/seq_scan_plan.h" +#include "catalog/system_catalogs.h" +#include "catalog/column_catalog.h" +#include "binder/bind_node_visitor.h" +#include "catalog/catalog.h" +#include "common/logger.h" +#include "concurrency/transaction_manager_factory.h" +#include "executor/plan_executor.h" +#include "gmock/gtest/gtest.h" +#include "optimizer/optimizer.h" +#include "optimizer/rule.h" +#include "parser/postgresparser.h" +#include "planner/plan_util.h" +#include "optimizer/stats/stats_storage.h" +#include "traffic_cop/traffic_cop.h" namespace peloton { namespace network { class PelotonRpcServerImpl final : public PelotonService::Server { + private: + static std::atomic_int counter_; + protected: - kj::Promise createIndex(CreateIndexContext) override { - // TODO(tianyu) Write actual index code - LOG_DEBUG("Received rpc to create index"); + kj::Promise dropIndex(DropIndexContext request) override { + auto database_oid = request.getParams().getRequest().getDatabaseOid(); + auto index_oid = request.getParams().getRequest().getIndexOid(); + LOG_TRACE("Database oid: %d", database_oid); + LOG_TRACE("Index oid: %d", index_oid); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + // Drop index. Fail if it doesn't exist. + auto catalog = catalog::Catalog::GetInstance(); + try { + catalog->DropIndex(database_oid, index_oid, txn); + } catch (CatalogException e) { + LOG_ERROR("Drop Index Failed"); + txn_manager.AbortTransaction(txn); + return kj::NEVER_DONE; + } + txn_manager.CommitTransaction(txn); return kj::READY_NOW; } -}; + // kj::Promise analyzeTableStats(AnalyzeTableStatsRequest req) override + // { + // auto &txn_manager = + // concurrency::TransactionManagerFactory::GetInstance(); + // auto txn = txn_manager.BeginTransaction(); + // // Generate column stats for all the tables before we begin. + // // TODO[vamshi] + // // Instead of collecting stats for every table, collect them only for + // the + // // tables + // // we are analyzing i.e. tables that are referenced in the current + // workload. + // optimizer::StatsStorage *stats_storage = + // optimizer::StatsStorage::GetInstance(); + // ResultType result = stats_storage->AnalyzeStatsForAllTables(txn); + // if (result != ResultType::SUCCESS) { + // LOG_ERROR( + // "Cannot generate stats for table columns. Not performing index " + // "suggestion..."); + // txn_manager.AbortTransaction(txn); + // return; + // } + // txn_manager.CommitTransaction(txn); + // } + + kj::Promise createIndex(CreateIndexContext request) override { + LOG_DEBUG("Received RPC to create index"); + + auto database_oid = request.getParams().getRequest().getDatabaseOid(); + auto table_oid = request.getParams().getRequest().getTableOid(); + auto col_oids = request.getParams().getRequest().getKeyAttrOids(); + auto index_name = request.getParams().getRequest().getIndexName(); + + std::vector col_oid_vector; + LOG_DEBUG("Database oid: %d", database_oid); + LOG_DEBUG("Table oid: %d", table_oid); + for (auto col : col_oids) { + LOG_DEBUG("Col oid: %d", col); + col_oid_vector.push_back(col); + } + + // Create transaction to query the catalog. + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + // Get the existing table so that we can find its oid and the cols oids. + std::shared_ptr table_object; + try { + table_object = catalog::Catalog::GetInstance()->GetTableObject( + database_oid, table_oid, txn); + } catch (CatalogException e) { + LOG_ERROR("Exception ocurred while getting table: %s", + e.GetMessage().c_str()); + PELOTON_ASSERT(false); + } + + auto table_name = table_object->GetTableName(); + auto col_obj_pairs = table_object->GetColumnObjects(); + + // Done with the transaction. + txn_manager.CommitTransaction(txn); + + // Get all the column names from the oids. + std::vector column_names; + for (auto col_oid : col_oid_vector) { + auto found_itr = col_obj_pairs.find(col_oid); + if (found_itr != col_obj_pairs.end()) { + auto col_obj = found_itr->second; + column_names.push_back(col_obj->GetColumnName()); + } else { + PELOTON_ASSERT(false); + } + } + + // Create "CREATE INDEX" query. + std::ostringstream oss; + oss << "CREATE INDEX " << index_name.cStr() << " ON "; + oss << table_name << "("; + for (auto i = 0UL; i < column_names.size(); i++) { + oss << column_names[i]; + if (i < (column_names.size() - 1)) { + oss << ","; + } + } + oss << ")"; + + LOG_DEBUG("Executing Create Index Query: %s", oss.str().c_str()); + + // Execute the SQL query + std::vector result; + std::vector tuple_descriptor; + std::string error_message; + int rows_affected; + + ExecuteSQLQuery(oss.str(), result, tuple_descriptor, rows_affected, + error_message); + LOG_INFO("Execute query done"); + + return kj::READY_NOW; + } + + static void UtilTestTaskCallback(void *arg) { + std::atomic_int *count = static_cast(arg); + count->store(0); + } + + // TODO: Avoid using this function. + // Copied from SQL testing util. + // Execute a SQL query end-to-end + ResultType ExecuteSQLQuery(const std::string query, + std::vector &result, + std::vector &tuple_descriptor, + int &rows_changed, std::string &error_message) { + std::atomic_int counter_; + + LOG_INFO("Query: %s", query.c_str()); + // prepareStatement + std::string unnamed_statement = "unnamed"; + auto &peloton_parser = parser::PostgresParser::GetInstance(); + auto sql_stmt_list = peloton_parser.BuildParseTree(query); + PELOTON_ASSERT(sql_stmt_list); + if (!sql_stmt_list->is_valid) { + return ResultType::FAILURE; + } + + tcop::TrafficCop traffic_cop_(UtilTestTaskCallback, &counter_); + + auto statement = traffic_cop_.PrepareStatement(unnamed_statement, query, + std::move(sql_stmt_list)); + if (statement.get() == nullptr) { + traffic_cop_.setRowsAffected(0); + rows_changed = 0; + error_message = traffic_cop_.GetErrorMessage(); + return ResultType::FAILURE; + } + // Execute Statement + std::vector param_values; + bool unnamed = false; + std::vector result_format(statement->GetTupleDescriptor().size(), 0); + // SetTrafficCopCounter(); + counter_.store(1); + auto status = traffic_cop_.ExecuteStatement( + statement, param_values, unnamed, nullptr, result_format, result); + if (traffic_cop_.GetQueuing()) { + while (counter_.load() == 1) { + usleep(10); + } + traffic_cop_.ExecuteStatementPlanGetResult(); + status = traffic_cop_.ExecuteStatementGetResult(); + traffic_cop_.SetQueuing(false); + } + if (status == ResultType::SUCCESS) { + tuple_descriptor = statement->GetTupleDescriptor(); + } + LOG_INFO("Statement executed. Result: %s", + ResultTypeToString(status).c_str()); + rows_changed = traffic_cop_.getRowsAffected(); + return status; + } +}; class PelotonRpcHandlerTask : public DedicatedThreadTask { public: diff --git a/src/include/optimizer/optimizer.h b/src/include/optimizer/optimizer.h index 18608c06756..3fa3c1e6258 100644 --- a/src/include/optimizer/optimizer.h +++ b/src/include/optimizer/optimizer.h @@ -15,8 +15,8 @@ #include #include "optimizer/abstract_optimizer.h" -#include "optimizer/property_set.h" #include "optimizer/optimizer_metadata.h" +#include "optimizer/property_set.h" namespace peloton { @@ -53,6 +53,12 @@ struct QueryInfo { std::shared_ptr physical_props; }; +struct OptimizerPlanInfo { + OptimizerPlanInfo(){}; + std::unique_ptr plan; + double cost; +}; + //===--------------------------------------------------------------------===// // Optimizer //===--------------------------------------------------------------------===// @@ -77,6 +83,11 @@ class Optimizer : public AbstractOptimizer { const std::unique_ptr &parse_tree_list, concurrency::TransactionContext *txn) override; + // Used by What-if API + std::unique_ptr GetOptimizedPlanInfo( + std::shared_ptr parsed_statement, + concurrency::TransactionContext *txn); + void OptimizeLoop(int root_group_id, std::shared_ptr required_props); @@ -160,4 +171,4 @@ class Optimizer : public AbstractOptimizer { }; } // namespace optimizer -} // namespace peloton +} // namespace peloton \ No newline at end of file diff --git a/src/include/optimizer/stats_calculator.h b/src/include/optimizer/stats_calculator.h index 5aed2902671..ef4654812dd 100644 --- a/src/include/optimizer/stats_calculator.h +++ b/src/include/optimizer/stats_calculator.h @@ -2,11 +2,11 @@ // // Peloton // -// cost_and_stats_calculator.h +// stats_calculator.h // // Identification: src/include/optimizer/stats_calculator.h // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -26,8 +26,8 @@ class TableStats; */ class StatsCalculator : public OperatorVisitor { public: - void CalculateStats(GroupExpression *gexpr, ExprSet required_cols, - Memo *memo, concurrency::TransactionContext* txn); + void CalculateStats(GroupExpression *gexpr, ExprSet required_cols, Memo *memo, + concurrency::TransactionContext *txn); void Visit(const LogicalGet *) override; void Visit(const LogicalQueryDerivedGet *) override; @@ -68,14 +68,10 @@ class StatsCalculator : public OperatorVisitor { &predicate_stats, const std::vector &predicates); - double CalculateSelectivityForPredicate( - const std::shared_ptr predicate_table_stats, - const expression::AbstractExpression *expr); - GroupExpression *gexpr_; ExprSet required_cols_; Memo *memo_; - concurrency::TransactionContext* txn_; + concurrency::TransactionContext *txn_; }; } // namespace optimizer diff --git a/src/include/optimizer/util.h b/src/include/optimizer/util.h index 634e1297347..487420db2a4 100644 --- a/src/include/optimizer/util.h +++ b/src/include/optimizer/util.h @@ -6,7 +6,7 @@ // // Identification: src/include/optimizer/util.h // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -18,6 +18,7 @@ #include "expression/abstract_expression.h" #include "planner/abstract_plan.h" +#include "optimizer/stats/table_stats.h" namespace peloton { @@ -32,11 +33,11 @@ class DataTable; namespace optimizer { namespace util { - /** - * @brief Convert upper case letters into lower case in a string - * - * @param str The string to operate on - */ +/** + * @brief Convert upper case letters into lower case in a string + * + * @param str The string to operate on + */ inline void to_lower_string(std::string &str) { std::transform(str.begin(), str.end(), str.begin(), ::tolower); } @@ -109,7 +110,6 @@ expression::AbstractExpression *ConstructJoinPredicate( std::unordered_set &table_alias_set, MultiTablePredicates &join_predicates); - /** * @breif Check if there are any join columns in the join expression * For example, expr = (expr_1) AND (expr_2) AND (expr_3) @@ -160,6 +160,18 @@ void ExtractEquiJoinKeys( const std::unordered_set &left_alias, const std::unordered_set &right_alias); +/** + * @brief Calculate selectivity after applying predicates on a table + * + * @param predicate_table_stats the incoming table stats + * @param expr the predicate + * + * @return updated selectivity + */ +double CalculateSelectivityForPredicate( + const std::shared_ptr predicate_table_stats, + const expression::AbstractExpression *expr); + } // namespace util } // namespace optimizer } // namespace peloton diff --git a/src/include/planner/plan_util.h b/src/include/planner/plan_util.h index ca73beb46ea..259a5b989b3 100644 --- a/src/include/planner/plan_util.h +++ b/src/include/planner/plan_util.h @@ -56,14 +56,14 @@ class PlanUtil { const planner::AbstractPlan *plan); /** - * @brief Get the indexes affected by a given query - * @param CatalogCache - * @param SQLStatement - * @return set of affected index object ids - */ - static const std::set GetAffectedIndexes( + * @brief Get the indexes affected by a given query + * @param CatalogCache + * @param SQLStatement + * @return vector of affected index ids with triplet format + */ + static const std::vector GetAffectedIndexes( catalog::CatalogCache &catalog_cache, - const parser::SQLStatement &sql_stmt); + const parser::SQLStatement &sql_stmt, const bool ignore_primary = false); /** * @brief Get the columns affected by a given query diff --git a/src/main/peloton/peloton.cpp b/src/main/peloton/peloton.cpp index 8c5e0b204c6..c3572ee8190 100644 --- a/src/main/peloton/peloton.cpp +++ b/src/main/peloton/peloton.cpp @@ -6,18 +6,20 @@ // // Identification: src/main/peloton/peloton.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// #include #include +#include #include "common/init.h" #include "common/logger.h" #include "network/peloton_server.h" #include "settings/settings_manager.h" #include "brain/brain.h" +#include "brain/index_selection_job.h" // For GFlag's built-in help message flag DECLARE_bool(help); @@ -35,8 +37,9 @@ int RunPelotonServer() { peloton_server.SetupServer().ServerLoop(); } catch (peloton::ConnectionException &exception) { - //log error message and mark failure - peloton::LOG_ERROR("Cannot start server. Failure detail : %s\n", exception.GetMessage().c_str()); + // log error message and mark failure + peloton::LOG_ERROR("Cannot start server. Failure detail : %s\n", + exception.GetMessage().c_str()); return_code = EXIT_FAILURE; } @@ -45,32 +48,26 @@ int RunPelotonServer() { return return_code; } - int RunPelotonBrain() { // TODO(tianyu): boot up other peloton resources as needed here peloton::brain::Brain brain; evthread_use_pthreads(); // TODO(tianyu): register jobs here - struct timeval one_second; - one_second.tv_sec = 1; - one_second.tv_usec = 0; - - auto example_task = [](peloton::brain::BrainEnvironment *) { - // TODO(tianyu): Replace with real address - capnp::EzRpcClient client("localhost:15445"); - PelotonService::Client peloton_service = client.getMain(); - auto request = peloton_service.createIndexRequest(); - request.getRequest().setIndexKeys(42); - auto response = request.send().wait(client.getWaitScope()); - }; - - brain.RegisterJob(&one_second, "test", example_task); + struct timeval one_minute; + one_minute.tv_sec = 5; + one_minute.tv_usec = 0; + + // The handler for the Index Suggestion related RPC calls to create/drop + // indexes + // TODO[vamshi]: Remove this hard coding + auto num_queries_threshold = 2; + brain.RegisterJob( + &one_minute, "index_suggestion", num_queries_threshold); brain.Run(); return 0; } int main(int argc, char *argv[]) { - // Parse the command line flags ::google::ParseCommandLineNonHelpFlags(&argc, &argv, true); @@ -83,20 +80,43 @@ int main(int argc, char *argv[]) { try { // Print settings if (peloton::settings::SettingsManager::GetBool( - peloton::settings::SettingId::display_settings)) { + peloton::settings::SettingId::display_settings)) { auto &settings = peloton::settings::SettingsManager::GetInstance(); settings.ShowInfo(); } } catch (peloton::SettingsException &exception) { - peloton::LOG_ERROR("Cannot load settings. Failed with %s\n", exception.GetMessage().c_str()); - return EXIT_FAILURE; // TODO: Use an enum with exit error codes + peloton::LOG_ERROR("Cannot load settings. Failed with %s\n", + exception.GetMessage().c_str()); + return EXIT_FAILURE; // TODO: Use an enum with exit error codes } - int exit_code = 0; - if (peloton::settings::SettingsManager::GetBool( - peloton::settings::SettingId::brain)) - exit_code = RunPelotonBrain(); - else - exit_code = RunPelotonServer(); +// int exit_code = 0; +// if (peloton::settings::SettingsManager::GetBool( +// peloton::settings::SettingId::brain)) +// exit_code = RunPelotonBrain(); +// else { +// exit_code = RunPelotonServer(); + + // TODO[Siva]: Remove this from the final PR. Uncomment this to run brain + // and server in the same process for testing. This is a temporary to way to + // run both peloton server and the brain together to test the index suggestion + // at the brain end without catalog replication between the server and the + // brain + peloton::settings::SettingsManager::SetBool( + peloton::settings::SettingId::brain, true); + peloton::settings::SettingsManager::SetBool( + peloton::settings::SettingId::rpc_enabled, true); + + int exit_code = 0; + if (peloton::settings::SettingsManager::GetBool( + peloton::settings::SettingId::brain)) { + std::thread brain(RunPelotonBrain); + exit_code = RunPelotonServer(); + brain.join(); + } else { + exit_code = RunPelotonServer(); + } + + return exit_code; } diff --git a/src/optimizer/cost_calculator.cpp b/src/optimizer/cost_calculator.cpp index 56cbbecc64e..607086d02d7 100644 --- a/src/optimizer/cost_calculator.cpp +++ b/src/optimizer/cost_calculator.cpp @@ -2,11 +2,11 @@ // // Peloton // -// cost_and_stats_calculator.h +// cost_calculator.cpp // // Identification: src/optimizer/cost_calculator.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -14,7 +14,10 @@ #include +#include "catalog/column_catalog.h" #include "catalog/table_catalog.h" +#include "catalog/index_catalog.h" +#include "expression/tuple_value_expression.h" #include "optimizer/memo.h" #include "optimizer/operators.h" #include "optimizer/stats/cost.h" @@ -50,14 +53,73 @@ void CostCalculator::Visit(UNUSED_ATTRIBUTE const PhysicalIndexScan *op) { auto table_stats = std::dynamic_pointer_cast( StatsStorage::GetInstance()->GetTableStats( op->table_->GetDatabaseOid(), op->table_->GetTableOid(), txn_)); - if (table_stats->GetColumnCount() == 0 || table_stats->num_rows == 0) { + auto index_scan_rows = (double)table_stats->num_rows; + if (table_stats->GetColumnCount() == 0 || index_scan_rows == 0) { output_cost_ = 0.f; return; } + auto index_object = op->table_->GetIndexObject(op->index_id); + const auto &key_attr_list = index_object->GetKeyAttrs(); + // Loop over index to retrieve helpful index columns + // Consider all predicates that could be accelerated by the index, + // i.e. till the first column with no equality predicate on it + // index cols (a, b, c) + // example1 : predicates(a=1 AND b=2 AND c=3) index helps on both a, b and c + // example2 : predicates(a<1 AND b<=2 and c<3) index helps on only a + // example3 : predicates(a=1 AND b>2 AND c>3) index helps on a and b + bool has_non_equality_pred = false; + for (size_t idx = 0; idx < key_attr_list.size(); ++idx) { + // If index cannot further reduce scan range, break + if (idx == op->key_column_id_list.size() || + key_attr_list[idx] != op->key_column_id_list[idx]) { + break; + } + auto index_col_id = key_attr_list[idx]; + // Find the predicate and update scan rows accordingly + for (auto &predicate : op->predicates) { + auto &expr = predicate.expr; + // TODO(boweic): support non equality predicates + if (expr->GetExpressionType() != ExpressionType::COMPARE_EQUAL) { + has_non_equality_pred = true; + } + expression::AbstractExpression *tv_expr = nullptr; + if (expr->GetChild(0)->GetExpressionType() == + ExpressionType::VALUE_TUPLE) { + auto r_type = expr->GetChild(1)->GetExpressionType(); + if (r_type == ExpressionType::VALUE_CONSTANT || + r_type == ExpressionType::VALUE_PARAMETER) { + tv_expr = expr->GetModifiableChild(0); + } + } + if (expr->GetChild(1)->GetExpressionType() == + ExpressionType::VALUE_TUPLE) { + auto r_type = expr->GetChild(0)->GetExpressionType(); + if (r_type == ExpressionType::VALUE_CONSTANT || + r_type == ExpressionType::VALUE_PARAMETER) { + tv_expr = expr->GetModifiableChild(1); + } + } + if (tv_expr == nullptr) { + continue; + } + auto column_ref = + reinterpret_cast(tv_expr); + auto column_id = op->table_->GetColumnObject(column_ref->GetColumnName()) + ->GetColumnId(); + if (column_id != index_col_id) { + continue; + } + // update selectivity here + index_scan_rows *= + util::CalculateSelectivityForPredicate(table_stats, expr.get()); + } + if (has_non_equality_pred) { + break; + } + } // Index search cost + scan cost output_cost_ = std::log2(table_stats->num_rows) * DEFAULT_INDEX_TUPLE_COST + - memo_->GetGroupByID(gexpr_->GetGroupID())->GetNumRows() * - DEFAULT_TUPLE_COST; + index_scan_rows * DEFAULT_TUPLE_COST; } void CostCalculator::Visit(UNUSED_ATTRIBUTE const ExternalFileScan *) { @@ -93,7 +155,8 @@ void CostCalculator::Visit(UNUSED_ATTRIBUTE const PhysicalInnerHashJoin *op) { memo_->GetGroupByID(gexpr_->GetChildGroupId(0))->GetNumRows(); auto right_child_rows = memo_->GetGroupByID(gexpr_->GetChildGroupId(1))->GetNumRows(); - // TODO(boweic): Build (left) table should have different cost to probe table + // TODO(boweic): Build (left) table should have different cost to probe + // table output_cost_ = (left_child_rows + right_child_rows) * DEFAULT_TUPLE_COST; } void CostCalculator::Visit(UNUSED_ATTRIBUTE const PhysicalLeftHashJoin *op) {} diff --git a/src/optimizer/optimizer.cpp b/src/optimizer/optimizer.cpp index 2525915fcc1..9fdde28ec2b 100644 --- a/src/optimizer/optimizer.cpp +++ b/src/optimizer/optimizer.cpp @@ -6,7 +6,7 @@ // // Identification: src/optimizer/optimizer.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -21,16 +21,16 @@ #include "common/exception.h" #include "optimizer/binding.h" -#include "optimizer/input_column_deriver.h" #include "optimizer/operator_visitor.h" -#include "optimizer/optimize_context.h" -#include "optimizer/optimizer_task_pool.h" -#include "optimizer/plan_generator.h" #include "optimizer/properties.h" #include "optimizer/property_enforcer.h" #include "optimizer/query_to_operator_transformer.h" +#include "optimizer/input_column_deriver.h" +#include "optimizer/plan_generator.h" #include "optimizer/rule.h" #include "optimizer/rule_impls.h" +#include "optimizer/optimizer_task_pool.h" +#include "optimizer/optimize_context.h" #include "parser/create_statement.h" #include "planner/analyze_plan.h" @@ -113,7 +113,8 @@ shared_ptr Optimizer::BuildPelotonPlanTree( // Generate initial operator tree from query tree shared_ptr gexpr = InsertQueryTree(parse_tree, txn); GroupID root_id = gexpr->GetGroupID(); - // Get the physical properties the final plan must output + + // Get the physical properties and projected columns the final plan must have auto query_info = GetQueryInfo(parse_tree); try { @@ -136,6 +137,54 @@ shared_ptr Optimizer::BuildPelotonPlanTree( } } +// GetOptimizedQueryTree() +// Return an optimized physical query tree for the given parse tree along +// with the cost. +std::unique_ptr Optimizer::GetOptimizedPlanInfo( + std::shared_ptr parsed_statement, + concurrency::TransactionContext *txn) { + metadata_.txn = txn; + + // Generate initial operator tree to work with from the parsed + // statement object. + std::shared_ptr g_expr = + InsertQueryTree(parsed_statement.get(), txn); + GroupID root_id = g_expr->GetGroupID(); + + // Get the physical properties of the final plan that must be enforced + auto query_info = GetQueryInfo(parsed_statement.get()); + + // Start with the base expression and explore all the possible transformations + // and add them to the local context. + try { + OptimizeLoop(root_id, query_info.physical_props); + } catch (OptimizerException &e) { + LOG_WARN("Optimize Loop ended prematurely: %s", e.what()); + PELOTON_ASSERT(false); + } + + try { + auto best_plan = ChooseBestPlan(root_id, query_info.physical_props, + query_info.output_exprs); + auto info_obj = std::unique_ptr(new OptimizerPlanInfo()); + + // Get the cost. + auto group = GetMetadata().memo.GetGroupByID(root_id); + auto best_expr = group->GetBestExpression(query_info.physical_props); + + info_obj->cost = best_expr->GetCost(query_info.physical_props); + info_obj->plan = std::move(best_plan); + + // Reset memo after finishing the optimization + Reset(); + + return info_obj; + } catch (Exception &e) { + Reset(); + throw e; + } +} + void Optimizer::Reset() { metadata_ = OptimizerMetadata(); } unique_ptr Optimizer::HandleDDLStatement( @@ -231,29 +280,29 @@ shared_ptr Optimizer::InsertQueryTree( } QueryInfo Optimizer::GetQueryInfo(parser::SQLStatement *tree) { - auto GetQueryInfoHelper = - [](std::vector> &select_list, - std::unique_ptr &order_info, - std::vector &output_exprs, - std::shared_ptr &physical_props) { - // Extract output column - for (auto &expr : select_list) output_exprs.push_back(expr.get()); - - // Extract sort property - if (order_info != nullptr) { - std::vector sort_exprs; - std::vector sort_ascending; - for (auto &expr : order_info->exprs) { - sort_exprs.push_back(expr.get()); - } - for (auto &type : order_info->types) { - sort_ascending.push_back(type == parser::kOrderAsc); - } - if (!sort_exprs.empty()) - physical_props->AddProperty( - std::make_shared(sort_exprs, sort_ascending)); - } - }; + auto GetQueryInfoHelper = []( + std::vector> &select_list, + std::unique_ptr &order_info, + std::vector &output_exprs, + std::shared_ptr &physical_props) { + // Extract output column + for (auto &expr : select_list) output_exprs.push_back(expr.get()); + + // Extract sort property + if (order_info != nullptr) { + std::vector sort_exprs; + std::vector sort_ascending; + for (auto &expr : order_info->exprs) { + sort_exprs.push_back(expr.get()); + } + for (auto &type : order_info->types) { + sort_ascending.push_back(type == parser::kOrderAsc); + } + if (!sort_exprs.empty()) + physical_props->AddProperty( + std::make_shared(sort_exprs, sort_ascending)); + } + }; std::vector output_exprs; std::shared_ptr physical_props = std::make_shared(); diff --git a/src/optimizer/rule_impls.cpp b/src/optimizer/rule_impls.cpp index 33fb241df8d..1f249d8f17d 100644 --- a/src/optimizer/rule_impls.cpp +++ b/src/optimizer/rule_impls.cpp @@ -6,7 +6,7 @@ // // Identification: src/optimizer/rule_impls.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -312,9 +312,8 @@ void GetToIndexScan::Transform( // Check whether any index can fulfill predicate predicate evaluation if (!get->predicates.empty()) { - std::vector key_column_id_list; - std::vector expr_type_list; - std::vector value_list; + std::unordered_map> + type_value_pair_by_key_id; for (auto &pred : get->predicates) { auto expr = pred.expr.get(); if (expr->GetChildrenSize() != 2) continue; @@ -351,25 +350,25 @@ void GetToIndexScan::Transform( std::string col_name(column_ref->GetColumnName()); LOG_TRACE("Column name: %s", col_name.c_str()); auto column_id = get->table->GetColumnObject(col_name)->GetColumnId(); - key_column_id_list.push_back(column_id); - expr_type_list.push_back(expr_type); - + type::Value value; if (value_expr->GetExpressionType() == ExpressionType::VALUE_CONSTANT) { - value_list.push_back( - reinterpret_cast( - value_expr)->GetValue()); + value = reinterpret_cast( + value_expr) + ->GetValue(); LOG_TRACE("Value Type: %d", static_cast( reinterpret_cast( expr->GetModifiableChild(1))->GetValueType())); } else { - value_list.push_back( - type::ValueFactory::GetParameterOffsetValue( - reinterpret_cast( - value_expr)->GetValueIdx()).Copy()); + value = type::ValueFactory::GetParameterOffsetValue( + reinterpret_cast( + value_expr) + ->GetValueIdx()) + .Copy(); LOG_TRACE("Parameter offset: %s", (*value_list.rbegin()).GetInfo().c_str()); } + type_value_pair_by_key_id[column_id] = {expr_type, value}; } } // Loop predicates end @@ -384,12 +383,19 @@ void GetToIndexScan::Transform( std::unordered_set index_col_set( index_object->GetKeyAttrs().begin(), index_object->GetKeyAttrs().end()); - for (size_t offset = 0; offset < key_column_id_list.size(); offset++) { - auto col_id = key_column_id_list[offset]; - if (index_col_set.find(col_id) != index_col_set.end()) { - index_key_column_id_list.push_back(col_id); - index_expr_type_list.push_back(expr_type_list[offset]); - index_value_list.push_back(value_list[offset]); + // If the first index key column present in the predicate's column id map + // then we would let the cost model to decide if we want to use the index + const auto &key_attr_list = index_object->GetKeyAttrs(); + if (!key_attr_list.empty() && + type_value_pair_by_key_id.count(key_attr_list[0])) { + for (const auto &key_col_oid : key_attr_list) { + if (type_value_pair_by_key_id.count(key_col_oid)) { + const auto &type_value_pair = + type_value_pair_by_key_id[key_col_oid]; + index_key_column_id_list.push_back(key_col_oid); + index_expr_type_list.push_back(type_value_pair.first); + index_value_list.push_back(type_value_pair.second); + } } } // Add transformed plan diff --git a/src/optimizer/stats/selectivity.cpp b/src/optimizer/stats/selectivity.cpp index 474ae1a71da..0586ad31eb9 100644 --- a/src/optimizer/stats/selectivity.cpp +++ b/src/optimizer/stats/selectivity.cpp @@ -6,7 +6,7 @@ // // Identification: src/optimizer/stats/selectivity.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -84,7 +84,7 @@ double Selectivity::Equal(const std::shared_ptr &table_stats, auto column_stats = table_stats->GetColumnStats(condition.column_name); // LOG_INFO("column name %s", condition.column_name); if (std::isnan(value) || column_stats == nullptr) { - LOG_DEBUG("Calculate selectivity: return null"); + LOG_TRACE("Calculate selectivity: return null"); return DEFAULT_SELECTIVITY; } diff --git a/src/optimizer/stats_calculator.cpp b/src/optimizer/stats_calculator.cpp index 3cdb34c4d9d..f9d5685a3c3 100644 --- a/src/optimizer/stats_calculator.cpp +++ b/src/optimizer/stats_calculator.cpp @@ -2,11 +2,11 @@ // // Peloton // -// cost_and_stats_calculator.h +// stats_calculator.cpp // // Identification: src/optimizer/stats_calculator.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -42,8 +42,8 @@ void StatsCalculator::Visit(const LogicalGet *op) { return; } auto table_stats = std::dynamic_pointer_cast( - StatsStorage::GetInstance()->GetTableStats(op->table->GetDatabaseOid(), - op->table->GetTableOid(), txn_)); + StatsStorage::GetInstance()->GetTableStats( + op->table->GetDatabaseOid(), op->table->GetTableOid(), txn_)); // First, get the required stats of the base table std::unordered_map> required_stats; for (auto &col : required_cols_) { @@ -143,7 +143,8 @@ void StatsCalculator::Visit(const LogicalInnerJoin *op) { column_stats = std::make_shared( *left_child_group->GetStats(tv_expr->GetColFullName())); } else { - PELOTON_ASSERT(right_child_group->HasColumnStats(tv_expr->GetColFullName())); + PELOTON_ASSERT( + right_child_group->HasColumnStats(tv_expr->GetColFullName())); column_stats = std::make_shared( *right_child_group->GetStats(tv_expr->GetColFullName())); } @@ -251,96 +252,12 @@ void StatsCalculator::UpdateStatsForFilter( double selectivity = 1.f; for (auto &annotated_expr : predicates) { // Loop over conjunction exprs - selectivity *= CalculateSelectivityForPredicate(predicate_table_stats, - annotated_expr.expr.get()); + selectivity *= util::CalculateSelectivityForPredicate( + predicate_table_stats, annotated_expr.expr.get()); } // Update selectivity memo_->GetGroupByID(gexpr_->GetGroupID())->SetNumRows(num_rows * selectivity); } -// Calculate the selectivity given the predicate and the stats of columns in the -// predicate -double StatsCalculator::CalculateSelectivityForPredicate( - const std::shared_ptr predicate_table_stats, - const expression::AbstractExpression *expr) { - double selectivity = 1.f; - if (predicate_table_stats->GetColumnCount() == 0 || - predicate_table_stats->GetColumnStats(0)->num_rows == 0) { - return selectivity; - } - // Base case : Column Op Val - if ((expr->GetChild(0)->GetExpressionType() == ExpressionType::VALUE_TUPLE && - (expr->GetChild(1)->GetExpressionType() == - ExpressionType::VALUE_CONSTANT || - expr->GetChild(1)->GetExpressionType() == - ExpressionType::VALUE_PARAMETER)) || - (expr->GetChild(1)->GetExpressionType() == ExpressionType::VALUE_TUPLE && - (expr->GetChild(0)->GetExpressionType() == - ExpressionType::VALUE_CONSTANT || - expr->GetChild(0)->GetExpressionType() == - ExpressionType::VALUE_PARAMETER))) { - int right_index = - expr->GetChild(0)->GetExpressionType() == ExpressionType::VALUE_TUPLE - ? 1 - : 0; - - auto left_expr = expr->GetChild(1 - right_index); - PELOTON_ASSERT(left_expr->GetExpressionType() == ExpressionType::VALUE_TUPLE); - auto col_name = - reinterpret_cast(left_expr) - ->GetColFullName(); - - auto expr_type = expr->GetExpressionType(); - if (right_index == 0) { - switch (expr_type) { - case ExpressionType::COMPARE_LESSTHANOREQUALTO: - expr_type = ExpressionType::COMPARE_GREATERTHANOREQUALTO; - break; - case ExpressionType::COMPARE_LESSTHAN: - expr_type = ExpressionType::COMPARE_GREATERTHAN; - break; - case ExpressionType::COMPARE_GREATERTHANOREQUALTO: - expr_type = ExpressionType::COMPARE_LESSTHANOREQUALTO; - break; - case ExpressionType::COMPARE_GREATERTHAN: - expr_type = ExpressionType::COMPARE_LESSTHAN; - break; - default: - break; - } - } - - type::Value value; - if (expr->GetChild(right_index)->GetExpressionType() == - ExpressionType::VALUE_CONSTANT) { - value = reinterpret_cast( - expr->GetModifiableChild(right_index)) - ->GetValue(); - } else { - value = type::ValueFactory::GetParameterOffsetValue( - reinterpret_cast( - expr->GetModifiableChild(right_index)) - ->GetValueIdx()) - .Copy(); - } - ValueCondition condition(col_name, expr_type, value); - selectivity = - Selectivity::ComputeSelectivity(predicate_table_stats, condition); - } else if (expr->GetExpressionType() == ExpressionType::CONJUNCTION_AND || - expr->GetExpressionType() == ExpressionType::CONJUNCTION_OR) { - double left_selectivity = CalculateSelectivityForPredicate( - predicate_table_stats, expr->GetChild(0)); - double right_selectivity = CalculateSelectivityForPredicate( - predicate_table_stats, expr->GetChild(1)); - if (expr->GetExpressionType() == ExpressionType::CONJUNCTION_AND) { - selectivity = left_selectivity * right_selectivity; - } else { - selectivity = left_selectivity + right_selectivity - - left_selectivity * right_selectivity; - } - } - return selectivity; -} - } // namespace optimizer } // namespace peloton diff --git a/src/optimizer/util.cpp b/src/optimizer/util.cpp index 07685376b34..32fc9cabf6f 100644 --- a/src/optimizer/util.cpp +++ b/src/optimizer/util.cpp @@ -6,10 +6,12 @@ // // Identification: src/optimizer/util.cpp // -// Copyright (c) 2015-16, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// +#include "optimizer/stats/selectivity.h" +#include "optimizer/stats/value_condition.h" #include "optimizer/util.h" #include "catalog/query_metrics_catalog.h" @@ -143,8 +145,7 @@ std::unordered_map> ConstructSelectElementMap( std::vector> &select_list) { std::unordered_map> - res; + std::shared_ptr> res; for (auto &expr : select_list) { std::string alias; if (!expr->alias.empty()) { @@ -214,6 +215,89 @@ void ExtractEquiJoinKeys( } } +// Calculate the selectivity given the predicate and the stats of columns in the +// predicate +double CalculateSelectivityForPredicate( + const std::shared_ptr predicate_table_stats, + const expression::AbstractExpression *expr) { + double selectivity = 1.f; + if (predicate_table_stats->GetColumnCount() == 0 || + predicate_table_stats->GetColumnStats(0)->num_rows == 0) { + return selectivity; + } + // Base case : Column Op Val + if ((expr->GetChild(0)->GetExpressionType() == ExpressionType::VALUE_TUPLE && + (expr->GetChild(1)->GetExpressionType() == + ExpressionType::VALUE_CONSTANT || + expr->GetChild(1)->GetExpressionType() == + ExpressionType::VALUE_PARAMETER)) || + (expr->GetChild(1)->GetExpressionType() == ExpressionType::VALUE_TUPLE && + (expr->GetChild(0)->GetExpressionType() == + ExpressionType::VALUE_CONSTANT || + expr->GetChild(0)->GetExpressionType() == + ExpressionType::VALUE_PARAMETER))) { + int right_index = + expr->GetChild(0)->GetExpressionType() == ExpressionType::VALUE_TUPLE + ? 1 + : 0; + + auto left_expr = expr->GetChild(1 - right_index); + auto col_name = + reinterpret_cast(left_expr) + ->GetColFullName(); + + auto expr_type = expr->GetExpressionType(); + if (right_index == 0) { + switch (expr_type) { + case ExpressionType::COMPARE_LESSTHANOREQUALTO: + expr_type = ExpressionType::COMPARE_GREATERTHANOREQUALTO; + break; + case ExpressionType::COMPARE_LESSTHAN: + expr_type = ExpressionType::COMPARE_GREATERTHAN; + break; + case ExpressionType::COMPARE_GREATERTHANOREQUALTO: + expr_type = ExpressionType::COMPARE_LESSTHANOREQUALTO; + break; + case ExpressionType::COMPARE_GREATERTHAN: + expr_type = ExpressionType::COMPARE_LESSTHAN; + break; + default: + break; + } + } + + type::Value value; + if (expr->GetChild(right_index)->GetExpressionType() == + ExpressionType::VALUE_CONSTANT) { + value = reinterpret_cast( + expr->GetModifiableChild(right_index)) + ->GetValue(); + } else { + value = type::ValueFactory::GetParameterOffsetValue( + reinterpret_cast( + expr->GetModifiableChild(right_index)) + ->GetValueIdx()) + .Copy(); + } + ValueCondition condition(col_name, expr_type, value); + selectivity = + Selectivity::ComputeSelectivity(predicate_table_stats, condition); + } else if (expr->GetExpressionType() == ExpressionType::CONJUNCTION_AND || + expr->GetExpressionType() == ExpressionType::CONJUNCTION_OR) { + double left_selectivity = CalculateSelectivityForPredicate( + predicate_table_stats, expr->GetChild(0)); + double right_selectivity = CalculateSelectivityForPredicate( + predicate_table_stats, expr->GetChild(1)); + if (expr->GetExpressionType() == ExpressionType::CONJUNCTION_AND) { + selectivity = left_selectivity * right_selectivity; + } else { + selectivity = left_selectivity + right_selectivity - + left_selectivity * right_selectivity; + } + } + return selectivity; +} + } // namespace util } // namespace optimizer } // namespace peloton diff --git a/src/planner/plan_util.cpp b/src/planner/plan_util.cpp index cb013fb1531..5d06ec9dfd1 100644 --- a/src/planner/plan_util.cpp +++ b/src/planner/plan_util.cpp @@ -33,11 +33,14 @@ namespace peloton { namespace planner { -const std::set PlanUtil::GetAffectedIndexes( - catalog::CatalogCache &catalog_cache, - const parser::SQLStatement &sql_stmt) { - std::set index_oids; +const std::vector PlanUtil::GetAffectedIndexes( + catalog::CatalogCache &catalog_cache, const parser::SQLStatement &sql_stmt, + const bool ignore_primary) { + std::vector index_triplets; std::string db_name, table_name, schema_name; + std::shared_ptr db_object; + std::shared_ptr table_object; + oid_t db_oid, table_oid; switch (sql_stmt.GetType()) { // For INSERT, DELETE, all indexes are affected case StatementType::INSERT: { @@ -45,7 +48,11 @@ const std::set PlanUtil::GetAffectedIndexes( static_cast(sql_stmt); db_name = insert_stmt.GetDatabaseName(); table_name = insert_stmt.GetTableName(); + db_object = catalog_cache.GetDatabaseObject(db_name); + db_oid = db_object->GetDatabaseOid(); schema_name = insert_stmt.GetSchemaName(); + table_object = db_object->GetTableObject(table_name, schema_name); + table_oid = table_object->GetTableOid(); } PELOTON_FALLTHROUGH; case StatementType::DELETE: { @@ -54,23 +61,42 @@ const std::set PlanUtil::GetAffectedIndexes( static_cast(sql_stmt); db_name = delete_stmt.GetDatabaseName(); table_name = delete_stmt.GetTableName(); + db_object = catalog_cache.GetDatabaseObject(db_name); + db_oid = db_object->GetDatabaseOid(); schema_name = delete_stmt.GetSchemaName(); + table_object = db_object->GetTableObject(table_name, schema_name); + table_oid = table_object->GetTableOid(); } - auto indexes_map = catalog_cache.GetDatabaseObject(db_name) - ->GetTableObject(table_name, schema_name) - ->GetIndexObjects(); + auto indexes_map = table_object->GetIndexObjects(); for (auto &index : indexes_map) { - index_oids.insert(index.first); + bool add_index = true; + + if (ignore_primary) { + const auto col_oids = index.second->GetKeyAttrs(); + for (const auto col_oid : col_oids) { + if (table_object->GetColumnObject(col_oid)->IsPrimary()) { + add_index = false; + break; + } + } + } + + if (add_index) { + index_triplets.emplace_back(db_oid, table_oid, index.first); + } } + } break; case StatementType::UPDATE: { auto &update_stmt = static_cast(sql_stmt); db_name = update_stmt.table->GetDatabaseName(); table_name = update_stmt.table->GetTableName(); + db_object = catalog_cache.GetDatabaseObject(db_name); schema_name = update_stmt.table->GetSchemaName(); - auto table_object = catalog_cache.GetDatabaseObject(db_name) - ->GetTableObject(table_name, schema_name); + table_object = db_object->GetTableObject(table_name, schema_name); + db_oid = db_object->GetDatabaseOid(); + table_oid = table_object->GetTableOid(); auto &update_clauses = update_stmt.updates; std::set update_oids; @@ -78,6 +104,7 @@ const std::set PlanUtil::GetAffectedIndexes( LOG_TRACE("Affected column name for table(%s) in UPDATE query: %s", table_name.c_str(), update_clause->column.c_str()); auto col_object = table_object->GetColumnObject(update_clause->column); + update_oids.insert(col_object->GetColumnId()); } @@ -91,7 +118,20 @@ const std::set PlanUtil::GetAffectedIndexes( if (!SetUtil::IsDisjoint(key_attrs_set, update_oids)) { LOG_TRACE("Index (%s) is affected", index.second->GetIndexName().c_str()); - index_oids.insert(index.first); + bool add_index = true; + + if (ignore_primary) { + for (const auto col_oid : key_attrs) { + if (table_object->GetColumnObject(col_oid)->IsPrimary()) { + add_index = false; + break; + } + } + } + + if (add_index) { + index_triplets.emplace_back(db_oid, table_oid, index.first); + } } } } break; @@ -101,7 +141,7 @@ const std::set PlanUtil::GetAffectedIndexes( LOG_TRACE("Does not support finding affected indexes for query type: %d", static_cast(sql_stmt.GetType())); } - return (index_oids); + return (index_triplets); } const std::vector PlanUtil::GetIndexableColumns( @@ -183,7 +223,6 @@ const std::vector PlanUtil::GetIndexableColumns( LOG_ERROR("Error in BuildPelotonPlanTree: %s", e.what()); } - // TODO: should transaction commit or not? txn_manager.AbortTransaction(txn); } break; default: diff --git a/src/storage/data_table.cpp b/src/storage/data_table.cpp index 3660fcc2f79..999c39df119 100644 --- a/src/storage/data_table.cpp +++ b/src/storage/data_table.cpp @@ -6,7 +6,7 @@ // // Identification: src/storage/data_table.cpp // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -386,7 +386,7 @@ bool DataTable::InsertTuple(const AbstractTuple *tuple, ItemPointer location, } PELOTON_ASSERT((*index_entry_ptr)->block == location.block && - (*index_entry_ptr)->offset == location.offset); + (*index_entry_ptr)->offset == location.offset); // Increase the table's number of tuples by 1 IncreaseTupleCount(1); @@ -1094,7 +1094,12 @@ void DataTable::DropIndexWithOid(const oid_t &index_oid) { indexes_.Update(index_offset, nullptr); // Drop index column info - indexes_columns_[index_offset].clear(); + // indexes_columns_[index_offset].clear(); + + // Doing this because StatsStorage::AnalyzeStatsForAllTables + // assumes that the set is completely erased when the index is + // deleted. + indexes_columns_.erase(indexes_columns_.begin() + index_offset); } void DataTable::DropIndexes() { diff --git a/src/storage/tile_group_header.cpp b/src/storage/tile_group_header.cpp index 1e0b450144e..56a4cb37017 100644 --- a/src/storage/tile_group_header.cpp +++ b/src/storage/tile_group_header.cpp @@ -1,3 +1,15 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// tile_group_header.cpp +// +// Identification: src/storage/tile_group_header.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + //===----------------------------------------------------------------------===// // // Peloton @@ -239,7 +251,8 @@ oid_t TileGroupHeader::GetActiveTupleCount() const { tuple_slot_id++) { txn_id_t tuple_txn_id = GetTransactionId(tuple_slot_id); if (tuple_txn_id != INVALID_TXN_ID) { - PELOTON_ASSERT(tuple_txn_id == INITIAL_TXN_ID); + // TODO Copying what Tiyanu did + // PELOTON_ASSERT(tuple_txn_id == INITIAL_TXN_ID); active_tuple_slots++; } } diff --git a/src/traffic_cop/traffic_cop.cpp b/src/traffic_cop/traffic_cop.cpp index 7bfffebb4c0..2fb0b70fa96 100644 --- a/src/traffic_cop/traffic_cop.cpp +++ b/src/traffic_cop/traffic_cop.cpp @@ -6,7 +6,7 @@ // // Identification: src/traffic_cop/traffic_cop.cpp // -// Copyright (c) 2015-17, Carnegie Mellon University Database Group +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group // //===----------------------------------------------------------------------===// @@ -305,8 +305,10 @@ std::shared_ptr TrafficCop::PrepareStatement( tcop_txn_state_.emplace(txn, ResultType::SUCCESS); } + // Log the query only if we have a statement. if (settings::SettingsManager::GetBool(settings::SettingId::brain)) { - tcop_txn_state_.top().first->AddQueryString(query_string.c_str()); + tcop_txn_state_.top().first->AddQueryString( + query_string.c_str()); } // TODO(Tianyi) Move Statement Planing into Statement's method diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt index 94291523cdd..1385289866e 100644 --- a/test/CMakeLists.txt +++ b/test/CMakeLists.txt @@ -48,6 +48,7 @@ set(TESTING_UTIL_STATS ${PROJECT_SOURCE_DIR}/test/statistics/testing_stats_ set(TESTING_UTIL_SQL ${PROJECT_SOURCE_DIR}/test/sql/testing_sql_util.cpp) set(TESTING_UTIL_INDEX ${PROJECT_SOURCE_DIR}/test/index/testing_index_util.cpp) set(TESTING_UTIL_CODEGEN ${PROJECT_SOURCE_DIR}/test/codegen/testing_codegen_util.cpp) +set(TESTING_UTIL_INDEX_SELECTION ${PROJECT_SOURCE_DIR}/test/brain/testing_index_selection_util.cpp) add_library(peloton-test-common EXCLUDE_FROM_ALL ${gmock_srcs} ${HARNESS} ${TESTING_UTIL_EXECUTOR} @@ -58,6 +59,7 @@ add_library(peloton-test-common EXCLUDE_FROM_ALL ${gmock_srcs} ${HARNESS} ${TESTING_UTIL_INDEX} ${TESTING_UTIL_SQL} ${TESTING_UTIL_CODEGEN} + ${TESTING_UTIL_INDEX_SELECTION} ) # --[ Add "make check" target @@ -71,37 +73,37 @@ add_custom_target(check COMMAND ${CMAKE_CTEST_COMMAND} ${CTEST_FLAGS} --verbose) foreach(test_src ${test_srcs} ) #message("test_src = " ${test_src}) - + # get test file name - get_filename_component(test_bare_name ${test_src} NAME) + get_filename_component(test_bare_name ${test_src} NAME) string(REPLACE ".cpp" "" test_bare_name_without_extension ${test_bare_name}) string(REPLACE "\"" "" test_name ${test_bare_name_without_extension}) - + # create executable add_executable(${test_name} EXCLUDE_FROM_ALL ${test_src}) add_dependencies(check ${test_name}) - + #message("Correctness test: " ${test_name}) - + # link libraries - target_link_libraries(${test_name} peloton peloton-test-common) + target_link_libraries(${test_name} peloton peloton-test-common) - # set target properties + # set target properties set_target_properties(${test_name} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/test" COMMAND ${test_name} - ) - + ) + # add test add_test(${test_name} ${CMAKE_BINARY_DIR}/test/${test_name} --gtest_color=yes --gtest_output=xml:${CMAKE_BINARY_DIR}/test/${test_name}.xml) - + # leak suppression / whitelist set_property(TEST ${test_name} PROPERTY ENVIRONMENT "LSAN_OPTIONS=suppressions=${PROJECT_SOURCE_DIR}/test/leak_suppr.txt") - + endforeach(test_src ${test_srcs}) ################################################################################## @@ -112,32 +114,32 @@ endforeach(test_src ${test_srcs}) foreach(perf_src ${perf_srcs} ) list(REMOVE_ITEM test_srcs ${perf_src}) - + #message("test_srcs = " ${test_srcs}) #message("perf_src = " ${perf_src}) - - get_filename_component(perf_bare_name ${perf_src} NAME) + + get_filename_component(perf_bare_name ${perf_src} NAME) string(REPLACE ".cpp" "" perf_bare_name_without_extension ${perf_bare_name}) string(REPLACE "\"" "" perf_name ${perf_bare_name_without_extension}) - + # create executable add_executable(${perf_name} EXCLUDE_FROM_ALL ${perf_src}) add_dependencies(check ${perf_name}) - + #message("Performance test: " ${perf_name}) - + # link libraries - target_link_libraries(${perf_name} peloton peloton-test-common) + target_link_libraries(${perf_name} peloton peloton-test-common) - # set target properties + # set target properties set_target_properties(${perf_name} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/test" COMMAND ${perf_name} - ) - + ) + # add test add_test(${perf_name} ${CMAKE_BINARY_DIR}/test/${perf_name} --gtest_color=yes --gtest_output=xml:${CMAKE_BINARY_DIR}/test/${perf_name}.xml) - + endforeach(perf_src ${perf_srcs}) diff --git a/test/brain/compressed_idx_config_test.cpp b/test/brain/compressed_idx_config_test.cpp new file mode 100644 index 00000000000..608b604e01e --- /dev/null +++ b/test/brain/compressed_idx_config_test.cpp @@ -0,0 +1,368 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// compressed_idx_config_test.cpp +// +// Identification: test/brain/compressed_idx_config_test.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/testing_index_selection_util.h" +#include "brain/index_selection.h" +#include "brain/indextune/lspi/lspi_common.h" +#include "brain/indextune/compressed_index_config.h" +#include "brain/indextune/compressed_index_config_util.h" +#include "catalog/catalog.h" +#include "catalog/database_catalog.h" +#include "catalog/index_catalog.h" +#include "catalog/table_catalog.h" +#include "common/harness.h" +#include "concurrency/transaction_manager_factory.h" +#include "util/file_util.h" + +namespace peloton { +namespace test { + +//===--------------------------------------------------------------------===// +// RL Framework Tests +//===--------------------------------------------------------------------===// + +class CompressedIdxConfigTest : public PelotonTest { + public: + CompressedIdxConfigTest() + : catalog_{catalog::Catalog::GetInstance()}, + txn_manager_{&concurrency::TransactionManagerFactory::GetInstance()} {} + + /** + * @brief: Get the OID of a table by its name + */ + oid_t GetTableOid(const std::string &db_name, const std::string &table_name) { + auto txn = txn_manager_->BeginTransaction(); + const auto table_oid = catalog_->GetDatabaseObject(db_name, txn) + ->GetTableObject(table_name, DEFAULT_SCHEMA_NAME) + ->GetTableOid(); + txn_manager_->CommitTransaction(txn); + return table_oid; + } + + private: + catalog::Catalog *catalog_; + concurrency::TransactionManager *txn_manager_; +}; + +TEST_F(CompressedIdxConfigTest, CompressedRepresentationTest) { + /**This test checks for correctness of the compressed container + * representation*/ + std::string database_name = DEFAULT_DB_NAME; + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + // Initialization + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + auto config = testing_util.GetQueryStringsWorkload( + index_selection::QueryStringsWorkloadType::MultiTableNoop); + + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + std::string table_name_1 = table_schemas[0].table_name; + std::string table_name_2 = table_schemas[1].table_name; + std::string table_name_3 = table_schemas[2].table_name; + + + auto index_ab_t1 = testing_util.CreateHypotheticalIndex(table_name_1, {"a", "b"}); + auto index_bc_t1 = testing_util.CreateHypotheticalIndex(table_name_1, {"b", "c"}); + auto index_ac_t2 = testing_util.CreateHypotheticalIndex(table_name_2, {"a", "c"}); + + auto index_objs = {index_ab_t1, index_bc_t1, index_ac_t2}; + + for(auto index_obj: index_objs) { + testing_util.CreateIndex(index_obj); + } + + auto comp_idx_config = + brain::CompressedIndexConfigContainer(database_name, ignore_table_oids); + LOG_DEBUG("bitset: %s", comp_idx_config.ToString().c_str()); + EXPECT_EQ(comp_idx_config.GetConfigurationCount(), 48); + // 2 created + PK index being created by default + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name_1)), + 3); + // 1 created + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name_2)), + 1); + // No index created + EXPECT_TRUE( + comp_idx_config.EmptyConfig(GetTableOid(database_name, table_name_3))); + + // Now check that bitset positions exactly align with Indexes present + for (const auto &index_obj : index_objs) { + size_t global_offset = comp_idx_config.GetGlobalOffset(index_obj); + const auto new_idx_obj = comp_idx_config.GetIndex(global_offset); + EXPECT_TRUE(comp_idx_config.IsSet(index_obj)); + std::set idx_obj_cols(index_obj->column_oids.begin(), + index_obj->column_oids.end()); + std::set new_idx_obj_cols(new_idx_obj->column_oids.begin(), + new_idx_obj->column_oids.end()); + EXPECT_EQ(index_obj->db_oid, new_idx_obj->db_oid); + EXPECT_EQ(index_obj->table_oid, new_idx_obj->table_oid); + EXPECT_EQ(idx_obj_cols, new_idx_obj_cols); + } +} + +TEST_F(CompressedIdxConfigTest, ConfigEnumerationTest) { + +} + +TEST_F(CompressedIdxConfigTest, AddSimpleCandidatesTest) { + std::string database_name = DEFAULT_DB_NAME; + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + // Initialization + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + auto config = testing_util.GetQueryStringsWorkload( + index_selection::QueryStringsWorkloadType::SingleTableNoop); + + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + std::string table_name = table_schemas[0].table_name; + + auto index_ab = testing_util.CreateHypotheticalIndex(table_name, {"a", "b"}); + testing_util.CreateIndex(index_ab); + auto index_bc = testing_util.CreateHypotheticalIndex(table_name, {"b", "c"}); + testing_util.CreateIndex(index_bc); + + auto comp_idx_config = + brain::CompressedIndexConfigContainer(database_name, ignore_table_oids); + LOG_DEBUG("bitset: %s", comp_idx_config.ToString().c_str()); + // Total configuration = total number of permutations: 1 * 3! + 3 * 2! + 3 * + // 1! + 1 = 16 + EXPECT_EQ(comp_idx_config.GetConfigurationCount(), 16); + // 2 created + PK index being created by default + EXPECT_FALSE( + comp_idx_config.EmptyConfig(GetTableOid(database_name, table_name))); + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name)), + 3); + + std::string query_string = query_strings[0]; + boost::dynamic_bitset<> add_candidates_simple; + brain::CompressedIndexConfigUtil::AddCandidates( + comp_idx_config, query_string, add_candidates_simple, + brain::CandidateSelectionType::Simple); + + auto index_empty = testing_util.CreateHypotheticalIndex(table_name, {}); + auto index_b = testing_util.CreateHypotheticalIndex(table_name, {"b"}); + auto index_c = + testing_util.CreateHypotheticalIndex(table_name, {"c"}); + + std::vector> + add_expect_indexes_simple = {index_empty, index_b, index_c}; + + auto add_expect_bitset_simple = + brain::CompressedIndexConfigUtil::GenerateBitSet( + comp_idx_config, add_expect_indexes_simple); + + EXPECT_EQ(*add_expect_bitset_simple, add_candidates_simple); +} + +TEST_F(CompressedIdxConfigTest, AddAutoAdminCandidatesTest) { + std::string database_name = DEFAULT_DB_NAME; + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + // Initialization + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + auto config = testing_util.GetQueryStringsWorkload( + index_selection::QueryStringsWorkloadType::SingleTableNoop); + + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + std::string table_name = table_schemas[0].table_name; + + auto index_ab = testing_util.CreateHypotheticalIndex(table_name, {"a", "b"}); + testing_util.CreateIndex(index_ab); + auto index_bc = testing_util.CreateHypotheticalIndex(table_name, {"b", "c"}); + testing_util.CreateIndex(index_bc); + + auto comp_idx_config = + brain::CompressedIndexConfigContainer(database_name, ignore_table_oids); + LOG_DEBUG("bitset: %s", comp_idx_config.ToString().c_str()); + // Total configuration = total number of permutations: 1 * 3! + 3 * 2! + 3 * + // 1! + 1 = 16 + EXPECT_EQ(comp_idx_config.GetConfigurationCount(), 16); + // 2 created + PK index being created by default + EXPECT_FALSE( + comp_idx_config.EmptyConfig(GetTableOid(database_name, table_name))); + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name)), + 3); + size_t max_index_cols = 2; // multi-column index limit + size_t enumeration_threshold = 2; // naive enumeration + size_t num_indexes = 1; // essentially get all possible indexes + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + std::string query_string = query_strings[0]; + boost::dynamic_bitset<> add_candidates; + // TODO(saatviks): Indexes generated seem a bit weird - need to recheck whats happening here + // When turning up `num_indexes` to 10, this doesnt recommend 1, 2, (1, 2) and (2, 1)? + // Logs show correct set, but actual returned seem to be from 1 iteration less + brain::CompressedIndexConfigUtil::AddCandidates( + comp_idx_config, query_string, add_candidates, + brain::CandidateSelectionType::AutoAdmin, 0, knobs); + + auto index_empty = testing_util.CreateHypotheticalIndex(table_name, {}); + auto index_b = testing_util.CreateHypotheticalIndex(table_name, {"b"}); + + std::vector> + add_expect_indexes = {index_empty, index_b}; + + auto add_expect_bitset = + brain::CompressedIndexConfigUtil::GenerateBitSet( + comp_idx_config, add_expect_indexes); + + EXPECT_EQ(*add_expect_bitset, add_candidates); +} + +TEST_F(CompressedIdxConfigTest, AddExhaustiveCandidatesTest) { + std::string database_name = DEFAULT_DB_NAME; + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + // Initialization + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + auto config = testing_util.GetQueryStringsWorkload( + index_selection::QueryStringsWorkloadType::SingleTableNoop); + + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + std::string table_name = table_schemas[0].table_name; + + auto index_ab = testing_util.CreateHypotheticalIndex(table_name, {"a", "b"}); + testing_util.CreateIndex(index_ab); + auto index_bc = testing_util.CreateHypotheticalIndex(table_name, {"b", "c"}); + testing_util.CreateIndex(index_bc); + + auto comp_idx_config = + brain::CompressedIndexConfigContainer(database_name, ignore_table_oids); + LOG_DEBUG("bitset: %s", comp_idx_config.ToString().c_str()); + // Total configuration = total number of permutations: 1 * 3! + 3 * 2! + 3 * + // 1! + 1 = 16 + EXPECT_EQ(comp_idx_config.GetConfigurationCount(), 16); + // 2 created + PK index being created by default + EXPECT_FALSE( + comp_idx_config.EmptyConfig(GetTableOid(database_name, table_name))); + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name)), + 3); + + std::string query_string = query_strings[0]; + boost::dynamic_bitset<> add_candidates_exhaustive; + brain::CompressedIndexConfigUtil::AddCandidates( + comp_idx_config, query_string, add_candidates_exhaustive, + brain::CandidateSelectionType::Exhaustive, 2); + + auto index_empty = testing_util.CreateHypotheticalIndex(table_name, {}); + auto index_b = testing_util.CreateHypotheticalIndex(table_name, {"b"}); + auto index_c = testing_util.CreateHypotheticalIndex(table_name, {"c"}); + auto index_cb = testing_util.CreateHypotheticalIndex(table_name, {"c", "b"}); + + std::vector> + add_expect_indexes_exhaustive = {index_empty, index_b, index_c, index_bc, index_cb}; + + auto add_expect_bitset_exhaustive = + brain::CompressedIndexConfigUtil::GenerateBitSet( + comp_idx_config, add_expect_indexes_exhaustive); + + EXPECT_EQ(*add_expect_bitset_exhaustive, add_candidates_exhaustive); +} + +TEST_F(CompressedIdxConfigTest, DropCandidatesTest) { + std::string database_name = DEFAULT_DB_NAME; + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + // Initialization + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + auto config = testing_util.GetQueryStringsWorkload( + index_selection::QueryStringsWorkloadType::SingleTableNoop); + + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + std::string table_name = table_schemas[0].table_name; + + auto index_ab = testing_util.CreateHypotheticalIndex(table_name, {"a", "b"}); + testing_util.CreateIndex(index_ab); + auto index_bc = testing_util.CreateHypotheticalIndex(table_name, {"b", "c"}); + testing_util.CreateIndex(index_bc); + + auto comp_idx_config = + brain::CompressedIndexConfigContainer(database_name, ignore_table_oids); + LOG_DEBUG("bitset: %s", comp_idx_config.ToString().c_str()); + // Total configuration = total number of permutations: 1 * 3! + 3 * 2! + 3 * + // 1! + 1 = 16 + EXPECT_EQ(comp_idx_config.GetConfigurationCount(), 16); + // 2 created + PK index being created by default + EXPECT_FALSE( + comp_idx_config.EmptyConfig(GetTableOid(database_name, table_name))); + EXPECT_EQ( + comp_idx_config.GetNumIndexes(GetTableOid(database_name, table_name)), + 3); + + std::string query_string = query_strings[0]; + boost::dynamic_bitset<> drop_candidates; + brain::CompressedIndexConfigUtil::DropCandidates( + comp_idx_config, query_string, drop_candidates); + + // since b is primary key, we will ignore index {a, b} + std::vector> + drop_expect_indexes = {}; + + auto drop_expect_bitset = brain::CompressedIndexConfigUtil::GenerateBitSet( + comp_idx_config, drop_expect_indexes); + + EXPECT_EQ(*drop_expect_bitset, drop_candidates); + +} + +} // namespace test +} // namespace peloton diff --git a/test/brain/index_selection_test.cpp b/test/brain/index_selection_test.cpp new file mode 100644 index 00000000000..65e2efb9312 --- /dev/null +++ b/test/brain/index_selection_test.cpp @@ -0,0 +1,653 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// index_selection_test.cpp +// +// Identification: test/brain/index_selection_test.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include + +#include "binder/bind_node_visitor.h" +#include "brain/index_selection.h" +#include "brain/what_if_index.h" +#include "catalog/index_catalog.h" +#include "common/harness.h" +#include "concurrency/transaction_manager_factory.h" +#include "optimizer/stats/column_stats.h" +#include "optimizer/stats/stats_storage.h" +#include "optimizer/stats/table_stats.h" +#include "sql/testing_sql_util.h" + +#include "brain/testing_index_selection_util.h" + +namespace peloton { +namespace test { + +using namespace index_selection; + +//===--------------------------------------------------------------------===// +// IndexSelectionTest +//===--------------------------------------------------------------------===// + +class IndexSelectionTest : public PelotonTest {}; + +/** + * @brief Verify if admissible index count is correct for a given + * query workload. + */ +TEST_F(IndexSelectionTest, AdmissibleIndexesTest) { + // Parameters + std::string table_name = "table1"; + std::string database_name = DEFAULT_DB_NAME; + long num_tuples = 10; + + size_t max_index_cols = 2; + size_t enumeration_threshold = 2; + size_t num_indexes = 10; + + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + + TableSchema schema(table_name, {{"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + TestingIndexSelectionUtil testing_util(database_name); + testing_util.CreateTable(schema); + testing_util.InsertIntoTable(schema, num_tuples); + + // Form the query strings + std::vector query_strs; + std::vector admissible_indexes; + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a < 1 or b > 4 GROUP BY a"); + // 2 indexes will be choosen in GetAdmissibleIndexes - a, b + admissible_indexes.push_back(2); + query_strs.push_back("SELECT a, b, c FROM " + table_name + + " WHERE a < 1 or b > 4 ORDER BY a"); + admissible_indexes.push_back(2); + query_strs.push_back("DELETE FROM " + table_name + " WHERE a < 1 or b > 4"); + admissible_indexes.push_back(2); + query_strs.push_back("UPDATE " + table_name + + " SET a = 45 WHERE a < 1 or b > 4"); + + admissible_indexes.push_back(2); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + // Create a new workload + brain::Workload workload(query_strs, database_name, txn); + EXPECT_GT(workload.Size(), 0); + + // Verify the admissible indexes. + auto queries = workload.GetQueries(); + for (unsigned long i = 0; i < queries.size(); i++) { + brain::Workload w(queries[i], workload.GetDatabaseName()); + brain::IndexSelection is(w, knobs, txn); + + brain::IndexConfiguration ic; + is.GetAdmissibleIndexes(queries[i].first, ic); + LOG_TRACE("Admissible indexes %ld, %s", i, ic.ToString().c_str()); + auto indexes = ic.GetIndexes(); + EXPECT_EQ(ic.GetIndexCount(), admissible_indexes[i]); + } + txn_manager.CommitTransaction(txn); +} + +/** + * @brief Tests the first iteration of the candidate index generation + * algorithm i.e. generating single column candidate indexes per query. + */ +TEST_F(IndexSelectionTest, CandidateIndexGenerationTest) { + std::string database_name = DEFAULT_DB_NAME; + + // Config knobs + size_t max_index_cols = 1; + size_t enumeration_threshold = 2; + size_t num_indexes = 10; + int num_rows = 2000; + + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + + TestingIndexSelectionUtil testing_util(database_name); + auto config = + testing_util.GetQueryStringsWorkload(QueryStringsWorkloadType::SingleTableTwoColW1); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + } + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + brain::Workload workload(query_strings, database_name, txn); + EXPECT_EQ(workload.Size(), query_strings.size()); + + // Generate candidate configurations. + // The table doesn't have any tuples, so the admissible indexes won't help + // any of the queries --> candidate set should be 0. + brain::IndexConfiguration candidate_config; + brain::IndexConfiguration admissible_config; + + brain::IndexSelection index_selection(workload, knobs, txn); + index_selection.GenerateCandidateIndexes(candidate_config, admissible_config, + workload); + + LOG_TRACE("Admissible Index Count: %ld", admissible_config.GetIndexCount()); + LOG_TRACE("Admissible Indexes: %s", admissible_config.ToString().c_str()); + LOG_TRACE("Candidate Indexes: %s", candidate_config.ToString().c_str()); + + EXPECT_EQ(admissible_config.GetIndexCount(), 2); + // TODO: There is no data in the table. Indexes should not help. Should return + // 0. But currently, the cost with index for a query if 0.0 if there are no + // rows in the table where as the cost without the index is 1.0. This needs to + // be fixed in the cost model. Or is this behaviour of optimizer fine? + // EXPECT_EQ(candidate_config.GetIndexCount(), 0); + EXPECT_EQ(candidate_config.GetIndexCount(), 2); + + // Insert tuples into the tables. + for (auto table_schema : table_schemas) { + testing_util.InsertIntoTable(table_schema, num_rows); + } + + candidate_config.Clear(); + admissible_config.Clear(); + + brain::IndexSelection is(workload, knobs, txn); + is.GenerateCandidateIndexes(candidate_config, admissible_config, workload); + + LOG_TRACE("Admissible Index Count: %ld", admissible_config.GetIndexCount()); + LOG_TRACE("Admissible Indexes: %s", admissible_config.ToString().c_str()); + LOG_TRACE("Candidate Indexes: %s", candidate_config.ToString().c_str()); + EXPECT_EQ(admissible_config.GetIndexCount(), 2); + // Indexes help reduce the cost of the queries, so they get selected. + EXPECT_EQ(candidate_config.GetIndexCount(), 2); + + auto admissible_indexes = admissible_config.GetIndexes(); + auto candidate_indexes = candidate_config.GetIndexes(); + + // Columns - a and c + std::set expected_cols = {0, 2}; + + for (auto col : expected_cols) { + std::vector cols = {col}; + bool found = false; + for (auto index : admissible_indexes) { + found |= (index->column_oids == cols); + } + EXPECT_TRUE(found); + + found = false; + for (auto index : candidate_indexes) { + found |= (index->column_oids == cols); + } + EXPECT_TRUE(found); + } + + txn_manager.CommitTransaction(txn); +} + +/** + * @brief Tests multi column index generation from a set of candidate indexes. + */ +TEST_F(IndexSelectionTest, MultiColumnIndexGenerationTest) { + std::string database_name = DEFAULT_DB_NAME; + + brain::IndexConfiguration candidates; + brain::IndexConfiguration single_column_indexes; + brain::IndexConfiguration result; + brain::IndexConfiguration expected; + brain::Workload workload(database_name); + + size_t max_index_cols = 5; + size_t enumeration_threshold = 2; + size_t num_indexes = 10; + + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + brain::IndexSelection index_selection(workload, knobs, txn); + + std::vector cols; + + // Database: 1 + // Table: 1 + // Column: 1 + auto a11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, 1)); + // Column: 2 + auto b11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, 2)); + // Column: 3 + auto c11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, 3)); + // Column: 1, 2 + cols = {1, 2}; + auto ab11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, cols)); + // Column: 1, 3 + cols = {1, 3}; + auto ac11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, cols)); + // Column: 2, 3 + cols = {2, 3}; + auto bc11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, cols)); + // Column: 2, 1 + cols = {2, 1}; + auto ba11 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 1, cols)); + + // Database: 1 + // Table: 2 + // Column: 1 + auto a12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, 1)); + // Column: 2 + auto b12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, 2)); + // Column: 3 + auto c12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, 3)); + // Column: 2, 3 + cols = {2, 3}; + auto bc12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 1, 3 + cols = {1, 3}; + auto ac12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 3, 1 + cols = {3, 1}; + auto ca12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 3, 2 + cols = {3, 2}; + auto cb12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 1, 2, 3 + cols = {1, 2, 3}; + auto abc12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 2, 3, 1 + cols = {2, 3, 1}; + auto bca12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + // Column: 1, 3, 2 + cols = {1, 3, 2}; + auto acb12 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(1, 2, cols)); + + // Database: 2 + // Table: 1 + // Column: 1 + auto a21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, 1)); + // Column: 2 + auto b21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, 2)); + // Column: 3 + auto c21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, 3)); + // Column: 1, 2 + cols = {1, 2}; + auto ab21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, cols)); + // Column: 1, 3 + cols = {1, 3}; + auto ac21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, cols)); + // Column: 1, 2, 3 + cols = {1, 2, 3}; + auto abc21 = index_selection.AddConfigurationToPool( + brain::HypotheticalIndexObject(2, 1, cols)); + + std::set> indexes; + + indexes = {a11, b11, c11, a12, b12, c12, a21, b21, c21}; + single_column_indexes = {indexes}; + + indexes = {a11, b11, bc12, ac12, c12, a21, abc21}; + candidates = {indexes}; + + index_selection.GenerateMultiColumnIndexes(candidates, single_column_indexes, + result); + + // candidates union (candidates * single_column_indexes) + indexes = {// candidates + a11, b11, bc12, ac12, c12, a21, abc21, + // crossproduct + ab11, ac11, ba11, bc11, bca12, acb12, ca12, cb12, ab21, ac21}; + expected = {indexes}; + + auto chosen_indexes = result.GetIndexes(); + auto expected_indexes = expected.GetIndexes(); + + for (auto index : chosen_indexes) { + int count = 0; + for (auto expected_index : expected_indexes) { + auto index_object = *(index.get()); + auto expected_index_object = *(expected_index.get()); + if (index_object == expected_index_object) count++; + } + EXPECT_EQ(1, count); + } + EXPECT_EQ(expected_indexes.size(), chosen_indexes.size()); + + txn_manager.CommitTransaction(txn); +} + +/** + * @brief end-to-end test which takes in a workload of queries + * and spits out the set of indexes that are the best ones for the + * workload. + */ +TEST_F(IndexSelectionTest, IndexSelectionTest1) { + std::string database_name = DEFAULT_DB_NAME; + + int num_rows = 2000; // number of rows to be inserted. + + TestingIndexSelectionUtil testing_util(database_name); + auto config = + testing_util.GetQueryStringsWorkload(QueryStringsWorkloadType::SingleTableTwoColW2); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create and populate tables. + for (auto table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, num_rows); + } + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + brain::Workload workload(query_strings, database_name, txn); + EXPECT_EQ(workload.Size(), query_strings.size()); + + brain::IndexConfiguration best_config; + std::set> expected_indexes; + brain::IndexConfiguration expected_config; + + /** Test 1 + * Choose only 1 index with 1 column + * it should choose {B} + */ + size_t max_index_cols = 1; // multi-column index limit + size_t enumeration_threshold = 2; // naive enumeration threshold + size_t num_indexes = 1; // top num_indexes will be returned. + + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + + brain::IndexSelection is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(1, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"b"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 2 + * Choose 2 indexes with 1 column + * it should choose {A} and {B} + */ + max_index_cols = 1; + enumeration_threshold = 2; + num_indexes = 2; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(2, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"a"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"b"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 3 + * Choose 1 index with up to 2 columns + * it should choose {BA} + */ + max_index_cols = 2; + enumeration_threshold = 2; + num_indexes = 1; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(1, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"b", "a"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 4 + * Choose 2 indexes with up to 2 columns + * it should choose {AB} and {BC} + */ + max_index_cols = 2; + enumeration_threshold = 2; + num_indexes = 2; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(2, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"a", "b"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"b", "c"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 5 + * Choose 4 indexes with up to 2 columns + * it should choose {AB}, {BC} from exhaustive and {AC} or {CA} from greedy + * more indexes donot give any added benefit + */ + max_index_cols = 2; + enumeration_threshold = 2; + num_indexes = 4; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(3, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"a", "b"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"a", "c"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"b", "c"}, &is)}; + expected_config = {expected_indexes}; + + std::set> + alternate_expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"a", "b"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"c", "a"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"b", "c"}, &is)}; + brain::IndexConfiguration alternate_expected_config = { + alternate_expected_indexes}; + + // It can choose either AC or CA based on the distribution of C and A + EXPECT_TRUE((expected_config == best_config) || + (alternate_expected_config == best_config)); + + /** Test 6 + * Choose 1 index with up to 3 columns + * it should choose {BA} + * more indexes / columns donot give any added benefit + */ + max_index_cols = 3; + enumeration_threshold = 2; + num_indexes = 1; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(1, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"b", "a"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 7 + * Choose 2 indexes with up to 2 columns + * it should choose {BA} and {AC} + * This has a naive threshold of 1, it chooses BA from exhaustive + * enumeration and AC greedily + */ + max_index_cols = 2; + enumeration_threshold = 1; + num_indexes = 2; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(2, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy2", {"b", "a"}, &is), + testing_util.CreateHypotheticalIndex("dummy2", {"a", "c"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + txn_manager.CommitTransaction(txn); +} + +/** + * @brief end-to-end test which takes in a workload of queries + * and spits out the set of indexes that are the best ones for more + * complex workloads. + */ +TEST_F(IndexSelectionTest, IndexSelectionTest2) { + std::string database_name = DEFAULT_DB_NAME; + int num_rows = 2000; // number of rows to be inserted. + + TestingIndexSelectionUtil testing_util(database_name); + auto config = + testing_util.GetQueryStringsWorkload(QueryStringsWorkloadType::SingleTableThreeColW); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create and populate tables. + for (auto table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, num_rows); + } + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + brain::Workload workload(query_strings, database_name, txn); + EXPECT_EQ(workload.Size(), query_strings.size()); + + brain::IndexConfiguration best_config; + std::set> expected_indexes; + brain::IndexConfiguration expected_config; + + /** Test 1 + * Choose only 1 index with up to 3 column + * it should choose {BCA} + */ + size_t max_index_cols = 3; + size_t enumeration_threshold = 2; + size_t num_indexes = 1; + brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, + num_indexes}; + brain::IndexSelection is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(1, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy3", {"b", "c", "a"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + /** Test 2 + * Choose only 2 indexes with up to 3 column + * it should choose some permutation of {BCA} and {DEF} + */ + max_index_cols = 3; + enumeration_threshold = 2; + num_indexes = 2; + knobs = {max_index_cols, enumeration_threshold, num_indexes}; + is = {workload, knobs, txn}; + + is.GetBestIndexes(best_config); + + LOG_TRACE("Best Indexes: %s", best_config.ToString().c_str()); + LOG_TRACE("Best Index Count: %ld", best_config.GetIndexCount()); + + EXPECT_EQ(2, best_config.GetIndexCount()); + + expected_indexes = { + testing_util.CreateHypotheticalIndex("dummy3", {"b", "c", "a"}, &is), + testing_util.CreateHypotheticalIndex("dummy3", {"d", "e", "f"}, &is)}; + expected_config = {expected_indexes}; + + EXPECT_TRUE(expected_config == best_config); + + txn_manager.CommitTransaction(txn); +} + +} // namespace test +} // namespace peloton diff --git a/test/brain/lspi_test.cpp b/test/brain/lspi_test.cpp new file mode 100644 index 00000000000..8db0f1a09bd --- /dev/null +++ b/test/brain/lspi_test.cpp @@ -0,0 +1,804 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// lspi_test.cpp +// +// Identification: test/brain/lspi_test.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/indextune/lspi/lspi_common.h" +#include "brain/indextune/lspi/lspi_tuner.h" +#include "brain/indextune/lspi/lstdq.h" +#include "brain/indextune/lspi/rlse.h" +#include "brain/util/eigen_util.h" +#include "common/harness.h" +#include "brain/testing_index_selection_util.h" +#include "common/timer.h" + +namespace peloton { +namespace test { + +//===--------------------------------------------------------------------===// +// Tensorflow Tests +//===--------------------------------------------------------------------===// + +class LSPITests : public PelotonTest {}; + +/** + * @brief: Attempt to fit y = m*x with Recursive Least Squares + */ +TEST_F(LSPITests, RLSETest) { + // + int NUM_SAMPLES = 500; + int LOG_INTERVAL = 100; + int m = 3; + vector_eig data_in = vector_eig::LinSpaced(NUM_SAMPLES, 0, NUM_SAMPLES - 1); + vector_eig data_out = data_in.array() * m; + vector_eig loss_vector = vector_eig::Zero(LOG_INTERVAL); + float prev_loss = std::numeric_limits::max(); + auto model = brain::RLSEModel(1); + for (int i = 0; i < NUM_SAMPLES; i++) { + vector_eig feat_vec = data_in.segment(i, 1); + double value_true = data_out(i); + double value_pred = model.Predict(feat_vec); + double loss = fabs(value_pred - value_true); + loss_vector(i % LOG_INTERVAL) = loss; + model.Update(feat_vec, value_true); + if ((i + 1) % LOG_INTERVAL == 0) { + float curr_loss = loss_vector.array().mean(); + LOG_DEBUG("Loss at %d: %.5f", i + 1, curr_loss); + EXPECT_LE(curr_loss, prev_loss); + prev_loss = curr_loss; + } + } +} + +/** + * @brief: The suite of simple tuning tests uses the + * `testing_index_selection_util` to build a cyclic query workload + * and observe improvement in cost metrics over time. + * As a sanity check, at every CATALOG_SYNC_INTERVAL, + * we perform a hard check that the average cost metric + * in this interval has been lower than previous upto a threshold. + * + * We also perform a run of the workload with and without the tuning enabled + * and perform a hard check that the overall cost should be lower with tuning. + * + * In addition these microworkloads serve as a useful way to analyze the + * behavior + * of the tuner. + * TODO(saatviks): Add analysis and observations here? + */ + +// TEST_F(LSPITests, BenchmarkTest) { +// // ** Initialization ** // +// std::string database_name = DEFAULT_DB_NAME; +// size_t MAX_INDEX_SIZE = 3; +// int CATALOG_SYNC_INTERVAL = 2; +// // This threshold depends on #rows in the tables +// double MIN_COST_THRESH = 1000.0; +// size_t MAX_NUMINDEXES_WHATIF = 100; +// bool DRY_RUN_MODE = true; +// int TBL_ROWS = 1000; +// auto timer = Timer>(); +// std::vector batch_costs; +// std::vector batch_queries; +// +// index_selection::TestingIndexSelectionUtil testing_util(database_name); +// +// std::set ignore_table_oids; +// brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, +// ignore_table_oids); +// +// auto config = testing_util.GetCyclicWorkload( +// {index_selection::QueryStringsWorkloadType::SingleTableTwoColW2}, 2); +// auto table_schemas = config.first; +// auto query_strings = config.second; +// +// // Create all the required tables for this workloads. +// for (auto &table_schema : table_schemas) { +// testing_util.CreateTable(table_schema); +// testing_util.InsertIntoTable(table_schema, TBL_ROWS); +// } +// +// // ** No Tuning ** // +// brain::LSPIIndexTuner index_tuner(database_name, ignore_table_oids, false, +// MAX_INDEX_SIZE, DRY_RUN_MODE); +// vector_eig query_costs_notuning = vector_eig::Zero(query_strings.size()); +// vector_eig search_time_notuning = vector_eig::Zero(query_strings.size()); +// +// LOG_DEBUG("Run without Tuning:"); +// for (size_t i = 1; i <= query_strings.size(); i++) { +// auto query = query_strings[i - 1]; +// +// auto index_config = +// brain::CompressedIndexConfigUtil::ToIndexConfiguration( +// *index_tuner.GetConfigContainer()); +// +// // Measure the What-If Index cost +// auto cost = +// testing_util.WhatIfIndexCost(query, index_config, database_name); +// +// // No tuning performed here +// query_costs_notuning[i - 1] = cost; +// } +// +// // ** Exhaustive What-If Tuning Setup(Closest to Ideal) ** // +// +// size_t max_index_cols = MAX_INDEX_SIZE; // multi-column index limit +// size_t enumeration_threshold = MAX_INDEX_SIZE; // naive enumeration +// threshold +// size_t num_indexes = +// MAX_NUMINDEXES_WHATIF; // top num_indexes will be returned. +// +// brain::IndexSelectionKnobs knobs = {max_index_cols, enumeration_threshold, +// num_indexes}; +// brain::IndexConfiguration best_config; +// vector_eig query_costs_exhaustivewhatif = +// vector_eig::Zero(query_strings.size()); +// vector_eig search_time_exhaustivewhatif = +// vector_eig::Zero(query_strings.size()); +// auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); +// +// // Cleanup +// batch_queries.clear(); +// +// brain::Workload w(database_name); +// auto txn = txn_manager.BeginTransaction(); +// brain::IndexSelection is = {w, knobs, txn}; +// is.GetBestIndexes(best_config); +// txn_manager.CommitTransaction(txn); +// LOG_DEBUG("Index: %s", best_config.ToString().c_str()); +// LOG_DEBUG("Run with Exhaustive What-If Search:"); +// for (size_t i = 1; i <= query_strings.size(); i++) { +// auto query = query_strings[i - 1]; +// +// // Measure the What-If Index cost +// +// batch_queries.push_back(query); +// double cost = +// testing_util.WhatIfIndexCost(query, best_config, database_name); +// query_costs_exhaustivewhatif[i - 1] = cost; +// +// // Perform tuning +// if (i % CATALOG_SYNC_INTERVAL == 0) { +// LOG_DEBUG("Exhaustive What-If Tuning..."); +// timer.Reset(); +// timer.Start(); +// txn = txn_manager.BeginTransaction(); +// brain::Workload workload(batch_queries, database_name, txn); +// is = {workload, knobs, txn}; +// is.GetBestIndexes(best_config); +// txn_manager.CommitTransaction(txn); +// timer.Stop(); +// search_time_exhaustivewhatif[i-1] = timer.GetDuration(); +// batch_queries.clear(); +// batch_costs.clear(); +// } +// } +// batch_costs.clear(); +// batch_queries.clear(); +// +// // ** Exhaustive What-If Tuning Setup without dropping indexes (Closest to +// // Ideal) ** // +// +// brain::IndexConfiguration best_config_nodropping; +// brain::IndexConfiguration prev_config_nodropping; +// vector_eig query_costs_exhaustivewhatif_nodropping = +// vector_eig::Zero(query_strings.size()); +// vector_eig search_time_exhaustivewhatif_nodropping = +// vector_eig::Zero(query_strings.size()); +// +// brain::Workload w_nodropping(database_name); +// txn = txn_manager.BeginTransaction(); +// brain::IndexSelection is_nodropping = {w_nodropping, knobs, txn}; +// is_nodropping.GetBestIndexes(best_config_nodropping); +// txn_manager.CommitTransaction(txn); +// +// LOG_DEBUG("Index: %s", best_config_nodropping.ToString().c_str()); +// LOG_DEBUG("Run with Exhaustive What-If Search without dropping indexes:"); +// for (size_t i = 1; i <= query_strings.size(); i++) { +// auto query = query_strings[i - 1]; +// +// // Measure the What-If Index cost +// +// batch_queries.push_back(query); +// double cost = testing_util.WhatIfIndexCost(query, best_config_nodropping, +// database_name); +// query_costs_exhaustivewhatif_nodropping[i - 1] = cost; +// +// // Perform tuning +// if (i % CATALOG_SYNC_INTERVAL == 0) { +// LOG_DEBUG("Exhaustive What-If Tuning..."); +// txn = txn_manager.BeginTransaction(); +// prev_config_nodropping.Set(best_config_nodropping); +// timer.Reset(); +// timer.Start(); +// brain::Workload workload(batch_queries, database_name, txn); +// is_nodropping = {workload, knobs, txn}; +// is_nodropping.GetBestIndexes(best_config_nodropping); +// timer.Stop(); +// best_config_nodropping.Merge(prev_config_nodropping); +// txn_manager.CommitTransaction(txn); +// +// search_time_exhaustivewhatif_nodropping[i - 1] = timer.GetDuration(); +// batch_queries.clear(); +// batch_costs.clear(); +// } +// } +// batch_costs.clear(); +// batch_queries.clear(); +// +// // ** LSPI Tuning Setup(Exhaustive: with max add candidate search) ** // +// brain::LSPIIndexTuner index_tuner_exhaustive( +// database_name, ignore_table_oids, false, MAX_INDEX_SIZE, DRY_RUN_MODE); +// double prev_cost = DBL_MAX; +// vector_eig cost_vector_lspiexhaustive = +// vector_eig::Zero(CATALOG_SYNC_INTERVAL); +// vector_eig query_costs_lspiexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig search_time_lspiexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig numconfigadds_lspiexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig numconfigdrops_lspiexhaustive = +// vector_eig::Zero(query_strings.size()); +// +// +// LOG_DEBUG("Run with LSPI(Exhaustive) Tuning:"); +// for (size_t i = 1; i <= query_strings.size(); i++) { +// auto query = query_strings[i - 1]; +// +// auto index_config = +// brain::CompressedIndexConfigUtil::ToIndexConfiguration( +// *index_tuner_exhaustive.GetConfigContainer()); +// +// // Measure the What-If Index cost +// auto cost = +// testing_util.WhatIfIndexCost(query, index_config, database_name); +// +// batch_queries.push_back(query); +// batch_costs.push_back(cost); +// query_costs_lspiexhaustive[i - 1] = cost; +// cost_vector_lspiexhaustive[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; +// +// // Perform tuning +// if (i % CATALOG_SYNC_INTERVAL == 0) { +// const boost::dynamic_bitset<> prev_config( +// *index_tuner_exhaustive.GetConfigContainer() +// ->GetCurrentIndexConfig()); +// LOG_DEBUG("COREIL Tuning..."); +// timer.Reset(); +// timer.Start(); +// index_tuner_exhaustive.Tune(batch_queries, batch_costs); +// timer.Stop(); +// search_time_lspiexhaustive[i - 1] = timer.GetDuration(); +// const boost::dynamic_bitset<> cur_config( +// *index_tuner_exhaustive.GetConfigContainer() +// ->GetCurrentIndexConfig()); +// const auto drop_bitset = prev_config - cur_config; +// const auto add_bitset = cur_config - prev_config; +// numconfigadds_lspiexhaustive[i - 1] = add_bitset.count(); +// numconfigdrops_lspiexhaustive[i - 1] = drop_bitset.count(); +// LOG_DEBUG("#Dropped Indexes: %lu, #Added Indexes: %lu", +// drop_bitset.count(), add_bitset.count()); +// +// batch_queries.clear(); +// batch_costs.clear(); +// double mean_cost = cost_vector_lspiexhaustive.array().mean(); +// LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); +// EXPECT_LE(mean_cost, prev_cost); +// prev_cost = std::max(mean_cost, MIN_COST_THRESH); +// } +// } +// batch_costs.clear(); +// batch_queries.clear(); +// +// // ** LSPI Tuning Setup(Non-Exhaustive: with only single-column indexes) ** +// // +// brain::LSPIIndexTuner index_tuner_nonexhaustive( +// database_name, ignore_table_oids, true, MAX_INDEX_SIZE, DRY_RUN_MODE); +// prev_cost = DBL_MAX; +// vector_eig cost_vector_lspinonexhaustive = +// vector_eig::Zero(CATALOG_SYNC_INTERVAL); +// vector_eig query_costs_lspinonexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig search_time_lspinonexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig numconfigadds_lspinonexhaustive = +// vector_eig::Zero(query_strings.size()); +// vector_eig numconfigdrops_lspinonexhaustive = +// vector_eig::Zero(query_strings.size()); +// +// LOG_DEBUG("Run with LSPI(Non-Exhaustive) Tuning:"); +// for (size_t i = 1; i <= query_strings.size(); i++) { +// auto query = query_strings[i - 1]; +// +// auto index_config = +// brain::CompressedIndexConfigUtil::ToIndexConfiguration( +// *index_tuner_nonexhaustive.GetConfigContainer()); +// +// // Measure the What-If Index cost +// auto cost = +// testing_util.WhatIfIndexCost(query, index_config, database_name); +// +// batch_queries.push_back(query); +// batch_costs.push_back(cost); +// query_costs_lspinonexhaustive[i - 1] = cost; +// cost_vector_lspinonexhaustive[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; +// +// // Perform tuning +// if (i % CATALOG_SYNC_INTERVAL == 0) { +// const boost::dynamic_bitset<> prev_config( +// *index_tuner_nonexhaustive.GetConfigContainer() +// ->GetCurrentIndexConfig()); +// LOG_DEBUG("LSPI Tuning(Non-Exhaustive)..."); +// timer.Reset(); +// timer.Start(); +// index_tuner_nonexhaustive.Tune(batch_queries, batch_costs); +// timer.Stop(); +// search_time_lspinonexhaustive[i - 1] = timer.GetDuration(); +// const boost::dynamic_bitset<> cur_config( +// *index_tuner_nonexhaustive.GetConfigContainer() +// ->GetCurrentIndexConfig()); +// const auto drop_bitset = prev_config - cur_config; +// const auto add_bitset = cur_config - prev_config; +// numconfigadds_lspinonexhaustive[i-1] = add_bitset.count(); +// numconfigdrops_lspinonexhaustive[i-1] = drop_bitset.count(); +// LOG_DEBUG("#Dropped Indexes: %lu, #Added Indexes: %lu", +// drop_bitset.count(), add_bitset.count()); +// batch_queries.clear(); +// batch_costs.clear(); +// double mean_cost = cost_vector_lspinonexhaustive.array().mean(); +// LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); +// EXPECT_LE(mean_cost, prev_cost); +// prev_cost = std::max(mean_cost, MIN_COST_THRESH); +// } +// } +// +// // For analysis +// // TODO: This is tooooooooooooo overloaded!! +// LOG_DEBUG("Overall Cost Trend for SingleTableTwoColW1 Workload:"); +// for (size_t i = 0; i < query_strings.size(); i++) { +// LOG_DEBUG( +// "%zu\t" +// "No Tuning Cost: %f\tLSPI(Exhaustive) Tuning Cost: " +// "%f\tWhatIf(Exhaustive) Tuning Cost: %f\tLSPI(Non-Exhaustive) Tuning " +// "Cost: %f\tWhatIf(Exhaustive No-Dropping) Tuning Cost: %f\t" +// "No Tuning Time: %f\tLSPI(Exhaustive) Tuning Time: " +// "%f\tWhatIf(Exhaustive) Tuning Time: %f\tLSPI(Non-Exhaustive) Tuning " +// "Time: %f\tWhatIf(Exhaustive No-Dropping) Tuning Time: %f\t" +// "LSPI(Exhaustive) Adds: %f\tLSPI(Exhaustive) Drops: %f\t" +// "LSPI(Non-Exhaustive) Adds: %f\tLSPI(Non-Exhaustive) Drops: %f\t" +// "%s", +// i, query_costs_notuning[i], query_costs_lspiexhaustive[i], +// query_costs_exhaustivewhatif[i], query_costs_lspinonexhaustive[i], +// query_costs_exhaustivewhatif_nodropping[i], search_time_notuning[i], +// search_time_lspiexhaustive[i], search_time_exhaustivewhatif[i], +// search_time_lspinonexhaustive[i], +// search_time_exhaustivewhatif_nodropping[i], +// numconfigadds_lspiexhaustive[i], numconfigdrops_lspiexhaustive[i], +// numconfigadds_lspinonexhaustive[i], +// numconfigdrops_lspinonexhaustive[i], +// query_strings[i].c_str()); +// } +// float tuning_overall_cost_lspiexhaustive = +// query_costs_lspiexhaustive.array().sum(); +// float tuning_overall_cost_lspinonexhaustive = +// query_costs_lspinonexhaustive.array().sum(); +// float notuning_overall_cost = query_costs_notuning.array().sum(); +// float tuning_overall_cost_exhaustivewhatif = +// query_costs_exhaustivewhatif.array().sum(); +// float tuning_overall_cost_exhaustivewhatif_nodropping = +// query_costs_exhaustivewhatif_nodropping.array().sum(); +// LOG_DEBUG( +// "No Tuning Cost Total: %f, LSPI(Exhaustive) Tuning Cost Total: %f, " +// "WhatIf(Exhaustive) Tuning Cost: %f, LSPI(Non-Exhaustive) Tuning Cost +// Total: %f," +// "WhatIf(Exhaustive No-Dropping) Tuning Cost: %f", +// notuning_overall_cost, tuning_overall_cost_lspiexhaustive, +// tuning_overall_cost_exhaustivewhatif, +// tuning_overall_cost_lspinonexhaustive, +// tuning_overall_cost_exhaustivewhatif_nodropping); +// EXPECT_LT(tuning_overall_cost_lspiexhaustive, notuning_overall_cost); +// EXPECT_LT(tuning_overall_cost_lspinonexhaustive, notuning_overall_cost); +//} + +/** +* @brief: The suite of simple tuning tests uses the +* `testing_index_selection_util` to build a cyclic query workload +* and observe improvement in cost metrics over time. +* As a sanity check, at every CATALOG_SYNC_INTERVAL, +* we perform a hard check that the average cost metric +* in this interval has been lower than previous upto a threshold. +* +* We also perform a run of the workload with and without the tuning enabled +* and perform a hard check that the overall cost should be lower with tuning. +* +* In addition these microworkloads serve as a useful way to analyze the behavior +* of the tuner. +* TODO(saatviks): Add analysis and observations here? +*/ + +TEST_F(LSPITests, TuneTestTwoColTable1) { + std::string database_name = DEFAULT_DB_NAME; + size_t MAX_INDEX_SIZE = 3; + int CATALOG_SYNC_INTERVAL = 2; + // This threshold depends on #rows in the tables + double MIN_COST_THRESH = 0.04; + int TBL_ROWS = 100; + std::set> add_set, drop_set; + + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + + auto config = testing_util.GetCyclicWorkload( + {index_selection::QueryStringsWorkloadType::SingleTableTwoColW1}, 2); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, TBL_ROWS); + } + + brain::LSPIIndexTuner index_tuner(database_name, ignore_table_oids, brain::CandidateSelectionType::Exhaustive, + MAX_INDEX_SIZE); + vector_eig query_costs_no_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run without Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + // No tuning performed here + query_costs_no_tuning[i - 1] = cost; + } + + std::vector batch_costs; + std::vector batch_queries; + double prev_cost = DBL_MAX; + vector_eig cost_vector = vector_eig::Zero(CATALOG_SYNC_INTERVAL); + vector_eig query_costs_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run with Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + batch_queries.push_back(query); + batch_costs.push_back(cost); + query_costs_tuning[i - 1] = cost; + cost_vector[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; + + // Perform tuning + if (i % CATALOG_SYNC_INTERVAL == 0) { + LOG_DEBUG("Tuning..."); + index_tuner.Tune(batch_queries, batch_costs, add_set, drop_set); + batch_queries.clear(); + batch_costs.clear(); + double mean_cost = cost_vector.array().mean(); + LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); + EXPECT_LE(mean_cost, prev_cost); + prev_cost = std::max(mean_cost, MIN_COST_THRESH); + } + } + + // For analysis + LOG_DEBUG("Overall Cost Trend for SingleTableTwoColW1 Workload:"); + for (size_t i = 0; i < query_strings.size(); i++) { + LOG_DEBUG("%zu\tWithout Tuning: %f\tWith Tuning: %f\t%s", i, + query_costs_no_tuning[i], query_costs_tuning[i], + query_strings[i].c_str()); + } + float tuning_overall_cost = query_costs_tuning.array().sum(); + float notuning_overall_cost = query_costs_no_tuning.array().sum(); + LOG_DEBUG("With Tuning: %f, Without Tuning: %f", tuning_overall_cost, + notuning_overall_cost); + EXPECT_LT(tuning_overall_cost, notuning_overall_cost); +} + +TEST_F(LSPITests, TuneTestTwoColTable2) { + std::string database_name = DEFAULT_DB_NAME; + size_t MAX_INDEX_SIZE = 3; + int CATALOG_SYNC_INTERVAL = 2; + // This threshold depends on #rows in the tables + double MIN_COST_THRESH = 0.05; + int TBL_ROWS = 100; + std::set> add_set, drop_set; + + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + + auto config = testing_util.GetCyclicWorkload( + {index_selection::QueryStringsWorkloadType::SingleTableTwoColW2}, 2); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, TBL_ROWS); + } + + brain::LSPIIndexTuner index_tuner(database_name, ignore_table_oids, brain::CandidateSelectionType::Exhaustive, + MAX_INDEX_SIZE); + vector_eig query_costs_no_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run without Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + // No tuning performed here + query_costs_no_tuning[i - 1] = cost; + } + + std::vector batch_costs; + std::vector batch_queries; + double prev_cost = DBL_MAX; + vector_eig cost_vector = vector_eig::Zero(CATALOG_SYNC_INTERVAL); + vector_eig query_costs_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run with Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + batch_queries.push_back(query); + batch_costs.push_back(cost); + query_costs_tuning[i - 1] = cost; + cost_vector[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; + + // Perform tuning + if (i % CATALOG_SYNC_INTERVAL == 0) { + LOG_DEBUG("Tuning..."); + index_tuner.Tune(batch_queries, batch_costs, add_set, drop_set); + batch_queries.clear(); + batch_costs.clear(); + double mean_cost = cost_vector.array().mean(); + LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); + EXPECT_LE(mean_cost, prev_cost); + prev_cost = std::max(mean_cost, MIN_COST_THRESH); + } + } + + // For analysis + LOG_DEBUG("Overall Cost Trend for SingleTableTwoColW2 Workload:"); + for (size_t i = 0; i < query_strings.size(); i++) { + LOG_DEBUG("%zu\tWithout Tuning: %f\tWith Tuning: %f\t%s", i, + query_costs_no_tuning[i], query_costs_tuning[i], + query_strings[i].c_str()); + } + float tuning_overall_cost = query_costs_tuning.array().sum(); + float notuning_overall_cost = query_costs_no_tuning.array().sum(); + LOG_DEBUG("With Tuning: %f, Without Tuning: %f", tuning_overall_cost, + notuning_overall_cost); + EXPECT_LT(tuning_overall_cost, notuning_overall_cost); +} + +TEST_F(LSPITests, TuneTestThreeColTable) { + std::string database_name = DEFAULT_DB_NAME; + size_t MAX_INDEX_SIZE = 3; + int CATALOG_SYNC_INTERVAL = 2; + // This threshold depends on #rows in the tables + double MIN_COST_THRESH = 0.05; + int TBL_ROWS = 100; + std::set> add_set, drop_set; + + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + + auto config = testing_util.GetCyclicWorkload( + {index_selection::QueryStringsWorkloadType::SingleTableThreeColW}, 2); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, TBL_ROWS); + } + + brain::LSPIIndexTuner index_tuner(database_name, ignore_table_oids, brain::CandidateSelectionType::Exhaustive, + MAX_INDEX_SIZE); + vector_eig query_costs_no_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run without Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + // No tuning performed here + query_costs_no_tuning[i - 1] = cost; + } + + std::vector batch_costs; + std::vector batch_queries; + double prev_cost = DBL_MAX; + vector_eig cost_vector = vector_eig::Zero(CATALOG_SYNC_INTERVAL); + vector_eig query_costs_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run with Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + batch_queries.push_back(query); + batch_costs.push_back(cost); + query_costs_tuning[i - 1] = cost; + cost_vector[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; + + // Perform tuning + if (i % CATALOG_SYNC_INTERVAL == 0) { + LOG_DEBUG("Tuning..."); + index_tuner.Tune(batch_queries, batch_costs, add_set, drop_set); + batch_queries.clear(); + batch_costs.clear(); + double mean_cost = cost_vector.array().mean(); + LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); + EXPECT_LE(mean_cost, prev_cost); + prev_cost = std::max(mean_cost, MIN_COST_THRESH); + } + } + + // For analysis + LOG_DEBUG("Overall Cost Trend for SingleTableThreeColW Workload:"); + for (size_t i = 0; i < query_strings.size(); i++) { + LOG_DEBUG("%zu\tWithout Tuning: %f\tWith Tuning: %f\t%s", i, + query_costs_no_tuning[i], query_costs_tuning[i], + query_strings[i].c_str()); + } + float tuning_overall_cost = query_costs_tuning.array().sum(); + float notuning_overall_cost = query_costs_no_tuning.array().sum(); + LOG_DEBUG("With Tuning: %f, Without Tuning: %f", tuning_overall_cost, + notuning_overall_cost); + EXPECT_LT(tuning_overall_cost, notuning_overall_cost); +} + +TEST_F(LSPITests, TuneTestMultiColMultiTable) { + std::string database_name = DEFAULT_DB_NAME; + size_t MAX_INDEX_SIZE = 3; + int CATALOG_SYNC_INTERVAL = 2; + // This threshold depends on #rows in the tables + double MIN_COST_THRESH = 100.0; + int TBL_ROWS = 100; + std::set> add_set, drop_set; + + index_selection::TestingIndexSelectionUtil testing_util(database_name); + + std::set ignore_table_oids; + brain::CompressedIndexConfigUtil::GetIgnoreTables(database_name, + ignore_table_oids); + + auto config = testing_util.GetCyclicWorkload( + {index_selection::QueryStringsWorkloadType::MultiTableMultiColW}, 2); + auto table_schemas = config.first; + auto query_strings = config.second; + + // Create all the required tables for this workloads. + for (auto &table_schema : table_schemas) { + testing_util.CreateTable(table_schema); + testing_util.InsertIntoTable(table_schema, TBL_ROWS); + } + + brain::LSPIIndexTuner index_tuner(database_name, ignore_table_oids, brain::CandidateSelectionType::Exhaustive, + MAX_INDEX_SIZE); + vector_eig query_costs_no_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run without Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + // No tuning performed here + query_costs_no_tuning[i - 1] = cost; + } + + std::vector batch_costs; + std::vector batch_queries; + double prev_cost = DBL_MAX; + vector_eig cost_vector = vector_eig::Zero(CATALOG_SYNC_INTERVAL); + vector_eig query_costs_tuning = vector_eig::Zero(query_strings.size()); + + LOG_DEBUG("Run with Tuning:"); + for (size_t i = 1; i <= query_strings.size(); i++) { + auto query = query_strings[i - 1]; + + auto index_config = brain::CompressedIndexConfigUtil::ToIndexConfiguration( + *index_tuner.GetConfigContainer()); + + // Measure the What-If Index cost + auto cost = + testing_util.WhatIfIndexCost(query, index_config, database_name); + + batch_queries.push_back(query); + batch_costs.push_back(cost); + query_costs_tuning[i - 1] = cost; + cost_vector[(i - 1) % CATALOG_SYNC_INTERVAL] = cost; + + // Perform tuning + if (i % CATALOG_SYNC_INTERVAL == 0) { + LOG_DEBUG("Tuning..."); + index_tuner.Tune(batch_queries, batch_costs, add_set, drop_set); + batch_queries.clear(); + batch_costs.clear(); + double mean_cost = cost_vector.array().mean(); + LOG_DEBUG("Iter: %zu, Avg Cost: %f", i, mean_cost); + EXPECT_LE(mean_cost, prev_cost); + prev_cost = std::max(mean_cost, MIN_COST_THRESH); + } + } + + // For analysis + LOG_DEBUG("Overall Cost Trend for MultiTableMultiColW Workload:"); + for (size_t i = 0; i < query_strings.size(); i++) { + LOG_DEBUG("%zu\tWithout Tuning: %f\tWith Tuning: %f\t%s", i, + query_costs_no_tuning[i], query_costs_tuning[i], + query_strings[i].c_str()); + } + float tuning_overall_cost = query_costs_tuning.array().sum(); + float notuning_overall_cost = query_costs_no_tuning.array().sum(); + LOG_DEBUG("With Tuning: %f, Without Tuning: %f", tuning_overall_cost, + notuning_overall_cost); + EXPECT_LT(tuning_overall_cost, notuning_overall_cost); +} + +} // namespace test +} // namespace peloton diff --git a/test/brain/tensorflow_test.cpp b/test/brain/tensorflow_test.cpp index c0c659f5bb9..a00855067fb 100644 --- a/test/brain/tensorflow_test.cpp +++ b/test/brain/tensorflow_test.cpp @@ -34,10 +34,39 @@ TEST_F(TensorflowTests, BasicTFTest) { } TEST_F(TensorflowTests, BasicEigenTest) { - Eigen::MatrixXd m = Eigen::MatrixXd::Random(2, 2); + /** + * Notes on Eigen: + * 1. Don't use 'auto'!! + */ + // Eigen Matrix + matrix_eig m = matrix_eig::Random(2, 2); EXPECT_EQ(m.rows(), 2); EXPECT_EQ(m.cols(), 2); EXPECT_TRUE(m.IsRowMajor); + // Eigen Vector + vector_eig v = vector_eig::Random(2); + EXPECT_EQ(v.rows(), 2); + EXPECT_EQ(v.cols(), 1); + // Transpose(if you try to store as `vec_eig` it will be 2x1) + matrix_eig vT = v.transpose(); + EXPECT_EQ(vT.rows(), 1); + EXPECT_EQ(vT.cols(), 2); + // Matrix multiplication(1) + vector_eig vTv = vT * v; + EXPECT_EQ(vTv.rows(), 1); + EXPECT_EQ(vTv.cols(), 1); + // Matrix multiplication(2) + matrix_eig vvT = v * vT; + EXPECT_EQ(vvT.rows(), 2); + EXPECT_EQ(vvT.cols(), 2); + // Element-wise multiplication + matrix_eig mvvT = m.array() * vvT.array(); + EXPECT_EQ(mvvT.rows(), 2); + EXPECT_EQ(mvvT.cols(), 2); + EXPECT_EQ(m(0, 0) * vvT(0, 0), mvvT(0, 0)); + EXPECT_EQ(m(0, 1) * vvT(0, 1), mvvT(0, 1)); + EXPECT_EQ(m(1, 0) * vvT(1, 0), mvvT(1, 0)); + EXPECT_EQ(m(1, 1) * vvT(1, 1), mvvT(1, 1)); } TEST_F(TensorflowTests, SineWavePredictionTest) { diff --git a/test/brain/testing_index_selection_util.cpp b/test/brain/testing_index_selection_util.cpp new file mode 100644 index 00000000000..cbf7f3bf894 --- /dev/null +++ b/test/brain/testing_index_selection_util.cpp @@ -0,0 +1,444 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// testing_index_selection_util.cpp +// +// Identification: test/brain/testing_index_selection_util.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/testing_index_selection_util.h" +#include "brain/what_if_index.h" +#include "common/harness.h" +#include "optimizer/stats/stats_storage.h" +#include "sql/testing_sql_util.h" +#include "planner/index_scan_plan.h" + +namespace peloton { + +namespace test { + +namespace index_selection { + +TestingIndexSelectionUtil::TestingIndexSelectionUtil(std::string db_name) + : database_name_(db_name) { + srand(time(NULL)); + CreateDatabase(); +} + +TestingIndexSelectionUtil::~TestingIndexSelectionUtil() { + for (auto it = tables_created_.begin(); it != tables_created_.end(); it++) { + DropTable(it->first); + } + DropDatabase(); +} + +std::pair, std::vector> +TestingIndexSelectionUtil::GetQueryStringsWorkload( + QueryStringsWorkloadType type) { + std::vector query_strs; + std::vector table_schemas; + std::string table_name; + // Procedure to add a new workload: + // 1. Create all the table schemas required for the workload queries. + // 2. Create all the required workload query strings. + // Note on Naming of workloads: TableColW + switch (type) { + case QueryStringsWorkloadType::SingleTableNoop: { + table_name = "dummy0"; + table_schemas.emplace_back( + table_name, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGERPKEY}, + {"c", TupleValueType::INTEGER}}); + // This query string is not actually executed - only used for testing + // add/drop candidates + query_strs.push_back("UPDATE dummy0 SET a = 0 WHERE b = 1 AND c = 2"); + break; + } + case QueryStringsWorkloadType::SingleTableTwoColW1: { + table_name = "dummy1"; + table_schemas.emplace_back( + table_name, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a = 160 and a = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE c = 190 and c = 250"); + query_strs.push_back("SELECT a, b, c FROM " + table_name + + " WHERE a = 190 and c = 250"); + break; + } + case QueryStringsWorkloadType::SingleTableTwoColW2: { + table_name = "dummy2"; + table_schemas.emplace_back( + table_name, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + query_strs.push_back("SELECT * FROM " + table_name + " WHERE a = 160"); + query_strs.push_back("SELECT * FROM " + table_name + " WHERE b = 190"); + query_strs.push_back("SELECT * FROM " + table_name + " WHERE b = 81"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a = 190 and b = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a = 190 and b = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 190 and a = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 190 and c = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 190 and c = 250"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a = 190 and c = 250"); + break; + } + case QueryStringsWorkloadType::SingleTableThreeColW: { + table_name = "dummy3"; + table_schemas.emplace_back( + table_name, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}, + {"e", TupleValueType::INTEGER}, + {"f", TupleValueType::INTEGER}, + {"g", TupleValueType::INTEGER}}); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE a = 160 and b = 199 and c = 1009"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 190 and a = 677 and c = 987"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 81 and c = 123 and a = 122"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 81 and c = 123 and d = 122"); + query_strs.push_back("SELECT * FROM " + table_name + " WHERE b = 81"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE b = 81 and c = 12"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE d = 81 and e = 123 and f = 122"); + query_strs.push_back("SELECT * FROM " + table_name + " WHERE d = 81"); + query_strs.push_back("SELECT * FROM " + table_name + + " WHERE d = 81 and e = 12"); + break; + } + case QueryStringsWorkloadType::MultiTableNoop: { + std::string table_name_1 = "dummy1"; + table_schemas.emplace_back( + table_name_1, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGERPKEY}, + {"c", TupleValueType::INTEGER}}); + std::string table_name_2 = "dummy2"; + table_schemas.emplace_back( + table_name_2, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}}); + std::string table_name_3 = "dummy3"; + table_schemas.emplace_back( + table_name_3, + std::initializer_list>{ + {"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}}); + // No workload + break; + } + case QueryStringsWorkloadType::MultiTableMultiColW: { + std::string table_name_1 = "d_student"; + table_schemas.emplace_back( + table_name_1, + std::initializer_list>{ + {"name", TupleValueType::STRING}, + {"gpa", TupleValueType::INTEGER}, + {"id", TupleValueType::INTEGER}, + {"cgpa", TupleValueType::INTEGER}}); + std::string table_name_2 = "d_college"; + table_schemas.emplace_back( + table_name_2, + std::initializer_list>{ + {"name", TupleValueType::STRING}, + {"city", TupleValueType::STRING}, + {"county", TupleValueType::STRING}, + {"state", TupleValueType::STRING}, + {"country", TupleValueType::STRING}, + {"enrolment", TupleValueType::INTEGER}}); + std::string table_name_3 = "d_course"; + table_schemas.emplace_back( + table_name_3, + std::initializer_list>{ + {"name", TupleValueType::STRING}, + {"id", TupleValueType::INTEGER}}); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE name = 'vamshi' and id = 40"); + query_strs.push_back("SELECT * FROM " + table_name_1 + " WHERE id = 100"); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE name = 'siva' and id = 50"); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE name = 'priyatham' and id = 60"); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE id = 69 and name = 'vamshi'"); + query_strs.push_back("SELECT * FROM " + table_name_1 + " WHERE id = 4"); + query_strs.push_back("SELECT * FROM " + table_name_1 + " WHERE id = 10"); + query_strs.push_back("SELECT cgpa FROM " + table_name_1 + + " WHERE name = 'vam'"); + query_strs.push_back("SELECT name FROM " + table_name_1 + + " WHERE cgpa = 3"); + query_strs.push_back("SELECT name FROM " + table_name_1 + + " WHERE cgpa = 9 and gpa = 9"); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE cgpa = 9 and gpa = 9 and name = 'vam'"); + query_strs.push_back("SELECT * FROM " + table_name_1 + + " WHERE gpa = 9 and name = 'vam' and cgpa = 9"); + query_strs.push_back("SELECT country FROM " + table_name_2 + + " WHERE name = 'cmu'"); + query_strs.push_back("UPDATE " + table_name_2 + + " set name = 'cmu' where country = 'usa'"); + query_strs.push_back("UPDATE " + table_name_2 + + " set name = 'berkeley' where country = 'usa'"); + query_strs.push_back("DELETE FROM " + table_name_1 + + " where name = 'vam'"); + query_strs.push_back("DELETE FROM " + table_name_2 + + " where name = 'vam'"); + query_strs.push_back("DELETE FROM " + table_name_1 + " where id = 1"); + query_strs.push_back( + "SELECT * FROM d_student s inner join d_college c on s.name = " + "c.name inner join d_course co on c.name = co.name"); + // The below 2(especially last one is prohibitively expensive) + // Unable to understand whether What-If is correctly measuring - since + // difference is minimal with or without indexes :/ +// query_strs.push_back( +// "SELECT * FROM d_student join d_college on d_student.name = " +// "d_college.name"); +// query_strs.push_back("SELECT * FROM " + table_name_1 + " t1 ," + +// table_name_2 + " t2 where t1.name = 'vam'"); + break; + } + default: + PELOTON_ASSERT(false); + } + return std::make_pair(table_schemas, query_strs); +} + +std::pair, std::vector> +TestingIndexSelectionUtil::GetCyclicWorkload(std::vector workload_types, + int num_cycles) { + // Using table names to prevent duplication + std::set schemas_processed; + std::vector query_strs; + std::vector table_schemas; + for(const auto &w_type: workload_types) { + auto config = GetQueryStringsWorkload(w_type); + auto config_schemas = config.first; + for(const auto &table_schema: config_schemas) { + if(schemas_processed.find(table_schema.table_name) == schemas_processed.end()) { + schemas_processed.insert(table_schema.table_name); + table_schemas.push_back(table_schema); + } + } + auto config_queries = config.second; + query_strs.insert(query_strs.end(), config_queries.begin(), config_queries.end()); + } + for(int i = 0; i < num_cycles - 1; i++) { + query_strs.insert(query_strs.end(), query_strs.begin(), query_strs.end()); + } + return std::make_pair(table_schemas, query_strs); +} + +// Creates a new table with the provided schema. +void TestingIndexSelectionUtil::CreateTable(TableSchema schema) { + // Create table. + std::ostringstream s_stream; + s_stream << "CREATE TABLE " << schema.table_name << " ("; + for (auto i = 0UL; i < schema.cols.size(); i++) { + s_stream << schema.cols[i].first; + s_stream << " "; + switch (schema.cols[i].second) { + case TupleValueType::FLOAT: + s_stream << "FLOAT"; + break; + case TupleValueType::INTEGER: + s_stream << "INT"; + break; + case TupleValueType::STRING: + s_stream << "VARCHAR(30)"; + break; + case TupleValueType::INTEGERPKEY: + s_stream << "INT PRIMARY KEY"; + break; + default: + PELOTON_ASSERT(false); + } + if (i < (schema.cols.size() - 1)) { + s_stream << ", "; + } + } + s_stream << ");"; + LOG_DEBUG("Create table: %s", s_stream.str().c_str()); + TestingSQLUtil::ExecuteSQLQuery(s_stream.str()); +} + +// Inserts specified number of tuples into the table with random values. +void TestingIndexSelectionUtil::InsertIntoTable(TableSchema schema, + long num_tuples) { + // Insert tuples into table + for (int i = 0; i < num_tuples; i++) { + std::ostringstream oss; + oss << "INSERT INTO " << schema.table_name << " VALUES ("; + for (auto col = 0UL; col < schema.cols.size(); col++) { + auto type = schema.cols[col].second; + switch (type) { + case TupleValueType::INTEGER: + oss << rand() % 1000; + break; + case TupleValueType::INTEGERPKEY: + oss << rand() % 1000; + break; + case TupleValueType::FLOAT: + oss << (float)(rand() % 100); + break; + case TupleValueType::STRING: + oss << "'str" << rand() % RAND_MAX << "'"; + break; + default: + PELOTON_ASSERT(false); + } + if (col < (schema.cols.size() - 1)) { + oss << ", "; + } + } + oss << ");"; + LOG_TRACE("Inserting: %s", oss.str().c_str()); + TestingSQLUtil::ExecuteSQLQuery(oss.str()); + } + GenerateTableStats(); +} + +void TestingIndexSelectionUtil::GenerateTableStats() { + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + optimizer::StatsStorage *stats_storage = + optimizer::StatsStorage::GetInstance(); + ResultType result = stats_storage->AnalyzeStatsForAllTables(txn); + PELOTON_ASSERT(result == ResultType::SUCCESS); + (void)result; + txn_manager.CommitTransaction(txn); +} + +// Factory method +// Returns a what-if index on the columns at the given +// offset of the table. +std::shared_ptr +TestingIndexSelectionUtil::CreateHypotheticalIndex( + std::string table_name, std::vector index_col_names, + brain::IndexSelection *is) { + // We need transaction to get table object. + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + // Get the existing table so that we can find its oid and the cols oids. + auto table_object = catalog::Catalog::GetInstance()->GetTableObject( + database_name_, "public", table_name, txn); + auto col_obj_pairs = table_object->GetColumnObjects(); + + std::vector col_ids; + auto database_oid = table_object->GetDatabaseOid(); + auto table_oid = table_object->GetTableOid(); + + // Find the column oids. + for (auto col_name : index_col_names) { + for (auto it = col_obj_pairs.begin(); it != col_obj_pairs.end(); it++) { + LOG_DEBUG("Table id: %d, Column id: %d, Offset: %d, Name: %s", + it->second->GetTableOid(), it->second->GetColumnId(), + it->second->GetColumnOffset(), + it->second->GetColumnName().c_str()); + if (col_name == it->second->GetColumnName()) { + col_ids.push_back(it->second->GetColumnId()); + } + } + } + PELOTON_ASSERT(col_ids.size() == index_col_names.size()); + + std::shared_ptr index_obj; + + if (is == nullptr) { + auto obj_ptr = + new brain::HypotheticalIndexObject(database_oid, table_oid, col_ids); + index_obj = std::shared_ptr(obj_ptr); + } else { + auto obj = brain::HypotheticalIndexObject(database_oid, table_oid, col_ids); + index_obj = is->AddConfigurationToPool(obj); + } + + txn_manager.CommitTransaction(txn); + return index_obj; +} + +void TestingIndexSelectionUtil::CreateDatabase() { + std::string create_db_str = "CREATE DATABASE " + database_name_ + ";"; + TestingSQLUtil::ExecuteSQLQuery(create_db_str); +} + +void TestingIndexSelectionUtil::DropDatabase() { + std::string create_str = "DROP DATABASE " + database_name_ + ";"; + TestingSQLUtil::ExecuteSQLQuery(create_str); +} + +void TestingIndexSelectionUtil::DropTable(std::string table_name) { + std::string create_str = "DROP TABLE " + table_name + ";"; + TestingSQLUtil::ExecuteSQLQuery(create_str); +} + +void TestingIndexSelectionUtil::CreateIndex(std::shared_ptr index_obj) { + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + auto catalog = catalog::Catalog::GetInstance(); + std::string table_name = + catalog->GetTableObject(index_obj->db_oid, index_obj->table_oid, txn)->GetTableName(); + + catalog->CreateIndex(database_name_, DEFAULT_SCHEMA_NAME, table_name, + index_obj->column_oids, index_obj->ToString(), false, + IndexType::BWTREE, txn); + txn_manager.CommitTransaction(txn); +} + +double TestingIndexSelectionUtil::WhatIfIndexCost(std::string query, + brain::IndexConfiguration &config, + std::string database_name) { + std::unique_ptr stmt_list( + parser::PostgresParser::ParseSQLString(query)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, database_name)); + + // Get the first statement. + auto sql_statement = std::shared_ptr( + stmt_list->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement.get()); + auto cost = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + database_name, txn)->cost; + txn_manager.CommitTransaction(txn); + return cost; +} + +} // namespace index_selection +} // namespace test +} // namespace peloton diff --git a/test/brain/what_if_index_test.cpp b/test/brain/what_if_index_test.cpp new file mode 100644 index 00000000000..2f93955f71c --- /dev/null +++ b/test/brain/what_if_index_test.cpp @@ -0,0 +1,490 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// what_if_index_test.cpp +// +// Identification: test/brain/what_if_index_test.cpp +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "brain/what_if_index.h" +#include "common/harness.h" +#include "optimizer/stats/stats_storage.h" +#include "sql/testing_sql_util.h" +#include "planner/index_scan_plan.h" + +#include "brain/testing_index_selection_util.h" + +namespace peloton { +namespace test { + +using namespace index_selection; + +//===--------------------------------------------------------------------===// +// WhatIfIndex Tests +//===--------------------------------------------------------------------===// +class WhatIfIndexTests : public PelotonTest { + public: + WhatIfIndexTests() {} +}; + +TEST_F(WhatIfIndexTests, SingleColTest) { + std::string db_name = DEFAULT_DB_NAME; + int num_rows = 100; + + TableSchema schema("table1", {{"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + + TestingIndexSelectionUtil testing_util(db_name); + testing_util.CreateTable(schema); + testing_util.InsertIntoTable(schema, num_rows); + + // Form the query. + std::string query("SELECT a from " + schema.table_name + + " WHERE b = 100 and c = 5;"); + LOG_TRACE("Query: %s", query.c_str()); + + brain::IndexConfiguration config; + + std::unique_ptr stmt_list( + parser::PostgresParser::ParseSQLString(query)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto parser = parser::PostgresParser::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, DEFAULT_DB_NAME)); + + // Get the first statement. + auto sql_statement = std::shared_ptr( + stmt_list.get()->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement.get()); + + // 1. Get the optimized plan tree without the indexes (sequential scan) + auto result = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement, config, DEFAULT_DB_NAME, txn); + auto cost_without_index = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + LOG_TRACE("Cost of the query without indexes: %lf", cost_without_index); + EXPECT_NE(result->plan, nullptr); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + // 2. Get the optimized plan tree with 1 hypothetical indexes (indexes) + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b"})); + + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_1 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with 1 index: %lf", cost_with_index_1); + EXPECT_NE(result->plan, nullptr); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + // 3. Get the optimized plan tree with 2 hypothetical indexes (indexes) + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"c"})); + + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_2 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with 2 indexes: %lf", cost_with_index_2); + + EXPECT_LT(cost_with_index_1, cost_without_index); + EXPECT_LT(cost_with_index_2, cost_without_index); + EXPECT_NE(result->plan, nullptr); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + txn_manager.CommitTransaction(txn); +} + +/** + * @brief This test checks if a hypothetical index on multiple columns + * helps a particular query. + */ +TEST_F(WhatIfIndexTests, MultiColumnTest1) { + std::string db_name = DEFAULT_DB_NAME; + int num_rows = 1000; + + TableSchema schema("table1", {{"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + TestingIndexSelectionUtil testing_util(db_name); + testing_util.CreateTable(schema); + testing_util.InsertIntoTable(schema, num_rows); + + // Form the query + std::string query("SELECT a from " + schema.table_name + + " WHERE b = 200 and c = 100;"); + LOG_TRACE("Query: %s", query.c_str()); + + brain::IndexConfiguration config; + + std::unique_ptr stmt_list( + parser::PostgresParser::ParseSQLString(query)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto parser = parser::PostgresParser::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, DEFAULT_DB_NAME)); + + // Get the first statement. + auto sql_statement = std::shared_ptr( + stmt_list.get()->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement.get()); + + // Get the optimized plan tree without the indexes (sequential scan) + auto result = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement, config, DEFAULT_DB_NAME, txn); + auto cost_without_index = result->cost; + LOG_TRACE("Cost of the query without indexes {}: %lf", cost_without_index); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + // Insert hypothetical catalog objects + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"a", "c"})); + + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_1 = result->cost; + LOG_TRACE("Cost of the query with index {'a', 'c'}: %lf", cost_with_index_1); + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + EXPECT_DOUBLE_EQ(cost_without_index, cost_with_index_1); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"a", "b"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_2 = result->cost; + LOG_TRACE("Cost of the query with index {'a', 'b'}: %lf", cost_with_index_2); + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + EXPECT_DOUBLE_EQ(cost_without_index, cost_with_index_2); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b", "c"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_3 = result->cost; + LOG_TRACE("Cost of the query with index {'b', 'c'}: %lf", cost_with_index_3); + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + EXPECT_GT(cost_without_index, cost_with_index_3); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_4 = result->cost; + EXPECT_LE(cost_with_index_3, cost_with_index_4); + + // The cost of using one index {1} should be greater than the cost + // of using both the indexes {1, 2} for the query. + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'b'}: %lf", cost_with_index_4); + LOG_TRACE("%s", result->plan->GetInfo().c_str()); + + txn_manager.CommitTransaction(txn); +} + +TEST_F(WhatIfIndexTests, MultiColumnTest2) { + std::string db_name = DEFAULT_DB_NAME; + int num_rows = 1000; + + TableSchema schema("table1", {{"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}, + {"e", TupleValueType::INTEGER}, + {"f", TupleValueType::INTEGER}}); + TestingIndexSelectionUtil testing_util(db_name); + testing_util.CreateTable(schema); + testing_util.InsertIntoTable(schema, num_rows); + + // Form the query. + std::string query("SELECT a from " + schema.table_name + + " WHERE b = 500 AND e = 100;"); + LOG_TRACE("Query: %s", query.c_str()); + + brain::IndexConfiguration config; + + std::unique_ptr stmt_list( + parser::PostgresParser::ParseSQLString(query)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto parser = parser::PostgresParser::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, DEFAULT_DB_NAME)); + + // Get the first statement. + auto sql_statement = std::shared_ptr( + stmt_list.get()->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement.get()); + + // Get the optimized plan tree without the indexes (sequential scan) + auto result = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement, config, DEFAULT_DB_NAME, txn); + auto cost_without_index = result->cost; + LOG_TRACE("Cost of the query without indexes: %lf", cost_without_index); + + // Insert hypothetical catalog objects + // Index on cols a, b, c, d, e. + config.AddIndexObject(testing_util.CreateHypotheticalIndex( + schema.table_name, {"a", "b", "c", "d", "e"})); + + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_1 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + LOG_TRACE("Cost of the query with index {'a', 'b', 'c', 'd', 'e'}: %lf", + cost_with_index_1); + EXPECT_DOUBLE_EQ(cost_without_index, cost_with_index_1); + + config.Clear(); + config.AddIndexObject(testing_util.CreateHypotheticalIndex( + schema.table_name, {"a", "c", "d", "f"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_2 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + LOG_TRACE("Cost of the query with index {'a', 'c', 'd', 'f'}: %lf", + cost_with_index_2); + EXPECT_DOUBLE_EQ(cost_without_index, cost_with_index_2); + + config.Clear(); + config.AddIndexObject(testing_util.CreateHypotheticalIndex( + schema.table_name, {"a", "b", "d", "e"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_3 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + LOG_TRACE("Cost of the query with index {'a', 'b', 'd', 'e'}: %lf", + cost_with_index_3); + EXPECT_DOUBLE_EQ(cost_without_index, cost_with_index_3); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b", "c", "e"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_4 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'b', 'c', 'e'}: %lf", + cost_with_index_4); + EXPECT_GT(cost_without_index, cost_with_index_4); + + config.Clear(); + config.AddIndexObject(testing_util.CreateHypotheticalIndex( + schema.table_name, {"b", "c", "d", "e"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_5 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'b', 'c', 'd', 'e'}: %lf", + cost_with_index_5); + EXPECT_GT(cost_without_index, cost_with_index_5); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b", "e"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_6 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'b', 'e'}: %lf", cost_with_index_6); + EXPECT_GT(cost_without_index, cost_with_index_6); + EXPECT_GT(cost_with_index_5, cost_with_index_6); + EXPECT_GT(cost_with_index_4, cost_with_index_6); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"e"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_7 = result->cost; + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'e'} : %lf", cost_with_index_7); + EXPECT_GT(cost_without_index, cost_with_index_7); + EXPECT_GT(cost_with_index_7, cost_with_index_6); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"b"})); + result = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_8 = result->cost; + LOG_TRACE("Cost of the query with index {'b'}: %lf", cost_with_index_8); + EXPECT_EQ(result->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + EXPECT_GT(cost_without_index, cost_with_index_8); + EXPECT_GT(cost_with_index_8, cost_with_index_6); + + txn_manager.CommitTransaction(txn); +} + +/** + * @brief This test checks if a hypothetical index on multiple columns + * helps a particular query. + */ +TEST_F(WhatIfIndexTests, MultiColumnTest3) { + std::string db_name = DEFAULT_DB_NAME; + int num_rows = 1000; + + TableSchema schema("table1", {{"a", TupleValueType::INTEGER}, + {"b", TupleValueType::INTEGER}, + {"c", TupleValueType::INTEGER}, + {"d", TupleValueType::INTEGER}}); + TestingIndexSelectionUtil testing_util(db_name); + testing_util.CreateTable(schema); + testing_util.InsertIntoTable(schema, num_rows); + + // Form the query + std::string query1("SELECT a from " + schema.table_name + + " WHERE a = 50 and b = 200 and c = 100 and d = 50;"); + std::string query2("SELECT a from " + schema.table_name + + " WHERE c = 100 and a = 50 and d = 1 and b = 123;"); + std::string query3("SELECT a from " + schema.table_name + + " WHERE d = 100 and c = 50 and b = 1 and a = 13;"); + LOG_TRACE("Query1: %s", query1.c_str()); + LOG_TRACE("Query2: %s", query2.c_str()); + LOG_TRACE("Query3: %s", query3.c_str()); + + brain::IndexConfiguration config; + + std::unique_ptr stmt_list1( + parser::PostgresParser::ParseSQLString(query1)); + std::unique_ptr stmt_list2( + parser::PostgresParser::ParseSQLString(query2)); + std::unique_ptr stmt_list3( + parser::PostgresParser::ParseSQLString(query3)); + + auto &txn_manager = concurrency::TransactionManagerFactory::GetInstance(); + auto parser = parser::PostgresParser::GetInstance(); + auto txn = txn_manager.BeginTransaction(); + + std::unique_ptr binder( + new binder::BindNodeVisitor(txn, DEFAULT_DB_NAME)); + + // Get the first statement. + auto sql_statement1 = std::shared_ptr( + stmt_list1.get()->PassOutStatement(0)); + auto sql_statement2 = std::shared_ptr( + stmt_list2.get()->PassOutStatement(0)); + auto sql_statement3 = std::shared_ptr( + stmt_list3.get()->PassOutStatement(0)); + + binder->BindNameToNode(sql_statement1.get()); + binder->BindNameToNode(sql_statement2.get()); + binder->BindNameToNode(sql_statement3.get()); + + // Get the optimized plan tree without the indexes (sequential scan) + auto result1 = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement1, config, DEFAULT_DB_NAME, txn); + auto cost_without_index = result1->cost; + LOG_TRACE("Cost of the query without indexes {}: %lf", cost_without_index); + LOG_TRACE("%s", result1->plan->GetInfo().c_str()); + EXPECT_EQ(result1->plan->GetPlanNodeType(), PlanNodeType::SEQSCAN); + + // Insert hypothetical catalog objects + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"a"})); + + result1 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement1, config, + DEFAULT_DB_NAME, txn); + auto result2 = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement2, config, DEFAULT_DB_NAME, txn); + auto result3 = brain::WhatIfIndex::GetCostAndBestPlanTree( + sql_statement3, config, DEFAULT_DB_NAME, txn); + auto cost_with_index_1_1 = result1->cost; + auto cost_with_index_1_2 = result2->cost; + auto cost_with_index_1_3 = result3->cost; + LOG_TRACE("Cost of the query with index {'a'}: %lf", cost_with_index_1_1); + EXPECT_EQ(result1->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + EXPECT_GT(cost_without_index, cost_with_index_1_1); + EXPECT_DOUBLE_EQ(cost_with_index_1_1, cost_with_index_1_2); + EXPECT_DOUBLE_EQ(cost_with_index_1_2, cost_with_index_1_3); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"a", "b"})); + result1 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement1, config, + DEFAULT_DB_NAME, txn); + result2 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement2, config, + DEFAULT_DB_NAME, txn); + result3 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement3, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_2_1 = result1->cost; + auto cost_with_index_2_2 = result2->cost; + auto cost_with_index_2_3 = result3->cost; + EXPECT_EQ(result1->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'a', 'b'}: %lf", + cost_with_index_2_1); + EXPECT_GT(cost_without_index, cost_with_index_2_1); + EXPECT_GT(cost_with_index_1_1, cost_with_index_2_1); + EXPECT_DOUBLE_EQ(cost_with_index_2_1, cost_with_index_2_2); + EXPECT_DOUBLE_EQ(cost_with_index_2_2, cost_with_index_2_3); + + config.Clear(); + config.AddIndexObject( + testing_util.CreateHypotheticalIndex(schema.table_name, {"a", "b", "c"})); + result1 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement1, config, + DEFAULT_DB_NAME, txn); + result2 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement2, config, + DEFAULT_DB_NAME, txn); + result3 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement3, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_3_1 = result1->cost; + auto cost_with_index_3_2 = result2->cost; + auto cost_with_index_3_3 = result3->cost; + EXPECT_EQ(result1->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'a', 'b', 'c'}: %lf", + cost_with_index_3_1); + EXPECT_GT(cost_without_index, cost_with_index_3_1); + EXPECT_GT(cost_with_index_2_1, cost_with_index_3_1); + EXPECT_DOUBLE_EQ(cost_with_index_3_1, cost_with_index_3_2); + EXPECT_DOUBLE_EQ(cost_with_index_3_2, cost_with_index_3_3); + + config.Clear(); + config.AddIndexObject(testing_util.CreateHypotheticalIndex( + schema.table_name, {"a", "b", "c", "d"})); + result1 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement1, config, + DEFAULT_DB_NAME, txn); + result2 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement2, config, + DEFAULT_DB_NAME, txn); + result3 = brain::WhatIfIndex::GetCostAndBestPlanTree(sql_statement3, config, + DEFAULT_DB_NAME, txn); + auto cost_with_index_4_1 = result1->cost; + auto cost_with_index_4_2 = result2->cost; + auto cost_with_index_4_3 = result3->cost; + EXPECT_EQ(result1->plan->GetPlanNodeType(), PlanNodeType::INDEXSCAN); + LOG_TRACE("Cost of the query with index {'a', 'b', 'c', 'd'}: %lf", + cost_with_index_4_1); + EXPECT_GT(cost_without_index, cost_with_index_4_1); + EXPECT_GT(cost_with_index_3_1, cost_with_index_4_1); + // TODO(saatviks): Changed from DOUBLE to FLOAT_EQ + EXPECT_FLOAT_EQ(cost_with_index_4_1, cost_with_index_4_2); + EXPECT_FLOAT_EQ(cost_with_index_4_2, cost_with_index_4_3); + + txn_manager.CommitTransaction(txn); +} + +} // namespace test +} // namespace peloton diff --git a/test/include/brain/testing_index_selection_util.h b/test/include/brain/testing_index_selection_util.h new file mode 100644 index 00000000000..9f3ff68e36b --- /dev/null +++ b/test/include/brain/testing_index_selection_util.h @@ -0,0 +1,164 @@ +//===----------------------------------------------------------------------===// +// +// Peloton +// +// testing_index_selection_util.h +// +// Identification: test/include/brain/testing_index_selection_util.h +// +// Copyright (c) 2015-2018, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "brain/index_selection_util.h" +#include "brain/index_selection.h" +#include "brain/what_if_index.h" + +namespace peloton { +namespace test { + +namespace index_selection { + +/** + * Table column type. + */ +enum class TupleValueType { INTEGER, FLOAT, STRING, INTEGERPKEY }; + +/** + * Represents workload types used in the test cases. + */ +enum class QueryStringsWorkloadType { SingleTableTwoColW1, SingleTableTwoColW2, + SingleTableThreeColW, MultiTableMultiColW, SingleTableNoop, MultiTableNoop }; + +/** + * Represents the schema for creating tables in the test cases. + */ +class TableSchema { + public: + std::vector> cols; + std::unordered_map col_offset_map; + std::string table_name; + + TableSchema(){}; + TableSchema(std::string table_name, + std::vector> columns) { + auto i = 0UL; + for (auto col : columns) { + cols.push_back(col); + col_offset_map[col.first] = i; + i++; + } + this->table_name = table_name; + } +}; + +/** + * Utility class for testing Index Selection (auto-index). + */ +class TestingIndexSelectionUtil { + public: + /** + * Creates a database. + * @param db_name + */ + TestingIndexSelectionUtil(std::string db_name); + + /** + * Drops all tables and the database. + */ + ~TestingIndexSelectionUtil(); + + /** + * Inserts specified number of tuples. + * @param schema schema of the table to be created + * @param num_tuples number of tuples to be inserted with random values. + */ + void InsertIntoTable(TableSchema schema, long num_tuples); + + /** + * Create a new table.s + * @param schema + */ + void CreateTable(TableSchema schema); + + /** + * Factory method to create a hypothetical index object. The returned object + * can be used in the catalog or catalog cache. + * @param table_name + * @param index_col_names + * @return + */ + std::shared_ptr CreateHypotheticalIndex( + std::string table_name, std::vector cols, + brain::IndexSelection *is = nullptr); + + /** + * Given a hypothetical index object, this method creates an actual index + * corresponding to that hypothetical index object + * @param index_obj: hypothetical index object + */ + void CreateIndex(std::shared_ptr index_obj); + + /** + * Return a micro workload + * This function returns queries and the respective table schemas + * User of this function must create all of the returned tables. + * @param workload_type type of the workload to be returned + * @return workload query strings along with the table schema + */ + std::pair, std::vector> + GetQueryStringsWorkload(QueryStringsWorkloadType workload_type); + + /** + * Return a cyclic workload + * This function accepts a list of workload types and number of + * workload cycles and returns the representative cyclic workload + * eg. ((W1, W2), 3) -> (W1, W2, W1, W2, W1, W2) + * @param workload_types sequence of the workloads - you can assume one cycle involves running such a sequence + * @return workload query strings along with the table schema + */ + std::pair, std::vector> + GetCyclicWorkload(std::vector workload_types, int num_cycles); + + /** + * Get the an estimate of cost of running a query on a given + * index configuration by the cost model(Available via What-If API) + * @param query: the query string + * @param config: Index configuration + * @param database_name: DB name + * @return: "What-If" Index cost + */ + double WhatIfIndexCost(std::string query, + brain::IndexConfiguration &config, + std::string database_name); + + private: + std::string database_name_; + std::unordered_map tables_created_; + + /** + * Create the database + */ + void CreateDatabase(); + + /** + * Drop the database + */ + void DropDatabase(); + + /** + * Drop the table + */ + void DropTable(std::string table_name); + + /** + * Generate stats for all the tables in the system. + */ + void GenerateTableStats(); +}; +} + +} // namespace test +} // namespace peloton diff --git a/test/planner/plan_util_test.cpp b/test/planner/plan_util_test.cpp index 77df6f54e88..2014560699b 100644 --- a/test/planner/plan_util_test.cpp +++ b/test/planner/plan_util_test.cpp @@ -40,6 +40,7 @@ TEST_F(PlanUtilTests, GetAffectedIndexesTest) { auto txn = txn_manager.BeginTransaction(); catalog->CreateDatabase(TEST_DB_NAME, txn); + auto db = catalog->GetDatabaseWithName(TEST_DB_NAME, txn); // Insert a table first auto id_column = catalog::Column( type::TypeId::INTEGER, type::Type::GetTypeSize(type::TypeId::INTEGER), @@ -62,6 +63,8 @@ TEST_F(PlanUtilTests, GetAffectedIndexesTest) { txn_manager.CommitTransaction(txn); txn = txn_manager.BeginTransaction(); + oid_t db_oid = db->GetOid(); + oid_t table_oid = source_table->GetOid(); oid_t col_id = source_table->GetSchema()->GetColumnID(id_column.column_name); std::vector source_col_ids; source_col_ids.push_back(col_id); @@ -99,68 +102,78 @@ TEST_F(PlanUtilTests, GetAffectedIndexesTest) { // An update query affecting both indexes std::string query_string = "UPDATE test_table SET id = 0;"; - std::unique_ptr stmt(new Statement("UPDATE", query_string)); auto &peloton_parser = parser::PostgresParser::GetInstance(); auto sql_stmt_list = peloton_parser.BuildParseTree(query_string); auto sql_stmt = sql_stmt_list->GetStatement(0); static_cast(sql_stmt) ->table->TryBindDatabaseName(TEST_DB_NAME); - std::set affected_indexes = + std::vector affected_indexes = planner::PlanUtil::GetAffectedIndexes(txn->catalog_cache, *sql_stmt); + std::set affected_indexes_set(affected_indexes.begin(), + affected_indexes.end()); // id and first_name are affected - EXPECT_EQ(2, static_cast(affected_indexes.size())); - std::set expected_oids{id_idx_oid, fname_idx_oid}; - EXPECT_EQ(expected_oids, affected_indexes); + EXPECT_EQ(2, static_cast(affected_indexes_set.size())); + std::set expected_oids; + expected_oids.emplace(db_oid, table_oid, id_idx_oid); + expected_oids.emplace(db_oid, table_oid, fname_idx_oid); + EXPECT_EQ(expected_oids, affected_indexes_set); // Update query affecting only one index query_string = "UPDATE test_table SET first_name = '';"; - stmt.reset(new Statement("UPDATE", query_string)); sql_stmt_list = peloton_parser.BuildParseTree(query_string); sql_stmt = sql_stmt_list->GetStatement(0); static_cast(sql_stmt) ->table->TryBindDatabaseName(TEST_DB_NAME); affected_indexes = planner::PlanUtil::GetAffectedIndexes(txn->catalog_cache, *sql_stmt); + affected_indexes_set = std::set( + affected_indexes.begin(), affected_indexes.end()); // only first_name is affected - EXPECT_EQ(1, static_cast(affected_indexes.size())); - expected_oids = std::set({fname_idx_oid}); - EXPECT_EQ(expected_oids, affected_indexes); + EXPECT_EQ(1, static_cast(affected_indexes_set.size())); + expected_oids.clear(); + expected_oids.emplace(db_oid, table_oid, fname_idx_oid); + EXPECT_EQ(expected_oids, affected_indexes_set); // ====== DELETE statements check === query_string = "DELETE FROM test_table;"; - stmt.reset(new Statement("DELETE", query_string)); sql_stmt_list = peloton_parser.BuildParseTree(query_string); sql_stmt = sql_stmt_list->GetStatement(0); static_cast(sql_stmt) ->TryBindDatabaseName(TEST_DB_NAME); affected_indexes = planner::PlanUtil::GetAffectedIndexes(txn->catalog_cache, *sql_stmt); + affected_indexes_set = std::set( + affected_indexes.begin(), affected_indexes.end()); // all indexes are affected - EXPECT_EQ(2, static_cast(affected_indexes.size())); - expected_oids = std::set({id_idx_oid, fname_idx_oid}); - EXPECT_EQ(expected_oids, affected_indexes); + EXPECT_EQ(2, static_cast(affected_indexes_set.size())); + expected_oids.clear(); + expected_oids.emplace(db_oid, table_oid, fname_idx_oid); + expected_oids.emplace(db_oid, table_oid, id_idx_oid); + EXPECT_EQ(expected_oids, affected_indexes_set); // ========= INSERT statements check == query_string = "INSERT INTO test_table VALUES (1, 'pel', 'ton');"; - stmt.reset(new Statement("INSERT", query_string)); sql_stmt_list = peloton_parser.BuildParseTree(query_string); sql_stmt = sql_stmt_list->GetStatement(0); static_cast(sql_stmt) ->TryBindDatabaseName(TEST_DB_NAME); affected_indexes = planner::PlanUtil::GetAffectedIndexes(txn->catalog_cache, *sql_stmt); + affected_indexes_set = std::set( + affected_indexes.begin(), affected_indexes.end()); // all indexes are affected - EXPECT_EQ(2, static_cast(affected_indexes.size())); - expected_oids = std::set({id_idx_oid, fname_idx_oid}); - EXPECT_EQ(expected_oids, affected_indexes); + EXPECT_EQ(2, static_cast(affected_indexes_set.size())); + expected_oids.clear(); + expected_oids.emplace(db_oid, table_oid, fname_idx_oid); + expected_oids.emplace(db_oid, table_oid, id_idx_oid); + EXPECT_EQ(expected_oids, affected_indexes_set); // ========= SELECT statement check == query_string = "SELECT * FROM test_table;"; - stmt.reset(new Statement("SELECT", query_string)); sql_stmt_list = peloton_parser.BuildParseTree(query_string); sql_stmt = sql_stmt_list->GetStatement(0); affected_indexes = @@ -179,9 +192,8 @@ TEST_F(PlanUtilTests, GetIndexableColumnsTest) { auto txn = txn_manager.BeginTransaction(); catalog->CreateDatabase(TEST_DB_COLUMNS, txn); - auto db = catalog->GetDatabaseWithName(TEST_DB_COLUMNS, txn); - oid_t database_id = db->GetOid(); auto db_object = catalog->GetDatabaseObject(TEST_DB_COLUMNS, txn); + oid_t database_id = db_object->GetDatabaseOid(); int table_count = db_object->GetTableObjects().size(); txn_manager.CommitTransaction(txn); @@ -204,6 +216,7 @@ TEST_F(PlanUtilTests, GetIndexableColumnsTest) { // Obtain ids for the table and columns txn = txn_manager.BeginTransaction(); + auto source_table = catalog->GetTableWithName( TEST_DB_COLUMNS, DEFAULT_SCHEMA_NAME, "test_table", txn); txn_manager.CommitTransaction(txn);