First drop of MapOp refactor to support device augment.

Refactored MapOp to in preparation to support device augment

Minor fix
pull/3549/head
anthonyaje 5 years ago
parent f2401cd0f9
commit a7eaf2736a

@ -66,6 +66,7 @@ add_dependencies(kernels core)
add_dependencies(engine-datasetops-source core)
add_dependencies(engine-datasetops-source-sampler core)
add_dependencies(engine-datasetops core)
add_dependencies(engine-datasetops-mapop core)
add_dependencies(engine-opt core)
add_dependencies(engine-perf core)
add_dependencies(engine-gnn core)
@ -89,6 +90,7 @@ set(submodules
$<TARGET_OBJECTS:cpp-API>
$<TARGET_OBJECTS:engine-datasetops-source>
$<TARGET_OBJECTS:engine-datasetops-source-sampler>
$<TARGET_OBJECTS:engine-datasetops-mapop>
$<TARGET_OBJECTS:engine-gnn>
$<TARGET_OBJECTS:engine-perf>
$<TARGET_OBJECTS:engine-datasetops>

@ -27,7 +27,7 @@
#include "minddata/dataset/engine/datasetops/source/voc_op.h"
// Dataset operator headers (in alphabetical order)
#include "minddata/dataset/engine/datasetops/batch_op.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/map_op/map_op.h"
#include "minddata/dataset/engine/datasetops/project_op.h"
#include "minddata/dataset/engine/datasetops/rename_op.h"
#include "minddata/dataset/engine/datasetops/repeat_op.h"
@ -537,9 +537,6 @@ std::vector<std::shared_ptr<DatasetOp>> MapDataset::Build() {
// A vector containing shared pointer to the Dataset Ops that this object will create
std::vector<std::shared_ptr<DatasetOp>> node_ops;
// Currently default is true, and this is not exposed to user.
bool perf_mode = true;
std::vector<std::shared_ptr<TensorOp>> tensor_ops;
// Build tensorOp from tensorOperation vector
@ -550,8 +547,7 @@ std::vector<std::shared_ptr<DatasetOp>> MapDataset::Build() {
// This parameter will be removed with next rebase
std::vector<std::string> col_orders;
auto map_op =
std::make_shared<MapOp>(input_columns_, output_columns_, tensor_ops, num_workers_, connector_que_size_, perf_mode);
auto map_op = std::make_shared<MapOp>(input_columns_, output_columns_, tensor_ops, num_workers_, connector_que_size_);
if (!project_columns_.empty()) {
auto project_op = std::make_shared<ProjectOp>(project_columns_);
node_ops.push_back(project_op);

@ -39,7 +39,7 @@
#include "minddata/dataset/engine/datasetops/batch_op.h"
#include "minddata/dataset/engine/datasetops/dataset_op.h"
#include "minddata/dataset/engine/datasetops/device_queue_op.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/map_op/map_op.h"
#include "minddata/dataset/engine/datasetops/project_op.h"
#include "minddata/dataset/engine/datasetops/rename_op.h"
#include "minddata/dataset/engine/datasetops/repeat_op.h"

@ -26,6 +26,9 @@ namespace dataset {
using uchar = unsigned char;
using dsize_t = int64_t;
// Target devices to perform map operation
enum class MapTargetDevice { kCpu, kGpu, kDvpp };
// Possible dataset types for holding the data and client type
enum class DatasetType { kUnknown, kArrow, kTf };

@ -19,8 +19,8 @@ target_include_directories(engine PRIVATE ${pybind11_INCLUDE_DIRS})
if (ENABLE_TDTQUE)
add_dependencies(engine engine-datasetops engine-datasetops-source engine-tdt engine-opt engine-gnn engine-perf
engine-cache-client engine-cache-server)
engine-cache-client engine-cache-server engine-datasetops-mapop)
else ()
add_dependencies(engine engine-datasetops engine-datasetops-source engine-opt engine-gnn engine-perf
engine-cache-client engine-cache-server)
engine-cache-client engine-cache-server engine-datasetops-mapop)
endif ()

@ -1,4 +1,5 @@
add_subdirectory(source)
add_subdirectory(map_op)
file(GLOB_RECURSE _CURRENT_SRC_FILES RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} "*.cc")
set_property(SOURCE ${_CURRENT_SRC_FILES} PROPERTY COMPILE_DEFINITIONS SUBMODULE_ID=mindspore::SubModuleId::SM_MD)
@ -9,7 +10,6 @@ set(DATASET_ENGINE_DATASETOPS_SRC_FILES
pipeline_op.cc
batch_op.cc
device_queue_op.cc
map_op.cc
project_op.cc
rename_op.cc
repeat_op.cc
@ -37,4 +37,3 @@ if (ENABLE_PYTHON)
endif()
add_library(engine-datasetops OBJECT ${DATASET_ENGINE_DATASETOPS_SRC_FILES})

@ -0,0 +1,10 @@
file(GLOB_RECURSE _CURRENT_SRC_FILES RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} "*.cc")
set_property(SOURCE ${_CURRENT_SRC_FILES} PROPERTY COMPILE_DEFINITIONS SUBMODULE_ID=mindspore::SubModuleId::SM_MD)
set(DATASET_ENGINE_DATASETOPS_MAPOP_SRC_FILES
map_op.cc
cpu_map_job.cc
gpu_map_job.cc
)
add_library(engine-datasetops-mapop OBJECT ${DATASET_ENGINE_DATASETOPS_MAPOP_SRC_FILES})

@ -0,0 +1,56 @@
/**
* Copyright 2020 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.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <memory>
#include <vector>
#include <utility>
#include "minddata/dataset/engine/datasetops/map_op/cpu_map_job.h"
namespace mindspore {
namespace dataset {
// Constructor
CpuMapJob::CpuMapJob() = default;
// Constructor
CpuMapJob::CpuMapJob(std::vector<std::shared_ptr<TensorOp>> operations) : MapJob(operations) {}
// Destructor
CpuMapJob::~CpuMapJob() = default;
// A function to execute a cpu map job
Status CpuMapJob::Run(std::vector<TensorRow> in, std::vector<TensorRow> *out) {
int32_t num_rows = in.size();
for (int32_t row = 0; row < num_rows; row++) {
TensorRow input_row = in[row];
TensorRow result_row;
for (size_t i = 0; i < ops_.size(); i++) {
// Call compute function for cpu
RETURN_IF_NOT_OK(ops_[i]->Compute(input_row, &result_row));
// Assign result_row to to_process for the next TensorOp processing, except for the last TensorOp in the list.
if (i + 1 < ops_.size()) {
input_row = std::move(result_row);
}
}
out->push_back(std::move(result_row));
}
return Status::OK();
}
} // namespace dataset
} // namespace mindspore

@ -0,0 +1,43 @@
/**
* Copyright 2020 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.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef DATASET_ENGINE_DATASETOPS_MAP_OP_CPU_MAP_JOB_H_
#define DATASET_ENGINE_DATASETOPS_MAP_OP_CPU_MAP_JOB_H_
#include <memory>
#include <vector>
#include "minddata/dataset/engine/datasetops/map_op/map_job.h"
namespace mindspore {
namespace dataset {
class CpuMapJob : public MapJob {
public:
// Constructor
CpuMapJob();
// Constructor
explicit CpuMapJob(std::vector<std::shared_ptr<TensorOp>> operations);
// Destructor
~CpuMapJob();
// A pure virtual run function to execute a cpu map job
Status Run(std::vector<TensorRow> in, std::vector<TensorRow> *out) override;
};
} // namespace dataset
} // namespace mindspore
#endif // DATASET_ENGINE_DATASETOPS_MAP_OP_CPU_MAP_JOB_H_

@ -0,0 +1,37 @@
/**
* Copyright 2020 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.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <memory>
#include <vector>
#include "minddata/dataset/engine/datasetops/map_op/gpu_map_job.h"
namespace mindspore {
namespace dataset {
// Constructor
GpuMapJob::GpuMapJob(std::vector<std::shared_ptr<TensorOp>> operations) : MapJob(operations) {}
// Destructor
GpuMapJob::~GpuMapJob() = default;
// A function to execute a cpu map job
Status GpuMapJob::Run(std::vector<TensorRow> in, std::vector<TensorRow> *out) {
// Do nothing for now
return Status::OK();
}
} // namespace dataset
} // namespace mindspore

@ -0,0 +1,40 @@
/**
* Copyright 2020 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.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef DATASET_ENGINE_DATASETOPS_MAP_OP_GPU_MAP_JOB_H_
#define DATASET_ENGINE_DATASETOPS_MAP_OP_GPU_MAP_JOB_H_
#include <memory>
#include <vector>
#include "minddata/dataset/engine/datasetops/map_op/map_job.h"
namespace mindspore {
namespace dataset {
class GpuMapJob : public MapJob {
public:
// Constructor
explicit GpuMapJob(std::vector<std::shared_ptr<TensorOp>> operations);
// Destructor
~GpuMapJob();
// A pure virtual run function to execute a cpu map job
Status Run(std::vector<TensorRow> in, std::vector<TensorRow> *out) override;
};
} // namespace dataset
} // namespace mindspore
#endif // DATASET_ENGINE_DATASETOPS_MAP_OP_GPU_MAP_JOB_H_

@ -0,0 +1,55 @@
/**
* Copyright 2020 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.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef DATASET_ENGINE_DATASETOPS_MAP_OP_MAP_JOB_H_
#define DATASET_ENGINE_DATASETOPS_MAP_OP_MAP_JOB_H_
#include <memory>
#include <vector>
#include "minddata/dataset/kernels/tensor_op.h"
#include "minddata/dataset/core/tensor.h"
#include "minddata/dataset/core/tensor_row.h"
#include "minddata/dataset/util/status.h"
namespace mindspore {
namespace dataset {
class MapJob {
public:
// Constructor
explicit MapJob(std::vector<std::shared_ptr<TensorOp>> operations) : ops_(operations) {}
// Constructor
MapJob() = default;
// Destructor
~MapJob() = default;
Status AddOperation(std::shared_ptr<TensorOp> operation) {
ops_.push_back(operation);
return Status::OK();
}
// A pure virtual run function to execute a particular map job
virtual Status Run(std::vector<TensorRow> in, std::vector<TensorRow> *out) = 0;
protected:
std::vector<std::shared_ptr<TensorOp>> ops_;
};
} // namespace dataset
} // namespace mindspore
#endif // DATASET_ENGINE_DATASETOPS_MAP_OP_MAP_JOB_H_

@ -24,6 +24,7 @@
#include "minddata/dataset/engine/datasetops/parallel_op.h"
#include "minddata/dataset/kernels/tensor_op.h"
#include "minddata/dataset/util/queue.h"
#include "minddata/dataset/engine/datasetops/map_op/map_job.h"
namespace mindspore {
namespace dataset {
@ -107,13 +108,6 @@ class MapOp : public ParallelOp {
return *this;
}
// Setter method.
// @return Builder setter method returns reference to the builder.
Builder &SetPerformanceMode(bool perf_mode) {
build_perf_mode_ = perf_mode;
return *this;
}
// The builder "build" method creates the final object.
// @param ptr The shared_ptr to the new MapOp object
// @return Status
@ -125,7 +119,6 @@ class MapOp : public ParallelOp {
std::vector<std::shared_ptr<TensorOp>> build_tensor_funcs_;
int32_t build_num_workers_;
int32_t build_op_connector_size_;
bool build_perf_mode_; // Default true.
// Check if the required parameters are set by the builder.
// @return Status The error code return
@ -140,8 +133,7 @@ class MapOp : public ParallelOp {
// @param num_workers The number of worker threads.
// @param op_connector_size The size of each queue in the connector.
MapOp(const std::vector<std::string> &in_col_names, const std::vector<std::string> &out_col_names,
std::vector<std::shared_ptr<TensorOp>> tensor_funcs, int32_t num_workers, int32_t op_connector_size,
bool perf_mode);
std::vector<std::shared_ptr<TensorOp>> tensor_funcs, int32_t num_workers, int32_t op_connector_size);
// Destructor
~MapOp() = default;
@ -164,6 +156,8 @@ class MapOp : public ParallelOp {
// Class functor operator () override.
// All dataset ops operate by launching a thread (see ExecutionTree). This class functor will
// provide the master loop that drives the logic for performing the work
// This main thread creates local queues, pulls databuffers from the previous
// op's Connector and distributes them to the local queues. Workers pull from the local queues.
// @return Status The error code return
Status operator()() override;
@ -189,12 +183,24 @@ class MapOp : public ParallelOp {
const auto &TFuncs() const { return tfuncs_; }
private:
// Local queues where worker threads can pop from.
// Popping directly from the Connector can block if the previous designated threads haven't pop.
// Setting the size of these queues to 0 is essentially the same as pulling directly from Connector.
QueueList<std::unique_ptr<DataBuffer>> local_queues_;
// 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 {
std::vector<std::shared_ptr<MapJob>> jobs;
std::unique_ptr<DataBuffer> databuffer;
};
// A helper function to create jobs for workers.
Status GenerateWorkerJob(const std::unique_ptr<MapWorkerJob> *worker_job);
// Static variables to be ready by worker threads, no modification and readonly
// 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<DataBuffer> *db,
std::vector<std::shared_ptr<MapJob>> *job_list);
// Local queues where worker threads get a job from
QueueList<std::unique_ptr<MapWorkerJob>> local_queues_;
// Tensorops to be read and applied by worker threads
std::vector<std::shared_ptr<TensorOp>> tfuncs_;
// Variable to store the column name that the tensorOps are consuming
@ -209,13 +215,6 @@ class MapOp : public ParallelOp {
// Indices of the columns to process.
std::vector<size_t> to_process_indices_;
// Performance mode is when the main thread creates local queues, pulls databuffers from the previous
// op's Connector and distributes them to the local queues. Workers pull from the local queues.
// If this flag is false, each worker pulls directly from the Connector. This use less resources
// (thread and memory), but when the computation cost is heavy (e.g. DecodeOp) and fluctuating, it can
// cause additional blocking because pop calls to Connector from the threads are synchronized to enforce the order.
bool perf_mode_;
// 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
@ -224,25 +223,12 @@ class MapOp : public ParallelOp {
// @return Status The error code return
Status WorkerEntry(int32_t worker_id) override; // In: workerId assigned by tree_
// Private helper function for getting the next buffer
// When PerformanceMode is enabled, workers pop from the local queue.
// Otherwise, workers pop from the first child output Connector.
// @param p_buffer - the buffer to return
// @return Status return code
Status FetchNextBuffer(std::unique_ptr<DataBuffer> *p_buffer, int32_t worker_id) {
if (perf_mode_) {
RETURN_IF_NOT_OK(local_queues_[worker_id]->PopFront(p_buffer));
} else {
RETURN_IF_NOT_OK(child_[0]->GetNextBuffer(p_buffer, worker_id));
}
return Status::OK();
}
// Private function for worker thread to perform TensorOp's compute function and get the result.
// @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(DataBuffer *in_buffer, TensorQTable *new_tensor_table,
const std::vector<std::shared_ptr<MapJob>> &job_list);
// Private function that create the final column name to index mapping and
// get indices of the columns this mapop does not use.

@ -17,7 +17,7 @@
#include <memory>
#include "minddata/dataset/engine/opt/optional/tensor_op_fusion_pass.h"
#include "minddata/dataset/kernels/image/decode_op.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/map_op/map_op.h"
#include "minddata/dataset/kernels/image/random_crop_decode_resize_op.h"
namespace mindspore {

@ -24,7 +24,7 @@
#include "minddata/dataset/engine/datasetops/dataset_op.h"
#include "minddata/dataset/engine/datasetops/device_queue_op.h"
#include "minddata/dataset/engine/datasetops/epoch_ctrl_op.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/map_op/map_op.h"
#include "minddata/dataset/engine/datasetops/project_op.h"
#include "minddata/dataset/engine/datasetops/rename_op.h"
#include "minddata/dataset/engine/datasetops/repeat_op.h"

@ -645,16 +645,14 @@ TEST_F(MindDataTestMapOp, ImageFolder_Decode_Repeat_Resize) {
map_decode_builder.SetInColNames({"image"})
.SetOutColNames({})
.SetTensorFuncs(func_list)
.SetNumWorkers(14)
.SetPerformanceMode(false);
.SetNumWorkers(14);
rc = map_decode_builder.Build(&map_decode_map);
EXPECT_TRUE(rc.IsOk());
map_resize_builder.SetInColNames({"image"})
.SetOutColNames({})
.SetTensorFuncs(func_list2)
.SetNumWorkers(15)
.SetPerformanceMode(false);
.SetNumWorkers(15);
rc = map_resize_builder.Build(&map_resize_op);
EXPECT_TRUE(rc.IsOk());
@ -739,5 +737,3 @@ TEST_F(MindDataTestMapOp, ImageFolder_Decode_Repeat_Resize_NoInputColumns) {
}
EXPECT_TRUE(i == 88);
}

@ -19,7 +19,6 @@
#include <string>
#include "minddata/dataset/core/client.h"
#include "minddata/dataset/core/constants.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/rename_op.h"
#include "common/common.h"
#include "common/utils.h"

@ -23,7 +23,6 @@
#include <thread>
#include "minddata/dataset/core/client.h"
#include "minddata/dataset/core/constants.h"
#include "minddata/dataset/engine/datasetops/map_op.h"
#include "minddata/dataset/engine/datasetops/zip_op.h"
#include "minddata/dataset/core/tensor.h"
#include "minddata/dataset/core/config_manager.h"

Loading…
Cancel
Save