diff --git a/CMakeLists.txt b/CMakeLists.txt index 87e7bc3c7..471135127 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -324,37 +324,33 @@ set(P3_FILES "src/include/execution/executors/index_scan_executor.h" "src/include/execution/executors/insert_executor.h" "src/include/execution/executors/limit_executor.h" + "src/include/execution/executors/nested_index_join_executor.h" "src/include/execution/executors/nested_loop_join_executor.h" "src/include/execution/executors/seq_scan_executor.h" - "src/include/execution/executors/sort_executor.h" - "src/include/execution/executors/topn_executor.h" - "src/include/execution/executors/topn_per_group_executor.h" + "src/include/execution/executors/external_merge_sort_executor.h" "src/include/execution/executors/update_executor.h" - "src/include/execution/executors/window_function_executor.h" "src/execution/aggregation_executor.cpp" - "src/execution/window_function_executor.cpp" "src/execution/delete_executor.cpp" "src/execution/filter_executor.cpp" "src/execution/hash_join_executor.cpp" "src/execution/index_scan_executor.cpp" "src/execution/insert_executor.cpp" "src/execution/limit_executor.cpp" + "src/execution/nested_index_join_executor.cpp" "src/execution/nested_loop_join_executor.cpp" "src/execution/seq_scan_executor.cpp" - "src/execution/sort_executor.cpp" - "src/execution/topn_executor.cpp" - "src/execution/topn_per_group_executor.cpp" + "src/execution/external_merge_sort_executor.cpp" "src/execution/update_executor.cpp" + "src/include/execution/execution_common.h" "src/include/optimizer/optimizer.h" "src/include/optimizer/optimizer_internal.h" + "src/execution/execution_common.cpp" "src/optimizer/nlj_as_hash_join.cpp" "src/optimizer/optimizer_custom_rules.cpp" - "src/optimizer/sort_limit_as_topn.cpp" "src/optimizer/optimizer_internal.cpp" "src/optimizer/seqscan_as_indexscan.cpp" "src/optimizer/column_pruning.cpp" "src/common/bustub_ddl.cpp" - "src/include/execution/plans/topn_per_group_plan.h" ${P2_FILES} ) @@ -375,8 +371,6 @@ set(P4_FILES "src/concurrency/transaction_manager.cpp" "src/include/concurrency/watermark.h" "src/concurrency/watermark.cpp" - "src/include/execution/execution_common.h" - "src/execution/execution_common.cpp" ${P3_FILES} ) diff --git a/src/catalog/table_generator.cpp b/src/catalog/table_generator.cpp index 7e6223874..02c8f860d 100644 --- a/src/catalog/table_generator.cpp +++ b/src/catalog/table_generator.cpp @@ -62,7 +62,7 @@ auto TableGenerator::MakeValues(ColumnInsertMeta *col_meta, uint32_t count) -> s } } -void TableGenerator::FillTable(TableInfo *info, TableInsertMeta *table_meta) { +void TableGenerator::FillTable(const std::shared_ptr &info, TableInsertMeta *table_meta) { uint32_t num_inserted = 0; uint32_t batch_size = 128; while (num_inserted < table_meta->num_rows_) { diff --git a/src/common/bustub_ddl.cpp b/src/common/bustub_ddl.cpp index 7cf63c5f0..96e5f3b42 100644 --- a/src/common/bustub_ddl.cpp +++ b/src/common/bustub_ddl.cpp @@ -45,7 +45,7 @@ namespace bustub { void BusTubInstance::HandleCreateStatement(Transaction *txn, const CreateStatement &stmt, ResultWriter &writer) { std::unique_lock l(catalog_lock_); auto info = catalog_->CreateTable(txn, stmt.table_, Schema(stmt.columns_)); - IndexInfo *index = nullptr; + std::shared_ptr index = nullptr; if (!stmt.primary_key_.empty()) { std::vector col_ids; for (const auto &col : stmt.primary_key_) { @@ -106,7 +106,7 @@ void BusTubInstance::HandleIndexStatement(Transaction *txn, const IndexStatement } std::unique_lock l(catalog_lock_); - IndexInfo *info = nullptr; + std::shared_ptr info = nullptr; if (stmt.index_type_.empty()) { info = catalog_->CreateIndex( diff --git a/src/common/bustub_instance.cpp b/src/common/bustub_instance.cpp index 1ec6e9a97..4bd25123b 100644 --- a/src/common/bustub_instance.cpp +++ b/src/common/bustub_instance.cpp @@ -162,7 +162,7 @@ void BusTubInstance::CmdDbgMvcc(const std::vector ¶ms, ResultWr writer.OneCell("table " + table + " not found"); return; } - TxnMgrDbg("\\dbgmvcc", txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("\\dbgmvcc", txn_manager_.get(), table_info.get(), table_info->table_.get()); } void BusTubInstance::CmdDisplayTables(ResultWriter &writer) { @@ -175,7 +175,7 @@ void BusTubInstance::CmdDisplayTables(ResultWriter &writer) { writer.EndHeader(); for (const auto &name : table_names) { writer.BeginRow(); - const auto *table_info = catalog_->GetTable(name); + const auto table_info = catalog_->GetTable(name); writer.WriteCell(fmt::format("{}", table_info->oid_)); writer.WriteCell(table_info->name_); writer.WriteCell(table_info->schema_.ToString()); @@ -194,7 +194,7 @@ void BusTubInstance::CmdDisplayIndices(ResultWriter &writer) { writer.WriteHeaderCell("index_cols"); writer.EndHeader(); for (const auto &table_name : table_names) { - for (const auto *index_info : catalog_->GetTableIndexes(table_name)) { + for (const auto &index_info : catalog_->GetTableIndexes(table_name)) { writer.BeginRow(); writer.WriteCell(table_name); writer.WriteCell(fmt::format("{}", index_info->index_oid_)); diff --git a/src/execution/CMakeLists.txt b/src/execution/CMakeLists.txt index cab89f5c0..172744693 100644 --- a/src/execution/CMakeLists.txt +++ b/src/execution/CMakeLists.txt @@ -4,6 +4,7 @@ add_library( aggregation_executor.cpp delete_executor.cpp execution_common.cpp + external_merge_sort_executor.cpp executor_factory.cpp filter_executor.cpp fmt_impl.cpp diff --git a/src/execution/execution_common.cpp b/src/execution/execution_common.cpp index 0df9d1a75..617027b3b 100644 --- a/src/execution/execution_common.cpp +++ b/src/execution/execution_common.cpp @@ -10,6 +10,19 @@ namespace bustub { +TupleComparator::TupleComparator(std::vector order_bys) : order_bys_(std::move(order_bys)) {} + +auto TupleComparator::operator()(const SortEntry &entry_a, const SortEntry &entry_b) const -> bool { return false; } + +auto GenerateSortKey(const Tuple &tuple, const std::vector &order_bys, const Schema &schema) -> SortKey { + return {}; +} + +/** + * Above are all you need for P3. + * You can ignore the remaining part of this file until P4. + */ + auto ReconstructTuple(const Schema *schema, const Tuple &base_tuple, const TupleMeta &base_meta, const std::vector &undo_logs) -> std::optional { UNIMPLEMENTED("not implemented"); diff --git a/src/execution/executor_factory.cpp b/src/execution/executor_factory.cpp index dd98b4f09..d83bb085e 100644 --- a/src/execution/executor_factory.cpp +++ b/src/execution/executor_factory.cpp @@ -18,6 +18,7 @@ #include "execution/executors/abstract_executor.h" #include "execution/executors/aggregation_executor.h" #include "execution/executors/delete_executor.h" +#include "execution/executors/external_merge_sort_executor.h" #include "execution/executors/filter_executor.h" #include "execution/executors/hash_join_executor.h" #include "execution/executors/index_scan_executor.h" @@ -166,7 +167,7 @@ auto ExecutorFactory::CreateExecutor(ExecutorContext *exec_ctx, const AbstractPl case PlanType::Sort: { const auto *sort_plan = dynamic_cast(plan.get()); auto child = ExecutorFactory::CreateExecutor(exec_ctx, sort_plan->GetChildPlan()); - return std::make_unique(exec_ctx, sort_plan, std::move(child)); + return std::make_unique>(exec_ctx, sort_plan, std::move(child)); } // Create a new topN executor diff --git a/src/execution/external_merge_sort_executor.cpp b/src/execution/external_merge_sort_executor.cpp new file mode 100644 index 000000000..a3c8b6533 --- /dev/null +++ b/src/execution/external_merge_sort_executor.cpp @@ -0,0 +1,39 @@ +//===----------------------------------------------------------------------===// +// +// BusTub +// +// external_merge_sort_executor.cpp +// +// Identification: src/execution/external_merge_sort_executor.cpp +// +// Copyright (c) 2015-2024, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#include "execution/executors/external_merge_sort_executor.h" +#include +#include +#include +#include "common/config.h" +#include "execution/plans/sort_plan.h" + +namespace bustub { + +template +ExternalMergeSortExecutor::ExternalMergeSortExecutor(ExecutorContext *exec_ctx, const SortPlanNode *plan, + std::unique_ptr &&child_executor) + : AbstractExecutor(exec_ctx), cmp_(plan->GetOrderBy()) {} + +template +void ExternalMergeSortExecutor::Init() { + throw NotImplementedException("ExternalMergeSortExecutor is not implemented"); +} + +template +auto ExternalMergeSortExecutor::Next(Tuple *tuple, RID *rid) -> bool { + return false; +} + +template class ExternalMergeSortExecutor<2>; + +} // namespace bustub diff --git a/src/execution/fmt_impl.cpp b/src/execution/fmt_impl.cpp index ea3eb0b14..3279118b7 100644 --- a/src/execution/fmt_impl.cpp +++ b/src/execution/fmt_impl.cpp @@ -72,7 +72,9 @@ auto UpdatePlanNode::PlanNodeToString() const -> std::string { } auto SortPlanNode::PlanNodeToString() const -> std::string { - return fmt::format("Sort {{ order_bys={} }}", order_bys_); + // Note(f24): A sort plan node will be converted to an external merge sort executor in + // Fall 2024. So `ExternalMergeSort` is returned instead of `Sort`. + return fmt::format("ExternalMergeSort {{ order_bys={} }}", order_bys_); } auto LimitPlanNode::PlanNodeToString() const -> std::string { return fmt::format("Limit {{ limit={} }}", limit_); } diff --git a/src/execution/hash_join_executor.cpp b/src/execution/hash_join_executor.cpp index 032bcefe7..469098cde 100644 --- a/src/execution/hash_join_executor.cpp +++ b/src/execution/hash_join_executor.cpp @@ -19,7 +19,7 @@ HashJoinExecutor::HashJoinExecutor(ExecutorContext *exec_ctx, const HashJoinPlan std::unique_ptr &&right_child) : AbstractExecutor(exec_ctx) { if (!(plan->GetJoinType() == JoinType::LEFT || plan->GetJoinType() == JoinType::INNER)) { - // Note for 2023 Fall: You ONLY need to implement left join and inner join. + // Note for Fall 2024: You ONLY need to implement left join and inner join. throw bustub::NotImplementedException(fmt::format("join type {} not supported", plan->GetJoinType())); } } diff --git a/src/include/binder/bound_order_by.h b/src/include/binder/bound_order_by.h index ef2614e2c..3cde5c7e5 100644 --- a/src/include/binder/bound_order_by.h +++ b/src/include/binder/bound_order_by.h @@ -14,6 +14,7 @@ #include "binder/bound_expression.h" #include "common/exception.h" +#include "execution/expressions/abstract_expression.h" #include "fmt/format.h" namespace bustub { @@ -28,6 +29,8 @@ enum class OrderByType : uint8_t { DESC = 3, /**< Descending order by type. */ }; +using OrderBy = std::pair; + /** * BoundOrderBy is an item in the ORDER BY clause. */ diff --git a/src/include/catalog/catalog.h b/src/include/catalog/catalog.h index 3591356d0..80ff3e984 100644 --- a/src/include/catalog/catalog.h +++ b/src/include/catalog/catalog.h @@ -112,11 +112,12 @@ struct IndexInfo { */ class Catalog { public: - /** Indicates that an operation returning a `TableInfo*` failed */ - static constexpr TableInfo *NULL_TABLE_INFO{nullptr}; + /** Indicates that an operation returning a `std::shared_ptr` failed */ + static inline const std::shared_ptr NULL_TABLE_INFO{nullptr}; - /** Indicates that an operation returning a `IndexInfo*` failed */ - static constexpr IndexInfo *NULL_INDEX_INFO{nullptr}; + /** Indicates that an operation returning a `std::shared_ptr` failed */ + // const std::shared_ptr NULL_INDEX_INFO{nullptr}; + static inline const std::shared_ptr NULL_INDEX_INFO{nullptr}; /** * Construct a new Catalog instance. @@ -133,10 +134,10 @@ class Catalog { * @param table_name The name of the new table, note that all tables beginning with `__` are reserved for the system. * @param schema The schema of the new table * @param create_table_heap whether to create a table heap for the new table - * @return A (non-owning) pointer to the metadata for the table + * @return A shared pointer to the metadata for the table */ auto CreateTable(Transaction *txn, const std::string &table_name, const Schema &schema, bool create_table_heap = true) - -> TableInfo * { + -> std::shared_ptr { if (table_names_.count(table_name) != 0) { return NULL_TABLE_INFO; } @@ -157,15 +158,14 @@ class Catalog { const auto table_oid = next_table_oid_.fetch_add(1); // Construct the table information - auto meta = std::make_unique(schema, table_name, std::move(table), table_oid); - auto *tmp = meta.get(); + auto meta = std::make_shared(schema, table_name, std::move(table), table_oid); // Update the internal tracking mechanisms - tables_.emplace(table_oid, std::move(meta)); + tables_.emplace(table_oid, meta); table_names_.emplace(table_name, table_oid); index_names_.emplace(table_name, std::unordered_map{}); - return tmp; + return meta; } /** @@ -173,7 +173,7 @@ class Catalog { * @param table_name The name of the table * @return A (non-owning) pointer to the metadata for the table */ - auto GetTable(const std::string &table_name) const -> TableInfo * { + auto GetTable(const std::string &table_name) const -> std::shared_ptr { auto table_oid = table_names_.find(table_name); if (table_oid == table_names_.end()) { // Table not found @@ -183,21 +183,21 @@ class Catalog { auto meta = tables_.find(table_oid->second); BUSTUB_ASSERT(meta != tables_.end(), "Broken Invariant"); - return (meta->second).get(); + return meta->second; } /** * Query table metadata by OID * @param table_oid The OID of the table to query - * @return A (non-owning) pointer to the metadata for the table + * @return A shared pointer to the metadata for the table */ - auto GetTable(table_oid_t table_oid) const -> TableInfo * { + auto GetTable(table_oid_t table_oid) const -> std::shared_ptr { auto meta = tables_.find(table_oid); if (meta == tables_.end()) { return NULL_TABLE_INFO; } - return (meta->second).get(); + return meta->second; } /** @@ -210,13 +210,13 @@ class Catalog { * @param key_attrs Key attributes * @param keysize Size of the key * @param hash_function The hash function for the index - * @return A (non-owning) pointer to the metadata of the new table + * @return A shared pointer to the metadata of the new table */ template auto CreateIndex(Transaction *txn, const std::string &index_name, const std::string &table_name, const Schema &schema, const Schema &key_schema, const std::vector &key_attrs, std::size_t keysize, HashFunction hash_function, bool is_primary_key = false, - IndexType index_type = IndexType::HashTableIndex) -> IndexInfo * { + IndexType index_type = IndexType::BPlusTreeIndex) -> std::shared_ptr { // Reject the creation request for nonexistent table if (table_names_.find(table_name) == table_names_.end()) { return NULL_INDEX_INFO; @@ -257,7 +257,7 @@ class Catalog { } // Populate the index with all tuples in table heap - auto *table_meta = GetTable(table_name); + auto table_meta = GetTable(table_name); for (auto iter = table_meta->table_->MakeIterator(); !iter.IsEnd(); ++iter) { auto [meta, tuple] = iter.GetTuple(); // we have to silently ignore the error here for a lot of reasons... @@ -268,15 +268,12 @@ class Catalog { const auto index_oid = next_index_oid_.fetch_add(1); // Construct index information; IndexInfo takes ownership of the Index itself - auto index_info = std::make_unique(key_schema, index_name, std::move(index), index_oid, table_name, + auto index_info = std::make_shared(key_schema, index_name, std::move(index), index_oid, table_name, keysize, is_primary_key, index_type); - auto *tmp = index_info.get(); - // Update internal tracking - indexes_.emplace(index_oid, std::move(index_info)); + indexes_.emplace(index_oid, index_info); table_indexes.emplace(index_name, index_oid); - - return tmp; + return index_info; } /** @@ -285,7 +282,7 @@ class Catalog { * @param table_name The name of the table on which to perform query * @return A (non-owning) pointer to the metadata for the index */ - auto GetIndex(const std::string &index_name, const std::string &table_name) -> IndexInfo * { + auto GetIndex(const std::string &index_name, const std::string &table_name) -> std::shared_ptr { auto table = index_names_.find(table_name); if (table == index_names_.end()) { BUSTUB_ASSERT((table_names_.find(table_name) == table_names_.end()), "Broken Invariant"); @@ -302,7 +299,7 @@ class Catalog { auto index = indexes_.find(index_meta->second); BUSTUB_ASSERT((index != indexes_.end()), "Broken Invariant"); - return index->second.get(); + return index->second; } /** @@ -311,7 +308,7 @@ class Catalog { * @param table_oid The OID of the table on which to perform query * @return A (non-owning) pointer to the metadata for the index */ - auto GetIndex(const std::string &index_name, const table_oid_t table_oid) -> IndexInfo * { + auto GetIndex(const std::string &index_name, const table_oid_t table_oid) -> std::shared_ptr { // Locate the table metadata for the specified table OID auto table_meta = tables_.find(table_oid); if (table_meta == tables_.end()) { @@ -327,36 +324,36 @@ class Catalog { * @param index_oid The OID of the index for which to query * @return A (non-owning) pointer to the metadata for the index */ - auto GetIndex(index_oid_t index_oid) -> IndexInfo * { + auto GetIndex(index_oid_t index_oid) -> std::shared_ptr { auto index = indexes_.find(index_oid); if (index == indexes_.end()) { return NULL_INDEX_INFO; } - return index->second.get(); + return index->second; } /** * Get all of the indexes for the table identified by `table_name`. * @param table_name The name of the table for which indexes should be retrieved - * @return A vector of IndexInfo* for each index on the given table, empty vector + * @return A vector of std::shared_ptr for each index on the given table, empty vector * in the event that the table exists but no indexes have been created for it */ - auto GetTableIndexes(const std::string &table_name) const -> std::vector { + auto GetTableIndexes(const std::string &table_name) const -> std::vector> { // Ensure the table exists if (table_names_.find(table_name) == table_names_.end()) { - return std::vector{}; + return std::vector>{}; } auto table_indexes = index_names_.find(table_name); BUSTUB_ASSERT((table_indexes != index_names_.end()), "Broken Invariant"); - std::vector indexes{}; + std::vector> indexes{}; indexes.reserve(table_indexes->second.size()); for (const auto &index_meta : table_indexes->second) { auto index = indexes_.find(index_meta.second); BUSTUB_ASSERT((index != indexes_.end()), "Broken Invariant"); - indexes.push_back(index->second.get()); + indexes.push_back(index->second); } return indexes; @@ -375,12 +372,8 @@ class Catalog { [[maybe_unused]] LockManager *lock_manager_; [[maybe_unused]] LogManager *log_manager_; - /** - * Map table identifier -> table metadata. - * - * NOTE: `tables_` owns all table metadata. - */ - std::unordered_map> tables_; + /** Map table identifier -> table metadata. */ + std::unordered_map> tables_; /** Map table name -> table identifiers. */ std::unordered_map table_names_; @@ -388,12 +381,8 @@ class Catalog { /** The next table identifier to be used. */ std::atomic next_table_oid_{0}; - /** - * Map index identifier -> index metadata. - * - * NOTE: that `indexes_` owns all index metadata. - */ - std::unordered_map> indexes_; + /** Map index identifier -> index metadata. */ + std::unordered_map> indexes_; /** Map table name -> index names -> index identifiers. */ std::unordered_map> index_names_; diff --git a/src/include/catalog/table_generator.h b/src/include/catalog/table_generator.h index dc65acef4..dc61ea4ad 100644 --- a/src/include/catalog/table_generator.h +++ b/src/include/catalog/table_generator.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -101,7 +102,7 @@ class TableGenerator { : name_(name), num_rows_(num_rows), col_meta_(std::move(col_meta)) {} }; - void FillTable(TableInfo *info, TableInsertMeta *table_meta); + void FillTable(const std::shared_ptr &info, TableInsertMeta *table_meta); auto MakeValues(ColumnInsertMeta *col_meta, uint32_t count) -> std::vector; diff --git a/src/include/common/bustub_instance.h b/src/include/common/bustub_instance.h index f0f9e1617..f1ae30b5d 100644 --- a/src/include/common/bustub_instance.h +++ b/src/include/common/bustub_instance.h @@ -242,7 +242,7 @@ class BusTubInstance { auto MakeExecutorContext(Transaction *txn, bool is_modify) -> std::unique_ptr; public: - explicit BusTubInstance(const std::filesystem::path &db_file_name, size_t bpm_size = 128); + explicit BusTubInstance(const std::filesystem::path &db_file_name, size_t bpm_size = BUFFER_POOL_SIZE); explicit BusTubInstance(size_t bpm_size = 128); diff --git a/src/include/common/config.h b/src/include/common/config.h index 433399ac2..be5a655b4 100644 --- a/src/include/common/config.h +++ b/src/include/common/config.h @@ -36,7 +36,7 @@ static constexpr int INVALID_TXN_ID = -1; // invalid transaction id static constexpr int INVALID_LSN = -1; // invalid log sequence number static constexpr int BUSTUB_PAGE_SIZE = 4096; // size of a data page in byte -static constexpr int BUFFER_POOL_SIZE = 10; // size of buffer pool +static constexpr int BUFFER_POOL_SIZE = 128; // size of buffer pool static constexpr int DEFAULT_DB_IO_SIZE = 16; // starting size of file on disk static constexpr int LOG_BUFFER_SIZE = ((BUFFER_POOL_SIZE + 1) * BUSTUB_PAGE_SIZE); // size of a log buffer in byte static constexpr int BUCKET_SIZE = 50; // size of extendible hash bucket diff --git a/src/include/execution/execution_common.h b/src/include/execution/execution_common.h index 584f8afdb..d0d0a6c57 100644 --- a/src/include/execution/execution_common.h +++ b/src/include/execution/execution_common.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include +#include "binder/bound_order_by.h" #include "catalog/catalog.h" #include "catalog/schema.h" #include "concurrency/transaction.h" @@ -10,13 +12,42 @@ namespace bustub { +/** The SortKey defines a list of values that sort is based on */ +using SortKey = std::vector; +/** The SortEntry defines a key-value pairs for sorting tuples and corresponding RIDs */ +using SortEntry = std::pair; + +/** The Tuple Comparator provides a comparison function for SortEntry */ +class TupleComparator { + public: + explicit TupleComparator(std::vector order_bys); + + /** TODO(P3): Implement the comparison method */ + auto operator()(const SortEntry &entry_a, const SortEntry &entry_b) const -> bool; + + private: + std::vector order_bys_; +}; + +/** + * Generate sort key for a tuple based on the order by expressions. + * + * TODO(P3): Implement this method. + */ +auto GenerateSortKey(const Tuple &tuple, const std::vector &order_bys, const Schema &schema) -> SortKey; + +/** + * Above are all you need for P3. + * You can ignore the remaining part of this file until P4. + */ + auto ReconstructTuple(const Schema *schema, const Tuple &base_tuple, const TupleMeta &base_meta, const std::vector &undo_logs) -> std::optional; void TxnMgrDbg(const std::string &info, TransactionManager *txn_mgr, const TableInfo *table_info, TableHeap *table_heap); -// Add new functions as needed... You are likely need to define some more functions. +// TODO(P4): Add new functions as needed... You are likely need to define some more functions. // // To give you a sense of what can be shared across executors / transaction manager, here are the // list of helper function names that we defined in the reference solution. You should come up with diff --git a/src/include/execution/executors/external_merge_sort_executor.h b/src/include/execution/executors/external_merge_sort_executor.h new file mode 100644 index 000000000..71b94f9fc --- /dev/null +++ b/src/include/execution/executors/external_merge_sort_executor.h @@ -0,0 +1,168 @@ +//===----------------------------------------------------------------------===// +// +// BusTub +// +// external_merge_sort_executor.h +// +// Identification: src/include/execution/executors/external_merge_sort_executor.h +// +// Copyright (c) 2015-2024, Carnegie Mellon University Database Group +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include +#include +#include +#include +#include "common/config.h" +#include "common/macros.h" +#include "execution/execution_common.h" +#include "execution/executors/abstract_executor.h" +#include "execution/plans/sort_plan.h" +#include "storage/table/tuple.h" + +namespace bustub { + +/** + * Page to hold the intermediate data for external merge sort. + * + * Only fixed-length data will be supported in Fall 2024. + */ +class SortPage { + public: + /** + * TODO: Define and implement the methods for reading data from and writing data to the sort + * page. Feel free to add other helper methods. + */ + private: + /** + * TODO: Define the private members. You may want to have some necessary metadata for + * the sort page before the start of the actual data. + */ +}; + +/** + * A data structure that holds the sorted tuples as a run during external merge sort. + * Tuples might be stored in multiple pages, and tuples are ordered both within one page + * and across pages. + */ +class MergeSortRun { + public: + MergeSortRun() = default; + MergeSortRun(std::vector pages, BufferPoolManager *bpm) : pages_(std::move(pages)), bpm_(bpm) {} + + auto GetPageCount() -> size_t { return pages_.size(); } + + /** Iterator for iterating on the sorted tuples in one run. */ + class Iterator { + friend class MergeSortRun; + + public: + Iterator() = default; + + /** + * Advance the iterator to the next tuple. If the current sort page is exhausted, move to the + * next sort page. + * + * TODO: Implement this method. + */ + auto operator++() -> Iterator & { return *this; } + + /** + * Dereference the iterator to get the current tuple in the sorted run that the iterator is + * pointing to. + * + * TODO: Implement this method. + */ + auto operator*() -> Tuple { return {}; } + + /** + * Checks whether two iterators are pointing to the same tuple in the same sorted run. + * + * TODO: Implement this method. + */ + auto operator==(const Iterator &other) const -> bool { return false; } + + /** + * Checks whether two iterators are pointing to different tuples in a sorted run or iterating + * on different sorted runs. + * + * TODO: Implement this method. + */ + auto operator!=(const Iterator &other) const -> bool { return false; } + + private: + explicit Iterator(const MergeSortRun *run) : run_(run) {} + + /** The sorted run that the iterator is iterating on. */ + [[maybe_unused]] const MergeSortRun *run_; + + /** + * TODO: Add your own private members here. You may want something to record your current + * position in the sorted run. Also feel free to add additional constructors to initialize + * your private members. + */ + }; + + /** + * Get an iterator pointing to the beginning of the sorted run, i.e. the first tuple. + * + * TODO: Implement this method. + */ + auto Begin() -> Iterator { return {}; } + + /** + * Get an iterator pointing to the end of the sorted run, i.e. the position after the last tuple. + * + * TODO: Implement this method. + */ + auto End() -> Iterator { return {}; } + + private: + /** The page IDs of the sort pages that store the sorted tuples. */ + std::vector pages_; + /** + * The buffer pool manager used to read sort pages. The buffer pool manager is responsible for + * deleting the sort pages when they are no longer needed. + */ + [[maybe_unused]] BufferPoolManager *bpm_; +}; + +/** + * ExternalMergeSortExecutor executes an external merge sort. + * + * In Fall 2024, only 2-way external merge sort is required. + */ +template +class ExternalMergeSortExecutor : public AbstractExecutor { + public: + ExternalMergeSortExecutor(ExecutorContext *exec_ctx, const SortPlanNode *plan, + std::unique_ptr &&child_executor); + + /** Initialize the external merge sort */ + void Init() override; + + /** + * Yield the next tuple from the external merge sort. + * @param[out] tuple The next tuple produced by the external merge sort. + * @param[out] rid The next tuple RID produced by the external merge sort. + * @return `true` if a tuple was produced, `false` if there are no more tuples + */ + auto Next(Tuple *tuple, RID *rid) -> bool override; + + /** @return The output schema for the external merge sort */ + auto GetOutputSchema() const -> const Schema & override { return plan_->OutputSchema(); } + + private: + /** The sort plan node to be executed */ + const SortPlanNode *plan_; + + /** Compares tuples based on the order-bys */ + TupleComparator cmp_; + + /** TODO: You will want to add your own private members here. */ +}; + +} // namespace bustub diff --git a/src/include/execution/plans/sort_plan.h b/src/include/execution/plans/sort_plan.h index fc33053c8..902d00294 100644 --- a/src/include/execution/plans/sort_plan.h +++ b/src/include/execution/plans/sort_plan.h @@ -36,8 +36,7 @@ class SortPlanNode : public AbstractPlanNode { * @param child The child plan node * @param order_bys The sort expressions and their order by types. */ - SortPlanNode(SchemaRef output, AbstractPlanNodeRef child, - std::vector> order_bys) + SortPlanNode(SchemaRef output, AbstractPlanNodeRef child, std::vector order_bys) : AbstractPlanNode(std::move(output), {std::move(child)}), order_bys_(std::move(order_bys)) {} /** @return The type of the plan node */ @@ -50,11 +49,11 @@ class SortPlanNode : public AbstractPlanNode { } /** @return Get sort by expressions */ - auto GetOrderBy() const -> const std::vector> & { return order_bys_; } + auto GetOrderBy() const -> const std::vector & { return order_bys_; } BUSTUB_PLAN_NODE_CLONE_WITH_CHILDREN(SortPlanNode); - std::vector> order_bys_; + std::vector order_bys_; protected: auto PlanNodeToString() const -> std::string override; diff --git a/src/include/execution/plans/topn_per_group_plan.h b/src/include/execution/plans/topn_per_group_plan.h index 29c4c6144..57f543320 100644 --- a/src/include/execution/plans/topn_per_group_plan.h +++ b/src/include/execution/plans/topn_per_group_plan.h @@ -38,7 +38,7 @@ class TopNPerGroupPlanNode : public AbstractPlanNode { * @param n Retain n elements. */ TopNPerGroupPlanNode(SchemaRef output, AbstractPlanNodeRef child, std::vector group_bys, - std::vector> order_bys, std::size_t n) + std::vector order_bys, std::size_t n) : AbstractPlanNode(std::move(output), {std::move(child)}), order_bys_(std::move(order_bys)), group_bys_(std::move(group_bys)), @@ -51,7 +51,7 @@ class TopNPerGroupPlanNode : public AbstractPlanNode { auto GetN() const -> size_t { return n_; } /** @return Get order by expressions */ - auto GetOrderBy() const -> const std::vector> & { return order_bys_; } + auto GetOrderBy() const -> const std::vector & { return order_bys_; } /** @return Get group by expressions */ auto GetGroupBy() const -> const std::vector & { return group_bys_; } @@ -64,7 +64,7 @@ class TopNPerGroupPlanNode : public AbstractPlanNode { BUSTUB_PLAN_NODE_CLONE_WITH_CHILDREN(TopNPerGroupPlanNode); - std::vector> order_bys_; + std::vector order_bys_; std::vector group_bys_; std::size_t n_; diff --git a/src/include/execution/plans/topn_plan.h b/src/include/execution/plans/topn_plan.h index 6adc04248..04a3079a8 100644 --- a/src/include/execution/plans/topn_plan.h +++ b/src/include/execution/plans/topn_plan.h @@ -37,8 +37,7 @@ class TopNPlanNode : public AbstractPlanNode { * @param order_bys The sort expressions and their order by types. * @param n Retain n elements. */ - TopNPlanNode(SchemaRef output, AbstractPlanNodeRef child, - std::vector> order_bys, std::size_t n) + TopNPlanNode(SchemaRef output, AbstractPlanNodeRef child, std::vector order_bys, std::size_t n) : AbstractPlanNode(std::move(output), {std::move(child)}), order_bys_(std::move(order_bys)), n_{n} {} /** @return The type of the plan node */ @@ -48,7 +47,7 @@ class TopNPlanNode : public AbstractPlanNode { auto GetN() const -> size_t { return n_; } /** @return Get order by expressions */ - auto GetOrderBy() const -> const std::vector> & { return order_bys_; } + auto GetOrderBy() const -> const std::vector & { return order_bys_; } /** @return The child plan node */ auto GetChildPlan() const -> AbstractPlanNodeRef { @@ -58,7 +57,7 @@ class TopNPlanNode : public AbstractPlanNode { BUSTUB_PLAN_NODE_CLONE_WITH_CHILDREN(TopNPlanNode); - std::vector> order_bys_; + std::vector order_bys_; std::size_t n_; protected: diff --git a/src/include/execution/plans/window_plan.h b/src/include/execution/plans/window_plan.h index b67cf2380..9790db3cf 100644 --- a/src/include/execution/plans/window_plan.h +++ b/src/include/execution/plans/window_plan.h @@ -61,8 +61,7 @@ class WindowFunctionPlanNode : public AbstractPlanNode { WindowFunctionPlanNode(SchemaRef output_schema, AbstractPlanNodeRef child, std::vector window_func_indexes, std::vector columns, std::vector> partition_bys, - std::vector>> order_bys, - std::vector functions, + std::vector> order_bys, std::vector functions, std::vector window_func_types) : AbstractPlanNode(std::move(output_schema), {std::move(child)}), columns_(std::move(columns)) { for (uint32_t i = 0; i < window_func_indexes.size(); i++) { @@ -88,7 +87,7 @@ class WindowFunctionPlanNode : public AbstractPlanNode { AbstractExpressionRef function_; WindowFunctionType type_; std::vector partition_by_; - std::vector> order_by_; + std::vector order_by_; }; /** all columns expressions */ diff --git a/src/include/storage/table/tuple.h b/src/include/storage/table/tuple.h index 2d44cc0bf..4f22d21c1 100644 --- a/src/include/storage/table/tuple.h +++ b/src/include/storage/table/tuple.h @@ -65,6 +65,9 @@ class Tuple { // constructor for creating a new tuple based on input value Tuple(std::vector values, const Schema *schema); + // constructor for creating a new tuple by copying fron existing bytes + Tuple(RID rid, const char *data, uint32_t size); + Tuple(const Tuple &other) = default; // move constructor diff --git a/src/optimizer/nlj_as_index_join.cpp b/src/optimizer/nlj_as_index_join.cpp index b0413ab47..d209deff9 100644 --- a/src/optimizer/nlj_as_index_join.cpp +++ b/src/optimizer/nlj_as_index_join.cpp @@ -24,7 +24,7 @@ namespace bustub { auto Optimizer::MatchIndex(const std::string &table_name, uint32_t index_key_idx) -> std::optional> { const auto key_attrs = std::vector{index_key_idx}; - for (const auto *index_info : catalog_.GetTableIndexes(table_name)) { + for (const auto &index_info : catalog_.GetTableIndexes(table_name)) { if (key_attrs == index_info->index_->GetKeyAttrs()) { return std::make_optional(std::make_tuple(index_info->index_oid_, index_info->name_)); } diff --git a/src/optimizer/optimizer.cpp b/src/optimizer/optimizer.cpp index 7a5a865c7..cc6011401 100644 --- a/src/optimizer/optimizer.cpp +++ b/src/optimizer/optimizer.cpp @@ -11,8 +11,8 @@ auto Optimizer::Optimize(const AbstractPlanNodeRef &plan) -> AbstractPlanNodeRef auto p = plan; p = OptimizeMergeProjection(p); p = OptimizeMergeFilterNLJ(p); + p = OptimizeNLJAsIndexJoin(p); p = OptimizeOrderByAsIndexScan(p); - p = OptimizeSortLimitAsTopN(p); p = OptimizeMergeFilterScan(p); p = OptimizeSeqScanAsIndexScan(p); return p; diff --git a/src/optimizer/order_by_index_scan.cpp b/src/optimizer/order_by_index_scan.cpp index 24d739529..076d0cc46 100644 --- a/src/optimizer/order_by_index_scan.cpp +++ b/src/optimizer/order_by_index_scan.cpp @@ -54,10 +54,10 @@ auto Optimizer::OptimizeOrderByAsIndexScan(const AbstractPlanNodeRef &plan) -> A if (child_plan->GetType() == PlanType::SeqScan) { const auto &seq_scan = dynamic_cast(*child_plan); - const auto *table_info = catalog_.GetTable(seq_scan.GetTableOid()); + const auto table_info = catalog_.GetTable(seq_scan.GetTableOid()); const auto indices = catalog_.GetTableIndexes(table_info->name_); - for (const auto *index : indices) { + for (const auto &index : indices) { const auto &columns = index->index_->GetKeyAttrs(); if (order_by_column_ids == columns) { return std::make_shared(optimized_plan->output_schema_, table_info->oid_, diff --git a/src/planner/plan_select.cpp b/src/planner/plan_select.cpp index c8829ae94..67c32eec6 100644 --- a/src/planner/plan_select.cpp +++ b/src/planner/plan_select.cpp @@ -117,7 +117,7 @@ auto Planner::PlanSelect(const SelectStatement &statement) -> AbstractPlanNodeRe // Plan ORDER BY if (!statement.sort_.empty()) { - std::vector> order_bys; + std::vector order_bys; for (const auto &order_by : statement.sort_) { auto [_, expr] = PlanExpression(*order_by->expr_, {plan}); auto abstract_expr = std::move(expr); diff --git a/src/planner/plan_window_function.cpp b/src/planner/plan_window_function.cpp index 2b8e866a0..856082077 100644 --- a/src/planner/plan_window_function.cpp +++ b/src/planner/plan_window_function.cpp @@ -28,14 +28,13 @@ namespace bustub { // TODO(chi): clang-tidy on macOS will suggest changing it to const reference. Looks like a bug. -void CheckOrderByCompatible( - const std::vector>> &order_by_exprs) { +void CheckOrderByCompatible(const std::vector> &order_by_exprs) { if (order_by_exprs.empty()) { // either or window functions not having order by clause return; } // or all order by clause are the same - std::vector> first_order_by = order_by_exprs[0]; + std::vector first_order_by = order_by_exprs[0]; for (auto &order_by : order_by_exprs) { if (order_by.size() != first_order_by.size()) { throw Exception("order by clause of window functions are not compatible"); @@ -63,7 +62,7 @@ auto Planner::PlanSelectWindow(const SelectStatement &statement, AbstractPlanNod std::vector window_func_indexes; std::vector window_func_types; std::vector> partition_by_exprs; - std::vector>> order_by_exprs; + std::vector> order_by_exprs; std::vector arg_exprs; for (uint32_t i = 0; i < statement.select_list_.size(); i++) { @@ -111,7 +110,7 @@ auto Planner::PlanSelectWindow(const SelectStatement &statement, AbstractPlanNod throw Exception("order by clause is mandatory for rank function"); } - std::vector> order_by; + std::vector order_by; for (const auto &item : window_call.order_bys_) { auto [_, expr] = PlanExpression(*item->expr_, {child}); auto abstract_expr = std::move(expr); diff --git a/src/storage/table/tuple.cpp b/src/storage/table/tuple.cpp index e5ea0fa8c..3694847aa 100644 --- a/src/storage/table/tuple.cpp +++ b/src/storage/table/tuple.cpp @@ -12,6 +12,7 @@ #include #include +#include #include #include #include @@ -60,6 +61,12 @@ Tuple::Tuple(std::vector values, const Schema *schema) { } } +Tuple::Tuple(RID rid, const char *data, uint32_t size) { + rid_ = rid; + data_.resize(size); + memcpy(data_.data(), data, size); +} + auto Tuple::GetValue(const Schema *schema, const uint32_t column_idx) const -> Value { assert(schema); const TypeId column_type = schema->GetColumn(column_idx).GetType(); diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt index 72648a9d1..81609100d 100644 --- a/test/CMakeLists.txt +++ b/test/CMakeLists.txt @@ -56,7 +56,7 @@ set(BUSTUB_SLT_SOURCES "${PROJECT_SOURCE_DIR}/test/sql/p3.02-insert.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.03-update.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.04-delete.slt" - "${PROJECT_SOURCE_DIR}/test/sql/p3.05-index-scan.slt" + "${PROJECT_SOURCE_DIR}/test/sql/p3.05-index-scan-btree.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.06-empty-table.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.07-simple-agg.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.08-group-agg-1.slt" @@ -72,7 +72,7 @@ set(BUSTUB_SLT_SOURCES "${PROJECT_SOURCE_DIR}/test/sql/p3.18-integration-1.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.19-integration-2.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.20-window-function.slt" - "${PROJECT_SOURCE_DIR}/test/sql/p3.leaderboard-q1-window.slt" + "${PROJECT_SOURCE_DIR}/test/sql/p3.leaderboard-q1.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.leaderboard-q2.slt" "${PROJECT_SOURCE_DIR}/test/sql/p3.leaderboard-q3.slt" ) diff --git a/test/txn/txn_abort_serializable_test.cpp b/test/txn/txn_abort_serializable_test.cpp index 7e4325e5a..2969f3159 100644 --- a/test/txn/txn_abort_serializable_test.cpp +++ b/test/txn/txn_abort_serializable_test.cpp @@ -24,7 +24,7 @@ TEST(TxnBonusTest, DISABLED_SerializableTest) { // NOLINT auto txn_read = BeginTxnSerializable(*bustub, "txn_read"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET a = 0 WHERE a = 1")); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET a = 1 WHERE a = 0")); - TxnMgrDbg("after two updates", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after two updates", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_read, ExecuteTxn(*bustub, _var, _txn, "SELECT * FROM maintable WHERE a = 0")); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); WithTxn(txn3, CommitTxn(*bustub, _var, _txn, EXPECT_FAIL)); @@ -56,7 +56,7 @@ TEST(TxnBonusTest, DISABLED_ConcurrentSerializableTest) { // NOLINT auto txn3 = BeginTxnSerializable(*bustub, "txn3"); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET a = 1 WHERE a = 0")); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET a = 0 WHERE a = 1")); - TxnMgrDbg("after two updates", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after two updates", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); std::vector commit_threads; const int thread_cnt = 2; @@ -97,7 +97,7 @@ TEST(TxnBonusTest, DISABLED_AbortTest) { // NOLINT auto txn1 = BeginTxn(*bustub, "txn1"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1, 233), (2, 2333)")); WithTxn(txn1, AbortTxn(*bustub, _var, _txn)); - TxnMgrDbg("after abort", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after abort", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn2 = BeginTxn(*bustub, "txn2"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1, 2333), (2, 23333), (3, 233)")); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM maintable", @@ -106,9 +106,9 @@ TEST(TxnBonusTest, DISABLED_AbortTest) { // NOLINT {2, 23333}, {3, 233}, })); - TxnMgrDbg("after insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn3 = BeginTxn(*bustub, "txn3"); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM maintable", IntResult{ @@ -116,7 +116,7 @@ TEST(TxnBonusTest, DISABLED_AbortTest) { // NOLINT {2, 23333}, {3, 233}, })); - TableHeapEntryNoMoreThan(*bustub, table_info, 3); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 3); // test continues on Gradescope... } } diff --git a/test/txn/txn_executor_test.cpp b/test/txn/txn_executor_test.cpp index d9f921007..7afe56acb 100644 --- a/test/txn/txn_executor_test.cpp +++ b/test/txn/txn_executor_test.cpp @@ -17,7 +17,7 @@ TEST(TxnExecutorTest, DISABLED_InsertTest) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1)")); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2)")); - TxnMgrDbg("after insertion", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insertion", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); const std::string query = "SELECT a FROM maintable"; fmt::println(stderr, "A: check scan txn1"); @@ -40,7 +40,7 @@ TEST(TxnExecutorTest, DISABLED_InsertCommitTest) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1)")); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2)")); - TxnMgrDbg("after insertion", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insertion", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); const std::string query = "SELECT a FROM maintable"; fmt::println(stderr, "A: check scan txn1"); @@ -48,7 +48,7 @@ TEST(TxnExecutorTest, DISABLED_InsertCommitTest) { // NOLINT fmt::println(stderr, "B: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2}})); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit txn1", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit txn1", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn_ref = BeginTxn(*bustub, "txn_ref"); @@ -58,14 +58,14 @@ TEST(TxnExecutorTest, DISABLED_InsertCommitTest) { // NOLINT fmt::println(stderr, "D: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2}})); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (3)")); - TxnMgrDbg("after insert into txn3", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insert into txn3", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "E: check scan txn3"); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {3}})); fmt::println(stderr, "F: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2}})); WithTxn(txn3, CommitTxn(*bustub, _var, _txn)); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit txn2", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit txn2", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn4 = BeginTxn(*bustub, "txn4"); fmt::println(stderr, "G: check scan txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}, {3}})); @@ -83,18 +83,18 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteTest) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2)")); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (3)")); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 3")); - TxnMgrDbg("after 3 insert + 1 delete", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after 3 insert + 1 delete", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "A: check scan txn1"); const auto query = "SELECT a FROM maintable"; WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn_ref = BeginTxn(*bustub, "txn_ref"); auto txn2 = BeginTxn(*bustub, "txn2"); fmt::println(stderr, "B: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 2")); - TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}})); auto txn4 = BeginTxn(*bustub, "txn4"); fmt::println(stderr, "C: check scan txn4"); @@ -103,7 +103,7 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteTest) { // NOLINT WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (5)")); WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (6)")); WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 6")); - TxnMgrDbg("after txn4 modification", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn4 modification", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "D: check scan txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}, {4}, {5}})); fmt::println(stderr, "E: check scan txn2"); @@ -112,7 +112,7 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteTest) { // NOLINT WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}})); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); WithTxn(txn4, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn5 = BeginTxn(*bustub, "txn5"); fmt::println(stderr, "F: check scan txn5"); WithTxn(txn5, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {4}, {5}})); @@ -132,24 +132,24 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteConflictTest) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2)")); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (3)")); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 3")); - TxnMgrDbg("after 3 insert + 1 delete", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after 3 insert + 1 delete", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "A: check scan txn1"); const auto query = "SELECT a FROM maintable"; WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn2 = BeginTxn(*bustub, "txn2"); fmt::println(stderr, "B: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 2")); - TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}})); auto txn3 = BeginTxn(*bustub, "txn3"); fmt::println(stderr, "C: check scan txn3"); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); fmt::println(stderr, "D: taint txn3"); WithTxn(txn3, ExecuteTxnTainted(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 2")); - TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn4 = BeginTxn(*bustub, "txn4"); fmt::println(stderr, "E: check scan txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}})); @@ -157,7 +157,7 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteConflictTest) { // NOLINT WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (5)")); WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (6)")); WithTxn(txn4, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 6")); - TxnMgrDbg("after txn4 modification", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn4 modification", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "F: check scan txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {2}, {4}, {5}})); fmt::println(stderr, "G: check scan txn2"); @@ -166,16 +166,16 @@ TEST(TxnExecutorTest, DISABLED_InsertDeleteConflictTest) { // NOLINT WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}})); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); WithTxn(txn4, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn5 = BeginTxn(*bustub, "txn5"); fmt::println(stderr, "H: check scan txn5"); WithTxn(txn5, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1}, {4}, {5}})); fmt::println(stderr, "I: commit txn 6"); auto txn6 = BeginTxn(*bustub, "txn6"); WithTxn(txn6, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 5")); - TxnMgrDbg("after txn6 deletes", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn6 deletes", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn6, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn6 commits", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn6 commits", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "J: taint txn5"); WithTxn(txn5, ExecuteTxnTainted(*bustub, _var, _txn, "DELETE FROM maintable WHERE a = 5")); auto txn7 = BeginTxn(*bustub, "txn7"); @@ -195,53 +195,53 @@ TEST(TxnExecutorTest, DISABLED_UpdateTest1) { // NOLINT auto txn_ref = BeginTxn(*bustub, "txn_ref"); auto txn1 = BeginTxn(*bustub, "txn1"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO table1 VALUES (1, 1, 1)")); - TxnMgrDbg("after insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); const std::string query = "SELECT * FROM table1"; fmt::println(stderr, "A: 1st update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET b = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 2, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "B: 2nd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET b = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "C1: 3rd update, not real update..."); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET a = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "C2: the real 3rd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET a = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "D: 4th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET b = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 1, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "E: 5th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET a = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{3, 1, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "F: 6th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET a = 4, b = 4, c = 4")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{4, 4, 4}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); fmt::println(stderr, "G: delete"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE from table1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn1, CheckUndoLogNum(*bustub, _var, _txn, 0)); @@ -250,7 +250,7 @@ TEST(TxnExecutorTest, DISABLED_UpdateTest1) { // NOLINT fmt::println(stderr, "H: check scan txn2"); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TableHeapEntryNoMoreThan(*bustub, table_info, 1); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 1); } TEST(TxnExecutorTest, DISABLED_UpdateTest2) { // NOLINT @@ -262,55 +262,55 @@ TEST(TxnExecutorTest, DISABLED_UpdateTest2) { // NOLINT auto txn0 = BeginTxn(*bustub, "txn0"); WithTxn(txn0, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO table2 VALUES (1, 1, 1)")); WithTxn(txn0, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after insert and commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insert and commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn1 = BeginTxn(*bustub, "txn1"); auto txn_ref = BeginTxn(*bustub, "txn_ref"); const std::string query = "SELECT * FROM table2"; fmt::println(stderr, "A: 1st update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 2, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "B: 2nd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "C1: 3rd update, not real update..."); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "C2: the real 3rd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 3, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "D: 4th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 1, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "E: 5th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{3, 1, 1}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "F: 6th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 4, b = 4, c = 4")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{4, 4, 4}})); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 3)); fmt::println(stderr, "G: delete"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE from table2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 3)); @@ -322,7 +322,7 @@ TEST(TxnExecutorTest, DISABLED_UpdateTest2) { // NOLINT WithTxn(txn_ref, CommitTxn(*bustub, _var, _txn)); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TableHeapEntryNoMoreThan(*bustub, table_info, 1); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 1); } TEST(TxnExecutorTest, DISABLED_UpdateTestWithUndoLog) { // NOLINT @@ -338,62 +338,62 @@ TEST(TxnExecutorTest, DISABLED_UpdateTestWithUndoLog) { // NOLINT auto txn01 = BeginTxn(*bustub, "txn01"); WithTxn(txn01, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 1, b = 1, c = 1")); WithTxn(txn01, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after insert, update, and commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after insert, update, and commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn1 = BeginTxn(*bustub, "txn1"); auto txn_ref_1 = BeginTxn(*bustub, "txn_ref_1"); const std::string query = "SELECT * FROM table2"; fmt::println(stderr, "A: 1st update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 2, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "B: 2nd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "C1: 3rd update, not real update..."); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 3, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 1)); fmt::println(stderr, "C2: the real 3rd update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 3, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "D: 4th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET b = 1")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 1, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "E: 5th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 3")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{3, 1, 1}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 2)); fmt::println(stderr, "F: 6th update"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table2 SET a = 4, b = 4, c = 4")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{4, 4, 4}})); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); WithTxn(txn1, CheckUndoLogColumn(*bustub, _var, _txn, 3)); fmt::println(stderr, "G: delete"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE from table2")); - TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn_ref_0, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0, 0}})); WithTxn(txn_ref_1, QueryShowResult(*bustub, _var, _txn, query, IntResult{{1, 1, 1}})); @@ -408,7 +408,7 @@ TEST(TxnExecutorTest, DISABLED_UpdateTestWithUndoLog) { // NOLINT WithTxn(txn_ref_1, CommitTxn(*bustub, _var, _txn)); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TableHeapEntryNoMoreThan(*bustub, table_info, 1); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 1); } TEST(TxnExecutorTest, DISABLED_UpdateConflict) { // NOLINT @@ -421,17 +421,17 @@ TEST(TxnExecutorTest, DISABLED_UpdateConflict) { // NOLINT WithTxn(txn0, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO table1 VALUES (0, 0, 0)")); WithTxn(txn0, CommitTxn(*bustub, _var, _txn)); auto txn_ref = BeginTxn(*bustub, "txn_ref"); - TxnMgrDbg("after initialize", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after initialize", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn1 = BeginTxn(*bustub, "txn1"); auto txn2 = BeginTxn(*bustub, "txn2"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET a = 1")); - TxnMgrDbg("after 1st update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after 1st update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, ExecuteTxnTainted(*bustub, _var, _txn, "UPDATE table1 SET b = 2")); - TxnMgrDbg("after txn tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM table1", IntResult{{0, 0, 0}})); - TableHeapEntryNoMoreThan(*bustub, table_info, 1); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 1); } { fmt::println(stderr, "--- UpdateConflict2: complex case with version chain ---"); @@ -441,7 +441,7 @@ TEST(TxnExecutorTest, DISABLED_UpdateConflict) { // NOLINT auto txn0 = BeginTxn(*bustub, "txn0"); WithTxn(txn0, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO table1 VALUES (0, 0, 0), (1, 1, 1)")); WithTxn(txn0, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after initialize", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after initialize", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn1 = BeginTxn(*bustub, "txn1"); auto txn2 = BeginTxn(*bustub, "txn2"); auto txn3 = BeginTxn(*bustub, "txn3"); @@ -450,17 +450,17 @@ TEST(TxnExecutorTest, DISABLED_UpdateConflict) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET b = 233 WHERE a = 0")); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "UPDATE table1 SET b = 2333 WHERE a = 1")); - TxnMgrDbg("after updates", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after updates", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn3, ExecuteTxnTainted(*bustub, _var, _txn, "UPDATE table1 SET b = 2 WHERE a = 0")); - TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn4, ExecuteTxnTainted(*bustub, _var, _txn, "UPDATE table1 SET b = 2 WHERE a = 1")); - TxnMgrDbg("after txn4 tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn4 tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_ref, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM table1", IntResult{{0, 0, 0}, {1, 1, 1}})); auto txn5 = BeginTxn(*bustub, "txn5"); WithTxn(txn5, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM table1", IntResult{{0, 233, 0}, {1, 2333, 1}})); - TableHeapEntryNoMoreThan(*bustub, table_info, 2); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 2); } } @@ -504,7 +504,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT auto txn_watermark_at_3 = BeginTxn(*bustub, "txn_watermark_at_3"); auto txn_watermark_at_3_id = txn_watermark_at_3->GetTransactionId(); BumpCommitTs(*bustub, 2); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn_watermark_at_1, @@ -516,11 +516,11 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT fmt::println(stderr, "A: first GC"); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "B: second GC"); GarbageCollection(*bustub); TxnMgrDbg("after garbage collection (yes, we call it twice without doing anything...)", bustub->txn_manager_.get(), - table_info, table_info->table_.get()); + table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnExists(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnExists(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -540,7 +540,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT fmt::println(stderr, "C: 3rd GC"); WithTxn(txn_watermark_at_0, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnExists(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -559,7 +559,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT fmt::println(stderr, "D: 4th GC"); WithTxn(txn_watermark_at_1, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -576,7 +576,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT fmt::println(stderr, "E: 5th GC"); WithTxn(txn_watermark_at_2, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnGCed(*bustub, _var, txn_watermark_at_2_id)); @@ -591,7 +591,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollection) { // NOLINT fmt::println(stderr, "F: 6th GC"); WithTxn(txn_watermark_at_3, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnGCed(*bustub, _var, txn_watermark_at_2_id)); @@ -646,7 +646,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT auto txn_watermark_at_3 = BeginTxn(*bustub, "txn_watermark_at_3"); auto txn_watermark_at_3_id = txn_watermark_at_3->GetTransactionId(); BumpCommitTs(*bustub, 2); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, QueryShowResult(*bustub, _var, _txn, query, empty_table)); WithTxn(txn_watermark_at_1, @@ -658,11 +658,11 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT fmt::println(stderr, "A: first GC"); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); fmt::println(stderr, "B: second GC"); GarbageCollection(*bustub); TxnMgrDbg("after garbage collection (yes, we call it twice without doing anything...)", bustub->txn_manager_.get(), - table_info, table_info->table_.get()); + table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnExists(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnExists(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -684,9 +684,9 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT WithTxn(txn5, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM table1 WHERE a = 12")); WithTxn(txn5, ExecuteTxnTainted(*bustub, _var, _txn, "DELETE FROM table1 WHERE a = 11")); WithTxn(txn6, ExecuteTxnTainted(*bustub, _var, _txn, "DELETE FROM table1 WHERE a = 11")); - TxnMgrDbg("after txn5 + txn6 tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn5 + txn6 tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnExists(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnExists(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -708,7 +708,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT fmt::println(stderr, "D: 4th GC"); WithTxn(txn_watermark_at_0, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnExists(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -729,7 +729,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT fmt::println(stderr, "E: 5th GC"); WithTxn(txn_watermark_at_1, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnExists(*bustub, _var, txn_watermark_at_2_id)); @@ -748,7 +748,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT fmt::println(stderr, "F: 6th GC"); WithTxn(txn_watermark_at_2, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnGCed(*bustub, _var, txn_watermark_at_2_id)); @@ -765,7 +765,7 @@ TEST(TxnExecutorTest, DISABLED_GarbageCollectionWithTainted) { // NOLINT fmt::println(stderr, "G: 7th GC"); WithTxn(txn_watermark_at_3, CommitTxn(*bustub, _var, _txn)); GarbageCollection(*bustub); - TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after garbage collection", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn_watermark_at_0, EnsureTxnGCed(*bustub, _var, txn_watermark_at_0_id)); WithTxn(txn_watermark_at_1, EnsureTxnGCed(*bustub, _var, txn_watermark_at_1_id)); WithTxn(txn_watermark_at_2, EnsureTxnGCed(*bustub, _var, txn_watermark_at_2_id)); diff --git a/test/txn/txn_index_concurrent_test.cpp b/test/txn/txn_index_concurrent_test.cpp index 90213d2f0..384ad08dc 100644 --- a/test/txn/txn_index_concurrent_test.cpp +++ b/test/txn/txn_index_concurrent_test.cpp @@ -86,7 +86,7 @@ TEST(TxnIndexTest, DISABLED_IndexConcurrentInsertTest) { // NOLINT } auto query_txn = BeginTxn(*bustub, "query_txn"); WithTxn(query_txn, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM maintable", expected_rows)); - auto entry = TableHeapEntry(*bustub, bustub->catalog_->GetTable("maintable")); + auto entry = TableHeapEntry(*bustub, bustub->catalog_->GetTable("maintable").get()); fmt::println(stderr, "{} entries in the table heap", entry); if (n == trials - 1) { SimpleStreamWriter writer(std::cerr); @@ -126,7 +126,7 @@ TEST(TxnIndexTest, DISABLED_IndexConcurrentUpdateTest) { // NOLINT const int thread_cnt = 8; const int number_cnt = 20; Execute(*bustub, generate_insert_sql(number_cnt), false); - TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable"), number_cnt); + TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable").get(), number_cnt); update_threads.reserve(thread_cnt); std::map> operation_result; std::mutex result_mutex; @@ -183,7 +183,7 @@ TEST(TxnIndexTest, DISABLED_IndexConcurrentUpdateTest) { // NOLINT } auto query_txn = BeginTxn(*bustub, "query_txn"); WithTxn(query_txn, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM maintable", expected_rows)); - TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable"), number_cnt); + TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable").get(), number_cnt); if (n == trials - 1 || n == trials - 2) { SimpleStreamWriter writer(std::cerr); fmt::println(stderr, "--- the following data might be manually inspected by TAs ---"); @@ -214,7 +214,7 @@ TEST(TxnIndexTest, DISABLED_IndexConcurrentUpdateAbortTest) { // NOLINT Execute(*bustub, "CREATE TABLE maintable(a int primary key, b int)"); std::vector update_threads; Execute(*bustub, generate_insert_sql(number_cnt), false); - TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable"), number_cnt); + TableHeapEntryNoMoreThan(*bustub, bustub->catalog_->GetTable("maintable").get(), number_cnt); update_threads.reserve(thread_cnt); std::map> operation_result; std::mutex result_mutex; @@ -273,10 +273,10 @@ TEST(TxnIndexTest, DISABLED_IndexConcurrentUpdateAbortTest) { // NOLINT std::terminate(); } } - auto *table_info = bustub->catalog_->GetTable("maintable"); + auto table_info = bustub->catalog_->GetTable("maintable"); auto query_txn = BeginTxn(*bustub, "query_txn"); WithTxn(query_txn, QueryShowResult(*bustub, _var, _txn, "SELECT * FROM maintable", expected_rows)); - TableHeapEntryNoMoreThan(*bustub, table_info, number_cnt); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), number_cnt); if (n >= trials - 2) { SimpleStreamWriter writer(std::cerr); fmt::println(stderr, "--- the following data might be manually inspected by TAs ---"); diff --git a/test/txn/txn_index_test.cpp b/test/txn/txn_index_test.cpp index b04bf94f6..162685589 100644 --- a/test/txn/txn_index_test.cpp +++ b/test/txn/txn_index_test.cpp @@ -22,37 +22,37 @@ TEST(TxnIndexTest, DISABLED_IndexInsertTest) { // NOLINT auto table_info = bustub->catalog_->GetTable("maintable"); auto txn1 = BeginTxn(*bustub, "txn1"); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1, 0)")); - TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn1, QueryShowResult(*bustub, _var, _txn, query, IntResult{ {1, 0}, })); WithTxn(txn1, ExecuteTxnTainted(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1, 1)")); - TxnMgrDbg("after txn1 taint", bustub->txn_manager_.get(), table_info, table_info->table_.get()); - TableHeapEntryNoMoreThan(*bustub, table_info, 1); + TxnMgrDbg("after txn1 taint", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 1); auto txn2 = BeginTxn(*bustub, "txn2"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2, 2)")); - TxnMgrDbg("after txn2 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{ {2, 2}, })); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn2 commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); - TableHeapEntryNoMoreThan(*bustub, table_info, 2); + TxnMgrDbg("after txn2 commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 2); auto txn3 = BeginTxn(*bustub, "txn3"); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (3, 3)")); - TxnMgrDbg("after txn3 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, query, IntResult{ {2, 2}, {3, 3}, })); WithTxn(txn3, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn3 commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); - TableHeapEntryNoMoreThan(*bustub, table_info, 3); + TxnMgrDbg("after txn3 commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 3); auto txn4 = BeginTxn(*bustub, "txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, @@ -61,19 +61,19 @@ TEST(TxnIndexTest, DISABLED_IndexInsertTest) { // NOLINT {3, 3}, })); WithTxn(txn4, ExecuteTxnTainted(*bustub, _var, _txn, "INSERT INTO maintable VALUES (3, 4)")); - TxnMgrDbg("after txn4 taint", bustub->txn_manager_.get(), table_info, table_info->table_.get()); - TableHeapEntryNoMoreThan(*bustub, table_info, 3); + TxnMgrDbg("after txn4 taint", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 3); auto txn5 = BeginTxn(*bustub, "txn5"); WithTxn(txn5, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (4, 4)")); - TxnMgrDbg("after txn5 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn5 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn5, QueryShowResult(*bustub, _var, _txn, query, IntResult{ {2, 2}, {3, 3}, {4, 4}, })); - TableHeapEntryNoMoreThan(*bustub, table_info, 4); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 4); auto txn6 = BeginTxn(*bustub, "txn6"); WithTxn(txn6, QueryShowResult(*bustub, _var, _txn, query, @@ -97,16 +97,16 @@ TEST(TxnIndexTest, DISABLED_InsertDeleteTest) { // NOLINT WithTxn(txn1, QueryIndex(*bustub, _var, _txn, query, "col1", std::vector{1, 2, 3, 4}, IntResult{{1, 0}, {2, 0}, {3, 0}, {4, 0}})); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn1_reverify = BeginTxn(*bustub, "txn1_reverify"); auto txn2 = BeginTxn(*bustub, "txn2"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable")); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{})); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 delete", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); - TableHeapEntryNoMoreThan(*bustub, table_info, 4); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 4); // hidden tests in-between @@ -136,7 +136,7 @@ TEST(TxnIndexTest, DISABLED_UpdateTest) { // NOLINT WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (4, 0), (6, 0)")); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE col1 = 5")); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE col1 = 6")); - TxnMgrDbg("after preparation", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after preparation", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); // at this point, we have (4, 0) inserted, (5, 0) deleted, and (6, 0) self inserted and deleted. return {txn1_reverify, txn2_reverify, txn3}; }; @@ -163,9 +163,9 @@ TEST(TxnIndexTest, DISABLED_UpdateTest) { // NOLINT IntResult{{1, 0}, {}, {}, {4, 0}, {}, {}})); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (2, 1), (5, 1), (3, 1), (6, 1)")); - TxnMgrDbg("after txn3 insert operations", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 insert operations", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET col2 = col2 + 10")); - TxnMgrDbg("after txn3 update operations", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 update operations", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, query, IntResult{ {1, 10}, @@ -185,7 +185,7 @@ TEST(TxnIndexTest, DISABLED_UpdateTest) { // NOLINT {6, 11}, })); WithTxn(txn3, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after commit", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn4 = BeginTxn(*bustub, "txn4"); WithTxn(txn4, QueryShowResult(*bustub, _var, _txn, query, @@ -207,7 +207,7 @@ TEST(TxnIndexTest, DISABLED_UpdateTest) { // NOLINT {6, 11}, })); reverify(bustub, txn1_reverify, txn2_reverify, query); - TableHeapEntryNoMoreThan(*bustub, table_info, 6); + TableHeapEntryNoMoreThan(*bustub, table_info.get(), 6); } // hidden tests... } @@ -223,16 +223,16 @@ TEST(GradingTxnIndexTest, DISABLED_IndexUpdateConflictTest) { // NOLINT WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (1, 0), (2, 0), (3, 0)")); WithTxn(txn1, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE col1 = 2")); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn2 = BeginTxn(*bustub, "txn2"); auto txn3 = BeginTxn(*bustub, "txn3"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "INSERT INTO maintable VALUES (4, 0)")); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE col1 = 1")); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "DELETE FROM maintable WHERE col1 = 3")); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn2 modification", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 modification", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); WithTxn(txn3, ExecuteTxnTainted(*bustub, _var, _txn, "UPDATE maintable SET col2 = 2 WHERE col1 = 1")); - TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn3 tainted", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); // hidden tests... } @@ -249,14 +249,14 @@ TEST(TxnIndexTest, DISABLED_UpdatePrimaryKeyTest) { // NOLINT WithTxn(txn1, QueryIndex(*bustub, _var, _txn, query, "col1", std::vector{1, 2, 3, 4}, IntResult{{1, 0}, {2, 0}, {3, 0}, {4, 0}})); WithTxn(txn1, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn1 insert", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn2 = BeginTxn(*bustub, "txn2"); WithTxn(txn2, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET col1 = col1 + 1")); WithTxn(txn2, QueryShowResult(*bustub, _var, _txn, query, IntResult{{2, 0}, {3, 0}, {4, 0}, {5, 0}})); WithTxn(txn2, QueryIndex(*bustub, _var, _txn, query, "col1", std::vector{1, 2, 3, 4, 5}, IntResult{{}, {2, 0}, {3, 0}, {4, 0}, {5, 0}})); WithTxn(txn2, CommitTxn(*bustub, _var, _txn)); - TxnMgrDbg("after txn2 update", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("after txn2 update", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto txn3 = BeginTxn(*bustub, "txn3"); WithTxn(txn3, ExecuteTxn(*bustub, _var, _txn, "UPDATE maintable SET col1 = col1 - 2")); WithTxn(txn3, QueryShowResult(*bustub, _var, _txn, query, IntResult{{0, 0}, {1, 0}, {2, 0}, {3, 0}})); diff --git a/test/txn/txn_scan_test.cpp b/test/txn/txn_scan_test.cpp index 930911a16..adb9ef809 100644 --- a/test/txn/txn_scan_test.cpp +++ b/test/txn/txn_scan_test.cpp @@ -199,7 +199,7 @@ TEST(TxnScanTest, DISABLED_ScanTest) { // NOLINT Tuple{{IntNull(), DoubleNull(), BoolNull()}, schema.get()}); bustub->txn_manager_->UpdateUndoLink(rid4, prev_log_5, nullptr); - TxnMgrDbg("before verify scan", bustub->txn_manager_.get(), table_info, table_info->table_.get()); + TxnMgrDbg("before verify scan", bustub->txn_manager_.get(), table_info.get(), table_info->table_.get()); auto query = "SELECT * FROM maintable"; fmt::println(stderr, "A: Verify txn0");