From 3ec480118e5d9b830d4c1b0661aa730c8ccc5e51 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Sat, 25 Jun 2022 12:09:41 -0400 Subject: [PATCH 01/28] Implement Slice, Head and Tail Operation in both centralize and distributed environment Signed-off-by: Arup Sarker --- cpp/src/cylon/compute/aggregate_kernels.hpp | 3 + cpp/src/cylon/net/utils.cpp | 1 + cpp/src/cylon/table.cpp | 118 ++++++++++++++++++ cpp/src/cylon/table.hpp | 26 ++++ .../flat_hash_map/bytell_hash_map.hpp | 2 + cpp/src/examples/CMakeLists.txt | 3 + cpp/src/examples/head_example.cpp | 107 ++++++++++++++++ cpp/src/examples/join_example.cpp | 35 +++++- cpp/src/examples/slice_example.cpp | 112 +++++++++++++++++ cpp/src/examples/tail_example.cpp | 107 ++++++++++++++++ 10 files changed, 511 insertions(+), 3 deletions(-) create mode 100644 cpp/src/examples/head_example.cpp create mode 100644 cpp/src/examples/slice_example.cpp create mode 100644 cpp/src/examples/tail_example.cpp diff --git a/cpp/src/cylon/compute/aggregate_kernels.hpp b/cpp/src/cylon/compute/aggregate_kernels.hpp index 0e9f4c58d..b676d10ac 100644 --- a/cpp/src/cylon/compute/aggregate_kernels.hpp +++ b/cpp/src/cylon/compute/aggregate_kernels.hpp @@ -19,6 +19,9 @@ #include #include #include +#include +#include + #include "cylon/util/macros.hpp" diff --git a/cpp/src/cylon/net/utils.cpp b/cpp/src/cylon/net/utils.cpp index 91b6134f2..63e34600a 100644 --- a/cpp/src/cylon/net/utils.cpp +++ b/cpp/src/cylon/net/utils.cpp @@ -12,6 +12,7 @@ * limitations under the License. */ +#include #include "utils.hpp" diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 65450a8b8..650a02f77 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -277,7 +277,11 @@ static inline Status shuffle_two_tables_by_hashing(const std::shared_ptr &ctx, const std::string &path, std::shared_ptr &tableOut, const cylon::io::config::CSVReadOptions &options) { arrow::Result> result = cylon::io::read_csv(ctx, path, options); + + LOG(INFO) << "Reading Inside FromCSV"; + if (result.ok()) { + LOG(INFO) << "CSV file reading is OK"; std::shared_ptr &table = result.ValueOrDie(); if (table->column(0)->chunks().size() > 1) { const auto &combine_res = table->CombineChunks(ToArrowPool(ctx)); @@ -289,6 +293,7 @@ Status FromCSV(const std::shared_ptr &ctx, const std::string &path } // slice the table if required if (options.IsSlice() && ctx->GetWorldSize() > 1) { + LOG(INFO) << "Slice the table if required"; int32_t rows_per_worker = table->num_rows() / ctx->GetWorldSize(); int32_t remainder = table->num_rows() % ctx->GetWorldSize(); @@ -1756,5 +1761,118 @@ Status WriteParquet(const std::shared_ptr &ctx_, return Status(Code::OK); } + +/** + * Slice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ + + +Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out) { +#ifdef CYLON_DEBUG + auto p1 = std::chrono::high_resolution_clock::now(); +#endif + const auto &ctx = in->GetContext(); + auto pool = cylon::ToArrowPool(ctx); + std::shared_ptr out_table, in_table = in->get_table(); + std::vector cols = {0}; + Unique(in, cols, out, true); + + + if (!in->Empty()) { + if (in_table->column(0)->num_chunks() > 1) { + CYLON_ASSIGN_OR_RAISE(in_table, in_table->CombineChunks(pool)) + } + + std::unique_ptr row_comp; + RETURN_CYLON_STATUS_IF_FAILED(TableRowIndexEqualTo::Make(in_table, cols, &row_comp)); + + std::unique_ptr row_hash; + RETURN_CYLON_STATUS_IF_FAILED(TableRowIndexHash::Make(in_table, cols, &row_hash)); + + const int64_t num_rows = length; + ska::bytell_hash_set + rows_set(num_rows, *row_hash, *row_comp); + + arrow::Int64Builder filter(pool); + RETURN_CYLON_STATUS_IF_ARROW_FAILED(filter.Reserve(num_rows)); +#ifdef CYLON_DEBUG + auto p2 = std::chrono::high_resolution_clock::now(); +#endif + for (int64_t row = offset; row < offset + num_rows; ++row) { + const auto &res = rows_set.insert(row); + if (res.second) { + filter.UnsafeAppend(row); + } + } +#ifdef CYLON_DEBUG + auto p3 = std::chrono::high_resolution_clock::now(); +#endif + rows_set.clear(); +#ifdef CYLON_DEBUG + auto p4 = std::chrono::high_resolution_clock::now(); +#endif + CYLON_ASSIGN_OR_RAISE(auto take_arr, filter.Finish()); + CYLON_ASSIGN_OR_RAISE(auto take_res, arrow::compute::Take(in_table, take_arr)) + out_table = take_res.table(); + +#ifdef CYLON_DEBUG + auto p5 = std::chrono::high_resolution_clock::now(); + LOG(INFO) << "P1 " << std::chrono::duration_cast(p2 - p1).count() + << " P2 " << std::chrono::duration_cast(p3 - p2).count() + << " P3 " << std::chrono::duration_cast(p4 - p3).count() + << " P4 " << std::chrono::duration_cast(p5 - p4).count() + << " tot " << std::chrono::duration_cast(p5 - p1).count() + << " tot " << rows_set.load_factor() << " " << rows_set.bucket_count(); +#endif + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), out); +} + + +/** + * Head the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + +Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if(num_rows > 0 && table_size > 0) { + return Slice(table, 0, num_rows, output); + } + else + LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + +} + +/** + * Tail the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + +Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + LOG(INFO) << "Input Table size " << table_size; + + if(num_rows > 0 && table_size > 0) { + return Slice(table, table_size-num_rows, num_rows, output); + } + else + LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + +} + #endif } // namespace cylon diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index 1b433eb01..37c1b6b3f 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -479,6 +479,32 @@ Status WriteParquet(const std::shared_ptr &ctx, std::shared_ptr &table, const std::string &path, const io::config::ParquetOptions &options = cylon::io::config::ParquetOptions()); +/** + * Slice the part of table to create a single table + * @param tables, offset, length + * @return new sliced table + */ +Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out); + +/** + * Head the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + +Status Head(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr &output); + +/** + * Tail the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + +Status Tail(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr &output); + #endif // BUILD_CYLON_PARQUET } // namespace cylon diff --git a/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp b/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp index 3a31c726d..9314eb433 100644 --- a/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp +++ b/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp @@ -13,6 +13,8 @@ #include #include #include +#include +#include #include diff --git a/cpp/src/examples/CMakeLists.txt b/cpp/src/examples/CMakeLists.txt index bc88311a3..bc6166e2a 100644 --- a/cpp/src/examples/CMakeLists.txt +++ b/cpp/src/examples/CMakeLists.txt @@ -75,6 +75,9 @@ cylon_add_exe(multi_idx_join_example) cylon_add_exe(parquet_union_example) cylon_add_exe(parquet_join_example) cylon_add_exe(dist_sort_example) +cylon_add_exe(slice_example) +cylon_add_exe(head_example) +cylon_add_exe(tail_example) if (CYLON_UCX) cylon_add_exe(ucx_join_example) diff --git a/cpp/src/examples/head_example.cpp b/cpp/src/examples/head_example.cpp new file mode 100644 index 000000000..dd323617f --- /dev/null +++ b/cpp/src/examples/head_example.cpp @@ -0,0 +1,107 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include + +#include +#include +#include +#include + +#include "example_utils.hpp" + + +int main(int argc, char *argv[]) { + if ((argc < 5 && std::string(argv[1]) == "f")) { + LOG(ERROR) << "./head_example f [n | o] csv_file num_rows" << std::endl + << "./head_example f [n | o] csv_file num_rows" << std::endl; + return 1; + } + + if ((argc < 6 && std::string(argv[1]) == "m")) { + LOG(ERROR) << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl + << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl; + return 1; + } + + auto start_start = std::chrono::steady_clock::now(); + auto mpi_config = std::make_shared(); + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + + std::shared_ptr in_table, head_table; + auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); + cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; + + std::string mem = std::string(argv[1]); + std::string ops_param = std::string(argv[2]); + int64_t num_rows = 0; + + bool ops = true; + if (ops_param == "o") { + ops = true; + } else if (ops_param == "n") { + ops = false; + } + + if (mem == "m") { + uint64_t count = std::stoull(argv[3]); + double dup = std::stod(argv[4]); + cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); + num_rows = std::stoull(argv[5]); + } else if (mem == "f") { + LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); + cylon::FromCSV(ctx, std::string(argv[3]), in_table); + + num_rows = std::stoull(argv[4]); + } + ctx->Barrier(); + auto read_end_time = std::chrono::steady_clock::now(); + //in_table->Print(); + LOG(INFO) << "Read tables in " + << std::chrono::duration_cast( + read_end_time - start_start).count() << "[ms]"; + + auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, + 0, + 0, + algorithm, + "l_", + "r_"); + cylon::Status status; + + // Arup: Code block for slice operation + + if (ops) { + status = cylon::Head(in_table, num_rows, head_table); + } else { + status = cylon::Head(in_table, num_rows, head_table); //Todos: Need to figure it out for distributed head + } + if (!status.is_ok()) { + LOG(INFO) << "Table Head is failed "; + ctx->Finalize(); + return 1; + } + auto slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Head table has : " << head_table->Rows(); + LOG(INFO) << "Head is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + std::vector sliced_column_names = head_table->ColumnNames(); + + head_table->Print(); + + ctx->Finalize(); + return 0; +} diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index d59abf698..2998eaee8 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -36,7 +36,7 @@ int main(int argc, char *argv[]) { auto mpi_config = std::make_shared(); auto ctx = cylon::CylonContext::InitDistributed(mpi_config); - std::shared_ptr first_table, second_table, joined; + std::shared_ptr first_table, second_table, joined, sliced; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; @@ -65,8 +65,13 @@ int main(int argc, char *argv[]) { double dup = std::stod(argv[4]); cylon::examples::create_two_in_memory_tables(count, dup,ctx,first_table,second_table); } else if (mem == "f") { - cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); - cylon::FromCSV(ctx, std::string(argv[4]) + std::to_string(ctx->GetRank()) + ".csv", second_table); + LOG(INFO) << "Load From first CSV file" << std::string(argv[3]); + cylon::FromCSV(ctx, std::string(argv[3]), first_table); + cylon::FromCSV(ctx, std::string(argv[4]), second_table); + + //cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); + //cylon::FromCSV(ctx, std::string(argv[4]) + std::to_string(ctx->GetRank()) + ".csv", second_table); + if (argc == 6) { if (!strcmp(argv[5], "hash")) { LOG(INFO) << "Hash join algorithm"; @@ -78,9 +83,11 @@ int main(int argc, char *argv[]) { } ctx->Barrier(); auto read_end_time = std::chrono::steady_clock::now(); + //first_table->Print(); LOG(INFO) << "Read tables in " << std::chrono::duration_cast( read_end_time - start_start).count() << "[ms]"; + auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, 0, 0, @@ -105,6 +112,28 @@ int main(int argc, char *argv[]) { << std::chrono::duration_cast( join_end_time - read_end_time).count() << "[ms]"; std::vector column_names = joined->ColumnNames(); + + // Arup: Code block for slice operation + + if (ops) { + status = cylon::Slice(first_table, 10, 5, sliced); + } else { + status = cylon::Slice(second_table, 10, 5, sliced); + } + if (!status.is_ok()) { + LOG(INFO) << "Table Slice is failed "; + ctx->Finalize(); + return 1; + } + auto slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Sliced table has : " << sliced->Rows(); + LOG(INFO) << "Sliced is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + std::vector sliced_column_names = sliced->ColumnNames(); + + sliced->Print(); + ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp new file mode 100644 index 000000000..d062bcb27 --- /dev/null +++ b/cpp/src/examples/slice_example.cpp @@ -0,0 +1,112 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include + +#include +#include +#include +#include + +#include "example_utils.hpp" + + +int main(int argc, char *argv[]) { + if ((argc < 6 && std::string(argv[1]) == "f")) { + LOG(ERROR) << "./slice_example f [n | o] csv_file offset length" << std::endl + << "./slice_example f [n | o] csv_file offset length" << std::endl; + return 1; + } + + if ((argc < 7 && std::string(argv[1]) == "m")) { + LOG(ERROR) << "./slice_example m [n | o] num_tuples_per_worker 0.0-1.0 offset length" << std::endl + << "./slice_example m [n | o] num_tuples_per_worker 0.0-1.0 offset length" << std::endl; + return 1; + } + + auto start_start = std::chrono::steady_clock::now(); + auto mpi_config = std::make_shared(); + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + + std::shared_ptr in_table, joined, sliced; + auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); + cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; + + std::string mem = std::string(argv[1]); + std::string ops_param = std::string(argv[2]); + int64_t offset = 0, length = 0; + + bool ops = true; + if (ops_param == "o") { + ops = true; + } else if (ops_param == "n") { + ops = false; + } + + if (mem == "m") { + uint64_t count = std::stoull(argv[3]); + double dup = std::stod(argv[4]); + cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); + offset = std::stoull(argv[5]); + length = std::stoull(argv[6]); + } else if (mem == "f") { + LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); + cylon::FromCSV(ctx, std::string(argv[3]) , in_table); + + //cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); + //cylon::FromCSV(ctx, std::string(argv[4]) + std::to_string(ctx->GetRank()) + ".csv", second_table); + + offset = std::stoull(argv[4]); + length = std::stoull(argv[5]); + } + ctx->Barrier(); + auto read_end_time = std::chrono::steady_clock::now(); + //in_table->Print(); + LOG(INFO) << "Read tables in " + << std::chrono::duration_cast( + read_end_time - start_start).count() << "[ms]"; + + auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, + 0, + 0, + algorithm, + "l_", + "r_"); + cylon::Status status; + + // Arup: Code block for slice operation + + if (ops) { + status = cylon::Slice(in_table, offset, length, sliced); + } else { + status = cylon::Slice(in_table, offset, length, sliced); + } + if (!status.is_ok()) { + LOG(INFO) << "Table Slice is failed "; + ctx->Finalize(); + return 1; + } + auto slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Sliced table has : " << sliced->Rows(); + LOG(INFO) << "Sliced is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + std::vector sliced_column_names = sliced->ColumnNames(); + + sliced->Print(); + + ctx->Finalize(); + return 0; +} diff --git a/cpp/src/examples/tail_example.cpp b/cpp/src/examples/tail_example.cpp new file mode 100644 index 000000000..e6974bf01 --- /dev/null +++ b/cpp/src/examples/tail_example.cpp @@ -0,0 +1,107 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include + +#include +#include +#include +#include + +#include "example_utils.hpp" + + +int main(int argc, char *argv[]) { + if ((argc < 5 && std::string(argv[1]) == "f")) { + LOG(ERROR) << "./head_example f [n | o] csv_file num_rows" << std::endl + << "./head_example f [n | o] csv_file num_rows" << std::endl; + return 1; + } + + if ((argc < 6 && std::string(argv[1]) == "m")) { + LOG(ERROR) << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl + << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl; + return 1; + } + + auto start_start = std::chrono::steady_clock::now(); + auto mpi_config = std::make_shared(); + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + + std::shared_ptr in_table, tail_table; + auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); + cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; + + std::string mem = std::string(argv[1]); + std::string ops_param = std::string(argv[2]); + int64_t num_rows = 0; + + bool ops = true; + if (ops_param == "o") { + ops = true; + } else if (ops_param == "n") { + ops = false; + } + + if (mem == "m") { + uint64_t count = std::stoull(argv[3]); + double dup = std::stod(argv[4]); + cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); + num_rows = std::stoull(argv[5]); + } else if (mem == "f") { + LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); + cylon::FromCSV(ctx, std::string(argv[3]), in_table); + + num_rows = std::stoull(argv[4]); + } + ctx->Barrier(); + auto read_end_time = std::chrono::steady_clock::now(); + //in_table->Print(); + LOG(INFO) << "Read tables in " + << std::chrono::duration_cast( + read_end_time - start_start).count() << "[ms]"; + + auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, + 0, + 0, + algorithm, + "l_", + "r_"); + cylon::Status status; + + // Arup: Code block for slice operation + + if (ops) { + status = cylon::Tail(in_table, num_rows, tail_table); + } else { + status = cylon::Tail(in_table, num_rows, tail_table); //Todos: Need to figure it out for distributed Tail + } + if (!status.is_ok()) { + LOG(INFO) << "Table Tail is failed "; + ctx->Finalize(); + return 1; + } + auto slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Tail table has : " << tail_table->Rows(); + LOG(INFO) << "Tail is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + std::vector sliced_column_names = tail_table->ColumnNames(); + + tail_table->Print(); + + ctx->Finalize(); + return 0; +} From 1b451cdab8cb838719dba6aaaee0041e947e7b0d Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Sat, 25 Jun 2022 14:26:18 -0400 Subject: [PATCH 02/28] Add Distributed Slice with arrow api Signed-off-by: Arup Sarker --- cpp/src/cylon/table.cpp | 39 +++++++++++++++++++++++++++--- cpp/src/cylon/table.hpp | 15 ++++++++++-- cpp/src/examples/join_example.cpp | 21 ---------------- cpp/src/examples/slice_example.cpp | 4 +-- 4 files changed, 50 insertions(+), 29 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 650a02f77..d98bd6ab9 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1763,13 +1763,13 @@ Status WriteParquet(const std::shared_ptr &ctx_, } /** - * Slice the part of table to create a single table + * Local_Slice the part of table to create a single table * @param table, offset and length * @return new sliced table */ -Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, +Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out) { #ifdef CYLON_DEBUG auto p1 = std::chrono::high_resolution_clock::now(); @@ -1834,6 +1834,37 @@ Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, } +/** + * Distributed_Slice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ + + +Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out) { + + const auto &ctx = in->GetContext(); + std::shared_ptr out_table, in_table = in->get_table(); + + if (!in->Empty()) { + + auto sliced = in_table->columns(); + int64_t num_rows = length; + for (auto& column : sliced) { + column = column->Slice(offset, length); + num_rows = column->length(); + } + out_table = arrow::Table::Make(in_table->schema(), std::move(sliced), num_rows); + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), out); +} + + + + /** * Head the part of table to create a single table with specific number of rows * @param tables, number of rows @@ -1846,7 +1877,7 @@ Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_p const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return Slice(table, 0, num_rows, output); + return Distributed_Slice(table, 0, num_rows, output); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); @@ -1867,7 +1898,7 @@ Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_p LOG(INFO) << "Input Table size " << table_size; if(num_rows > 0 && table_size > 0) { - return Slice(table, table_size-num_rows, num_rows, output); + return Distributed_Slice(table, table_size-num_rows, num_rows, output); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index 37c1b6b3f..a0d5ac9d2 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -480,13 +480,24 @@ Status WriteParquet(const std::shared_ptr &ctx, const std::string &path, const io::config::ParquetOptions &options = cylon::io::config::ParquetOptions()); /** - * Slice the part of table to create a single table + * Local_Slice the part of table to create a single table * @param tables, offset, length * @return new sliced table */ -Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, +Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out); +/** + * Distributed_Slice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ + + +Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out); + + /** * Head the part of table to create a single table with specific number of rows * @param tables, number of rows diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index 2998eaee8..cf1e22e85 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -113,27 +113,6 @@ int main(int argc, char *argv[]) { join_end_time - read_end_time).count() << "[ms]"; std::vector column_names = joined->ColumnNames(); - // Arup: Code block for slice operation - - if (ops) { - status = cylon::Slice(first_table, 10, 5, sliced); - } else { - status = cylon::Slice(second_table, 10, 5, sliced); - } - if (!status.is_ok()) { - LOG(INFO) << "Table Slice is failed "; - ctx->Finalize(); - return 1; - } - auto slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Sliced table has : " << sliced->Rows(); - LOG(INFO) << "Sliced is done in " - << std::chrono::duration_cast( - slice_end_time - read_end_time).count() << "[ms]"; - std::vector sliced_column_names = sliced->ColumnNames(); - - sliced->Print(); - ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index d062bcb27..e97233ae8 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -89,9 +89,9 @@ int main(int argc, char *argv[]) { // Arup: Code block for slice operation if (ops) { - status = cylon::Slice(in_table, offset, length, sliced); + status = cylon::Local_Slice(in_table, offset, length, sliced); } else { - status = cylon::Slice(in_table, offset, length, sliced); + status = cylon::Distributed_Slice(in_table, offset, length, sliced); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; From 68cfe5d30cf9c92f88954a7dab105373d5845b01 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Sat, 9 Jul 2022 21:31:23 -0400 Subject: [PATCH 03/28] [Cylon] Implementation of Distributive Slice, Head and Tail Signed-off-by: Arup Sarker --- cpp/src/cylon/ctx/cylon_context.cpp | 22 --- cpp/src/cylon/ctx/cylon_context.hpp | 1 - cpp/src/cylon/table.cpp | 177 +++++++++++------- cpp/src/cylon/table.hpp | 7 +- cpp/src/examples/compute_example.cpp | 7 +- cpp/src/examples/groupby_example.cpp | 6 +- cpp/src/examples/groupby_perf.cpp | 6 +- cpp/src/examples/groupby_pipeline_example.cpp | 6 +- cpp/src/examples/head_example.cpp | 10 +- cpp/src/examples/indexing_example.cpp | 140 ++++++++++++-- cpp/src/examples/intersect_example.cpp | 6 +- cpp/src/examples/join_example.cpp | 6 +- cpp/src/examples/multi_idx_join_example.cpp | 8 +- .../examples/multicolumn_sorting_example.cpp | 6 +- cpp/src/examples/parquet_join_example.cpp | 6 +- cpp/src/examples/parquet_union_example.cpp | 6 +- cpp/src/examples/project_example.cpp | 6 +- cpp/src/examples/select_example.cpp | 6 +- cpp/src/examples/slice_example.cpp | 19 +- cpp/src/examples/sorting_example.cpp | 6 +- cpp/src/examples/subtract_example.cpp | 6 +- cpp/src/examples/tail_example.cpp | 10 +- cpp/src/examples/ucx_join_example.cpp | 6 +- cpp/src/examples/union_example.cpp | 6 +- cpp/src/examples/unique_example.cpp | 6 +- cpp/test/custom_mpi_comm_test.cpp | 12 +- cpp/test/equal_test.cpp | 4 + 27 files changed, 368 insertions(+), 139 deletions(-) diff --git a/cpp/src/cylon/ctx/cylon_context.cpp b/cpp/src/cylon/ctx/cylon_context.cpp index a76a7e444..ff554e8c3 100644 --- a/cpp/src/cylon/ctx/cylon_context.cpp +++ b/cpp/src/cylon/ctx/cylon_context.cpp @@ -38,28 +38,6 @@ CylonContext::CylonContext(bool distributed) { this->is_distributed = distributed; } -std::shared_ptr CylonContext::InitDistributed(const std::shared_ptr &config) { - if (config->Type() == net::CommType::MPI) { - auto ctx = std::make_shared(true); - ctx->communicator = std::make_shared(&ctx); - ctx->communicator->Init(config); - return ctx; - } - -#ifdef BUILD_CYLON_UCX - else if (config->Type() == net::CommType::UCX) { - auto ctx = std::make_shared(true); - ctx->communicator = std::make_shared(&ctx); - ctx->communicator->Init(config); - return ctx; - } -#endif - else { - throw "Unsupported communication type"; - } - return nullptr; -} - Status CylonContext::InitDistributed(const std::shared_ptr &config, std::shared_ptr *ctx) { switch (config->Type()) { diff --git a/cpp/src/cylon/ctx/cylon_context.hpp b/cpp/src/cylon/ctx/cylon_context.hpp index 24a28b21c..ccc921274 100644 --- a/cpp/src/cylon/ctx/cylon_context.hpp +++ b/cpp/src/cylon/ctx/cylon_context.hpp @@ -54,7 +54,6 @@ class CylonContext { * @param config Configuration to be passed on to the cylon::net::Communicator * @return */ - static std::shared_ptr InitDistributed(const std::shared_ptr &config); static Status InitDistributed(const std::shared_ptr &config, std::shared_ptr *ctx); diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index d98bd6ab9..4ffba2a07 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1472,8 +1472,11 @@ Status Equals(const std::shared_ptr &a, const std::shared_ptr &a, const std::shared_ptr &b, std::shared_ptr *b_out) { + LOG(INFO) << "I am at the RepartitionToMatchOtherTable"; int64_t num_row = a->Rows(); + LOG(INFO) << "Table-A size: " << num_row; + LOG(INFO) << "Table-B size: " << b->Rows(); if (num_row == 0) { *b_out = a; return Status::OK(); @@ -1483,8 +1486,11 @@ static Status RepartitionToMatchOtherTable(const std::shared_ptr & std::vector rows_per_partition; std::shared_ptr output; + + LOG(INFO) << "Execute before allgather api in RepartitionToMatchOtherTable"; RETURN_CYLON_STATUS_IF_FAILED( a->GetContext()->GetCommunicator()->Allgather(num_row_scalar, &output)); + LOG(INFO) << "Execute after allgather api in RepartitionToMatchOtherTable"; auto *data_ptr = std::static_pointer_cast(output->data())->raw_values(); @@ -1501,6 +1507,8 @@ Status DistributedEquals(const std::shared_ptr &a, bool subResult; RETURN_CYLON_STATUS_IF_FAILED(VerifyTableSchema(a->get_table(), b->get_table())); + LOG(INFO) << "I am at the DistributedEquals"; + if (!ordered) { int col = a->Columns(); std::vector indices(col); @@ -1556,6 +1564,7 @@ Status Repartition(const std::shared_ptr &table, auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + RETURN_CYLON_STATUS_IF_FAILED( table->GetContext()->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); @@ -1771,62 +1780,11 @@ Status WriteParquet(const std::shared_ptr &ctx_, Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out) { -#ifdef CYLON_DEBUG - auto p1 = std::chrono::high_resolution_clock::now(); -#endif const auto &ctx = in->GetContext(); - auto pool = cylon::ToArrowPool(ctx); std::shared_ptr out_table, in_table = in->get_table(); - std::vector cols = {0}; - Unique(in, cols, out, true); - if (!in->Empty()) { - if (in_table->column(0)->num_chunks() > 1) { - CYLON_ASSIGN_OR_RAISE(in_table, in_table->CombineChunks(pool)) - } - - std::unique_ptr row_comp; - RETURN_CYLON_STATUS_IF_FAILED(TableRowIndexEqualTo::Make(in_table, cols, &row_comp)); - - std::unique_ptr row_hash; - RETURN_CYLON_STATUS_IF_FAILED(TableRowIndexHash::Make(in_table, cols, &row_hash)); - - const int64_t num_rows = length; - ska::bytell_hash_set - rows_set(num_rows, *row_hash, *row_comp); - - arrow::Int64Builder filter(pool); - RETURN_CYLON_STATUS_IF_ARROW_FAILED(filter.Reserve(num_rows)); -#ifdef CYLON_DEBUG - auto p2 = std::chrono::high_resolution_clock::now(); -#endif - for (int64_t row = offset; row < offset + num_rows; ++row) { - const auto &res = rows_set.insert(row); - if (res.second) { - filter.UnsafeAppend(row); - } - } -#ifdef CYLON_DEBUG - auto p3 = std::chrono::high_resolution_clock::now(); -#endif - rows_set.clear(); -#ifdef CYLON_DEBUG - auto p4 = std::chrono::high_resolution_clock::now(); -#endif - CYLON_ASSIGN_OR_RAISE(auto take_arr, filter.Finish()); - CYLON_ASSIGN_OR_RAISE(auto take_res, arrow::compute::Take(in_table, take_arr)) - out_table = take_res.table(); - -#ifdef CYLON_DEBUG - auto p5 = std::chrono::high_resolution_clock::now(); - LOG(INFO) << "P1 " << std::chrono::duration_cast(p2 - p1).count() - << " P2 " << std::chrono::duration_cast(p3 - p2).count() - << " P3 " << std::chrono::duration_cast(p4 - p3).count() - << " P4 " << std::chrono::duration_cast(p5 - p4).count() - << " tot " << std::chrono::duration_cast(p5 - p1).count() - << " tot " << rows_set.load_factor() << " " << rows_set.bucket_count(); -#endif + out_table = in_table->Slice(offset, length); } else { out_table = in_table; } @@ -1841,21 +1799,85 @@ Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t len */ -Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out) { +Status Distributed_Slice(const std::shared_ptr &in, int64_t offset, int64_t length, + std::shared_ptr &out, int order) { const auto &ctx = in->GetContext(); std::shared_ptr out_table, in_table = in->get_table(); + auto num_row = in->Rows(); if (!in->Empty()) { + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + + + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); + + int total_partition = sizes_cols->length(); + LOG(INFO) << "Total Length: " << total_partition; + sizes.resize(sizes_cols->length()); + std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); + + + int current_partition = ctx->GetRank(); + LOG(INFO) << "Current Partion: " << current_partition << " and Size: " << sizes[current_partition]; + + std::vector offset_vec; + std::vector length_vec; + + if(!order) { + LOG(INFO) << "from 0 to size"; + for(int i = 0; i < total_partition; i++) { + if(offset + length > sizes[i]) { + if(sizes[i] - offset <= 0) + { + offset_vec.push_back(sizes[i]); + length_vec.push_back(0); + offset = 0; + } + else { + offset_vec.push_back(offset); + length_vec.push_back(sizes[i] - offset); + + length = length - sizes[i] + offset; + offset = 0; + } + } + else { + offset_vec.push_back(offset); + length_vec.push_back(length); + length = 0; + offset = 0; + } + } + } + else { + LOG(INFO) << "from size to 0"; + for(int i = 0; i < total_partition; i++){ + if(length > sizes[i]) { + offset_vec.push_back(0); + length_vec.push_back(sizes[i]); + + length = length - sizes[i]; + } + else { + offset_vec.push_back(sizes[i] - length); + length_vec.push_back(length); + length = 0; + } + } + } + + out_table = in_table->Slice(offset_vec[current_partition], length_vec[current_partition]); - auto sliced = in_table->columns(); - int64_t num_rows = length; - for (auto& column : sliced) { - column = column->Slice(offset, length); - num_rows = column->length(); - } - out_table = arrow::Table::Make(in_table->schema(), std::move(sliced), num_rows); } else { out_table = in_table; } @@ -1877,7 +1899,21 @@ Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_p const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return Distributed_Slice(table, 0, num_rows, output); + return Local_Slice(table, 0, num_rows, output); + } + else + LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + +} + +Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + int order = 0; + + if(num_rows > 0 && table_size > 0) { + return Distributed_Slice(table, 0, num_rows, output, order); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); @@ -1898,7 +1934,22 @@ Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_p LOG(INFO) << "Input Table size " << table_size; if(num_rows > 0 && table_size > 0) { - return Distributed_Slice(table, table_size-num_rows, num_rows, output); + return Local_Slice(table, table_size-num_rows, num_rows, output); + } + else + LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + +} + +Status Distributed_Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + int order = 1; + LOG(INFO) << "Input Table size " << table_size; + + if(num_rows > 0 && table_size > 0) { + return Distributed_Slice(table, table_size-num_rows, num_rows, output, order); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index a0d5ac9d2..60e1f1bec 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -495,7 +495,7 @@ Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t len Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out); + std::shared_ptr &out, int order); /** @@ -506,6 +506,8 @@ Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64 Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); +Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr &output); /** * Tail the part of table to create a single table with specific number of rows @@ -516,6 +518,9 @@ Status Head(const std::shared_ptr
&table, int64_t num_rows, Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); +Status Distributed_Tail(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr &output); + #endif // BUILD_CYLON_PARQUET } // namespace cylon diff --git a/cpp/src/examples/compute_example.cpp b/cpp/src/examples/compute_example.cpp index a67461ee4..a95f1eaa4 100644 --- a/cpp/src/examples/compute_example.cpp +++ b/cpp/src/examples/compute_example.cpp @@ -44,7 +44,12 @@ int main() { int32_t agg_index = 1; auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr table; std::shared_ptr result; diff --git a/cpp/src/examples/groupby_example.cpp b/cpp/src/examples/groupby_example.cpp index 293d2e213..10e527cb0 100644 --- a/cpp/src/examples/groupby_example.cpp +++ b/cpp/src/examples/groupby_example.cpp @@ -40,7 +40,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/groupby_perf.cpp b/cpp/src/examples/groupby_perf.cpp index d33657c7e..f8d87a712 100644 --- a/cpp/src/examples/groupby_perf.cpp +++ b/cpp/src/examples/groupby_perf.cpp @@ -35,7 +35,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/groupby_pipeline_example.cpp b/cpp/src/examples/groupby_pipeline_example.cpp index 253d8f9cc..902548f3f 100644 --- a/cpp/src/examples/groupby_pipeline_example.cpp +++ b/cpp/src/examples/groupby_pipeline_example.cpp @@ -56,7 +56,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = cylon::net::MPIConfig::Make(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } arrow::MemoryPool *pool = arrow::default_memory_pool(); std::shared_ptr left_table; diff --git a/cpp/src/examples/head_example.cpp b/cpp/src/examples/head_example.cpp index dd323617f..4de9db165 100644 --- a/cpp/src/examples/head_example.cpp +++ b/cpp/src/examples/head_example.cpp @@ -38,7 +38,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr in_table, head_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); @@ -86,7 +90,7 @@ int main(int argc, char *argv[]) { if (ops) { status = cylon::Head(in_table, num_rows, head_table); } else { - status = cylon::Head(in_table, num_rows, head_table); //Todos: Need to figure it out for distributed head + status = cylon::Distributed_Head(in_table, num_rows, head_table); } if (!status.is_ok()) { LOG(INFO) << "Table Head is failed "; @@ -101,7 +105,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = head_table->ColumnNames(); head_table->Print(); - + sleep(1); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/indexing_example.cpp b/cpp/src/examples/indexing_example.cpp index d2f937840..294fac960 100644 --- a/cpp/src/examples/indexing_example.cpp +++ b/cpp/src/examples/indexing_example.cpp @@ -285,7 +285,12 @@ int arrow_indexer_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -352,7 +357,12 @@ int arrow_indexer_str_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -412,7 +422,12 @@ int arrow_indexer_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -475,7 +490,12 @@ int arrow_indexer_str_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -538,7 +558,12 @@ int arrow_indexer_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -600,7 +625,12 @@ int arrow_indexer_str_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -662,7 +692,12 @@ int arrow_indexer_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -723,7 +758,12 @@ int arrow_indexer_str_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -784,7 +824,12 @@ int arrow_indexer_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -847,7 +892,12 @@ int arrow_indexer_str_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -910,7 +960,12 @@ int arrow_indexer_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -972,7 +1027,12 @@ int arrow_indexer_str_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1039,7 +1099,12 @@ int arrow_iloc_indexer_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1099,7 +1164,12 @@ int arrow_iloc_indexer_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1162,7 +1232,12 @@ int arrow_iloc_indexer_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1224,7 +1299,12 @@ int arrow_iloc_indexer_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1285,7 +1365,12 @@ int arrow_iloc_indexer_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1349,7 +1434,12 @@ int arrow_iloc_indexer_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1426,7 +1516,12 @@ int arrow_filter_example() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; @@ -1543,7 +1638,12 @@ int arrow_range_indexer_test() { separator(func_title); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } cylon::Status status; diff --git a/cpp/src/examples/intersect_example.cpp b/cpp/src/examples/intersect_example.cpp index c55b3e1b5..483410241 100644 --- a/cpp/src/examples/intersect_example.cpp +++ b/cpp/src/examples/intersect_example.cpp @@ -32,7 +32,11 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index cf1e22e85..206baae8d 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -34,7 +34,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, joined, sliced; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/multi_idx_join_example.cpp b/cpp/src/examples/multi_idx_join_example.cpp index dd869a0c8..b7959d422 100644 --- a/cpp/src/examples/multi_idx_join_example.cpp +++ b/cpp/src/examples/multi_idx_join_example.cpp @@ -32,7 +32,7 @@ } while (0) int main(int argc, char *argv[]) { - if (argc < 6) { + if (argc < 5) { LOG(ERROR) << "./multi_idx_join_example m [n | o] num_tuples_per_worker 0.0-1.0" << std::endl << "./multi_idx_join_example m [n | o] num_tuples_per_worker 0.0-1.0" << std::endl << "./multi_idx_join_example f [n | o] csv_file1 csv_file2" << std::endl @@ -42,7 +42,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, joined; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/multicolumn_sorting_example.cpp b/cpp/src/examples/multicolumn_sorting_example.cpp index 4023d7111..0828fa228 100644 --- a/cpp/src/examples/multicolumn_sorting_example.cpp +++ b/cpp/src/examples/multicolumn_sorting_example.cpp @@ -31,7 +31,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); CYLON_UNUSED(start_start); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/parquet_join_example.cpp b/cpp/src/examples/parquet_join_example.cpp index 918ec65a3..bf56910ce 100644 --- a/cpp/src/examples/parquet_join_example.cpp +++ b/cpp/src/examples/parquet_join_example.cpp @@ -27,7 +27,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, joined; diff --git a/cpp/src/examples/parquet_union_example.cpp b/cpp/src/examples/parquet_union_example.cpp index fccd641aa..1bc23094d 100644 --- a/cpp/src/examples/parquet_union_example.cpp +++ b/cpp/src/examples/parquet_union_example.cpp @@ -30,7 +30,11 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, unioned_table; diff --git a/cpp/src/examples/project_example.cpp b/cpp/src/examples/project_example.cpp index 02363f354..7ee3adc58 100644 --- a/cpp/src/examples/project_example.cpp +++ b/cpp/src/examples/project_example.cpp @@ -29,7 +29,11 @@ int main(int argc, char *argv[]) { } auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr table, project; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/select_example.cpp b/cpp/src/examples/select_example.cpp index 8a8b6038e..4c18d8fe5 100644 --- a/cpp/src/examples/select_example.cpp +++ b/cpp/src/examples/select_example.cpp @@ -30,7 +30,11 @@ int main(int argc, char *argv[]) { } auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr table, select; auto read_options = cylon::io::config::CSVReadOptions().UseThreads( diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index e97233ae8..bdd721407 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -35,10 +35,16 @@ int main(int argc, char *argv[]) { << "./slice_example m [n | o] num_tuples_per_worker 0.0-1.0 offset length" << std::endl; return 1; } - + LOG(INFO) << "Starting main() function"; auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } + std::shared_ptr in_table, joined, sliced; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); @@ -61,8 +67,9 @@ int main(int argc, char *argv[]) { cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); offset = std::stoull(argv[5]); length = std::stoull(argv[6]); + LOG(INFO) << "Load From in-memory size: " << std::string(argv[3]); } else if (mem == "f") { - LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); + LOG(INFO) << "Load From the CSV file: " << std::string(argv[3]); cylon::FromCSV(ctx, std::string(argv[3]) , in_table); //cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); @@ -87,11 +94,11 @@ int main(int argc, char *argv[]) { cylon::Status status; // Arup: Code block for slice operation - + int order = 0; if (ops) { status = cylon::Local_Slice(in_table, offset, length, sliced); } else { - status = cylon::Distributed_Slice(in_table, offset, length, sliced); + status = cylon::Distributed_Slice(in_table, offset, length, sliced, order); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; @@ -106,7 +113,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = sliced->ColumnNames(); sliced->Print(); - + sleep(1); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/sorting_example.cpp b/cpp/src/examples/sorting_example.cpp index d65549c49..cd231d94d 100644 --- a/cpp/src/examples/sorting_example.cpp +++ b/cpp/src/examples/sorting_example.cpp @@ -32,7 +32,11 @@ int main(int argc, char *argv[]) { CYLON_UNUSED(start_start); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/subtract_example.cpp b/cpp/src/examples/subtract_example.cpp index 90f4ceb08..ee1abcabd 100644 --- a/cpp/src/examples/subtract_example.cpp +++ b/cpp/src/examples/subtract_example.cpp @@ -32,7 +32,11 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/tail_example.cpp b/cpp/src/examples/tail_example.cpp index e6974bf01..b193499ef 100644 --- a/cpp/src/examples/tail_example.cpp +++ b/cpp/src/examples/tail_example.cpp @@ -38,7 +38,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr in_table, tail_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); @@ -86,7 +90,7 @@ int main(int argc, char *argv[]) { if (ops) { status = cylon::Tail(in_table, num_rows, tail_table); } else { - status = cylon::Tail(in_table, num_rows, tail_table); //Todos: Need to figure it out for distributed Tail + status = cylon::Distributed_Tail(in_table, num_rows, tail_table); } if (!status.is_ok()) { LOG(INFO) << "Table Tail is failed "; @@ -101,7 +105,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = tail_table->ColumnNames(); tail_table->Print(); - + sleep(1); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/ucx_join_example.cpp b/cpp/src/examples/ucx_join_example.cpp index 12c2ee8c8..34e02fccd 100644 --- a/cpp/src/examples/ucx_join_example.cpp +++ b/cpp/src/examples/ucx_join_example.cpp @@ -27,7 +27,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, joined; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/union_example.cpp b/cpp/src/examples/union_example.cpp index 28af0a423..b59c42203 100644 --- a/cpp/src/examples/union_example.cpp +++ b/cpp/src/examples/union_example.cpp @@ -35,7 +35,11 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/unique_example.cpp b/cpp/src/examples/unique_example.cpp index a70af7a32..b0880b5e9 100644 --- a/cpp/src/examples/unique_example.cpp +++ b/cpp/src/examples/unique_example.cpp @@ -33,7 +33,11 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return 1; + } std::shared_ptr table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/test/custom_mpi_comm_test.cpp b/cpp/test/custom_mpi_comm_test.cpp index a003df19b..119fce9bd 100644 --- a/cpp/test/custom_mpi_comm_test.cpp +++ b/cpp/test/custom_mpi_comm_test.cpp @@ -49,7 +49,11 @@ TEST_CASE("custom mpi communicator") { REQUIRE(MPI_Comm_split(MPI_COMM_WORLD, color, l_rank, &new_comm) == MPI_SUCCESS); auto mpi_config = cylon::net::MPIConfig::Make(new_comm); - auto ctx = cylon::CylonContext::InitDistributed(mpi_config); + std::shared_ptr ctx; + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return; + } REQUIRE(l_rank == ctx->GetRank()); if (color == 0) { @@ -67,7 +71,11 @@ TEST_CASE("custom mpi communicator") { REQUIRE(MPI_Comm_split(MPI_COMM_WORLD, color, l_rank, &new_comm) == MPI_SUCCESS); mpi_config = cylon::net::MPIConfig::Make(new_comm); - ctx = cylon::CylonContext::InitDistributed(mpi_config); + + if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { + std::cerr << "ctx init failed! " << std::endl; + return; + } REQUIRE(l_rank == ctx->GetRank()); REQUIRE(ctx->GetWorldSize() == 2); diff --git a/cpp/test/equal_test.cpp b/cpp/test/equal_test.cpp index 5dae01fe9..78193db0d 100644 --- a/cpp/test/equal_test.cpp +++ b/cpp/test/equal_test.cpp @@ -109,9 +109,11 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result)); + printf("testing unordered equal\n"); REQUIRE(result); CHECK_CYLON_STATUS(DistributedEquals(table2, table3, result)); + printf("testing unordered equal\n"); REQUIRE(!result); } @@ -119,6 +121,7 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result, false)); + printf("testing unordered equal\n"); REQUIRE(result); } @@ -126,6 +129,7 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table2, table3, result, false)); + printf("testing unordered equal\n"); REQUIRE(result); } } From bde8bab4c37fffeb19c6884569f49b18c1ef6b13 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Thu, 14 Jul 2022 13:45:34 -0400 Subject: [PATCH 04/28] [Cylon] Revert the changes based on the upstream Signed-off-by: Arup Sarker --- cpp/src/cylon/ctx/cylon_context.cpp | 22 +++ cpp/src/cylon/ctx/cylon_context.hpp | 1 + cpp/src/examples/compute_example.cpp | 7 +- cpp/src/examples/groupby_example.cpp | 6 +- cpp/src/examples/groupby_perf.cpp | 6 +- cpp/src/examples/groupby_pipeline_example.cpp | 6 +- cpp/src/examples/indexing_example.cpp | 140 +++--------------- cpp/src/examples/intersect_example.cpp | 6 +- cpp/src/examples/join_example.cpp | 6 +- cpp/src/examples/multi_idx_join_example.cpp | 8 +- .../examples/multicolumn_sorting_example.cpp | 6 +- cpp/src/examples/parquet_join_example.cpp | 6 +- cpp/src/examples/parquet_union_example.cpp | 6 +- cpp/src/examples/project_example.cpp | 6 +- cpp/src/examples/select_example.cpp | 6 +- cpp/src/examples/sorting_example.cpp | 6 +- cpp/src/examples/subtract_example.cpp | 6 +- cpp/src/examples/ucx_join_example.cpp | 6 +- cpp/src/examples/union_example.cpp | 6 +- cpp/src/examples/unique_example.cpp | 6 +- cpp/test/custom_mpi_comm_test.cpp | 12 +- cpp/test/equal_test.cpp | 4 - 22 files changed, 63 insertions(+), 221 deletions(-) diff --git a/cpp/src/cylon/ctx/cylon_context.cpp b/cpp/src/cylon/ctx/cylon_context.cpp index ff554e8c3..a76a7e444 100644 --- a/cpp/src/cylon/ctx/cylon_context.cpp +++ b/cpp/src/cylon/ctx/cylon_context.cpp @@ -38,6 +38,28 @@ CylonContext::CylonContext(bool distributed) { this->is_distributed = distributed; } +std::shared_ptr CylonContext::InitDistributed(const std::shared_ptr &config) { + if (config->Type() == net::CommType::MPI) { + auto ctx = std::make_shared(true); + ctx->communicator = std::make_shared(&ctx); + ctx->communicator->Init(config); + return ctx; + } + +#ifdef BUILD_CYLON_UCX + else if (config->Type() == net::CommType::UCX) { + auto ctx = std::make_shared(true); + ctx->communicator = std::make_shared(&ctx); + ctx->communicator->Init(config); + return ctx; + } +#endif + else { + throw "Unsupported communication type"; + } + return nullptr; +} + Status CylonContext::InitDistributed(const std::shared_ptr &config, std::shared_ptr *ctx) { switch (config->Type()) { diff --git a/cpp/src/cylon/ctx/cylon_context.hpp b/cpp/src/cylon/ctx/cylon_context.hpp index ccc921274..24a28b21c 100644 --- a/cpp/src/cylon/ctx/cylon_context.hpp +++ b/cpp/src/cylon/ctx/cylon_context.hpp @@ -54,6 +54,7 @@ class CylonContext { * @param config Configuration to be passed on to the cylon::net::Communicator * @return */ + static std::shared_ptr InitDistributed(const std::shared_ptr &config); static Status InitDistributed(const std::shared_ptr &config, std::shared_ptr *ctx); diff --git a/cpp/src/examples/compute_example.cpp b/cpp/src/examples/compute_example.cpp index a95f1eaa4..a67461ee4 100644 --- a/cpp/src/examples/compute_example.cpp +++ b/cpp/src/examples/compute_example.cpp @@ -44,12 +44,7 @@ int main() { int32_t agg_index = 1; auto mpi_config = std::make_shared(); - - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr table; std::shared_ptr result; diff --git a/cpp/src/examples/groupby_example.cpp b/cpp/src/examples/groupby_example.cpp index 10e527cb0..293d2e213 100644 --- a/cpp/src/examples/groupby_example.cpp +++ b/cpp/src/examples/groupby_example.cpp @@ -40,11 +40,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/groupby_perf.cpp b/cpp/src/examples/groupby_perf.cpp index f8d87a712..d33657c7e 100644 --- a/cpp/src/examples/groupby_perf.cpp +++ b/cpp/src/examples/groupby_perf.cpp @@ -35,11 +35,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/groupby_pipeline_example.cpp b/cpp/src/examples/groupby_pipeline_example.cpp index 902548f3f..253d8f9cc 100644 --- a/cpp/src/examples/groupby_pipeline_example.cpp +++ b/cpp/src/examples/groupby_pipeline_example.cpp @@ -56,11 +56,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = cylon::net::MPIConfig::Make(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); arrow::MemoryPool *pool = arrow::default_memory_pool(); std::shared_ptr left_table; diff --git a/cpp/src/examples/indexing_example.cpp b/cpp/src/examples/indexing_example.cpp index 294fac960..d2f937840 100644 --- a/cpp/src/examples/indexing_example.cpp +++ b/cpp/src/examples/indexing_example.cpp @@ -285,12 +285,7 @@ int arrow_indexer_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -357,12 +352,7 @@ int arrow_indexer_str_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -422,12 +412,7 @@ int arrow_indexer_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -490,12 +475,7 @@ int arrow_indexer_str_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -558,12 +538,7 @@ int arrow_indexer_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -625,12 +600,7 @@ int arrow_indexer_str_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -692,12 +662,7 @@ int arrow_indexer_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -758,12 +723,7 @@ int arrow_indexer_str_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -824,12 +784,7 @@ int arrow_indexer_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -892,12 +847,7 @@ int arrow_indexer_str_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -960,12 +910,7 @@ int arrow_indexer_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1027,12 +972,7 @@ int arrow_indexer_str_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1099,12 +1039,7 @@ int arrow_iloc_indexer_test_1() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1164,12 +1099,7 @@ int arrow_iloc_indexer_test_2() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1232,12 +1162,7 @@ int arrow_iloc_indexer_test_3() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1299,12 +1224,7 @@ int arrow_iloc_indexer_test_4() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1365,12 +1285,7 @@ int arrow_iloc_indexer_test_5() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1434,12 +1349,7 @@ int arrow_iloc_indexer_test_6() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1516,12 +1426,7 @@ int arrow_filter_example() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; @@ -1638,12 +1543,7 @@ int arrow_range_indexer_test() { separator(func_title); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); cylon::Status status; diff --git a/cpp/src/examples/intersect_example.cpp b/cpp/src/examples/intersect_example.cpp index 483410241..c55b3e1b5 100644 --- a/cpp/src/examples/intersect_example.cpp +++ b/cpp/src/examples/intersect_example.cpp @@ -32,11 +32,7 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index 206baae8d..cf1e22e85 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -34,11 +34,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, joined, sliced; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/multi_idx_join_example.cpp b/cpp/src/examples/multi_idx_join_example.cpp index b7959d422..dd869a0c8 100644 --- a/cpp/src/examples/multi_idx_join_example.cpp +++ b/cpp/src/examples/multi_idx_join_example.cpp @@ -32,7 +32,7 @@ } while (0) int main(int argc, char *argv[]) { - if (argc < 5) { + if (argc < 6) { LOG(ERROR) << "./multi_idx_join_example m [n | o] num_tuples_per_worker 0.0-1.0" << std::endl << "./multi_idx_join_example m [n | o] num_tuples_per_worker 0.0-1.0" << std::endl << "./multi_idx_join_example f [n | o] csv_file1 csv_file2" << std::endl @@ -42,11 +42,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, joined; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/multicolumn_sorting_example.cpp b/cpp/src/examples/multicolumn_sorting_example.cpp index 0828fa228..4023d7111 100644 --- a/cpp/src/examples/multicolumn_sorting_example.cpp +++ b/cpp/src/examples/multicolumn_sorting_example.cpp @@ -31,11 +31,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); CYLON_UNUSED(start_start); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/parquet_join_example.cpp b/cpp/src/examples/parquet_join_example.cpp index bf56910ce..918ec65a3 100644 --- a/cpp/src/examples/parquet_join_example.cpp +++ b/cpp/src/examples/parquet_join_example.cpp @@ -27,11 +27,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, joined; diff --git a/cpp/src/examples/parquet_union_example.cpp b/cpp/src/examples/parquet_union_example.cpp index 1bc23094d..fccd641aa 100644 --- a/cpp/src/examples/parquet_union_example.cpp +++ b/cpp/src/examples/parquet_union_example.cpp @@ -30,11 +30,7 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, unioned_table; diff --git a/cpp/src/examples/project_example.cpp b/cpp/src/examples/project_example.cpp index 7ee3adc58..02363f354 100644 --- a/cpp/src/examples/project_example.cpp +++ b/cpp/src/examples/project_example.cpp @@ -29,11 +29,7 @@ int main(int argc, char *argv[]) { } auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr table, project; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/select_example.cpp b/cpp/src/examples/select_example.cpp index 4c18d8fe5..8a8b6038e 100644 --- a/cpp/src/examples/select_example.cpp +++ b/cpp/src/examples/select_example.cpp @@ -30,11 +30,7 @@ int main(int argc, char *argv[]) { } auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr table, select; auto read_options = cylon::io::config::CSVReadOptions().UseThreads( diff --git a/cpp/src/examples/sorting_example.cpp b/cpp/src/examples/sorting_example.cpp index cd231d94d..d65549c49 100644 --- a/cpp/src/examples/sorting_example.cpp +++ b/cpp/src/examples/sorting_example.cpp @@ -32,11 +32,7 @@ int main(int argc, char *argv[]) { CYLON_UNUSED(start_start); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/subtract_example.cpp b/cpp/src/examples/subtract_example.cpp index ee1abcabd..90f4ceb08 100644 --- a/cpp/src/examples/subtract_example.cpp +++ b/cpp/src/examples/subtract_example.cpp @@ -32,11 +32,7 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/ucx_join_example.cpp b/cpp/src/examples/ucx_join_example.cpp index 34e02fccd..12c2ee8c8 100644 --- a/cpp/src/examples/ucx_join_example.cpp +++ b/cpp/src/examples/ucx_join_example.cpp @@ -27,11 +27,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, joined; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/union_example.cpp b/cpp/src/examples/union_example.cpp index b59c42203..28af0a423 100644 --- a/cpp/src/examples/union_example.cpp +++ b/cpp/src/examples/union_example.cpp @@ -35,11 +35,7 @@ int main(int argc, char *argv[]) { auto start_time = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr first_table, second_table, unioned_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/src/examples/unique_example.cpp b/cpp/src/examples/unique_example.cpp index b0880b5e9..a70af7a32 100644 --- a/cpp/src/examples/unique_example.cpp +++ b/cpp/src/examples/unique_example.cpp @@ -33,11 +33,7 @@ int main(int argc, char *argv[]) { auto start_start = std::chrono::steady_clock::now(); auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); std::shared_ptr table, output; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); diff --git a/cpp/test/custom_mpi_comm_test.cpp b/cpp/test/custom_mpi_comm_test.cpp index 119fce9bd..a003df19b 100644 --- a/cpp/test/custom_mpi_comm_test.cpp +++ b/cpp/test/custom_mpi_comm_test.cpp @@ -49,11 +49,7 @@ TEST_CASE("custom mpi communicator") { REQUIRE(MPI_Comm_split(MPI_COMM_WORLD, color, l_rank, &new_comm) == MPI_SUCCESS); auto mpi_config = cylon::net::MPIConfig::Make(new_comm); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return; - } + auto ctx = cylon::CylonContext::InitDistributed(mpi_config); REQUIRE(l_rank == ctx->GetRank()); if (color == 0) { @@ -71,11 +67,7 @@ TEST_CASE("custom mpi communicator") { REQUIRE(MPI_Comm_split(MPI_COMM_WORLD, color, l_rank, &new_comm) == MPI_SUCCESS); mpi_config = cylon::net::MPIConfig::Make(new_comm); - - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return; - } + ctx = cylon::CylonContext::InitDistributed(mpi_config); REQUIRE(l_rank == ctx->GetRank()); REQUIRE(ctx->GetWorldSize() == 2); diff --git a/cpp/test/equal_test.cpp b/cpp/test/equal_test.cpp index 78193db0d..5dae01fe9 100644 --- a/cpp/test/equal_test.cpp +++ b/cpp/test/equal_test.cpp @@ -109,11 +109,9 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result)); - printf("testing unordered equal\n"); REQUIRE(result); CHECK_CYLON_STATUS(DistributedEquals(table2, table3, result)); - printf("testing unordered equal\n"); REQUIRE(!result); } @@ -121,7 +119,6 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result, false)); - printf("testing unordered equal\n"); REQUIRE(result); } @@ -129,7 +126,6 @@ TEST_CASE("Distributed equal testing - reparition needed", "[distributed equal]" if(table1->GetContext()->GetWorldSize() != 2) return; bool result; CHECK_CYLON_STATUS(DistributedEquals(table2, table3, result, false)); - printf("testing unordered equal\n"); REQUIRE(result); } } From 2a4a1dd2eec2441047c47a80cd943c25d3ee8dcb Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Tue, 19 Jul 2022 14:12:49 -0400 Subject: [PATCH 05/28] [Cylon] Implement Slice with new Logic Signed-off-by: Arup Sarker --- cpp/src/cylon/table.cpp | 102 ++++++++++++++++++++++++++++- cpp/src/examples/head_example.cpp | 2 +- cpp/src/examples/slice_example.cpp | 2 +- cpp/src/examples/tail_example.cpp | 2 +- cpp/test/slice_test.cpp | 96 +++++++++++++++++++++++++++ 5 files changed, 200 insertions(+), 4 deletions(-) create mode 100644 cpp/test/slice_test.cpp diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index d67f34086..69fef590a 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -1815,6 +1816,106 @@ Status Distributed_Slice(const std::shared_ptr &in, int64_t offset auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); + + int64_t total_partition = sizes_cols->length(); + LOG(INFO) << "Total Length: " << total_partition; + sizes.resize(sizes_cols->length()); + std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); + + int64_t minLimit = 0; + int64_t rank = ctx->GetRank(); + int64_t prod = std::accumulate(data_ptr, data_ptr + rank, minLimit); + + LOG(INFO) << "Current Partion: " << rank << " and Size: " << sizes[rank]; + int64_t curSize = sizes[rank]; + int64_t minx = std::min(offset - prod, curSize); + + + + int64_t x = std::max(minLimit, minx); + int64_t x_maxy = std::max(offset + total_partition - prod, minLimit); + int64_t y = std::min(curSize, x_maxy); + + //$x = max(0, min(K-L_i, l_i)) + //$x+y = min(l_i, max(K+L-L_i, 0) + + /* std::vector offset_vec; + std::vector length_vec; + + if(!order) { + LOG(INFO) << "from 0 to size"; + for(int i = 0; i < total_partition; i++) { + if(offset + length > sizes[i]) { + if(sizes[i] - offset <= 0) + { + offset_vec.push_back(sizes[i]); + length_vec.push_back(0); + offset = 0; + } + else { + offset_vec.push_back(offset); + length_vec.push_back(sizes[i] - offset); + + length = length - sizes[i] + offset; + offset = 0; + } + } + else { + offset_vec.push_back(offset); + length_vec.push_back(length); + length = 0; + offset = 0; + } + } + } + else { + LOG(INFO) << "from size to 0"; + for(int i = 0; i < total_partition; i++){ + if(length > sizes[i]) { + offset_vec.push_back(0); + length_vec.push_back(sizes[i]); + + length = length - sizes[i]; + } + else { + offset_vec.push_back(sizes[i] - length); + length_vec.push_back(length); + length = 0; + } + } + } + */ + + out_table = in_table->Slice(x, y); + + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), out); +} + + +Status Distributed_Slice1(const std::shared_ptr &in, int64_t offset, int64_t length, + std::shared_ptr &out, int order) { + + const auto &ctx = in->GetContext(); + std::shared_ptr out_table, in_table = in->get_table(); + auto num_row = in->Rows(); + + if (!in->Empty()) { + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); auto *data_ptr = @@ -1886,7 +1987,6 @@ Status Distributed_Slice(const std::shared_ptr &in, int64_t offset - /** * Head the part of table to create a single table with specific number of rows * @param tables, number of rows diff --git a/cpp/src/examples/head_example.cpp b/cpp/src/examples/head_example.cpp index 4de9db165..070e00e80 100644 --- a/cpp/src/examples/head_example.cpp +++ b/cpp/src/examples/head_example.cpp @@ -105,7 +105,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = head_table->ColumnNames(); head_table->Print(); - sleep(1); + sleep(3); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index bdd721407..820159678 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -113,7 +113,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = sliced->ColumnNames(); sliced->Print(); - sleep(1); + sleep(3); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/tail_example.cpp b/cpp/src/examples/tail_example.cpp index b193499ef..8ccdbe6c3 100644 --- a/cpp/src/examples/tail_example.cpp +++ b/cpp/src/examples/tail_example.cpp @@ -105,7 +105,7 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = tail_table->ColumnNames(); tail_table->Print(); - sleep(1); + sleep(3); ctx->Finalize(); return 0; } diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp new file mode 100644 index 000000000..616375cf0 --- /dev/null +++ b/cpp/test/slice_test.cpp @@ -0,0 +1,96 @@ +/* +* Licensed under the Apache License, Version 2.0 (the "License"); +* you may not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +#include "common/test_header.hpp" +#include "test_utils.hpp" +#include + +namespace cylon { +namespace test { +TEST_CASE("Slice testing", "[equal]") { + std::string path1 = "../data/input/csv1_0.csv"; + std::string path2 = "../data/input/csv1_1.csv"; + std::string path3 = "../data/input/csv1_0_shuffled.csv"; + std::string path4 = "../data/input/csv1_0_col_order_change.csv"; + std::shared_ptr
table1, table2, table3, table4; + + auto read_options = io::config::CSVReadOptions().UseThreads(false); + + CHECK_CYLON_STATUS(FromCSV(ctx, std::vector{path1, path2, path3, path4}, + std::vector *>{&table1, &table2, &table3, &table4}, + read_options)); + + SECTION("testing ordered equal") { + bool result; + CHECK_CYLON_STATUS(Equals(table1, table1, result)); + REQUIRE(result == true); + + CHECK_CYLON_STATUS(Equals(table1, table2, result)); + REQUIRE(result == false); + + CHECK_CYLON_STATUS(Equals(table1, table3, result)); + REQUIRE(result == false); + + CHECK_CYLON_STATUS(Equals(table1, table4, result)); + REQUIRE(result == false); + } + + SECTION("testing unordered equal") { + bool result; + CHECK_CYLON_STATUS(Equals(table1, table1, result, false)); + REQUIRE(result == true); + + CHECK_CYLON_STATUS(Equals(table1, table2, result, false)); + REQUIRE(result == false); + + CHECK_CYLON_STATUS(Equals(table1, table3, result, false)); + REQUIRE(result == true); + + CHECK_CYLON_STATUS(Equals(table1, table4, result, false)); + REQUIRE(result == false); + } +} + +TEST_CASE("Distributed Slice testing", "[distributed slice]") { + std::string path1 = "../data/input/csv1_" + std::to_string(RANK) +".csv"; + std::string path2 = "../data/input/csv2_" + std::to_string(RANK) +".csv"; + std::shared_ptr
table1, table2; + + auto read_options = io::config::CSVReadOptions().UseThreads(false); + + CHECK_CYLON_STATUS(FromCSV(ctx, std::vector{path1, path2}, + std::vector *>{&table1, &table2}, + read_options)); + + SECTION("testing ordered equal") { + bool result; + CHECK_CYLON_STATUS(DistributedEquals(table1, table1, result)); + REQUIRE(result == true); + + CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result)); + REQUIRE(result == false); + } + + SECTION("testing unordered equal") { + bool result; + CHECK_CYLON_STATUS(DistributedEquals(table1, table1, result, false)); + REQUIRE(result == true); + + CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result, false)); + REQUIRE(result == false); + } +} + +} +} \ No newline at end of file From 3e399748268056c8c7e9e7bf889baa2bdee41129 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Wed, 20 Jul 2022 22:50:18 -0400 Subject: [PATCH 06/28] [Cylon] Implement new logic for slice and add a basic test case Signed-off-by: Arup Sarker --- cpp/src/cylon/table.cpp | 197 ++++------------------------- cpp/src/cylon/table.hpp | 6 +- cpp/src/examples/slice_example.cpp | 9 +- cpp/test/CMakeLists.txt | 5 + cpp/test/slice_test.cpp | 51 ++------ 5 files changed, 50 insertions(+), 218 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 69fef590a..39c7ec6b4 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1794,199 +1794,60 @@ Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t len /** - * Distributed_Slice the part of table to create a single table + * DistributedSlice the part of table to create a single table * @param table, offset and length * @return new sliced table */ -Status Distributed_Slice(const std::shared_ptr &in, int64_t offset, int64_t length, - std::shared_ptr &out, int order) { +Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, + std::shared_ptr &out) { const auto &ctx = in->GetContext(); std::shared_ptr out_table, in_table = in->get_table(); auto num_row = in->Rows(); - - if (!in->Empty()) { - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); - - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); - - int64_t total_partition = sizes_cols->length(); - LOG(INFO) << "Total Length: " << total_partition; - sizes.resize(sizes_cols->length()); - std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); - - int64_t minLimit = 0; - int64_t rank = ctx->GetRank(); - int64_t prod = std::accumulate(data_ptr, data_ptr + rank, minLimit); - - LOG(INFO) << "Current Partion: " << rank << " and Size: " << sizes[rank]; - int64_t curSize = sizes[rank]; - int64_t minx = std::min(offset - prod, curSize); + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - int64_t x = std::max(minLimit, minx); - int64_t x_maxy = std::max(offset + total_partition - prod, minLimit); - int64_t y = std::min(curSize, x_maxy); - - //$x = max(0, min(K-L_i, l_i)) - //$x+y = min(l_i, max(K+L-L_i, 0) - - /* std::vector offset_vec; - std::vector length_vec; - - if(!order) { - LOG(INFO) << "from 0 to size"; - for(int i = 0; i < total_partition; i++) { - if(offset + length > sizes[i]) { - if(sizes[i] - offset <= 0) - { - offset_vec.push_back(sizes[i]); - length_vec.push_back(0); - offset = 0; - } - else { - offset_vec.push_back(offset); - length_vec.push_back(sizes[i] - offset); - - length = length - sizes[i] + offset; - offset = 0; - } - } - else { - offset_vec.push_back(offset); - length_vec.push_back(length); - length = 0; - offset = 0; - } - } - } - else { - LOG(INFO) << "from size to 0"; - for(int i = 0; i < total_partition; i++){ - if(length > sizes[i]) { - offset_vec.push_back(0); - length_vec.push_back(sizes[i]); - - length = length - sizes[i]; - } - else { - offset_vec.push_back(sizes[i] - length); - length_vec.push_back(length); - length = 0; - } - } - } - */ - - out_table = in_table->Slice(x, y); - } else { - out_table = in_table; - } - return Table::FromArrowTable(ctx, std::move(out_table), out); -} + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); -Status Distributed_Slice1(const std::shared_ptr &in, int64_t offset, int64_t length, - std::shared_ptr &out, int order) { + int64_t L = length; + LOG(INFO) << "Total Length: " << L; + sizes.resize(sizes_cols->length()); + std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); - const auto &ctx = in->GetContext(); - std::shared_ptr out_table, in_table = in->get_table(); - auto num_row = in->Rows(); + //$x = max(zero_0, min(K-L_i, sl_i)) + //$x+y = min(sl_i, max(K+L-L_i, zero_0)) - if (!in->Empty()) { + int64_t K = offset; + int64_t zero_0 = 0; + int64_t rank = ctx->GetRank(); + int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + LOG(INFO) << "Current Partion: " << rank << " and Size: " << sizes[rank] << "L_i: " << L_i; - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - + int64_t sl_i = *(data_ptr + rank); - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); + int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); + int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)); - int total_partition = sizes_cols->length(); - LOG(INFO) << "Total Length: " << total_partition; - sizes.resize(sizes_cols->length()); - std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); - - - int current_partition = ctx->GetRank(); - LOG(INFO) << "Current Partion: " << current_partition << " and Size: " << sizes[current_partition]; - - std::vector offset_vec; - std::vector length_vec; - - if(!order) { - LOG(INFO) << "from 0 to size"; - for(int i = 0; i < total_partition; i++) { - if(offset + length > sizes[i]) { - if(sizes[i] - offset <= 0) - { - offset_vec.push_back(sizes[i]); - length_vec.push_back(0); - offset = 0; - } - else { - offset_vec.push_back(offset); - length_vec.push_back(sizes[i] - offset); - length = length - sizes[i] + offset; - offset = 0; - } - } - else { - offset_vec.push_back(offset); - length_vec.push_back(length); - length = 0; - offset = 0; - } - } - } - else { - LOG(INFO) << "from size to 0"; - for(int i = 0; i < total_partition; i++){ - if(length > sizes[i]) { - offset_vec.push_back(0); - length_vec.push_back(sizes[i]); - - length = length - sizes[i]; - } - else { - offset_vec.push_back(sizes[i] - length); - length_vec.push_back(length); - length = 0; - } - } - } - - out_table = in_table->Slice(offset_vec[current_partition], length_vec[current_partition]); + out_table = in_table->Slice(x, y); - } else { - out_table = in_table; - } return Table::FromArrowTable(ctx, std::move(out_table), out); } - /** * Head the part of table to create a single table with specific number of rows * @param tables, number of rows @@ -2010,10 +1871,9 @@ Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, s std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - int order = 0; if(num_rows > 0 && table_size > 0) { - return Distributed_Slice(table, 0, num_rows, output, order); + return DistributedSlice(table, 0, num_rows, output); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); @@ -2045,11 +1905,10 @@ Status Distributed_Tail(const std::shared_ptr
&table, int64_t num_rows, s std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - int order = 1; LOG(INFO) << "Input Table size " << table_size; if(num_rows > 0 && table_size > 0) { - return Distributed_Slice(table, table_size-num_rows, num_rows, output, order); + return DistributedSlice(table, table_size-num_rows, num_rows, output); } else LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index 82523c7bb..8366ff198 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -490,14 +490,14 @@ Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t len std::shared_ptr &out); /** - * Distributed_Slice the part of table to create a single table + * DistributedSlice the part of table to create a single table * @param table, offset and length * @return new sliced table */ -Status Distributed_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out, int order); +Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out); /** diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index 820159678..7ad5d1b89 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -93,12 +93,11 @@ int main(int argc, char *argv[]) { "r_"); cylon::Status status; - // Arup: Code block for slice operation - int order = 0; + // Code block for slice operation if (ops) { status = cylon::Local_Slice(in_table, offset, length, sliced); } else { - status = cylon::Distributed_Slice(in_table, offset, length, sliced, order); + status = cylon::DistributedSlice(in_table, offset, length, sliced); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; @@ -106,14 +105,14 @@ int main(int argc, char *argv[]) { return 1; } auto slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Sliced table has : " << sliced->Rows(); + LOG(INFO) << ctx->GetRank() << " Partition with " << "Sliced table has : " << sliced->Rows(); LOG(INFO) << "Sliced is done in " << std::chrono::duration_cast( slice_end_time - read_end_time).count() << "[ms]"; std::vector sliced_column_names = sliced->ColumnNames(); sliced->Print(); - sleep(3); + sleep(2); ctx->Finalize(); return 0; } diff --git a/cpp/test/CMakeLists.txt b/cpp/test/CMakeLists.txt index b709ec1c8..2696d1650 100644 --- a/cpp/test/CMakeLists.txt +++ b/cpp/test/CMakeLists.txt @@ -154,6 +154,11 @@ cylon_add_test(equal_test) cylon_run_test(equal_test 1 mpi) cylon_run_test(equal_test 2 mpi) +# slice test +cylon_add_test(slice_test) +cylon_run_test(slice_test 1 mpi) +cylon_run_test(slice_test 2 mpi) + # flatten array test cylon_add_test(flatten_array_test) cylon_run_test(flatten_array_test 1 mpi) diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 616375cf0..6b2ffdbae 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -23,7 +23,7 @@ TEST_CASE("Slice testing", "[equal]") { std::string path2 = "../data/input/csv1_1.csv"; std::string path3 = "../data/input/csv1_0_shuffled.csv"; std::string path4 = "../data/input/csv1_0_col_order_change.csv"; - std::shared_ptr
table1, table2, table3, table4; + std::shared_ptr
table1, table2, table3, table4, out; auto read_options = io::config::CSVReadOptions().UseThreads(false); @@ -31,41 +31,22 @@ TEST_CASE("Slice testing", "[equal]") { std::vector *>{&table1, &table2, &table3, &table4}, read_options)); - SECTION("testing ordered equal") { - bool result; - CHECK_CYLON_STATUS(Equals(table1, table1, result)); - REQUIRE(result == true); + SECTION("Testing Local Slice") { - CHECK_CYLON_STATUS(Equals(table1, table2, result)); - REQUIRE(result == false); + CHECK_CYLON_STATUS(Local_Slice(table1, 13, 8, out)); - CHECK_CYLON_STATUS(Equals(table1, table3, result)); - REQUIRE(result == false); + CHECK_CYLON_STATUS(Local_Slice(table2, 15, 5, out)); - CHECK_CYLON_STATUS(Equals(table1, table4, result)); - REQUIRE(result == false); - } - - SECTION("testing unordered equal") { - bool result; - CHECK_CYLON_STATUS(Equals(table1, table1, result, false)); - REQUIRE(result == true); - - CHECK_CYLON_STATUS(Equals(table1, table2, result, false)); - REQUIRE(result == false); - - CHECK_CYLON_STATUS(Equals(table1, table3, result, false)); - REQUIRE(result == true); + CHECK_CYLON_STATUS(Local_Slice(table3, 0, 10, out)); - CHECK_CYLON_STATUS(Equals(table1, table4, result, false)); - REQUIRE(result == false); + CHECK_CYLON_STATUS(Local_Slice(table4, 2, 15, out)); } } TEST_CASE("Distributed Slice testing", "[distributed slice]") { std::string path1 = "../data/input/csv1_" + std::to_string(RANK) +".csv"; std::string path2 = "../data/input/csv2_" + std::to_string(RANK) +".csv"; - std::shared_ptr
table1, table2; + std::shared_ptr
table1, table2, out; auto read_options = io::config::CSVReadOptions().UseThreads(false); @@ -73,22 +54,10 @@ TEST_CASE("Distributed Slice testing", "[distributed slice]") { std::vector *>{&table1, &table2}, read_options)); - SECTION("testing ordered equal") { - bool result; - CHECK_CYLON_STATUS(DistributedEquals(table1, table1, result)); - REQUIRE(result == true); - - CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result)); - REQUIRE(result == false); - } - - SECTION("testing unordered equal") { - bool result; - CHECK_CYLON_STATUS(DistributedEquals(table1, table1, result, false)); - REQUIRE(result == true); + SECTION("Testing Distributed Slice") { + CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, out)); - CHECK_CYLON_STATUS(DistributedEquals(table1, table2, result, false)); - REQUIRE(result == false); + CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, out)); } } From db5595c8e9acb7c8f30eb9d6c6c50d89d094e744 Mon Sep 17 00:00:00 2001 From: arupcsedu Date: Thu, 21 Jul 2022 14:22:56 -0400 Subject: [PATCH 07/28] Update cpp/src/cylon/table.cpp Co-authored-by: niranda perera --- cpp/src/cylon/table.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 39c7ec6b4..e638ffdba 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1891,7 +1891,6 @@ Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_p std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - LOG(INFO) << "Input Table size " << table_size; if(num_rows > 0 && table_size > 0) { return Local_Slice(table, table_size-num_rows, num_rows, output); From bf286da1b57600f10b7b1484576e0176837d69b5 Mon Sep 17 00:00:00 2001 From: arupcsedu Date: Thu, 21 Jul 2022 14:24:24 -0400 Subject: [PATCH 08/28] Update cpp/src/cylon/table.cpp Co-authored-by: niranda perera --- cpp/src/cylon/table.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index e638ffdba..fb9d47424 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1832,9 +1832,6 @@ Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - - LOG(INFO) << "Current Partion: " << rank << " and Size: " << sizes[rank] << "L_i: " << L_i; - int64_t sl_i = *(data_ptr + rank); From adca09e5d43187c8ca3f14952295279ea0747754 Mon Sep 17 00:00:00 2001 From: arupcsedu Date: Thu, 21 Jul 2022 14:25:21 -0400 Subject: [PATCH 09/28] Update cpp/src/cylon/table.cpp Co-authored-by: niranda perera --- cpp/src/cylon/table.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index fb9d47424..2c7b0bfe7 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1821,13 +1821,6 @@ Status DistributedSlice(const std::shared_ptr &in, int64_t offset, ->raw_values(); int64_t L = length; - LOG(INFO) << "Total Length: " << L; - sizes.resize(sizes_cols->length()); - std::copy(data_ptr, data_ptr + sizes_cols->length(), sizes.data()); - - //$x = max(zero_0, min(K-L_i, sl_i)) - //$x+y = min(sl_i, max(K+L-L_i, zero_0)) - int64_t K = offset; int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); From 62fc15935deb81ceb12ffaa129d55ab41b2f377b Mon Sep 17 00:00:00 2001 From: arupcsedu Date: Thu, 21 Jul 2022 14:28:25 -0400 Subject: [PATCH 10/28] Update cpp/src/examples/CMakeLists.txt Co-authored-by: niranda perera --- cpp/src/examples/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/src/examples/CMakeLists.txt b/cpp/src/examples/CMakeLists.txt index 6d7f74cc8..510e3a1c8 100644 --- a/cpp/src/examples/CMakeLists.txt +++ b/cpp/src/examples/CMakeLists.txt @@ -76,8 +76,6 @@ cylon_add_exe(parquet_union_example) cylon_add_exe(parquet_join_example) cylon_add_exe(dist_sort_example) cylon_add_exe(slice_example) -cylon_add_exe(head_example) -cylon_add_exe(tail_example) if (CYLON_UCX) cylon_add_exe(ucx_join_example) From bd47628ce9b94b674f31d79a0437d5bc6ec5484f Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Thu, 21 Jul 2022 15:04:47 -0400 Subject: [PATCH 11/28] [Cylon] Removed unneccessary data copy and logs Signed-off-by: Arup Sarker --- cpp/src/cylon/table.cpp | 29 ++++++++++++++++------------- cpp/src/cylon/table.hpp | 8 ++++---- cpp/src/examples/head_example.cpp | 5 ++--- cpp/src/examples/slice_example.cpp | 3 +-- cpp/src/examples/tail_example.cpp | 5 ++--- cpp/test/slice_test.cpp | 8 ++++---- 6 files changed, 29 insertions(+), 29 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 2c7b0bfe7..4e7c34000 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1773,13 +1773,13 @@ Status WriteParquet(const std::shared_ptr &ctx_, } /** - * Local_Slice the part of table to create a single table + * LocalSlice the part of table to create a single table * @param table, offset and length * @return new sliced table */ -Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, +Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out) { const auto &ctx = in->GetContext(); std::shared_ptr out_table, in_table = in->get_table(); @@ -1825,6 +1825,11 @@ Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); +<<<<<<< HEAD +======= + + +>>>>>>> [Cylon] Removed unneccessary data copy and logs int64_t sl_i = *(data_ptr + rank); @@ -1850,14 +1855,14 @@ Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_p const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return Local_Slice(table, 0, num_rows, output); + return LocalSlice(table, 0, num_rows, output); } else - LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + return cylon::Status(Code::IOError); } -Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { +Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); @@ -1866,7 +1871,7 @@ Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, s return DistributedSlice(table, 0, num_rows, output); } else - LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + return cylon::Status(Code::IOError); } @@ -1880,27 +1885,25 @@ Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_p std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - - + if(num_rows > 0 && table_size > 0) { - return Local_Slice(table, table_size-num_rows, num_rows, output); + return LocalSlice(table, table_size-num_rows, num_rows, output); } else - LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + return cylon::Status(Code::IOError); } -Status Distributed_Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { +Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - LOG(INFO) << "Input Table size " << table_size; if(num_rows > 0 && table_size > 0) { return DistributedSlice(table, table_size-num_rows, num_rows, output); } else - LOG_AND_RETURN_ERROR(Code::ValueError, "Number of row should be greater than 0"); + return cylon::Status(Code::IOError); } diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index 8366ff198..dad3fb168 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -482,11 +482,11 @@ Status WriteParquet(const std::shared_ptr &ctx, const std::string &path, const io::config::ParquetOptions &options = cylon::io::config::ParquetOptions()); /** - * Local_Slice the part of table to create a single table + * LocalSlice the part of table to create a single table * @param tables, offset, length * @return new sliced table */ -Status Local_Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, +Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out); /** @@ -508,7 +508,7 @@ Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_ Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); -Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, +Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); /** @@ -520,7 +520,7 @@ Status Distributed_Head(const std::shared_ptr
&table, int64_t num_rows, Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); -Status Distributed_Tail(const std::shared_ptr
&table, int64_t num_rows, +Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output); #endif // BUILD_CYLON_PARQUET diff --git a/cpp/src/examples/head_example.cpp b/cpp/src/examples/head_example.cpp index 070e00e80..df94b6f16 100644 --- a/cpp/src/examples/head_example.cpp +++ b/cpp/src/examples/head_example.cpp @@ -85,12 +85,12 @@ int main(int argc, char *argv[]) { "r_"); cylon::Status status; - // Arup: Code block for slice operation + //Code block for slice operation if (ops) { status = cylon::Head(in_table, num_rows, head_table); } else { - status = cylon::Distributed_Head(in_table, num_rows, head_table); + status = cylon::DistributedHead(in_table, num_rows, head_table); } if (!status.is_ok()) { LOG(INFO) << "Table Head is failed "; @@ -105,7 +105,6 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = head_table->ColumnNames(); head_table->Print(); - sleep(3); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index 7ad5d1b89..08bbdaccd 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -95,7 +95,7 @@ int main(int argc, char *argv[]) { // Code block for slice operation if (ops) { - status = cylon::Local_Slice(in_table, offset, length, sliced); + status = cylon::LocalSlice(in_table, offset, length, sliced); } else { status = cylon::DistributedSlice(in_table, offset, length, sliced); } @@ -112,7 +112,6 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = sliced->ColumnNames(); sliced->Print(); - sleep(2); ctx->Finalize(); return 0; } diff --git a/cpp/src/examples/tail_example.cpp b/cpp/src/examples/tail_example.cpp index 8ccdbe6c3..cd87c8980 100644 --- a/cpp/src/examples/tail_example.cpp +++ b/cpp/src/examples/tail_example.cpp @@ -85,12 +85,12 @@ int main(int argc, char *argv[]) { "r_"); cylon::Status status; - // Arup: Code block for slice operation + //Code block for slice operation if (ops) { status = cylon::Tail(in_table, num_rows, tail_table); } else { - status = cylon::Distributed_Tail(in_table, num_rows, tail_table); + status = cylon::DistributedTail(in_table, num_rows, tail_table); } if (!status.is_ok()) { LOG(INFO) << "Table Tail is failed "; @@ -105,7 +105,6 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = tail_table->ColumnNames(); tail_table->Print(); - sleep(3); ctx->Finalize(); return 0; } diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 6b2ffdbae..b2e9b1da1 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -33,13 +33,13 @@ TEST_CASE("Slice testing", "[equal]") { SECTION("Testing Local Slice") { - CHECK_CYLON_STATUS(Local_Slice(table1, 13, 8, out)); + CHECK_CYLON_STATUS(LocalSlice(table1, 13, 8, out)); - CHECK_CYLON_STATUS(Local_Slice(table2, 15, 5, out)); + CHECK_CYLON_STATUS(LocalSlice(table2, 15, 5, out)); - CHECK_CYLON_STATUS(Local_Slice(table3, 0, 10, out)); + CHECK_CYLON_STATUS(LocalSlice(table3, 0, 10, out)); - CHECK_CYLON_STATUS(Local_Slice(table4, 2, 15, out)); + CHECK_CYLON_STATUS(LocalSlice(table4, 2, 15, out)); } } From a194f539b6ac7aaa99d344ced72ba23fea6750ba Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Thu, 21 Jul 2022 15:11:49 -0400 Subject: [PATCH 12/28] [Cylon] Fix merge conflict Signed-off-by: Arup Sarker --- cpp/src/cylon/table.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 4e7c34000..f99cd4042 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1825,11 +1825,7 @@ Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); -<<<<<<< HEAD -======= - ->>>>>>> [Cylon] Removed unneccessary data copy and logs int64_t sl_i = *(data_ptr + rank); @@ -1885,7 +1881,7 @@ Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_p std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); - + if(num_rows > 0 && table_size > 0) { return LocalSlice(table, table_size-num_rows, num_rows, output); } From 60c50dffc41999d51d8442e3e1b992572f07621d Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Mon, 25 Jul 2022 23:36:24 -0400 Subject: [PATCH 13/28] [Cylon] Refactoring Slice logic into seperate file Signed-off-by: Arup Sarker --- cpp/src/cylon/CMakeLists.txt | 1 + cpp/src/cylon/indexing/slice.cpp | 169 +++++++++++++++++++++++++++++++ cpp/src/cylon/table.cpp | 130 ------------------------ 3 files changed, 170 insertions(+), 130 deletions(-) create mode 100644 cpp/src/cylon/indexing/slice.cpp diff --git a/cpp/src/cylon/CMakeLists.txt b/cpp/src/cylon/CMakeLists.txt index 59602311d..332ea6f7d 100644 --- a/cpp/src/cylon/CMakeLists.txt +++ b/cpp/src/cylon/CMakeLists.txt @@ -102,6 +102,7 @@ add_library(cylon SHARED indexing/index_utils.hpp indexing/indexer.cpp indexing/indexer.hpp + indexing/slice.cpp io/arrow_io.cpp io/arrow_io.hpp io/csv_read_config.cpp diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp new file mode 100644 index 000000000..3002e7755 --- /dev/null +++ b/cpp/src/cylon/indexing/slice.cpp @@ -0,0 +1,169 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace cylon { + /** + * LocalSlice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ + + + Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr &out) { + const auto &ctx = in->GetContext(); + std::shared_ptr out_table, in_table = in->get_table(); + + if (!in->Empty()) { + out_table = in_table->Slice(offset, length); + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), out); + } + + + /** + * DistributedSlice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ + + + Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, + std::shared_ptr &out) { + + const auto &ctx = in->GetContext(); + std::shared_ptr out_table, in_table = in->get_table(); + auto num_row = in->Rows(); + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + + + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); + + int64_t L = length; + int64_t K = offset; + int64_t zero_0 = 0; + int64_t rank = ctx->GetRank(); + int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); + + int64_t sl_i = *(data_ptr + rank); + + + int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); + int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)) - x; + + + out_table = in_table->Slice(x, y); + + return Table::FromArrowTable(ctx, std::move(out_table), out); + } + + + /** + * Head the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + + Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if(num_rows > 0 && table_size > 0) { + return LocalSlice(table, 0, num_rows, output); + } + else + return cylon::Status(Code::IOError); + + } + + Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if(num_rows > 0 && table_size > 0) { + return DistributedSlice(table, 0, num_rows, output); + } + else + return cylon::Status(Code::IOError); + + } + + /** + * Tail the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ + + Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if(num_rows > 0 && table_size > 0) { + return LocalSlice(table, table_size-num_rows, num_rows, output); + } + else + return cylon::Status(Code::IOError); + + } + + Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if(num_rows > 0 && table_size > 0) { + return DistributedSlice(table, table_size-num_rows, num_rows, output); + } + else + return cylon::Status(Code::IOError); + + } +} \ No newline at end of file diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index f99cd4042..97eb4876d 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1772,136 +1772,6 @@ Status WriteParquet(const std::shared_ptr &ctx_, return Status(Code::OK); } -/** - * LocalSlice the part of table to create a single table - * @param table, offset and length - * @return new sliced table - */ - - -Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out) { - const auto &ctx = in->GetContext(); - std::shared_ptr out_table, in_table = in->get_table(); - - if (!in->Empty()) { - out_table = in_table->Slice(offset, length); - } else { - out_table = in_table; - } - return Table::FromArrowTable(ctx, std::move(out_table), out); -} - - -/** - * DistributedSlice the part of table to create a single table - * @param table, offset and length - * @return new sliced table - */ - - -Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, - std::shared_ptr &out) { - - const auto &ctx = in->GetContext(); - std::shared_ptr out_table, in_table = in->get_table(); - auto num_row = in->Rows(); - - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); - - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); - - int64_t L = length; - int64_t K = offset; - int64_t zero_0 = 0; - int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - - int64_t sl_i = *(data_ptr + rank); - - - int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); - int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)); - - - out_table = in_table->Slice(x, y); - - return Table::FromArrowTable(ctx, std::move(out_table), out); -} - - -/** - * Head the part of table to create a single table with specific number of rows - * @param tables, number of rows - * @return new table - */ - -Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return LocalSlice(table, 0, num_rows, output); - } - else - return cylon::Status(Code::IOError); - -} - -Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, 0, num_rows, output); - } - else - return cylon::Status(Code::IOError); - -} - -/** - * Tail the part of table to create a single table with specific number of rows - * @param tables, number of rows - * @return new table - */ - -Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return LocalSlice(table, table_size-num_rows, num_rows, output); - } - else - return cylon::Status(Code::IOError); - -} - -Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, table_size-num_rows, num_rows, output); - } - else - return cylon::Status(Code::IOError); - -} #endif } // namespace cylon From 06d380dbcb2efc2cadbb7e7c6e7ed5874e84740d Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Wed, 27 Jul 2022 14:21:43 -0400 Subject: [PATCH 14/28] [Cylon] Update new test cases Signed-off-by: Arup Sarker --- cpp/test/slice_test.cpp | 122 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 122 insertions(+) diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index b2e9b1da1..591c86875 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -17,7 +17,129 @@ #include namespace cylon { + + +void testDistSlice(const std::vector& sort_cols, + const std::vector& sort_order, + std::shared_ptr
& global_table, + std::shared_ptr
& table) { + std::shared_ptr
out; + auto ctx = table->GetContext(); + std::shared_ptr arrow_output; + + CHECK_CYLON_STATUS(DistributedSort(table, sort_cols, out, sort_order, + {0, 0, SortOptions::INITIAL_SAMPLE})); + + std::vector> gathered; + CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, + &gathered)); + + if (RANK == 0) { + std::shared_ptr
exp, result; + // local sort the global table + CHECK_CYLON_STATUS(Sort(global_table, sort_cols, exp, sort_order)); + + CHECK_CYLON_STATUS(Merge(gathered, result)); + + CHECK_ARROW_EQUAL(exp->get_table(), result->get_table()); + } +} + namespace test { + +TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) { + auto type = default_type_instance(); + auto schema = arrow::schema({{arrow::field("a", type)}, + {arrow::field("b", arrow::float32())}}); + auto global_arrow_table = TableFromJSON(schema, {R"([{"a": 3, "b":0.025}, + {"a": 26, "b":0.394}, + {"a": 51, "b":0.755}, + {"a": 20, "b":0.030}, + {"a": 33, "b":0.318}, + {"a": 12, "b":0.813}, + {"a": 72, "b":0.968}, + {"a": 29, "b":0.291}, + {"a": 41, "b":0.519}, + {"a": 29, "b":0.291}, + {"a": 41, "b":0.519}, + {"a": 43, "b":0.419}, + {"a": 57, "b":0.153}, + {"a": 25, "b":0.479}, + {"a": 26, "b":0.676}, + {"a": 70, "b":0.504}, + {"a": 7, "b":0.232}, + {"a": 45, "b":0.734}, + {"a": 61, "b":0.685}, + {"a": 57, "b":0.314}, + {"a": 59, "b": 0.837}, + {"a": 67, "b": 0.086}, + {"a": 14, "b": 0.193}, + {"a": 21, "b": 0.853}, + {"a": 10, "b": 0.808}, + {"a": 13, "b": 0.085}, + {"a": 31, "b": 0.122}, + {"a": 20, "b": 0.689}, + {"a": 37, "b": 0.491}, + {"a": 62, "b": 0.262}, + {"a": 1 , "b": 0.868}, + {"a": 19, "b": 0.422}, + {"a": 64, "b": 0.528}, + {"a": 37, "b": 0.834}, + {"a": 33, "b": 0.010}, + {"a": 76, "b": 0.927}, + {"a": 4 , "b": 0.529}, + {"a": 13, "b": 0.201}, + {"a": 45, "b": 0.898}, + {"a": 67, "b": 0.407}])"}); + + int64_t rows_per_tab = global_arrow_table->num_rows() / WORLD_SZ; + std::shared_ptr
table1; + CHECK_CYLON_STATUS(Table::FromArrowTable( + ctx, global_arrow_table->Slice(RANK * rows_per_tab, rows_per_tab), + table1)); + std::shared_ptr
global_table; + LOG(INFO) << "HERE!!!"; + + CHECK_CYLON_STATUS( + Table::FromArrowTable(ctx, global_arrow_table, global_table)); + + SECTION("dist_slice_test_1_single_table") { + testDistSlice({0, 1}, {true, true}, global_table, table1); + } + + SECTION("dist_slice_test_2_multiple_table") { + testDistSlice({0, 1}, {true, false}, global_table, table1); + } + + SECTION("dist_sort_test_3_skipped_two_table") { + testDistSlice({1, 0}, {false, false}, global_table, table1); + } + + SECTION("dist_sort_test_4_one_empty_table") { + if (RANK == 0) { + auto pool = cylon::ToArrowPool(ctx); + + std::shared_ptr arrow_empty_table; + auto arrow_status = util::CreateEmptyTable(table1->get_table()->schema(), + &arrow_empty_table, pool); + auto empty_table = std::make_shared
(ctx, arrow_empty_table); + table1 = empty_table; + } + + std::shared_ptr
out, out2; + auto ctx = table1->GetContext(); + std::shared_ptr arrow_output; + auto status = DistributedSort(table1, {1, 0}, out, {0, 0}); + REQUIRE(status.is_ok()); + status = DistributedSort(table1, {1, 0}, out2, {0, 0}, + {0, 0, SortOptions::INITIAL_SAMPLE}); + REQUIRE(status.is_ok()); + bool eq; + status = DistributedEquals(out, out2, eq); + REQUIRE(eq); + } +} + TEST_CASE("Slice testing", "[equal]") { std::string path1 = "../data/input/csv1_0.csv"; std::string path2 = "../data/input/csv1_1.csv"; From 0f4acce2e1181d1951069a311707255c0c9d971f Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Fri, 29 Jul 2022 12:57:33 -0400 Subject: [PATCH 15/28] [Cylon] Add multiple test cases for slice operation Signed-off-by: Arup Sarker --- cpp/test/slice_test.cpp | 54 +++++++++++++++++++---------------------- 1 file changed, 25 insertions(+), 29 deletions(-) diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 591c86875..9bab22338 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -19,29 +19,28 @@ namespace cylon { -void testDistSlice(const std::vector& sort_cols, - const std::vector& sort_order, - std::shared_ptr
& global_table, - std::shared_ptr
& table) { - std::shared_ptr
out; +void testDistSlice(std::shared_ptr
& global_table, + std::shared_ptr
& table, + int64_t offset, + int64_t length) { + std::shared_ptr
out, global_out; auto ctx = table->GetContext(); std::shared_ptr arrow_output; - CHECK_CYLON_STATUS(DistributedSort(table, sort_cols, out, sort_order, - {0, 0, SortOptions::INITIAL_SAMPLE})); + CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, out)); std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, &gathered)); + + CHECK_CYLON_STATUS(LocalSlice(global_table, offset, length, global_out)); if (RANK == 0) { - std::shared_ptr
exp, result; - // local sort the global table - CHECK_CYLON_STATUS(Sort(global_table, sort_cols, exp, sort_order)); + std::shared_ptr
result; CHECK_CYLON_STATUS(Merge(gathered, result)); - CHECK_ARROW_EQUAL(exp->get_table(), result->get_table()); + CHECK_ARROW_EQUAL(global_out->get_table(), result->get_table()); } } @@ -104,40 +103,37 @@ TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) Table::FromArrowTable(ctx, global_arrow_table, global_table)); SECTION("dist_slice_test_1_single_table") { - testDistSlice({0, 1}, {true, true}, global_table, table1); + testDistSlice(global_table, table1, 2, 5); } SECTION("dist_slice_test_2_multiple_table") { - testDistSlice({0, 1}, {true, false}, global_table, table1); + testDistSlice(global_table, table1, 1, 15); } SECTION("dist_sort_test_3_skipped_two_table") { - testDistSlice({1, 0}, {false, false}, global_table, table1); + testDistSlice(global_table, table1, 15, 8); } SECTION("dist_sort_test_4_one_empty_table") { - if (RANK == 0) { - auto pool = cylon::ToArrowPool(ctx); - - std::shared_ptr arrow_empty_table; - auto arrow_status = util::CreateEmptyTable(table1->get_table()->schema(), - &arrow_empty_table, pool); - auto empty_table = std::make_shared
(ctx, arrow_empty_table); - table1 = empty_table; - } + + auto pool = cylon::ToArrowPool(ctx); + + std::shared_ptr arrow_empty_table; + auto arrow_status = util::CreateEmptyTable(table1->get_table()->schema(), + &arrow_empty_table, pool); + auto empty_table = std::make_shared
(ctx, arrow_empty_table); + table1 = empty_table; std::shared_ptr
out, out2; auto ctx = table1->GetContext(); std::shared_ptr arrow_output; - auto status = DistributedSort(table1, {1, 0}, out, {0, 0}); + auto status = DistributedSlice(table1, 3, 10, out); REQUIRE(status.is_ok()); - status = DistributedSort(table1, {1, 0}, out2, {0, 0}, - {0, 0, SortOptions::INITIAL_SAMPLE}); + status = DistributedSlice(table1, 15, 5, out2); REQUIRE(status.is_ok()); - bool eq; - status = DistributedEquals(out, out2, eq); - REQUIRE(eq); + CHECK_ARROW_EQUAL(out->get_table(), out2->get_table()); } + } TEST_CASE("Slice testing", "[equal]") { From 170d4c77718049f52c143aa336bcbcb0262f3f0e Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Tue, 2 Aug 2022 22:41:15 -0400 Subject: [PATCH 16/28] [Cylon] Refactoring slice operation Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 136 +++++++++--------- cpp/src/cylon/table.cpp | 7 - cpp/src/cylon/table.hpp | 4 +- .../flat_hash_map/bytell_hash_map.hpp | 1 - cpp/src/examples/join_example.cpp | 2 +- cpp/src/examples/slice_example.cpp | 49 ++++++- cpp/test/slice_test.cpp | 10 +- 7 files changed, 123 insertions(+), 86 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 3002e7755..56811fd48 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -37,23 +37,25 @@ namespace cylon { /** - * LocalSlice the part of table to create a single table + * Slice the part of table to create a single table * @param table, offset and length * @return new sliced table */ - Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, + Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out) { - const auto &ctx = in->GetContext(); - std::shared_ptr out_table, in_table = in->get_table(); - - if (!in->Empty()) { - out_table = in_table->Slice(offset, length); - } else { - out_table = in_table; - } - return Table::FromArrowTable(ctx, std::move(out_table), out); + const auto &ctx = in->GetContext(); + + std::shared_ptr out_table; + const auto& in_table = in->get_table(); + + if (!in->Empty()) { + out_table = in_table->Slice(offset, length); + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), out); } @@ -67,39 +69,37 @@ namespace cylon { Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, std::shared_ptr &out) { - const auto &ctx = in->GetContext(); - std::shared_ptr out_table, in_table = in->get_table(); - auto num_row = in->Rows(); - - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + const auto &ctx = in->GetContext(); + std::shared_ptr out_table; + auto num_row = in->Rows(); + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - int64_t L = length; - int64_t K = offset; - int64_t zero_0 = 0; - int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); - int64_t sl_i = *(data_ptr + rank); + int64_t L = length; + int64_t K = offset; + int64_t zero_0 = 0; + int64_t rank = ctx->GetRank(); + int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); + int64_t sl_i = *(data_ptr + rank); - int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); - int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)) - x; + int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); + int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)) - x; - out_table = in_table->Slice(x, y); - return Table::FromArrowTable(ctx, std::move(out_table), out); + return Slice(in, x, y, out); } @@ -111,27 +111,27 @@ namespace cylon { Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); - if(num_rows > 0 && table_size > 0) { - return LocalSlice(table, 0, num_rows, output); - } - else - return cylon::Status(Code::IOError); + if(num_rows > 0 && table_size > 0) { + return Slice(table, 0, num_rows, output); + } + else + return cylon::Status(Code::IOError); - } + } - Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); - if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, 0, num_rows, output); - } - else - return cylon::Status(Code::IOError); + if(num_rows > 0 && table_size > 0) { + return DistributedSlice(table, 0, num_rows, output); + } + else + return cylon::Status(Code::IOError); } @@ -143,27 +143,27 @@ namespace cylon { Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); - if(num_rows > 0 && table_size > 0) { - return LocalSlice(table, table_size-num_rows, num_rows, output); - } - else - return cylon::Status(Code::IOError); + if(num_rows > 0 && table_size > 0) { + return Slice(table, table_size-num_rows, num_rows, output); + } + else + return cylon::Status(Code::IOError); - } + } - Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); - if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, table_size-num_rows, num_rows, output); - } - else - return cylon::Status(Code::IOError); + if(num_rows > 0 && table_size > 0) { + return DistributedSlice(table, table_size-num_rows, num_rows, output); + } + else + return cylon::Status(Code::IOError); - } + } } \ No newline at end of file diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 97eb4876d..f373e4860 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -1473,11 +1473,8 @@ Status Equals(const std::shared_ptr &a, const std::shared_ptr &a, const std::shared_ptr &b, std::shared_ptr *b_out) { - LOG(INFO) << "I am at the RepartitionToMatchOtherTable"; int64_t num_row = a->Rows(); - LOG(INFO) << "Table-A size: " << num_row; - LOG(INFO) << "Table-B size: " << b->Rows(); if (num_row == 0) { *b_out = a; return Status::OK(); @@ -1488,10 +1485,8 @@ static Status RepartitionToMatchOtherTable(const std::shared_ptr & std::vector rows_per_partition; std::shared_ptr output; - LOG(INFO) << "Execute before allgather api in RepartitionToMatchOtherTable"; RETURN_CYLON_STATUS_IF_FAILED( a->GetContext()->GetCommunicator()->Allgather(num_row_scalar, &output)); - LOG(INFO) << "Execute after allgather api in RepartitionToMatchOtherTable"; auto *data_ptr = std::static_pointer_cast(output->data())->raw_values(); @@ -1508,8 +1503,6 @@ Status DistributedEquals(const std::shared_ptr &a, bool subResult; RETURN_CYLON_STATUS_IF_FAILED(VerifyTableSchema(a->get_table(), b->get_table())); - LOG(INFO) << "I am at the DistributedEquals"; - if (!ordered) { int col = a->Columns(); std::vector indices(col); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index dad3fb168..e940b9d31 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -482,11 +482,11 @@ Status WriteParquet(const std::shared_ptr &ctx, const std::string &path, const io::config::ParquetOptions &options = cylon::io::config::ParquetOptions()); /** - * LocalSlice the part of table to create a single table + * Slice the part of table to create a single table * @param tables, offset, length * @return new sliced table */ -Status LocalSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, +Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, std::shared_ptr &out); /** diff --git a/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp b/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp index 9314eb433..2cd3b1931 100644 --- a/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp +++ b/cpp/src/cylon/thridparty/flat_hash_map/bytell_hash_map.hpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index 7867d5a87..944d83da4 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -40,7 +40,7 @@ int main(int argc, char *argv[]) { return 1; } - std::shared_ptr first_table, second_table, joined, sliced; + std::shared_ptr first_table, second_table, joined; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index 08bbdaccd..f57fc8d5b 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -24,6 +24,7 @@ int main(int argc, char *argv[]) { + if ((argc < 6 && std::string(argv[1]) == "f")) { LOG(ERROR) << "./slice_example f [n | o] csv_file offset length" << std::endl << "./slice_example f [n | o] csv_file offset length" << std::endl; @@ -46,7 +47,7 @@ int main(int argc, char *argv[]) { } - std::shared_ptr in_table, joined, sliced; + std::shared_ptr in_table, joined, sliced, head_table, tail_table; auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; @@ -95,7 +96,7 @@ int main(int argc, char *argv[]) { // Code block for slice operation if (ops) { - status = cylon::LocalSlice(in_table, offset, length, sliced); + status = cylon::Slice(in_table, offset, length, sliced); } else { status = cylon::DistributedSlice(in_table, offset, length, sliced); } @@ -112,6 +113,50 @@ int main(int argc, char *argv[]) { std::vector sliced_column_names = sliced->ColumnNames(); sliced->Print(); + + int64_t num_rows = 10; + //Code block for head operation + + if (ops) { + status = cylon::Head(in_table, num_rows, head_table); + } else { + status = cylon::DistributedHead(in_table, num_rows, head_table); + } + if (!status.is_ok()) { + LOG(INFO) << "Table Head is failed "; + ctx->Finalize(); + return 1; + } + slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Head table has : " << head_table->Rows(); + LOG(INFO) << "Head is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + sliced_column_names = head_table->ColumnNames(); + + head_table->Print(); + + //Code block for tail operation + + if (ops) { + status = cylon::Tail(in_table, num_rows, tail_table); + } else { + status = cylon::DistributedTail(in_table, num_rows, tail_table); + } + if (!status.is_ok()) { + LOG(INFO) << "Table Tail is failed "; + ctx->Finalize(); + return 1; + } + slice_end_time = std::chrono::steady_clock::now(); + LOG(INFO) << "Tail table has : " << tail_table->Rows(); + LOG(INFO) << "Tail is done in " + << std::chrono::duration_cast( + slice_end_time - read_end_time).count() << "[ms]"; + sliced_column_names = tail_table->ColumnNames(); + + tail_table->Print(); + ctx->Finalize(); return 0; } diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 9bab22338..3266e42eb 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -33,7 +33,7 @@ void testDistSlice(std::shared_ptr
& global_table, CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, &gathered)); - CHECK_CYLON_STATUS(LocalSlice(global_table, offset, length, global_out)); + CHECK_CYLON_STATUS(Slice(global_table, offset, length, global_out)); if (RANK == 0) { std::shared_ptr
result; @@ -151,13 +151,13 @@ TEST_CASE("Slice testing", "[equal]") { SECTION("Testing Local Slice") { - CHECK_CYLON_STATUS(LocalSlice(table1, 13, 8, out)); + CHECK_CYLON_STATUS(Slice(table1, 13, 8, out)); - CHECK_CYLON_STATUS(LocalSlice(table2, 15, 5, out)); + CHECK_CYLON_STATUS(Slice(table2, 15, 5, out)); - CHECK_CYLON_STATUS(LocalSlice(table3, 0, 10, out)); + CHECK_CYLON_STATUS(Slice(table3, 0, 10, out)); - CHECK_CYLON_STATUS(LocalSlice(table4, 2, 15, out)); + CHECK_CYLON_STATUS(Slice(table4, 2, 15, out)); } } From 8c28aa810a27a2469addfe37e3e4d23668f344b2 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Wed, 3 Aug 2022 11:33:29 -0400 Subject: [PATCH 17/28] [Cylon] Fix error message and un-necessary example files Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 9 ++- cpp/src/examples/head_example.cpp | 110 ------------------------------ cpp/src/examples/tail_example.cpp | 110 ------------------------------ 3 files changed, 4 insertions(+), 225 deletions(-) delete mode 100644 cpp/src/examples/head_example.cpp delete mode 100644 cpp/src/examples/tail_example.cpp diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 56811fd48..2fb8902d6 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -118,8 +118,7 @@ namespace cylon { return Slice(table, 0, num_rows, output); } else - return cylon::Status(Code::IOError); - + return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); } Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { @@ -131,7 +130,7 @@ namespace cylon { return DistributedSlice(table, 0, num_rows, output); } else - return cylon::Status(Code::IOError); + return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); } @@ -150,7 +149,7 @@ namespace cylon { return Slice(table, table_size-num_rows, num_rows, output); } else - return cylon::Status(Code::IOError); + return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); } @@ -163,7 +162,7 @@ namespace cylon { return DistributedSlice(table, table_size-num_rows, num_rows, output); } else - return cylon::Status(Code::IOError); + return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); } } \ No newline at end of file diff --git a/cpp/src/examples/head_example.cpp b/cpp/src/examples/head_example.cpp deleted file mode 100644 index df94b6f16..000000000 --- a/cpp/src/examples/head_example.cpp +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include -#include - -#include -#include -#include -#include - -#include "example_utils.hpp" - - -int main(int argc, char *argv[]) { - if ((argc < 5 && std::string(argv[1]) == "f")) { - LOG(ERROR) << "./head_example f [n | o] csv_file num_rows" << std::endl - << "./head_example f [n | o] csv_file num_rows" << std::endl; - return 1; - } - - if ((argc < 6 && std::string(argv[1]) == "m")) { - LOG(ERROR) << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl - << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl; - return 1; - } - - auto start_start = std::chrono::steady_clock::now(); - auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } - - std::shared_ptr in_table, head_table; - auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); - cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; - - std::string mem = std::string(argv[1]); - std::string ops_param = std::string(argv[2]); - int64_t num_rows = 0; - - bool ops = true; - if (ops_param == "o") { - ops = true; - } else if (ops_param == "n") { - ops = false; - } - - if (mem == "m") { - uint64_t count = std::stoull(argv[3]); - double dup = std::stod(argv[4]); - cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); - num_rows = std::stoull(argv[5]); - } else if (mem == "f") { - LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); - cylon::FromCSV(ctx, std::string(argv[3]), in_table); - - num_rows = std::stoull(argv[4]); - } - ctx->Barrier(); - auto read_end_time = std::chrono::steady_clock::now(); - //in_table->Print(); - LOG(INFO) << "Read tables in " - << std::chrono::duration_cast( - read_end_time - start_start).count() << "[ms]"; - - auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, - 0, - 0, - algorithm, - "l_", - "r_"); - cylon::Status status; - - //Code block for slice operation - - if (ops) { - status = cylon::Head(in_table, num_rows, head_table); - } else { - status = cylon::DistributedHead(in_table, num_rows, head_table); - } - if (!status.is_ok()) { - LOG(INFO) << "Table Head is failed "; - ctx->Finalize(); - return 1; - } - auto slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Head table has : " << head_table->Rows(); - LOG(INFO) << "Head is done in " - << std::chrono::duration_cast( - slice_end_time - read_end_time).count() << "[ms]"; - std::vector sliced_column_names = head_table->ColumnNames(); - - head_table->Print(); - ctx->Finalize(); - return 0; -} diff --git a/cpp/src/examples/tail_example.cpp b/cpp/src/examples/tail_example.cpp deleted file mode 100644 index cd87c8980..000000000 --- a/cpp/src/examples/tail_example.cpp +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#include -#include - -#include -#include -#include -#include - -#include "example_utils.hpp" - - -int main(int argc, char *argv[]) { - if ((argc < 5 && std::string(argv[1]) == "f")) { - LOG(ERROR) << "./head_example f [n | o] csv_file num_rows" << std::endl - << "./head_example f [n | o] csv_file num_rows" << std::endl; - return 1; - } - - if ((argc < 6 && std::string(argv[1]) == "m")) { - LOG(ERROR) << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl - << "./head_example m [n | o] num_tuples_per_worker 0.0-1.0 num_rows" << std::endl; - return 1; - } - - auto start_start = std::chrono::steady_clock::now(); - auto mpi_config = std::make_shared(); - std::shared_ptr ctx; - if (!cylon::CylonContext::InitDistributed(mpi_config, &ctx).is_ok()) { - std::cerr << "ctx init failed! " << std::endl; - return 1; - } - - std::shared_ptr in_table, tail_table; - auto read_options = cylon::io::config::CSVReadOptions().UseThreads(false).BlockSize(1 << 30); - cylon::join::config::JoinAlgorithm algorithm = cylon::join::config::JoinAlgorithm::SORT; - - std::string mem = std::string(argv[1]); - std::string ops_param = std::string(argv[2]); - int64_t num_rows = 0; - - bool ops = true; - if (ops_param == "o") { - ops = true; - } else if (ops_param == "n") { - ops = false; - } - - if (mem == "m") { - uint64_t count = std::stoull(argv[3]); - double dup = std::stod(argv[4]); - cylon::examples::create_in_memory_tables(count, dup,ctx,in_table); - num_rows = std::stoull(argv[5]); - } else if (mem == "f") { - LOG(INFO) << "Load From the CSV file" << std::string(argv[3]); - cylon::FromCSV(ctx, std::string(argv[3]), in_table); - - num_rows = std::stoull(argv[4]); - } - ctx->Barrier(); - auto read_end_time = std::chrono::steady_clock::now(); - //in_table->Print(); - LOG(INFO) << "Read tables in " - << std::chrono::duration_cast( - read_end_time - start_start).count() << "[ms]"; - - auto join_config = cylon::join::config::JoinConfig(cylon::join::config::JoinType::INNER, - 0, - 0, - algorithm, - "l_", - "r_"); - cylon::Status status; - - //Code block for slice operation - - if (ops) { - status = cylon::Tail(in_table, num_rows, tail_table); - } else { - status = cylon::DistributedTail(in_table, num_rows, tail_table); - } - if (!status.is_ok()) { - LOG(INFO) << "Table Tail is failed "; - ctx->Finalize(); - return 1; - } - auto slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Tail table has : " << tail_table->Rows(); - LOG(INFO) << "Tail is done in " - << std::chrono::duration_cast( - slice_end_time - read_end_time).count() << "[ms]"; - std::vector sliced_column_names = tail_table->ColumnNames(); - - tail_table->Print(); - ctx->Finalize(); - return 0; -} From 9ee391472df02811d483e42a032f7ba03e542c18 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Sat, 6 Aug 2022 12:37:57 -0400 Subject: [PATCH 18/28] [Cylon] Implement tail operation with new logic Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 36 +++++++++++++++++++++++++++--- cpp/src/examples/slice_example.cpp | 2 +- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 2fb8902d6..2e1b5c7b4 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -153,16 +153,46 @@ namespace cylon { } - Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, table_size-num_rows, num_rows, output); + const auto &ctx = table->GetContext(); + std::shared_ptr out_table; + auto num_row = table->Rows(); + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); + + + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + + auto *data_ptr = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); + + int64_t L = num_rows; + int64_t zero_0 = 0; + int64_t rank = ctx->GetRank(); + int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); + + int64_t sl_i = *(data_ptr + rank); + + + int64_t y = std::max(zero_0, std::min(L - L_i, sl_i)); + int64_t x = sl_i - y; + + + return Slice(table, x, y, output); } else return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); - } + } + } \ No newline at end of file diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index f57fc8d5b..b6cb57205 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -114,7 +114,7 @@ int main(int argc, char *argv[]) { sliced->Print(); - int64_t num_rows = 10; + int64_t num_rows = 30; //Code block for head operation if (ops) { From 1d54430ee86e539fffe9fe39a29aee6160a09695 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Wed, 10 Aug 2022 14:40:40 -0400 Subject: [PATCH 19/28] [Cylon] Implement Tail operation by fixing the logic Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 49 ++++++++++++++---------------- cpp/src/cylon/table.hpp | 2 +- cpp/src/examples/slice_example.cpp | 6 ++-- cpp/test/slice_test.cpp | 10 +++--- 4 files changed, 31 insertions(+), 36 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 2e1b5c7b4..45a3c9665 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -66,39 +66,41 @@ namespace cylon { */ - Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, - std::shared_ptr &out) { + Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, const long int *data_ptr, std::shared_ptr &out) { const auto &ctx = in->GetContext(); - std::shared_ptr out_table; - auto num_row = in->Rows(); - - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + auto *data_ptr_st = data_ptr; + + if(data_ptr_st == nullptr) { + std::shared_ptr out_table; + auto num_row = in->Rows(); + + std::vector sizes; + std::shared_ptr sizes_cols; + RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); + + auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); + data_ptr_st = + std::static_pointer_cast(sizes_cols->data()) + ->raw_values(); + } int64_t L = length; int64_t K = offset; int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); + int64_t L_i = std::accumulate(data_ptr_st, data_ptr_st + rank, zero_0); - int64_t sl_i = *(data_ptr + rank); + int64_t sl_i = *(data_ptr_st + rank); int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)) - x; - return Slice(in, x, y, out); } @@ -127,7 +129,7 @@ namespace cylon { const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, 0, num_rows, output); + return DistributedSlice(table, 0, num_rows, nullptr, output); } else return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); @@ -178,17 +180,10 @@ namespace cylon { int64_t L = num_rows; int64_t zero_0 = 0; - int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - - int64_t sl_i = *(data_ptr + rank); - - - int64_t y = std::max(zero_0, std::min(L - L_i, sl_i)); - int64_t x = sl_i - y; + int64_t L_g = std::accumulate(data_ptr, data_ptr + sizes_cols->length(), zero_0); - return Slice(table, x, y, output); + return DistributedSlice(table, L_g - L, L, data_ptr, output); } else return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index e940b9d31..68c6b720a 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -497,7 +497,7 @@ Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out); + const long int *data_ptr, std::shared_ptr &out); /** diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index b6cb57205..1397edce2 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -98,7 +98,7 @@ int main(int argc, char *argv[]) { if (ops) { status = cylon::Slice(in_table, offset, length, sliced); } else { - status = cylon::DistributedSlice(in_table, offset, length, sliced); + status = cylon::DistributedSlice(in_table, offset, length, nullptr, sliced); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; @@ -128,7 +128,7 @@ int main(int argc, char *argv[]) { return 1; } slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Head table has : " << head_table->Rows(); + LOG(INFO) << ctx->GetRank() << " Partition with " << "Head table has : " << head_table->Rows(); LOG(INFO) << "Head is done in " << std::chrono::duration_cast( slice_end_time - read_end_time).count() << "[ms]"; @@ -149,7 +149,7 @@ int main(int argc, char *argv[]) { return 1; } slice_end_time = std::chrono::steady_clock::now(); - LOG(INFO) << "Tail table has : " << tail_table->Rows(); + LOG(INFO) << ctx->GetRank() << " Partition with " << "Tail table has : " << tail_table->Rows(); LOG(INFO) << "Tail is done in " << std::chrono::duration_cast( slice_end_time - read_end_time).count() << "[ms]"; diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 3266e42eb..cc70aa526 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -27,7 +27,7 @@ void testDistSlice(std::shared_ptr
& global_table, auto ctx = table->GetContext(); std::shared_ptr arrow_output; - CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, out)); + CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, nullptr, out)); std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, @@ -127,9 +127,9 @@ TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) std::shared_ptr
out, out2; auto ctx = table1->GetContext(); std::shared_ptr arrow_output; - auto status = DistributedSlice(table1, 3, 10, out); + auto status = DistributedSlice(table1, 3, 10, nullptr, out); REQUIRE(status.is_ok()); - status = DistributedSlice(table1, 15, 5, out2); + status = DistributedSlice(table1, 15, 5, nullptr, out2); REQUIRE(status.is_ok()); CHECK_ARROW_EQUAL(out->get_table(), out2->get_table()); } @@ -173,9 +173,9 @@ TEST_CASE("Distributed Slice testing", "[distributed slice]") { read_options)); SECTION("Testing Distributed Slice") { - CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, out)); + CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, nullptr, out)); - CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, out)); + CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, nullptr, out)); } } From ffc70d3e39b4cb9d54ac761e5df7819d39ba2522 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Thu, 11 Aug 2022 18:28:07 -0400 Subject: [PATCH 20/28] Add sliceImple methods and clear unnecessary logs Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 28 +++++++++++++++++----------- cpp/src/cylon/table.hpp | 2 +- cpp/src/examples/join_example.cpp | 9 ++------- cpp/src/examples/slice_example.cpp | 2 +- cpp/test/slice_test.cpp | 10 +++++----- 5 files changed, 26 insertions(+), 25 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 45a3c9665..1a37899f5 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -65,13 +65,11 @@ namespace cylon { * @return new sliced table */ - - Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, const long int *data_ptr, std::shared_ptr &out) { + Status DistributedSliceImpl(const std::shared_ptr &in, int64_t offset, int64_t length, int64_t *data_ptr, std::shared_ptr &out) { const auto &ctx = in->GetContext(); - auto *data_ptr_st = data_ptr; - if(data_ptr_st == nullptr) { + if(data_ptr == nullptr) { std::shared_ptr out_table; auto num_row = in->Rows(); @@ -84,18 +82,18 @@ namespace cylon { RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - data_ptr_st = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); + data_ptr = + const_cast (std::static_pointer_cast(sizes_cols->data()) + ->raw_values()); } int64_t L = length; int64_t K = offset; int64_t zero_0 = 0; int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr_st, data_ptr_st + rank, zero_0); + int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - int64_t sl_i = *(data_ptr_st + rank); + int64_t sl_i = *(data_ptr + rank); int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); @@ -103,6 +101,14 @@ namespace cylon { return Slice(in, x, y, out); } + + + + Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, std::shared_ptr &out) { + + return DistributedSliceImpl(in, offset, length, nullptr, out); + + } /** @@ -129,7 +135,7 @@ namespace cylon { const int64_t table_size = in_table->num_rows(); if(num_rows > 0 && table_size > 0) { - return DistributedSlice(table, 0, num_rows, nullptr, output); + return DistributedSliceImpl(table, 0, num_rows, nullptr, output); } else return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); @@ -183,7 +189,7 @@ namespace cylon { int64_t L_g = std::accumulate(data_ptr, data_ptr + sizes_cols->length(), zero_0); - return DistributedSlice(table, L_g - L, L, data_ptr, output); + return DistributedSliceImpl(table, L_g - L, L, const_cast (data_ptr), output); } else return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index 68c6b720a..d2fcb9d31 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -497,7 +497,7 @@ Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, - const long int *data_ptr, std::shared_ptr &out); + std::shared_ptr &out); /** diff --git a/cpp/src/examples/join_example.cpp b/cpp/src/examples/join_example.cpp index 944d83da4..4130af99a 100644 --- a/cpp/src/examples/join_example.cpp +++ b/cpp/src/examples/join_example.cpp @@ -69,12 +69,8 @@ int main(int argc, char *argv[]) { double dup = std::stod(argv[4]); cylon::examples::create_two_in_memory_tables(count, dup,ctx,first_table,second_table); } else if (mem == "f") { - LOG(INFO) << "Load From first CSV file" << std::string(argv[3]); - cylon::FromCSV(ctx, std::string(argv[3]), first_table); - cylon::FromCSV(ctx, std::string(argv[4]), second_table); - - //cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); - //cylon::FromCSV(ctx, std::string(argv[4]) + std::to_string(ctx->GetRank()) + ".csv", second_table); + cylon::FromCSV(ctx, std::string(argv[3]) + std::to_string(ctx->GetRank()) + ".csv", first_table); + cylon::FromCSV(ctx, std::string(argv[4]) + std::to_string(ctx->GetRank()) + ".csv", second_table); if (argc == 6) { if (!strcmp(argv[5], "hash")) { @@ -87,7 +83,6 @@ int main(int argc, char *argv[]) { } ctx->Barrier(); auto read_end_time = std::chrono::steady_clock::now(); - //first_table->Print(); LOG(INFO) << "Read tables in " << std::chrono::duration_cast( read_end_time - start_start).count() << "[ms]"; diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index 1397edce2..b7c3f3327 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -98,7 +98,7 @@ int main(int argc, char *argv[]) { if (ops) { status = cylon::Slice(in_table, offset, length, sliced); } else { - status = cylon::DistributedSlice(in_table, offset, length, nullptr, sliced); + status = cylon::DistributedSlice(in_table, offset, length, sliced); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index cc70aa526..3266e42eb 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -27,7 +27,7 @@ void testDistSlice(std::shared_ptr
& global_table, auto ctx = table->GetContext(); std::shared_ptr arrow_output; - CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, nullptr, out)); + CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, out)); std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, @@ -127,9 +127,9 @@ TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) std::shared_ptr
out, out2; auto ctx = table1->GetContext(); std::shared_ptr arrow_output; - auto status = DistributedSlice(table1, 3, 10, nullptr, out); + auto status = DistributedSlice(table1, 3, 10, out); REQUIRE(status.is_ok()); - status = DistributedSlice(table1, 15, 5, nullptr, out2); + status = DistributedSlice(table1, 15, 5, out2); REQUIRE(status.is_ok()); CHECK_ARROW_EQUAL(out->get_table(), out2->get_table()); } @@ -173,9 +173,9 @@ TEST_CASE("Distributed Slice testing", "[distributed slice]") { read_options)); SECTION("Testing Distributed Slice") { - CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, nullptr, out)); + CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, out)); - CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, nullptr, out)); + CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, out)); } } From e02f598f2008b90719337ae1ecb363c5c05fc7b1 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Fri, 12 Aug 2022 13:12:27 -0400 Subject: [PATCH 21/28] [Cylon] Change the output table parameter from address to pointer Signed-off-by: Arup Sarker --- cpp/src/cylon/indexing/slice.cpp | 16 ++++++++-------- cpp/src/cylon/table.hpp | 12 ++++++------ cpp/src/examples/slice_example.cpp | 12 ++++++------ cpp/test/slice_test.cpp | 20 ++++++++++---------- 4 files changed, 30 insertions(+), 30 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 1a37899f5..3a368d77d 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -44,7 +44,7 @@ namespace cylon { Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out) { + std::shared_ptr *out) { const auto &ctx = in->GetContext(); std::shared_ptr out_table; @@ -55,7 +55,7 @@ namespace cylon { } else { out_table = in_table; } - return Table::FromArrowTable(ctx, std::move(out_table), out); + return Table::FromArrowTable(ctx, std::move(out_table), *out); } @@ -65,7 +65,7 @@ namespace cylon { * @return new sliced table */ - Status DistributedSliceImpl(const std::shared_ptr &in, int64_t offset, int64_t length, int64_t *data_ptr, std::shared_ptr &out) { + Status DistributedSliceImpl(const std::shared_ptr &in, int64_t offset, int64_t length, int64_t *data_ptr, std::shared_ptr *out) { const auto &ctx = in->GetContext(); @@ -104,7 +104,7 @@ namespace cylon { - Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, std::shared_ptr &out) { + Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, std::shared_ptr *out) { return DistributedSliceImpl(in, offset, length, nullptr, out); @@ -117,7 +117,7 @@ namespace cylon { * @return new table */ - Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); @@ -129,7 +129,7 @@ namespace cylon { return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); } - Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); @@ -148,7 +148,7 @@ namespace cylon { * @return new table */ - Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); @@ -161,7 +161,7 @@ namespace cylon { } - Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr &output) { + Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { std::shared_ptr in_table = table->get_table(); const int64_t table_size = in_table->num_rows(); diff --git a/cpp/src/cylon/table.hpp b/cpp/src/cylon/table.hpp index d2fcb9d31..cfd67113f 100644 --- a/cpp/src/cylon/table.hpp +++ b/cpp/src/cylon/table.hpp @@ -487,7 +487,7 @@ Status WriteParquet(const std::shared_ptr &ctx, * @return new sliced table */ Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out); + std::shared_ptr *out); /** * DistributedSlice the part of table to create a single table @@ -497,7 +497,7 @@ Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr &out); + std::shared_ptr *out); /** @@ -507,9 +507,9 @@ Status DistributedSlice(const std::shared_ptr
&in, int64_t offset, int64_ */ Status Head(const std::shared_ptr
&table, int64_t num_rows, - std::shared_ptr &output); + std::shared_ptr *output); Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, - std::shared_ptr &output); + std::shared_ptr *output); /** * Tail the part of table to create a single table with specific number of rows @@ -518,10 +518,10 @@ Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, */ Status Tail(const std::shared_ptr
&table, int64_t num_rows, - std::shared_ptr &output); + std::shared_ptr *output); Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, - std::shared_ptr &output); + std::shared_ptr *output); #endif // BUILD_CYLON_PARQUET diff --git a/cpp/src/examples/slice_example.cpp b/cpp/src/examples/slice_example.cpp index b7c3f3327..df2a254b3 100644 --- a/cpp/src/examples/slice_example.cpp +++ b/cpp/src/examples/slice_example.cpp @@ -96,9 +96,9 @@ int main(int argc, char *argv[]) { // Code block for slice operation if (ops) { - status = cylon::Slice(in_table, offset, length, sliced); + status = cylon::Slice(in_table, offset, length, &sliced); } else { - status = cylon::DistributedSlice(in_table, offset, length, sliced); + status = cylon::DistributedSlice(in_table, offset, length, &sliced); } if (!status.is_ok()) { LOG(INFO) << "Table Slice is failed "; @@ -118,9 +118,9 @@ int main(int argc, char *argv[]) { //Code block for head operation if (ops) { - status = cylon::Head(in_table, num_rows, head_table); + status = cylon::Head(in_table, num_rows, &head_table); } else { - status = cylon::DistributedHead(in_table, num_rows, head_table); + status = cylon::DistributedHead(in_table, num_rows, &head_table); } if (!status.is_ok()) { LOG(INFO) << "Table Head is failed "; @@ -139,9 +139,9 @@ int main(int argc, char *argv[]) { //Code block for tail operation if (ops) { - status = cylon::Tail(in_table, num_rows, tail_table); + status = cylon::Tail(in_table, num_rows, &tail_table); } else { - status = cylon::DistributedTail(in_table, num_rows, tail_table); + status = cylon::DistributedTail(in_table, num_rows, &tail_table); } if (!status.is_ok()) { LOG(INFO) << "Table Tail is failed "; diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index 3266e42eb..ddc6a546c 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -27,13 +27,13 @@ void testDistSlice(std::shared_ptr
& global_table, auto ctx = table->GetContext(); std::shared_ptr arrow_output; - CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, out)); + CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, &out)); std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, &gathered)); - CHECK_CYLON_STATUS(Slice(global_table, offset, length, global_out)); + CHECK_CYLON_STATUS(Slice(global_table, offset, length, &global_out)); if (RANK == 0) { std::shared_ptr
result; @@ -127,9 +127,9 @@ TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) std::shared_ptr
out, out2; auto ctx = table1->GetContext(); std::shared_ptr arrow_output; - auto status = DistributedSlice(table1, 3, 10, out); + auto status = DistributedSlice(table1, 3, 10, &out); REQUIRE(status.is_ok()); - status = DistributedSlice(table1, 15, 5, out2); + status = DistributedSlice(table1, 15, 5, &out2); REQUIRE(status.is_ok()); CHECK_ARROW_EQUAL(out->get_table(), out2->get_table()); } @@ -151,13 +151,13 @@ TEST_CASE("Slice testing", "[equal]") { SECTION("Testing Local Slice") { - CHECK_CYLON_STATUS(Slice(table1, 13, 8, out)); + CHECK_CYLON_STATUS(Slice(table1, 13, 8, &out)); - CHECK_CYLON_STATUS(Slice(table2, 15, 5, out)); + CHECK_CYLON_STATUS(Slice(table2, 15, 5, &out)); - CHECK_CYLON_STATUS(Slice(table3, 0, 10, out)); + CHECK_CYLON_STATUS(Slice(table3, 0, 10, &out)); - CHECK_CYLON_STATUS(Slice(table4, 2, 15, out)); + CHECK_CYLON_STATUS(Slice(table4, 2, 15, &out)); } } @@ -173,9 +173,9 @@ TEST_CASE("Distributed Slice testing", "[distributed slice]") { read_options)); SECTION("Testing Distributed Slice") { - CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, out)); + CHECK_CYLON_STATUS(DistributedSlice(table1, 10, 15, &out)); - CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, out)); + CHECK_CYLON_STATUS(DistributedSlice(table2, 12, 8, &out)); } } From f13b4e13f95c30cd0b72bdfb59664cbf7220184f Mon Sep 17 00:00:00 2001 From: niranda Date: Sun, 14 Aug 2022 12:11:22 -0400 Subject: [PATCH 22/28] Squashing following commits removing std out fixing errors more logs more logs adding logs attempting to fix macos error cosmetic changes cosmetic changes --- cpp/src/cylon/indexing/slice.cpp | 304 ++++++++++++++----------------- cpp/test/CMakeLists.txt | 1 + cpp/test/slice_test.cpp | 19 +- 3 files changed, 144 insertions(+), 180 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 3a368d77d..22b6b704c 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -12,188 +12,150 @@ * limitations under the License. */ -#include -#include -#include - -#include -#include #include -#include -#include #include +#include + #include -#include -#include -#include #include -#include -#include #include #include #include -#include namespace cylon { - /** - * Slice the part of table to create a single table - * @param table, offset and length - * @return new sliced table - */ - - - Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, - std::shared_ptr *out) { - const auto &ctx = in->GetContext(); - - std::shared_ptr out_table; - const auto& in_table = in->get_table(); - - if (!in->Empty()) { - out_table = in_table->Slice(offset, length); - } else { - out_table = in_table; - } - return Table::FromArrowTable(ctx, std::move(out_table), *out); - } - - - /** - * DistributedSlice the part of table to create a single table - * @param table, offset and length - * @return new sliced table - */ - - Status DistributedSliceImpl(const std::shared_ptr &in, int64_t offset, int64_t length, int64_t *data_ptr, std::shared_ptr *out) { - - const auto &ctx = in->GetContext(); - - if(data_ptr == nullptr) { - std::shared_ptr out_table; - auto num_row = in->Rows(); - - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); - - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - - data_ptr = - const_cast (std::static_pointer_cast(sizes_cols->data()) - ->raw_values()); - } - - int64_t L = length; - int64_t K = offset; - int64_t zero_0 = 0; - int64_t rank = ctx->GetRank(); - int64_t L_i = std::accumulate(data_ptr, data_ptr + rank, zero_0); - - int64_t sl_i = *(data_ptr + rank); - - - int64_t x = std::max(zero_0, std::min(K - L_i, sl_i)); - int64_t y = std::min(sl_i, std::max(K + L - L_i, zero_0)) - x; - - return Slice(in, x, y, out); - } - - - - Status DistributedSlice(const std::shared_ptr &in, int64_t offset, int64_t length, std::shared_ptr *out) { - - return DistributedSliceImpl(in, offset, length, nullptr, out); - - } - - /** - * Head the part of table to create a single table with specific number of rows - * @param tables, number of rows - * @return new table - */ +static constexpr int64_t kZero = 0; - Status Head(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return Slice(table, 0, num_rows, output); - } - else - return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); - } - - Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return DistributedSliceImpl(table, 0, num_rows, nullptr, output); - } - else - return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); - - } - - /** - * Tail the part of table to create a single table with specific number of rows - * @param tables, number of rows - * @return new table - */ - - Status Tail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - return Slice(table, table_size-num_rows, num_rows, output); - } - else - return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); - - } - - Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, std::shared_ptr *output) { - - std::shared_ptr in_table = table->get_table(); - const int64_t table_size = in_table->num_rows(); - - if(num_rows > 0 && table_size > 0) { - const auto &ctx = table->GetContext(); - std::shared_ptr out_table; - auto num_row = table->Rows(); - - std::vector sizes; - std::shared_ptr sizes_cols; - RETURN_CYLON_STATUS_IF_FAILED(Column::FromVector(sizes, sizes_cols)); - - auto num_row_scalar = std::make_shared(arrow::MakeScalar(num_row)); - - - RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator()->Allgather(num_row_scalar, &sizes_cols)); - - auto *data_ptr = - std::static_pointer_cast(sizes_cols->data()) - ->raw_values(); - - int64_t L = num_rows; - int64_t zero_0 = 0; - - int64_t L_g = std::accumulate(data_ptr, data_ptr + sizes_cols->length(), zero_0); - - return DistributedSliceImpl(table, L_g - L, L, const_cast (data_ptr), output); - } - else - return cylon::Status(Code::IOError, "Number of tailed row should be greater than zero with minimum table elements"); - - } +/** + * Slice the part of table to create a single table + * @param table, offset and length + * @return new sliced table + */ +Status Slice(const std::shared_ptr
&in, int64_t offset, int64_t length, + std::shared_ptr
*out) { + const auto &ctx = in->GetContext(); + const auto &in_table = in->get_table(); + + std::shared_ptr out_table; + if (!in->Empty()) { + out_table = in_table->Slice(offset, length); + } else { + out_table = in_table; + } + return Table::FromArrowTable(ctx, std::move(out_table), *out); +} + +/** + * DistributedSlice the part of table to create a single table + * @param table, global_offset and global_length + * @return new sliced table + */ +Status distributed_slice_impl(const std::shared_ptr
&in, + int64_t global_offset, + int64_t global_length, + int64_t *partition_lengths, + std::shared_ptr
*out) { + const auto &ctx = in->GetContext(); + std::shared_ptr partition_len_col; + + if (partition_lengths == nullptr) { + const auto &num_row_scalar = Scalar::Make(arrow::MakeScalar(in->Rows())); + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator() + ->Allgather(num_row_scalar, &partition_len_col)); + + partition_lengths = + const_cast(std::static_pointer_cast(partition_len_col->data()) + ->raw_values()); + } + + int64_t rank = ctx->GetRank(); + int64_t prefix_sum = std::accumulate(partition_lengths, partition_lengths + rank, kZero); + int64_t this_length = *(partition_lengths + rank); + assert(this_length == in->Rows()); + + int64_t local_offset = std::max(kZero, std::min(global_offset - prefix_sum, this_length)); + int64_t local_length = + std::min(this_length, std::max(global_offset + global_length - prefix_sum, kZero)) + - local_offset; + + return Slice(in, local_offset, local_length, out); +} + +Status DistributedSlice(const std::shared_ptr
&in, + int64_t offset, + int64_t length, + std::shared_ptr
*out) { + return distributed_slice_impl(in, offset, length, nullptr, out); +} + +/** + * Head the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ +Status Head(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr
*output) { + if (num_rows >= 0) { + return Slice(table, 0, num_rows, output); + } else + return {Code::Invalid, "Number of head rows should be >=0"}; +} + +Status DistributedHead(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr
*output) { + + std::shared_ptr in_table = table->get_table(); + + if (num_rows >= 0) { + return distributed_slice_impl(table, 0, num_rows, nullptr, output); + } else { + return {Code::Invalid, "Number of head rows should be >=0"}; + } +} + +/** + * Tail the part of table to create a single table with specific number of rows + * @param tables, number of rows + * @return new table + */ +Status Tail(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr
*output) { + + std::shared_ptr in_table = table->get_table(); + const int64_t table_size = in_table->num_rows(); + + if (num_rows >= 0) { + return Slice(table, table_size - num_rows, num_rows, output); + } else { + return {Code::Invalid, "Number of tailed rows should be >=0"}; + } +} + +Status DistributedTail(const std::shared_ptr
&table, int64_t num_rows, + std::shared_ptr
*output) { + if (num_rows >= 0) { + const auto &ctx = table->GetContext(); + std::shared_ptr partition_len_col; + const auto &num_row_scalar = Scalar::Make(arrow::MakeScalar(table->Rows())); + RETURN_CYLON_STATUS_IF_FAILED(ctx->GetCommunicator() + ->Allgather(num_row_scalar, &partition_len_col)); + assert(ctx->GetWorldSize() == partition_len_col->length()); + auto *partition_lengths = + std::static_pointer_cast(partition_len_col->data()) + ->raw_values(); + + int64_t dist_length = + std::accumulate(partition_lengths, partition_lengths + ctx->GetWorldSize(), kZero); + + return distributed_slice_impl(table, + dist_length - num_rows, + num_rows, + const_cast (partition_lengths), + output); + } else { + return {Code::Invalid, "Number of tailed rows should be >=0"}; + } +} } \ No newline at end of file diff --git a/cpp/test/CMakeLists.txt b/cpp/test/CMakeLists.txt index 2696d1650..04c32720a 100644 --- a/cpp/test/CMakeLists.txt +++ b/cpp/test/CMakeLists.txt @@ -158,6 +158,7 @@ cylon_run_test(equal_test 2 mpi) cylon_add_test(slice_test) cylon_run_test(slice_test 1 mpi) cylon_run_test(slice_test 2 mpi) +cylon_run_test(slice_test 4 mpi) # flatten array test cylon_add_test(flatten_array_test) diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index ddc6a546c..ec0391633 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -23,32 +23,34 @@ void testDistSlice(std::shared_ptr
& global_table, std::shared_ptr
& table, int64_t offset, int64_t length) { - std::shared_ptr
out, global_out; + std::shared_ptr
out; auto ctx = table->GetContext(); std::shared_ptr arrow_output; CHECK_CYLON_STATUS(DistributedSlice(table, offset, length, &out)); + INFO(RANK << "slice res:" << out->Rows()); std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, &gathered)); - - CHECK_CYLON_STATUS(Slice(global_table, offset, length, &global_out)); if (RANK == 0) { - std::shared_ptr
result; + std::shared_ptr
result, global_out; + CHECK_CYLON_STATUS(Slice(global_table, offset, length, &global_out)); + for (size_t i = 0; i < gathered.size(); i++) { + INFO("gathered " << i << ":" << gathered[i]->Rows()); + } + REQUIRE(WORLD_SZ == (int) gathered.size()); CHECK_CYLON_STATUS(Merge(gathered, result)); - CHECK_ARROW_EQUAL(global_out->get_table(), result->get_table()); } } namespace test { -TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) { - auto type = default_type_instance(); - auto schema = arrow::schema({{arrow::field("a", type)}, +TEST_CASE("Dist Slice testing", "[dist slice]") { + auto schema = arrow::schema({{arrow::field("a", arrow::int64())}, {arrow::field("b", arrow::float32())}}); auto global_arrow_table = TableFromJSON(schema, {R"([{"a": 3, "b":0.025}, {"a": 26, "b":0.394}, @@ -97,7 +99,6 @@ TEMPLATE_LIST_TEST_CASE("Dist Slice testing", "[dist slice]", ArrowNumericTypes) ctx, global_arrow_table->Slice(RANK * rows_per_tab, rows_per_tab), table1)); std::shared_ptr
global_table; - LOG(INFO) << "HERE!!!"; CHECK_CYLON_STATUS( Table::FromArrowTable(ctx, global_arrow_table, global_table)); From e6b73062a32ff86abdb899a5025edeb09bea9e38 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Tue, 16 Aug 2022 11:08:01 -0400 Subject: [PATCH 23/28] Minor fixes (#596) * remove gloo default hostname * minor change gloo * adding gloo-mpi test * adding ucc cyton * Update setup.py * Update setup.py * adding ucc test * adding multi env test * cosmetic changes * adding regular sampling cython * adding UCC barrier * adding macos11 tag for CI * fixing windows error * trying to fix macos ci * trying to fix macos issue * Revert "trying to fix macos issue" This reverts commit cda5c2cd3354a6ddf05d6adfcf327c3e9f7fffc5. * attempting to fix macos ci * style-check * adding gloo timeout * adding custom mpiexec cmake var --- .github/workflows/macos.yml | 2 +- build.py | 29 ++++++++-- conda/environments/cylon_MacOS.yml | 2 +- cpp/CMakeLists.txt | 4 ++ cpp/src/cylon/net/gloo/gloo_communicator.cpp | 14 ++++- cpp/src/cylon/net/gloo/gloo_communicator.hpp | 7 ++- cpp/src/cylon/net/mpi/mpi_communicator.cpp | 3 +- cpp/src/cylon/net/ucx/ucx_communicator.cpp | 53 +++++++++++++----- cpp/src/cylon/net/ucx/ucx_communicator.hpp | 5 +- cpp/src/cylon/table.cpp | 18 ++---- cpp/test/CMakeLists.txt | 3 + cpp/test/sync_comms_test.cpp | 15 +++++ python/pycylon/pycylon/api/lib.pxd | 9 ++- python/pycylon/pycylon/api/lib.pyx | 16 +++++- python/pycylon/pycylon/ctx/context.pyx | 7 +++ python/pycylon/pycylon/data/table.pxd | 10 +++- python/pycylon/pycylon/data/table.pyx | 22 +++++--- python/pycylon/pycylon/net/gloo_config.pxd | 1 + python/pycylon/pycylon/net/gloo_config.pyx | 15 +++++ python/pycylon/pycylon/net/ucx_config.pxd | 31 +++++++++++ python/pycylon/pycylon/net/ucx_config.pyx | 28 ++++++++++ python/pycylon/setup.py | 3 +- python/pycylon/test/test_all.py | 25 +++++++++ python/pycylon/test/test_gloo.py | 2 +- python/pycylon/test/test_gloo_mpi.py | 46 ++++++++++++++++ .../test/test_mpi_multiple_env_init.py | 55 +++++++++++++++++++ python/pycylon/test/test_ucx_mpi.py | 44 +++++++++++++++ python/pygcylon/setup.py | 2 +- 28 files changed, 416 insertions(+), 55 deletions(-) create mode 100644 python/pycylon/pycylon/net/ucx_config.pxd create mode 100644 python/pycylon/pycylon/net/ucx_config.pyx create mode 100644 python/pycylon/test/test_gloo_mpi.py create mode 100644 python/pycylon/test/test_mpi_multiple_env_init.py create mode 100644 python/pycylon/test/test_ucx_mpi.py diff --git a/.github/workflows/macos.yml b/.github/workflows/macos.yml index 24c6a49ad..775a10979 100644 --- a/.github/workflows/macos.yml +++ b/.github/workflows/macos.yml @@ -20,7 +20,7 @@ jobs: fail-fast: false matrix: include: - - os: macos-10.15 + - os: macos-11 steps: - uses: actions/checkout@v2 diff --git a/build.py b/build.py index cf2eecfa6..8506cf733 100644 --- a/build.py +++ b/build.py @@ -89,6 +89,8 @@ def check_conda_prefix(): parser.add_argument("-ipath", help='Install directory') parser.add_argument("--verbose", help='Set verbosity', default=False, action="store_true") +parser.add_argument("-j", help='Parallel build threads', default=os.cpu_count(), + dest='parallel', type=int) args = parser.parse_args() @@ -107,9 +109,7 @@ def on_off(arg): RUN_CPP_TESTS = args.test RUN_PYTHON_TESTS = args.pytest CMAKE_FLAGS = args.cmake_flags -CPPLINT_COMMAND = "\"-DCMAKE_CXX_CPPLINT=cpplint;--linelength=100;--headers=h," \ - "hpp;--filter=-legal/copyright,-build/c++11,-runtime/references\" " if \ - args.style_check else " " +PARALLEL = args.parallel # arrow build expects /s even on windows BUILD_PYTHON = args.python @@ -127,6 +127,16 @@ def on_off(arg): PYTHON_EXEC = sys.executable +if args.style_check: + cmd = f'{PYTHON_EXEC} -m pip install cpplint' + res = subprocess.run(cmd, shell=True, cwd=PYTHON_SOURCE_DIR) + check_status(res.returncode, "cpplint install") + + CPPLINT_COMMAND = "-DCMAKE_CXX_CPPLINT=\"cpplint;--linelength=100;--headers=h," \ + "hpp;--filter=-legal/copyright,-build/c++11,-runtime/references\" " +else: + CPPLINT_COMMAND = " " + CMAKE_BOOL_FLAGS = {'CYLON_GLOO', 'CYLON_UCX', 'CYLON_UCC'} CMAKE_FALSE_OPTIONS = {'0', 'FALSE', 'OFF', 'N', 'NO', 'IGNORE', 'NOTFOUND'} @@ -169,6 +179,7 @@ def print_line(): logger.info(f"Python exec : {PYTHON_EXEC}") logger.info(f"Build mode : {CPP_BUILD_MODE}") logger.info(f"Build path : {BUILD_DIR}") +logger.info(f"Build threads : {PARALLEL}") logger.info(f"Install path : {INSTALL_DIR}") logger.info(f"CMake flags : {CMAKE_FLAGS}") logger.info(f" -CYLON_GLOO : {CYLON_GLOO}") @@ -206,7 +217,7 @@ def build_cpp(): res = subprocess.call(cmake_command, cwd=BUILD_DIR, shell=True) check_status(res, "C++ cmake generate") - cmake_build_command = f'cmake --build . --parallel {os.cpu_count()} --config {CPP_BUILD_MODE}' + cmake_build_command = f'cmake --build . --parallel {PARALLEL} --config {CPP_BUILD_MODE}' logger.info(f"Build command: {cmake_build_command}") res = subprocess.call(cmake_build_command, cwd=BUILD_DIR, shell=True) check_status(res, "C++ cmake build") @@ -254,6 +265,14 @@ def python_test(): env['LD_LIBRARY_PATH'] = os.path.join(GLOO_PREFIX, "lib") + os.pathsep + \ env['LD_LIBRARY_PATH'] + + if CYLON_UCC: + env['CYLON_UCC'] = str(CYLON_UCC) + env['UCC_PREFIX'] = UCC_PREFIX + env['LD_LIBRARY_PATH'] = os.path.join(UCC_PREFIX, "lib") + os.pathsep + \ + os.path.join(UCC_PREFIX, "lib", "ucc") + os.pathsep + \ + env['LD_LIBRARY_PATH'] + elif OS_NAME == 'Darwin': if 'DYLD_LIBRARY_PATH' in env: env['DYLD_LIBRARY_PATH'] = str(Path(INSTALL_DIR, "lib")) + os.pathsep \ @@ -279,7 +298,7 @@ def build_python(): conda_prefix = check_conda_prefix() - python_build_command = f'{PYTHON_EXEC} setup.py install --force' + python_build_command = f'{PYTHON_EXEC} -m pip install -v --upgrade .' env = os.environ env["CYLON_PREFIX"] = str(BUILD_DIR) if os.name == 'posix': diff --git a/conda/environments/cylon_MacOS.yml b/conda/environments/cylon_MacOS.yml index 8af49b18a..e1c2a797d 100644 --- a/conda/environments/cylon_MacOS.yml +++ b/conda/environments/cylon_MacOS.yml @@ -12,7 +12,7 @@ dependencies: - numpy>=1.16 - pandas>=1.0 - fsspec - - setuptools>=40.0,<60.0 + - setuptools>=40.0 # they are not needed for using pygcylon or compiling it - pytest - pytest-mpi diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index d7337f0b4..9b6e7c2ec 100755 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -155,6 +155,10 @@ message("Finding MPI") find_package(MPI REQUIRED COMPONENTS CXX) message(STATUS "MPI include dir: ${MPI_CXX_INCLUDE_PATH}") message(STATUS "MPI libs: ${MPI_CXX_LIBRARIES}") + +if (CYLON_CUSTOM_MPIRUN) + set(MPIEXEC_EXECUTABLE ${CYLON_CUSTOM_MPIRUN}) +endif (CYLON_CUSTOM_MPIRUN) message(STATUS "MPI executable: ${MPIEXEC_EXECUTABLE}") include_directories(SYSTEM ${MPI_CXX_INCLUDE_PATH}) diff --git a/cpp/src/cylon/net/gloo/gloo_communicator.cpp b/cpp/src/cylon/net/gloo/gloo_communicator.cpp index 7af55b28f..12d880398 100644 --- a/cpp/src/cylon/net/gloo/gloo_communicator.cpp +++ b/cpp/src/cylon/net/gloo/gloo_communicator.cpp @@ -57,6 +57,9 @@ Status GlooCommunicator::Make(const std::shared_ptr &config, } else { // MPI is not initialized. Ask gloo to initialize MPI gloo_ctx = gloo::mpi::Context::createManaged(); } + if (gloo_config->timeout_ != kTimoutNotSet) { + gloo_ctx->setTimeout(std::chrono::seconds(gloo_config->timeout_)); + } ((gloo::mpi::Context &) *gloo_ctx).connectFullMesh(dev); #else return {Code::Invalid, "Gloo does not contain mpi headers!"}; @@ -69,8 +72,10 @@ Status GlooCommunicator::Make(const std::shared_ptr &config, gloo_ctx = std::make_shared(gloo_config->rank_, gloo_config->world_size_); + if (gloo_config->timeout_ != kTimoutNotSet) { + gloo_ctx->setTimeout(std::chrono::seconds(gloo_config->timeout_)); + } ((gloo::rendezvous::Context &) *gloo_ctx).connectFullMesh(*prefix_store, dev); - } *out = std::make_shared(pool, std::move(gloo_ctx)); return Status::OK(); @@ -193,5 +198,12 @@ int GlooConfig::world_size() const { return world_size_; } +void GlooConfig::SetTimeout(int timeout) { + GlooConfig::timeout_ = std::chrono::seconds(timeout); +} +const std::chrono::seconds &GlooConfig::timeout() const { + return GlooConfig::timeout_; +} + } } diff --git a/cpp/src/cylon/net/gloo/gloo_communicator.hpp b/cpp/src/cylon/net/gloo/gloo_communicator.hpp index 1f5b9a825..c7523e00a 100644 --- a/cpp/src/cylon/net/gloo/gloo_communicator.hpp +++ b/cpp/src/cylon/net/gloo/gloo_communicator.hpp @@ -33,6 +33,8 @@ namespace net { class GlooCommunicator; +static constexpr std::chrono::seconds kTimoutNotSet(0); + class GlooConfig : public CommConfig { public: explicit GlooConfig(int rank = 0, int world_size = 1, bool use_mpi = false); @@ -43,7 +45,9 @@ class GlooConfig : public CommConfig { int rank() const; int world_size() const; + const std::chrono::seconds &timeout() const; + void SetTimeout(int timeout); void SetTcpHostname(const std::string &tcp_hostname); void SetTcpIface(const std::string &tcp_iface); void SetTcpAiFamily(int tcp_ai_family); @@ -63,6 +67,7 @@ class GlooConfig : public CommConfig { int rank_; int world_size_; bool use_mpi_; + std::chrono::seconds timeout_ = kTimoutNotSet; #ifdef GLOO_USE_MPI /* @@ -74,7 +79,7 @@ class GlooConfig : public CommConfig { #endif //GLOO_USE_MPI // tcp attr - std::string tcp_hostname_ = "localhost"; + std::string tcp_hostname_; std::string tcp_iface_; int tcp_ai_family_ = AF_UNSPEC; diff --git a/cpp/src/cylon/net/mpi/mpi_communicator.cpp b/cpp/src/cylon/net/mpi/mpi_communicator.cpp index ba2b5768e..5a723e346 100644 --- a/cpp/src/cylon/net/mpi/mpi_communicator.cpp +++ b/cpp/src/cylon/net/mpi/mpi_communicator.cpp @@ -91,12 +91,13 @@ Status MPICommunicator::Make(const std::shared_ptr &config, void MPICommunicator::Finalize() { // finalize only if we initialized MPI - if (!externally_init) { + if (!externally_init && !IsFinalized()) { int finalized; MPI_Finalized(&finalized); if (!finalized) { MPI_Finalize(); } + finalized = true; } } void MPICommunicator::Barrier() { diff --git a/cpp/src/cylon/net/ucx/ucx_communicator.cpp b/cpp/src/cylon/net/ucx/ucx_communicator.cpp index 964d76efb..0e1771a83 100644 --- a/cpp/src/cylon/net/ucx/ucx_communicator.cpp +++ b/cpp/src/cylon/net/ucx/ucx_communicator.cpp @@ -27,6 +27,8 @@ namespace cylon { namespace net { +static constexpr int kBarrierFlag = UINT32_MAX; + void mpi_check_and_finalize() { int mpi_finalized; MPI_Finalized(&mpi_finalized); @@ -89,7 +91,8 @@ Status UCXCommunicator::AllReduce(const std::shared_ptr &column, return {Code::NotImplemented, "Allreduce not implemented for ucx"}; } -UCXCommunicator::UCXCommunicator(MemoryPool *pool) : Communicator(pool, -1, -1) {} +UCXCommunicator::UCXCommunicator(MemoryPool *pool, bool externally_init) + : Communicator(pool, -1, -1), externally_init(externally_init) {} Status UCXCommunicator::AllReduce(const std::shared_ptr &values, net::ReduceOp reduce_op, @@ -117,10 +120,16 @@ Status UCXCommunicator::Allgather(const std::shared_ptr &value, Status UCXCommunicator::Make(const std::shared_ptr &config, MemoryPool *pool, std::shared_ptr *out) { CYLON_UNUSED(config); - *out = std::make_shared(pool); - auto &comm = static_cast(**out); - // Check init functions + // MPI init int initialized; + MPI_Initialized(&initialized); + if (!initialized) { + RETURN_CYLON_STATUS_IF_MPI_FAILED(MPI_Init(nullptr, nullptr)); + } + + *out = std::make_shared(pool, initialized); + auto &comm = static_cast(**out); + // Int variable used when iterating int sIndx; // Address of the UCP Worker for receiving @@ -133,12 +142,6 @@ Status UCXCommunicator::Make(const std::shared_ptr &config, MemoryPo // Variable to hold the current ucp address ucp_address_t *address; - // MPI init - MPI_Initialized(&initialized); - if (!initialized) { - RETURN_CYLON_STATUS_IF_MPI_FAILED(MPI_Init(nullptr, nullptr)); - } - // Get the rank for checking send to self, and initializations MPI_Comm_rank(MPI_COMM_WORLD, &comm.rank); MPI_Comm_size(MPI_COMM_WORLD, &comm.world_size); @@ -205,7 +208,7 @@ Status UCXCommunicator::Make(const std::shared_ptr &config, MemoryPo } void UCXCommunicator::Finalize() { - if (!this->IsFinalized()) { + if (!externally_init && !IsFinalized()) { ucp_cleanup(ucpContext); mpi_check_and_finalize(); finalized = true; @@ -329,14 +332,36 @@ void UCXUCCCommunicator::Finalize() { } } ucc_context_destroy(uccContext); - mpi_check_and_finalize(); - ucx_comm_->Finalize(); + ucx_comm_->Finalize(); // this will handle MPI_Finalize finalized = true; } } void UCXUCCCommunicator::Barrier() { - return ucx_comm_->Barrier(); + ucc_coll_args_t args_; + ucc_coll_req_h req; + + args_.mask = 0; + args_.coll_type = UCC_COLL_TYPE_BARRIER; + + ucc_status_t status; + status = ucc_collective_init(&args_, &req, uccTeam); + assert(status == UCC_OK); + + status = ucc_collective_post(req); + assert(status == UCC_OK); + + status = ucc_collective_test(req); + while (status > UCC_OK) { // loop until status == 0 + status = ucc_context_progress(uccContext); + assert(status >= UCC_OK); // should be 0, 1 or 2 + + status = ucc_collective_test(req); + } + assert(status == UCC_OK); + + status = ucc_collective_finalize(req); + assert(status == UCC_OK); } Status UCXUCCCommunicator::AllGather(const std::shared_ptr
&table, diff --git a/cpp/src/cylon/net/ucx/ucx_communicator.hpp b/cpp/src/cylon/net/ucx/ucx_communicator.hpp index 57ba07578..74abb5200 100644 --- a/cpp/src/cylon/net/ucx/ucx_communicator.hpp +++ b/cpp/src/cylon/net/ucx/ucx_communicator.hpp @@ -29,15 +29,15 @@ namespace cylon { namespace net { class UCXConfig : public CommConfig { + public: CommType Type() override; - public: static std::shared_ptr Make(); }; class UCXCommunicator : public Communicator { public: - explicit UCXCommunicator(MemoryPool *pool); + explicit UCXCommunicator(MemoryPool *pool, bool externally_init); ~UCXCommunicator() override = default; std::unique_ptr CreateChannel() const override; @@ -78,6 +78,7 @@ class UCXCommunicator : public Communicator { std::unordered_map endPointMap; // UCP Context - Holds a UCP communication instance's global information. ucp_context_h ucpContext{}; + bool externally_init = false; }; #ifdef BUILD_CYLON_UCC diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 07964dfc8..d018cf186 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -504,9 +504,8 @@ Status MergeSortedTable(const std::vector> &tables, const std::vector &sort_orders, std::shared_ptr
&out) { std::shared_ptr
concatenated; - std::vector table_indices(tables.size()), - table_end_indices(tables.size()); - int acc = 0; + std::vector table_indices(tables.size()), table_end_indices(tables.size()); + int64_t acc = 0; for (size_t i = 0; i < table_indices.size(); i++) { table_indices[i] = acc; acc += tables[i]->Rows(); @@ -515,7 +514,7 @@ Status MergeSortedTable(const std::vector> &tables, RETURN_CYLON_STATUS_IF_FAILED(Merge(tables, concatenated)); - if(concatenated->GetContext()->GetWorldSize() > 4) { + if (concatenated->GetContext()->GetWorldSize() > 4) { return Sort(concatenated, sort_columns, out, sort_orders); } @@ -530,7 +529,7 @@ Status MergeSortedTable(const std::vector> &tables, std::priority_queue, decltype(comp)> pq(comp); - for (size_t i = 0; i < tables.size(); i++) { + for (int i = 0; i < (int) tables.size(); i++) { if (table_indices[i] < table_end_indices[i]) { pq.push(i); } @@ -541,13 +540,9 @@ Status MergeSortedTable(const std::vector> &tables, arrow::Int64Builder filter(pool); RETURN_CYLON_STATUS_IF_ARROW_FAILED(filter.Reserve(concatenated->Rows())); - std::vector temp_v; - while (!pq.empty()) { int t = pq.top(); pq.pop(); - // std::cout<> &tables, } CYLON_ASSIGN_OR_RAISE(auto take_arr, filter.Finish()); - CYLON_ASSIGN_OR_RAISE( - auto take_res, - (arrow::compute::Take(concatenated->get_table(), take_arr))); + CYLON_ASSIGN_OR_RAISE(auto take_res, + (arrow::compute::Take(concatenated->get_table(), take_arr))); out = std::make_shared
(ctx, take_res.table()); diff --git a/cpp/test/CMakeLists.txt b/cpp/test/CMakeLists.txt index b709ec1c8..2db81f853 100644 --- a/cpp/test/CMakeLists.txt +++ b/cpp/test/CMakeLists.txt @@ -238,5 +238,8 @@ if (CYLON_UCX) cylon_run_test(aggregate_test 1 ucx) cylon_run_test(aggregate_test 2 ucx) cylon_run_test(aggregate_test 4 ucx) + + cylon_run_test(sync_comms_test 1 ucx) + cylon_run_test(sync_comms_test 4 ucx) endif (CYLON_UCC) endif (CYLON_UCX) \ No newline at end of file diff --git a/cpp/test/sync_comms_test.cpp b/cpp/test/sync_comms_test.cpp index 76911e1ca..62d810cb6 100644 --- a/cpp/test/sync_comms_test.cpp +++ b/cpp/test/sync_comms_test.cpp @@ -12,12 +12,22 @@ * limitations under the License. */ #include +#include +#include #include "common/test_header.hpp" namespace cylon { namespace test { +TEST_CASE("barrier", "[sync comms]") { + srand((unsigned) time(nullptr)); + int i = (rand() % 2000) + 1; + + std::this_thread::sleep_for(std::chrono::duration(i)); + ctx->Barrier(); +} + enum GenType { Empty, Null, NonEmpty }; void generate_table(std::shared_ptr *schema, @@ -113,6 +123,11 @@ TEST_CASE("all gather table", "[sync comms]") { } TEST_CASE("gather table", "[sync comms]") { + // todo: UCC doesnt support gatherv for the moment #599 + if (ctx->GetCommType() == net::UCX){ + return; + } + std::shared_ptr schema; std::shared_ptr in_table; generate_table(&schema, &in_table); diff --git a/python/pycylon/pycylon/api/lib.pxd b/python/pycylon/pycylon/api/lib.pxd index ad6368bea..96cad6ca3 100644 --- a/python/pycylon/pycylon/api/lib.pxd +++ b/python/pycylon/pycylon/api/lib.pxd @@ -25,6 +25,8 @@ from pycylon.net.comm_config cimport CCommConfig from pycylon.net.mpi_config cimport CMPIConfig IF CYTHON_GLOO: from pycylon.net.gloo_config cimport CGlooConfig +IF CYTHON_UCX & CYTHON_UCC: + from pycylon.net.ucx_config cimport CUCXConfig from pycylon.io.csv_read_config cimport CCSVReadOptions from pycylon.io.csv_read_config import CSVReadOptions from pycylon.io.csv_read_config cimport CSVReadOptions @@ -59,6 +61,9 @@ cdef api shared_ptr[CMPIConfig] pycylon_unwrap_mpi_config(object config) IF CYTHON_GLOO: cdef api shared_ptr[CGlooConfig] pycylon_unwrap_gloo_config(object config) +IF CYTHON_UCX & CYTHON_UCC: + cdef api shared_ptr[CUCXConfig] pycylon_unwrap_ucx_config(object config) + cdef api shared_ptr[CTable] pycylon_unwrap_table(object table) cdef api shared_ptr[CDataType] pycylon_unwrap_data_type(object data_type) @@ -67,7 +72,7 @@ cdef api CCSVReadOptions pycylon_unwrap_csv_read_options(object csv_read_options cdef api CCSVWriteOptions pycylon_unwrap_csv_write_options(object csv_write_options) -cdef api CSortOptions * pycylon_unwrap_sort_options(object sort_options) +cdef api shared_ptr[CSortOptions] pycylon_unwrap_sort_options(object sort_options) cdef api shared_ptr[CBaseArrowIndex] pycylon_unwrap_base_arrow_index(object base_arrow_index) @@ -87,7 +92,7 @@ cdef api object pycylon_wrap_layout(const CLayout & layout) cdef api object pycylon_wrap_data_type(const shared_ptr[CDataType] & data_type) -cdef api object pycylon_wrap_sort_options(CSortOptions *sort_options) +cdef api object pycylon_wrap_sort_options(const shared_ptr[CSortOptions] &sort_options) cdef api object pycylon_wrap_base_arrow_index(const shared_ptr[CBaseArrowIndex] & base_arrow_index) diff --git a/python/pycylon/pycylon/api/lib.pyx b/python/pycylon/pycylon/api/lib.pyx index a1130f1d3..67bec2700 100644 --- a/python/pycylon/pycylon/api/lib.pyx +++ b/python/pycylon/pycylon/api/lib.pyx @@ -29,6 +29,9 @@ from pycylon.net.mpi_config cimport MPIConfig IF CYTHON_GLOO: from pycylon.net.gloo_config import GlooMPIConfig, GlooStandaloneConfig from pycylon.net.gloo_config cimport CGlooConfig, GlooMPIConfig, GlooStandaloneConfig +IF CYTHON_UCX & CYTHON_UCC: + from pycylon.net.ucx_config import UCXConfig + from pycylon.net.ucx_config cimport CUCXConfig, UCXConfig from pycylon.io.csv_read_config cimport CCSVReadOptions from pycylon.io.csv_read_config import CSVReadOptions from pycylon.io.csv_read_config cimport CSVReadOptions @@ -121,6 +124,13 @@ IF CYTHON_GLOO: else: raise ValueError('Passed object is not an instance of GlooConfig') +IF CYTHON_UCX & CYTHON_UCC: + cdef api shared_ptr[CUCXConfig] pycylon_unwrap_ucx_config(object config): + if isinstance(config, UCXConfig): + return ( config).ucx_config_shd_ptr + else: + raise ValueError('Passed object is not an instance of UcxConfig') + cdef api CCSVReadOptions pycylon_unwrap_csv_read_options(object csv_read_options): cdef CSVReadOptions csvrdopt if pyclon_is_csv_read_options(csv_read_options): @@ -145,13 +155,13 @@ cdef api shared_ptr[CDataType] pycylon_unwrap_data_type(object data_type): else: raise ValueError('Passed object is not an instance of DataType') -cdef api CSortOptions * pycylon_unwrap_sort_options(object sort_options): +cdef api shared_ptr[CSortOptions] pycylon_unwrap_sort_options(object sort_options): cdef SortOptions so if pyclon_is_sort_options(sort_options): so = sort_options return so.thisPtr else: - raise ValueError('Passed object is not an instance of DataType') + raise ValueError('Passed object is not an instance of SortOptions') cdef api shared_ptr[CBaseArrowIndex] pycylon_unwrap_base_arrow_index(object base_arrow_index): cdef BaseArrowIndex bi @@ -196,7 +206,7 @@ cdef api object pycylon_wrap_data_type(const shared_ptr[CDataType] & cdata_type) data_type.init(cdata_type) return data_type -cdef api object pycylon_wrap_sort_options(CSortOptions *csort_options): +cdef api object pycylon_wrap_sort_options(const shared_ptr[CSortOptions] &csort_options): cdef SortOptions sort_options = SortOptions.__new__(SortOptions) sort_options.init(csort_options) return sort_options diff --git a/python/pycylon/pycylon/ctx/context.pyx b/python/pycylon/pycylon/ctx/context.pyx index f3e9b71c6..c993947ee 100644 --- a/python/pycylon/pycylon/ctx/context.pyx +++ b/python/pycylon/pycylon/ctx/context.pyx @@ -20,6 +20,9 @@ from pycylon.ctx.context cimport CCylonContext from pycylon.api.lib cimport pycylon_unwrap_mpi_config IF CYTHON_GLOO: from pycylon.api.lib cimport pycylon_unwrap_gloo_config + +IF CYTHON_UCX & CYTHON_UCC: + from pycylon.api.lib cimport pycylon_unwrap_ucx_config from pycylon.net import CommType from pycylon.net.mpi_config cimport CMPIConfig from pycylon.net.mpi_config import MPIConfig @@ -82,6 +85,10 @@ cdef class CylonContext: if config.comm_type == CommType.GLOO: return pycylon_unwrap_gloo_config(config) + IF CYTHON_UCX & CYTHON_UCC: + if config.comm_type == CommType.UCX: + return pycylon_unwrap_ucx_config(config) + raise ValueError(f"Unsupported distributed comm config {config}") def get_rank(self) -> int: diff --git a/python/pycylon/pycylon/data/table.pxd b/python/pycylon/pycylon/data/table.pxd index 228fae890..084454edf 100644 --- a/python/pycylon/pycylon/data/table.pxd +++ b/python/pycylon/pycylon/data/table.pxd @@ -127,16 +127,22 @@ cdef extern from "../../../../cpp/src/cylon/table.hpp" namespace "cylon": cdef extern from "../../../../cpp/src/cylon/table.hpp" namespace "cylon": cdef cppclass CSortOptions "cylon::SortOptions": + enum CSortMethod: + CREGULAR_SAMPLE 'cylon::SortOptions::SortMethod::REGULAR_SAMPLE', + CINITIAL_SAMPLE 'cylon::SortOptions::SortMethod::INITIAL_SAMPLE' + int num_bins long num_samples + CSortMethod sort_method + @ staticmethod CSortOptions Defaults() cdef class SortOptions: cdef: - CSortOptions *thisPtr - void init(self, CSortOptions *csort_options) + shared_ptr[CSortOptions] thisPtr + void init(self, const shared_ptr[CSortOptions] &csort_options) cdef class Table: cdef: diff --git a/python/pycylon/pycylon/data/table.pyx b/python/pycylon/pycylon/data/table.pyx index 356baf83e..9144ebbcc 100644 --- a/python/pycylon/pycylon/data/table.pyx +++ b/python/pycylon/pycylon/data/table.pyx @@ -471,7 +471,7 @@ cdef class Table: """ cdef shared_ptr[CTable] output - cdef CSortOptions *csort_options + cdef shared_ptr[CSortOptions] csort_options cdef vector[int] sort_index cdef vector[cpp_bool] order_directions @@ -507,7 +507,7 @@ cdef class Table: else: csort_options = pycylon_unwrap_sort_options(SortOptions(0, 0)) cdef CStatus status = DistributedSort(self.table_shd_ptr, sort_index, output, - order_directions, csort_options[0]) + order_directions, csort_options.get()[0]) if status.is_ok(): return pycylon_wrap_table(output) else: @@ -2728,7 +2728,7 @@ cdef class SortOptions: """ Sort Operations for Distributed Sort """ - def __cinit__(self, num_bins: int = 0, num_samples: int = 0): + def __cinit__(self, num_bins: int = 0, num_samples: int = 0, sampling: str = 'regular'): ''' Initializes the CSortOptions struct Args: @@ -2738,9 +2738,17 @@ cdef class SortOptions: Returns: None ''' - self.thisPtr = new CSortOptions() - self.thisPtr.num_bins = num_bins - self.thisPtr.num_samples = num_samples + self.thisPtr = make_shared[CSortOptions]() + self.thisPtr.get().num_bins = num_bins + self.thisPtr.get().num_samples = num_samples - cdef void init(self, CSortOptions *csort_options): + sampling = sampling.lower() + if sampling == 'regular': + self.thisPtr.get().sort_method = CSortOptions.CSortMethod.CREGULAR_SAMPLE + elif sampling == 'initial': + self.thisPtr.get().sort_method = CSortOptions.CSortMethod.CINITIAL_SAMPLE + else: + raise ValueError(f'unknown sampling method for sorting: {sampling}') + + cdef void init(self, const shared_ptr[CSortOptions] &csort_options): self.thisPtr = csort_options diff --git a/python/pycylon/pycylon/net/gloo_config.pxd b/python/pycylon/pycylon/net/gloo_config.pxd index dbec281cf..f24b68ae8 100644 --- a/python/pycylon/pycylon/net/gloo_config.pxd +++ b/python/pycylon/pycylon/net/gloo_config.pxd @@ -31,6 +31,7 @@ IF CYTHON_GLOO: void SetTcpAiFamily(int tcp_ai_family) void SetFileStorePath(const string & file_store_path) void SetStorePrefix(const string & store_prefix) + void SetTimeout(int timeout) @staticmethod shared_ptr[CGlooConfig] MakeWithMpi(MPI_Comm comm); diff --git a/python/pycylon/pycylon/net/gloo_config.pyx b/python/pycylon/pycylon/net/gloo_config.pyx index 361fca365..8a7534fe4 100644 --- a/python/pycylon/pycylon/net/gloo_config.pyx +++ b/python/pycylon/pycylon/net/gloo_config.pyx @@ -38,6 +38,18 @@ IF CYTHON_GLOO: def comm_type(self): return self.gloo_config_shd_ptr.get().Type() + def set_tcp_hostname(self, hostname: str): + self.gloo_config_shd_ptr.get().SetTcpHostname(hostname.encode()) + + def set_tcp_iface(self, iface: str): + self.gloo_config_shd_ptr.get().SetTcpIface(iface.encode()) + + def set_tcp_ai_family(self, ai_family: int): + self.gloo_config_shd_ptr.get().SetTcpAiFamily(ai_family) + + def set_timeout(self, timeout: int): + self.gloo_config_shd_ptr.get().SetTimeout(timeout) + cdef class GlooStandaloneConfig(CommConfig): """ GlooConfig Type mapping from libCylon to PyCylon @@ -73,3 +85,6 @@ IF CYTHON_GLOO: def set_store_prefix(self, prefix: str): self.gloo_config_shd_ptr.get().SetStorePrefix(prefix.encode()) + + def set_timeout(self, timeout: int): + self.gloo_config_shd_ptr.get().SetTimeout(timeout) \ No newline at end of file diff --git a/python/pycylon/pycylon/net/ucx_config.pxd b/python/pycylon/pycylon/net/ucx_config.pxd new file mode 100644 index 000000000..a6f1b36c8 --- /dev/null +++ b/python/pycylon/pycylon/net/ucx_config.pxd @@ -0,0 +1,31 @@ +## +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +## + +IF CYTHON_UCX & CYTHON_UCC: + from libcpp.memory cimport shared_ptr + + from pycylon.net.comm_type cimport CCommType + from pycylon.net.comm_config cimport CommConfig + + cdef extern from "../../../../cpp/src/cylon/net/ucx/ucx_communicator.hpp" namespace "cylon::net": + cdef cppclass CUCXConfig "cylon::net::UCXConfig": + CCommType Type() + + @ staticmethod + shared_ptr[CUCXConfig] Make(); + + + cdef class UCXConfig(CommConfig): + cdef: + shared_ptr[CUCXConfig] ucx_config_shd_ptr diff --git a/python/pycylon/pycylon/net/ucx_config.pyx b/python/pycylon/pycylon/net/ucx_config.pyx new file mode 100644 index 000000000..ed04440c3 --- /dev/null +++ b/python/pycylon/pycylon/net/ucx_config.pyx @@ -0,0 +1,28 @@ +## +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +## + +IF CYTHON_UCX & CYTHON_UCC: + from pycylon.net.comm_config cimport CommConfig + from pycylon.net.ucx_config cimport CUCXConfig + + cdef class UCXConfig(CommConfig): + """ + GlooConfig Type mapping from libCylon to PyCylon + """ + def __cinit__(self): + self.ucx_config_shd_ptr = CUCXConfig.Make() + + @property + def comm_type(self): + return self.ucx_config_shd_ptr.get().Type() diff --git a/python/pycylon/setup.py b/python/pycylon/setup.py index 1cb302144..7e4149466 100644 --- a/python/pycylon/setup.py +++ b/python/pycylon/setup.py @@ -132,7 +132,7 @@ macros = [] # compile_time_env serves as preprocessor macros. ref: https://github.com/cython/cython/issues/2488 -compile_time_env = {'CYTHON_GLOO': False, 'CYTHON_UCC': False} +compile_time_env = {'CYTHON_GLOO': False, 'CYTHON_UCC': False, 'CYTHON_UCX': False} if CYLON_GLOO: libraries.append('gloo') library_directories.append(os.path.join(GLOO_PREFIX, 'lib')) @@ -147,6 +147,7 @@ _include_dirs.append(os.path.join(UCC_PREFIX, 'include')) macros.append(('BUILD_CYLON_UCX', '1')) macros.append(('BUILD_CYLON_UCC', '1')) + compile_time_env['CYTHON_UCX'] = True compile_time_env['CYTHON_UCC'] = True print('Libraries :', libraries) diff --git a/python/pycylon/test/test_all.py b/python/pycylon/test/test_all.py index b8fb158ed..61cba082f 100644 --- a/python/pycylon/test/test_all.py +++ b/python/pycylon/test/test_all.py @@ -14,6 +14,7 @@ import os + import numpy as np print("-------------------------------------------------") @@ -265,12 +266,14 @@ def test_dist_aggregate(): "python/pycylon/test/test_dist_aggregate.py")) assert responses[-1] == 0 + def test_dist_io(): print("34. Dist IO") responses.append(os.system(get_mpi_command() + " -n 4 python -m pytest --with-mpi " "python/pycylon/test/test_io.py")) assert responses[-1] == 0 + if os.environ.get('CYLON_GLOO'): def test_gloo(): print("35. Gloo") @@ -278,6 +281,28 @@ def test_gloo(): assert responses[-1] == 0 + def test_gloo_mpi(): + print("36. Gloo") + responses.append(os.system( + f"{get_mpi_command()} -n 4 python -m pytest python/pycylon/test/test_gloo_mpi.py")) + assert responses[-1] == 0 + +if os.environ.get('CYLON_UCC'): + def test_ucx_mpi(): + print("37. UCX MPI") + responses.append(os.system( + f"{get_mpi_command()} -n 4 python -m pytest python/pycylon/test/test_ucx_mpi.py")) + assert responses[-1] == 0 + +if os.environ.get('CYLON_GLOO') and os.environ.get('CYLON_UCC'): + def test_mpi_multiple_env_init(): + print("38. Create and destroy multiple environments in MPI") + responses.append(os.system( + f"{get_mpi_command()} -n 4 python -m pytest " + f"python/pycylon/test/test_mpi_multiple_env_init.py")) + assert responses[-1] == 0 + + def test_all(): ar = np.array(responses) total = len(responses) diff --git a/python/pycylon/test/test_gloo.py b/python/pycylon/test/test_gloo.py index 2d1e422eb..47ffe1192 100644 --- a/python/pycylon/test/test_gloo.py +++ b/python/pycylon/test/test_gloo.py @@ -29,7 +29,7 @@ from pycylon.net.gloo_config import GlooStandaloneConfig FILE_STORE_PATH = os.path.join(tempfile.gettempdir(), 'gloo') -WORLD_SIZE = 1 +WORLD_SIZE = 4 ROWS = 5 diff --git a/python/pycylon/test/test_gloo_mpi.py b/python/pycylon/test/test_gloo_mpi.py new file mode 100644 index 000000000..0c62e283d --- /dev/null +++ b/python/pycylon/test/test_gloo_mpi.py @@ -0,0 +1,46 @@ +## +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +## + +""" +Run test: +>> mpirun -n 4 python -m pytest --with-mpi -q python/pycylon/test/test_gloo.py +""" + +import pandas as pd +import pytest +from numpy.random import default_rng + +from pycylon import CylonEnv, DataFrame +from pycylon.net.gloo_config import GlooMPIConfig + +ROWS = 5 + + +@pytest.mark.mpi +def test_gloo_mpi(): # confirms that the code is under main function + conf = GlooMPIConfig() + env = CylonEnv(config=conf) + + rng = default_rng(seed=ROWS) + data1 = rng.integers(0, ROWS, size=(ROWS, 2)) + data2 = rng.integers(0, ROWS, size=(ROWS, 2)) + + df1 = DataFrame(pd.DataFrame(data1).add_prefix("col")) + df2 = DataFrame(pd.DataFrame(data2).add_prefix("col")) + + print("Distributed Merge") + df3 = df1.merge(right=df2, on=[0], env=env) + print(f'res len {len(df3)}') + + env.finalize() diff --git a/python/pycylon/test/test_mpi_multiple_env_init.py b/python/pycylon/test/test_mpi_multiple_env_init.py new file mode 100644 index 000000000..76aff0e6a --- /dev/null +++ b/python/pycylon/test/test_mpi_multiple_env_init.py @@ -0,0 +1,55 @@ +## +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +## + +""" +Run test +>> mpirun -n 2 python -m pytest --with-mpi -q python/pycylon/test/test_mpi_multiple_env_init.py +""" +import pandas as pd +import pytest +from mpi4py import MPI +from numpy.random import default_rng + +from pycylon import CylonEnv, DataFrame +from pycylon.net import MPIConfig +from pycylon.net.gloo_config import GlooMPIConfig +from pycylon.net.ucx_config import UCXConfig + + +def create_and_destroy_env(conf): + env = CylonEnv(config=conf) + print(f"{conf.comm_type} rank: {env.rank} world size: {env.world_size}") + r = 10 + + rng = default_rng() + data1 = rng.integers(0, r, size=(r, 2)) + data2 = rng.integers(0, r, size=(r, 2)) + + df1 = DataFrame(pd.DataFrame(data1).add_prefix("col")) + df2 = DataFrame(pd.DataFrame(data2).add_prefix("col")) + + print(len(df1.merge(right=df2, on=[0], env=env))) + env.finalize() + + +@pytest.mark.mpi +def test_mpi_multiple_env_init(): + comm = MPI.COMM_WORLD + rank = comm.rank + world_sz = comm.size + print(f"mpi rank: {rank} world size: {world_sz}") + + create_and_destroy_env(MPIConfig()) + create_and_destroy_env(GlooMPIConfig()) + create_and_destroy_env(UCXConfig()) diff --git a/python/pycylon/test/test_ucx_mpi.py b/python/pycylon/test/test_ucx_mpi.py new file mode 100644 index 000000000..dc1528f5c --- /dev/null +++ b/python/pycylon/test/test_ucx_mpi.py @@ -0,0 +1,44 @@ +## +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +## + +""" +Run test: +>> mpirun -n 4 python -m pytest -q python/pycylon/test/test_ucx_mpi.py +""" + +import pandas as pd +from numpy.random import default_rng + +from pycylon import CylonEnv, DataFrame +from pycylon.net.ucx_config import UCXConfig + +ROWS = 5 + + +def test_ucx_mpi(): # confirms that the code is under main function + conf = UCXConfig() + env = CylonEnv(config=conf) + + rng = default_rng(seed=env.rank) + data1 = rng.integers(0, ROWS, size=(ROWS, 2)) + data2 = rng.integers(0, ROWS, size=(ROWS, 2)) + + df1 = DataFrame(pd.DataFrame(data1).add_prefix("col")) + df2 = DataFrame(pd.DataFrame(data2).add_prefix("col")) + + print("Distributed Merge") + df3 = df1.merge(right=df2, on=[0], env=env) + print(f'res len {len(df3)}') + + env.finalize() \ No newline at end of file diff --git a/python/pygcylon/setup.py b/python/pygcylon/setup.py index be66a2f4c..662d58b25 100644 --- a/python/pygcylon/setup.py +++ b/python/pygcylon/setup.py @@ -163,7 +163,7 @@ def build_extensions(self): packages = find_packages(include=["pygcylon", "pygcylon.*"]) -compile_time_env = {'CYTHON_GLOO': False, 'CYTHON_UCC': False} +compile_time_env = {'CYTHON_GLOO': False, 'CYTHON_UCC': False, 'CYTHON_UCX': False} setup( name="pygcylon", packages=packages, From 6a03ab3aee5b4d224d1811bdad192436b86e05f9 Mon Sep 17 00:00:00 2001 From: Arup Sarker Date: Wed, 17 Aug 2022 17:15:55 -0400 Subject: [PATCH 24/28] [Cylon] Handle corner case for slice test Signed-off-by: Arup Sarker --- cpp/test/slice_test.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index ec0391633..ae6e8110b 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -23,6 +23,7 @@ void testDistSlice(std::shared_ptr
& global_table, std::shared_ptr
& table, int64_t offset, int64_t length) { + std::shared_ptr
out; auto ctx = table->GetContext(); std::shared_ptr arrow_output; @@ -33,15 +34,18 @@ void testDistSlice(std::shared_ptr
& global_table, std::vector> gathered; CHECK_CYLON_STATUS(ctx->GetCommunicator()->Gather(out, /*root*/0, /*gather_from_root*/true, &gathered)); - if (RANK == 0) { std::shared_ptr
result, global_out; CHECK_CYLON_STATUS(Slice(global_table, offset, length, &global_out)); + if(length <= 0) + return; + for (size_t i = 0; i < gathered.size(); i++) { INFO("gathered " << i << ":" << gathered[i]->Rows()); } REQUIRE(WORLD_SZ == (int) gathered.size()); + CHECK_CYLON_STATUS(Merge(gathered, result)); CHECK_ARROW_EQUAL(global_out->get_table(), result->get_table()); } @@ -115,7 +119,11 @@ TEST_CASE("Dist Slice testing", "[dist slice]") { testDistSlice(global_table, table1, 15, 8); } - SECTION("dist_sort_test_4_one_empty_table") { + SECTION("dist_slice_test_4_multiple_table") { + testDistSlice(global_table, table1, 1, 0); + } + + SECTION("dist_sort_test_5_one_empty_table") { auto pool = cylon::ToArrowPool(ctx); From 4bc27f936ca55b9a2a9b6fc099c25bc32dc1938d Mon Sep 17 00:00:00 2001 From: Gregor von Laszewski Date: Thu, 18 Aug 2022 16:05:13 -0400 Subject: [PATCH 25/28] Create README-summit.md (#602) * Create README-summit.md Detailed description of how to compile cylon on summit * Update README-summit.md * Update README-summit.md --- README-summit.md | 103 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 README-summit.md diff --git a/README-summit.md b/README-summit.md new file mode 100644 index 000000000..1428d3e3e --- /dev/null +++ b/README-summit.md @@ -0,0 +1,103 @@ +# SUMMIT Installation guide + +This document describes the instalation on summit. +The instalition is first described in detail and then an abbreviated +instalation with a Makefile is provided. + +Notation: + +* `login$` denotes the login node +* `compute$` donates a compute node + +## Details + +### Activites on the login node + +```bash +login$ module purge +login$ module load gcc/9.3.0 +login$ module load spectrum-mpi/10.4.0.3-20210112 +login$ module load python/3.8-anaconda3 +``` + +Next check the python version + +```bash +login$ which python +> /sw/summit/python/3.8/anaconda3/2020.07-rhel8/bin/python +login$ python -V +> Python 3.8.3 +``` + +Now create a venv called `CYLON` in the home directory and activate it + +```bash +login$ python -m venv ~/CYLON +login$ source ~/CYLON/bin/activate +``` + +```bash +login$ which python +> ~/CYLON/bin/python +login$ python -V +> Python 3.8.3 +``` + +Update pip and install pytest: + +```bash +login$ pip install pip -U +login$ pip install pytest +``` + +Checkout the cylon code + +```bash +cd ~git clone https://github.com/cylondata/cylon.git +``` + +Now we obtain two interactive compute nodes + +```bash +login$ bsub -Is -W 0:30 -nnodes 2 -P gen150_bench $SHELL +``` + +After you get them you will be in a compute node + +### Activities on the compute node + +```bash +compute$ module purge +compute$ module load gcc/9.3.0 +compute$ module load spectrum-mpi/10.4.0.3-20210112 +compute$ module load python/3.8-anaconda3 +compute$ source ~/CYLON/bin/activate +compute$ export CC=`which gcc` +compute$ export CXX=`which g++` +compute$ export ARROW_DEFAULT_MEMORY_POOL=system +compute$ CC=gcc MPICC=mpicc pip install --no-binary mpi4py install mpi4py +compute$ pip install pytest-mpi +compute$ pip install cmake +compute$ pip install numpy +compute$ ./build.sh -pyenv ~/CYLON -bpath $(pwd)/build --cpp --test --cmake-flags "-DMPI_C_COMPILER=$(which mpicc) -DMPI_CXX_COMPILER=$(which mpicxx)" -j 4 +``` + +The compilation will take some time. After it is completed you can conduct a test with + +```bash +compute$ cd ~/cylon/build +compute$ jsrun -n 1 -c 4 -a 1 ./build/bin/join_example m n 1000 0.9 +``` + +If everything is ok, you will see at the end of the test output + +``` +... +================================================ +All tests passed (66 assertions in 4 test cases) +``` + + +### Running batch scripts + +Please note that the module load and the source of the CYLON venv must be at the beginning of each batsch script you want to use cylon in. From f20c11954aa5e310d949a7305982a6c65d38260f Mon Sep 17 00:00:00 2001 From: Gregor von Laszewski Date: Thu, 18 Aug 2022 16:44:17 -0400 Subject: [PATCH 26/28] Update README-summit.md (#603) add cmake path --- README-summit.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/README-summit.md b/README-summit.md index 1428d3e3e..977d7b228 100644 --- a/README-summit.md +++ b/README-summit.md @@ -79,6 +79,9 @@ compute$ CC=gcc MPICC=mpicc pip install --no-binary mpi4py install mpi4py compute$ pip install pytest-mpi compute$ pip install cmake compute$ pip install numpy +compute$ pip install cmake +compute$ export PATH=/ccs/home/gregorvl/.local/summit/anaconda3/2020.07/3.8/bin:$PATH + compute$ ./build.sh -pyenv ~/CYLON -bpath $(pwd)/build --cpp --test --cmake-flags "-DMPI_C_COMPILER=$(which mpicc) -DMPI_CXX_COMPILER=$(which mpicxx)" -j 4 ``` From d99a6f2fd0e2284db4c6842090dffef5795d0733 Mon Sep 17 00:00:00 2001 From: niranda perera Date: Thu, 18 Aug 2022 20:14:36 -0400 Subject: [PATCH 27/28] adding custom mpirun params cmake var (#604) * adding custom mpirun params cmake var * minor change --- cpp/CMakeLists.txt | 11 +++++++++++ cpp/test/CMakeLists.txt | 6 +++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 9b6e7c2ec..f4ea961f1 100755 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -161,6 +161,17 @@ if (CYLON_CUSTOM_MPIRUN) endif (CYLON_CUSTOM_MPIRUN) message(STATUS "MPI executable: ${MPIEXEC_EXECUTABLE}") +if (CYLON_CUSTOM_MPIRUN_PARAMS) + set(MPIEXEC_EXECUTABLE_PARAMS ${CYLON_CUSTOM_MPIRUN_PARAMS}) +else () + if (WIN32) + set(MPIEXEC_EXECUTABLE_PARAMS "") + else () + set(MPIEXEC_EXECUTABLE_PARAMS --allow-run-as-root --oversubscribe) + endif () +endif (CYLON_CUSTOM_MPIRUN_PARAMS) +message(STATUS "MPI executable params: ${MPIEXEC_EXECUTABLE_PARAMS}") + include_directories(SYSTEM ${MPI_CXX_INCLUDE_PATH}) # Glog diff --git a/cpp/test/CMakeLists.txt b/cpp/test/CMakeLists.txt index 2db81f853..9eda441cb 100644 --- a/cpp/test/CMakeLists.txt +++ b/cpp/test/CMakeLists.txt @@ -73,11 +73,11 @@ endfunction(cylon_add_test) function(cylon_run_test TESTNAME parallelism comm_args) set(exec_name "${TESTNAME}_${parallelism}_${comm_args}") if (WIN32) - set(test_params -np ${parallelism} "${CMAKE_BINARY_DIR}/bin/${CMAKE_BUILD_TYPE}/${TESTNAME}" --comm "${comm_args}") + set(test_params -n ${parallelism} "${CMAKE_BINARY_DIR}/bin/${CMAKE_BUILD_TYPE}/${TESTNAME}" --comm "${comm_args}") else () - set(test_params --allow-run-as-root --oversubscribe -np ${parallelism} "${CMAKE_BINARY_DIR}/bin/${TESTNAME}" --comm "${comm_args}") + set(test_params -n ${parallelism} "${CMAKE_BINARY_DIR}/bin/${TESTNAME}" --comm "${comm_args}") endif () - add_test(NAME ${exec_name} COMMAND ${MPIEXEC_EXECUTABLE} ${test_params}) + add_test(NAME ${exec_name} COMMAND ${MPIEXEC_EXECUTABLE} ${MPIEXEC_EXECUTABLE_PARAMS} ${test_params}) endfunction(cylon_run_test) #Add tests as follows ... From b50efb20c3394fc2418ff5a083d58e7201aaf0ca Mon Sep 17 00:00:00 2001 From: niranda perera Date: Fri, 19 Aug 2022 11:46:05 -0400 Subject: [PATCH 28/28] changing merge to support empty tables --- cpp/src/cylon/indexing/slice.cpp | 13 +++++++++-- cpp/src/cylon/table.cpp | 10 +++++---- cpp/test/slice_test.cpp | 37 ++++++++++++++++++++------------ 3 files changed, 40 insertions(+), 20 deletions(-) diff --git a/cpp/src/cylon/indexing/slice.cpp b/cpp/src/cylon/indexing/slice.cpp index 22b6b704c..73ac1f08c 100644 --- a/cpp/src/cylon/indexing/slice.cpp +++ b/cpp/src/cylon/indexing/slice.cpp @@ -15,8 +15,6 @@ #include #include -#include - #include #include #include @@ -71,6 +69,17 @@ Status distributed_slice_impl(const std::shared_ptr
&in, int64_t rank = ctx->GetRank(); int64_t prefix_sum = std::accumulate(partition_lengths, partition_lengths + rank, kZero); + int64_t total_length = std::accumulate(partition_lengths + rank, + partition_lengths + ctx->GetWorldSize(), + prefix_sum); + if (global_offset > total_length) { + return {Code::Invalid, "global offset exceeds total length of the dist table"}; + } + // adjust global length if it exceeds total_length + if (global_offset + global_length > total_length) { + global_length = total_length - global_offset; + } + int64_t this_length = *(partition_lengths + rank); assert(this_length == in->Rows()); diff --git a/cpp/src/cylon/table.cpp b/cpp/src/cylon/table.cpp index 6d163f157..507f72f45 100644 --- a/cpp/src/cylon/table.cpp +++ b/cpp/src/cylon/table.cpp @@ -385,12 +385,14 @@ Status Merge(const std::vector> &ctables, std::vector> tables; tables.reserve(ctables.size()); for (const auto &t: ctables) { - if (t->Rows()) { - std::shared_ptr arrow; - t->ToArrowTable(arrow); - tables.push_back(std::move(arrow)); + if (!t->Empty()) { + tables.push_back(t->get_table()); } } + if (tables.empty()) { // means all tables are empty. return a cylon table from 0th table + tableOut = ctables[0]; + return Status::OK(); + } const auto &ctx = ctables[0]->GetContext(); const auto &concat_res = arrow::ConcatenateTables(tables); diff --git a/cpp/test/slice_test.cpp b/cpp/test/slice_test.cpp index ae6e8110b..27c79ba7a 100644 --- a/cpp/test/slice_test.cpp +++ b/cpp/test/slice_test.cpp @@ -38,9 +38,6 @@ void testDistSlice(std::shared_ptr
& global_table, std::shared_ptr
result, global_out; CHECK_CYLON_STATUS(Slice(global_table, offset, length, &global_out)); - if(length <= 0) - return; - for (size_t i = 0; i < gathered.size(); i++) { INFO("gathered " << i << ":" << gathered[i]->Rows()); } @@ -123,24 +120,36 @@ TEST_CASE("Dist Slice testing", "[dist slice]") { testDistSlice(global_table, table1, 1, 0); } - SECTION("dist_sort_test_5_one_empty_table") { + SECTION("dist_slice_test_5_multiple_table") { + testDistSlice(global_table, table1, 0, 0); + } + + SECTION("dist_slice_test_5_multiple_table") { + testDistSlice(global_table, table1, global_table->Rows(), INT64_MAX); + } + SECTION("dist_slice_test_5_multiple_table") { + testDistSlice(global_table, table1, global_table->Rows() - 1, 1000); + } + + SECTION("failing case") { + REQUIRE_FALSE(DistributedSlice(table1, 1000, 1000, nullptr).is_ok()); + } + + SECTION("dist_sort_test_5_one_empty_table") { auto pool = cylon::ToArrowPool(ctx); std::shared_ptr arrow_empty_table; auto arrow_status = util::CreateEmptyTable(table1->get_table()->schema(), - &arrow_empty_table, pool); - auto empty_table = std::make_shared
(ctx, arrow_empty_table); - table1 = empty_table; + &arrow_empty_table, pool); + REQUIRE(arrow_status.ok()); + auto empty_table = std::make_shared
(ctx, std::move(arrow_empty_table)); - std::shared_ptr
out, out2; - auto ctx = table1->GetContext(); + std::shared_ptr
out1, out2; std::shared_ptr arrow_output; - auto status = DistributedSlice(table1, 3, 10, &out); - REQUIRE(status.is_ok()); - status = DistributedSlice(table1, 15, 5, &out2); - REQUIRE(status.is_ok()); - CHECK_ARROW_EQUAL(out->get_table(), out2->get_table()); + CHECK_CYLON_STATUS(DistributedSlice(empty_table, 0, 10, &out1)); + CHECK_CYLON_STATUS(DistributedSlice(empty_table, 0, 5, &out2)); + CHECK_ARROW_EQUAL(out1->get_table(), out2->get_table()); } }