diff --git a/mindspore/ccsrc/minddata/dataset/core/tensor_row.cc b/mindspore/ccsrc/minddata/dataset/core/tensor_row.cc index 8c3aa3864e..83701a46ad 100644 --- a/mindspore/ccsrc/minddata/dataset/core/tensor_row.cc +++ b/mindspore/ccsrc/minddata/dataset/core/tensor_row.cc @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,15 +21,21 @@ namespace mindspore { namespace dataset { -TensorRow::TensorRow() noexcept : id_(kDefaultRowId), path_({}) {} +TensorRow::TensorRow() noexcept : id_(kDefaultRowId), path_({}), tensor_row_flag_(kFlagNone) {} -TensorRow::TensorRow(size_type n, TensorRow::value_type t) noexcept : id_(kDefaultRowId), path_({}), row_(n, t) {} +TensorRow::TensorRow(size_type n, TensorRow::value_type t) noexcept + : id_(kDefaultRowId), path_({}), row_(n, t), tensor_row_flag_(kFlagNone) {} -TensorRow::TensorRow(const TensorRow::vector_type &v) : id_(kDefaultRowId), path_({}), row_(v) {} +TensorRow::TensorRow(const TensorRow::vector_type &v) + : id_(kDefaultRowId), path_({}), row_(v), tensor_row_flag_(kFlagNone) {} -TensorRow::TensorRow(row_id_type id, const std::initializer_list &lst) : id_(id), path_({}), row_(lst) {} +TensorRow::TensorRow(row_id_type id, const std::initializer_list &lst) + : id_(id), path_({}), row_(lst), tensor_row_flag_(kFlagNone) {} -TensorRow::TensorRow(const TensorRow &tr) : id_(tr.id_), path_(tr.path_), row_(tr.row_) {} +TensorRow::TensorRow(const TensorRow &tr) + : id_(tr.id_), path_(tr.path_), row_(tr.row_), tensor_row_flag_(tr.tensor_row_flag_) {} + +TensorRow::TensorRow(TensorRow::TensorRowFlags flag) : tensor_row_flag_(flag) {} TensorRow &TensorRow::operator=(const TensorRow &tr) { if (this == &tr) { @@ -38,23 +44,27 @@ TensorRow &TensorRow::operator=(const TensorRow &tr) { row_ = tr.row_; id_ = tr.id_; path_ = tr.path_; + tensor_row_flag_ = tr.tensor_row_flag_; return *this; } TensorRow &TensorRow::operator=(const std::initializer_list &lst) { row_ = lst; + tensor_row_flag_ = kFlagNone; return *this; } -TensorRow::TensorRow(TensorRow::vector_type &&v) noexcept : id_(kDefaultRowId), path_({}), row_(std::move(v)) {} +TensorRow::TensorRow(TensorRow::vector_type &&v) noexcept + : id_(kDefaultRowId), path_({}), row_(std::move(v)), tensor_row_flag_(kFlagNone) {} TensorRow::TensorRow(row_id_type id, std::initializer_list &&lst) noexcept - : id_(id), path_({}), row_(std::move(lst)) {} + : id_(id), path_({}), row_(std::move(lst)), tensor_row_flag_(kFlagNone) {} TensorRow::TensorRow(TensorRow &&tr) noexcept { id_ = tr.id_; path_ = std::move(tr.path_); row_ = std::move(tr.row_); + tensor_row_flag_ = tr.tensor_row_flag_; } TensorRow &TensorRow::operator=(TensorRow &&tr) noexcept { @@ -65,11 +75,13 @@ TensorRow &TensorRow::operator=(TensorRow &&tr) noexcept { id_ = tr.id_; tr.id_ = kDefaultRowId; path_ = std::move(tr.path_); + tensor_row_flag_ = tr.tensor_row_flag_; return *this; } TensorRow &TensorRow::operator=(std::initializer_list &&lst) noexcept { row_ = std::move(lst); + tensor_row_flag_ = kFlagNone; return *this; } diff --git a/mindspore/ccsrc/minddata/dataset/core/tensor_row.h b/mindspore/ccsrc/minddata/dataset/core/tensor_row.h index c53899d735..5d58667b87 100644 --- a/mindspore/ccsrc/minddata/dataset/core/tensor_row.h +++ b/mindspore/ccsrc/minddata/dataset/core/tensor_row.h @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,14 @@ class TensorRow { public: static constexpr row_id_type kDefaultRowId = -1; // Default row id + enum TensorRowFlags : uint32_t { + kFlagNone = 0, + kFlagEOF = 1, // The buffer is an eof end-of-data msg + kFlagEOE = 1u << 1, // The buffer is an eoe end-of-epoch msg + kFlagWait = 1u << 2, // The buffer is an control signal for workers to suspend operations + kFlagQuit = 1u << 3 // The buffer is a control signal for workers to quit + }; + // Type definitions using size_type = dsize_t; using value_type = std::shared_ptr; @@ -222,10 +230,25 @@ class TensorRow { const_iterator end() const { return row_.end(); } + // Convenience getter functions for flag checking + bool eof() const { return (static_cast(tensor_row_flag_) & static_cast(kFlagEOF)); } + + bool eoe() const { return (static_cast(tensor_row_flag_) & static_cast(kFlagEOE)); } + + bool wait() const { return (static_cast(tensor_row_flag_) & static_cast(kFlagWait)); } + + bool quit() const { return (static_cast(tensor_row_flag_) & static_cast(kFlagQuit)); } + + TensorRowFlags Flags() { return tensor_row_flag_; } + + explicit TensorRow(TensorRowFlags); + protected: row_id_type id_; std::vector path_; std::vector> row_; + + TensorRowFlags tensor_row_flag_; }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.cc b/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.cc index b2ae11a13e..b052f8f1f5 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.cc @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -114,27 +114,6 @@ Status CacheClient::WriteRow(const TensorRow &row, row_id_type *row_id_from_serv return Status::OK(); } -Status CacheClient::WriteBuffer(std::unique_ptr &&in) const { - std::unique_ptr db_ptr = std::move(in); - auto num_rows = db_ptr->NumRows(); - // We will send the requests async first on all rows and do a final wait. - if (num_rows > 0) { - auto arr = std::make_unique[]>(num_rows); - for (auto i = 0; i < num_rows; ++i) { - TensorRow row; - RETURN_IF_NOT_OK(db_ptr->PopRow(&row)); - arr[i] = std::make_shared(this); - RETURN_IF_NOT_OK(arr[i]->SerializeCacheRowRequest(this, row)); - RETURN_IF_NOT_OK(PushRequest(arr[i])); - } - // Now we wait for them to come back - for (auto i = 0; i < num_rows; ++i) { - RETURN_IF_NOT_OK(arr[i]->Wait()); - } - } - return Status::OK(); -} - Status CacheClient::AsyncWriteRow(const TensorRow &row) { if (async_buffer_stream_ == nullptr) { return Status(StatusCode::kMDNotImplementedYet); @@ -143,34 +122,6 @@ Status CacheClient::AsyncWriteRow(const TensorRow &row) { return Status::OK(); } -Status CacheClient::AsyncWriteBuffer(std::unique_ptr &&in) { - if (async_buffer_stream_ == nullptr) { - return Status(StatusCode::kMDNotImplementedYet); - } else { - Status rc; - std::unique_ptr tensor_table = std::make_unique(); - auto num_rows = in->NumRows(); - if (num_rows > 0) { - for (auto i = 0; i < num_rows; ++i) { - TensorRow row; - RETURN_IF_NOT_OK(in->PopRow(&row)); - rc = AsyncWriteRow(row); - if (rc.StatusCode() == StatusCode::kMDNotImplementedYet) { - tensor_table->push_back(row); - } else if (rc.IsError()) { - return rc; - } - } - } - // If not all of them can be sent async, return what's left back to the caller. - if (!tensor_table->empty()) { - in->set_tensor_table(std::move(tensor_table)); - return Status(StatusCode::kMDNotImplementedYet); - } - } - return Status::OK(); -} - Status CacheClient::GetRows(const std::vector &row_id, TensorTable *out) const { RETURN_UNEXPECTED_IF_NULL(out); auto rq = std::make_shared(this, row_id); diff --git a/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.h b/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.h index 7c6f2ba524..98d4de247c 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.h +++ b/mindspore/ccsrc/minddata/dataset/engine/cache/cache_client.h @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -156,11 +156,6 @@ class CacheClient { /// \return return code Status WriteRow(const TensorRow &row, row_id_type *row_id_from_server = nullptr) const; - /// \brief Send a DataBuffer to the cache server - /// \param in Unique pointer of the DataBuffer to be cached - /// \return return code - Status WriteBuffer(std::unique_ptr &&in) const; - /// \brief Fetch a list of rows from the cache server. An empty TensorRow will be returned if there is /// any cache miss /// \param row_id A vector of row id's @@ -257,6 +252,9 @@ class CacheClient { return false; } + /// \brief Serialize a Tensor into the async buffer. + Status AsyncWriteRow(const TensorRow &row); + // Default size of the async write buffer constexpr static int64_t kAsyncBufferSize = 16 * 1048576L; // 16M constexpr static int32_t kNumAsyncBuffer = 3; @@ -269,8 +267,6 @@ class CacheClient { return Status::OK(); } - Status AsyncWriteBuffer(std::unique_ptr &&in); - private: mutable RWLock mux_; uint64_t cache_mem_sz_; @@ -354,9 +350,6 @@ class CacheClient { std::atomic next_addr_; }; std::shared_ptr async_buffer_stream_; - - /// \brief Serialize a Tensor into the async buffer. - Status AsyncWriteRow(const TensorRow &row); }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/cache/perf/cache_pipeline_run.cc b/mindspore/ccsrc/minddata/dataset/engine/cache/perf/cache_pipeline_run.cc index cf87e6b475..0005f1b342 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/cache/perf/cache_pipeline_run.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/cache/perf/cache_pipeline_run.cc @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -272,7 +272,6 @@ Status CachePipelineRun::WriterWorkerEntry(int32_t worker_id) { int64_t min_val = std::numeric_limits::max(); int64_t max_val = 0; int64_t total_val = 0; - int64_t cnt = 0; std::vector duration; duration.reserve(num_rows_ / num_pipelines_ / cfg_.num_parallel_workers()); bool resource_err = false; @@ -291,8 +290,6 @@ Status CachePipelineRun::WriterWorkerEntry(int32_t worker_id) { } // Once we hit resource error, we drain the io block. No point to send anything to the server. if (!resource_err) { - auto buffer = std::make_unique(cnt++, DataBuffer::kDeBFlagNone); - auto tensor_table = std::make_unique(); for (auto id : keys) { TensorRow row; std::shared_ptr element; @@ -305,29 +302,27 @@ Status CachePipelineRun::WriterWorkerEntry(int32_t worker_id) { *it = i; } row.push_back(std::move(element)); - tensor_table->push_back(std::move(row)); - } - buffer->set_tensor_table(std::move(tensor_table)); - // Measure the time to call WriteBuffer - auto start_tick = std::chrono::steady_clock::now(); - rc = cc_->AsyncWriteBuffer(std::move(buffer)); - auto end_tick = std::chrono::steady_clock::now(); - if (rc.IsError()) { - if (rc == StatusCode::kMDOutOfMemory || rc == StatusCode::kMDNoSpace) { - MS_LOG(WARNING) << "Pipeline number " << my_pipeline_ + 1 << " worker id " << worker_id << ": " - << rc.ToString(); - resource_err = true; - cc_->ServerRunningOutOfResources(); - continue; + // Measure the time to call WriteBuffer + auto start_tick = std::chrono::steady_clock::now(); + rc = cc_->AsyncWriteRow(std::move(row)); + auto end_tick = std::chrono::steady_clock::now(); + if (rc.IsError()) { + if (rc == StatusCode::kMDOutOfMemory || rc == StatusCode::kMDNoSpace) { + MS_LOG(WARNING) << "Pipeline number " << my_pipeline_ + 1 << " worker id " << worker_id << ": " + << rc.ToString(); + resource_err = true; + cc_->ServerRunningOutOfResources(); + continue; + } else { + return rc; + } } else { - return rc; + int64_t ms = std::chrono::duration_cast(end_tick - start_tick).count(); + min_val = std::min(min_val, ms); + max_val = std::max(max_val, ms); + duration.push_back(ms); + total_val += ms; } - } else { - int64_t ms = std::chrono::duration_cast(end_tick - start_tick).count(); - min_val = std::min(min_val, ms); - max_val = std::max(max_val, ms); - duration.push_back(ms); - total_val += ms; } } } while (true); diff --git a/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.cc b/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.cc index 3c654ea9c9..4139322b64 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.cc @@ -1,5 +1,5 @@ /** - * Copyright 2019 Huawei Technologies Co., Ltd + * Copyright 2019-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,13 +27,9 @@ namespace mindspore { namespace dataset { -// Constructor of the IteratorBase -IteratorBase::IteratorBase() : curr_buffer_(nullptr), eof_handled_(false) {} - -IteratorBase::~IteratorBase() = default; // Fetches one row of data from the iterator as a column map. -Status IteratorBase::GetNextAsMap(TensorMap *out_map) { +Status DatasetIterator::GetNextAsMap(TensorMap *out_map) { if (out_map == nullptr) { RETURN_STATUS_UNEXPECTED("Null output map in iterator!"); } @@ -67,63 +63,14 @@ Status IteratorBase::GetNextAsMap(TensorMap *out_map) { return Status::OK(); } - -// Fetches one row of data from the iterator. -// The base class version simply performs error handling and returns empty row. Actual -// functionality exists in the derived versions of this function. -Status IteratorBase::FetchNextTensorRow(TensorRow *out_row) { - if (out_row == nullptr) { - RETURN_STATUS_UNEXPECTED("Null output row in iterator!"); - } - - // clear the old tensor row - out_row->clear(); - - return Status::OK(); -} - -Status IteratorBase::GetNextAsOrderedPair(std::vector>> *vec) { - CHECK_FAIL_RETURN_UNEXPECTED(vec != nullptr && vec->empty(), "vec is null or non-empty."); - - TensorRow curr_row; - - RETURN_IF_NOT_OK(FetchNextTensorRow(&curr_row)); - RETURN_OK_IF_TRUE(curr_row.empty()); - - size_t num_cols = curr_row.size(); // num_cols is non-empty. - if (col_name_id_map_.empty()) col_name_id_map_ = this->GetColumnNameMap(); - // order the column names according to their ids - if (column_order_.empty()) { - const int32_t invalid_col_id = -1; - column_order_.resize(num_cols, {std::string(), invalid_col_id}); - for (const auto &itr : col_name_id_map_) { - int32_t ind = itr.second; - CHECK_FAIL_RETURN_UNEXPECTED(ind < num_cols && ind >= 0, "column id out of bounds."); - column_order_[ind] = std::make_pair(itr.first, ind); - } - // error check, make sure the ids in col_name_id_map are continuous and starts from 0 - for (const auto &col : column_order_) { - CHECK_FAIL_RETURN_UNEXPECTED(col.second != invalid_col_id, "column ids are not continuous."); - } - } - - vec->reserve(num_cols); - - for (const auto &col : column_order_) { - vec->emplace_back(std::make_pair(col.first, curr_row[col.second])); - } - - return Status::OK(); -} - // Constructor of the DatasetIterator DatasetIterator::DatasetIterator(std::shared_ptr exe_tree) - : IteratorBase(), - root_(exe_tree->root()), + : root_(exe_tree->root()), tracing_(nullptr), cur_batch_num_(0), cur_connector_size_(0), - cur_connector_capacity_(0) { + cur_connector_capacity_(0), + eof_handled_(false) { std::shared_ptr node; Status s = exe_tree->GetProfilingManager()->GetTracingNode(kDatasetIteratorTracingName, &node); if (s.IsOk()) { @@ -136,8 +83,11 @@ DatasetIterator::~DatasetIterator() = default; // Fetches one row of data from the iterator. Overrides the base class. This one fetches // from the tree root node directly. Status DatasetIterator::FetchNextTensorRow(TensorRow *out_row) { - // Common code init and error checking in the base class. - RETURN_IF_NOT_OK(IteratorBase::FetchNextTensorRow(out_row)); + if (out_row == nullptr) { + RETURN_STATUS_UNEXPECTED("Null output row in iterator!"); + } + // clear the old tensor row + out_row->clear(); bool isProfilingEnable = root_->Tree()->GetProfilingManager()->IsProfilingEnable(); @@ -149,41 +99,36 @@ Status DatasetIterator::FetchNextTensorRow(TensorRow *out_row) { } // Check if we need to get a new DataBuffer to iterate. - if (curr_buffer_ == nullptr || curr_buffer_->NumRows() == 0) { - if (tracing_ != nullptr) { - cur_connector_size_ = root_->ConnectorSize(); - cur_connector_capacity_ = root_->ConnectorCapacity(); - } - RETURN_IF_NOT_OK(root_->GetNextBuffer(&curr_buffer_)); - // Since GetNextBuffer was used rather than GetNextInput(), it means we need to manually - // handle eoe and eof messages here. - // - // An eoe buffer means we have iterated an epoch. - // The next buffer in the pipeline might be an EOF or a databuffer for next epoch - if (curr_buffer_->eoe()) { - MS_LOG(INFO) << "End of data iteration."; - curr_buffer_.reset(); // explicitly free the eoe buffer - if (isProfilingEnable) { - root_->Tree()->SetEpochEnd(); - } - return Status::OK(); + if (tracing_ != nullptr) { + cur_connector_size_ = root_->ConnectorSize(); + cur_connector_capacity_ = root_->ConnectorCapacity(); + } + RETURN_IF_NOT_OK(root_->GetNextRow(out_row)); + + // Since GetNextBuffer was used rather than GetNextInput(), it means we need to manually + // handle eoe and eof messages here. + // + // An eoe buffer means we have iterated an epoch. + // The next buffer in the pipeline might be an EOF or a databuffer for next epoch + if (out_row->eoe()) { + MS_LOG(INFO) << "End of data iteration."; + if (isProfilingEnable) { + root_->Tree()->SetEpochEnd(); } + return Status::OK(); + } - // An eof buffer means it is the end of execution and all operators are shutting down. - // Because there is no more data to return to the caller, this will change `eof_handled_` state and - // returns status unexpected error. - if (curr_buffer_->eof()) { - eof_handled_ = true; - curr_buffer_.reset(); // explicitly free the eof buffer - root_->Tree()->SetFinished(); - std::string err = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs."; - RETURN_STATUS_UNEXPECTED(err); - } + // An eof buffer means it is the end of execution and all operators are shutting down. + // Because there is no more data to return to the caller, this will change `eof_handled_` state and + // returns status unexpected error. + if (out_row->eof()) { + eof_handled_ = true; + root_->Tree()->SetFinished(); + std::string err = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs."; + RETURN_STATUS_UNEXPECTED(err); } - // If we got this far, now it's time to pop that next row for return to caller - RETURN_IF_NOT_OK(curr_buffer_->PopRow(out_row)); if (tracing_ != nullptr) { cur_batch_num_++; tracing_->Record(CONNECTOR_DEPTH, cur_connector_capacity_, cur_batch_num_, cur_connector_size_, @@ -192,33 +137,6 @@ Status DatasetIterator::FetchNextTensorRow(TensorRow *out_row) { return Status::OK(); } -Status DatasetIterator::GetOutputShapes(std::vector *out_shapes) { - if (out_shapes == nullptr) { - RETURN_STATUS_UNEXPECTED("Null output shape argument"); - } - if (device_queue_row_.empty()) { - RETURN_IF_NOT_OK(FetchNextTensorRow(&device_queue_row_)); - } - for (const auto ts : device_queue_row_) { - out_shapes->push_back(ts->shape()); - } - - return Status::OK(); -} - -Status DatasetIterator::GetOutputTypes(std::vector *out_types) { - if (out_types == nullptr) { - RETURN_STATUS_UNEXPECTED("Null output type argument"); - } - if (device_queue_row_.empty()) { - RETURN_IF_NOT_OK(FetchNextTensorRow(&device_queue_row_)); - } - for (const auto ts : device_queue_row_) { - out_types->push_back(ts->type()); - } - return Status::OK(); -} - // Getter std::unordered_map DatasetIterator::GetColumnNameMap() const { return root_->column_name_id_map(); @@ -226,15 +144,16 @@ std::unordered_map DatasetIterator::GetColumnNameMap() con // Constructor of the ChildIterator ChildIterator::ChildIterator(DatasetOp *current_op, int32_t worker_id, int32_t child_idx) - : IteratorBase(), current_op_(current_op), child_idx_(child_idx), worker_id_(worker_id), end_epoch_(false) {} + : current_op_(current_op), child_idx_(child_idx), worker_id_(worker_id), end_epoch_(false), eof_handled_(false) {} ChildIterator::~ChildIterator() { current_op_ = nullptr; } // Fetches one row of data from the iterator. Overrides the base class. This one fetches // only from the child/worker id as given from the constructor. Status ChildIterator::FetchNextTensorRow(TensorRow *out_row) { - // Common code init and error checking in the base class. - RETURN_IF_NOT_OK(IteratorBase::FetchNextTensorRow(out_row)); + RETURN_UNEXPECTED_IF_NULL(out_row); + // clear the old tensor row + out_row->clear(); // Once eof is handled, always return empty row. Class must be destroyed and recreated if you // want to iterate again. @@ -243,32 +162,24 @@ Status ChildIterator::FetchNextTensorRow(TensorRow *out_row) { RETURN_STATUS_UNEXPECTED(err); } - // Check if we need to get a new DataBuffer to iterate. - if (curr_buffer_ == nullptr || curr_buffer_->NumRows() == 0) { - // GetNextInput() depends on current_op's EoeReceived. So, EOE buffer might be already be handled and - // this child iterator might not see EOE buffer. - RETURN_IF_NOT_OK(current_op_->GetNextInput(&curr_buffer_, worker_id_, child_idx_)); - - // If an eoe is picked up here, we simply return an empty vector and it's up to the - // caller to decide what it wants to do next. - if (curr_buffer_->eoe()) { - MS_LOG(DEBUG) << "Child iterator picked up EOE."; - end_epoch_ = true; - return Status::OK(); - } else { - end_epoch_ = false; - } - - if (curr_buffer_->eof()) { - MS_LOG(DEBUG) << "Child iterator picked up EOF."; - eof_handled_ = true; - return Status::OK(); - } + RETURN_IF_NOT_OK(current_op_->child(child_idx_)->GetNextRow(out_row, worker_id_)); + // If an eoe is picked up here, we simply return an empty vector and it's up to the + // caller to decide what it wants to do next.TensorRow + if (out_row->eoe()) { + MS_LOG(DEBUG) << "(" << current_op_->NameWithID() << ", " << child_idx_ << ")" + << "Child iterator picked up EOE."; + end_epoch_ = true; + return Status::OK(); + } else { + end_epoch_ = false; } - // If we got this far, now it's time to pop that next row for return to caller - RETURN_IF_NOT_OK(curr_buffer_->PopRow(out_row)); - + if (out_row->eof()) { + MS_LOG(DEBUG) << "(" << current_op_->NameWithID() << ", " << child_idx_ << ")" + << "Child iterator picked up EOF."; + eof_handled_ = true; + *out_row = TensorRow(TensorRow::kFlagEOF); + } return Status::OK(); } @@ -285,11 +196,12 @@ Status ChildIterator::Drain() { return Status::OK(); } MS_LOG(DEBUG) << "Child draining buffers until eoe."; + TensorRow row; // else we drain until eoe or eof, eof here is for sanity check - while (!curr_buffer_->eoe() && !curr_buffer_->eof()) { - RETURN_IF_NOT_OK(current_op_->GetNextInput(&curr_buffer_, worker_id_, child_idx_)); + while (!row.eoe() && !row.eof()) { + RETURN_IF_NOT_OK(current_op_->child(child_idx_)->GetNextRow(&row, worker_id_)); } - if (curr_buffer_->eof()) { + if (row.eof()) { return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, "Child iterator picked up EOF in drain."); } return Status::OK(); diff --git a/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.h b/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.h index 4bca84938b..54b0768b19 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.h +++ b/mindspore/ccsrc/minddata/dataset/engine/dataset_iterator.h @@ -1,5 +1,5 @@ /** - * Copyright 2019 Huawei Technologies Co., Ltd + * Copyright 2019-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,18 +35,21 @@ using TensorMap = std::unordered_map>; // forward declare class ExecutionTree; -class DataBuffer; - -// IteratorBase class is used to iterate data from an executionTree one row at a time. -// The base class provides the general interface, whereas derived classes provide slightly -// different implementations. -class IteratorBase { +// The DatasetIterator derived class is for fetching rows off the end/root of the execution tree. +class DatasetIterator { public: - // Constructor of IteratorBase - IteratorBase(); + // Constructor of the DatasetIterator + // @param exe_tree The execution tree we want to pull/iterate the data from using it's root node. + explicit DatasetIterator(std::shared_ptr exe_tree); // Destructor - virtual ~IteratorBase(); + ~DatasetIterator(); + + // Getter + // @return The string to column id mapping. + std::unordered_map GetColumnNameMap() const; + + bool eof_handled() const { return eof_handled_; } // Fetches one row of data from the iterator. // the base class version simply performs error handling and returns empty row. Actual @@ -57,63 +60,12 @@ class IteratorBase { // @note The position of a Tensor/column might be different from the initial column order // in corresponding Dataset Op. User must be aware that MapOp, ZipOps, and others might change // the column ordering. - virtual Status FetchNextTensorRow(TensorRow *out_row); + Status FetchNextTensorRow(TensorRow *out_row); // Fetches one row of data from the iterator as a column map. // @return A unordered map from column name to shared pointer to Tensor. Status GetNextAsMap(TensorMap *out_map); - /// \brief return column_name, tensor pair in the order of its column id. - /// \param[out] vec - /// \return Error code - Status GetNextAsOrderedPair(std::vector>> *vec); - - // Getter - // @return T/F if this iterator is completely done after getting an eof - bool eof_handled() const { return eof_handled_; } - - // Getter - // @return The string to column id mapping. - virtual std::unordered_map GetColumnNameMap() const = 0; - - protected: - std::unique_ptr curr_buffer_; // holds the current buffer - bool eof_handled_; // T/F if this op got an eof - std::unordered_map col_name_id_map_; - std::vector> column_order_; // key: column name, val: column id -}; - -// The DatasetIterator derived class is for fetching rows off the end/root of the execution tree. -class DatasetIterator : public IteratorBase { - public: - // Constructor of the DatasetIterator - // @param exe_tree The execution tree we want to pull/iterate the data from using it's root node. - explicit DatasetIterator(std::shared_ptr exe_tree); - - // Destructor - ~DatasetIterator(); - - // Fetches one row of data from the iterator. Overrides the base class. This one fetches - // from the tree root node directly. - // @param out_row - A TensorRow (vector of shared pointers to Tensors). If any of the of data - // messages are encountered (such as eoe or eof), then an empty TensorRow is returned back. - // @return Status The status code returned - Status FetchNextTensorRow(TensorRow *out_row) override; - - // Fetches the next tensor row into device row, and returns it's shape. - // @param out_shapes - A vector of tensor shapes (one shape per column) - // @return Status The status code returned - Status GetOutputShapes(std::vector *out_shapes); - - // Fetches the next tensor row into device row, and returns it's shape. - // @param outShapes - A vector of tensor shapes (one shape per column) - // @return Status The status code returned - Status GetOutputTypes(std::vector *out_types); - - // Getter - // @return The string to column id mapping. - std::unordered_map GetColumnNameMap() const override; - private: std::shared_ptr root_; // saves the root of the executionTree TensorRow device_queue_row_; @@ -121,11 +73,14 @@ class DatasetIterator : public IteratorBase { int32_t cur_batch_num_; // current batch number,used for profiling int32_t cur_connector_size_; // current connector size of root op,used for profiling int32_t cur_connector_capacity_; // current connector capacity of root op, used for profiling + bool eof_handled_; // T/F if this op got an eof + std::unordered_map col_name_id_map_; + std::vector> column_order_; // key: column name, val: column id }; // The ChildIterator derived class is for fetching rows from intermediate nodes of execution tree. // This one should only be used by internal Dataset operators, rather than an end-user. -class ChildIterator : public IteratorBase { +class ChildIterator { public: // Constructor of the DatasetIterator // @param current_op - The parent op from which we'll fetch from it's children. @@ -141,7 +96,7 @@ class ChildIterator : public IteratorBase { // @param out_row - A TensorRow (vector of shared pointers to Tensors). If any of the of data // messages are encountered (such as eoe or eof), then an empty TensorRow is returned back. // @return Status The status code returned - Status FetchNextTensorRow(TensorRow *out_row) override; + Status FetchNextTensorRow(TensorRow *out_row); // This function drains buffer until next eoe has been received. // It will be a no-op if the previous row returned is empty. @@ -150,16 +105,21 @@ class ChildIterator : public IteratorBase { // Getter // @return The string to column id mapping. - std::unordered_map GetColumnNameMap() const override; + std::unordered_map GetColumnNameMap() const; // Return T/F if end of epoch bool end_of_epoch() { return end_epoch_; } + // Getter + // @return T/F if this iterator is completely done after getting an eof + bool eof_handled() const { return eof_handled_; } + private: DatasetOp *current_op_; // The parent operator. We consume from it's children. int32_t child_idx_; // The specific child this iterator will fetch from. int32_t worker_id_; // The worker id uses for fetching the child data. bool end_epoch_; // the flag used when an empty row has been returned. + bool eof_handled_; // T/F if this op got an eof }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.cc index 7d92d569ce..af3d8defc3 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.cc @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -71,9 +71,10 @@ Status BarrierOp::operator()() { // Loop until eof is true while (!eof_) { - // Create new table to put the new tensor rows - std::unique_ptr curr_table = std::make_unique(); - RETURN_IF_NOT_OK(prepare(curr_table.get())); + RETURN_IF_NOT_OK(prepare()); + // read the first row + TensorRow new_row; + RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); // If an eof got picked up during the above prepare, then we're done if (eof_) { @@ -82,92 +83,36 @@ Status BarrierOp::operator()() { // we have to output new buffer with possibly different buffer size, possibly one row while (!clean_up_) { - // 1. If a previous loop iteration sent the current table out, then create a new one. - - if (curr_table == nullptr) { - curr_table = std::make_unique(); - } + // 2 Block + RETURN_IF_NOT_OK(blockCond()); - // 2 fill the table. Note: clean_up mode might get turned on if epoch is finished - RETURN_IF_NOT_OK(fillBuffer(curr_table.get())); - - // 3 create and update buffer and send it to the out connector - if (!curr_table->empty()) { - std::unique_ptr curr_buffer = std::make_unique(buffer_id_, DataBuffer::kDeBFlagNone); - curr_buffer->set_tensor_table(std::move(curr_table)); - MS_LOG(DEBUG) << "Barrier operator finished one buffer, pushing, rows " << curr_buffer->NumRows() << ", cols " - << curr_buffer->NumCols() << ", map " << column_name_id_map_.size() << "."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(curr_buffer))); - buffer_id_++; - } + MS_LOG(DEBUG) << "Barrier operator finished one row, pushing, cols " << new_row.size() << ", map " + << column_name_id_map_.size() << "."; + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); + RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); } - // 4 handle drain state. if (clean_up_) { MS_LOG(DEBUG) << "Barrier operator sending epoch ending signal."; - // Send the eoe up. - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOE)))); + // 3 Send the eoe up. + RETURN_IF_NOT_OK(out_connector_->SendEOE()); } } - // 5 handle eof + // 4 handle eof // propagate eof here. MS_LOG(INFO) << "Barrier operator got EOF, propagating."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOF)))); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } // Handles preprocessing of the main loop, used when starting new epoch -Status BarrierOp::prepare(TensorQTable *const table) { +Status BarrierOp::prepare() { MS_LOG(DEBUG) << "Barrier operator prepares for new epoch."; clean_up_ = false; - buffer_id_ = 0; - if (table == nullptr) { - return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, - "BarrierOp prepare phase requires a tensor table."); - } - // fill initial row - TensorRow new_row = {}; - // use iterator to get next row and invoke pyfunc wait - RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); - - // If the first row fetching resulted in eof, then we are done. - if (eof_) { - return Status::OK(); - } - if (new_row.empty()) { - // This epoch is empty - return Status::OK(); - } - // Pack this first row into our tensor table - // first row we also have to check if we should block - RETURN_IF_NOT_OK(blockCond()); - - table->push_back(std::move(new_row)); - // the update code below shouldn't do anything bad if the column name already exists. return Status::OK(); } -// fillBuffer always expects a new table to fill -Status BarrierOp::fillBuffer(TensorQTable *const table) { - if (table == nullptr) { - return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, "BarrierOp fillBuffer null table pointer."); - } - TensorRow new_row = {}; - while (table->size() < static_cast(rows_per_buffer_)) { - RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); - // Early exit the loop if we got empty row from any of our child iterations - if (new_row.empty()) { - return Status::OK(); - } - // else we got a row so pack it into the tensor table. - RETURN_IF_NOT_OK(blockCond()); - - table->push_back(std::move(new_row)); - } - return Status::OK(); -} - // function executes a py_func and blocks until condition becomes true. Status BarrierOp::blockCond() { { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.h index e650298af5..6b13654db2 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/barrier_op.h @@ -1,5 +1,5 @@ /** - * Copyright 2020 Huawei Technologies Co., Ltd + * Copyright 2020-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -139,7 +139,7 @@ class BarrierOp : public PipelineOp { // Handles preprocessing of the main loop, used when starting new epoch // @param table - a table of tensors to be moved into a buffer - Status prepare(TensorQTable *const table); + Status prepare(); // This function calls takes a table repeatedly adds rows to it. // @param table - a table of tensors to be moved into a buffer @@ -152,7 +152,7 @@ class BarrierOp : public PipelineOp { Status blockCond(); private: - // clean up variable to return imcomplete buffer + // clean up variable to return incomplete buffer bool clean_up_; // end of file state, we stop reading data and shut down bool eof_; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.cc index 9d2c0efb3c..ac402eec0d 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.cc @@ -182,24 +182,21 @@ void BatchOp::Print(std::ostream &out, bool show_all) const { } } -Status BatchOp::BatchRows(const std::unique_ptr *src, const std::unique_ptr *dest, - dsize_t batch_size) { +Status BatchOp::BatchRows(const std::unique_ptr *src, TensorRow *dest, dsize_t batch_size) { if ((*src)->size() != batch_size) { RETURN_STATUS_UNEXPECTED("[Internal Batch ERROR] Source table size does not match the batch_size"); } if (batch_size == 1) { - TensorRow row = std::move((*src)->front()); - row.setPath({}); + *dest = std::move((*src)->front()); (*src)->pop_front(); - (*dest)->push_back(row); - for (const auto &tensor : (*dest)->front()) { + + for (const auto &tensor : (*dest)) { RETURN_IF_NOT_OK(tensor->ExpandDim(0)); } return Status::OK(); } - TensorRow batched_row; auto num_columns = (*src)->front().size(); for (size_t i = 0; i < num_columns; i++) { std::shared_ptr first_tensor = (*src)->at(0).at(i); // first row, column i @@ -234,11 +231,9 @@ Status BatchOp::BatchRows(const std::unique_ptr *src, const std::u } RETURN_IF_NOT_OK(Tensor::CreateFromVector(strings, new_shape, &new_tensor)); } - batched_row.emplace_back(new_tensor); + dest->emplace_back(new_tensor); } - (*dest)->emplace_back(batched_row); - return Status::OK(); } @@ -248,30 +243,26 @@ Status BatchOp::WorkerEntry(int32_t workerId) { RETURN_IF_NOT_OK(worker_queues_[workerId]->PopFront(&table_pair)); while (table_pair.second.ctrl_ != batchCtrl::kQuit) { if (table_pair.second.ctrl_ == batchCtrl::kEOE) { - RETURN_IF_NOT_OK(out_connector_->Add(workerId, std::make_unique(0, DataBuffer::kDeBFlagEOE))); + RETURN_IF_NOT_OK(out_connector_->SendEOE(workerId)); } else if (table_pair.second.ctrl_ == batchCtrl::kEOF) { - RETURN_IF_NOT_OK(out_connector_->Add(workerId, std::make_unique(0, DataBuffer::kDeBFlagEOF))); + RETURN_IF_NOT_OK(out_connector_->SendEOF(workerId)); } else if (table_pair.second.ctrl_ == batchCtrl::kNoCtrl) { - std::unique_ptr db = nullptr; - RETURN_IF_NOT_OK(MakeBatchedBuffer(std::move(table_pair), &db)); - RETURN_IF_NOT_OK(out_connector_->Add(workerId, std::move(db))); + TensorRow new_row; + RETURN_IF_NOT_OK(MakeBatchedBuffer(std::move(table_pair), &new_row)); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row), workerId)); } RETURN_IF_NOT_OK(worker_queues_[workerId]->PopFront(&table_pair)); } return Status::OK(); } -Status BatchOp::MakeBatchedBuffer(std::pair, CBatchInfo> table_pair, - std::unique_ptr *db) { +Status BatchOp::MakeBatchedBuffer(std::pair, CBatchInfo> table_pair, TensorRow *new_row) { RETURN_UNEXPECTED_IF_NULL(table_pair.first); #ifdef ENABLE_PYTHON if (!in_col_names_.empty()) RETURN_IF_NOT_OK(MapColumns(&table_pair)); // pass it through pyfunc #endif if (pad_) RETURN_IF_NOT_OK(PadColumns(&table_pair.first, pad_info_, column_name_id_map_)); // do padding if needed - (*db) = std::make_unique(table_pair.second.batch_num_, DataBuffer::kDeBFlagNone); - std::unique_ptr dest_table = std::make_unique(); - RETURN_IF_NOT_OK(BatchRows(&table_pair.first, &dest_table, table_pair.first->size())); - (*db)->set_tensor_table(std::move(dest_table)); + RETURN_IF_NOT_OK(BatchRows(&table_pair.first, new_row, table_pair.first->size())); return Status::OK(); } @@ -575,14 +566,14 @@ int64_t BatchOp::GetTreeBatchSize() { return start_batch_size_; } -Status BatchOp::GetNextRow(TensorRow *row) { +Status BatchOp::GetNextRowPullMode(TensorRow *row) { std::unique_ptr table = std::make_unique(); child_iterator_ = std::make_unique(this, 0, 0); int32_t cur_batch_size = 0; RETURN_IF_NOT_OK(GetBatchSize(&cur_batch_size, CBatchInfo(0, batch_num_, batch_cnt_))); for (int i = 0; i < cur_batch_size; i++) { TensorRow new_row; - RETURN_IF_NOT_OK(child_[0]->GetNextRow(&new_row)); + RETURN_IF_NOT_OK(child_[0]->GetNextRowPullMode(&new_row)); if (!new_row.empty()) { table->emplace_back(new_row); if (table->size() == static_cast(cur_batch_size)) break; @@ -592,13 +583,10 @@ Status BatchOp::GetNextRow(TensorRow *row) { } } } - std::unique_ptr out = std::make_unique(); RETURN_UNEXPECTED_IF_NULL(table); if (pad_) RETURN_IF_NOT_OK(PadColumns(&table, pad_info_, column_name_id_map_)); // do padding if needed if (!table->empty()) { - RETURN_IF_NOT_OK(BatchRows(&table, &out, table->size())); - CHECK_FAIL_RETURN_UNEXPECTED(out->size() == 1, "Batch returned 2 rows while 1 row was expected."); - *row = out->back(); + RETURN_IF_NOT_OK(BatchRows(&table, row, table->size())); batch_cnt_++; batch_num_++; } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.h index 196e125a30..454d9403e2 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/batch_op.h @@ -203,8 +203,7 @@ class BatchOp : public ParallelOp { // @param int32_t size - batch_size // @param const std::unordered_map& column_name_id_map - column names to index mapping // @return Status The status code returned - static Status BatchRows(const std::unique_ptr *src, const std::unique_ptr *dest, - dsize_t batch_size); + static Status BatchRows(const std::unique_ptr *src, TensorRow *dest, dsize_t batch_size); // @param table // @param const PadInfo &pad_info pad info @@ -226,8 +225,7 @@ class BatchOp : public ParallelOp { // Generate buffer with batched tensors // @return Status The status code returned - Status MakeBatchedBuffer(std::pair, CBatchInfo> table_pair, - std::unique_ptr *db); + Status MakeBatchedBuffer(std::pair, CBatchInfo> table_pair, TensorRow *new_row); #ifdef ENABLE_PYTHON // Function that calls pyfunc to perform map on batch @@ -259,7 +257,7 @@ class BatchOp : public ParallelOp { // @return Status The status code returned Status LaunchThreadsAndInitOp(); - Status GetNextRow(TensorRow *row) override; + Status GetNextRowPullMode(TensorRow *row) override; #ifdef ENABLE_PYTHON // Invoke batch size function with current BatchInfo to generate batch size. diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/bucket_batch_by_length_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/bucket_batch_by_length_op.cc index f6a81ff1c2..24515f411c 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/bucket_batch_by_length_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/bucket_batch_by_length_op.cc @@ -136,11 +136,11 @@ Status BucketBatchByLengthOp::operator()() { } // need to send EOE manually since we set state to idle in EoeRecieved() - std::unique_ptr eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(¤t_row)); } + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } @@ -198,13 +198,11 @@ Status BucketBatchByLengthOp::PadAndBatchBucket(int32_t bucket_index, int32_t ba // PadColumns will change the data in bucket RETURN_IF_NOT_OK(BatchOp::PadColumns(bucket, pad_info_copy, column_name_id_map_)); - std::unique_ptr batched_bucket = std::make_unique(); + TensorRow batched_bucket; RETURN_IF_NOT_OK(BatchOp::BatchRows(bucket, &batched_bucket, batch_size)); (*bucket)->clear(); - std::unique_ptr batched_buffer = std::make_unique(batch_count_, DataBuffer::kDeBFlagNone); - batched_buffer->set_tensor_table(std::move(batched_bucket)); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(batched_buffer))); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(batched_bucket), 0)); batch_count_++; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_sentence_piece_vocab_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_sentence_piece_vocab_op.cc index 98ea0aca1a..924b71f9b1 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_sentence_piece_vocab_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_sentence_piece_vocab_op.cc @@ -57,6 +57,7 @@ Status BuildSentencePieceVocabOp::operator()() { RETURN_IF_NOT_OK(sentence_queue_->EmplaceBack(new_row)); RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); CHECK_FAIL_RETURN_UNEXPECTED(!eoe_warning, "no op should be after from_dataset (repeat detected)"); eoe_warning = true; } @@ -91,8 +92,8 @@ Status BuildSentencePieceVocabOp::SentenceThread() { } vocab_->set_model_proto(model_proto); } - RETURN_IF_NOT_OK(out_connector_->Add(0, std::make_unique(0, DataBuffer::kDeBFlagEOE))); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::make_unique(0, DataBuffer::kDeBFlagEOF))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_vocab_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_vocab_op.cc index 3d5f929952..cb96bdc93c 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_vocab_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/build_vocab_op.cc @@ -112,6 +112,7 @@ Status BuildVocabOp::operator()() { RETURN_IF_NOT_OK(distributor_queue_->EmplaceBack(new_row)); RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); CHECK_FAIL_RETURN_UNEXPECTED(!eoe_warning, "no op should be after from_dataset (repeat detected)"); eoe_warning = true; } @@ -184,8 +185,8 @@ Status BuildVocabOp::CollectorThread() { for (const std::string &sp_tk : special_tokens_) vocab_->append_word(sp_tk); } - RETURN_IF_NOT_OK(out_connector_->Add(0, std::make_unique(0, DataBuffer::kDeBFlagEOE))); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::make_unique(0, DataBuffer::kDeBFlagEOF))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); // then use std::nth_element to partial sort return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_base_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_base_op.cc index 156e16341b..3150b06556 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_base_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_base_op.cc @@ -174,7 +174,6 @@ Status CacheBase::FetchSamplesToWorkers() { } Status CacheBase::FetchFromCache(int32_t worker_id) { - int64_t buffer_id = worker_id; std::unique_ptr blk; do { RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&blk)); @@ -185,9 +184,9 @@ Status CacheBase::FetchFromCache(int32_t worker_id) { wait_for_workers_post_.Set(); } } else if (blk->eof()) { - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::make_unique(0, DataBuffer::kDeBFlagEOF))); + RETURN_IF_NOT_OK(out_connector_->SendEOF(worker_id)); } else if (blk->eoe()) { - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::make_unique(0, DataBuffer::kDeBFlagEOE))); + RETURN_IF_NOT_OK(out_connector_->SendEOE(worker_id)); } else { std::vector keys; RETURN_IF_NOT_OK(blk->GetKeys(&keys)); @@ -195,8 +194,6 @@ Status CacheBase::FetchFromCache(int32_t worker_id) { // empty key is a quit signal for workers break; } - std::unique_ptr db = std::make_unique(buffer_id, DataBuffer::kDeBFlagNone); - std::unique_ptr que = std::make_unique(); for (auto row_id : keys) { TensorRow row; // Block until the row shows up in the pool. @@ -209,11 +206,8 @@ Status CacheBase::FetchFromCache(int32_t worker_id) { RETURN_STATUS_UNEXPECTED(errMsg); } } - que->push_back(std::move(row)); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(row), worker_id)); } - db->set_tensor_table(std::move(que)); - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::move(db))); - buffer_id += num_workers_; } } while (true); return Status::OK(); diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_merge_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_merge_op.cc index a696e7aa5b..0436a3e96a 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_merge_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_merge_op.cc @@ -76,30 +76,21 @@ Status CacheMergeOp::operator()() { // until it shows up in the pool. Status CacheMergeOp::WorkerEntry(int32_t worker_id) { TaskManager::FindMe()->Post(); - std::shared_ptr cache_hit_stream = child_[kCacheHitChildIdx]; - std::unique_ptr db_ptr; - RETURN_IF_NOT_OK(cache_hit_stream->GetNextBuffer(&db_ptr, worker_id)); - while (!db_ptr->eof()) { - if (db_ptr->eoe()) { + TensorRow new_row; + auto child_iterator = std::make_unique(this, worker_id, kCacheHitChildIdx); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&new_row)); + while (!new_row.eof()) { + if (new_row.eoe()) { RETURN_IF_NOT_OK(EoeReceived(worker_id)); - db_ptr.reset(); - RETURN_IF_NOT_OK(cache_hit_stream->GetNextBuffer(&db_ptr, worker_id)); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&new_row)); } else { - // See if there is any missing row - auto tbl = std::make_unique(); - while (db_ptr->NumRows() > 0) { - TensorRow row; - RETURN_IF_NOT_OK(db_ptr->PopRow(&row)); - if (row.empty()) { - auto row_id = row.getId(); - // Block until the row shows up in the pool. - RETURN_IF_NOT_OK(cache_miss_.PopFront(row_id, &row)); - } - tbl->push_back(std::move(row)); + if (new_row.empty()) { + auto row_id = new_row.getId(); + // Block until the row shows up in the pool. + RETURN_IF_NOT_OK(cache_miss_.PopFront(row_id, &new_row)); } - db_ptr->set_tensor_table(std::move(tbl)); - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::move(db_ptr))); - RETURN_IF_NOT_OK(cache_hit_stream->GetNextBuffer(&db_ptr, worker_id)); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row), worker_id)); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&new_row)); } } RETURN_IF_NOT_OK(EofReceived(worker_id)); @@ -111,16 +102,16 @@ Status CacheMergeOp::CacheMissWorkerEntry(int32_t workerId) { // We will simply pop TensorRow from the stream and insert them into the pool and // wake up any worker that is awaiting on the missing TensorRow. // If we see an eoe, ignore it. For eof, we exit. - std::shared_ptr cache_missing_stream = child_[kCacheMissChildIdx]; // Before we start, cache the schema at the server. Pick one of the workers // do it. The schema should have been done at prepare time. if (workerId == 0) { RETURN_IF_NOT_OK(cache_client_->CacheSchema(column_name_id_map())); } - std::unique_ptr db_ptr; - RETURN_IF_NOT_OK(cache_missing_stream->GetNextBuffer(&db_ptr, workerId)); - while (!db_ptr->eof()) { - if (db_ptr->eoe()) { + TensorRow new_row; + auto child_iterator = std::make_unique(this, workerId, kCacheMissChildIdx); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&new_row)); + while (!new_row.eof()) { + if (new_row.eoe()) { // Ignore it. MS_LOG(DEBUG) << "Ignore eoe"; // However we need to flush any left over from the async write buffer. But any error @@ -135,36 +126,32 @@ Status CacheMergeOp::CacheMissWorkerEntry(int32_t workerId) { } } } else { - while (db_ptr->NumRows() > 0) { - TensorRow row; - RETURN_IF_NOT_OK(db_ptr->PopRow(&row)); - row_id_type row_id = row.getId(); - if (row_id < 0) { - std::string errMsg = "Expect positive row id: " + std::to_string(row_id); - RETURN_STATUS_UNEXPECTED(errMsg); - } - if (cache_missing_rows_) { - // Technically number of this row shows up in the cache miss stream is equal to the number - // of P() call. However the cleaner wants it too. So we need an extra copy. - TensorRowCacheRequest *rq; - RETURN_IF_NOT_OK(GetRq(row_id, &rq)); - if (rq->GetState() == TensorRowCacheRequest::State::kEmpty) { - // We will send the request async. But any error we most - // likely ignore and continue. - Status rc; - rc = rq->AsyncSendCacheRequest(cache_client_, row); - if (rc.IsOk()) { - RETURN_IF_NOT_OK(io_que_->EmplaceBack(row_id)); - } else if (rc == StatusCode::kMDOutOfMemory || rc == kMDNoSpace) { - cache_missing_rows_ = false; - cache_client_->ServerRunningOutOfResources(); - } + row_id_type row_id = new_row.getId(); + if (row_id < 0) { + std::string errMsg = "Expect positive row id: " + std::to_string(row_id); + RETURN_STATUS_UNEXPECTED(errMsg); + } + if (cache_missing_rows_) { + // Technically number of this row shows up in the cache miss stream is equal to the number + // of P() call. However the cleaner wants it too. So we need an extra copy. + TensorRowCacheRequest *rq; + RETURN_IF_NOT_OK(GetRq(row_id, &rq)); + if (rq->GetState() == TensorRowCacheRequest::State::kEmpty) { + // We will send the request async. But any error we most + // likely ignore and continue. + Status rc; + rc = rq->AsyncSendCacheRequest(cache_client_, new_row); + if (rc.IsOk()) { + RETURN_IF_NOT_OK(io_que_->EmplaceBack(row_id)); + } else if (rc == StatusCode::kMDOutOfMemory || rc == kMDNoSpace) { + cache_missing_rows_ = false; + cache_client_->ServerRunningOutOfResources(); } } - RETURN_IF_NOT_OK(cache_miss_.Add(row_id, std::move(row))); } + RETURN_IF_NOT_OK(cache_miss_.Add(row_id, std::move(new_row))); } - RETURN_IF_NOT_OK(cache_missing_stream->GetNextBuffer(&db_ptr, workerId)); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&new_row)); } return Status::OK(); } @@ -265,14 +252,14 @@ Status CacheMergeOp::Builder::Build(std::shared_ptr *ptr) { Status CacheMergeOp::EoeReceived(int32_t worker_id) { // Send the eoe up. MS_LOG(DEBUG) << "Cache merge sending eoe"; - return DatasetOp::EoeReceived(worker_id); + return out_connector_->SendEOE(worker_id); } // Base-class override for handling cases when an eof is received. Status CacheMergeOp::EofReceived(int32_t worker_id) { // Send the eof up. MS_LOG(DEBUG) << "Cache merge sending eof"; - return DatasetOp::EofReceived(worker_id); + return out_connector_->SendEOF(worker_id); } Status CacheMergeOp::GetRq(row_id_type row_id, CacheMergeOp::TensorRowCacheRequest **out) { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_op.cc index 77db6adc1b..cccf77f343 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/cache_op.cc @@ -21,6 +21,7 @@ #include "minddata/dataset/include/constants.h" #include "minddata/dataset/core/global_context.h" #include "minddata/dataset/engine/datasetops/repeat_op.h" +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/data_buffer.h" #include "minddata/dataset/engine/execution_tree.h" #include "minddata/dataset/util/log_adapter.h" @@ -104,15 +105,16 @@ Status CacheOp::CacheAllRows(int32_t worker_id) { } MS_LOG(INFO) << "CacheOp first epoch SAVE mode started. Worker: " << worker_id; // SAVE mode loop - std::unique_ptr db_ptr; - RETURN_IF_NOT_OK(this->GetNextInput(&db_ptr, worker_id, 0)); - while (!db_ptr->eof()) { - if (!db_ptr->eoe()) { + TensorRow row; + auto child_iterator = std::make_unique(this, worker_id, 0); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&row)); + while (!row.eof()) { + if (!row.eoe()) { Status rc; // Do the Async write if we attach to the shared memory. - rc = cache_client_->AsyncWriteBuffer(std::move(db_ptr)); + rc = cache_client_->AsyncWriteRow(row); if (rc.StatusCode() == StatusCode::kMDNotImplementedYet) { - RETURN_IF_NOT_OK(cache_client_->WriteBuffer(std::move(db_ptr))); + RETURN_IF_NOT_OK(cache_client_->WriteRow(row)); } else if (rc.IsError()) { return rc; } @@ -122,12 +124,13 @@ Status CacheOp::CacheAllRows(int32_t worker_id) { // the eoe to indicate the end of the epoch, we should next expect to get the eof. // Drain this eof so that we don't leave it sitting there on a connector that we'll never fetch // from again. - RETURN_IF_NOT_OK(this->GetNextInput(&db_ptr, worker_id, 0)); - if (!db_ptr->eof()) { + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&row)); + if (!row.eof()) { RETURN_STATUS_UNEXPECTED("Cache op expects to get an eof after eoe from child."); } + break; } - RETURN_IF_NOT_OK(this->GetNextInput(&db_ptr, worker_id, 0)); + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&row)); } } // Let the main guy know we are done. diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.cc index abb808c91e..557d32a6f7 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.cc @@ -78,9 +78,12 @@ void ConcatOp::Print(std::ostream &out, bool show_all) const { // Main entry point for Concat Status ConcatOp::operator()() { - children_num_ = static_cast(child_.size()); TaskManager::FindMe()->Post(); - std::unique_ptr buf; + children_num_ = static_cast(child_.size()); + for (int32_t i = 0; i < children_num_; i++) { + children_iterators_.push_back(std::make_unique(this, 0, i)); + } + TensorRow new_row; int eof_count = 0; int sample_number = 0; bool is_not_mappable = true; @@ -95,26 +98,26 @@ Status ConcatOp::operator()() { while (eof_count == 0) { for (int i = 0; i < children_num_; i++) { - // 1. Read the first buffer - RETURN_IF_NOT_OK(child_[i]->GetNextBuffer(&buf)); - if (buf->eof()) { + // 1. Read the first row + RETURN_IF_NOT_OK(children_iterators_[i]->FetchNextTensorRow(&new_row)); + if (new_row.eof()) { eof_count++; continue; } // 2. Do verification as for column name, column data type and rank of column data - if (!buf->eoe()) { - RETURN_IF_NOT_OK(Verify(i, buf)); + if (!new_row.eoe()) { + RETURN_IF_NOT_OK(Verify(i, new_row)); } // 3. Put the data into output_connector if (!children_flag_and_nums_.empty()) { is_not_mappable = children_flag_and_nums_[i].first; is_not_mappable_or_second_ne_zero = is_not_mappable || (!children_flag_and_nums_[i].second); } - while (!buf->eoe() && !buf->eof()) { + while (!new_row.eoe() && !new_row.eof()) { // if dataset is not mappable or generator dataset which source is yield, cannot get the number of samples in // python layer), we use filtering to get data if (sample_number % num_shard == shard_index && is_not_mappable_or_second_ne_zero) { - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(buf))); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); } else if (!is_not_mappable_or_second_ne_zero) { // if dataset is mappable or generator dataset which source is not yield, // get the start and end subscripts of valid values @@ -122,7 +125,7 @@ Status ConcatOp::operator()() { // determine whether the data allocated to the current shard id is false data if (f(fv, sv, shard_index)) { - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(buf))); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); } } @@ -131,7 +134,7 @@ Status ConcatOp::operator()() { sample_number++; } - RETURN_IF_NOT_OK(child_[i]->GetNextBuffer(&buf)); + RETURN_IF_NOT_OK(children_iterators_[i]->FetchNextTensorRow(&new_row)); } // if dataset is mappable,We don't use filtering to pick data. @@ -143,8 +146,7 @@ Status ConcatOp::operator()() { // 4. Add eoe buffer after get buffer from all child if (eof_count == 0) { - auto eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); } UpdateRepeatAndEpochCounter(); } @@ -152,15 +154,11 @@ Status ConcatOp::operator()() { "Something went wrong, eof count does not match the number of children."); // 5. Add eof buffer in the end manually MS_LOG(DEBUG) << "Add the eof buffer manually in the end."; - auto eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eof_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } -Status ConcatOp::Verify(int32_t id, const std::unique_ptr &buf) { - TensorRow new_row; - RETURN_IF_NOT_OK(buf->GetRow(0, &new_row)); - +Status ConcatOp::Verify(int32_t id, const TensorRow &new_row) { if (id == 0) { // Obtain the data type and data rank in child[0] for (auto item : new_row) { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.h index 6222b32a53..2962a450ba 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/concat_op.h @@ -21,6 +21,7 @@ #include #include #include +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/pipeline_op.h" #include "minddata/dataset/engine/datasetops/source/sampler/distributed_sampler.h" @@ -111,7 +112,7 @@ class ConcatOp : public PipelineOp { Status GetNumClasses(int64_t *num_classes) override; private: - Status Verify(int32_t id, const std::unique_ptr &buf); + Status Verify(int32_t id, const TensorRow &tensor_row); int32_t children_num_; // The num of child of parent node. std::unordered_map column_name_id_; // Mapping between col index and col name @@ -120,6 +121,8 @@ class ConcatOp : public PipelineOp { std::shared_ptr sampler_; std::vector> children_flag_and_nums_; std::vector> children_start_end_index_; + + std::vector> children_iterators_; // Iterator for fetching. }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.cc index 1e0d591d83..9234ae8858 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.cc @@ -252,45 +252,15 @@ void DatasetOp::Print(std::ostream &out, bool show_all) const { } } -Status DatasetOp::GetNextRow(TensorRow *row) { +Status DatasetOp::GetNextRowPullMode(TensorRow *row) { RETURN_UNEXPECTED_IF_NULL(child_[0]); - return child_[0]->GetNextRow(row); + return child_[0]->GetNextRowPullMode(row); } // Gets the next buffer from the given child -Status DatasetOp::GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) { +Status DatasetOp::GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) { // pop is a blocked call and will throw an interruption if the whole group shuts down. - RETURN_IF_NOT_OK(out_connector_->PopWithRetry(static_cast(worker_id), p_buffer, retry_if_eoe)); - return Status::OK(); -} - -// Gets the next buffer from the given child . This function also has built-in eoe and eof -// message handling so that child classes don't have to manually code pass-through logic when -// those messages are received. -Status DatasetOp::GetNextInput(std::unique_ptr *p_buffer, int32_t worker_id, int32_t child_index) { - if (child_.size() == 0) { - return this->GetNextBuffer(p_buffer, worker_id); - } - CHECK_FAIL_RETURN_UNEXPECTED(child_index < child_.size(), - "Invalid data, child index too big : " + std::to_string(child_index)); - std::shared_ptr child = child_[child_index]; - std::unique_ptr buf; - RETURN_IF_NOT_OK(child->GetNextBuffer(&buf, worker_id)); - // Loop until non EOE is received - while (buf->eoe()) { - UpdateRepeatAndEpochCounter(); - RETURN_IF_NOT_OK(EoeReceived(worker_id)); - if (state_ == OpState::kDeOpIdle) { - *p_buffer = std::move(buf); - return Status::OK(); - } - RETURN_IF_NOT_OK(child->GetNextBuffer(&buf, worker_id)); - } - // Check if the last buf is next eof - if (buf->eof()) { - RETURN_IF_NOT_OK(EofReceived(worker_id)); - } - *p_buffer = std::move(buf); + RETURN_IF_NOT_OK(out_connector_->PopWithRetry(static_cast(worker_id), row, retry_if_eoe)); return Status::OK(); } @@ -328,18 +298,12 @@ Status DatasetOp::GetClassIndexing(std::vector eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - return (out_connector_->Add(static_cast(worker_id), std::move(eoe_buffer))); -} +Status DatasetOp::EoeReceived(int32_t worker_id) { return out_connector_->SendEOE(worker_id); } // Performs handling for when an eof message is received. // The base class implementation simply flows the eof message to output. Derived classes // may override if they need to perform special eof handling. -Status DatasetOp::EofReceived(int32_t worker_id) { - std::unique_ptr eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - return (out_connector_->Add(static_cast(worker_id), std::move(eof_buffer))); -} +Status DatasetOp::EofReceived(int32_t worker_id) { return out_connector_->SendEOF(worker_id); } // During tree prepare phase, operators may have specific post-operations to perform depending on their role. Status DatasetOp::PrepareOperator() { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.h index 2f3f44e628..26b5d9f0c9 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/dataset_op.h @@ -129,7 +129,7 @@ class DatasetOp : public std::enable_shared_from_this { /// \param show_all - A bool to control if you want to show all info or just a summary virtual void Print(std::ostream &out, bool show_all) const; - virtual Status GetNextRow(TensorRow *row); + virtual Status GetNextRowPullMode(TensorRow *row); /// \brief << Stream output operator overload /// \notes This allows you to write the debug print info using stream operators @@ -149,35 +149,17 @@ class DatasetOp : public std::enable_shared_from_this { virtual Status operator()() = 0; /// \brief Gets the next buffer from the given child - /// \notes See GetNextInput for similar function that has built-in message handling - /// \param p_buffer - The shared pointer for the fetched buffer to return (by reference) - /// \param worker_id - The worker id - /// \return Status The status code returned - virtual Status GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id) { - return GetNextBuffer(p_buffer, worker_id, false); - } - - /// \brief Gets the next buffer from the given child - /// \notes See GetNextInput for similar function that has built-in message handling - /// \param p_buffer - The shared pointer for the fetched buffer to return (by reference) + /// \param row[out] - Fetched TensorRow + /// \param worker_id[in] - The worker id, default to 0. /// \return Status The status code returned - virtual Status GetNextBuffer(std::unique_ptr *p_buffer) { return GetNextBuffer(p_buffer, 0, false); } + virtual Status GetNextRow(TensorRow *row, int32_t worker_id = 0) { return GetNextRow(row, worker_id, false); } /// \brief Gets the next buffer from the given child - /// \notes See GetNextInput for similar function that has built-in message handling - /// \param p_buffer - The shared pointer for the fetched buffer to return (by reference) - /// \param worker_id - The worker id + /// \param row[out] - Fetched TensorRow + /// \param worker_id[in] - The worker id, default to 0. /// \param retry_if_eoe Set this flag to true to allow calling pop() again after the first pop() returns EOE. /// \return Status The status code returned - virtual Status GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe); - - /// \brief Gets the next buffer from the given child . This function also has built-in eoe and eof - /// message handling so that child classes don't have to manually code pass-through logic when - /// those messages are received. - /// \param p_buffer - The shared pointer for the fetched buffer to return (by reference) - /// \param worker_id - The worker id - /// \return Status The status code returned - Status GetNextInput(std::unique_ptr *p_buffer, int32_t worker_id = 0, int32_t child_index = 0); + virtual Status GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe); /// \brief Gets the batch size /// \return Status - The status code return diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.cc index 644b791803..8095c55525 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.cc @@ -93,21 +93,18 @@ Status DeviceQueueOp::EoeReceived(int32_t worker_id) { return Status::OK(); } -Status DeviceQueueOp::CheckExceptions(const std::unique_ptr &buffer) const { - // this method checks if the buffer meets the conditions to be sent to TDT - if (buffer->NumRows() != 0) { - TensorRow row; - buffer->GetRow(0, &row); - for (const auto &item : row) { - CHECK_FAIL_RETURN_UNEXPECTED(item->type().IsNumeric(), "Invalid data, cannot send string tensor to device."); - CHECK_FAIL_RETURN_UNEXPECTED(item->HasData(), "Invalid data, cannot send tensor with no data to device."); - } +Status DeviceQueueOp::CheckExceptions(const TensorRow &row) const { + // this method checks if the row meets the conditions to be sent to TDT + for (const auto &item : row) { + CHECK_FAIL_RETURN_UNEXPECTED(item->type().IsNumeric(), "Invalid data, cannot send string tensor to device."); + CHECK_FAIL_RETURN_UNEXPECTED(item->HasData(), "Invalid data, cannot send tensor with no data to device."); } return Status::OK(); } Status DeviceQueueOp::operator()() { TaskManager::FindMe()->Post(); + child_iterator_ = std::make_unique(this, 0, 0); #ifdef ENABLE_DUMP_IR if (md_channel_info_ == nullptr) { @@ -163,43 +160,39 @@ Status DeviceQueueOp::SendDataToAscend() { md_channel_info_->RecordBatchQueue(ChildOpConnectorSize()); md_channel_info_->RecordPreprocessBatch(0); #endif - std::unique_ptr current_buffer; - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); - - while (!current_buffer->eof() && !is_break_loop) { - while (!current_buffer->eoe() && !is_break_loop) { - RETURN_IF_NOT_OK(CheckExceptions(current_buffer)); - TensorRow currRow; - for (int row_id = 0; row_id < current_buffer->NumRows(); row_id++) { - RETURN_IF_NOT_OK(current_buffer->GetRow(row_id, &currRow)); - WaitContinueSignal(); + TensorRow curr_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&curr_row)); + while (!curr_row.eof() && !is_break_loop) { + while (!curr_row.eoe() && !is_break_loop) { + RETURN_IF_NOT_OK(CheckExceptions(curr_row)); + WaitContinueSignal(); #ifdef ENABLE_DUMP_IR - md_channel_info_->RecordBatchQueue(ChildOpConnectorSize()); - md_channel_info_->RecordPreprocessBatch(send_batch); - md_channel_info_->RecordPushStartTime(); + md_channel_info_->RecordBatchQueue(ChildOpConnectorSize()); + md_channel_info_->RecordPreprocessBatch(send_batch); + md_channel_info_->RecordPushStartTime(); #endif - RETURN_IF_NOT_OK(SendRowToTdt(currRow, isProfilingEnable, &tdt_cost)); - ProfilingRecorder(isProfilingEnable, profiling_node, send_batch, tdt_cost, &batch_start_time, &end_time, - connector_capacity, connector_size); - send_batch++; + RETURN_IF_NOT_OK(SendRowToTdt(curr_row, isProfilingEnable, &tdt_cost)); + ProfilingRecorder(isProfilingEnable, profiling_node, send_batch, tdt_cost, &batch_start_time, &end_time, + connector_capacity, connector_size); + send_batch++; #ifdef ENABLE_DUMP_IR - md_channel_info_->RecordBatchQueue(ChildOpConnectorSize()); - md_channel_info_->RecordPreprocessBatch(send_batch); - md_channel_info_->RecordPushEndTime(); + md_channel_info_->RecordBatchQueue(ChildOpConnectorSize()); + md_channel_info_->RecordPreprocessBatch(send_batch); + md_channel_info_->RecordPushEndTime(); #endif - if (total_batch_ > 0 && send_batch >= total_batch_) { - is_break_loop = true; - break; - } + if (total_batch_ > 0 && send_batch >= total_batch_) { + is_break_loop = true; + break; } + if (isProfilingEnable) { connector_size = ChildOpConnectorSize(); connector_capacity = ChildOpConnectorCapacity(); } - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&curr_row)); } - if (current_buffer->eoe() && send_epoch_end_) { + if (curr_row.eoe() && send_epoch_end_) { TensorRow currRow; auto status = tdtInstancePtr->hostPush(currRow, true, channel_name_, isProfilingEnable, tdt_cost, ACL_TENSOR_DATA_END_OF_SEQUENCE); @@ -219,7 +212,7 @@ Status DeviceQueueOp::SendDataToAscend() { connector_capacity = ChildOpConnectorCapacity(); tree_->SetEpochEnd(); } - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&curr_row)); } // now we use this flag to judge whether exception raised. @@ -444,27 +437,23 @@ Status DeviceQueueOp::WorkerEntry(int32_t worker_id) { // Every thread use cuda api should SetThreadDevice RETURN_IF_NOT_OK(SetThreadDevice()); TaskManager::FindMe()->Post(); - std::unique_ptr current_buffer; + TensorRow current_row; uint32_t batch_num = 0; - RETURN_IF_NOT_OK(receive_queues_[worker_id]->PopFront(¤t_buffer)); - while (!current_buffer->quit() && !GpuBufferMgr::GetInstance().IsClosed()) { - TensorRow curr_row; - for (int row_id = 0; row_id < current_buffer->NumRows() && !GpuBufferMgr::GetInstance().IsClosed(); row_id++) { - RETURN_IF_NOT_OK(current_buffer->GetRow(row_id, &curr_row)); - std::vector items; - for (int i = 0; i < curr_row.size(); i++) { - device::DataItemGpu data_item; - data_item.data_len_ = static_cast(curr_row[i]->SizeInBytes()); - data_item.data_ptr_ = nullptr; - data_item.worker_id_ = worker_id; - items.push_back(data_item); - } - RETURN_IF_NOT_OK(MallocForGPUData(&items, curr_row, worker_id)); - RETURN_IF_NOT_OK(gpu_item_connector_->Add(worker_id, std::move(items))); - batch_num++; + RETURN_IF_NOT_OK(receive_queues_[worker_id]->PopFront(¤t_row)); + while (!current_row.quit() && !GpuBufferMgr::GetInstance().IsClosed()) { + std::vector items; + for (int i = 0; i < current_row.size(); i++) { + device::DataItemGpu data_item; + data_item.data_len_ = static_cast(current_row[i]->SizeInBytes()); + data_item.data_ptr_ = nullptr; + data_item.worker_id_ = worker_id; + items.push_back(data_item); } + RETURN_IF_NOT_OK(MallocForGPUData(&items, current_row, worker_id)); + RETURN_IF_NOT_OK(gpu_item_connector_->Add(worker_id, std::move(items))); + batch_num++; - RETURN_IF_NOT_OK(receive_queues_[worker_id]->PopFront(¤t_buffer)); + RETURN_IF_NOT_OK(receive_queues_[worker_id]->PopFront(¤t_row)); } MS_LOG(INFO) << "Device queue worker id " << worker_id << "proc " << batch_num << "batch."; @@ -477,31 +466,31 @@ Status DeviceQueueOp::WorkerEntry(int32_t worker_id) { Status DeviceQueueOp::SendDataToGPU() { RETURN_IF_NOT_OK(LaunchParallelCopyThread()); MS_LOG(INFO) << "Device queue, sending data to GPU."; - std::unique_ptr current_buffer; - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); + TensorRow current_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(¤t_row)); int64_t num_buf = 0; bool is_break_loop = false; - while (!current_buffer->eof() && !is_break_loop && !GpuBufferMgr::GetInstance().IsClosed()) { - while (!current_buffer->eoe() && !is_break_loop && !GpuBufferMgr::GetInstance().IsClosed()) { - RETURN_IF_NOT_OK(CheckExceptions(current_buffer)); - RETURN_IF_NOT_OK(receive_queues_[num_buf++ % num_workers_]->Add(std::move(current_buffer))); + while (!current_row.eof() && !is_break_loop && !GpuBufferMgr::GetInstance().IsClosed()) { + while (!current_row.eoe() && !is_break_loop && !GpuBufferMgr::GetInstance().IsClosed()) { + RETURN_IF_NOT_OK(CheckExceptions(current_row)); + RETURN_IF_NOT_OK(receive_queues_[num_buf++ % num_workers_]->Add(std::move(current_row))); if (!TaskManager::FindMe()->Interrupted() && !GpuBufferMgr::GetInstance().IsClosed()) { - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(¤t_row)); } else { is_break_loop = true; } } if (!TaskManager::FindMe()->Interrupted() && !GpuBufferMgr::GetInstance().IsClosed()) { - RETURN_IF_NOT_OK(GetNextInput(¤t_buffer)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(¤t_row)); } else { is_break_loop = true; } } for (uint32_t index = 0; index < num_workers_; index++) { - auto quit = std::make_unique(0, DataBuffer::kDeBFlagQuit); - RETURN_IF_NOT_OK(receive_queues_[num_buf++ % num_workers_]->Add(std::move(quit))); + TensorRow quit_flag(TensorRow::kFlagQuit); + RETURN_IF_NOT_OK(receive_queues_[num_buf++ % num_workers_]->Add(std::move(quit_flag))); } MS_LOG(INFO) << "Device queue receive " << num_buf - num_workers_ << " batch."; @@ -537,10 +526,9 @@ Status DeviceQueueOp::SendDataToCPU() { MS_LOG(INFO) << "Device queue, sending data to CPU."; int64_t total_batch = 0; - std::unique_ptr child_iterator = std::make_unique(this, 0, 0); - while (!(child_iterator->eof_handled())) { + while (!(child_iterator_->eof_handled())) { TensorRow curr_row; - RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&curr_row)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&curr_row)); if (!curr_row.empty()) { for (auto &tensor : curr_row) { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.h index 3135fc57df..91ef088b0b 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/device_queue_op.h @@ -23,6 +23,8 @@ #include "minddata/dataset/engine/datasetops/pipeline_op.h" #include "minddata/dataset/engine/datasetops/repeat_op.h" +#include "minddata/dataset/engine/dataset_iterator.h" + #include "minddata/dataset/engine/perf/device_queue_tracing.h" #include "minddata/dataset/util/status.h" #ifdef ENABLE_DUMP_IR @@ -182,9 +184,9 @@ class DeviceQueueOp : public PipelineOp { std::string Name() const override { return kDeviceQueueOp; } private: - // Name: checkExceptions(DataBuffer); - // Description: Check whether the dataBuffer meets the condition for performing DeviceQueueOp - Status CheckExceptions(const std::unique_ptr &buffer) const; + // Name: checkExceptions(TensorRow); + // Description: Check whether the TensorRow meets the condition for performing DeviceQueueOp + Status CheckExceptions(const TensorRow &row) const; private: #ifdef ENABLE_TDTQUE @@ -204,7 +206,7 @@ class DeviceQueueOp : public PipelineOp { Status WorkerEntry(int32_t worker_id); Status SetThreadDevice(); - QueueList> receive_queues_; + QueueList receive_queues_; std::vector> pool_; std::unique_ptr gpu_item_connector_; uint32_t num_workers_; @@ -216,6 +218,8 @@ class DeviceQueueOp : public PipelineOp { #endif Status SendDataToCPU(); + + std::unique_ptr child_iterator_; std::string channel_name_; DeviceType device_type_; const int32_t device_id_; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.cc index 273ea60633..bf9652c43c 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.cc @@ -61,24 +61,21 @@ void EpochCtrlOp::Print(std::ostream &out, bool show_all) const { } } -Status EpochCtrlOp::GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) { +Status EpochCtrlOp::GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) { if (child_.empty()) { RETURN_STATUS_UNEXPECTED("EpochCtrlOp can't be the leaf node."); } - std::unique_ptr buf; - // `retry_if_eoe` is false because EpochCtrlOp does not eat EOE. - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf, worker_id, false)); + RETURN_IF_NOT_OK(child_[0]->GetNextRow(row, worker_id, false)); // Only intercept EOE for EoeReceived processing, after that the EOE is forwarded to next op. // Other databuffers containing data or EOF will simply be forwarded. // EOF can simply be forwarded because this op does not spawn any thread, thus does not require clean up. - if (buf->eoe()) { + if (row->eoe()) { RETURN_IF_NOT_OK(EoeReceived(worker_id)); } - *p_buffer = std::move(buf); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.h index f2ef8f6697..5c6b58f250 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/epoch_ctrl_op.h @@ -59,7 +59,7 @@ class EpochCtrlOp : public RepeatOp { // Since EpochCtrlOp is derived from RepeatOp which is an inlined op, getting a buffer from us // will simply bounce you to get a buffer from our child. // Epoch Control Op does not eat the EOE, it will pass the EOE to the next op. - Status GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) override; + Status GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) override; // Base-class override for handling cases when an eoe is received. // @param worker_id - The worker id diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.cc index d9c9d2acf6..455ab7c3b1 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.cc @@ -23,7 +23,6 @@ #include "minddata/dataset/core/global_context.h" #include "minddata/dataset/core/tensor.h" #include "minddata/dataset/engine/data_buffer.h" -#include "minddata/dataset/engine/execution_tree.h" #include "minddata/dataset/kernels/tensor_op.h" #include "minddata/dataset/util/log_adapter.h" #include "minddata/dataset/util/task_manager.h" @@ -80,8 +79,8 @@ Status FilterOp::EofReceived(int32_t) { return Status::OK(); } Status FilterOp::EoeReceived(int32_t) { return Status::OK(); } // Validating if each of the input_columns exists in the DataBuffer. -Status FilterOp::ValidateInColumns(const std::vector *input_columns) { - for (const auto &inCol : *input_columns) { +Status FilterOp::ValidateInColumns(const std::vector &input_columns) { + for (const auto &inCol : input_columns) { bool found = column_name_id_map_.find(inCol) != column_name_id_map_.end() ? true : false; if (!found) { std::string err_msg = "Invalid parameter, column name: " + inCol + " does not exist in the dataset columns."; @@ -111,68 +110,51 @@ void FilterOp::Print(std::ostream &out, bool show_all) const { } Status FilterOp::WorkerEntry(int32_t worker_id) { + std::unique_ptr child_iterator = std::make_unique(this, worker_id, 0); + // Handshake with TaskManager that thread creation is successful. TaskManager::FindMe()->Post(); - std::unique_ptr in_buffer; bool worker_stop = false; while (worker_stop == false) { - // Getting a databuffer to work on. - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&in_buffer, worker_id)); - if (in_buffer->eoe()) { - filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_buffer), filterCtrl::kFilterEoe)); + // Getting a TensorRow to work on. + TensorRow in_row; + RETURN_IF_NOT_OK(child_iterator->FetchNextTensorRow(&in_row)); + + if (in_row.eoe()) { + RETURN_IF_NOT_OK(filter_queues_[worker_id]->EmplaceBack(std::make_pair(in_row, filterCtrl::kFilterEoe))); continue; - } else if (in_buffer->eof()) { - filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_buffer), filterCtrl::kFilterEof)); + } else if (in_row.eof()) { + RETURN_IF_NOT_OK(filter_queues_[worker_id]->EmplaceBack(std::make_pair(in_row, filterCtrl::kFilterEof))); worker_stop = true; continue; } - RETURN_IF_NOT_OK(CheckColumns(in_buffer.get(), &in_columns_)); + RETURN_IF_NOT_OK(ValidateInColumns(in_columns_)); - // if the databuffer was all filtered, it is marked as kFilterEmpty. - // if the databuffer was partially filtered, it is marked as kFilterPartial. - // if the databuffer was not filtered, it is marked as kFilterFull. - int32_t num_rows = in_buffer->NumRows(); - std::unique_ptr new_tensor_table; - RETURN_IF_NOT_OK(WorkerCompute(in_buffer.get(), &new_tensor_table)); + bool result; + RETURN_IF_NOT_OK(WorkerCompute(in_row, &result)); - if (new_tensor_table->empty()) { - RETURN_IF_NOT_OK( - filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_buffer), filterCtrl::kFilterEmpty))); - } else if (new_tensor_table->size() == num_rows) { - in_buffer->set_tensor_table(std::move(new_tensor_table)); + if (result) RETURN_IF_NOT_OK( - filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_buffer), filterCtrl::kFilterFull))); - } else { // kFilterPartial - in_buffer->set_tensor_table(std::move(new_tensor_table)); + filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_row), filterCtrl::kFilterFull))); + else RETURN_IF_NOT_OK( - filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_buffer), filterCtrl::kFilterPartial))); - } + filter_queues_[worker_id]->EmplaceBack(std::make_pair(std::move(in_row), filterCtrl::kFilterEmpty))); } return Status::OK(); } -Status FilterOp::WorkerCompute(DataBuffer *in_buffer, std::unique_ptr *out) { - *out = std::make_unique(); - int32_t num_rows = in_buffer->NumRows(); - for (int32_t i = 0; i < num_rows; i++) { - TensorRow to_process; - TensorRow cur_row; - RETURN_IF_NOT_OK(in_buffer->PopRow(&cur_row)); - if (in_columns_.empty() == true) { - MS_LOG(INFO) << "Input columns in filter operator is empty, will apply to the all column in the current table."; - to_process = cur_row; - } else { - (void)std::transform( - in_columns_.begin(), in_columns_.end(), std::back_inserter(to_process), - [&cur_row, this](const auto &it) -> std::shared_ptr { return cur_row[column_name_id_map_[it]]; }); - } - bool predicate = true; - RETURN_IF_NOT_OK(InvokePredicateFunc(to_process, &predicate)); - if (predicate) { - (*out)->push_back(std::move(cur_row)); - } +Status FilterOp::WorkerCompute(const TensorRow &in_row, bool *out_predicate) { + TensorRow to_process; + if (in_columns_.empty() == true) { + MS_LOG(INFO) << "Input columns in filter operator is empty, will apply to the all column in the current table."; + to_process = in_row; + } else { + (void)std::transform( + in_columns_.begin(), in_columns_.end(), std::back_inserter(to_process), + [&in_row, this](const auto &it) -> std::shared_ptr { return in_row[column_name_id_map_[it]]; }); } + RETURN_IF_NOT_OK(InvokePredicateFunc(to_process, out_predicate)); return Status::OK(); } @@ -190,20 +172,24 @@ Status FilterOp::Collector() { bool collector_stop = false; uint64_t task_id_cnt = 0; uint64_t out_id_cnt = 0; - std::pair, filterCtrl> in_pair; + std::pair in_pair; while (collector_stop == false) { uint32_t w_id = task_id_cnt % num_workers_; RETURN_IF_NOT_OK(filter_queues_[w_id]->PopFront(&in_pair)); if (in_pair.second == filterCtrl::kFilterFull || in_pair.second == filterCtrl::kFilterPartial || in_pair.second == filterCtrl::kFilterEoe) { - if (in_pair.second == filterCtrl::kFilterEoe) UpdateRepeatAndEpochCounter(); uint32_t out_task_id = out_id_cnt % num_workers_; - RETURN_IF_NOT_OK(out_connector_->Add(static_cast(out_task_id), std::move(in_pair.first))); + if (in_pair.second == filterCtrl::kFilterEoe) { + UpdateRepeatAndEpochCounter(); + RETURN_IF_NOT_OK(out_connector_->SendEOE(static_cast(out_task_id))); + } else { + RETURN_IF_NOT_OK(out_connector_->Add(std::move(in_pair.first), static_cast(out_task_id))); + } out_id_cnt++; task_id_cnt++; } else if (in_pair.second == filterCtrl::kFilterEof) { uint32_t out_task_id = out_id_cnt % num_workers_; - RETURN_IF_NOT_OK(out_connector_->Add(static_cast(out_task_id), std::move(in_pair.first))); + RETURN_IF_NOT_OK(out_connector_->SendEOF(static_cast(out_task_id))); collector_stop = true; } else { // kFilterEmpty task_id_cnt++; @@ -212,18 +198,6 @@ Status FilterOp::Collector() { return Status::OK(); } -// Private function for checking the column legality. -Status FilterOp::CheckColumns(const DataBuffer *in_buf, const std::vector *input_columns) { - int32_t num_rows = in_buf->NumRows(); - int32_t num_cols = in_buf->NumCols(); - if (num_rows == 0 || num_cols == 0) { - RETURN_STATUS_UNEXPECTED("FilterOp is getting an empty DataBuffer."); - } - // Check if there is invalid column name in the inColumns. - RETURN_IF_NOT_OK(ValidateInColumns(input_columns)); - return Status::OK(); -} - Status FilterOp::CheckInput(const TensorRow &input) const { for (auto &item : input) { if (item == nullptr) { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.h index ab09ac1229..ca746c28c3 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/filter_op.h @@ -21,6 +21,7 @@ #include #include #include +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/parallel_op.h" #include "minddata/dataset/kernels/tensor_op.h" #include "minddata/dataset/util/queue.h" @@ -133,7 +134,7 @@ class FilterOp : public ParallelOp { std::vector in_columns_; // Internal queue for filter. - QueueList, filterCtrl>> filter_queues_; + QueueList> filter_queues_; // Private function for worker/thread to loop continuously. It comprises the main // logic of FilterOp, getting the data from previous Op, validating user specified column names, @@ -143,11 +144,10 @@ class FilterOp : public ParallelOp { Status WorkerEntry(int32_t worker_id) override; // In: workerId assigned by tree_ // Filter the data by predicate function . - // @param in_buffer input data buffer. - // @param to_proess_indices Indices of columns to be processed. - // @param out data buffer that are filtered by predicate. + // @param in_row input row. + // @param out_predicate result boolean to filter or not. // @return Status The status code returned - Status WorkerCompute(DataBuffer *in_buffer, std::unique_ptr *out); + Status WorkerCompute(const TensorRow &in_row, bool *out_predicate); // Collector databuffer. // @return Status The status code returned @@ -167,14 +167,7 @@ class FilterOp : public ParallelOp { // exist in the DataBuffer. // @param input_columns The vector of input column names used in the current thread. // @return Status The status code returned - Status ValidateInColumns(const std::vector *input_columns); - - // Private function for checking the column legality - // @param in_buf A raw pointer to the DataBuffer. A raw pointer is fine because this function does not manage memory - // and is not shared with other threads. - // @param[out] to_process_indices Indices of columns that will feed to predicate. - // @param input_columns The vector of input column names used in the current thread. - Status CheckColumns(const DataBuffer *in_buf, const std::vector *input_columns); + Status ValidateInColumns(const std::vector &input_columns); }; } // namespace dataset diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.cc index 7aa8c68405..d49f5344a0 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.cc @@ -101,13 +101,12 @@ void MapOp::Print(std::ostream &out, bool show_all) const { } // A helper function that fetch worker map job from local queues and extract the data and map job list -Status MapOp::FetchNextWork(uint32_t worker_id, std::unique_ptr *db, - std::vector> *job_list) { +Status MapOp::FetchNextWork(uint32_t worker_id, TensorRow *row, std::vector> *job_list) { std::unique_ptr worker_job; // Fetch the next worker job and data buffer RETURN_IF_NOT_OK(local_queues_[worker_id]->PopFront(&worker_job)); // Extract the databuffer and job list from the map worker job. - *db = std::move(worker_job->databuffer); + *row = std::move(worker_job->tensor_row); *job_list = std::move(worker_job->jobs); return Status::OK(); @@ -166,21 +165,22 @@ Status MapOp::operator()() { RETURN_IF_NOT_OK(callback_manager_.Begin(CallbackParam(0, ep_step, total_step))); - std::unique_ptr buff; + child_iterator_ = std::make_unique(this, 0, 0); + TensorRow new_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buff, 0)); - while (!buff->eof()) { + while (!new_row.eof()) { if (op_current_repeats_ % op_num_repeats_per_epoch() == 0) { RETURN_IF_NOT_OK(callback_manager_.EpochBegin(CallbackParam(op_current_epochs_ + 1, ep_step, total_step))); } - while (!buff->eoe()) { + while (!new_row.eoe()) { ep_step++; total_step++; // Create an empty map worker job to be populated by a databuffer and map jobs RETURN_IF_NOT_OK(callback_manager_.StepBegin(CallbackParam(op_current_epochs_ + 1, ep_step, total_step))); - std::unique_ptr worker_job = std::make_unique(std::move(buff)); + std::unique_ptr worker_job = std::make_unique(std::move(new_row)); // Populate map worker job for a worker to execute RETURN_IF_NOT_OK(GenerateWorkerJob(&worker_job)); @@ -190,7 +190,7 @@ Status MapOp::operator()() { RETURN_IF_NOT_OK(callback_manager_.StepEnd(CallbackParam(op_current_epochs_ + 1, ep_step, total_step))); - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buff, 0)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } // check whether this is the end of a real epoch (not all eoe signals end of epoch) @@ -200,19 +200,20 @@ Status MapOp::operator()() { ep_step = 0; } // Propagate the eoe buffer to worker - std::unique_ptr worker_job = std::make_unique(std::move(buff)); + std::unique_ptr worker_job = std::make_unique(std::move(new_row)); RETURN_IF_NOT_OK(local_queues_[num_buf++ % num_workers_]->Add(std::move(worker_job))); UpdateRepeatAndEpochCounter(); - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buff, 0)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } // End() is commented out because it might never be called due to the lack of EOF when EpochCtrl is -1 // Handle eof logic, this code might never be reached if epoch_ctrl = -1. - std::unique_ptr worker_job = std::make_unique(std::move(buff)); + std::unique_ptr worker_job = std::make_unique(std::move(new_row)); RETURN_IF_NOT_OK(local_queues_[num_buf++ % num_workers_]->Add(std::move(worker_job))); // Quit all workers, this code might never be reached if EpochCtrl is -1. for (int32_t wkr_id = 0; wkr_id < num_workers_; wkr_id++) { - auto quit = std::make_unique(std::make_unique(0, DataBuffer::kDeBFlagQuit)); + TensorRow quit_flag(TensorRow::kFlagQuit); + auto quit = std::make_unique(quit_flag); RETURN_IF_NOT_OK(local_queues_[num_buf++ % num_workers_]->Add(std::move(quit))); } @@ -227,78 +228,73 @@ Status MapOp::WorkerEntry(int32_t worker_id) { // Handshake with TaskManager that thread creation is successful. TaskManager::FindMe()->Post(); - std::unique_ptr in_buffer; + TensorRow in_row; std::vector> job_list; // Fetch next data buffer and map job list - RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_buffer, &job_list)); + RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_row, &job_list)); // Now that init work is done, drop into the main fetching loop. // Map op does not use child iterator, and it needs to manually handle eoe and eof's itself // rather than use the base-class defaults. while (true) { // Handle special logic where buffer carries a ctrl flag. - if (in_buffer->buffer_flags() != DataBuffer::kDeBFlagNone) { - if (in_buffer->wait()) { + if (in_row.Flags() != TensorRow::kFlagNone) { + if (in_row.wait()) { // When worker receives the signal from master thread, it increments a atomic int // The last guy who increments the counter, wakes up master thread if (++num_workers_paused_ == num_workers_) { wait_for_workers_post_.Set(); } // This will block the worker until master thread gives it a new work - } else if (in_buffer->eoe()) { + } else if (in_row.eoe()) { // Calling base class EoeReceived to forward eoe buffer. - RETURN_IF_NOT_OK(EoeReceived(worker_id)); - } else if (in_buffer->eof()) { + RETURN_IF_NOT_OK(out_connector_->SendEOE(worker_id)); + } else if (in_row.eof()) { // Calling base class EofReceived to forward eof buffer. - RETURN_IF_NOT_OK(EofReceived(worker_id)); - } else if (in_buffer->quit()) { + RETURN_IF_NOT_OK(out_connector_->SendEOF(worker_id)); + } else if (in_row.quit()) { break; } - RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_buffer, &job_list)); + RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_row, &job_list)); continue; } - CHECK_FAIL_RETURN_UNEXPECTED(in_buffer->NumRows() * in_buffer->NumCols() != 0, "MapOp got an empty DataBuffer."); - std::unique_ptr new_tensor_table(std::make_unique()); + CHECK_FAIL_RETURN_UNEXPECTED(in_row.size() != 0, "MapOp got an empty TensorRow."); + TensorRow out_row; // Perform the compute function of TensorOp(s) and store the result in new_tensor_table. - RETURN_IF_NOT_OK(WorkerCompute(in_buffer.get(), new_tensor_table.get(), job_list)); + RETURN_IF_NOT_OK(WorkerCompute(in_row, &out_row, job_list)); // Replace the TensorTable in DataBuffer with the new one. - in_buffer->set_tensor_table(std::move(new_tensor_table)); // Push the buffer onto the connector for next operator to consume. - RETURN_IF_NOT_OK(out_connector_->Add(static_cast(worker_id), std::move(in_buffer))); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(out_row), static_cast(worker_id))); // Fetch next data buffer and map job list - RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_buffer, &job_list)); + RETURN_IF_NOT_OK(FetchNextWork(worker_id, &in_row, &job_list)); } return Status::OK(); } -Status MapOp::WorkerCompute(DataBuffer *in_buffer, TensorQTable *new_tensor_table, +Status MapOp::WorkerCompute(const TensorRow &in_row, TensorRow *out_row, const std::vector> &job_list) { - int32_t num_rows = in_buffer->NumRows(); - int32_t num_cols = in_buffer->NumCols(); + int32_t num_cols = in_row.size(); std::vector job_input_table; std::vector original_table; - - // Prepare the data that we need from in_buffer - for (int32_t r = 0; r < num_rows; r++) { - // to_process : A vector of Tensors only holding cols in input_columns. - // cur_row : A vector of Tensors holding all the cols from DataBuffer. - TensorRow to_process, cur_row; - RETURN_IF_NOT_OK(in_buffer->PopRow(&cur_row)); - // From the current row, select the Tensor that need to be passed to TensorOp - (void)std::transform(to_process_indices_.begin(), to_process_indices_.end(), std::back_inserter(to_process), - [&cur_row](const auto &it) { return std::move(cur_row[it]); }); - to_process.setId(cur_row.getId()); - std::vector cur_row_path = cur_row.getPath(); - if (cur_row_path.size() > 0) { - std::vector to_process_path; - (void)std::transform(to_process_indices_.begin(), to_process_indices_.end(), std::back_inserter(to_process_path), - [&cur_row_path](const auto &it) { return cur_row_path[it]; }); - to_process.setPath(to_process_path); - } - job_input_table.push_back(std::move(to_process)); - original_table.push_back(std::move(cur_row)); + TensorRow to_process; + // Prepare the data that we need from in_row + // to_process : A vector of Tensors only holding cols in input_columns. + // cur_row : A vector of Tensors holding all the cols from DataBuffer. + + // From the current row, select the Tensor that need to be passed to TensorOp + (void)std::transform(to_process_indices_.begin(), to_process_indices_.end(), std::back_inserter(to_process), + [&in_row](const auto &it) { return std::move(in_row[it]); }); + to_process.setId(in_row.getId()); + std::vector cur_row_path = in_row.getPath(); + if (cur_row_path.size() > 0) { + std::vector to_process_path; + (void)std::transform(to_process_indices_.begin(), to_process_indices_.end(), std::back_inserter(to_process_path), + [&cur_row_path](const auto &it) { return cur_row_path[it]; }); + to_process.setPath(to_process_path); } + job_input_table.push_back(std::move(to_process)); + original_table.push_back(std::move(in_row)); // Variable to keep the result after executing the job. std::vector result_table; @@ -319,26 +315,22 @@ Status MapOp::WorkerCompute(DataBuffer *in_buffer, TensorQTable *new_tensor_tabl } // Merging the data processed by job (result_table) with the data that are not used. - for (int32_t r = 0; r < num_rows; r++) { - TensorRow out_row; - if (in_columns_.size() == out_columns_.size()) { - // Place the processed tensor back into the original index of the input tensor - for (size_t i = 0; i < result_table[r].size(); i++) { - original_table[r][to_process_indices_[i]] = std::move(result_table[r][i]); - } - out_row = std::move(original_table[r]); - } else { - // Append the data in the original table that we did not use to the end of each row in result_table. - for (int32_t i = 0; i < num_cols; i++) { - if (keep_input_columns_[i]) { - result_table[r].push_back(std::move(original_table[r][i])); - } + if (in_columns_.size() == out_columns_.size()) { + // Place the processed tensor back into the original index of the input tensor + for (size_t i = 0; i < result_table[0].size(); i++) { + original_table[0][to_process_indices_[i]] = std::move(result_table[0][i]); + } + *out_row = std::move(original_table[0]); + } else { + // Append the data in the original table that we did not use to the end of each row in result_table. + for (int32_t i = 0; i < num_cols; i++) { + if (keep_input_columns_[i]) { + result_table[0].push_back(std::move(original_table[0][i])); } - out_row = std::move(result_table[r]); } - // Add this final out_row to our new TensorTable. - new_tensor_table->push_back(std::move(out_row)); + *out_row = std::move(result_table[0]); } + return Status::OK(); } @@ -451,8 +443,8 @@ Status MapOp::WaitForWorkers() { num_workers_paused_ = 0; for (int32_t wkr_id = 0; wkr_id < num_workers_; wkr_id++) { // a special buffer (id=-1, empty, none flag) is used to signal that worker needs to pause. - RETURN_IF_NOT_OK(local_queues_[wkr_id]->Add( - std::make_unique(std::make_unique(0, DataBuffer::kDeBFlagWait)))); + TensorRow waitRow(TensorRow::kFlagWait); + RETURN_IF_NOT_OK(local_queues_[wkr_id]->Add(std::make_unique(waitRow))); } // wait until all workers are done processing their work in local_queue_ RETURN_IF_NOT_OK(wait_for_workers_post_.Wait()); diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.h index 1eca689591..2ed9fdf972 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/map_op/map_op.h @@ -24,6 +24,7 @@ #include #include "minddata/dataset/callback/ds_callback.h" +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/map_op/map_job.h" #include "minddata/dataset/engine/datasetops/parallel_op.h" #include "minddata/dataset/kernels/tensor_op.h" @@ -192,17 +193,16 @@ class MapOp : public ParallelOp { // A unit of job for map worker thread. // MapWorkerJob holds a list of MapJob where each MapJob can be a CpuMapJob, GpuMapJob or DvppMapJob. struct MapWorkerJob { - explicit MapWorkerJob(std::unique_ptr db) : databuffer(std::move(db)) {} + explicit MapWorkerJob(TensorRow tr) : tensor_row(std::move(tr)) {} std::vector> jobs; - std::unique_ptr databuffer; + TensorRow tensor_row; }; // A helper function to create jobs for workers. Status GenerateWorkerJob(const std::unique_ptr *worker_job); // A helper function that fetch worker map job from local queues and extract the data and map job list - Status FetchNextWork(uint32_t worker_id, std::unique_ptr *db, - std::vector> *job_list); + Status FetchNextWork(uint32_t worker_id, TensorRow *row, std::vector> *job_list); // Local queues where worker threads get a job from QueueList> local_queues_; @@ -222,6 +222,8 @@ class MapOp : public ParallelOp { // Indices of the columns to process. std::vector to_process_indices_; + std::unique_ptr child_iterator_; // An iterator for fetching. + // Private function for worker/thread to loop continuously. It comprises the main // logic of MapOp: getting the data from previous Op, validating user specified column names, // applying a list of TensorOps to each of the data, process the results and then @@ -234,7 +236,7 @@ class MapOp : public ParallelOp { // @param in_buffer A raw pointer to the DataBuffer. A raw pointer is fine because this function doesn't manage memory // and is not shared with other threads. // @param[out] new_tensor_table A new Tensor Table to be populated in this function. - Status WorkerCompute(DataBuffer *in_buffer, TensorQTable *new_tensor_table, + Status WorkerCompute(const TensorRow &in_row, TensorRow *out_row, const std::vector> &job_list); // Private function that create the final column name to index mapping and diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.cc index 413175d87c..57742b8ead 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.cc @@ -67,32 +67,25 @@ void ProjectOp::Print(std::ostream &out, bool show_all) const { } // Gets a buffer from the child operator and projects the buffer. -Status ProjectOp::GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) { - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(p_buffer, worker_id, retry_if_eoe)); - if (!((*p_buffer)->eoe()) && !((*p_buffer)->eof())) { - RETURN_IF_NOT_OK(Project(p_buffer)); +Status ProjectOp::GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) { + RETURN_IF_NOT_OK(child_[0]->GetNextRow(row, worker_id, retry_if_eoe)); + if (!row->eoe() && !row->eof()) { + *row = Project(*row); } - if ((*p_buffer)->eoe()) { + if (row->eoe()) { UpdateRepeatAndEpochCounter(); } return Status::OK(); } -Status ProjectOp::Project(std::unique_ptr *data_buffer) { - std::unique_ptr new_tensor_table = std::make_unique(); - while ((*data_buffer)->NumRows() > 0) { - TensorRow current_row; - RETURN_IF_NOT_OK((*data_buffer)->PopRow(¤t_row)); - TensorRow new_row; - (void)std::transform(projected_column_indices_.begin(), projected_column_indices_.end(), - std::back_inserter(new_row), [¤t_row](uint32_t x) { return current_row[x]; }); - // Now if columns changed after map, we don't know which column we should keep, - // so temporarily we don't support print file_path after ProjectOp. - new_row.setPath({}); - new_tensor_table->push_back(new_row); - } - (*data_buffer)->set_tensor_table(std::move(new_tensor_table)); - return Status::OK(); +TensorRow ProjectOp::Project(const TensorRow &row) { + TensorRow new_row; + (void)std::transform(projected_column_indices_.begin(), projected_column_indices_.end(), std::back_inserter(new_row), + [&row](uint32_t x) { return row[x]; }); + // Now if columns changed after map, we don't know which column we should keep, + // so temporarily we don't support print file_path after ProjectOp. + new_row.setPath({}); + return new_row; } // Class functor operator () override. @@ -152,10 +145,10 @@ Status ProjectOp::ComputeColMap() { return Status::OK(); } -Status ProjectOp::GetNextRow(TensorRow *row) { +Status ProjectOp::GetNextRowPullMode(TensorRow *row) { ComputeColMap(); TensorRow new_row; - RETURN_IF_NOT_OK(child_[0]->GetNextRow(&new_row)); + RETURN_IF_NOT_OK(child_[0]->GetNextRowPullMode(&new_row)); (void)std::transform(projected_column_indices_.begin(), projected_column_indices_.end(), std::back_inserter(*row), [&new_row](uint32_t x) { return new_row[x]; }); // Now if columns changed after map, we don't know which column we should keep, diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.h index d741530f6d..5b530c906e 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/project_op.h @@ -81,7 +81,7 @@ class ProjectOp : public PipelineOp { // Gets a buffer from the child node and projects that buffer. The caller is typically our parent node. // @param p_buffer - output pointer to the projected buffer. // @param worker_id - The worker id - Status GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) override; + Status GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) override; // Base-class override. Return the number of workers in the first parent. // @param workerId - The worker id @@ -101,7 +101,7 @@ class ProjectOp : public PipelineOp { // @return Status The status code returned Status EofReceived(int32_t worker_id) override; - Status GetNextRow(TensorRow *row) override; + Status GetNextRowPullMode(TensorRow *row) override; // Op name getter // @return Name of the current Op @@ -111,7 +111,7 @@ class ProjectOp : public PipelineOp { std::vector columns_to_project_; std::vector projected_column_indices_; - Status Project(std::unique_ptr *data_buffer); + TensorRow Project(const TensorRow &row); // Computing the assignment of the column name map. // @return - Status diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.cc index bde4bc9549..f5a5eb337b 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.cc @@ -59,32 +59,24 @@ RenameOp::~RenameOp() {} // main entry point for rename Status RenameOp::operator()() { TaskManager::FindMe()->Post(); - std::unique_ptr curr_buffer; - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); - if (curr_buffer->buffer_flags() != DataBuffer::kDeBFlagNone) { - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(curr_buffer))); - std::string err_msg = "Rename first buffer got was control signal"; - // if 1st eoe or eof, pass it on then return - RETURN_STATUS_UNEXPECTED(err_msg); - } + child_iterator_ = std::make_unique(this, 0, 0); - while (curr_buffer->eof() == false) { - while (curr_buffer->eoe() == false) { - // push the renamed input buffer - MS_LOG(DEBUG) << "Rename operator pushing next buffer."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(curr_buffer))); - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); - } // end of while eoe loop + TensorRow new_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); - // we got eoe, now try again until we get eof + while (!new_row.eof()) { + while (!new_row.eoe()) { + MS_LOG(DEBUG) << "Rename operator pushing next buffer."; + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); + } + RETURN_IF_NOT_OK(out_connector_->SendEOE()); MS_LOG(DEBUG) << "Rename operator EOE Received."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOE)))); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); MS_LOG(DEBUG) << "Rename operator fetching buffer after EOE."; - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); - } // end of while eof loop - - MS_LOG(DEBUG) << "Rename opeerator EOF Received."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOF)))); + } + RETURN_IF_NOT_OK(out_connector_->SendEOF()); + MS_LOG(DEBUG) << "Rename operator EOF Received."; return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.h index 6b35618deb..6898026a49 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/rename_op.h @@ -21,6 +21,7 @@ #include #include #include "minddata/dataset/core/tensor.h" +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/pipeline_op.h" #include "minddata/dataset/util/status.h" @@ -125,6 +126,8 @@ class RenameOp : public PipelineOp { // Variable to store the output column names std::vector out_columns_; + + std::unique_ptr child_iterator_; // An iterator for fetching. }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.cc index 8fd3d2a883..190885710c 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.cc @@ -78,27 +78,24 @@ void RepeatOp::Print(std::ostream &out, bool show_all) const { // a buffer from our child. // This function sets the `retryIfEoe` flag when popping from the child connector. This way, // this function will retry to pop the connector again and will get the non-EOE buffer if any. -Status RepeatOp::GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) { +Status RepeatOp::GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) { if (child_.empty()) { RETURN_STATUS_UNEXPECTED("Pipeline init failed, RepeatOp can't be the first op in pipeline."); } - std::unique_ptr buf; - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf, worker_id, true)); + RETURN_IF_NOT_OK(child_[0]->GetNextRow(row, worker_id, true)); // Loop until non EOE is received - while (buf->eoe()) { + while (row->eoe()) { RETURN_IF_NOT_OK(EoeReceived(worker_id)); if (state_ == OpState::kDeOpIdle) { - *p_buffer = std::move(buf); return Status::OK(); } - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf, worker_id, true)); + RETURN_IF_NOT_OK(child_[0]->GetNextRow(row, worker_id, true)); } // Check if the last buf is next eof - if (buf->eof()) { + if (row->eof()) { RETURN_IF_NOT_OK(EofReceived(worker_id)); } - *p_buffer = std::move(buf); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.h index 09c305073b..8c248c8087 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/repeat_op.h @@ -91,7 +91,7 @@ class RepeatOp : public PipelineOp { // @param worker_id - The worker id // @param retry_if_eoe Set this flag to true to allow calling pop() again after the first pop() returns EOE. // @return Status The status code returned - Status GetNextBuffer(std::unique_ptr *p_buffer, int32_t worker_id, bool retry_if_eoe) override; + Status GetNextRow(TensorRow *row, int32_t worker_id, bool retry_if_eoe) override; // Base-class override for handling cases when an eoe is received. // @param worker_id - The worker id diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/shuffle_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/shuffle_op.cc index 0d8cd168f7..842468c892 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/shuffle_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/shuffle_op.cc @@ -152,6 +152,7 @@ Status ShuffleOp::operator()() { // This is our main loop exit condition, when the iterator has no more data completely. if (child_iterator_->eof_handled()) { + RETURN_IF_NOT_OK(out_connector_->SendEOF()); break; } @@ -170,21 +171,11 @@ Status ShuffleOp::operator()() { // tensor table. We remove the data from the shuffle buffer, leaving that slot // in the table as an empty vector int64_t random_slot = rng_() % (shuffle_last_row_idx_ + 1); - new_buffer_table->push_back(std::move((*shuffle_buffer_)[random_slot])); + TensorRow random_row = std::move((*shuffle_buffer_)[random_slot]); + MS_LOG(DEBUG) << "Shuffle operator sending a row to output."; + RETURN_IF_NOT_OK(out_connector_->Add(std::move(random_row))); // Step 3) - // If the output tensor table is at the requested size, then create a buffer for it - // and send this buffer on it's way up the pipeline. Special case is if this is the - // last row then we also send it. - if (new_buffer_table->size() == rows_per_buffer_ || shuffle_last_row_idx_ == 0) { - auto new_buffer = std::make_unique(buffer_counter_, DataBuffer::kDeBFlagNone); - new_buffer->set_tensor_table(std::move(new_buffer_table)); - buffer_counter_++; - MS_LOG(DEBUG) << "Shuffle operator sending a buffer to output."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(new_buffer))); - } - - // Step 4) // Take the last row from shuffle buffer, and swap it into the row position that was // just vacated. This makes the shuffle buffer contiguous, with an empty slot at the // tail of the shuffle buffer. @@ -192,7 +183,7 @@ Status ShuffleOp::operator()() { (*shuffle_buffer_)[random_slot] = std::move((*shuffle_buffer_)[shuffle_last_row_idx_]); } - // Step 5) + // Step 4) // Refill the last slot of the shuffle buffer with the next row from input if we are in the // active state. // If we are in the draining state, we do not need to fetch another row to replace the one we @@ -218,14 +209,14 @@ Status ShuffleOp::operator()() { // Since we overloaded eoeReceived function, we are responsible to flow the EOE up the // pipeline manually now that we are done draining the shuffle buffer MS_LOG(DEBUG) << "Shuffle operator sending EOE."; - auto eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); // Do not wait for any reset to be flown down from operators above us. // Instead, manually update ourselves and then go reloop to start fetching from child operator // right away. Any Reset() from the parent will still perform common reset actions. RETURN_IF_NOT_OK(this->SelfReset()); } + return Status::OK(); } @@ -252,6 +243,7 @@ Status ShuffleOp::InitShuffleBuffer() { if (child_iterator_->eof_handled()) { MS_LOG(DEBUG) << "Shuffle operator init picked up EOF. No more epochs."; + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.cc index 3a260755f4..0249408692 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.cc @@ -19,9 +19,9 @@ #include "minddata/dataset/core/config_manager.h" #include "minddata/dataset/engine/data_buffer.h" +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/skip_op.h" #include "minddata/dataset/engine/db_connector.h" -#include "minddata/dataset/engine/execution_tree.h" #include "minddata/dataset/util/log_adapter.h" namespace mindspore { @@ -69,57 +69,32 @@ void SkipOp::Print(std::ostream &out, bool show_all) const { } } -// Base-class override for handling cases when an eoe is received. -Status SkipOp::EoeReceived(int32_t worker_id) { - skip_count_ = 0; - state_ = OpState::kDeOpIdle; - return Status::OK(); -} - // main entry point for skip Status SkipOp::operator()() { TaskManager::FindMe()->Post(); - std::unique_ptr curr_buffer; - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); + child_iterator_ = std::make_unique(this, 0, 0); - while (curr_buffer->eof() == false) { + TensorRow new_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); + while (!new_row.eof()) { // Reset count skip_count_ = 0; - while (curr_buffer->eoe() == false) { + while (!new_row.eoe()) { // Drop first count rows - while (skip_count_ < max_skips_) { - if (curr_buffer->eoe() || curr_buffer->eof()) { - break; - } - // Consider the rows of buffer more than one - TensorRow drop_row; - int row_num = curr_buffer->NumRows(); - int drop_num = row_num + skip_count_ < max_skips_ ? row_num : max_skips_ - skip_count_; - skip_count_ += drop_num; - for (int i = 0; i < drop_num; i++) { - RETURN_IF_NOT_OK(curr_buffer->PopRow(&drop_row)); - } - if (curr_buffer->NumRows() == 0) { - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); - } + if (skip_count_ < max_skips_) { + skip_count_++; + } else { + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); } - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(curr_buffer))); - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } // we got eoe, now try again until we got eof MS_LOG(DEBUG) << "Skip operator EOE Received."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOE)))); - RETURN_IF_NOT_OK(GetNextInput(&curr_buffer)); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } - MS_LOG(DEBUG) << "Skip operator EOF Received."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOF)))); - return Status::OK(); -} - -// Base-class override for handling cases when an eof is received. -Status SkipOp::EofReceived(int32_t worker_id) { - MS_LOG(DEBUG) << "Skip operator EOF received, do nothing now."; + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.h index 05d95c880e..12068180cf 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/skip_op.h @@ -66,14 +66,6 @@ class SkipOp : public PipelineOp { // @return Status The status code returned Status operator()() override; - // Base-class override for handling cases when an eoe is received. - // @param worker_id - The worker id - Status EoeReceived(int32_t worker_id) override; - - // Base-class override for handling cases when an eof is received. - // @param worker_id - The worker id - Status EofReceived(int32_t worker_id) override; - // Op name getter // @return Name of the current Op std::string Name() const override { return kSkipOp; } @@ -81,6 +73,8 @@ class SkipOp : public PipelineOp { private: int32_t max_skips_; // The number of skips that the user requested int32_t skip_count_; // A counter for the current number of executed skips + + std::unique_ptr child_iterator_; // An iterator for fetching. }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.cc index f6a8d3ad7f..968afc0b70 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.cc @@ -485,7 +485,7 @@ Status AlbumOp::ComputeColMap() { return Status::OK(); } -Status AlbumOp::GetNextRow(TensorRow *row) { +Status AlbumOp::GetNextRowPullMode(TensorRow *row) { if (image_rows_.empty()) PrescanEntry(); if (sample_ids_ == nullptr) { RETURN_IF_NOT_OK(this->InitSampler()); diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.h index 8174c749f3..bfe88760cb 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/album_op.h @@ -267,7 +267,7 @@ class AlbumOp : public MappableLeafOp { /// \return Status The status code returned Status LaunchThreadsAndInitOp() override; - Status GetNextRow(TensorRow *row) override; + Status GetNextRowPullMode(TensorRow *row) override; /// Private function for computing the assignment of the column name map. /// \return Status The status code returned diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/clue_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/clue_op.cc index e03679b47b..76b43176e3 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/clue_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/clue_op.cc @@ -145,11 +145,8 @@ Status ClueOp::LoadFile(const std::string &file, int64_t start_offset, int64_t e RETURN_STATUS_UNEXPECTED("Invalid file, failed to open file: " + file); } - int64_t rows_each_buffer = 0; int64_t rows_total = 0; std::string line; - std::unique_ptr cur_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - std::unique_ptr tensor_table = std::make_unique(); while (getline(handle, line)) { if (line.empty()) { @@ -177,31 +174,18 @@ Status ClueOp::LoadFile(const std::string &file, int64_t start_offset, int64_t e // Add file path info std::vector file_path(cols_count, file); tRow.setPath(file_path); - tensor_table->push_back(std::move(tRow)); int cout = 0; for (auto &p : cols_to_keyword_) { std::shared_ptr tensor; RETURN_IF_NOT_OK(GetValue(js, p.second, &tensor)); - (*tensor_table)[rows_each_buffer][cout] = std::move(tensor); + tRow[cout] = std::move(tensor); cout++; } - rows_each_buffer++; rows_total++; - if (rows_each_buffer == rows_per_buffer_) { - cur_buffer->set_tensor_table(std::move(tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(cur_buffer))); - - cur_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - tensor_table = std::make_unique(); - rows_each_buffer = 0; - } + RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(tRow))); } - if (rows_each_buffer > 0) { - cur_buffer->set_tensor_table(std::move(tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(cur_buffer))); - } return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.cc index 8a5fefe7b7..801844741a 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.cc @@ -101,20 +101,17 @@ CsvOp::CsvParser::CsvParser(int32_t worker_id, JaggedConnector *connector, int64 file_path_(file_path), cur_state_(START_OF_FILE), pos_(0), - cur_row_(0), cur_col_(0), total_rows_(0), start_offset_(0), end_offset_(std::numeric_limits::max()), err_message_("unknown") { - cur_buffer_ = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); InitCsvParser(); } void CsvOp::CsvParser::Reset() { cur_state_ = START_OF_FILE; pos_ = 0; - cur_row_ = 0; cur_col_ = 0; } @@ -170,11 +167,11 @@ int CsvOp::CsvParser::PutRecord(int c) { Tensor::CreateScalar(s, &t); break; } - if (cur_col_ >= (*tensor_table_)[cur_row_].size()) { + if (cur_col_ >= cur_row_.size()) { err_message_ = "Number of file columns does not match the tensor table"; return -1; } - (*tensor_table_)[cur_row_][cur_col_] = std::move(t); + cur_row_[cur_col_] = std::move(t); pos_ = 0; cur_col_++; return 0; @@ -203,18 +200,10 @@ int CsvOp::CsvParser::PutRow(int c) { } total_rows_++; - cur_row_++; cur_col_ = 0; - if (cur_row_ == csv_rows_per_buffer_) { - cur_buffer_->set_tensor_table(std::move(tensor_table_)); + buffer_connector_->Add(worker_id_, std::move(cur_row_)); - buffer_connector_->Add(worker_id_, std::move(cur_buffer_)); - - cur_buffer_ = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - tensor_table_ = std::make_unique(); - cur_row_ = 0; - } return 0; } @@ -230,11 +219,6 @@ int CsvOp::CsvParser::EndFile(int c) { return ret; } } - - if (cur_row_ > 0) { - cur_buffer_->set_tensor_table(std::move(tensor_table_)); - buffer_connector_->Add(worker_id_, std::move(cur_buffer_)); - } return 0; } @@ -345,8 +329,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_ = std::make_unique(); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); this->str_buf_[0] = c; this->pos_ = 1; return 0; @@ -357,8 +340,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_ = std::make_unique(); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); return this->PutRecord(c); }}}, {{State::START_OF_FILE, Message::MS_QUOTE}, @@ -367,8 +349,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_ = std::make_unique(); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); this->pos_ = 0; return 0; }}}, @@ -454,7 +435,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); } this->str_buf_[0] = c; this->pos_ = 1; @@ -467,7 +448,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); } return this->PutRecord(c); }}}, @@ -478,7 +459,7 @@ Status CsvOp::CsvParser::InitCsvParser() { TensorRow row(column_default_.size(), nullptr); std::vector file_path(column_default_.size(), file_path_); row.setPath(file_path); - this->tensor_table_->push_back(row); + this->cur_row_ = std::move(row); } return 0; }}}, diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.h index 8e4e302cac..a6a61c3501 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/csv_op.h @@ -133,7 +133,6 @@ class CsvOp : public NonMappableLeafOp { std::vector> column_default_; State cur_state_; size_t pos_; - int cur_row_; int cur_col_; int64_t total_rows_; int64_t start_offset_; @@ -141,8 +140,7 @@ class CsvOp : public NonMappableLeafOp { StateDiagram sd; StateDiagram sdl; std::vector str_buf_; - std::unique_ptr tensor_table_; - std::unique_ptr cur_buffer_; + TensorRow cur_row_; std::string err_message_; std::string file_path_; }; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.cc index 55b4709d6b..08f5a6abc5 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.cc @@ -41,19 +41,18 @@ Status GeneratorOp::Builder::SanityCheck() { Status GeneratorOp::Builder::Build(std::shared_ptr *ptr) { RETURN_IF_NOT_OK(SanityCheck()); *ptr = std::make_shared(build_generator_function_, build_column_names_, build_column_types_, - build_prefetch_size_, build_buffer_size_, build_op_connector_size_, nullptr); + build_prefetch_size_, build_op_connector_size_, nullptr); return (*ptr)->Init(); } GeneratorOp::GeneratorOp(py::function generator_function, std::vector column_names, - std::vector column_types, int32_t prefetch_size, int32_t buffer_size, - int32_t connector_size, std::shared_ptr sampler) + std::vector column_types, int32_t prefetch_size, int32_t connector_size, + std::shared_ptr sampler) : PipelineOp(connector_size, std::move(sampler)), generator_function_(generator_function), column_names_(column_names), column_types_(column_types), prefetch_size_(prefetch_size), - buffer_size_(buffer_size), buffer_id_(0), generator_counter_(0) {} @@ -145,16 +144,6 @@ Status GeneratorOp::PyRowToTensorRow(py::object py_data, TensorRow *tensor_row) return Status(StatusCode::kSuccess, ""); } -Status GeneratorOp::FillBuffer(TensorQTable *tt) { - for (int i = 0; i < buffer_size_; i++) { - TensorRow row; - RETURN_IF_NOT_OK(PyRowToTensorRow(generator_.attr("__next__")(), &row)); - tt->push_back(std::move(row)); - generator_counter_++; - } - return Status::OK(); -} - // Entry point for Generator, called by launch() // Note that this function is very easy to break because of the Python GIL mechanism // The master thread has the following workflow @@ -192,23 +181,22 @@ Status GeneratorOp::operator()() { // Handshake with TaskManager to synchronize thread creation TaskManager::FindMe()->Post(); RETURN_IF_NOT_OK(wp_.Register(tree_->AllTasks())); - std::unique_ptr fetched_buffer; int64_t num_rows_sampled = sampler_ ? sampler_->CalculateNumSamples(num_rows_) : num_rows_; RETURN_IF_NOT_OK(Init()); bool eof = false; while (!eof) { - // Create new buffer each iteration - fetched_buffer = std::make_unique(buffer_id_++, DataBuffer::kDeBFlagNone); - std::unique_ptr fetched_table = std::make_unique(); + // Create new row each iteration bool eoe = false; + TensorRow new_row; { py::gil_scoped_acquire gil_acquire; if (Py_IsInitialized() == 0) { return Status(StatusCode::kMDPythonInterpreterFailure, "Python Interpreter is finalized"); } try { - RETURN_IF_NOT_OK(FillBuffer(fetched_table.get())); + RETURN_IF_NOT_OK(PyRowToTensorRow(generator_.attr("__next__")(), &new_row)); + generator_counter_++; } catch (py::error_already_set &e) { eoe = e.matches(PyExc_StopIteration); // Restore exception to python @@ -226,20 +214,18 @@ Status GeneratorOp::operator()() { } } } - if (fetched_table->size() > 0) { - fetched_buffer->set_tensor_table(std::move(fetched_table)); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(fetched_buffer))); - } + if (!new_row.empty()) RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); + if (eoe) { // Push out EOE upon StopIteration exception from generator MS_LOG(DEBUG) << "Generator operator sends out EOE."; std::unique_ptr eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); if (IsLastIteration()) { // If last repeat or not repeated, push out EOF and exit master loop MS_LOG(DEBUG) << "Generator operator sends out EOF."; std::unique_ptr eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eof_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); MS_LOG(DEBUG) << "Generator operator main execution loop complete."; eof = true; } else { diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.h index b2c5abdcd4..0885a4e49d 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/generator_op.h @@ -94,7 +94,7 @@ class GeneratorOp : public PipelineOp, public RandomAccessOp { }; GeneratorOp(py::function generator_function, std::vector column_names, - std::vector column_types, int32_t prefetch_size, int32_t buffer_size, int32_t connector_size, + std::vector column_types, int32_t prefetch_size, int32_t connector_size, std::shared_ptr sampler); ~GeneratorOp() = default; @@ -135,7 +135,6 @@ class GeneratorOp : public PipelineOp, public RandomAccessOp { std::vector column_names_; std::vector column_types_; int32_t prefetch_size_; - int32_t buffer_size_; int64_t generator_counter_; py::object generator_; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.cc index b17d885bec..5f6a131d79 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.cc @@ -14,11 +14,9 @@ * limitations under the License. */ #include "minddata/dataset/engine/datasetops/source/mappable_leaf_op.h" -#include #include #include "utils/ms_utils.h" #include "minddata/dataset/core/config_manager.h" -#include "minddata/dataset/core/tensor_shape.h" #include "minddata/dataset/engine/datasetops/source/sampler/sequential_sampler.h" #include "minddata/dataset/engine/db_connector.h" #include "minddata/dataset/engine/execution_tree.h" @@ -28,44 +26,34 @@ namespace dataset { MappableLeafOp::MappableLeafOp(int32_t num_wkrs, int32_t queue_size, std::shared_ptr sampler, int32_t rows_per_buffer) - : ParallelOp(num_wkrs, queue_size, std::move(sampler)), - row_cnt_(0), - buf_cnt_(0), - rows_per_buffer_(rows_per_buffer) {} + : ParallelOp(num_wkrs, queue_size, std::move(sampler)), rows_per_buffer_(rows_per_buffer) {} // Main logic, Register Queue with TaskGroup, launch all threads and do the functor's work Status MappableLeafOp::operator()() { RETURN_IF_NOT_OK(LaunchThreadsAndInitOp()); std::unique_ptr sampler_buffer; RETURN_IF_NOT_OK(sampler_->GetNextSample(&sampler_buffer)); - while (true) { // each iterator is 1 epoch - std::vector keys; - keys.reserve(rows_per_buffer_); + int64_t row_cnt = 0; + while (true) { // each iteration is 1 epoch, breaks when IsLastIteration() is true while (sampler_buffer->eoe() == false) { TensorRow sample_row; RETURN_IF_NOT_OK(sampler_buffer->PopRow(&sample_row)); std::shared_ptr sample_ids = sample_row[0]; for (auto itr = sample_ids->begin(); itr != sample_ids->end(); ++itr) { - if ((*itr) >= num_rows_) continue; // index out of bound, skipping - keys.push_back(*itr); - row_cnt_++; - if (row_cnt_ % rows_per_buffer_ == 0) { - RETURN_IF_NOT_OK( - io_block_queues_[buf_cnt_++ % num_workers_]->Add(std::make_unique(keys, IOBlock::kDeIoBlockNone))); - keys.clear(); + if ((*itr) >= num_rows_) { + MS_LOG(WARNING) << "Skipping sample with ID: " << *itr << " since it is out of bound: " << num_rows_; + continue; // index out of bound, skipping } + RETURN_IF_NOT_OK( + io_block_queues_[row_cnt++ % num_workers_]->Add(std::make_unique(*itr, IOBlock::kDeIoBlockNone))); } RETURN_IF_NOT_OK(sampler_->GetNextSample(&sampler_buffer)); } - if (keys.empty() == false) { - RETURN_IF_NOT_OK( - io_block_queues_[(buf_cnt_++) % num_workers_]->Add(std::make_unique(keys, IOBlock::kDeIoBlockNone))); - } if (IsLastIteration()) { std::unique_ptr eoe_block = std::make_unique(IOBlock::kDeIoBlockFlagEoe); std::unique_ptr eof_block = std::make_unique(IOBlock::kDeIoBlockFlagEof); - RETURN_IF_NOT_OK(io_block_queues_[(buf_cnt_++) % num_workers_]->Add(std::move(eoe_block))); - RETURN_IF_NOT_OK(io_block_queues_[(buf_cnt_++) % num_workers_]->Add(std::move(eof_block))); + RETURN_IF_NOT_OK(io_block_queues_[(row_cnt++) % num_workers_]->Add(std::move(eoe_block))); + RETURN_IF_NOT_OK(io_block_queues_[(row_cnt++) % num_workers_]->Add(std::move(eof_block))); for (int32_t i = 0; i < num_workers_; ++i) { RETURN_IF_NOT_OK( io_block_queues_[i]->Add(std::make_unique(std::vector(), IOBlock::kDeIoBlockNone))); @@ -73,7 +61,7 @@ Status MappableLeafOp::operator()() { return Status::OK(); } else { // not the last repeat. RETURN_IF_NOT_OK( - io_block_queues_[(buf_cnt_++) % num_workers_]->Add(std::make_unique(IOBlock::kDeIoBlockFlagEoe))); + io_block_queues_[(row_cnt++) % num_workers_]->Add(std::make_unique(IOBlock::kDeIoBlockFlagEoe))); } if (epoch_sync_flag_) { @@ -104,49 +92,34 @@ Status MappableLeafOp::InitSampler() { } // contains the main logic of pulling a IOBlock from IOBlockQueue, load a buffer and push the buffer to out_connector_ -// IMPORTANT: 1 IOBlock produces 1 DataBuffer +// IMPORTANT: 1 IOBlock produces 1 row Status MappableLeafOp::WorkerEntry(int32_t worker_id) { TaskManager::FindMe()->Post(); - int64_t buffer_id = worker_id; std::unique_ptr io_block; RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&io_block)); while (io_block != nullptr) { - if (io_block->wait() == true) { + if (io_block->wait()) { // Sync io_block is a signal that master thread wants us to pause and sync with other workers. // The last guy who comes to this sync point should reset the counter and wake up the master thread. if (++num_workers_paused_ == num_workers_) { wait_for_workers_post_.Set(); } - } else if (io_block->eoe() == true) { - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::make_unique(0, DataBuffer::kDeBFlagEOE))); - buffer_id = worker_id; - } else if (io_block->eof() == true) { - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::make_unique(0, DataBuffer::kDeBFlagEOF))); + } else if (io_block->eoe()) { + RETURN_IF_NOT_OK(out_connector_->SendEOE(worker_id)); + } else if (io_block->eof()) { + RETURN_IF_NOT_OK(out_connector_->SendEOF(worker_id)); } else { std::vector keys; RETURN_IF_NOT_OK(io_block->GetKeys(&keys)); - if (keys.empty() == true) return Status::OK(); // empty key is a quit signal for workers - std::unique_ptr db = std::make_unique(buffer_id, DataBuffer::kDeBFlagNone); - RETURN_IF_NOT_OK(LoadBuffer(keys, &db)); - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::move(db))); - buffer_id += num_workers_; + if (keys.empty()) return Status::OK(); // empty key is a quit signal for workers + TensorRow trow; + RETURN_IF_NOT_OK(this->LoadTensorRow(keys[0], &trow)); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(trow), worker_id)); } RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&io_block)); } RETURN_STATUS_UNEXPECTED("Unexpected nullptr received in worker"); } -// Looping over LoadTensorRow to make 1 DataBuffer. 1 function call produces 1 buffer -Status MappableLeafOp::LoadBuffer(const std::vector &keys, std::unique_ptr *db) { - std::unique_ptr deq = std::make_unique(); - TensorRow trow; - for (const int64_t &key : keys) { - RETURN_IF_NOT_OK(this->LoadTensorRow(key, &trow)); - deq->push_back(std::move(trow)); - } - (*db)->set_tensor_table(std::move(deq)); - return Status::OK(); -} - } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.h index 897fb76123..9ff9a61c10 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mappable_leaf_op.h @@ -52,53 +52,47 @@ using FolderImagesPair = std::shared_ptr exts - set of file extensions to read, if empty, read everything under the dir - // @param td::unique_ptr sampler - sampler tells the source what to read + /// Constructor + /// \param int32_t num_wkrs - Num of workers reading images in parallel + /// \param int32_t queue_size - connector queue size + /// \param td::unique_ptr sampler - sampler tells the source what to read MappableLeafOp(int32_t num_wkrs, int32_t queue_size, std::shared_ptr sampler, int32_t rows_per_buffer); - // Destructor. + /// Destructor. ~MappableLeafOp() = default; - // Main Loop of MappableLeaf - // Master thread: Fill IOBlockQueue, then goes to sleep - // Worker thread: pulls IOBlock from IOBlockQueue, work on it then put buffer to mOutConnector - // @return Status The status code returned + /// Main Loop of MappableLeaf + /// Master thread: Fill IOBlockQueue, then goes to sleep + /// Worker thread: pulls IOBlock from IOBlockQueue, work on it then put row to out_connector_ + /// \return Status The status code returned Status operator()() override; - // Op name getter - // @return Name of the current Op + /// Op name getter + /// @return Name of the current Op std::string Name() const override { return "MappableLeafPp"; } protected: - // Initialize Sampler, calls sampler->Init() within - // @return Status The status code returned + /// Initialize Sampler, calls sampler->Init() within + /// @return Status The status code returned Status InitSampler(); - // // Called first when function is called - // // @return + /// Called first when function is called + /// \return Status The status code returned virtual Status LaunchThreadsAndInitOp() = 0; + /// Worker thread pulls a number of IOBlock from IOBlock Queue, make a row and push it to Connector + /// \param int32_t workerId - id of each worker + /// \return Status The status code returned Status WorkerEntry(int32_t workerId) override; - // @param const std::vector &keys - keys in ioblock - // @param std::unique_ptr db - // @return Status The status code returned - Status LoadBuffer(const std::vector &keys, std::unique_ptr *db); - - // Load a tensor row according to a pair - // @param row_id_type row_id - id for this tensor row - // @param ImageLabelPair pair - - // @param TensorRow row - loaded row - // @return Status The status code returned + /// Virtual function to Load a tensor row at location row_id + /// \param row_id_type row_id - id for this tensor row + /// \param TensorRow row - loaded row + /// \return Status The status code returned virtual Status LoadTensorRow(row_id_type row_id, TensorRow *row) = 0; - // reset Op - // @return Status The status code returned + /// Reset function to be called after every epoch to reset the source op after + /// \return Status The status code returned Status Reset() override; int32_t rows_per_buffer_; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.cc index 470b505c73..00140c38cc 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.cc @@ -70,10 +70,9 @@ Status MindRecordOp::Builder::Build(std::shared_ptr *ptr) { if (build_num_padded_ > 0) { sample_json = ToJson(build_sample_); } - new_mind_record_op = - std::make_shared(build_num_mind_record_workers_, build_rows_per_buffer_, build_dataset_file_, - build_load_dataset_, build_op_connector_queue_size_, build_columns_to_load_, - build_operators_, build_num_padded_, sample_json, build_sample_bytes_); + new_mind_record_op = std::make_shared( + build_num_mind_record_workers_, build_dataset_file_, build_load_dataset_, build_op_connector_queue_size_, + build_columns_to_load_, build_operators_, build_num_padded_, sample_json, build_sample_bytes_); RETURN_IF_NOT_OK(new_mind_record_op->Init()); *ptr = std::move(new_mind_record_op); @@ -111,13 +110,11 @@ mindrecord::json MindRecordOp::Builder::ToJson(const py::handle &obj) { } // Constructor of the MindRecordOp. -MindRecordOp::MindRecordOp(int32_t num_mind_record_workers, int32_t rows_per_buffer, - std::vector dataset_file, bool load_dataset, int32_t op_connector_queue_size, - const std::vector &columns_to_load, +MindRecordOp::MindRecordOp(int32_t num_mind_record_workers, std::vector dataset_file, bool load_dataset, + int32_t op_connector_queue_size, const std::vector &columns_to_load, const std::vector> &operators, int64_t num_padded, const mindrecord::json &sample_json, const std::map &sample_bytes) - : MappableLeafOp(num_mind_record_workers, op_connector_queue_size, std::make_shared(0, 0), - rows_per_buffer), + : MappableLeafOp(num_mind_record_workers, op_connector_queue_size, std::make_shared(0, 0), 1), dataset_file_(dataset_file), load_dataset_(load_dataset), columns_to_load_(columns_to_load), @@ -211,8 +208,7 @@ void MindRecordOp::Print(std::ostream &out, bool show_all) const { for (auto &file : dataset_file_) { out << file << " "; } - out << "\nNumber of rows : " << num_rows_ << "\nRows per buffer : " << rows_per_buffer_ - << "\nNumber of buffers : " << buffers_needed_ + out << "\nNumber of rows : " << num_rows_ << "\nNumber of buffers : " << buffers_needed_ << "\nNumber of ShardReader workers : " << num_mind_record_workers_ << "\n\n"; } } @@ -232,14 +228,12 @@ Status MindRecordOp::WorkerEntry(int32_t worker_id) { continue; } if (io_block->eoe()) { - RETURN_IF_NOT_OK( - out_connector_->Add(worker_id, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOE)))); + RETURN_IF_NOT_OK(out_connector_->SendEOE(worker_id)); RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&io_block)); continue; } if (io_block->eof()) { - RETURN_IF_NOT_OK( - out_connector_->Add(worker_id, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOF)))); + RETURN_IF_NOT_OK(out_connector_->SendEOF(worker_id)); RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&io_block)); continue; } @@ -256,52 +250,41 @@ Status MindRecordOp::WorkerEntry(int32_t worker_id) { return Status::OK(); // empty key is a quit signal for workers } - const uint64_t buffer_id = keys[0]; - std::unique_ptr fetched_buffer; + const uint64_t row_id = keys[0]; + TensorRow fetched_row; // Get the next buffer. Push it up to the output connector. - if (buffer_id % LOG_INTERVAL == 0) { - MS_LOG(DEBUG) << "MindRecord operator consumed buffer " << buffer_id << " by worker " << worker_id << "."; + if (row_id % LOG_INTERVAL == 0) { + MS_LOG(DEBUG) << "MindRecord operator consumed row " << row_id << " by worker " << worker_id << "."; } - RETURN_IF_NOT_OK(GetBufferFromReader(&fetched_buffer, buffer_id, worker_id)); - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::move(fetched_buffer))); + RETURN_IF_NOT_OK(GetRowFromReader(&fetched_row, row_id, worker_id)); + RETURN_IF_NOT_OK(out_connector_->Add(std::move(fetched_row), worker_id)); RETURN_IF_NOT_OK(io_block_queues_[worker_id]->PopFront(&io_block)); } RETURN_STATUS_UNEXPECTED("Unexpected nullptr received in worker."); } -Status MindRecordOp::GetBufferFromReader(std::unique_ptr *fetched_buffer, int64_t buffer_id, - int32_t worker_id) { - *fetched_buffer = std::make_unique(buffer_id, DataBuffer::kDeBFlagNone); - std::unique_ptr tensor_table = std::make_unique(); - for (int32_t i = 0; i < rows_per_buffer_; ++i) { - int32_t row_id = buffer_id * rows_per_buffer_ + i; - auto rc = shard_reader_->GetNextById(row_id, worker_id); - auto task_type = rc.first; - auto tupled_buffer = rc.second; - if (task_type == mindrecord::TaskType::kPaddedTask) { - TensorRow tensor_row; - RETURN_IF_NOT_OK(LoadTensorRow(&tensor_row, {}, mindrecord::json(), task_type)); - std::vector file_path(tensor_row.size(), dataset_file_[0]); - tensor_row.setPath(file_path); - tensor_table->push_back(std::move(tensor_row)); - } - if (tupled_buffer.empty()) break; - if (task_type == mindrecord::TaskType::kCommonTask) { - for (const auto &tupled_row : tupled_buffer) { - std::vector columns_blob = std::get<0>(tupled_row); - mindrecord::json columns_json = std::get<1>(tupled_row); - TensorRow tensor_row; - RETURN_IF_NOT_OK(LoadTensorRow(&tensor_row, columns_blob, columns_json, task_type)); - std::vector file_path(tensor_row.size(), dataset_file_[0]); - tensor_row.setPath(file_path); - tensor_table->push_back(std::move(tensor_row)); - } +Status MindRecordOp::GetRowFromReader(TensorRow *fetched_row, int64_t row_id, int32_t worker_id) { + *fetched_row = {}; + auto rc = shard_reader_->GetNextById(row_id, worker_id); + auto task_type = rc.first; + auto tupled_buffer = rc.second; + if (task_type == mindrecord::TaskType::kPaddedTask) { + RETURN_IF_NOT_OK(LoadTensorRow(fetched_row, {}, mindrecord::json(), task_type)); + std::vector file_path(fetched_row->size(), dataset_file_[0]); + fetched_row->setPath(file_path); + } + if (tupled_buffer.empty()) return Status::OK(); + if (task_type == mindrecord::TaskType::kCommonTask) { + for (const auto &tupled_row : tupled_buffer) { + std::vector columns_blob = std::get<0>(tupled_row); + mindrecord::json columns_json = std::get<1>(tupled_row); + RETURN_IF_NOT_OK(LoadTensorRow(fetched_row, columns_blob, columns_json, task_type)); + std::vector file_path(fetched_row->size(), dataset_file_[0]); + fetched_row->setPath(file_path); } } - // Replace the TensorTable in DataBuffer with the new one. - (*fetched_buffer)->set_tensor_table(std::move(tensor_table)); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.h index 89804eee21..0cf54d9863 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/mindrecord_op.h @@ -134,13 +134,12 @@ class MindRecordOp : public MappableLeafOp { // Constructor of the MindRecordOp. // @note The builder class should be used to call it // @param num_mind_record_workers - The number of workers for the op (run by ShardReader) - // @param rows_per_buffer - The requested number of rows per buffer // @param dataset_file - dataset files // @param op_connector_queue_size - The output connector queue size // @param columns_to_load - The list of columns to use (column name) // @param operators - ShardOperators for Shuffle, Category, Sample - MindRecordOp(int32_t num_mind_record_workers, int32_t rows_per_buffer, std::vector dataset_file, - bool load_dataset, int32_t op_connector_queue_size, const std::vector &columns_to_load, + MindRecordOp(int32_t num_mind_record_workers, std::vector dataset_file, bool load_dataset, + int32_t op_connector_queue_size, const std::vector &columns_to_load, const std::vector> &operators, int64_t num_padded_, const mindrecord::json &sample_json, const std::map &sample_bytes_); @@ -195,7 +194,7 @@ class MindRecordOp : public MappableLeafOp { std::string Name() const override { return "MindRecordOp"; } private: - Status GetBufferFromReader(std::unique_ptr *fetched_buffer, int64_t buffer_id, int32_t worker_id); + Status GetRowFromReader(TensorRow *fetched_row, int64_t row_id, int32_t worker_id); // Parses a single cell and puts the data into a tensor // @param tensor_row - the tensor row to put the parsed data in diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.cc index 3afaebdcff..226f0ddba6 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.cc @@ -1,5 +1,5 @@ /** - * Copyright 2019-2021 Huawei Technologies Co., Ltd + * Copyright 2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -77,7 +77,6 @@ Status NonMappableLeafOp::operator()() { NotifyToFillIOBlockQueue(); while (!finished_reading_dataset_) { - int64_t buffer_id = 0; int32_t workers_done = 0; int64_t rows_read = 0; { @@ -86,22 +85,14 @@ Status NonMappableLeafOp::operator()() { } while (workers_done < num_workers_) { - std::unique_ptr fetched_buffer; - RETURN_IF_NOT_OK(jagged_buffer_connector_->Pop(0, &fetched_buffer)); - if (fetched_buffer->eoe()) { + TensorRow fetched_row; + RETURN_IF_NOT_OK(jagged_buffer_connector_->Pop(0, &fetched_row)); + if (fetched_row.eoe()) { workers_done++; } else if (total_rows_ == 0 || rows_read < total_rows_) { - // we need to push a buffer - if (total_rows_ > 0 && rows_read + fetched_buffer->NumRows() > total_rows_) { - // this is last buffer we need, and we only need a part of it - int64_t rowsToRemove = fetched_buffer->NumRows() - (total_rows_ - rows_read); - RETURN_IF_NOT_OK(fetched_buffer->SliceOff(rowsToRemove)); - } - - rows_read += fetched_buffer->NumRows(); - fetched_buffer->set_id(buffer_id); - buffer_id++; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(fetched_buffer))); + // we need to push a row + RETURN_IF_NOT_OK(out_connector_->Add(std::move(fetched_row), 0)); + rows_read++; } else { // IOBlockQueue thread needs to: // -stop pushing stuff to IOBlockQueue @@ -126,23 +117,20 @@ Status NonMappableLeafOp::operator()() { } // all workers finished reading for this epoch, and we have read all the data from all workers - std::unique_ptr eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); if (IsLastIteration()) { finished_reading_dataset_ = true; NotifyToFillIOBlockQueue(); } else { jagged_buffer_connector_->DoReset(); - buffer_id = 0; // Self-reset to start a new iteration RETURN_IF_NOT_OK(Reset()); } UpdateRepeatAndEpochCounter(); } - std::unique_ptr eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eof_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); RETURN_IF_NOT_OK(PostEndOfData()); @@ -168,8 +156,8 @@ Status NonMappableLeafOp::WorkerEntry(int32_t worker_id) { MS_LOG(DEBUG) << Name() << " operator worker " << worker_id << " loaded file " << filename << "."; } } else { - std::unique_ptr eoe_buffer = std::make_unique(1, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(eoe_buffer))); + TensorRow eoe = TensorRow(TensorRow::kFlagEOE); + RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(eoe))); } RETURN_IF_NOT_OK(PopIoBlockQueue(worker_id, &io_block)); diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.h index c1775ec091..3b8645adfa 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/nonmappable_leaf_op.h @@ -1,5 +1,5 @@ /** - * Copyright 2019-2021 Huawei Technologies Co., Ltd + * Copyright 2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,12 +30,6 @@ #include "minddata/dataset/core/tensor.h" #include "minddata/dataset/engine/datasetops/parallel_op.h" -namespace dataengine { -class Example; -class Feature; -class BytesList; -} // namespace dataengine - namespace mindspore { namespace dataset { template diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.cc index ee7d5532a1..0158905bfa 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.cc @@ -46,8 +46,8 @@ RandomDataOp::Builder::Builder() Status RandomDataOp::Builder::Build(std::shared_ptr *out_op) { RETURN_IF_NOT_OK(SanityCheck()); - *out_op = std::make_shared(builder_num_workers_, builder_op_connector_size_, builder_rows_per_buffer_, - builder_total_rows_, std::move(builder_data_schema_)); + *out_op = std::make_shared(builder_num_workers_, builder_op_connector_size_, builder_total_rows_, + std::move(builder_data_schema_)); return Status::OK(); } @@ -61,13 +61,11 @@ Status RandomDataOp::Builder::SanityCheck() const { } // Constructor for RandomDataOp -RandomDataOp::RandomDataOp(int32_t num_workers, int32_t op_connector_size, int64_t rows_per_buffer, int64_t total_rows, +RandomDataOp::RandomDataOp(int32_t num_workers, int32_t op_connector_size, int64_t total_rows, std::unique_ptr data_schema) : ParallelOp(num_workers, op_connector_size), - buffer_id_(0), - rows_per_buffer_(rows_per_buffer), total_rows_(total_rows), - epoch_buffers_sent_(0), + epoch_rows_sent_(0), guys_in_(0), guys_out_(num_workers_), eoe_worker_id_(0), @@ -97,8 +95,7 @@ void RandomDataOp::Print(std::ostream &out, bool show_all) const { // Call the super class for displaying any common detailed info ParallelOp::Print(out, show_all); // Then show any custom derived-internal stuff - out << "\nTotal_rows: " << total_rows_ << "\nRows per buffer: " << rows_per_buffer_ << "\nSchema:\n" - << *data_schema_ << "\n\n"; + out << "\nTotal_rows: " << total_rows_ << " \nSchema:\n" << *data_schema_ << "\n\n"; } } @@ -147,18 +144,11 @@ Status RandomDataOp::operator()() { "RandomDataOp expects total_rows < num_workers. total_row=" + std::to_string(total_rows_) + ", num_workers=" + std::to_string(num_workers_) + " ."); - // First, compute how many buffers we'll need to satisfy the total row count. - // The only reason we do this is for the purpose of throttling worker count if needed. - int64_t buffers_needed = total_rows_ / rows_per_buffer_; - if (total_rows_ % rows_per_buffer_ != 0) { - buffers_needed++; - } - - // If the amount of workers we have exceeds the number of buffers to produce, then we'll have + // If the amount of workers we have exceeds the number of rows to produce, then we'll have // idle workers doing nothing. In that case, let's throttle the worker count. - if (num_workers_ > buffers_needed) { - MS_LOG(INFO) << "RandomDataOp throttling worker count from " << num_workers_ << "to " << buffers_needed; - num_workers_ = buffers_needed; + if (num_workers_ > total_rows_) { + MS_LOG(INFO) << "RandomDataOp throttling worker count from " << num_workers_ << "to " << total_rows_; + num_workers_ = total_rows_; num_producers_ = num_workers_; guys_out_ = num_workers_; // The output connector was already created with a different worker count. We have to drop and recreate @@ -181,18 +171,15 @@ Status RandomDataOp::operator()() { currentWorker = (currentWorker + 1) % num_workers_; } - // Next, compute the total buffer count. This stat is needed during reset logic + // Next, compute the total rows count. This stat is needed during reset logic for (int32_t w = 0; w < num_workers_; w++) { - int64_t worker_buffers = 0; - worker_buffers = worker_max_rows_[w] / rows_per_buffer_; - if (worker_max_rows_[w] % rows_per_buffer_ != 0) worker_buffers++; - epoch_buffers_sent_ += worker_buffers; + epoch_rows_sent_ += worker_max_rows_[w]; } // For the connector to work, we need to target the correct worker channel for the eoe. // This will initialize it for the first one. reset() handles for the rest of the epochs. - eoe_worker_id_ = epoch_buffers_sent_ % num_workers_; - epoch_buffers_sent_++; // Add the eoe buffer to the count for subsequent epochs + eoe_worker_id_ = epoch_rows_sent_ % num_workers_; + epoch_rows_sent_++; // Add the eoe row to the count for subsequent epochs // RandomDataOp doesn't need the master thread to stay around. Kick off the workers and then master exits. RETURN_IF_NOT_OK( @@ -228,16 +215,14 @@ Status RandomDataOp::EpochSync(int32_t worker_id, bool *quitting) { // Prepare for sync all_out_.Clear(); // Always flow eoe at the end - std::unique_ptr eoe_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOE); - RETURN_IF_NOT_OK(out_connector_->Add(eoe_worker_id_, std::move(eoe_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOE(eoe_worker_id_)); // If we're done then also flow the eof if (*quitting) { // The eof needs to be sent from the next sender in the round robin, so +1 int32_t eof_worker_id = (eoe_worker_id_ + 1) % num_workers_; MS_LOG(INFO) << "RandomDataOp worker " << worker_id << " has no more epochs. sending eof as worker " << eof_worker_id; - std::unique_ptr eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - RETURN_IF_NOT_OK(out_connector_->Add(eof_worker_id, std::move(eof_buffer))); + RETURN_IF_NOT_OK(out_connector_->SendEOF(eof_worker_id)); } } @@ -290,21 +275,12 @@ Status RandomDataOp::WorkerEntry(int32_t worker_id) { RETURN_IF_NOT_OK(CreateRandomRow(worker_id, &new_row)); // Add the row to our table - new_tensor_table->push_back(std::move(new_row)); worker_rows_packed_[worker_id]++; - // If the tensor table is at capacity then it's time to send it to output - if (new_tensor_table->size() == rows_per_buffer_) { - RETURN_IF_NOT_OK(PackAndSend(worker_id, std::move(new_tensor_table))); - } - } else { - // We've reached the total row count for this worker, so it's time for epoch sync. - // There is likely some records built but not sent yet, so take care of those first - // (this buffer will be smaller than rows_per_buffer) - if (new_tensor_table != nullptr && new_tensor_table->size() > 0) { - RETURN_IF_NOT_OK(PackAndSend(worker_id, std::move(new_tensor_table))); - } + // Send new_row out + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row), worker_id)); + } else { // Now, let's enter the epoch sync RETURN_IF_NOT_OK(EpochSync(worker_id, &quitting)); } @@ -315,14 +291,6 @@ Status RandomDataOp::WorkerEntry(int32_t worker_id) { return Status::OK(); } -// A helper function to stuff the tensor table into a buffer and send it to output connector -Status RandomDataOp::PackAndSend(int32_t worker_id, std::unique_ptr in_table) { - auto new_buffer = std::make_unique(GetNextBufferId(), DataBuffer::kDeBFlagNone); - new_buffer->set_tensor_table(std::move(in_table)); - RETURN_IF_NOT_OK(out_connector_->Add(worker_id, std::move(new_buffer))); - return Status::OK(); -} - // A helper function to create random data for the row Status RandomDataOp::CreateRandomRow(int32_t worker_id, TensorRow *new_row) { if (new_row == nullptr) { @@ -385,7 +353,6 @@ Status RandomDataOp::Reset() { worker_rows_packed_[w] = 0; worker_max_rows_[w] = 0; } - buffer_id_ = 0; // Re-assign round robin row counts, starting from the worker after the one that gave // the eoe last time @@ -396,7 +363,7 @@ Status RandomDataOp::Reset() { } // Compute which worker should get the eoe for the next epoch - eoe_worker_id_ = ((epoch_buffers_sent_ % num_workers_) + eoe_worker_id_) % num_workers_; + eoe_worker_id_ = ((epoch_rows_sent_ % num_workers_) + eoe_worker_id_) % num_workers_; // Wake up the workers to get them going again in a new epoch guys_out_ = 0; diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.h index 50d4ec4988..005e940fb4 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/random_data_op.h @@ -136,12 +136,11 @@ class RandomDataOp : public ParallelOp { * @note Private constructor. Must use builder to construct. * @param num_workers - The number of workers * @param op_connector_size - The size of the output connector - * @param rows_per_buffer - The number of rows in each DataBuffer * @param data_schema - A user-provided schema * @param total_rows - The total number of rows in the dataset * @return Builder - The modified builder by reference */ - RandomDataOp(int32_t num_workers, int32_t op_connector_size, int64_t rows_per_buffer, int64_t total_rows, + RandomDataOp(int32_t num_workers, int32_t op_connector_size, int64_t total_rows, std::unique_ptr data_schema); /** @@ -213,14 +212,6 @@ class RandomDataOp : public ParallelOp { */ Status EpochSync(int32_t worker_id, bool *quitting); - /** - * A helper function to stuff the tensor table into a buffer and send it to output connector - * @param worker_id - The worker id - * @param in_table - The tensor table to pack and send - * @return Status The status code returned - */ - Status PackAndSend(int32_t worker_id, std::unique_ptr in_table); - /** * A helper function to create random data for the row * @param worker_id - The worker id @@ -240,23 +231,12 @@ class RandomDataOp : public ParallelOp { return uniDist(rand_gen_); } - /** - * A quick inline for producing the next buffer id in sequence, threadsafe - * @return - The next buffer id. - */ - inline int32_t GetNextBufferId() { - std::unique_lock lock(buffer_id_mutex_); - return ++buffer_id_; - } - // Private function for computing the assignment of the column name map. // @return - Status Status ComputeColMap() override; - int32_t buffer_id_; - int64_t rows_per_buffer_; int64_t total_rows_; - int64_t epoch_buffers_sent_; + int64_t epoch_rows_sent_; std::atomic guys_in_; std::atomic guys_out_; int32_t eoe_worker_id_; @@ -266,7 +246,6 @@ class RandomDataOp : public ParallelOp { std::mt19937 rand_gen_; WaitPost epoch_sync_wait_post_; WaitPost all_out_; - std::mutex buffer_id_mutex_; }; // class RandomDataOp } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.cc index b22a563f9f..5aa7e6210d 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.cc @@ -117,10 +117,10 @@ Status TextFileOp::Init() { return Status::OK(); } -Status TextFileOp::LoadTensor(const std::string &line, std::unique_ptr *tensor_table, int64_t row) { +Status TextFileOp::LoadTensor(const std::string &line, TensorRow *out_row) { std::shared_ptr tensor; RETURN_IF_NOT_OK(Tensor::CreateScalar(line, &tensor)); - (**tensor_table)[row][0] = std::move(tensor); + (*out_row)[0] = std::move(tensor); return Status::OK(); } @@ -130,11 +130,8 @@ Status TextFileOp::LoadFile(const std::string &file, int64_t start_offset, int64 RETURN_STATUS_UNEXPECTED("Invalid file, failed to open file: " + file); } - int64_t rows_each_buffer = 0; int64_t rows_total = 0; std::string line; - std::unique_ptr cur_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - std::unique_ptr tensor_table = std::make_unique(); while (getline(handle, line)) { if (line.empty()) { @@ -152,23 +149,10 @@ Status TextFileOp::LoadFile(const std::string &file, int64_t start_offset, int64 TensorRow tRow(1, nullptr); tRow.setPath({file}); - tensor_table->push_back(std::move(tRow)); - RETURN_IF_NOT_OK(LoadTensor(line, &tensor_table, rows_each_buffer)); - rows_each_buffer++; - rows_total++; - if (rows_each_buffer == rows_per_buffer_) { - cur_buffer->set_tensor_table(std::move(tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(cur_buffer))); + RETURN_IF_NOT_OK(LoadTensor(line, &tRow)); + RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(tRow))); - cur_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - tensor_table = std::make_unique(); - rows_each_buffer = 0; - } - } - - if (rows_each_buffer > 0) { - cur_buffer->set_tensor_table(std::move(tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(cur_buffer))); + rows_total++; } return Status::OK(); diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.h index 50f91bd684..e6f6092c67 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/text_file_op.h @@ -173,7 +173,7 @@ class TextFileOp : public NonMappableLeafOp { // @param tensor_table - the tensor table to put the parsed data in. // @param row - the id of the row filled in the tensor table. // @return Status - the error code returned. - Status LoadTensor(const std::string &line, std::unique_ptr *tensor_table, int64_t row); + Status LoadTensor(const std::string &line, TensorRow *out_row); // Reads a text file and loads the data into multiple buffers. // @param file - the file to read. diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.cc index 00a1324d77..04ccd429a1 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.cc @@ -316,8 +316,6 @@ Status TFReaderOp::LoadFile(const std::string &filename, int64_t start_offset, i int64_t rows_read = 0; int64_t rows_total = 0; - std::unique_ptr current_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - std::unique_ptr new_tensor_table = std::make_unique(); while (reader.peek() != EOF) { if (!load_jagged_connector_) { @@ -336,6 +334,10 @@ Status TFReaderOp::LoadFile(const std::string &filename, int64_t start_offset, i std::string serialized_example; serialized_example.resize(record_length); (void)reader.read(&serialized_example[0], static_cast(record_length)); + + int32_t num_columns = data_schema_->NumColumns(); + TensorRow newRow(num_columns, nullptr); + if (start_offset == kInvalidOffset || (rows_total >= start_offset && rows_total < end_offset)) { dataengine::Example tf_file; if (!tf_file.ParseFromString(serialized_example)) { @@ -343,40 +345,24 @@ Status TFReaderOp::LoadFile(const std::string &filename, int64_t start_offset, i MS_LOG(DEBUG) << errMsg + ", details of string: " << serialized_example; RETURN_STATUS_UNEXPECTED(errMsg); } - int32_t num_columns = data_schema_->NumColumns(); - TensorRow newRow(num_columns, nullptr); + std::vector file_path(num_columns, filename); newRow.setPath(file_path); - new_tensor_table->push_back(std::move(newRow)); - RETURN_IF_NOT_OK(LoadExample(&tf_file, &new_tensor_table, rows_read)); + RETURN_IF_NOT_OK(LoadExample(&tf_file, &newRow)); rows_read++; + RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(newRow))); } // ignore crc footer (void)reader.ignore(static_cast(sizeof(int32_t))); rows_total++; - - if (rows_read == rows_per_buffer_) { - current_buffer->set_tensor_table(std::move(new_tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(current_buffer))); - - current_buffer = std::make_unique(0, DataBuffer::BufferFlags::kDeBFlagNone); - new_tensor_table = std::make_unique(); - rows_read = 0; - } - } - - if (rows_read > 0) { - current_buffer->set_tensor_table(std::move(new_tensor_table)); - RETURN_IF_NOT_OK(jagged_buffer_connector_->Add(worker_id, std::move(current_buffer))); } return Status::OK(); } // Parses a single row and puts the data into a tensor table. -Status TFReaderOp::LoadExample(const dataengine::Example *tf_file, std::unique_ptr *tensor_table, - int64_t row) { +Status TFReaderOp::LoadExample(const dataengine::Example *tf_file, TensorRow *out_row) { int32_t num_columns = data_schema_->NumColumns(); for (int32_t col = 0; col < num_columns; ++col) { const ColDescriptor current_col = data_schema_->column(col); @@ -387,16 +373,15 @@ Status TFReaderOp::LoadExample(const dataengine::Example *tf_file, std::unique_p RETURN_STATUS_UNEXPECTED("Invalid parameter, column name: " + current_col.name() + " does not exist."); } const dataengine::Feature &column_values_list = iter_column->second; - RETURN_IF_NOT_OK(LoadFeature(tensor_table, column_values_list, current_col, row, col)); + RETURN_IF_NOT_OK(LoadFeature(out_row, column_values_list, current_col, col)); } return Status::OK(); } // Parses a single cell and puts the data into a tensor table. -Status TFReaderOp::LoadFeature(const std::unique_ptr *tensor_table, - const dataengine::Feature &column_values_list, const ColDescriptor ¤t_col, - int64_t row, int32_t col) { +Status TFReaderOp::LoadFeature(TensorRow *tensor_row, const dataengine::Feature &column_values_list, + const ColDescriptor ¤t_col, int32_t col) { const dataengine::Feature::KindCase column_list_type = column_values_list.kind_case(); std::unique_ptr float_array; // For staging data from protobuf deserialization const unsigned char *data_ptr = nullptr; // Generic pointer used for populating the Tensor @@ -444,7 +429,7 @@ Status TFReaderOp::LoadFeature(const std::unique_ptr *tensor_table } } - (**tensor_table)[row][col] = std::move(ts); + (*tensor_row)[col] = std::move(ts); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.h index ccb80ed4e2..56cd516cf6 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/source/tf_reader_op.h @@ -233,15 +233,15 @@ class TFReaderOp : public NonMappableLeafOp { // @param tensor_table - the tensor table to put the parsed data in. // @param row - the id of the row filled in the tensor table. // @return Status - the error code returned. - Status LoadExample(const dataengine::Example *tf_file, std::unique_ptr *tensor_table, int64_t row); + Status LoadExample(const dataengine::Example *tf_file, TensorRow *out_row); // Parses a single cell and puts the data into a tensor table. // @param tensor_table - the tensor table to put the parsed data in. // @param column_values_list - the cell to parse. // @param current_col - the column descriptor containing the expected shape and type of the data. // @return Status - the error code returned. - Status LoadFeature(const std::unique_ptr *tensor_table, const dataengine::Feature &column_values_list, - const ColDescriptor ¤t_col, int64_t row, int32_t col); + Status LoadFeature(TensorRow *tensor_row, const dataengine::Feature &column_values_list, + const ColDescriptor ¤t_col, int32_t col); // Reads values from a bytes list // @param current_col - the column descriptor containing the expected shape and type of the data. diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.cc index bea084a0d0..9c10ee7b16 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.cc @@ -20,6 +20,7 @@ #include "utils/ms_utils.h" #include "minddata/dataset/core/config_manager.h" #include "minddata/dataset/engine/data_buffer.h" +#include "minddata/dataset/engine/dataset_iterator.h" #include "minddata/dataset/engine/datasetops/take_op.h" #include "minddata/dataset/engine/db_connector.h" #include "minddata/dataset/engine/execution_tree.h" @@ -69,60 +70,32 @@ void TakeOp::Print(std::ostream &out, bool show_all) const { // Main entry point for Take Status TakeOp::operator()() { TaskManager::FindMe()->Post(); - std::unique_ptr buf; - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf)); + child_iterator_ = std::make_unique(this, 0, 0); - while (buf->eof() == false) { - if (take_count_ == max_takes_) { - // Do drain Operation - while (!buf->eoe() && !buf->eof()) { - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf)); - } - } - - // Loop until non EOE is received - if (buf->eoe()) { - UpdateRepeatAndEpochCounter(); - take_count_ = 0; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(buf))); - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf)); - continue; - } + TensorRow new_row; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); - // Get buffer and push back when take_count is still small - if (take_count_ < max_takes_) { - std::unique_ptr p_buffer; - RETURN_IF_NOT_OK(FillBuffer(&buf, &p_buffer)); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(p_buffer))); + while (!new_row.eof()) { + while (!new_row.eoe()) { + if (take_count_ < max_takes_) { + RETURN_IF_NOT_OK(out_connector_->Add(std::move(new_row))); + take_count_++; + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); + } + if (take_count_ == max_takes_) { + RETURN_IF_NOT_OK(child_iterator_->Drain()); + break; + } } - RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(&buf)); + UpdateRepeatAndEpochCounter(); + take_count_ = 0; + RETURN_IF_NOT_OK(out_connector_->SendEOE()); + RETURN_IF_NOT_OK(child_iterator_->FetchNextTensorRow(&new_row)); } take_count_ = 0; MS_LOG(DEBUG) << "Meet the end and push-back eof buffer."; - auto eof_buffer = std::make_unique(0, DataBuffer::kDeBFlagEOF); - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(eof_buffer))); - return Status::OK(); -} - -// Function FillBuffer mainly prepare the buffer for returning -Status TakeOp::FillBuffer(std::unique_ptr *buffer, std::unique_ptr *data_buffer) { - int32_t buffer_size = (*buffer)->NumRows(); - if (take_count_ + buffer_size < max_takes_) { - *data_buffer = std::move(*buffer); - take_count_ = take_count_ + buffer_size; - } else { - MS_LOG(DEBUG) << "In last buffer: Push one buffer."; - std::unique_ptr new_tensor_table = std::make_unique(); - while (take_count_ < max_takes_) { - TensorRow new_row; - RETURN_IF_NOT_OK((*buffer)->PopRow(&new_row)); - take_count_++; - new_tensor_table->push_back(new_row); - } - (*buffer)->set_tensor_table(std::move(new_tensor_table)); - *data_buffer = std::move(*buffer); - } + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.h index 77173c6ee2..331ead5c7d 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/take_op.h @@ -86,7 +86,7 @@ class TakeOp : public PipelineOp { int32_t max_takes_; // The number of takes that the user requested int32_t take_count_; // A counter for the current number of executed takes - Status FillBuffer(std::unique_ptr *buffer, std::unique_ptr *data_buffer); + std::unique_ptr child_iterator_; // An iterator for fetching. }; } // namespace dataset } // namespace mindspore diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.cc b/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.cc index 498adcdcae..d329cdd102 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.cc @@ -72,96 +72,43 @@ Status ZipOp::operator()() { // Loop until eof is true while (!eof_) { - // Create tensor table and prepare it by fetching and packing the first zipped row into it. - std::unique_ptr curr_table = std::make_unique(); - RETURN_IF_NOT_OK(prepare(curr_table.get())); + // 1 Prepare new epoch + RETURN_IF_NOT_OK(prepare()); + // 2 fetch first row + TensorRow row; + RETURN_IF_NOT_OK(getNextTensorRow(&row)); - // If an eof got picked up during the above prepare, then we're done + // If an eof got picked up, then we're done if (eof_) { break; } while (!draining_) { - // 1. If a previous loop iteration sent the current table out, then create a new one. - if (curr_table == nullptr) { - curr_table = std::make_unique(); - } - - // 2 fill the table. Note: draining mode might get turned on if any of the child inputs were done - RETURN_IF_NOT_OK(fillBuffer(curr_table.get())); - - // 3 create and update buffer and send it to the out connector - if (!curr_table->empty()) { - std::unique_ptr curr_buffer = std::make_unique(buffer_id_, DataBuffer::kDeBFlagNone); - curr_buffer->set_tensor_table(std::move(curr_table)); - MS_LOG(DEBUG) << "Zip operator finished one buffer, pushing, rows " << curr_buffer->NumRows() << ", cols " - << curr_buffer->NumCols() << ", map " << column_name_id_map_.size() << "."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(curr_buffer))); - buffer_id_++; - } + // 3 send new row to the out connector + MS_LOG(DEBUG) << "Zip operator finished one row, pushing, cols " << row.size() << ", map " + << column_name_id_map_.size() << "."; + RETURN_IF_NOT_OK(out_connector_->Add(std::move(row))); + // 4 fetch one more row + RETURN_IF_NOT_OK(getNextTensorRow(&row)); } - - // 4 handle drain state. + // 5 handle drain state. if (draining_) { MS_LOG(DEBUG) << "Zip operator is now draining child inputs."; RETURN_IF_NOT_OK(drainPipeline()); // Now that we have drained child inputs, send the eoe up. - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOE)))); + RETURN_IF_NOT_OK(out_connector_->SendEOE()); } } - // 5 handle eof - // propagate eof here. + // 6 handle eof MS_LOG(DEBUG) << "Zip operator got EOF, propagating."; - RETURN_IF_NOT_OK(out_connector_->Add(0, std::move(std::make_unique(0, DataBuffer::kDeBFlagEOF)))); + RETURN_IF_NOT_OK(out_connector_->SendEOF()); return Status::OK(); } // Handles preprocessing of the main loop, used when starting new epoch -Status ZipOp::prepare(TensorQTable *const table) { +Status ZipOp::prepare() { MS_LOG(DEBUG) << "Zip operator prepares for new epoch."; draining_ = false; - buffer_id_ = 0; - if (table == nullptr) { - return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, - "Invalid data, ZipOp prepare phase requires a tensor table, but got nullptr."); - } - // fill initial row - TensorRow new_row; - RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); - - // If the first row fetching resulted in eof, then we are done. - if (eof_) { - return Status::OK(); - } - // One of our child iterators encounter EOE. Returns and proceed with draining phase. - if (new_row.empty()) { - return Status::OK(); - } - - // Pack this first row into our tensor table - table->push_back(std::move(new_row)); - - return Status::OK(); -} - -// fillBuffer always expects a new table to fill -Status ZipOp::fillBuffer(TensorQTable *const table) { - if (table == nullptr) { - return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, - "Invalid data, ZipOp fillBuffer null table pointer."); - } - TensorRow new_row; - while (table->size() < static_cast(rows_per_buffer_)) { - RETURN_IF_NOT_OK(getNextTensorRow(&new_row)); - // Early exit the loop if we got empty row from any of our child iterations - if (new_row.empty()) { - return Status::OK(); - } - // else we got a row so pack it into the tensor table. - // Currently we don't support printing error info after zip - new_row.setPath({}); - table->push_back(std::move(new_row)); - } return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.h b/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.h index 55f57ba496..2591e5cf47 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.h +++ b/mindspore/ccsrc/minddata/dataset/engine/datasetops/zip_op.h @@ -110,11 +110,7 @@ class ZipOp : public PipelineOp { private: // Handles preprocessing of the main loop, used when starting new epoch - Status prepare(TensorQTable *const table); - - // This function calls takes a table repeatedly adds rows to it. - // @param table a table of tensors to be moved into a buffer - Status fillBuffer(TensorQTable *const table); + Status prepare(); // Special handle case where an empty row has been received from child iterator // @note - we need to drain eoe signals from all children connectors. diff --git a/mindspore/ccsrc/minddata/dataset/engine/db_connector.h b/mindspore/ccsrc/minddata/dataset/engine/db_connector.h index d55d23d6e7..d2e50733b8 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/db_connector.h +++ b/mindspore/ccsrc/minddata/dataset/engine/db_connector.h @@ -1,5 +1,5 @@ /** - * Copyright 2019 Huawei Technologies Co., Ltd + * Copyright 2019-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,7 +26,7 @@ namespace mindspore { namespace dataset { // DbConnector is a derived class from Connector with added logic to handle EOE and EOF. // The Connector class itself is responsible to ensure deterministic order on every run. -class DbConnector : public Connector> { +class DbConnector : public Connector { public: // Constructor of DbConnector // @note DbConnector will create internal N number of blocking queues, where N = nProducers. @@ -35,7 +35,7 @@ class DbConnector : public Connector> { // @param n_consumers The number of thread consuming data from this DbConnector. // @param queue_capacity The number of element (DataBuffer) for each internal queue. DbConnector(int32_t n_producers, int32_t n_consumers, int32_t queue_capacity) - : Connector>(n_producers, n_consumers, queue_capacity), end_of_file_(false) {} + : Connector(n_producers, n_consumers, queue_capacity), end_of_file_(false) {} // Destructor of DbConnector ~DbConnector() = default; @@ -44,10 +44,19 @@ class DbConnector : public Connector> { // @note The caller of this add method should use std::move to pass the ownership to DbConnector. // @param worker_id The id of a worker thread calling this method. // @param el A rvalue reference to an element to be passed/added/pushed. - Status Add(int32_t worker_id, std::unique_ptr &&el) noexcept { - return (Connector>::Push(worker_id, std::move(el))); + Status Add(TensorRow &&el, int32_t worker_id = 0) noexcept { + return (Connector::Push(worker_id, std::move(el))); } + Status SendEOE(int32_t worker_id = 0) noexcept { + TensorRow eoe = TensorRow(TensorRow::kFlagEOE); + return Add(std::move(eoe), worker_id); + } + + Status SendEOF(int32_t worker_id = 0) noexcept { + TensorRow eof = TensorRow(TensorRow::kFlagEOF); + return Add(std::move(eof), worker_id); + } // Get a unique_ptr from the DbConnector. // @note After the first EOF Buffer is encountered, subsequent pop()s will return EOF Buffer. // This will provide/propagate the EOF to all consumer threads of this Connector. @@ -56,7 +65,7 @@ class DbConnector : public Connector> { // @param worker_id The id of a worker thread calling this method. // @param result The address of a unique_ptr where the popped element will be placed. // @param retry_if_eoe A flag to allow the same thread invoke pop() again if the current pop returns eoe buffer. - Status PopWithRetry(int32_t worker_id, std::unique_ptr *result, bool retry_if_eoe = false) noexcept { + Status PopWithRetry(int32_t worker_id, TensorRow *result, bool retry_if_eoe = false) noexcept { if (result == nullptr) { return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, "[ERROR] nullptr detected when getting data from db connector"); @@ -65,21 +74,17 @@ class DbConnector : public Connector> { RETURN_IF_NOT_OK(cv_.Wait(&lk, [this, worker_id]() { return (expect_consumer_ == worker_id) || end_of_file_; })); // Once an EOF message is encountered this flag will be set and we can return early. if (end_of_file_) { - *result = std::make_unique(0, DataBuffer::kDeBFlagEOF); + *result = TensorRow(TensorRow::kFlagEOF); } else { RETURN_IF_NOT_OK(queues_[pop_from_]->PopFront(result)); - if (*result == nullptr) { - return Status(StatusCode::kMDUnexpectedError, __LINE__, __FILE__, - "[ERROR] nullptr detected when getting data from db connector"); - } // Setting the internal flag once the first EOF is encountered. - if ((*result)->eof()) { + if (result->eof()) { end_of_file_ = true; } pop_from_ = (pop_from_ + 1) % num_producers_; } // Do not increment expect_consumer_ when result is eoe and retry_if_eoe is set. - if (!((*result)->eoe() && retry_if_eoe)) { + if (!(result->eoe() && retry_if_eoe)) { expect_consumer_ = (expect_consumer_ + 1) % num_consumers_; } } diff --git a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/generator_node.cc b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/generator_node.cc index 423b213953..15af578f51 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/generator_node.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/generator_node.cc @@ -84,7 +84,7 @@ Status GeneratorNode::Build(std::vector> *const node_ // GeneratorOp's constructor takes in a prefetch_size, which isn't being set by user nor is it being used by // GeneratorOp internally. Here it is given a zero which is the default in generator builder std::shared_ptr op = std::make_shared(generator_function_, column_names_, column_types_, 0, - rows_per_buffer_, connector_que_size_, sampler_rt); + connector_que_size_, sampler_rt); // set the number of rows from source length op->SetNumRows(source_len_); diff --git a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/minddata_node.cc b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/minddata_node.cc index 2cbf833dc5..624d211aa3 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/minddata_node.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/minddata_node.cc @@ -159,13 +159,13 @@ Status MindDataNode::Build(std::vector> *const node_o // else if pass a vector to MindData(), it will be treated as specified files to be read if (search_for_pattern_) { std::vector dataset_file_vec_ = {dataset_file_}; - mindrecord_op = std::make_shared(num_workers_, rows_per_buffer_, dataset_file_vec_, - search_for_pattern_, connector_que_size_, columns_list_, operators_, - num_padded_, padded_sample_, sample_bytes_); + mindrecord_op = + std::make_shared(num_workers_, dataset_file_vec_, search_for_pattern_, connector_que_size_, + columns_list_, operators_, num_padded_, padded_sample_, sample_bytes_); } else { - mindrecord_op = std::make_shared(num_workers_, rows_per_buffer_, dataset_files_, search_for_pattern_, - connector_que_size_, columns_list_, operators_, num_padded_, - padded_sample_, sample_bytes_); + mindrecord_op = + std::make_shared(num_workers_, dataset_files_, search_for_pattern_, connector_que_size_, + columns_list_, operators_, num_padded_, padded_sample_, sample_bytes_); } RETURN_IF_NOT_OK(mindrecord_op->Init()); diff --git a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/random_node.cc b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/random_node.cc index 4bf8de29ec..4034f4e9f1 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/random_node.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/ir/datasetops/source/random_node.cc @@ -107,8 +107,7 @@ Status RandomNode::Build(std::vector> *const node_ops } std::shared_ptr op; - op = std::make_shared(num_workers_, connector_que_size_, rows_per_buffer_, total_rows_, - std::move(data_schema_)); + op = std::make_shared(num_workers_, connector_que_size_, total_rows_, std::move(data_schema_)); op->set_total_repeats(GetTotalRepeats()); op->set_num_repeats_per_epoch(GetNumRepeatsPerEpoch()); node_ops->push_back(op); diff --git a/mindspore/ccsrc/minddata/dataset/engine/jagged_connector.h b/mindspore/ccsrc/minddata/dataset/engine/jagged_connector.h index 8871d0abf4..e181c564bd 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/jagged_connector.h +++ b/mindspore/ccsrc/minddata/dataset/engine/jagged_connector.h @@ -1,5 +1,5 @@ /** - * Copyright 2019 Huawei Technologies Co., Ltd + * Copyright 2019-2021 Huawei Technologies Co., Ltd * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,10 +27,10 @@ namespace mindspore { namespace dataset { -class JaggedConnector : public Connector> { +class JaggedConnector : public Connector { public: JaggedConnector(int32_t num_producers, int32_t num_consumers, int32_t queue_capacity) - : Connector>(num_producers, num_consumers, queue_capacity) { + : Connector(num_producers, num_consumers, queue_capacity) { for (int i = 0; i < num_producers; i++) { is_queue_finished_.push_back(false); } @@ -38,11 +38,11 @@ class JaggedConnector : public Connector> { ~JaggedConnector() = default; - Status Add(int32_t worker_d, std::unique_ptr &&element) noexcept { - return Connector>::Push(worker_d, std::move(element)); + Status Add(int32_t worker_d, TensorRow &&element) noexcept { + return Connector::Push(worker_d, std::move(element)); } - Status Pop(int32_t worker_id, std::unique_ptr *result) noexcept override { + Status Pop(int32_t worker_id, TensorRow *result) noexcept override { { MS_ASSERT(worker_id < num_consumers_); std::unique_lock lock(m_); @@ -53,7 +53,7 @@ class JaggedConnector : public Connector> { } RETURN_IF_NOT_OK(queues_[pop_from_]->PopFront(result)); - if ((*result)->eoe()) { + if (result->eoe()) { is_queue_finished_[pop_from_] = true; } @@ -77,7 +77,7 @@ class JaggedConnector : public Connector> { is_queue_finished_[i] = false; } - Connector>::Reset(); + Connector::Reset(); } private: diff --git a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.cc b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.cc index ad2d7f00e8..716a04bd9e 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.cc @@ -38,7 +38,7 @@ namespace mindspore { namespace dataset { -TreeAdapter::TreeAdapter(UsageFlag usage) : usage_(usage), tree_state_(kCompileStateInit) { +TreeAdapter::TreeAdapter(UsageFlag usage) : usage_(usage), tree_state_(kCompileStateInit), launched_(false) { optimize_ = common::GetEnv("OPTIMIZE") == "true"; // Initialize profiling parameters @@ -215,44 +215,24 @@ Status TreeAdapter::GetNext(TensorRow *row) { bool isProfilingEnable = tree_->GetProfilingManager()->IsProfilingEnable(); // When cur_db_ is a nullptr, it means this is the first call to get_next, launch ExecutionTree - if (cur_db_ == nullptr) { - RETURN_IF_NOT_OK(tree_->Launch()); - // Profiling - std::shared_ptr node; - Status s = tree_->GetProfilingManager()->GetTracingNode(kDatasetIteratorTracingName, &node); - if (s.IsOk()) { - tracing_ = std::dynamic_pointer_cast(node); - cur_connector_size_ = tree_->root()->ConnectorSize(); - cur_connector_capacity_ = tree_->root()->ConnectorCapacity(); - } - RETURN_IF_NOT_OK(tree_->root()->GetNextBuffer(&cur_db_)); // first buf can't be eof or empty buf with none flag - if (cur_db_->eoe()) { // return empty tensor if 1st buf is a ctrl buf (no rows) - MS_LOG(INFO) << "End of data iteration."; - if (isProfilingEnable) { - tree_->SetEpochEnd(); - } - return Status::OK(); - } + if (!launched_) { + RETURN_IF_NOT_OK(Launch()); } - CHECK_FAIL_RETURN_UNEXPECTED(!cur_db_->eof(), "EOF has already been reached."); - - if (cur_db_->NumRows() == 0) { // a new row is fetched if cur buf is empty or a ctrl buf - RETURN_IF_NOT_OK(tree_->root()->GetNextBuffer(&cur_db_)); - if (cur_db_->eoe()) { // return empty if this new buffer is a ctrl flag - MS_LOG(INFO) << "End of data iteration."; - if (isProfilingEnable) { - tree_->SetEpochEnd(); - } - return Status::OK(); - } - if (cur_db_->eof()) { - tree_->SetFinished(); - std::string err = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs."; - RETURN_STATUS_UNEXPECTED(err); + RETURN_IF_NOT_OK(tree_->root()->GetNextRow(row)); // first buf can't be eof or empty buf with none flag + if (row->eoe()) { // return empty tensor if 1st buf is a ctrl buf (no rows) + MS_LOG(INFO) << "End of data iteration."; + if (isProfilingEnable) { + tree_->SetEpochEnd(); } + return Status::OK(); } - RETURN_IF_NOT_OK(cur_db_->PopRow(row)); + if (row->eof()) { + tree_->SetFinished(); + std::string err = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs."; + RETURN_STATUS_UNEXPECTED(err); + } + // Record profiling info if (tracing_ != nullptr) { uint64_t end_time = ProfilingTime::GetCurMilliSecond(); @@ -263,9 +243,19 @@ Status TreeAdapter::GetNext(TensorRow *row) { return Status::OK(); } -Status TreeAdapter::Launch() const { +Status TreeAdapter::Launch() { CHECK_FAIL_RETURN_UNEXPECTED(tree_ != nullptr, "Tree is a nullptr."); - return tree_->Launch(); + RETURN_IF_NOT_OK(tree_->Launch()); + launched_ = true; + // Profiling + std::shared_ptr node; + Status s = tree_->GetProfilingManager()->GetTracingNode(kDatasetIteratorTracingName, &node); + if (s.IsOk()) { + tracing_ = std::dynamic_pointer_cast(node); + cur_connector_size_ = tree_->root()->ConnectorSize(); + cur_connector_capacity_ = tree_->root()->ConnectorCapacity(); + } + return Status::OK(); } } // namespace dataset diff --git a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.h b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.h index b2d46550d9..cffc3407f6 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.h +++ b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter.h @@ -64,7 +64,7 @@ class TreeAdapter { // to be able to launch a thread. BuildAndPrepare needs to be called before this function TaskGroup *const AllTasks() const { return tree_ ? tree_->AllTasks() : nullptr; } - Status Launch() const; + Status Launch(); // Set optional optimization pass void SetOptimize(bool value) { optimize_ = value; } @@ -88,7 +88,6 @@ class TreeAdapter { // This RECURSIVE function walks the (optimized) IR tree in DFS to build its corresponding Execution tree. Status BuildExecutionTreeRecur(std::shared_ptr ir, std::shared_ptr *op); - std::unique_ptr cur_db_; std::unordered_map column_name_map_; std::shared_ptr root_ir_; std::unique_ptr tree_; // current connector capacity of root op, used for profiling @@ -98,6 +97,7 @@ class TreeAdapter { int32_t cur_connector_size_; // current connector size of root op, used for profiling int32_t cur_connector_capacity_; // current connector capacity of root op, used for profiling UsageFlag usage_; // usage of this tree adapter (type of consumer) + bool launched_; // State flags for the lifecycle of the tree enum CompileState { kCompileStateInit = 0, // The freshly initialized state diff --git a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter_lite.cc b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter_lite.cc index 1865afbd92..0bef88eb1d 100644 --- a/mindspore/ccsrc/minddata/dataset/engine/tree_adapter_lite.cc +++ b/mindspore/ccsrc/minddata/dataset/engine/tree_adapter_lite.cc @@ -56,7 +56,7 @@ Status TreeAdapterLite::BuildTree(std::shared_ptr root_ir) { Status TreeAdapterLite::GetNextRow(TensorRow *row) { RETURN_UNEXPECTED_IF_NULL(root_); - RETURN_IF_NOT_OK(root_->GetNextRow(row)); + RETURN_IF_NOT_OK(root_->GetNextRowPullMode(row)); return Status::OK(); } diff --git a/mindspore/ccsrc/minddata/dataset/include/iterator.h b/mindspore/ccsrc/minddata/dataset/include/iterator.h index 799dc1d877..9f080f3206 100644 --- a/mindspore/ccsrc/minddata/dataset/include/iterator.h +++ b/mindspore/ccsrc/minddata/dataset/include/iterator.h @@ -31,7 +31,6 @@ namespace dataset { // Forward declare class ExecutionTree; -class DatasetIterator; class DatasetOp; class Tensor; diff --git a/tests/ut/cpp/dataset/cache_op_test.cc b/tests/ut/cpp/dataset/cache_op_test.cc index 1e1b66af78..44af34c222 100644 --- a/tests/ut/cpp/dataset/cache_op_test.cc +++ b/tests/ut/cpp/dataset/cache_op_test.cc @@ -277,7 +277,7 @@ TEST_F(MindDataTestCacheOp, DISABLED_TestRandomDataCache1) { rc = CacheOp::Builder() .SetNumWorkers(5) .SetClient(myClient) - .SetRowsPerBuffer(4) + .SetRowsPerBuffer(1) .SetSampler(std::move(seq_sampler)) .Build(&myCacheOp); ASSERT_TRUE(rc.IsOk()); diff --git a/tests/ut/python/dataset/test_epoch_ctrl.py b/tests/ut/python/dataset/test_epoch_ctrl.py index c59198424f..b36b3136b4 100644 --- a/tests/ut/python/dataset/test_epoch_ctrl.py +++ b/tests/ut/python/dataset/test_epoch_ctrl.py @@ -1,4 +1,4 @@ -# Copyright 2020 Huawei Technologies Co., Ltd +# Copyright 2020-2021 Huawei Technologies Co., Ltd # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -111,7 +111,7 @@ def test_decode_op(): with pytest.raises(RuntimeError) as info: iter2.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -227,7 +227,7 @@ def test_generator_dict_4(): with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -251,7 +251,7 @@ def test_generator_dict_4_1(): with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -277,7 +277,7 @@ def test_generator_dict_4_2(): with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -309,7 +309,7 @@ def test_generator_dict_5(): # now iter1 has been exhausted, c++ pipeline has been shut down. with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -418,7 +418,7 @@ def test_generator_tuple_4(): with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -450,7 +450,7 @@ def test_generator_tuple_5(): # now iter1 has been exhausted, c++ pipeline has been shut down. with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -484,7 +484,7 @@ def test_generator_tuple_repeat_1(): # now iter1 has been exhausted, c++ pipeline has been shut down. with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value) @@ -519,7 +519,7 @@ def test_generator_tuple_repeat_repeat_1(): # now iter1 has been exhausted, c++ pipeline has been shut down. with pytest.raises(RuntimeError) as info: iter1.__next__() - err_msg = "EOF buffer encountered. Users try to fetch data beyond the specified number of epochs." + err_msg = "EOF buffer encountered. User tries to fetch data beyond the specified number of epochs." assert err_msg in str(info.value)