From 9d24726370d130403b45cf19daedd7aa73dd5560 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 25 Feb 2021 12:30:48 +0000 Subject: [PATCH 001/106] graph engine demo --- .../fluid/distributed/service/CMakeLists.txt | 9 +- .../distributed/service/graph_brpc_client.cc | 137 +++++++ .../distributed/service/graph_brpc_client.h | 56 +++ .../distributed/service/graph_brpc_server.cc | 303 ++++++++++++++++ .../distributed/service/graph_brpc_server.h | 119 +++++++ .../distributed/service/graph_service.cc | 171 +++++++++ paddle/fluid/distributed/service/ps_client.cc | 5 +- paddle/fluid/distributed/service/ps_client.h | 27 +- .../fluid/distributed/service/sendrecv.proto | 2 + paddle/fluid/distributed/table/CMakeLists.txt | 8 +- .../distributed/table/common_graph_table.cc | 205 +++++++++++ .../distributed/table/common_graph_table.h | 123 +++++++ paddle/fluid/distributed/table/graph_node.cc | 54 +++ paddle/fluid/distributed/table/graph_node.h | 79 ++++ paddle/fluid/distributed/table/table.cc | 4 + paddle/fluid/distributed/table/table.h | 16 +- .../distributed/table/weighted_sampler.cc | 80 +++++ .../distributed/table/weighted_sampler.h | 54 +++ paddle/fluid/distributed/test/CMakeLists.txt | 5 +- .../fluid/distributed/test/graph_node_test.cc | 337 ++++++++++++++++++ paddle/fluid/pybind/CMakeLists.txt | 27 +- paddle/fluid/pybind/graph_service.cc | 31 ++ paddle/fluid/pybind/graph_service.h | 28 ++ paddle/fluid/pybind/pybind.cc | 97 ++--- 24 files changed, 1875 insertions(+), 102 deletions(-) create mode 100644 paddle/fluid/distributed/service/graph_brpc_client.cc create mode 100644 paddle/fluid/distributed/service/graph_brpc_client.h create mode 100644 paddle/fluid/distributed/service/graph_brpc_server.cc create mode 100644 paddle/fluid/distributed/service/graph_brpc_server.h create mode 100644 paddle/fluid/distributed/service/graph_service.cc create mode 100644 paddle/fluid/distributed/table/common_graph_table.cc create mode 100644 paddle/fluid/distributed/table/common_graph_table.h create mode 100644 paddle/fluid/distributed/table/graph_node.cc create mode 100644 paddle/fluid/distributed/table/graph_node.h create mode 100644 paddle/fluid/distributed/table/weighted_sampler.cc create mode 100644 paddle/fluid/distributed/table/weighted_sampler.h create mode 100644 paddle/fluid/distributed/test/graph_node_test.cc create mode 100644 paddle/fluid/pybind/graph_service.cc create mode 100644 paddle/fluid/pybind/graph_service.h diff --git a/paddle/fluid/distributed/service/CMakeLists.txt b/paddle/fluid/distributed/service/CMakeLists.txt index bb3f6f1174da9..459e194f76dae 100644 --- a/paddle/fluid/distributed/service/CMakeLists.txt +++ b/paddle/fluid/distributed/service/CMakeLists.txt @@ -24,11 +24,11 @@ set_source_files_properties(heter_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUT set_source_files_properties(client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +set_source_files_properties(graph_brpc_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +set_source_files_properties(graph_brpc_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) - -cc_library(downpour_server SRCS brpc_ps_server.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) -cc_library(downpour_client SRCS brpc_ps_client.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) +cc_library(downpour_server SRCS brpc_ps_server.cc graph_brpc_server.cc DEPS boost eigen3 table ${RPC_DEPS}) +cc_library(downpour_client SRCS brpc_ps_client.cc graph_brpc_client.cc DEPS boost eigen3 table ${RPC_DEPS}) cc_library(client SRCS ps_client.cc DEPS downpour_client boost ${RPC_DEPS}) cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) @@ -36,5 +36,6 @@ cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) cc_library(communicator SRCS communicator.cc DEPS scope client boost table math_function selected_rows_functor ${RPC_DEPS}) cc_library(ps_service SRCS service.cc DEPS communicator client server boost ${RPC_DEPS}) +cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) cc_library(heter_server SRCS heter_server.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) cc_library(heter_client SRCS heter_client.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc new file mode 100644 index 0000000000000..d0a068dcd63b0 --- /dev/null +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -0,0 +1,137 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include "Eigen/Dense" + +#include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "paddle/fluid/distributed/table/table.h" +#include "paddle/fluid/framework/archive.h" +#include "paddle/fluid/string/string_helper.h" +namespace paddle { +namespace distributed { +int GraphBrpcClient::get_server_index_by_id(uint64_t id) { + int shard_num = get_shard_num(); + size_t server_size = get_server_nums(); + int shard_per_server = shard_num % server_size == 0 + ? shard_num / server_size + : shard_num / server_size + 1; + return id % shard_num / shard_per_server; +} +// char* &buffer,int &actual_size +std::future GraphBrpcClient::sample(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, + int sample_size, + vector &res) { + int server_index = get_server_index_by_id(node_id); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(0, PS_GRAPH_SAMPLE); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + int start = 0; + while (start < bytes_size) { + GraphNode node; + node.recover_from_buffer(buffer + start); + start += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); + + return fut; +} + +std::future GraphBrpcClient::pull_graph_list(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, + int start, int size, + vector &res) { + int server_index = get_server_index_by_id(node_id); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + int index = 0; + while (index < bytes_size) { + GraphNode node; + node.recover_from_buffer(buffer + index); + index += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, sizeof(int)); + closure->request(0)->add_params((char *)&start, sizeof(int)); + closure->request(0)->add_params((char *)&size, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); + return fut; +} +int32_t GraphBrpcClient::initialize() { + set_shard_num(_config.shard_num()); + return BrpcPsClient::initialize(); +} +} +} \ No newline at end of file diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h new file mode 100644 index 0000000000000..9dac994f296e7 --- /dev/null +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -0,0 +1,56 @@ +// Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include +#include +#include + +#include "brpc/channel.h" +#include "brpc/controller.h" +#include "brpc/server.h" +#include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/ps_client.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/table.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/tensor_util.h" + +namespace paddle { +namespace distributed { + +class GraphBrpcClient : public BrpcPsClient { + public: + GraphBrpcClient() {} + virtual ~GraphBrpcClient() {} + virtual std::future sample(uint32_t table_id, uint64_t node_id, + GraphNodeType type, int sample_size, + vector &res); + virtual std::future pull_graph_list(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, int start, + int end, vector &res); + virtual int32_t initialize(); + int get_shard_num() { return shard_num; } + void set_shard_num(int shard_num) { this->shard_num = shard_num; } + int get_server_index_by_id(uint64_t id); + + private: + int shard_num; +}; + +} // namespace distributed +} // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc new file mode 100644 index 0000000000000..9e54b932b5301 --- /dev/null +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -0,0 +1,303 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/service/graph_brpc_server.h" +#include "paddle/fluid/distributed/service/brpc_ps_server.h" + +#include // NOLINT +#include "Eigen/Dense" +#include "butil/endpoint.h" +#include "iomanip" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/table.h" +#include "paddle/fluid/framework/archive.h" +#include "paddle/fluid/platform/profiler.h" +namespace paddle { +namespace distributed { + +int32_t GraphBrpcServer::initialize() { + auto &service_config = _config.downpour_server_param().service_param(); + if (!service_config.has_service_class()) { + LOG(ERROR) << "miss service_class in ServerServiceParameter"; + return -1; + } + auto *service = + CREATE_PSCORE_CLASS(PsBaseService, service_config.service_class()); + if (service == NULL) { + LOG(ERROR) << "service is unregistered, service_name:" + << service_config.service_class(); + return -1; + } + + _service.reset(service); + if (service->configure(this) != 0 || service->initialize() != 0) { + LOG(ERROR) << "service initialize failed, service_name:" + << service_config.service_class(); + return -1; + } + if (_server.AddService(service, brpc::SERVER_DOESNT_OWN_SERVICE) != 0) { + LOG(ERROR) << "service add to brpc failed, service:" + << service_config.service_class(); + return -1; + } + return 0; +} + +uint64_t GraphBrpcServer::start(const std::string &ip, uint32_t port) { + std::unique_lock lock(mutex_); + + std::string ip_port = ip + ":" + std::to_string(port); + VLOG(3) << "server of rank " << _rank << " starts at " << ip_port; + brpc::ServerOptions options; + + int num_threads = std::thread::hardware_concurrency(); + auto trainers = _environment->get_trainers(); + options.num_threads = trainers > num_threads ? trainers : num_threads; + + if (_server.Start(ip_port.c_str(), &options) != 0) { + LOG(ERROR) << "GraphBrpcServer start failed, ip_port=" << ip_port; + return 0; + } + VLOG(0) << "GraphBrpcServer::start registe_ps_server"; + _environment->registe_ps_server(ip, port, _rank); + VLOG(0) << "GraphBrpcServer::start wait"; + cv_.wait(lock, [&] { return stoped_; }); + + PSHost host; + host.ip = ip; + host.port = port; + host.rank = _rank; + VLOG(0) << "GraphBrpcServer::start return host.rank"; + return host.rank; +} + +int32_t GraphBrpcServer::port() { return _server.listen_address().port; } + +int32_t GraphBrpcService::initialize() { + _is_initialize_shard_info = false; + _service_handler_map[PS_STOP_SERVER] = &GraphBrpcService::stop_server; + _service_handler_map[PS_LOAD_ONE_TABLE] = &GraphBrpcService::load_one_table; + _service_handler_map[PS_LOAD_ALL_TABLE] = &GraphBrpcService::load_all_table; + + _service_handler_map[PS_PRINT_TABLE_STAT] = + &GraphBrpcService::print_table_stat; + _service_handler_map[PS_BARRIER] = &GraphBrpcService::barrier; + _service_handler_map[PS_START_PROFILER] = &GraphBrpcService::start_profiler; + _service_handler_map[PS_STOP_PROFILER] = &GraphBrpcService::stop_profiler; + + _service_handler_map[PS_PULL_GRAPH_LIST] = &GraphBrpcService::pull_graph_list; + _service_handler_map[PS_GRAPH_SAMPLE] = + &GraphBrpcService::graph_random_sample; + + // shard初始化,server启动后才可从env获取到server_list的shard信息 + initialize_shard_info(); + + return 0; +} + +#define CHECK_TABLE_EXIST(table, request, response) \ + if (table == NULL) { \ + std::string err_msg("table not found with table_id:"); \ + err_msg.append(std::to_string(request.table_id())); \ + set_response_code(response, -1, err_msg.c_str()); \ + return -1; \ + } + +int32_t GraphBrpcService::initialize_shard_info() { + if (!_is_initialize_shard_info) { + std::lock_guard guard(_initialize_shard_mutex); + if (_is_initialize_shard_info) { + return 0; + } + size_t shard_num = _server->environment()->get_ps_servers().size(); + auto &table_map = *(_server->table()); + for (auto itr : table_map) { + itr.second->set_shard(_rank, shard_num); + } + _is_initialize_shard_info = true; + } + return 0; +} + +void GraphBrpcService::service(google::protobuf::RpcController *cntl_base, + const PsRequestMessage *request, + PsResponseMessage *response, + google::protobuf::Closure *done) { + brpc::ClosureGuard done_guard(done); + std::string log_label("ReceiveCmd-"); + if (!request->has_table_id()) { + set_response_code(*response, -1, "PsRequestMessage.tabel_id is required"); + return; + } + + response->set_err_code(0); + response->set_err_msg(""); + auto *table = _server->table(request->table_id()); + brpc::Controller *cntl = static_cast(cntl_base); + auto itr = _service_handler_map.find(request->cmd_id()); + if (itr == _service_handler_map.end()) { + std::string err_msg( + "undefined cmd_id, should match PsCmdID in ps.proto, cmd_id:"); + err_msg.append(std::to_string(request->cmd_id())); + set_response_code(*response, -1, err_msg.c_str()); + return; + } + serviceFunc handler_func = itr->second; + int service_ret = (this->*handler_func)(table, *request, *response, cntl); + if (service_ret != 0) { + response->set_err_code(service_ret); + response->set_err_msg("server internal error"); + } +} + +int32_t GraphBrpcService::barrier(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + + if (request.params_size() < 1) { + set_response_code(response, -1, + "PsRequestMessage.params is requeired at " + "least 1 for num of sparse_key"); + return 0; + } + + auto trainer_id = request.client_id(); + auto barrier_type = request.params(0); + table->barrier(trainer_id, barrier_type); + return 0; +} + +int32_t GraphBrpcService::print_table_stat(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + std::pair ret = table->print_table_stat(); + paddle::framework::BinaryArchive ar; + ar << ret.first << ret.second; + std::string table_info(ar.Buffer(), ar.Length()); + response.set_data(table_info); + + return 0; +} + +int32_t GraphBrpcService::load_one_table(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 2) { + set_response_code( + response, -1, + "PsRequestMessage.datas is requeired at least 2 for path & load_param"); + return -1; + } + if (table->load(request.params(0), request.params(1)) != 0) { + set_response_code(response, -1, "table load failed"); + return -1; + } + return 0; +} + +int32_t GraphBrpcService::load_all_table(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + auto &table_map = *(_server->table()); + for (auto &itr : table_map) { + if (load_one_table(itr.second.get(), request, response, cntl) != 0) { + LOG(ERROR) << "load table[" << itr.first << "] failed"; + return -1; + } + } + return 0; +} + +int32_t GraphBrpcService::stop_server(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + auto *p_server = _server; + std::thread t_stop([p_server]() { + p_server->stop(); + LOG(INFO) << "Server Stoped"; + }); + t_stop.detach(); + return 0; +} + +int32_t GraphBrpcService::stop_profiler(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + platform::DisableProfiler(platform::EventSortingKey::kDefault, + string::Sprintf("server_%s_profile", _rank)); + return 0; +} + +int32_t GraphBrpcService::start_profiler(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + platform::EnableProfiler(platform::ProfilerState::kCPU); + return 0; +} + +int32_t GraphBrpcService::pull_graph_list(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 2) { + set_response_code(response, -1, + "pull_graph_list request requires at least 2 arguments"); + return 0; + } + uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); + std::string type_str = request.params(1); + GraphNodeType type = GraphNode::get_graph_node_type(type_str); + int start = *(int *)(request.params(2).c_str()); + int size = *(int *)(request.params(3).c_str()); + std::vector res_data; + char *buffer; + int actual_size; + table->pull_graph_list(node_id, type, start, size, buffer, actual_size); + cntl->response_attachment().append(buffer, actual_size); + return 0; +} +int32_t GraphBrpcService::graph_random_sample(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 3) { + set_response_code( + response, -1, + "graph_random_sample request requires at least 3 arguments"); + return 0; + } + uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); + std::string type_str = request.params(1); + int sample_size = *(uint64_t *)(request.params(2).c_str()); + GraphNodeType type = GraphNode::get_graph_node_type(type_str); + char *buffer; + int actual_size; + table->random_sample(node_id, type, sample_size, buffer, actual_size); + cntl->response_attachment().append(buffer, actual_size); + return 0; +} + +} // namespace distributed +} // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h new file mode 100644 index 0000000000000..b4056692f2ccf --- /dev/null +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -0,0 +1,119 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include "brpc/channel.h" +#include "brpc/controller.h" +#include "brpc/server.h" + +#include +#include +#include "paddle/fluid/distributed/service/brpc_ps_server.h" +#include "paddle/fluid/distributed/service/server.h" + +namespace paddle { +namespace distributed { +class GraphBrpcServer : public PSServer { + public: + GraphBrpcServer() {} + virtual ~GraphBrpcServer() {} + virtual uint64_t start(const std::string &ip, uint32_t port); + virtual int32_t stop() { + std::unique_lock lock(mutex_); + stoped_ = true; + cv_.notify_all(); + + _server.Stop(1000); + _server.Join(); + return 0; + } + virtual int32_t port(); + + private: + virtual int32_t initialize(); + + mutable std::mutex mutex_; + std::condition_variable cv_; + bool stoped_ = false; + brpc::Server _server; + std::shared_ptr _service; + std::vector> _pserver_channels; +}; +// class GraphBrpcServer : public GraphBrpcServer { +// public: +// GraphBrpcServer() {} +// virtual ~GraphBrpcServer() {} + +// private: +// virtual int32_t initialize(); +// }; +class GraphBrpcService; + +typedef int32_t (GraphBrpcService::*serviceFunc)( + Table *table, const PsRequestMessage &request, PsResponseMessage &response, + brpc::Controller *cntl); + +class GraphBrpcService : public PsBaseService { + public: + virtual int32_t initialize() override; + + virtual void service(::google::protobuf::RpcController *controller, + const PsRequestMessage *request, + PsResponseMessage *response, + ::google::protobuf::Closure *done) override; + + protected: + std::unordered_map _service_handler_map; + int32_t initialize_shard_info(); + int32_t pull_graph_list(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t graph_random_sample(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); + int32_t barrier(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t load_one_table(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t load_all_table(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t stop_server(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t start_profiler(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t stop_profiler(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + + int32_t print_table_stat(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + + private: + bool _is_initialize_shard_info; + std::mutex _initialize_shard_mutex; + std::unordered_map _msg_handler_map; + std::vector _ori_values; +}; +// class GraphBrpcService : public BrpcPsService { +// public: +// virtual int32_t initialize() override; +// int32_t pull_graph_list(Table *table, const PsRequestMessage &request, +// PsResponseMessage &response, brpc::Controller +// *cntl); +// int32_t graph_random_sample(Table *table, const PsRequestMessage &request, +// PsResponseMessage &response, brpc::Controller +// *cntl); +// }; + +} // namespace distributed +} // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_service.cc b/paddle/fluid/distributed/service/graph_service.cc new file mode 100644 index 0000000000000..2b3abe66b29f2 --- /dev/null +++ b/paddle/fluid/distributed/service/graph_service.cc @@ -0,0 +1,171 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +// #include +// #include // NOLINT +// #include // NOLINT +// #include +// #include +// #include +// #include "google/protobuf/text_format.h" + +// #include "gtest/gtest.h" +// #include "paddle/fluid/framework/lod_tensor.h" +// #include "paddle/fluid/framework/scope.h" +// #include "paddle/fluid/framework/tensor_util.h" +// #include "paddle/fluid/framework/variable.h" + +#include "paddle/fluid/distributed/ps.pb.h" +// #include "paddle/fluid/distributed/service/brpc_ps_client.h" +// #include "paddle/fluid/distributed/service/brpc_ps_server.h" +// #include "paddle/fluid/distributed/service/env.h" +// #include "paddle/fluid/distributed/service/ps_client.h" +// #include "paddle/fluid/distributed/service/sendrecv.pb.h" +// #include "paddle/fluid/distributed/service/service.h" +#include "paddle/fluid/distributed/table/graph_node.h" +// #include "paddle/fluid/framework/program_desc.h" +// #include "paddle/fluid/operators/math/math_function.h" +// #include "paddle/fluid/platform/place.h" +// #include "paddle/fluid/string/printf.h" +namespace paddle { +namespace distributed { +class graph_service { + std::vector keys; + std::vector server_list, port_list, host_sign_list; + int server_size, shard_num; + + public: + // void GetDownpourSparseTableProto( + // ::paddle::distributed::TableParameter* sparse_table_proto) { + // sparse_table_proto->set_table_id(0); + // sparse_table_proto->set_table_class("GraphTable"); + // sparse_table_proto->set_shard_num(256); + // sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); + // ::paddle::distributed::TableAccessorParameter* accessor_proto = + // sparse_table_proto->mutable_accessor(); + // ::paddle::distributed::CommonAccessorParameter* common_proto = + // sparse_table_proto->mutable_common(); + + // accessor_proto->set_accessor_class("CommMergeAccessor"); + // accessor_proto->set_fea_dim(0); + // accessor_proto->set_embedx_dim(10); + + // common_proto->set_name("sgd"); + // common_proto->set_table_name("MergedDense"); + // common_proto->set_trainer_num(1); + // common_proto->set_sync(false); + // common_proto->add_params("Param"); + // common_proto->add_dims(10); + // common_proto->add_initializers("uniform_random&0&-1.0&1.0"); + // common_proto->add_params("LearningRate"); + // common_proto->add_dims(1); + // common_proto->add_initializers("fill_constant&1.0"); + // } + + // ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { + // ::paddle::distributed::PSParameter worker_fleet_desc; + // worker_fleet_desc.set_shard_num(shard_num); + // ::paddle::distributed::WorkerParameter* worker_proto = + // worker_fleet_desc.mutable_worker_param(); + + // ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = + // worker_proto->mutable_downpour_worker_param(); + + // ::paddle::distributed::TableParameter* worker_sparse_table_proto = + // downpour_worker_proto->add_downpour_table_param(); + // GetDownpourSparseTableProto(worker_sparse_table_proto); + + // ::paddle::distributed::ServerParameter* server_proto = + // worker_fleet_desc.mutable_server_param(); + // ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + // server_proto->mutable_downpour_server_param(); + // ::paddle::distributed::ServerServiceParameter* server_service_proto = + // downpour_server_proto->mutable_service_param(); + // server_service_proto->set_service_class("GraphBrpcService"); + // server_service_proto->set_server_class("GraphBrpcServer"); + // server_service_proto->set_client_class("GraphBrpcClient"); + // server_service_proto->set_start_server_port(0); + // server_service_proto->set_server_thread_num(12); + + // ::paddle::distributed::TableParameter* server_sparse_table_proto = + // downpour_server_proto->add_downpour_table_param(); + // GetDownpourSparseTableProto(server_sparse_table_proto); + + // return worker_fleet_desc; + // } + void set_server_size(int server_size) { this->server_size = server_size; } + int get_server_size(int server_size) { return server_size; } + std::vector split(std::string &str, const char pattern) { + std::vector res; + std::stringstream input(str); + std::string temp; + while (std::getline(input, temp, pattern)) { + res.push_back(temp); + } + return res; + } + void start_client() { + // framework::Scope client_scope; + // platform::CPUPlace place; + // InitTensorsOnClient(&client_scope, &place, 100); + // std::map> + // dense_regions; + // dense_regions.insert( + // std::pair>(0, + // {})); + // auto regions = dense_regions[0]; + // framework::Variable* var = client_scope.FindVar("x"); + // framework::LoDTensor* tensor = var->GetMutable(); + + // RunClient(dense_regions); + // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + // paddle::distributed::PaddlePSEnvironment _ps_env; + // auto servers_ = host_sign_list_.size(); + // _ps_env = paddle::distributed::PaddlePSEnvironment(); + // _ps_env.set_ps_servers(&host_sign_list_, servers_); + // worker_ptr_ = std::shared_ptr( + // paddle::distributed::PSClientFactory::create(worker_proto)); + // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); + } + void init(std::string ips_str, int shard_num) { + std::istringstream stream(ips_str); + std::string ip, port; + server_size = 0; + std::vector ips_list = split(ips_str, ';'); + int index = 0; + for (auto ips : ips_list) { + auto ip_and_port = split(ips, ':'); + server_list.push_back(ip_and_port[0]); + port_list.push_back(ip_and_port[1]); + // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], + // ip_and_port[1], index); + // host_sign_list_.push_back(ph_host.serialize_to_string()); + index++; + } + start_client(); + } + + public: + void set_keys(std::vector keys) { // just for test + this->keys = keys; + } + std::vector get_keys(int start, int size) { // just for test + return std::vector(keys.begin() + start, keys.begin() + start + size); + } +}; +} +} \ No newline at end of file diff --git a/paddle/fluid/distributed/service/ps_client.cc b/paddle/fluid/distributed/service/ps_client.cc index 095b5dee0b28e..902c72c30ac5d 100644 --- a/paddle/fluid/distributed/service/ps_client.cc +++ b/paddle/fluid/distributed/service/ps_client.cc @@ -13,14 +13,17 @@ // limitations under the License. #include "paddle/fluid/distributed/service/ps_client.h" +#include +#include "brpc/server.h" #include "glog/logging.h" #include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { REGISTER_PSCORE_CLASS(PSClient, BrpcPsClient); - +REGISTER_PSCORE_CLASS(PSClient, GraphBrpcClient); int32_t PSClient::configure( const PSParameter &config, const std::map> ®ions, diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 50f5802c63a25..5a060f1dfa51d 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -24,16 +24,11 @@ #include "paddle/fluid/distributed/service/env.h" #include "paddle/fluid/distributed/service/sendrecv.pb.h" #include "paddle/fluid/distributed/table/accessor.h" +#include "paddle/fluid/distributed/table/graph_node.h" namespace paddle { namespace distributed { -class PSEnvironment; -class PsRequestMessage; -class PsResponseMessage; -class ValueAccessor; -struct Region; - using paddle::distributed::PsRequestMessage; using paddle::distributed::PsResponseMessage; @@ -75,8 +70,7 @@ class PSClient { int max_retry) = 0; // 触发table数据退场 - virtual std::future shrink(uint32_t table_id, - const std::string threshold) = 0; + virtual std::future shrink(uint32_t table_id) = 0; // 全量table进行数据load virtual std::future load(const std::string &epoch, @@ -160,6 +154,23 @@ class PSClient { promise.set_value(-1); return fut; } + // virtual std::future sample(uint32_t table_id, uint64_t node_id, + // GraphNodeType type, int sample_size,std::vector &res){ + // LOG(FATAL) << "Did not implement"; + // std::promise promise; + // std::future fut = promise.get_future(); + // promise.set_value(-1); + // return fut; + // } + // virtual std::future pull_graph_list(uint32_t table_id, uint64_t + // node_id, GraphNodeType type,int start, int size, std::vector + // &res){ + // LOG(FATAL) << "Did not implement"; + // std::promise promise; + // std::future fut = promise.get_future(); + // promise.set_value(-1); + // return fut; + // } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 6250f84c98754..52c2d7104b544 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -48,6 +48,8 @@ enum PsCmdID { PS_START_PROFILER = 27; PS_STOP_PROFILER = 28; PS_PUSH_GLOBAL_STEP = 29; + PS_PULL_GRAPH_LIST = 30; + PS_GRAPH_SAMPLE = 31; } message PsRequestMessage { diff --git a/paddle/fluid/distributed/table/CMakeLists.txt b/paddle/fluid/distributed/table/CMakeLists.txt index 1e98e193d54ae..779012ad1a8f2 100644 --- a/paddle/fluid/distributed/table/CMakeLists.txt +++ b/paddle/fluid/distributed/table/CMakeLists.txt @@ -1,13 +1,17 @@ set_property(GLOBAL PROPERTY TABLE_DEPS string_helper) get_property(TABLE_DEPS GLOBAL PROPERTY TABLE_DEPS) - +set_source_files_properties(weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(WeightedSampler SRCS weighted_sampler.cc) +set_source_files_properties(graph_node.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(graph_node SRCS graph_node.cc DEPS WeightedSampler) set_source_files_properties(common_dense_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(common_sparse_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(sparse_geo_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(barrier_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +set_source_files_properties(common_graph_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(common_table SRCS common_sparse_table.cc common_dense_table.cc sparse_geo_table.cc barrier_table.cc DEPS ${TABLE_DEPS} device_context string_helper simple_threadpool xxhash generator) +cc_library(common_table SRCS common_sparse_table.cc common_dense_table.cc sparse_geo_table.cc barrier_table.cc common_graph_table.cc DEPS ${TABLE_DEPS} graph_node device_context string_helper simple_threadpool xxhash generator) set_source_files_properties(tensor_accessor.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(tensor_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc new file mode 100644 index 0000000000000..1146b39fecd40 --- /dev/null +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -0,0 +1,205 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/common_graph_table.h" +#include +#include +#include "paddle/fluid/distributed/common/utils.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/string/printf.h" +#include "paddle/fluid/string/string_helper.h" +namespace paddle { +namespace distributed { +size_t GraphShard::get_size() { + size_t res = 0; + for (int i = 0; i < bucket_size; i++) { + res += bucket[i].size(); + } + return res; +} +list::iterator GraphShard::add_node(GraphNode *node) { + if (node_location.find({node->get_id(), node->get_graph_node_type()}) != + node_location.end()) + return node_location.find({node->get_id(), node->get_graph_node_type()}) + ->second; + int index = node->get_id() % shard_num % bucket_size; + list::iterator iter = + bucket[index].insert(bucket[index].end(), node); + node_location[{node->get_id(), node->get_graph_node_type()}] = iter; + return iter; +} +void GraphShard::add_neighboor(uint64_t id, GraphNodeType type, + GraphEdge *edge) { + (*add_node(new GraphNode(id, type, std::string(""))))->add_edge(edge); +} +GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { + if (node_location.find({id, type}) == node_location.end()) return NULL; + return *(node_location[{id, type}]); +} +int32_t GraphTable::load(const std::string &path, const std::string ¶m) { + rwlock_->WRLock(); + auto paths = paddle::string::split_string(path, ";"); + for (auto path : paths) { + std::ifstream file(path); + std::string line; + while (std::getline(file, line)) { + auto values = paddle::string::split_string(line, "\t"); + if (values.size() < 3) continue; + auto id = std::stoull(values[0]); + size_t shard_id = id % _shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + VLOG(0) << "will not load " << id << " from " << path + << ", please check id distribution"; + continue; + } + size_t index = shard_id - shard_start; + GraphNodeType type = GraphNode::get_graph_node_type(values[1]); + shards[index].add_node(new GraphNode(id, type, values[2])); + for (size_t i = 3; i < values.size(); i++) { + auto edge_arr = + paddle::string::split_string(values[i], ";"); + if (edge_arr.size() < 3) { + auto edge_id = std::stoull(edge_arr[0]); + GraphNodeType edge_type = GraphNode::get_graph_node_type(edge_arr[1]); + auto weight = std::stod(edge_arr[2]); + GraphEdge *edge = new GraphEdge(edge_id, edge_type, weight); + shards[index].add_neighboor(id, type, edge); + } + } + } + } + rwlock_->UNLock(); + return 0; +} +GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { + rwlock_->WRLock(); + size_t shard_id = id % _shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + return NULL; + } + size_t index = shard_id - shard_start; + + GraphNode *node = shards[index].find_node(id, type); + rwlock_->UNLock(); + return node; +} +int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, + int sample_size, char *&buffer, + int &actual_size) { + rwlock_->RDLock(); + GraphNode *node = find_node(node_id, type); + if (node == NULL) { + actual_size = 0; + rwlock_->UNLock(); + return 0; + } + vector res = node->sample_k(sample_size); + vector node_list; + int total_size = 0; + for (auto x : res) { + GraphNode temp; + temp.set_id(x->id); + temp.set_graph_node_type(x->type); + total_size += temp.get_size(); + node_list.push_back(temp); + } + buffer = new char[total_size]; + int index = 0; + for (auto x : node_list) { + x.to_buffer(buffer + index); + index += x.get_size(); + } + actual_size = total_size; + rwlock_->UNLock(); + return 0; +} +int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, + int &actual_size) { + rwlock_->RDLock(); + int thread_index = 0; + if (start < 0) start = 0; + int size = 0, cur_size; + if (total_size <= 0) { + actual_size = 0; + return 0; + } + std::vector>> tasks; + for (size_t i = 0; i < shards.size(); i++) { + cur_size = shards[i].get_size(); + if (size + cur_size <= start) { + size += cur_size; + continue; + } + if (size + cur_size - start >= total_size) { + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this, i, start, size, total_size]() -> vector { + return this->shards[i].get_batch(start - size, total_size); + })); + thread_index++; + break; + } else { + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this, i, start, size, total_size, + cur_size]() -> vector { + return this->shards[i].get_batch(start - size, + size + cur_size - start); + })); + total_size -= size + cur_size - start; + size += cur_size; + start = size; + } + } + for (size_t i = 0; i < tasks.size(); ++i) { + tasks[i].wait(); + } + size = 0; + vector> res; + for (size_t i = 0; i < tasks.size(); i++) { + res.push_back(tasks[i].get()); + for (size_t j = 0; j < res.back().size(); j++) { + size += res.back()[j]->get_size(); + } + } + buffer = new char[size]; + int index = 0; + for (size_t i = 0; i < res.size(); i++) { + for (size_t j = 0; j < res[i].size(); j++) { + res[i][j]->to_buffer(buffer + index); + index += res[i][j]->get_size(); + } + } + actual_size = size; + rwlock_->UNLock(); + return 0; +} +int32_t GraphTable::initialize() { + _shards_task_pool.resize(task_pool_size_); + for (size_t i = 0; i < _shards_task_pool.size(); ++i) { + _shards_task_pool[i].reset(new ::ThreadPool(1)); + } + server_num = _shard_num; + /* + _shard_num is actually server number here + when a server initialize its tables, it sets tables' _shard_num to server_num, + and _shard_idx to server + rank + */ + _shard_num = _config.shard_num(); + shard_num_per_table = sparse_local_shard_num(_shard_num, server_num); + shard_start = _shard_idx * shard_num_per_table; + shards.resize(shard_num_per_table); + return 0; +} +} +}; \ No newline at end of file diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h new file mode 100644 index 0000000000000..9adf155c50a27 --- /dev/null +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -0,0 +1,123 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include +#include +#include +#include +#include +#include // NOLINT +#include +#include +#include +#include +#include "Eigen/Dense" +#include "paddle/fluid/distributed/table/accessor.h" +#include "paddle/fluid/distributed/table/common_table.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/framework/rw_lock.h" +#include "paddle/fluid/string/string_helper.h" +namespace paddle { +namespace distributed { +struct pair_hash { + inline size_t operator()(const pair &p) const { + return p.first * 10007 + int(p.second); + } +}; +class GraphShard { + public: + static int bucket_low_bound; + static int gcd(int s, int t) { + if (s % t == 0) return t; + return gcd(t, s % t); + } + size_t get_size(); + GraphShard() {} + GraphShard(int shard_num) { + this->shard_num = shard_num; + bucket_size = init_bucket_size(shard_num); + bucket.resize(bucket_size); + } + vector get_batch(int start, int total_size) { + if (start < 0) start = 0; + int size = 0, cur_size; + vector res; + if (total_size <= 0) return res; + for (int i = 0; i < bucket_size; i++) { + cur_size = bucket[i].size(); + if (size + cur_size <= start) { + size += cur_size; + continue; + } + int read = 0; + list::iterator iter = bucket[i].begin(); + while (size + read < start) { + iter++; + read++; + } + read = 0; + while (iter != bucket[i].end() && read < total_size) { + res.push_back(*iter); + iter++; + read++; + } + if (read == total_size) break; + size += cur_size; + start = size; + total_size -= read; + } + return res; + } + int init_bucket_size(int shard_num) { + for (int i = bucket_low_bound;; i++) { + if (gcd(i, shard_num) == 1) return i; + } + return -1; + } + list::iterator add_node(GraphNode *node); + GraphNode *find_node(uint64_t id, GraphNodeType type); + void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); + + private: + unordered_map, list::iterator, + pair_hash> + node_location; + int bucket_size, shard_num; + vector> bucket; +}; +class GraphTable : public SparseTable { + public: + GraphTable() {} + virtual ~GraphTable() {} + virtual int32_t pull_graph_list(int start, int size, char *&buffer, + int &actual_size); + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, + int sampe_size, char *&buffer, + int &actual_size); + virtual int32_t initialize(); + int32_t load(const std::string &path, const std::string ¶m); + GraphNode *find_node(uint64_t id, GraphNodeType type); + + protected: + vector shards; + unordered_set id_set; + size_t shard_start, shard_end, server_num, shard_num_per_table; + std::unique_ptr rwlock_{nullptr}; + const int task_pool_size_ = 7; + std::vector> _shards_task_pool; +}; +} +}; diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc new file mode 100644 index 0000000000000..eb6ca577100a9 --- /dev/null +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -0,0 +1,54 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/graph_node.h" +#include +#include "paddle/fluid/distributed/table/weighted_sampler.h" +namespace paddle { +namespace distributed { +int GraphNode::enum_size = sizeof(int); +int GraphNode::id_size = sizeof(uint64_t); +int GraphNode::double_size = sizeof(double); +int GraphNode::int_size = sizeof(int); +int GraphNode::get_size() { + return feature.size() + id_size + enum_size + int_size; +} +void GraphNode::build_sampler() { + sampler = new WeightedSampler(); + GraphEdge** arr = edges.data(); + sampler->build((WeightedObject**)arr, 0, edges.size()); +} +void GraphNode::to_buffer(char* buffer) { + int size = get_size(); + memcpy(buffer, &size, int_size); + memcpy(buffer + int_size, feature.c_str(), feature.size()); + memcpy(buffer + int_size + feature.size(), &id, id_size); + int int_state = type; + memcpy(buffer + int_size + feature.size() + id_size, &int_state, enum_size); +} +void GraphNode::recover_from_buffer(char* buffer) { + int size; + memcpy(&size, buffer, int_size); + int feature_size = size - id_size - enum_size - int_size; + char str[feature_size + 1]; + memcpy(str, buffer + int_size, feature_size); + str[feature_size] = '\0'; + feature = str; + memcpy(&id, buffer + int_size + feature_size, id_size); + int int_state; + memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); + type = GraphNodeType(int_state); +} +} +} \ No newline at end of file diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h new file mode 100644 index 0000000000000..de17ab883e897 --- /dev/null +++ b/paddle/fluid/distributed/table/graph_node.h @@ -0,0 +1,79 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once +#include +#include "paddle/fluid/distributed/table/weighted_sampler.h" +namespace paddle { +namespace distributed { +enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; +class GraphEdge : public WeightedObject { + public: + double weight; + uint64_t id; + GraphNodeType type; + GraphEdge() {} + GraphEdge(uint64_t id, GraphNodeType type, double weight) + : weight(weight), id(id), type(type) {} +}; +class GraphNode { + public: + GraphNode() { sampler = NULL; } + GraphNode(uint64_t id, GraphNodeType type, std::string feature) + : id(id), type(type), feature(feature), sampler(NULL) {} + virtual ~GraphNode() {} + static int enum_size, id_size, int_size, double_size; + uint64_t get_id() { return id; } + void set_id(uint64_t id) { this->id = id; } + GraphNodeType get_graph_node_type() { return type; } + void set_graph_node_type(GraphNodeType type) { this->type = type; } + void set_feature(std::string feature) { this->feature = feature; } + std::string get_feature() { return feature; } + virtual int get_size(); + virtual void build_sampler(); + virtual void to_buffer(char *buffer); + virtual void recover_from_buffer(char *buffer); + virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } + static GraphNodeType get_graph_node_type(std::string &str) { + GraphNodeType type; + if (str == "user") + type = GraphNodeType::user; + else if (str == "item") + type = GraphNodeType::item; + else if (str == "query") + type = GraphNodeType::query; + else + type = GraphNodeType::unknown; + return type; + } + std::vector sample_k(int k) { + std::vector v; + if (sampler != NULL) { + auto res = sampler->sample_k(k); + for (auto x : res) { + v.push_back((GraphEdge *)x); + } + } + return v; + } + + protected: + uint64_t id; + GraphNodeType type; + std::string feature; + WeightedSampler *sampler; + std::vector edges; +}; +} +} diff --git a/paddle/fluid/distributed/table/table.cc b/paddle/fluid/distributed/table/table.cc index dfaaa6ffc12c2..dbdac32e3597c 100644 --- a/paddle/fluid/distributed/table/table.cc +++ b/paddle/fluid/distributed/table/table.cc @@ -14,10 +14,13 @@ #include "paddle/fluid/distributed/table/table.h" +#include +#include #include "glog/logging.h" #include "paddle/fluid/distributed/common/registerer.h" #include "paddle/fluid/distributed/table/common_dense_table.h" +#include "paddle/fluid/distributed/table/common_graph_table.h" #include "paddle/fluid/distributed/table/common_sparse_table.h" #include "paddle/fluid/distributed/table/sparse_geo_table.h" #include "paddle/fluid/distributed/table/tensor_accessor.h" @@ -75,5 +78,6 @@ int32_t Table::initialize_accessor() { _value_accesor.reset(accessor); return 0; } + } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 65c99d2bbd40d..77f5c3557f6cf 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -21,6 +21,7 @@ #include #include #include "paddle/fluid/distributed/table/accessor.h" +#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/platform/device_context.h" @@ -86,11 +87,23 @@ class Table { return 0; } + // only for graph table + virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, + int total_size, char *&buffer, + int &actual_size) { + return 0; + } + // only for graph table + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, + int sampe_size, char *&buffer, + int &actual_size) { + return 0; + } virtual int32_t pour() { return 0; } virtual void clear() = 0; virtual int32_t flush() = 0; - virtual int32_t shrink(const std::string ¶m) = 0; + virtual int32_t shrink() = 0; //指定加载路径 virtual int32_t load(const std::string &path, @@ -141,5 +154,6 @@ class TableManager { TableManager() {} ~TableManager() {} }; + } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc new file mode 100644 index 0000000000000..a2d2cfcaab8df --- /dev/null +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -0,0 +1,80 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/weighted_sampler.h" +namespace paddle { +namespace distributed { +void WeightedSampler::build(WeightedObject **v, int start, int end) { + count = 0; + if (start + 1 == end) { + left = right = NULL; + weight = v[start]->get_weight(); + object = v[start]; + count = 1; + + } else { + left = new WeightedSampler(); + right = new WeightedSampler(); + left->build(v, start, start + (end - start) / 2); + right->build(v, start + (end - start) / 2, end); + weight = left->weight + right->weight; + count = left->count + right->count; + } +} +vector WeightedSampler::sample_k(int k) { + if (k > count) { + k = count; + } + vector sample_result; + double subtract; + unordered_map subtract_weight_map; + unordered_map subtract_count_map; + while (k--) { + double query_weight = rand() % 100000 / 100000.0; + query_weight *= weight - subtract_weight_map[this]; + sample_result.push_back(sample(query_weight, subtract_weight_map, + subtract_count_map, subtract)); + } + return sample_result; +} +WeightedObject *WeightedSampler::sample( + double query_weight, + unordered_map &subtract_weight_map, + unordered_map &subtract_count_map, + double &subtract) { + if (left == NULL) { + subtract_weight_map[this] = weight; + subtract = weight; + subtract_count_map[this] = 1; + return object; + } + int left_count = left->count - subtract_count_map[left]; + int right_count = right->count - subtract_count_map[right]; + double left_subtract = subtract_weight_map[left]; + WeightedObject *return_id; + if (right_count == 0 || + left_count > 0 && left->weight - left_subtract >= query_weight) { + return_id = left->sample(query_weight, subtract_weight_map, + subtract_count_map, subtract); + } else { + return_id = + right->sample(query_weight - (left->weight - left_subtract), + subtract_weight_map, subtract_count_map, subtract); + } + subtract_weight_map[this] += subtract; + subtract_count_map[this]++; + return return_id; +} +} +} diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h new file mode 100644 index 0000000000000..2c9b428ffd90a --- /dev/null +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -0,0 +1,54 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once +#include +#include +#include +using namespace std; +namespace paddle { +namespace distributed { +class WeightedObject { + public: + WeightedObject() {} + virtual ~WeightedObject() {} + virtual unsigned long long get_id() { return id; } + virtual double get_weight() { return weight; } + + virtual void set_id(unsigned long long id) { this->id = id; } + virtual void set_weight(double weight) { this->weight = weight; } + + private: + unsigned long long id; + double weight; +}; + +class WeightedSampler { + public: + WeightedSampler *left, *right; + WeightedObject *object; + int count; + double weight; + void build(WeightedObject **v, int start, int end); + vector sample_k(int k); + + private: + WeightedObject *sample( + double query_weight, + unordered_map &subtract_weight_map, + unordered_map &subtract_count_map, + double &subtract); +}; +} +} diff --git a/paddle/fluid/distributed/test/CMakeLists.txt b/paddle/fluid/distributed/test/CMakeLists.txt index adedd049023da..fb352fdc7f509 100644 --- a/paddle/fluid/distributed/test/CMakeLists.txt +++ b/paddle/fluid/distributed/test/CMakeLists.txt @@ -11,7 +11,10 @@ set_source_files_properties(brpc_service_dense_sgd_test.cc PROPERTIES COMPILE_FL cc_test(brpc_service_dense_sgd_test SRCS brpc_service_dense_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) set_source_files_properties(brpc_service_sparse_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) +cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) + +set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +#cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc new file mode 100644 index 0000000000000..cfb9411636106 --- /dev/null +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -0,0 +1,337 @@ +/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + +#include +#include // NOLINT +#include +#include +#include +#include // NOLINT +#include "google/protobuf/text_format.h" + +#include "gtest/gtest.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/tensor_util.h" +#include "paddle/fluid/framework/variable.h" + +#include "paddle/fluid/distributed/ps.pb.h" +#include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/brpc_ps_server.h" +#include "paddle/fluid/distributed/service/env.h" +#include "paddle/fluid/distributed/service/ps_client.h" +#include "paddle/fluid/distributed/service/sendrecv.pb.h" +#include "paddle/fluid/distributed/service/service.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/framework/program_desc.h" +#include "paddle/fluid/operators/math/math_function.h" +#include "paddle/fluid/platform/place.h" +#include "paddle/fluid/string/printf.h" + +namespace framework = paddle::framework; +namespace platform = paddle::platform; +namespace operators = paddle::operators; +namespace math = paddle::operators::math; +namespace memory = paddle::memory; +namespace distributed = paddle::distributed; + +void testGraphToBuffer(); +std::string nodes[] = { + std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), + std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; +char file_name[] = "nodes.txt"; +void prepare_file(char file_name[]) { + ofstream ofile; + ofile.open(file_name); + for (auto x : nodes) { + ofile << x << endl; + } + ofile.close(); +} +distributed::GraphNodeType get_graph_node_type(std::string& str) { + distributed::GraphNodeType type; + if (str == "user") + type = distributed::GraphNodeType::user; + else if (str == "item") + type = distributed::GraphNodeType::item; + else if (str == "query") + type = distributed::GraphNodeType::query; + else + type = distributed::GraphNodeType::unknown; + return type; +} +void GetDownpourSparseTableProto( + ::paddle::distributed::TableParameter* sparse_table_proto) { + sparse_table_proto->set_table_id(0); + sparse_table_proto->set_table_class("GraphTable"); + sparse_table_proto->set_shard_num(256); + sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); + ::paddle::distributed::TableAccessorParameter* accessor_proto = + sparse_table_proto->mutable_accessor(); + ::paddle::distributed::CommonAccessorParameter* common_proto = + sparse_table_proto->mutable_common(); +} + +distributed::GraphNodeType get_graph_node_type(std::string str) { + distributed::GraphNodeType type; + if (str == "user") + type = distributed::GraphNodeType::user; + else if (str == "item") + type = distributed::GraphNodeType::item; + else if (str == "query") + type = distributed::GraphNodeType::query; + else + type = distributed::GraphNodeType::unknown; + return type; +} + +::paddle::distributed::PSParameter GetServerProto() { + // Generate server proto desc + ::paddle::distributed::PSParameter server_fleet_desc; + server_fleet_desc.set_shard_num(127); + ::paddle::distributed::ServerParameter* server_proto = + server_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); + + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto); + return server_fleet_desc; +} + +::paddle::distributed::PSParameter GetWorkerProto() { + ::paddle::distributed::PSParameter worker_fleet_desc; + worker_fleet_desc.set_shard_num(127); + ::paddle::distributed::WorkerParameter* worker_proto = + worker_fleet_desc.mutable_worker_param(); + + ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = + worker_proto->mutable_downpour_worker_param(); + + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto); + + ::paddle::distributed::ServerParameter* server_proto = + worker_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); + + ::paddle::distributed::TableParameter* server_sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(server_sparse_table_proto); + + return worker_fleet_desc; +} + +/*-------------------------------------------------------------------------*/ + +std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; +uint32_t port_ = 4209, port2 = 4210; + +std::vector host_sign_list_; + +std::shared_ptr pserver_ptr_, pserver_ptr2; + +std::shared_ptr worker_ptr_; + +void RunServer() { + LOG(INFO) << "init first server"; + ::paddle::distributed::PSParameter server_proto = GetServerProto(); + + auto _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&host_sign_list_, 2); // test + pserver_ptr_ = std::shared_ptr( + paddle::distributed::PSServerFactory::create(server_proto)); + std::vector empty_vec; + framework::ProgramDesc empty_prog; + empty_vec.push_back(empty_prog); + pserver_ptr_->configure(server_proto, _ps_env, 0, empty_vec); + LOG(INFO) << "first server, run start(ip,port)"; + pserver_ptr_->start(ip_, port_); + LOG(INFO) << "init first server Done"; +} + +void RunServer2() { + LOG(INFO) << "init second server"; + ::paddle::distributed::PSParameter server_proto2 = GetServerProto(); + + auto _ps_env2 = paddle::distributed::PaddlePSEnvironment(); + _ps_env2.set_ps_servers(&host_sign_list_, 2); // test + pserver_ptr2 = std::shared_ptr( + paddle::distributed::PSServerFactory::create(server_proto2)); + std::vector empty_vec2; + framework::ProgramDesc empty_prog2; + empty_vec2.push_back(empty_prog2); + pserver_ptr2->configure(server_proto2, _ps_env2, 1, empty_vec2); + pserver_ptr2->start(ip2, port2); +} + +void RunClient(std::map>& + dense_regions) { + ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + paddle::distributed::PaddlePSEnvironment _ps_env; + auto servers_ = host_sign_list_.size(); + _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&host_sign_list_, servers_); + worker_ptr_ = std::shared_ptr( + paddle::distributed::PSClientFactory::create(worker_proto)); + worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); +} + +void RunBrpcPushSparse() { + setenv("http_proxy", "", 1); + setenv("https_proxy", "", 1); + prepare_file(file_name); + auto ph_host = paddle::distributed::PSHost(ip_, port_, 0); + host_sign_list_.push_back(ph_host.serialize_to_string()); + + // test-start + auto ph_host2 = paddle::distributed::PSHost(ip2, port2, 1); + host_sign_list_.push_back(ph_host2.serialize_to_string()); + // test-end + // Srart Server + std::thread server_thread(RunServer); + std::thread server_thread2(RunServer2); + sleep(1); + + std::map> dense_regions; + dense_regions.insert( + std::pair>(0, {})); + auto regions = dense_regions[0]; + + RunClient(dense_regions); + + /*-----------------------Test Server Init----------------------------------*/ + LOG(INFO) << "Run pull_sparse_param"; + auto pull_status = + worker_ptr_->load(0, std::string(file_name), std::string("")); + pull_status.wait(); + vector v; + pull_status = + worker_ptr_->sample(0, 37, get_graph_node_type(string("user")), v); + pull_status.wait(); + for (auto g : v) { + std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + } + /*-----------------------Test Push Param----------------------------------*/ + + // LOG(INFO) << "Run push_sparse_param"; + // paddle::distributed::DownpourBrpcClosure* closure_push_param = + // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { + // int ret = 0; + // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; + // for (size_t i = 0; i < 2; ++i) { + // LOG(INFO) << "check response" <check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { + // ret = -1; + // break; + // } + // VLOG(0)<check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_PARAM); + // } + // closure->set_promise_value(ret); + // }); + // VLOG(0)<<"begin to push_sparse_param"; + // auto push_status = worker_ptr_->push_sparse_param( + // 0, fea_keys.data(), (const float**)fea_value_ptr.data(), + // fea_keys.size(), + // closure_push_param); + // push_status.wait(); + + // auto pull_param_status = worker_ptr_->pull_sparse( + // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); + // pull_param_status.wait(); + + // for (size_t idx = 0; idx < tensor->numel(); ++idx) { + // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); + // } + // LOG(INFO) << "first stage done"; + // /*-----------------------Test Push + // Grad----------------------------------*/ + + // paddle::distributed::DownpourBrpcClosure* closure_push_grad = + // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { + // int ret = 0; + // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; + // for (size_t i = 0; i < 2; ++i) { + // if (closure->check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { + // ret = -1; + // break; + // } + // } + // closure->set_promise_value(ret); + // }); + + // LOG(INFO) << "Run pull_sparse_grad"; + // std::vector push_g_vec; + // for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { + // push_g_vec.push_back(tensor->data() + i * 10); + // } + // auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( + // 0, fea_keys.data(), (const float**)push_g_vec.data(), + // fea_keys.size(), + // closure_push_grad); + // push_grad_status.wait(); + + // auto pull_update_status = worker_ptr_->pull_sparse( + // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); + // pull_update_status.wait(); + + // for (size_t idx = 0; idx < tensor->numel(); ++idx) { + // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); + // } + + LOG(INFO) << "Run stop_server"; + worker_ptr_->stop_server(); + LOG(INFO) << "Run finalize_worker"; + worker_ptr_->finalize_worker(); + server_thread.join(); + server_thread2.join(); + testGraphToBuffer(); +} + +void testGraphToBuffer() { + ::paddle::distributed::GraphNode s, s1; + s.set_feature("hhhh"); + s.set_id(65); + s.set_graph_node_type(::paddle::distributed::GraphNodeType(0)); + int size = s.get_size(); + char str[size]; + s.to_buffer(str); + s1.recover_from_buffer(str); + ASSERT_EQ(s.get_id(), s1.get_id()); + ASSERT_EQ((int)s.get_graph_node_type(), (int)s1.get_graph_node_type()); + VLOG(0) << s.get_feature(); + VLOG(0) << s1.get_feature(); +} +TEST(RunBrpcPushSparse, Run) { RunBrpcPushSparse(); } diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index bdf018db6f883..1877f9f971181 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -1,23 +1,13 @@ set(PYBIND_DEPS pybind python proto_desc memory executor fleet_wrapper box_wrapper prune feed_fetch_method pass_builder parallel_executor profiler layer tracer engine scope_pool analysis_predictor imperative_profiler imperative_flag save_load_util dlpack_tensor device_context - gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper custom_operator) - -if (WITH_GPU) - set(PYBIND_DEPS ${PYBIND_DEPS} dynload_cuda) - set(PYBIND_DEPS ${PYBIND_DEPS} cuda_device_guard) -endif() + gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper) if (WITH_NCCL) set(PYBIND_DEPS ${PYBIND_DEPS} nccl_wrapper) set(PYBIND_DEPS ${PYBIND_DEPS} reducer) endif() -if (WITH_XPU_BKCL) - set(PYBIND_DEPS ${PYBIND_DEPS} reducer) - set(PYBIND_DEPS ${PYBIND_DEPS} bkcl_context) -endif() - if(NOT WIN32) set(PYBIND_DEPS ${PYBIND_DEPS} data_loader) set(PYBIND_DEPS ${PYBIND_DEPS} mmap_allocator) @@ -41,6 +31,7 @@ set(PYBIND_SRCS heter_wrapper_py.cc ps_gpu_wrapper_py.cc gloo_wrapper_py.cc + graph_service.cc box_helper_py.cc data_set_py.cc imperative.cc @@ -85,10 +76,6 @@ if(WITH_PYTHON) list(APPEND OP_FUNCTION_GENERETOR_DEPS nccl_context) endif(WITH_NCCL) - if(WITH_XPU_BKCL) - list(APPEND OP_FUNCTION_GENERETOR_DEPS bkcl_context) - endif(WITH_XPU_BKCL) - add_executable(op_function_generator op_function_generator.cc) target_link_libraries(op_function_generator ${OP_FUNCTION_GENERETOR_DEPS}) get_property (os_dependency_modules GLOBAL PROPERTY OS_DEPENDENCY_MODULES) @@ -164,11 +151,11 @@ if(WITH_PYTHON) endif(WITH_MKLDNN) endif(WIN32) - if(WITH_ROCM_PLATFORM) - cc_library(paddle_pybind SHARED + if(WITH_AMD_GPU) + hip_library(paddle_pybind SHARED SRCS ${PYBIND_SRCS} - DEPS ${PYBIND_DEPS} - ${GLOB_OP_LIB} ${GLOB_OPERATOR_DEPS}) + DEPS ARCHIVE_START ${PYBIND_DEPS} + ${GLOB_OP_LIB} ${GLOB_OPERATOR_DEPS} ARCHIVE_END) else() cc_library(paddle_pybind SHARED SRCS ${PYBIND_SRCS} @@ -177,7 +164,7 @@ if(WITH_PYTHON) if(NOT APPLE AND NOT WIN32) target_link_libraries(paddle_pybind rt) endif(NOT APPLE AND NOT WIN32) - endif(WITH_ROCM_PLATFORM) + endif(WITH_AMD_GPU) get_property (os_dependency_modules GLOBAL PROPERTY OS_DEPENDENCY_MODULES) target_link_libraries(paddle_pybind ${os_dependency_modules}) diff --git a/paddle/fluid/pybind/graph_service.cc b/paddle/fluid/pybind/graph_service.cc new file mode 100644 index 0000000000000..e7172773b4dae --- /dev/null +++ b/paddle/fluid/pybind/graph_service.cc @@ -0,0 +1,31 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/pybind/graph_service.h" +#include "paddle/fluid/distributed/service/graph_service.cc" +namespace py = pybind11; +using paddle::distributed::graph_service; + +namespace paddle { +namespace pybind { + +void BindGraphService(py::module* m) { + py::class_(*m, "graph_service") + .def(py::init<>()) + .def("set_keys", &graph_service::set_keys) + .def("get_keys", &graph_service::get_keys); +} + +} // namespace pybind +} \ No newline at end of file diff --git a/paddle/fluid/pybind/graph_service.h b/paddle/fluid/pybind/graph_service.h new file mode 100644 index 0000000000000..560a01642a729 --- /dev/null +++ b/paddle/fluid/pybind/graph_service.h @@ -0,0 +1,28 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include "pybind11/pybind11.h" +#include "pybind11/stl.h" + +namespace py = pybind11; + +namespace paddle { +namespace pybind { + +void BindGraphService(py::module* m); + +} // namespace pybind +} // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 750fb6e225803..5e14ee0bb8b81 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -24,7 +24,6 @@ limitations under the License. */ #include #include -#include "paddle/fluid/framework/custom_operator.h" #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/framework/executor.h" #include "paddle/fluid/framework/feed_fetch_method.h" @@ -59,7 +58,6 @@ limitations under the License. */ #include "paddle/fluid/operators/py_func_op.h" #include "paddle/fluid/platform/cpu_helper.h" #include "paddle/fluid/platform/cpu_info.h" -#include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/dynload/dynamic_loader.h" #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/init.h" @@ -115,6 +113,7 @@ limitations under the License. */ #include "paddle/fluid/pybind/fleet_py.h" #endif +#include "paddle/fluid/pybind/graph_service.h" #include "pybind11/stl.h" DECLARE_bool(use_mkldnn); @@ -162,17 +161,6 @@ bool SupportsBfloat16() { #endif } -bool SupportsBfloat16FastPerformance() { -#ifndef PADDLE_WITH_MKLDNN - return false; -#else - if (platform::MayIUse(platform::cpu_isa_t::avx512_bf16)) - return true; - else - return false; -#endif -} - bool IsCompiledWithBrpc() { #ifndef PADDLE_WITH_DISTRIBUTE return false; @@ -398,7 +386,7 @@ PYBIND11_MODULE(core_noavx, m) { PyCapsule_GetPointer(dltensor->ptr(), "dltensor")); PyCapsule_SetName(dltensor->ptr(), "used_dltensor"); DLTensor dl = dmt->dl_tensor; - framework::Tensor tensor; + Tensor tensor; if (dl.ctx.device_type == kDLCPU) { paddle::framework::TensorFromDLPack(dl, &tensor); @@ -536,80 +524,77 @@ PYBIND11_MODULE(core_noavx, m) { BindImperative(&m); - py::class_(m, "Tensor", py::buffer_protocol()) - .def("__array__", - [](framework::Tensor &self) { return TensorToPyArray(self); }) + py::class_(m, "Tensor", py::buffer_protocol()) + .def("__array__", [](Tensor &self) { return TensorToPyArray(self); }) .def("_is_initialized", - [](const framework::Tensor &self) { return self.IsInitialized(); }) + [](const Tensor &self) { return self.IsInitialized(); }) .def("_get_dims", - [](const framework::Tensor &self) { return vectorize(self.dims()); }) + [](const Tensor &self) { return vectorize(self.dims()); }) .def("_set_dims", - [](framework::Tensor &self, const std::vector &dim) { + [](Tensor &self, const std::vector &dim) { self.Resize(make_ddim(dim)); }) .def("_set_layout", - [](framework::Tensor &self, const std::string &layout) { + [](Tensor &self, const std::string &layout) { self.set_layout(StringToDataLayout(layout)); }) .def("_alloc_float", - [](framework::Tensor &self, paddle::platform::CUDAPlace &place) { + [](Tensor &self, paddle::platform::CUDAPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](framework::Tensor &self, paddle::platform::XPUPlace &place) { + [](Tensor &self, paddle::platform::XPUPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](framework::Tensor &self, paddle::platform::CPUPlace &place) { + [](Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_double", - [](framework::Tensor &self, paddle::platform::CPUPlace &place) { + [](Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](framework::Tensor &self, paddle::platform::CPUPlace &place) { + [](Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](framework::Tensor &self, paddle::platform::XPUPlace &place) { + [](Tensor &self, paddle::platform::XPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](framework::Tensor &self, paddle::platform::CUDAPlace &place) { + [](Tensor &self, paddle::platform::CUDAPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](framework::Tensor &self, - paddle::platform::CUDAPinnedPlace &place) { + [](Tensor &self, paddle::platform::CUDAPinnedPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](framework::Tensor &self, - paddle::platform::CUDAPinnedPlace &place) { + [](Tensor &self, paddle::platform::CUDAPinnedPlace &place) { self.mutable_data(place); }) .def("_mutable_data", - [](framework::Tensor &self, paddle::platform::CPUPlace &place, + [](Tensor &self, paddle::platform::CPUPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](framework::Tensor &self, paddle::platform::XPUPlace &place, + [](Tensor &self, paddle::platform::XPUPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](framework::Tensor &self, paddle::platform::CUDAPlace &place, + [](Tensor &self, paddle::platform::CUDAPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](framework::Tensor &self, paddle::platform::CUDAPinnedPlace &place, + [](Tensor &self, paddle::platform::CUDAPinnedPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) - .def("_clear", &framework::Tensor::clear) + .def("_clear", &Tensor::clear) .def("set", SetTensorFromPyArray, py::arg("array"), py::arg("place"), py::arg("zero_copy") = false) .def("set", SetTensorFromPyArray, @@ -641,9 +626,7 @@ PYBIND11_MODULE(core_noavx, m) { t.set(np.ndarray([5, 30]), fluid.CPUPlace()) )DOC") - .def("shape", - [](framework::Tensor &self) { return vectorize(self.dims()); }, - R"DOC( + .def("shape", [](Tensor &self) { return vectorize(self.dims()); }, R"DOC( Return the shape of LoDTensor. Returns: @@ -661,7 +644,7 @@ PYBIND11_MODULE(core_noavx, m) { print(t.shape()) # [5, 30] )DOC") .def("_to_dlpack", - [](framework::Tensor &self) { + [](Tensor &self) { DLPackTensor dlpack_tensor(self, 1); DLManagedTensor *dmt = dlpack_tensor.ToCudfCompatibleDLManagedTensor(); @@ -686,22 +669,20 @@ PYBIND11_MODULE(core_noavx, m) { .def("_get_float_element", TensorGetElement) .def("_set_double_element", TensorSetElement) .def("_get_double_element", TensorGetElement) - .def("_place", [](framework::Tensor &self) { return self.place(); }) - .def("_dtype", [](framework::Tensor &self) { return self.type(); }) + .def("_place", [](Tensor &self) { return self.place(); }) + .def("_dtype", [](Tensor &self) { return self.type(); }) .def("_layout", - [](framework::Tensor &self) { - return DataLayoutToString(self.layout()); - }) - .def("_share_data_with", &framework::Tensor::ShareDataWith) + [](Tensor &self) { return DataLayoutToString(self.layout()); }) + .def("_share_data_with", &Tensor::ShareDataWith) .def("__getitem__", PySliceTensor, py::return_value_policy::reference) - .def("__str__", [](const framework::Tensor &self) { + .def("__str__", [](const Tensor &self) { std::stringstream ostr; ostr << self; return ostr.str(); }); // TODO(cql): add reference: en_user_guide_lod_tensor - py::class_(m, "LoDTensor", R"DOC( + py::class_(m, "LoDTensor", R"DOC( LoDTensor is a Tensor with optional LoD (Level of Details) information, it can be used for variable-length sequences, see :ref:`user_guide_lod_tensor` for details. @@ -785,8 +766,7 @@ PYBIND11_MODULE(core_noavx, m) { t = fluid.LoDTensor() )DOC") - .def("__array__", - [](framework::Tensor &self) { return TensorToPyArray(self); }) + .def("__array__", [](Tensor &self) { return TensorToPyArray(self); }) .def("__init__", [](LoDTensor &instance, const std::vector> &recursive_sequence_lengths) { @@ -1744,15 +1724,12 @@ All parameter, weight, gradient are variables in Paddle. m.def("init_gflags", framework::InitGflags); m.def("init_glog", framework::InitGLOG); m.def("load_op_library", framework::LoadOpLib); - m.def("load_op_meta_info_and_register_op", - framework::LoadOpMetaInfoAndRegisterOp); m.def("init_devices", []() { framework::InitDevices(); }); m.def("is_compiled_with_cuda", IsCompiledWithCUDA); m.def("is_compiled_with_xpu", IsCompiledWithXPU); m.def("is_compiled_with_mkldnn", IsCompiledWithMKLDNN); m.def("supports_bfloat16", SupportsBfloat16); - m.def("supports_bfloat16_fast_performance", SupportsBfloat16FastPerformance); m.def("is_compiled_with_brpc", IsCompiledWithBrpc); m.def("is_compiled_with_dist", IsCompiledWithDIST); m.def("_cuda_synchronize", [](const platform::CUDAPlace &place) { @@ -1974,10 +1951,6 @@ All parameter, weight, gradient are variables in Paddle. m.def("nvprof_init", platform::CudaProfilerInit); m.def("nvprof_start", platform::CudaProfilerStart); m.def("nvprof_stop", platform::CudaProfilerStop); - m.def("nvprof_nvtx_push", platform::CudaNvtxRangePush); - m.def("nvprof_nvtx_pop", platform::CudaNvtxRangePop); - m.def("nvprof_enable_record_event", platform::NvprofEnableRecordEvent); - m.def("nvprof_disable_record_event", platform::NvprofDisableRecordEvent); #endif #endif @@ -2015,13 +1988,6 @@ All parameter, weight, gradient are variables in Paddle. m.def("size_of_dtype", framework::SizeOfType); -#ifdef PADDLE_WITH_CUDA - m.def("set_cublas_switch", platform::SetAllowTF32Cublas); - m.def("get_cublas_switch", platform::AllowTF32Cublas); - m.def("set_cudnn_switch", platform::SetAllowTF32Cudnn); - m.def("get_cudnn_switch", platform::AllowTF32Cudnn); -#endif // PADDLE_WITH_CUDA - using VarQuantScale = std::unordered_map>; @@ -2879,6 +2845,7 @@ All parameter, weight, gradient are variables in Paddle. BindDistFleetWrapper(&m); BindPSHost(&m); BindCommunicatorContext(&m); + BindGraphService(&m); BindDistCommunicator(&m); BindHeterClient(&m); #endif From 7753f387b3e9e56ebaf91022aea585af2d6b2295 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Fri, 26 Feb 2021 03:54:23 +0000 Subject: [PATCH 002/106] upload unsaved changes --- .../distributed/service/graph_brpc_client.cc | 205 +++++++-------- .../distributed/service/graph_brpc_client.h | 44 ++-- .../distributed/service/graph_brpc_server.cc | 120 ++++----- .../distributed/service/graph_brpc_server.h | 34 +-- .../distributed/service/graph_service.cc | 239 ++++++++--------- paddle/fluid/distributed/service/ps_client.cc | 1 - paddle/fluid/distributed/service/ps_client.h | 31 +-- .../fluid/distributed/service/sendrecv.proto | 2 +- .../distributed/table/common_graph_table.cc | 245 +++++++++--------- .../distributed/table/common_graph_table.h | 143 +++++----- paddle/fluid/distributed/table/graph_node.cc | 76 +++--- paddle/fluid/distributed/table/graph_node.h | 139 +++++----- paddle/fluid/distributed/table/table.cc | 2 +- paddle/fluid/distributed/table/table.h | 12 +- .../distributed/table/weighted_sampler.cc | 128 ++++----- .../distributed/table/weighted_sampler.h | 81 +++--- paddle/fluid/distributed/test/CMakeLists.txt | 2 +- .../fluid/distributed/test/graph_node_test.cc | 236 ++++++++--------- 18 files changed, 809 insertions(+), 931 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index d0a068dcd63b0..05254acf8a104 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -1,17 +1,3 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - #include #include #include @@ -20,118 +6,115 @@ #include "Eigen/Dense" #include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/string/string_helper.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" namespace paddle { namespace distributed { -int GraphBrpcClient::get_server_index_by_id(uint64_t id) { - int shard_num = get_shard_num(); - size_t server_size = get_server_nums(); - int shard_per_server = shard_num % server_size == 0 - ? shard_num / server_size - : shard_num / server_size + 1; - return id % shard_num / shard_per_server; +int GraphBrpcClient::get_server_index_by_id(uint64_t id){ + int shard_num = get_shard_num(); + size_t server_size = get_server_nums(); + int shard_per_server = shard_num % server_size == 0 ? shard_num / server_size : shard_num / server_size + 1; + return id % shard_num / shard_per_server; } -// char* &buffer,int &actual_size -std::future GraphBrpcClient::sample(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, - int sample_size, - vector &res) { +//char* &buffer,int &actual_size +std::future GraphBrpcClient::sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,vector &res){ int server_index = get_server_index_by_id(node_id); - DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { - ret = -1; - } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_GRAPH_SAMPLE); - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - int start = 0; - while (start < bytes_size) { - GraphNode node; - node.recover_from_buffer(buffer + start); - start += node.get_size(); - res.push_back(node); - } - } - closure->set_promise_value(ret); - }); - auto promise = std::make_shared>(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + 1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " <<" "<check_response(0, PS_GRAPH_SAMPLE); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), + bytes_size); + int start = 0; + while(start < bytes_size){ + GraphNode node; + node.recover_from_buffer(buffer + start); + start += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); closure->add_promise(promise); - std::future fut = promise->get_future(); - ; - closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; - closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), - closure); + std::future fut = promise->get_future(); ; + closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, + sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, + sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), + closure->response(0), closure); + return fut; } -std::future GraphBrpcClient::pull_graph_list(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, - int start, int size, - vector &res) { +std:: future GraphBrpcClient::pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int size, vector &res){ int server_index = get_server_index_by_id(node_id); - DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { - ret = -1; - } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - int index = 0; - while (index < bytes_size) { - GraphNode node; - node.recover_from_buffer(buffer + index); - index += node.get_size(); - res.push_back(node); - } - } - closure->set_promise_value(ret); - }); - auto promise = std::make_shared>(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + 1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " <<" "<check_response(0, PS_PULL_GRAPH_LIST); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), + bytes_size); + int index = 0; + while(index < bytes_size){ + GraphNode node; + node.recover_from_buffer(buffer + index); + index += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); closure->add_promise(promise); - std::future fut = promise->get_future(); - ; - closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; - closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, sizeof(int)); - closure->request(0)->add_params((char *)&start, sizeof(int)); - closure->request(0)->add_params((char *)&size, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), - closure); - return fut; + std::future fut = promise->get_future(); ; + closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, + sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, + sizeof(int)); + closure->request(0)->add_params((char *)&start, + sizeof(int)); + closure->request(0)->add_params((char *)&size, + sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), + closure->response(0), closure); + return fut; } -int32_t GraphBrpcClient::initialize() { - set_shard_num(_config.shard_num()); - return BrpcPsClient::initialize(); +int32_t GraphBrpcClient::initialize(){ + set_shard_num(_config.shard_num()); + return BrpcPsClient::initialize(); } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 9dac994f296e7..1557e445d66af 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -21,35 +21,37 @@ #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" -#include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/ps_client.h" -#include "paddle/fluid/distributed/table/graph_node.h" -#include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/framework/tensor_util.h" +#include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { -class GraphBrpcClient : public BrpcPsClient { - public: - GraphBrpcClient() {} - virtual ~GraphBrpcClient() {} - virtual std::future sample(uint32_t table_id, uint64_t node_id, - GraphNodeType type, int sample_size, - vector &res); - virtual std::future pull_graph_list(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, int start, - int end, vector &res); - virtual int32_t initialize(); - int get_shard_num() { return shard_num; } - void set_shard_num(int shard_num) { this->shard_num = shard_num; } - int get_server_index_by_id(uint64_t id); - - private: - int shard_num; +class GraphBrpcClient: public BrpcPsClient{ +public: + GraphBrpcClient(){ + + } + virtual ~GraphBrpcClient(){ + + } + virtual std::future sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,vector &res); + virtual std::future pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int end, vector &res); + virtual int32_t initialize(); + int get_shard_num(){ + return shard_num; + } + void set_shard_num(int shard_num){ + this->shard_num = shard_num; + } + int get_server_index_by_id(uint64_t id); +private: + int shard_num; }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 9e54b932b5301..3239648ff1567 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -1,28 +1,14 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/service/graph_brpc_server.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" #include // NOLINT -#include "Eigen/Dense" -#include "butil/endpoint.h" #include "iomanip" -#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/platform/profiler.h" +#include "paddle/fluid/distributed/table/graph_node.h" +#include "Eigen/Dense" +#include "butil/endpoint.h" namespace paddle { namespace distributed { @@ -90,15 +76,13 @@ int32_t GraphBrpcService::initialize() { _service_handler_map[PS_LOAD_ONE_TABLE] = &GraphBrpcService::load_one_table; _service_handler_map[PS_LOAD_ALL_TABLE] = &GraphBrpcService::load_all_table; - _service_handler_map[PS_PRINT_TABLE_STAT] = - &GraphBrpcService::print_table_stat; + _service_handler_map[PS_PRINT_TABLE_STAT] = &GraphBrpcService::print_table_stat; _service_handler_map[PS_BARRIER] = &GraphBrpcService::barrier; _service_handler_map[PS_START_PROFILER] = &GraphBrpcService::start_profiler; _service_handler_map[PS_STOP_PROFILER] = &GraphBrpcService::stop_profiler; - _service_handler_map[PS_PULL_GRAPH_LIST] = &GraphBrpcService::pull_graph_list; - _service_handler_map[PS_GRAPH_SAMPLE] = - &GraphBrpcService::graph_random_sample; + _service_handler_map[PS_PULL_GRAPH_LIST] =&GraphBrpcService::pull_graph_list; + _service_handler_map[PS_GRAPH_SAMPLE] = &GraphBrpcService::graph_random_sample; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); @@ -131,9 +115,9 @@ int32_t GraphBrpcService::initialize_shard_info() { } void GraphBrpcService::service(google::protobuf::RpcController *cntl_base, - const PsRequestMessage *request, - PsResponseMessage *response, - google::protobuf::Closure *done) { + const PsRequestMessage *request, + PsResponseMessage *response, + google::protobuf::Closure *done) { brpc::ClosureGuard done_guard(done); std::string log_label("ReceiveCmd-"); if (!request->has_table_id()) { @@ -161,9 +145,11 @@ void GraphBrpcService::service(google::protobuf::RpcController *cntl_base, } } + + int32_t GraphBrpcService::barrier(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 1) { @@ -179,10 +165,12 @@ int32_t GraphBrpcService::barrier(Table *table, const PsRequestMessage &request, return 0; } + + int32_t GraphBrpcService::print_table_stat(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) std::pair ret = table->print_table_stat(); paddle::framework::BinaryArchive ar; @@ -194,9 +182,9 @@ int32_t GraphBrpcService::print_table_stat(Table *table, } int32_t GraphBrpcService::load_one_table(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 2) { set_response_code( @@ -212,9 +200,9 @@ int32_t GraphBrpcService::load_one_table(Table *table, } int32_t GraphBrpcService::load_all_table(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { auto &table_map = *(_server->table()); for (auto &itr : table_map) { if (load_one_table(itr.second.get(), request, response, cntl) != 0) { @@ -225,10 +213,13 @@ int32_t GraphBrpcService::load_all_table(Table *table, return 0; } + + + int32_t GraphBrpcService::stop_server(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { auto *p_server = _server; std::thread t_stop([p_server]() { p_server->stop(); @@ -239,48 +230,46 @@ int32_t GraphBrpcService::stop_server(Table *table, } int32_t GraphBrpcService::stop_profiler(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { platform::DisableProfiler(platform::EventSortingKey::kDefault, string::Sprintf("server_%s_profile", _rank)); return 0; } int32_t GraphBrpcService::start_profiler(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { platform::EnableProfiler(platform::ProfilerState::kCPU); return 0; } -int32_t GraphBrpcService::pull_graph_list(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { +int32_t GraphBrpcService::pull_graph_list(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl){ CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 2) { - set_response_code(response, -1, - "pull_graph_list request requires at least 2 arguments"); + set_response_code( + response, -1, + "pull_graph_list request requires at least 2 arguments"); return 0; } uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); - std::string type_str = request.params(1); - GraphNodeType type = GraphNode::get_graph_node_type(type_str); + std::string type_str = request.params(1); + GraphNodeType type = GraphNode::get_graph_node_type(type_str); int start = *(int *)(request.params(2).c_str()); int size = *(int *)(request.params(3).c_str()); std::vector res_data; char *buffer; int actual_size; - table->pull_graph_list(node_id, type, start, size, buffer, actual_size); - cntl->response_attachment().append(buffer, actual_size); + table->pull_graph_list(node_id, type, start, size, buffer,actual_size); + cntl->response_attachment().append(buffer, + actual_size); return 0; } -int32_t GraphBrpcService::graph_random_sample(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { +int32_t GraphBrpcService::graph_random_sample(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl){ CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 3) { set_response_code( @@ -290,14 +279,17 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, } uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); std::string type_str = request.params(1); - int sample_size = *(uint64_t *)(request.params(2).c_str()); + int sample_size = *(uint64_t *)(request.params(2).c_str()); GraphNodeType type = GraphNode::get_graph_node_type(type_str); char *buffer; int actual_size; - table->random_sample(node_id, type, sample_size, buffer, actual_size); - cntl->response_attachment().append(buffer, actual_size); - return 0; -} + table->random_sample(node_id,type,sample_size,buffer, actual_size); + cntl->response_attachment().append(buffer, + actual_size); + return 0; + } + + } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index b4056692f2ccf..54480e806036b 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -1,17 +1,3 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - #pragma once #include "brpc/channel.h" @@ -20,8 +6,8 @@ #include #include -#include "paddle/fluid/distributed/service/brpc_ps_server.h" #include "paddle/fluid/distributed/service/server.h" +#include "paddle/fluid/distributed/service/brpc_ps_server.h" namespace paddle { namespace distributed { @@ -65,6 +51,7 @@ typedef int32_t (GraphBrpcService::*serviceFunc)( Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); + class GraphBrpcService : public PsBaseService { public: virtual int32_t initialize() override; @@ -75,13 +62,12 @@ class GraphBrpcService : public PsBaseService { ::google::protobuf::Closure *done) override; protected: - std::unordered_map _service_handler_map; + std::unordered_map _service_handler_map; int32_t initialize_shard_info(); - int32_t pull_graph_list(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, brpc::Controller *cntl); - int32_t graph_random_sample(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl); + int32_t pull_graph_list(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); +int32_t graph_random_sample(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, @@ -108,11 +94,9 @@ class GraphBrpcService : public PsBaseService { // public: // virtual int32_t initialize() override; // int32_t pull_graph_list(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller -// *cntl); +// PsResponseMessage &response, brpc::Controller *cntl); // int32_t graph_random_sample(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller -// *cntl); +// PsResponseMessage &response, brpc::Controller *cntl); // }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_service.cc b/paddle/fluid/distributed/service/graph_service.cc index 2b3abe66b29f2..f8ce4dc3c8504 100644 --- a/paddle/fluid/distributed/service/graph_service.cc +++ b/paddle/fluid/distributed/service/graph_service.cc @@ -1,20 +1,6 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include +#include #include -#include +#include // #include // #include // NOLINT // #include // NOLINT @@ -43,129 +29,134 @@ // #include "paddle/fluid/string/printf.h" namespace paddle { namespace distributed { -class graph_service { - std::vector keys; - std::vector server_list, port_list, host_sign_list; - int server_size, shard_num; +class graph_service{ - public: - // void GetDownpourSparseTableProto( - // ::paddle::distributed::TableParameter* sparse_table_proto) { - // sparse_table_proto->set_table_id(0); - // sparse_table_proto->set_table_class("GraphTable"); - // sparse_table_proto->set_shard_num(256); - // sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); - // ::paddle::distributed::TableAccessorParameter* accessor_proto = - // sparse_table_proto->mutable_accessor(); - // ::paddle::distributed::CommonAccessorParameter* common_proto = - // sparse_table_proto->mutable_common(); + std::vector keys; + std::vector server_list,port_list,host_sign_list; + int server_size,shard_num; + public: + // void GetDownpourSparseTableProto( +// ::paddle::distributed::TableParameter* sparse_table_proto) { +// sparse_table_proto->set_table_id(0); +// sparse_table_proto->set_table_class("GraphTable"); +// sparse_table_proto->set_shard_num(256); +// sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); +// ::paddle::distributed::TableAccessorParameter* accessor_proto = +// sparse_table_proto->mutable_accessor(); +// ::paddle::distributed::CommonAccessorParameter* common_proto = +// sparse_table_proto->mutable_common(); - // accessor_proto->set_accessor_class("CommMergeAccessor"); - // accessor_proto->set_fea_dim(0); - // accessor_proto->set_embedx_dim(10); +// accessor_proto->set_accessor_class("CommMergeAccessor"); +// accessor_proto->set_fea_dim(0); +// accessor_proto->set_embedx_dim(10); - // common_proto->set_name("sgd"); - // common_proto->set_table_name("MergedDense"); - // common_proto->set_trainer_num(1); - // common_proto->set_sync(false); - // common_proto->add_params("Param"); - // common_proto->add_dims(10); - // common_proto->add_initializers("uniform_random&0&-1.0&1.0"); - // common_proto->add_params("LearningRate"); - // common_proto->add_dims(1); - // common_proto->add_initializers("fill_constant&1.0"); - // } +// common_proto->set_name("sgd"); +// common_proto->set_table_name("MergedDense"); +// common_proto->set_trainer_num(1); +// common_proto->set_sync(false); +// common_proto->add_params("Param"); +// common_proto->add_dims(10); +// common_proto->add_initializers("uniform_random&0&-1.0&1.0"); +// common_proto->add_params("LearningRate"); +// common_proto->add_dims(1); +// common_proto->add_initializers("fill_constant&1.0"); +// } - // ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { - // ::paddle::distributed::PSParameter worker_fleet_desc; - // worker_fleet_desc.set_shard_num(shard_num); - // ::paddle::distributed::WorkerParameter* worker_proto = - // worker_fleet_desc.mutable_worker_param(); +// ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { +// ::paddle::distributed::PSParameter worker_fleet_desc; +// worker_fleet_desc.set_shard_num(shard_num); +// ::paddle::distributed::WorkerParameter* worker_proto = +// worker_fleet_desc.mutable_worker_param(); - // ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = - // worker_proto->mutable_downpour_worker_param(); +// ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = +// worker_proto->mutable_downpour_worker_param(); - // ::paddle::distributed::TableParameter* worker_sparse_table_proto = - // downpour_worker_proto->add_downpour_table_param(); - // GetDownpourSparseTableProto(worker_sparse_table_proto); +// ::paddle::distributed::TableParameter* worker_sparse_table_proto = +// downpour_worker_proto->add_downpour_table_param(); +// GetDownpourSparseTableProto(worker_sparse_table_proto); - // ::paddle::distributed::ServerParameter* server_proto = - // worker_fleet_desc.mutable_server_param(); - // ::paddle::distributed::DownpourServerParameter* downpour_server_proto = - // server_proto->mutable_downpour_server_param(); - // ::paddle::distributed::ServerServiceParameter* server_service_proto = - // downpour_server_proto->mutable_service_param(); - // server_service_proto->set_service_class("GraphBrpcService"); - // server_service_proto->set_server_class("GraphBrpcServer"); - // server_service_proto->set_client_class("GraphBrpcClient"); - // server_service_proto->set_start_server_port(0); - // server_service_proto->set_server_thread_num(12); +// ::paddle::distributed::ServerParameter* server_proto = +// worker_fleet_desc.mutable_server_param(); +// ::paddle::distributed::DownpourServerParameter* downpour_server_proto = +// server_proto->mutable_downpour_server_param(); +// ::paddle::distributed::ServerServiceParameter* server_service_proto = +// downpour_server_proto->mutable_service_param(); +// server_service_proto->set_service_class("GraphBrpcService"); +// server_service_proto->set_server_class("GraphBrpcServer"); +// server_service_proto->set_client_class("GraphBrpcClient"); +// server_service_proto->set_start_server_port(0); +// server_service_proto->set_server_thread_num(12); - // ::paddle::distributed::TableParameter* server_sparse_table_proto = - // downpour_server_proto->add_downpour_table_param(); - // GetDownpourSparseTableProto(server_sparse_table_proto); +// ::paddle::distributed::TableParameter* server_sparse_table_proto = +// downpour_server_proto->add_downpour_table_param(); +// GetDownpourSparseTableProto(server_sparse_table_proto); - // return worker_fleet_desc; - // } - void set_server_size(int server_size) { this->server_size = server_size; } - int get_server_size(int server_size) { return server_size; } - std::vector split(std::string &str, const char pattern) { +// return worker_fleet_desc; +// } + void set_server_size(int server_size){ + this->server_size = server_size; + } + int get_server_size(int server_size){ + return server_size; + } + std::vector split(std::string &str, const char pattern) +{ std::vector res; - std::stringstream input(str); + std::stringstream input(str); std::string temp; - while (std::getline(input, temp, pattern)) { - res.push_back(temp); + while(std::getline(input, temp, pattern)) + { + res.push_back(temp); } return res; - } - void start_client() { - // framework::Scope client_scope; - // platform::CPUPlace place; - // InitTensorsOnClient(&client_scope, &place, 100); - // std::map> - // dense_regions; - // dense_regions.insert( - // std::pair>(0, - // {})); - // auto regions = dense_regions[0]; - // framework::Variable* var = client_scope.FindVar("x"); - // framework::LoDTensor* tensor = var->GetMutable(); +} + void start_client(){ +// framework::Scope client_scope; +// platform::CPUPlace place; +// InitTensorsOnClient(&client_scope, &place, 100); +// std::map> dense_regions; +// dense_regions.insert( +// std::pair>(0, {})); +// auto regions = dense_regions[0]; +// framework::Variable* var = client_scope.FindVar("x"); +// framework::LoDTensor* tensor = var->GetMutable(); - // RunClient(dense_regions); - // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); - // paddle::distributed::PaddlePSEnvironment _ps_env; - // auto servers_ = host_sign_list_.size(); - // _ps_env = paddle::distributed::PaddlePSEnvironment(); - // _ps_env.set_ps_servers(&host_sign_list_, servers_); - // worker_ptr_ = std::shared_ptr( - // paddle::distributed::PSClientFactory::create(worker_proto)); - // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); - } - void init(std::string ips_str, int shard_num) { - std::istringstream stream(ips_str); - std::string ip, port; - server_size = 0; - std::vector ips_list = split(ips_str, ';'); - int index = 0; - for (auto ips : ips_list) { - auto ip_and_port = split(ips, ':'); - server_list.push_back(ip_and_port[0]); - port_list.push_back(ip_and_port[1]); - // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], - // ip_and_port[1], index); - // host_sign_list_.push_back(ph_host.serialize_to_string()); - index++; +// RunClient(dense_regions); +// ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); +// paddle::distributed::PaddlePSEnvironment _ps_env; +// auto servers_ = host_sign_list_.size(); +// _ps_env = paddle::distributed::PaddlePSEnvironment(); +// _ps_env.set_ps_servers(&host_sign_list_, servers_); +// worker_ptr_ = std::shared_ptr( +// paddle::distributed::PSClientFactory::create(worker_proto)); +// worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); } - start_client(); - } + void init(std::string ips_str,int shard_num){ + std::istringstream stream(ips_str); + std::string ip,port; + server_size = 0; + std::vector ips_list = split(ips_str, ';'); + int index = 0; + for(auto ips:ips_list){ + auto ip_and_port = split(ips,':'); + server_list.push_back(ip_and_port[0]); + port_list.push_back(ip_and_port[1]); + // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], ip_and_port[1], index); + // host_sign_list_.push_back(ph_host.serialize_to_string()); + index++; + } + start_client(); + } + + public: + void set_keys(std::vector keys){//just for test + this->keys = keys; + } + std::vector get_keys(int start,int size){//just for test + return std::vector(keys.begin() + start, keys.begin() + start + size); + } + - public: - void set_keys(std::vector keys) { // just for test - this->keys = keys; - } - std::vector get_keys(int start, int size) { // just for test - return std::vector(keys.begin() + start, keys.begin() + start + size); - } }; -} + } } \ No newline at end of file diff --git a/paddle/fluid/distributed/service/ps_client.cc b/paddle/fluid/distributed/service/ps_client.cc index 902c72c30ac5d..68b240e5f1ac5 100644 --- a/paddle/fluid/distributed/service/ps_client.cc +++ b/paddle/fluid/distributed/service/ps_client.cc @@ -18,7 +18,6 @@ #include "glog/logging.h" #include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/graph_brpc_client.h" -#include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 5a060f1dfa51d..a104968493861 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -154,23 +154,20 @@ class PSClient { promise.set_value(-1); return fut; } - // virtual std::future sample(uint32_t table_id, uint64_t node_id, - // GraphNodeType type, int sample_size,std::vector &res){ - // LOG(FATAL) << "Did not implement"; - // std::promise promise; - // std::future fut = promise.get_future(); - // promise.set_value(-1); - // return fut; - // } - // virtual std::future pull_graph_list(uint32_t table_id, uint64_t - // node_id, GraphNodeType type,int start, int size, std::vector - // &res){ - // LOG(FATAL) << "Did not implement"; - // std::promise promise; - // std::future fut = promise.get_future(); - // promise.set_value(-1); - // return fut; - // } + // virtual std::future sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,std::vector &res){ + // LOG(FATAL) << "Did not implement"; + // std::promise promise; + // std::future fut = promise.get_future(); + // promise.set_value(-1); + // return fut; + // } + // virtual std::future pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int size, std::vector &res){ + // LOG(FATAL) << "Did not implement"; + // std::promise promise; + // std::future fut = promise.get_future(); + // promise.set_value(-1); + // return fut; + // } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 52c2d7104b544..5c77a472c4e9f 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -52,7 +52,7 @@ enum PsCmdID { PS_GRAPH_SAMPLE = 31; } -message PsRequestMessage { +message PsRequestMessage{ required uint32 cmd_id = 1; optional uint32 table_id = 2; repeated bytes params = 3; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 1146b39fecd40..43b44cdac5bbe 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -1,75 +1,59 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/common_graph_table.h" #include #include #include "paddle/fluid/distributed/common/utils.h" -#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" +#include "paddle/fluid/distributed/table/common_graph_table.h" namespace paddle { namespace distributed { -size_t GraphShard::get_size() { +size_t GraphShard::get_size(){ size_t res = 0; - for (int i = 0; i < bucket_size; i++) { + for(int i = 0;i < bucket_size;i++){ res += bucket[i].size(); } return res; } -list::iterator GraphShard::add_node(GraphNode *node) { - if (node_location.find({node->get_id(), node->get_graph_node_type()}) != - node_location.end()) - return node_location.find({node->get_id(), node->get_graph_node_type()}) - ->second; +list::iterator GraphShard::add_node(GraphNode *node){ + if(node_location.find({node->get_id(),node->get_graph_node_type()}) != node_location.end()) + return node_location.find({node->get_id(),node->get_graph_node_type()})->second; int index = node->get_id() % shard_num % bucket_size; - list::iterator iter = - bucket[index].insert(bucket[index].end(), node); - node_location[{node->get_id(), node->get_graph_node_type()}] = iter; + list::iterator iter = bucket[index].insert(bucket[index].end(), node); + node_location[{node->get_id(),node->get_graph_node_type()}] = iter; return iter; } -void GraphShard::add_neighboor(uint64_t id, GraphNodeType type, - GraphEdge *edge) { - (*add_node(new GraphNode(id, type, std::string(""))))->add_edge(edge); +void GraphShard:: add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge){ + + (*add_node(new GraphNode(id,type,std::string(""))))->add_edge(edge); } -GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { - if (node_location.find({id, type}) == node_location.end()) return NULL; - return *(node_location[{id, type}]); +GraphNode * GraphShard::find_node(uint64_t id, GraphNodeType type){ + if(node_location.find({id,type}) == node_location.end()) + return NULL; + return *(node_location[{id,type}]); } -int32_t GraphTable::load(const std::string &path, const std::string ¶m) { +int32_t GraphTable::load(const std::string& path, const std::string& param){ rwlock_->WRLock(); auto paths = paddle::string::split_string(path, ";"); - for (auto path : paths) { + for(auto path: paths){ std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); - if (values.size() < 3) continue; + if(values.size() < 3) + continue; auto id = std::stoull(values[0]); size_t shard_id = id % _shard_num; if (shard_id >= shard_end || shard_id < shard_start) { VLOG(0) << "will not load " << id << " from " << path - << ", please check id distribution"; + << ", please check id distribution"; continue; } size_t index = shard_id - shard_start; GraphNodeType type = GraphNode::get_graph_node_type(values[1]); - shards[index].add_node(new GraphNode(id, type, values[2])); - for (size_t i = 3; i < values.size(); i++) { - auto edge_arr = - paddle::string::split_string(values[i], ";"); - if (edge_arr.size() < 3) { + shards[index].add_node(new GraphNode(id, type, values[2])); + for(size_t i = 3;i < values.size();i++){ + auto edge_arr = paddle::string::split_string(values[i], ";"); + if(edge_arr.size() < 3){ auto edge_id = std::stoull(edge_arr[0]); GraphNodeType edge_type = GraphNode::get_graph_node_type(edge_arr[1]); auto weight = std::stod(edge_arr[2]); @@ -78,125 +62,130 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { } } } + for(auto &shard:shards){ + list::iter = shard.begin(); + while(iter != shard.end()){ + auto node = *iter; + + iter++; + } + } } rwlock_->UNLock(); return 0; } -GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { - rwlock_->WRLock(); - size_t shard_id = id % _shard_num; - if (shard_id >= shard_end || shard_id < shard_start) { - return NULL; - } - size_t index = shard_id - shard_start; - - GraphNode *node = shards[index].find_node(id, type); - rwlock_->UNLock(); - return node; -} -int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, - int sample_size, char *&buffer, - int &actual_size) { - rwlock_->RDLock(); - GraphNode *node = find_node(node_id, type); - if (node == NULL) { - actual_size = 0; +GraphNode * GraphTable::find_node(uint64_t id, GraphNodeType type){ + rwlock_->WRLock(); + size_t shard_id = id % _shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + return NULL; + } + size_t index = shard_id - shard_start; + + GraphNode *node = shards[index].find_node(id,type); rwlock_->UNLock(); - return 0; - } - vector res = node->sample_k(sample_size); - vector node_list; - int total_size = 0; - for (auto x : res) { - GraphNode temp; - temp.set_id(x->id); - temp.set_graph_node_type(x->type); - total_size += temp.get_size(); - node_list.push_back(temp); - } - buffer = new char[total_size]; - int index = 0; - for (auto x : node_list) { - x.to_buffer(buffer + index); - index += x.get_size(); + return node; + } - actual_size = total_size; - rwlock_->UNLock(); - return 0; +int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type,int sample_size, char* &buffer, int &actual_size){ + rwlock_->RDLock(); + GraphNode * node = find_node(node_id,type); + if(node == NULL){ + actual_size = 0; + rwlock_->UNLock(); + return 0; + } + vector res = node->sample_k(sample_size); + vector node_list; + int total_size = 0; + for(auto x:res){ + GraphNode temp; + temp.set_id(x->id); + temp.set_graph_node_type(x->type); + total_size += temp.get_size(); + node_list.push_back(temp); + } + buffer = new char[total_size]; + int index = 0; + for(auto x:node_list){ + x.to_buffer(buffer + index); + index += x.get_size(); + } + actual_size = total_size; + rwlock_->UNLock(); + return 0; } -int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, - int &actual_size) { - rwlock_->RDLock(); +int32_t GraphTable::pull_graph_list(int start, int total_size, char* &buffer,int &actual_size){ + rwlock_->RDLock(); int thread_index = 0; - if (start < 0) start = 0; - int size = 0, cur_size; - if (total_size <= 0) { - actual_size = 0; - return 0; - } - std::vector>> tasks; - for (size_t i = 0; i < shards.size(); i++) { - cur_size = shards[i].get_size(); - if (size + cur_size <= start) { - size += cur_size; - continue; - } - if (size + cur_size - start >= total_size) { - tasks.push_back(_shards_task_pool[thread_index]->enqueue( - [this, i, start, size, total_size]() -> vector { - return this->shards[i].get_batch(start - size, total_size); - })); - thread_index++; - break; - } else { - tasks.push_back(_shards_task_pool[thread_index]->enqueue( - [this, i, start, size, total_size, - cur_size]() -> vector { - return this->shards[i].get_batch(start - size, - size + cur_size - start); - })); - total_size -= size + cur_size - start; - size += cur_size; - start = size; - } - } + if(start < 0) + start = 0; + int size = 0, cur_size; + if(total_size <= 0){ + actual_size = 0; + return 0; + } + std::vector>> tasks; + for(size_t i = 0;i < shards.size();i++){ + cur_size = shards[i].get_size(); + if(size + cur_size <= start){ + size += cur_size; + continue; + } + if(size + cur_size - start >= total_size){ + + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this,i,start,size,total_size]() -> vector { + return this->shards[i].get_batch(start - size,total_size); + })); + thread_index++; + break; + } else { + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this,i,start,size,total_size,cur_size]() -> vector { + return this->shards[i].get_batch(start - size,size + cur_size - start); + })); + total_size -= size + cur_size - start; + size += cur_size; + start = size; + } + } for (size_t i = 0; i < tasks.size(); ++i) { tasks[i].wait(); } size = 0; - vector> res; - for (size_t i = 0; i < tasks.size(); i++) { + vector > res; + for(size_t i = 0;i < tasks.size();i++){ res.push_back(tasks[i].get()); - for (size_t j = 0; j < res.back().size(); j++) { + for(size_t j = 0;j < res.back().size();j++){ size += res.back()[j]->get_size(); } - } + } buffer = new char[size]; int index = 0; - for (size_t i = 0; i < res.size(); i++) { - for (size_t j = 0; j < res[i].size(); j++) { + for(size_t i = 0;i < res.size();i++){ + for(size_t j = 0;j < res[i].size();j++){ res[i][j]->to_buffer(buffer + index); index += res[i][j]->get_size(); } } actual_size = size; - rwlock_->UNLock(); - return 0; + rwlock_->UNLock(); + return 0; } -int32_t GraphTable::initialize() { +int32_t GraphTable::initialize(){ _shards_task_pool.resize(task_pool_size_); for (size_t i = 0; i < _shards_task_pool.size(); ++i) { _shards_task_pool[i].reset(new ::ThreadPool(1)); - } - server_num = _shard_num; + } + server_num = _shard_num; /* _shard_num is actually server number here - when a server initialize its tables, it sets tables' _shard_num to server_num, - and _shard_idx to server + when a server initialize its tables, it sets tables' _shard_num to server_num, and _shard_idx to server rank */ _shard_num = _config.shard_num(); - shard_num_per_table = sparse_local_shard_num(_shard_num, server_num); + shard_num_per_table = sparse_local_shard_num(_shard_num,server_num); shard_start = _shard_idx * shard_num_per_table; shards.resize(shard_num_per_table); return 0; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 9adf155c50a27..0e4f2b29acf7b 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -1,23 +1,8 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - #pragma once #include #include #include -#include #include #include // NOLINT #include @@ -25,99 +10,101 @@ #include #include #include "Eigen/Dense" +#include +#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/distributed/table/accessor.h" #include "paddle/fluid/distributed/table/common_table.h" -#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/framework/rw_lock.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { struct pair_hash { - inline size_t operator()(const pair &p) const { - return p.first * 10007 + int(p.second); - } + inline size_t operator()(const pair & p) const { + return p.first * 10007 + int(p.second); + } }; class GraphShard { - public: + public: static int bucket_low_bound; - static int gcd(int s, int t) { - if (s % t == 0) return t; + static int gcd(int s,int t){ + if(s % t == 0) + return t; return gcd(t, s % t); } size_t get_size(); - GraphShard() {} - GraphShard(int shard_num) { + GraphShard(){ + + } + GraphShard(int shard_num){ this->shard_num = shard_num; bucket_size = init_bucket_size(shard_num); bucket.resize(bucket_size); } - vector get_batch(int start, int total_size) { - if (start < 0) start = 0; - int size = 0, cur_size; - vector res; - if (total_size <= 0) return res; - for (int i = 0; i < bucket_size; i++) { - cur_size = bucket[i].size(); - if (size + cur_size <= start) { - size += cur_size; - continue; + vector get_batch(int start,int total_size){ + if(start < 0) + start = 0; + int size = 0, cur_size; + vector res; + if(total_size <= 0) + return res; + for(int i = 0;i < bucket_size;i++){ + cur_size = bucket[i].size(); + if(size + cur_size <= start){ + size += cur_size; + continue; + } + int read = 0; + list::iterator iter = bucket[i].begin(); + while(size + read < start){ + iter++; + read++; + } + read = 0; + while(iter != bucket[i].end() && read < total_size){ + res.push_back(*iter); + iter++; + read++; + } + if(read == total_size) + break; + size += cur_size; + start = size; + total_size -= read; } - int read = 0; - list::iterator iter = bucket[i].begin(); - while (size + read < start) { - iter++; - read++; - } - read = 0; - while (iter != bucket[i].end() && read < total_size) { - res.push_back(*iter); - iter++; - read++; - } - if (read == total_size) break; - size += cur_size; - start = size; - total_size -= read; - } - return res; + return res; } - int init_bucket_size(int shard_num) { - for (int i = bucket_low_bound;; i++) { - if (gcd(i, shard_num) == 1) return i; - } - return -1; + int init_bucket_size(int shard_num){ + for(int i = bucket_low_bound;;i++){ + if(gcd(i,shard_num) == 1) + return i; + } + return -1; } - list::iterator add_node(GraphNode *node); - GraphNode *find_node(uint64_t id, GraphNodeType type); - void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); + list::iterator add_node(GraphNode *node); + GraphNode * find_node(uint64_t id, GraphNodeType type); + void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); + private: + unordered_map,list::iterator , pair_hash> node_location; + int bucket_size, shard_num; + vector> bucket; - private: - unordered_map, list::iterator, - pair_hash> - node_location; - int bucket_size, shard_num; - vector> bucket; }; class GraphTable : public SparseTable { public: GraphTable() {} - virtual ~GraphTable() {} - virtual int32_t pull_graph_list(int start, int size, char *&buffer, - int &actual_size); - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, - int sampe_size, char *&buffer, - int &actual_size); + virtual ~GraphTable() {} + virtual int32_t pull_graph_list(int start, int size, char* &buffer,int &actual_size); + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, int sampe_size, char* &buffer, int &actual_size); virtual int32_t initialize(); - int32_t load(const std::string &path, const std::string ¶m); + int32_t load(const std::string& path, const std::string& param); GraphNode *find_node(uint64_t id, GraphNodeType type); - - protected: +protected: vector shards; unordered_set id_set; - size_t shard_start, shard_end, server_num, shard_num_per_table; + size_t shard_start,shard_end, server_num,shard_num_per_table; std::unique_ptr rwlock_{nullptr}; const int task_pool_size_ = 7; - std::vector> _shards_task_pool; + std::vector> _shards_task_pool; }; } -}; +}; \ No newline at end of file diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index eb6ca577100a9..929623656d90d 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -1,54 +1,40 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/graph_node.h" -#include -#include "paddle/fluid/distributed/table/weighted_sampler.h" +#include namespace paddle { namespace distributed { -int GraphNode::enum_size = sizeof(int); -int GraphNode::id_size = sizeof(uint64_t); -int GraphNode::double_size = sizeof(double); -int GraphNode::int_size = sizeof(int); -int GraphNode::get_size() { - return feature.size() + id_size + enum_size + int_size; +int GraphNode:: enum_size = sizeof(int); +int GraphNode:: id_size = sizeof(uint64_t); +int GraphNode:: double_size = sizeof(double); +int GraphNode:: int_size = sizeof(int); +int GraphNode::get_size(){ + return feature.size() + id_size + enum_size + int_size; } -void GraphNode::build_sampler() { - sampler = new WeightedSampler(); - GraphEdge** arr = edges.data(); - sampler->build((WeightedObject**)arr, 0, edges.size()); +void GraphNode::build_sampler(){ + sampler = new WeightedSampler(); + GraphEdge** arr = edges.data(); + sampler->build((WeightedObject **)arr, 0, edges.size()); + } -void GraphNode::to_buffer(char* buffer) { - int size = get_size(); - memcpy(buffer, &size, int_size); - memcpy(buffer + int_size, feature.c_str(), feature.size()); - memcpy(buffer + int_size + feature.size(), &id, id_size); - int int_state = type; - memcpy(buffer + int_size + feature.size() + id_size, &int_state, enum_size); +void GraphNode:: to_buffer(char * buffer){ + int size = get_size(); + memcpy(buffer, &size, int_size); + memcpy(buffer + int_size,feature.c_str(),feature.size()); + memcpy(buffer + int_size + feature.size(),&id,id_size); + int int_state = type; + memcpy(buffer + int_size + feature.size() + id_size,&int_state,enum_size); } -void GraphNode::recover_from_buffer(char* buffer) { - int size; - memcpy(&size, buffer, int_size); - int feature_size = size - id_size - enum_size - int_size; - char str[feature_size + 1]; - memcpy(str, buffer + int_size, feature_size); - str[feature_size] = '\0'; - feature = str; - memcpy(&id, buffer + int_size + feature_size, id_size); - int int_state; - memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); - type = GraphNodeType(int_state); +void GraphNode:: recover_from_buffer(char* buffer){ + int size; + memcpy(&size,buffer,int_size); + int feature_size = size - id_size - enum_size - int_size; + char str[feature_size + 1]; + memcpy(str,buffer + int_size,feature_size); + str[feature_size] = '\0'; + feature = str; + memcpy(&id,buffer + int_size + feature_size,id_size); + int int_state; + memcpy(&int_state,buffer + int_size + feature_size + id_size,enum_size); + type = GraphNodeType(int_state); } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index de17ab883e897..4571e10c34fbe 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -1,79 +1,84 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - #pragma once -#include #include "paddle/fluid/distributed/table/weighted_sampler.h" +#include namespace paddle { namespace distributed { -enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; -class GraphEdge : public WeightedObject { - public: - double weight; - uint64_t id; - GraphNodeType type; - GraphEdge() {} - GraphEdge(uint64_t id, GraphNodeType type, double weight) - : weight(weight), id(id), type(type) {} +enum GraphNodeType{ + user=0,item=1,query=2,unknown=3 }; -class GraphNode { - public: - GraphNode() { sampler = NULL; } - GraphNode(uint64_t id, GraphNodeType type, std::string feature) - : id(id), type(type), feature(feature), sampler(NULL) {} - virtual ~GraphNode() {} - static int enum_size, id_size, int_size, double_size; - uint64_t get_id() { return id; } - void set_id(uint64_t id) { this->id = id; } - GraphNodeType get_graph_node_type() { return type; } - void set_graph_node_type(GraphNodeType type) { this->type = type; } - void set_feature(std::string feature) { this->feature = feature; } - std::string get_feature() { return feature; } - virtual int get_size(); - virtual void build_sampler(); - virtual void to_buffer(char *buffer); - virtual void recover_from_buffer(char *buffer); - virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } - static GraphNodeType get_graph_node_type(std::string &str) { +class GraphEdge: public WeightedObject{ +public: + double weight; + uint64_t id; GraphNodeType type; - if (str == "user") - type = GraphNodeType::user; - else if (str == "item") - type = GraphNodeType::item; - else if (str == "query") - type = GraphNodeType::query; - else - type = GraphNodeType::unknown; + GraphEdge(){ + } + GraphEdge(uint64_t id, GraphNodeType type,double weight):weight(weight),id(id),type(type){ + } +}; +class GraphNode{ +public: +GraphNode(){ + sampler = NULL; +} +GraphNode(uint64_t id,GraphNodeType type,std::string feature):id(id),type(type),feature(feature),sampler(NULL){ +} +virtual ~GraphNode() {} +static int enum_size,id_size,int_size,double_size; +uint64_t get_id(){ + return id; +} +void set_id(uint64_t id){ + this->id = id; +} +GraphNodeType get_graph_node_type(){ return type; - } - std::vector sample_k(int k) { +} +void set_graph_node_type(GraphNodeType type){ + this->type = type; +} +void set_feature(std::string feature){ + this->feature = feature; +} +std::string get_feature(){ + return feature; +} +virtual int get_size(); +virtual void build_sampler(); +virtual void to_buffer(char* buffer); +virtual void recover_from_buffer(char* buffer); +virtual void add_edge(GraphEdge * edge){ + edges.push_back(edge); +} +static GraphNodeType get_graph_node_type(std::string &str){ + GraphNodeType type; + if(str == "user") + type = GraphNodeType::user; + else if(str == "item") + type = GraphNodeType::item; + else if(str == "query") + type = GraphNodeType:: query; + else + type = GraphNodeType::unknown; + return type; +} +std::vector sample_k(int k){ std::vector v; - if (sampler != NULL) { - auto res = sampler->sample_k(k); - for (auto x : res) { - v.push_back((GraphEdge *)x); - } + if(sampler != NULL){ + auto res = sampler->sample_k(k); + for(auto x: res){ + v.push_back((GraphEdge *)x); + } } return v; - } - - protected: - uint64_t id; - GraphNodeType type; - std::string feature; - WeightedSampler *sampler; - std::vector edges; -}; + } +protected: +uint64_t id; +GraphNodeType type; +std::string feature; +WeightedSampler *sampler; +std::vector edges; +}; } +} \ No newline at end of file diff --git a/paddle/fluid/distributed/table/table.cc b/paddle/fluid/distributed/table/table.cc index dbdac32e3597c..7ee9925b45fbe 100644 --- a/paddle/fluid/distributed/table/table.cc +++ b/paddle/fluid/distributed/table/table.cc @@ -20,11 +20,11 @@ #include "paddle/fluid/distributed/common/registerer.h" #include "paddle/fluid/distributed/table/common_dense_table.h" -#include "paddle/fluid/distributed/table/common_graph_table.h" #include "paddle/fluid/distributed/table/common_sparse_table.h" #include "paddle/fluid/distributed/table/sparse_geo_table.h" #include "paddle/fluid/distributed/table/tensor_accessor.h" #include "paddle/fluid/distributed/table/tensor_table.h" +#include "paddle/fluid/distributed/table/common_graph_table.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 77f5c3557f6cf..bf651c64d00e9 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -21,12 +21,12 @@ #include #include #include "paddle/fluid/distributed/table/accessor.h" -#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/place.h" #include "paddle/fluid/string/string_helper.h" +#include "paddle/fluid/distributed/table/graph_node.h" namespace paddle { namespace distributed { @@ -88,16 +88,12 @@ class Table { } // only for graph table - virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, - int total_size, char *&buffer, - int &actual_size) { + virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, int total_size, char* &buffer,int &actual_size){ return 0; } // only for graph table - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, - int sampe_size, char *&buffer, - int &actual_size) { - return 0; + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type,int sampe_size, char* &buffer, int &actual_size){ + return 0; } virtual int32_t pour() { return 0; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index a2d2cfcaab8df..cc61c6caa0267 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -1,80 +1,60 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { -void WeightedSampler::build(WeightedObject **v, int start, int end) { - count = 0; - if (start + 1 == end) { - left = right = NULL; - weight = v[start]->get_weight(); - object = v[start]; - count = 1; + void WeightedSampler::build(WeightedObject** v,int start,int end){ + count = 0; + if(start + 1 == end){ + left = right = NULL; + weight = v[start]->get_weight(); + object = v[start]; + count = 1; - } else { - left = new WeightedSampler(); - right = new WeightedSampler(); - left->build(v, start, start + (end - start) / 2); - right->build(v, start + (end - start) / 2, end); - weight = left->weight + right->weight; - count = left->count + right->count; - } -} -vector WeightedSampler::sample_k(int k) { - if (k > count) { - k = count; - } - vector sample_result; - double subtract; - unordered_map subtract_weight_map; - unordered_map subtract_count_map; - while (k--) { - double query_weight = rand() % 100000 / 100000.0; - query_weight *= weight - subtract_weight_map[this]; - sample_result.push_back(sample(query_weight, subtract_weight_map, - subtract_count_map, subtract)); - } - return sample_result; -} -WeightedObject *WeightedSampler::sample( - double query_weight, - unordered_map &subtract_weight_map, - unordered_map &subtract_count_map, - double &subtract) { - if (left == NULL) { - subtract_weight_map[this] = weight; - subtract = weight; - subtract_count_map[this] = 1; - return object; - } - int left_count = left->count - subtract_count_map[left]; - int right_count = right->count - subtract_count_map[right]; - double left_subtract = subtract_weight_map[left]; - WeightedObject *return_id; - if (right_count == 0 || - left_count > 0 && left->weight - left_subtract >= query_weight) { - return_id = left->sample(query_weight, subtract_weight_map, - subtract_count_map, subtract); - } else { - return_id = - right->sample(query_weight - (left->weight - left_subtract), - subtract_weight_map, subtract_count_map, subtract); - } - subtract_weight_map[this] += subtract; - subtract_count_map[this]++; - return return_id; -} + } else { + left = new WeightedSampler(); + right = new WeightedSampler(); + left->build(v,start, start + (end - start)/2); + right->build(v,start + (end - start)/2, end); + weight = left->weight + right->weight; + count = left->count + right->count; + } + } + vector WeightedSampler::sample_k(int k){ + if(k > count){ + k = count; + } + vector sample_result; + double subtract; + unordered_map subtract_weight_map; + unordered_map subtract_count_map; + while(k--){ + double query_weight = rand() % 100000/100000.0; + query_weight *= weight - subtract_weight_map[this]; + sample_result.push_back(sample(query_weight, subtract_weight_map,subtract_count_map, subtract)); + } + return sample_result; + + } + WeightedObject * WeightedSampler::sample(double query_weight, unordered_map &subtract_weight_map, unordered_map &subtract_count_map, double &subtract){ + if(left == NULL){ + subtract_weight_map[this] = weight; + subtract = weight; + subtract_count_map[this] = 1; + return object; + } + int left_count = left->count - subtract_count_map[left]; + int right_count = right->count - subtract_count_map[right]; + double left_subtract = subtract_weight_map[left]; + WeightedObject * return_id; + if(right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight){ + return_id = left->sample(query_weight, subtract_weight_map,subtract_count_map, subtract); + } else { + return_id = right->sample(query_weight - (left->weight - left_subtract),subtract_weight_map,subtract_count_map, subtract); + } + subtract_weight_map[this] += subtract; + subtract_count_map[this]++; + return return_id; + + } } } + diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 2c9b428ffd90a..838a034d2b4f6 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -1,54 +1,47 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - #pragma once -#include -#include -#include +#include +#include +#include using namespace std; namespace paddle { namespace distributed { -class WeightedObject { - public: - WeightedObject() {} - virtual ~WeightedObject() {} - virtual unsigned long long get_id() { return id; } - virtual double get_weight() { return weight; } +class WeightedObject{ +public: + WeightedObject(){ - virtual void set_id(unsigned long long id) { this->id = id; } - virtual void set_weight(double weight) { this->weight = weight; } + } + virtual ~WeightedObject(){ - private: - unsigned long long id; - double weight; -}; + } + virtual unsigned long long get_id(){ + return id; + } + virtual double get_weight(){ + return weight; + } + + virtual void set_id(unsigned long long id){ + this->id = id; + } + virtual void set_weight(double weight){ + this->weight = weight; + } +private: + unsigned long long id; + double weight; -class WeightedSampler { - public: - WeightedSampler *left, *right; - WeightedObject *object; - int count; - double weight; - void build(WeightedObject **v, int start, int end); - vector sample_k(int k); +}; - private: - WeightedObject *sample( - double query_weight, - unordered_map &subtract_weight_map, - unordered_map &subtract_count_map, - double &subtract); +class WeightedSampler{ +public: + WeightedSampler* left,*right; + WeightedObject* object; + int count; + double weight; + void build(WeightedObject** v,int start,int end); + vector sample_k(int k); +private: + WeightedObject* sample(double query_weight, unordered_map &subtract_weight_map, unordered_map &subtract_count_map, double &subtract); }; } -} +} \ No newline at end of file diff --git a/paddle/fluid/distributed/test/CMakeLists.txt b/paddle/fluid/distributed/test/CMakeLists.txt index fb352fdc7f509..73cb33a2e0f30 100644 --- a/paddle/fluid/distributed/test/CMakeLists.txt +++ b/paddle/fluid/distributed/test/CMakeLists.txt @@ -17,4 +17,4 @@ set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRI cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -#cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) +#cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) \ No newline at end of file diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index cfb9411636106..c335ce796878f 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -14,10 +14,10 @@ limitations under the License. */ #include #include // NOLINT -#include -#include #include #include // NOLINT +#include +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -47,29 +47,27 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = { - std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), - std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; +std::string nodes[] = {std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"),std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; char file_name[] = "nodes.txt"; -void prepare_file(char file_name[]) { +void prepare_file(char file_name[]){ ofstream ofile; - ofile.open(file_name); - for (auto x : nodes) { - ofile << x << endl; + ofile.open(file_name); + for(auto x: nodes){ + ofile<mutable_common(); } -distributed::GraphNodeType get_graph_node_type(std::string str) { - distributed::GraphNodeType type; - if (str == "user") - type = distributed::GraphNodeType::user; - else if (str == "item") - type = distributed::GraphNodeType::item; - else if (str == "query") - type = distributed::GraphNodeType::query; - else - type = distributed::GraphNodeType::unknown; - return type; +distributed::GraphNodeType get_graph_node_type(std::string str){ + distributed::GraphNodeType type; + if(str == "user") + type = distributed::GraphNodeType::user; + else if(str == "item") + type = distributed::GraphNodeType::item; + else if(str == "query") + type = distributed::GraphNodeType:: query; + else + type = distributed::GraphNodeType::unknown; + return type; } ::paddle::distributed::PSParameter GetServerProto() { @@ -120,7 +118,7 @@ ::paddle::distributed::PSParameter GetServerProto() { ::paddle::distributed::PSParameter GetWorkerProto() { ::paddle::distributed::PSParameter worker_fleet_desc; - worker_fleet_desc.set_shard_num(127); + worker_fleet_desc.set_shard_num(127); ::paddle::distributed::WorkerParameter* worker_proto = worker_fleet_desc.mutable_worker_param(); @@ -152,7 +150,7 @@ ::paddle::distributed::PSParameter GetWorkerProto() { /*-------------------------------------------------------------------------*/ -std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; +std::string ip_ = "127.0.0.1",ip2 = "127.0.0.1"; uint32_t port_ = 4209, port2 = 4210; std::vector host_sign_list_; @@ -166,31 +164,31 @@ void RunServer() { ::paddle::distributed::PSParameter server_proto = GetServerProto(); auto _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&host_sign_list_, 2); // test + _ps_env.set_ps_servers(&host_sign_list_, 2); //test pserver_ptr_ = std::shared_ptr( paddle::distributed::PSServerFactory::create(server_proto)); std::vector empty_vec; framework::ProgramDesc empty_prog; empty_vec.push_back(empty_prog); pserver_ptr_->configure(server_proto, _ps_env, 0, empty_vec); - LOG(INFO) << "first server, run start(ip,port)"; +LOG(INFO) << "first server, run start(ip,port)"; pserver_ptr_->start(ip_, port_); LOG(INFO) << "init first server Done"; } -void RunServer2() { +void RunServer2(){ LOG(INFO) << "init second server"; - ::paddle::distributed::PSParameter server_proto2 = GetServerProto(); + ::paddle::distributed::PSParameter server_proto2 = GetServerProto(); auto _ps_env2 = paddle::distributed::PaddlePSEnvironment(); - _ps_env2.set_ps_servers(&host_sign_list_, 2); // test + _ps_env2.set_ps_servers(&host_sign_list_, 2); //test pserver_ptr2 = std::shared_ptr( paddle::distributed::PSServerFactory::create(server_proto2)); std::vector empty_vec2; framework::ProgramDesc empty_prog2; empty_vec2.push_back(empty_prog2); pserver_ptr2->configure(server_proto2, _ps_env2, 1, empty_vec2); - pserver_ptr2->start(ip2, port2); + pserver_ptr2->start(ip2, port2); } void RunClient(std::map>& @@ -212,13 +210,13 @@ void RunBrpcPushSparse() { auto ph_host = paddle::distributed::PSHost(ip_, port_, 0); host_sign_list_.push_back(ph_host.serialize_to_string()); - // test-start + //test-start auto ph_host2 = paddle::distributed::PSHost(ip2, port2, 1); host_sign_list_.push_back(ph_host2.serialize_to_string()); - // test-end + //test-end // Srart Server std::thread server_thread(RunServer); - std::thread server_thread2(RunServer2); + std::thread server_thread2(RunServer2); sleep(1); std::map> dense_regions; @@ -230,86 +228,80 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ LOG(INFO) << "Run pull_sparse_param"; - auto pull_status = - worker_ptr_->load(0, std::string(file_name), std::string("")); + auto pull_status = worker_ptr_->load(0,std::string(file_name),std::string("")); pull_status.wait(); vector v; - pull_status = - worker_ptr_->sample(0, 37, get_graph_node_type(string("user")), v); + pull_status = worker_ptr_->sample(0,37,get_graph_node_type(string("user")),v); pull_status.wait(); - for (auto g : v) { - std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + for(auto g:v){ + std::cout<check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { - // ret = -1; - // break; - // } - // VLOG(0)<check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_PARAM); - // } - // closure->set_promise_value(ret); - // }); - // VLOG(0)<<"begin to push_sparse_param"; - // auto push_status = worker_ptr_->push_sparse_param( - // 0, fea_keys.data(), (const float**)fea_value_ptr.data(), - // fea_keys.size(), - // closure_push_param); - // push_status.wait(); - - // auto pull_param_status = worker_ptr_->pull_sparse( - // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); - // pull_param_status.wait(); - - // for (size_t idx = 0; idx < tensor->numel(); ++idx) { - // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); - // } - // LOG(INFO) << "first stage done"; - // /*-----------------------Test Push - // Grad----------------------------------*/ - - // paddle::distributed::DownpourBrpcClosure* closure_push_grad = - // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { - // int ret = 0; - // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; - // for (size_t i = 0; i < 2; ++i) { - // if (closure->check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { - // ret = -1; - // break; - // } - // } - // closure->set_promise_value(ret); - // }); - - // LOG(INFO) << "Run pull_sparse_grad"; - // std::vector push_g_vec; - // for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { - // push_g_vec.push_back(tensor->data() + i * 10); - // } - // auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( - // 0, fea_keys.data(), (const float**)push_g_vec.data(), - // fea_keys.size(), - // closure_push_grad); - // push_grad_status.wait(); - - // auto pull_update_status = worker_ptr_->pull_sparse( - // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); - // pull_update_status.wait(); - - // for (size_t idx = 0; idx < tensor->numel(); ++idx) { - // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); - // } +// LOG(INFO) << "Run push_sparse_param"; +// paddle::distributed::DownpourBrpcClosure* closure_push_param = +// new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { +// int ret = 0; +// auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; +// for (size_t i = 0; i < 2; ++i) { +// LOG(INFO) << "check response" <check_response( +// i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { +// ret = -1; +// break; +// } +// VLOG(0)<check_response( +// i, paddle::distributed::PS_PUSH_SPARSE_PARAM); +// } +// closure->set_promise_value(ret); +// }); +// VLOG(0)<<"begin to push_sparse_param"; +// auto push_status = worker_ptr_->push_sparse_param( +// 0, fea_keys.data(), (const float**)fea_value_ptr.data(), fea_keys.size(), +// closure_push_param); +// push_status.wait(); + +// auto pull_param_status = worker_ptr_->pull_sparse( +// fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); +// pull_param_status.wait(); + +// for (size_t idx = 0; idx < tensor->numel(); ++idx) { +// EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); +// } +// LOG(INFO) << "first stage done"; +// /*-----------------------Test Push Grad----------------------------------*/ + +// paddle::distributed::DownpourBrpcClosure* closure_push_grad = +// new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { +// int ret = 0; +// auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; +// for (size_t i = 0; i < 2; ++i) { +// if (closure->check_response( +// i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { +// ret = -1; +// break; +// } +// } +// closure->set_promise_value(ret); +// }); + +// LOG(INFO) << "Run pull_sparse_grad"; +// std::vector push_g_vec; +// for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { +// push_g_vec.push_back(tensor->data() + i * 10); +// } +// auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( +// 0, fea_keys.data(), (const float**)push_g_vec.data(), fea_keys.size(), +// closure_push_grad); +// push_grad_status.wait(); + +// auto pull_update_status = worker_ptr_->pull_sparse( +// fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); +// pull_update_status.wait(); + +// for (size_t idx = 0; idx < tensor->numel(); ++idx) { +// EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); +// } LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); @@ -320,8 +312,8 @@ void RunBrpcPushSparse() { testGraphToBuffer(); } -void testGraphToBuffer() { - ::paddle::distributed::GraphNode s, s1; +void testGraphToBuffer(){ + ::paddle::distributed::GraphNode s,s1; s.set_feature("hhhh"); s.set_id(65); s.set_graph_node_type(::paddle::distributed::GraphNodeType(0)); @@ -330,8 +322,10 @@ void testGraphToBuffer() { s.to_buffer(str); s1.recover_from_buffer(str); ASSERT_EQ(s.get_id(), s1.get_id()); - ASSERT_EQ((int)s.get_graph_node_type(), (int)s1.get_graph_node_type()); - VLOG(0) << s.get_feature(); - VLOG(0) << s1.get_feature(); + ASSERT_EQ((int)s.get_graph_node_type(),(int)s1.get_graph_node_type()); + VLOG(0)< Date: Sat, 27 Feb 2021 05:51:30 +0000 Subject: [PATCH 003/106] fix dependency error --- paddle/fluid/distributed/ps.proto | 1 + .../fluid/distributed/service/CMakeLists.txt | 59 +++- .../distributed/service/brpc_ps_client.cc | 2 +- .../distributed/service/brpc_ps_client.h | 35 ++- .../distributed/service/graph_brpc_client.cc | 205 +++++++------- .../distributed/service/graph_brpc_client.h | 45 ++-- .../distributed/service/graph_brpc_server.cc | 118 ++++---- .../distributed/service/graph_brpc_server.h | 34 ++- .../distributed/service/graph_py_service.cc | 46 ++++ .../distributed/service/graph_py_service.h | 148 ++++++++++ .../distributed/service/graph_service.cc | 162 ----------- paddle/fluid/distributed/service/ps_client.cc | 5 +- paddle/fluid/distributed/service/ps_client.h | 37 +-- .../fluid/distributed/service/sendrecv.proto | 2 +- .../distributed/table/common_graph_table.cc | 253 ++++++++++-------- .../distributed/table/common_graph_table.h | 164 +++++++----- paddle/fluid/distributed/table/graph_node.cc | 75 +++--- paddle/fluid/distributed/table/graph_node.h | 140 +++++----- paddle/fluid/distributed/table/table.cc | 4 +- paddle/fluid/distributed/table/table.h | 14 +- .../distributed/table/weighted_sampler.cc | 128 +++++---- .../distributed/table/weighted_sampler.h | 82 +++--- paddle/fluid/distributed/test/CMakeLists.txt | 25 +- .../fluid/distributed/test/graph_node_test.cc | 240 +++++++++-------- 24 files changed, 1141 insertions(+), 883 deletions(-) create mode 100644 paddle/fluid/distributed/service/graph_py_service.cc create mode 100644 paddle/fluid/distributed/service/graph_py_service.h delete mode 100644 paddle/fluid/distributed/service/graph_service.cc diff --git a/paddle/fluid/distributed/ps.proto b/paddle/fluid/distributed/ps.proto index 862ae4a504d9b..665098cd7d547 100644 --- a/paddle/fluid/distributed/ps.proto +++ b/paddle/fluid/distributed/ps.proto @@ -40,6 +40,7 @@ message PSParameter { optional ServerParameter server_param = 102; repeated DownpourTrainerParameter trainer_param = 301; optional FsClientParameter fs_client_param = 501; + optional int32 shard_num = 502; } message WorkerParameter { diff --git a/paddle/fluid/distributed/service/CMakeLists.txt b/paddle/fluid/distributed/service/CMakeLists.txt index 459e194f76dae..b211e474d00a9 100644 --- a/paddle/fluid/distributed/service/CMakeLists.txt +++ b/paddle/fluid/distributed/service/CMakeLists.txt @@ -1,3 +1,48 @@ +# set(BRPC_SRCS ps_client.cc server.cc) +# set_source_files_properties(${BRPC_SRCS}) + +# set(BRPC_DEPS brpc ssl crypto protobuf gflags glog zlib leveldb snappy gflags glog device_context) + +# brpc_library(sendrecv_rpc SRCS +# ${BRPC_SRCS} +# PROTO sendrecv.proto +# DEPS ${BRPC_DEPS} ) + +# set_property(GLOBAL PROPERTY RPC_DEPS sendrecv_rpc ${BRPC_DEPS} string_helper) + +# get_property(RPC_DEPS GLOBAL PROPERTY RPC_DEPS) + +# set_source_files_properties(communicator.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(service.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(brpc_ps_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(brpc_ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) + +# set_source_files_properties(brpc_utils.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(heter_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(heter_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) + +# set_source_files_properties(client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(graph_brpc_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# set_source_files_properties(graph_brpc_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) + +# cc_library(downpour_server SRCS brpc_ps_server.cc graph_brpc_server.cc DEPS boost eigen3 table ${RPC_DEPS}) +# cc_library(downpour_client SRCS brpc_ps_client.cc graph_brpc_client.cc DEPS boost eigen3 table ${RPC_DEPS}) + +# cc_library(downpour_server SRCS brpc_ps_server.cc DEPS boost eigen3 table ${RPC_DEPS}) +# cc_library(downpour_client SRCS brpc_ps_client.cc DEPS boost eigen3 table ${RPC_DEPS}) + +# cc_library(client SRCS ps_client.cc DEPS downpour_client boost ${RPC_DEPS}) +# cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) + +# cc_library(communicator SRCS communicator.cc DEPS scope client boost table math_function selected_rows_functor ${RPC_DEPS}) +# cc_library(ps_service SRCS service.cc DEPS communicator client server boost ${RPC_DEPS}) + +# cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) +# cc_library(heter_server SRCS heter_server.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) +# cc_library(heter_client SRCS heter_client.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) + set(BRPC_SRCS ps_client.cc server.cc) set_source_files_properties(${BRPC_SRCS}) @@ -26,9 +71,15 @@ set_source_files_properties(ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_C set_source_files_properties(server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(graph_brpc_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(graph_brpc_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) + +cc_library(downpour_server SRCS graph_brpc_server.cc brpc_ps_server.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) +cc_library(downpour_client SRCS graph_brpc_client.cc brpc_ps_client.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) -cc_library(downpour_server SRCS brpc_ps_server.cc graph_brpc_server.cc DEPS boost eigen3 table ${RPC_DEPS}) -cc_library(downpour_client SRCS brpc_ps_client.cc graph_brpc_client.cc DEPS boost eigen3 table ${RPC_DEPS}) +# cc_library(downpour_server1 SRCS graph_brpc_server.cc DEPS downpour_server ${RPC_DEPS}) +# cc_library(downpour_client1 SRCS graph_brpc_client.cc DEPS downpour_client ${RPC_DEPS}) +#cc_library(downpour_server SRCS brpc_ps_server.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) +#cc_library(downpour_client SRCS brpc_ps_client.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) cc_library(client SRCS ps_client.cc DEPS downpour_client boost ${RPC_DEPS}) cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) @@ -36,6 +87,8 @@ cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) cc_library(communicator SRCS communicator.cc DEPS scope client boost table math_function selected_rows_functor ${RPC_DEPS}) cc_library(ps_service SRCS service.cc DEPS communicator client server boost ${RPC_DEPS}) -cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) cc_library(heter_server SRCS heter_server.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) cc_library(heter_client SRCS heter_client.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) + +set_source_files_properties(graph_py_service.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(graph_py_service SRCS graph_py_service.cc DEPS ps_service) diff --git a/paddle/fluid/distributed/service/brpc_ps_client.cc b/paddle/fluid/distributed/service/brpc_ps_client.cc index 163526fe3b28c..5c226e6a0ddd0 100644 --- a/paddle/fluid/distributed/service/brpc_ps_client.cc +++ b/paddle/fluid/distributed/service/brpc_ps_client.cc @@ -990,4 +990,4 @@ int32_t BrpcPsClient::recv_and_save_table(const uint64_t table_id, } } // namespace distributed -} // namespace paddle +} // namespace paddle \ No newline at end of file diff --git a/paddle/fluid/distributed/service/brpc_ps_client.h b/paddle/fluid/distributed/service/brpc_ps_client.h index 8f9d2653864d1..5b6911564419d 100644 --- a/paddle/fluid/distributed/service/brpc_ps_client.h +++ b/paddle/fluid/distributed/service/brpc_ps_client.h @@ -170,9 +170,22 @@ class BrpcPsClient : public PSClient { virtual int32_t recv_and_save_table(const uint64_t table_id, const std::string &path); - private: + protected: + virtual size_t get_server_nums() { return _server_channels.size(); } + inline brpc::Channel *get_sparse_channel(size_t server_id) { + return _server_channels[server_id][0].get(); + } + inline brpc::Channel *get_dense_channel(size_t server_id) { + return _server_channels[server_id][1].get(); + } + inline brpc::Channel *get_cmd_channel(size_t server_id) { + return _server_channels[server_id][2].get(); + } virtual int32_t initialize() override; + private: + // virtual int32_t initialize() override; + inline uint32_t dense_dim_per_shard(uint32_t dense_dim_total, uint32_t shard_num) { return dense_dim_total / shard_num + 1; @@ -184,15 +197,15 @@ class BrpcPsClient : public PSClient { std::future send_save_cmd(uint32_t table_id, int cmd_id, const std::vector ¶m); - inline brpc::Channel *get_sparse_channel(size_t server_id) { - return _server_channels[server_id][0].get(); - } - inline brpc::Channel *get_dense_channel(size_t server_id) { - return _server_channels[server_id][1].get(); - } - inline brpc::Channel *get_cmd_channel(size_t server_id) { - return _server_channels[server_id][2].get(); - } + // inline brpc::Channel *get_sparse_channel(size_t server_id) { + // return _server_channels[server_id][0].get(); + // } + // inline brpc::Channel *get_dense_channel(size_t server_id) { + // return _server_channels[server_id][1].get(); + // } + // inline brpc::Channel *get_cmd_channel(size_t server_id) { + // return _server_channels[server_id][2].get(); + // } bool _running = false; bool _flushing = false; @@ -220,7 +233,7 @@ class BrpcPsClient : public PSClient { size_t num, void *done) override; - virtual size_t get_server_nums() { return _server_channels.size(); } + // virtual size_t get_server_nums() { return _server_channels.size(); } private: int32_t start_client_service(); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 05254acf8a104..e179f55c49037 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -1,3 +1,17 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + #include #include #include @@ -6,115 +20,118 @@ #include "Eigen/Dense" #include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/string/string_helper.h" -#include "paddle/fluid/distributed/service/graph_brpc_client.h" namespace paddle { namespace distributed { -int GraphBrpcClient::get_server_index_by_id(uint64_t id){ - int shard_num = get_shard_num(); - size_t server_size = get_server_nums(); - int shard_per_server = shard_num % server_size == 0 ? shard_num / server_size : shard_num / server_size + 1; - return id % shard_num / shard_per_server; + +int GraphBrpcClient::get_server_index_by_id(uint64_t id) { + int shard_num = get_shard_num(); + int shard_per_server = shard_num % server_size == 0 + ? shard_num / server_size + : shard_num / server_size + 1; + return id % shard_num / shard_per_server; } -//char* &buffer,int &actual_size -std::future GraphBrpcClient::sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,vector &res){ +// char* &buffer,int &actual_size +std::future GraphBrpcClient::sample(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, + int sample_size, + std::vector &res) { int server_index = get_server_index_by_id(node_id); - DownpourBrpcClosure *closure = new DownpourBrpcClosure( - 1, [&](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { - ret = -1; - } else { - VLOG(0) << "check sample response: " <<" "<check_response(0, PS_GRAPH_SAMPLE); - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), - bytes_size); - int start = 0; - while(start < bytes_size){ - GraphNode node; - node.recover_from_buffer(buffer + start); - start += node.get_size(); - res.push_back(node); - } - } - closure->set_promise_value(ret); - }); - auto promise = std::make_shared>(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(0, PS_GRAPH_SAMPLE); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + int start = 0; + while (start < bytes_size) { + GraphNode node; + node.recover_from_buffer(buffer + start); + start += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); closure->add_promise(promise); - std::future fut = promise->get_future(); ; - closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; - closure->request(0)->add_params((char *)&node_id, - sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, - sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), - closure->response(0), closure); - + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); return fut; } -std:: future GraphBrpcClient::pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int size, vector &res){ +std::future GraphBrpcClient::pull_graph_list( + uint32_t table_id, uint64_t node_id, GraphNodeType type, int start, + int size, std::vector &res) { int server_index = get_server_index_by_id(node_id); - DownpourBrpcClosure *closure = new DownpourBrpcClosure( - 1, [&](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { - ret = -1; - } else { - VLOG(0) << "check sample response: " <<" "<check_response(0, PS_PULL_GRAPH_LIST); - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), - bytes_size); - int index = 0; - while(index < bytes_size){ - GraphNode node; - node.recover_from_buffer(buffer + index); - index += node.get_size(); - res.push_back(node); - } - } - closure->set_promise_value(ret); - }); - auto promise = std::make_shared>(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + int index = 0; + while (index < bytes_size) { + GraphNode node; + node.recover_from_buffer(buffer + index); + index += node.get_size(); + res.push_back(node); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); closure->add_promise(promise); - std::future fut = promise->get_future(); ; - closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; - closure->request(0)->add_params((char *)&node_id, - sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, - sizeof(int)); - closure->request(0)->add_params((char *)&start, - sizeof(int)); - closure->request(0)->add_params((char *)&size, - sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), - closure->response(0), closure); - return fut; + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + int type_int = (int)type; + closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); + closure->request(0)->add_params((char *)&type_int, sizeof(int)); + closure->request(0)->add_params((char *)&start, sizeof(int)); + closure->request(0)->add_params((char *)&size, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); + return fut; } -int32_t GraphBrpcClient::initialize(){ - set_shard_num(_config.shard_num()); - return BrpcPsClient::initialize(); +int32_t GraphBrpcClient::initialize() { + set_shard_num(_config.shard_num()); + BrpcPsClient::initialize(); + server_size = get_server_nums(); + return 0; } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 1557e445d66af..0efb6bbabaec1 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -21,37 +21,36 @@ #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" +#include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/ps_client.h" +#include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/framework/tensor_util.h" -#include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/table/graph_node.h" -#include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { -class GraphBrpcClient: public BrpcPsClient{ -public: - GraphBrpcClient(){ - - } - virtual ~GraphBrpcClient(){ - - } - virtual std::future sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,vector &res); - virtual std::future pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int end, vector &res); - virtual int32_t initialize(); - int get_shard_num(){ - return shard_num; - } - void set_shard_num(int shard_num){ - this->shard_num = shard_num; - } - int get_server_index_by_id(uint64_t id); -private: - int shard_num; +class GraphBrpcClient : public BrpcPsClient { + public: + GraphBrpcClient() {} + virtual ~GraphBrpcClient() {} + virtual std::future sample(uint32_t table_id, uint64_t node_id, + GraphNodeType type, int sample_size, + std::vector &res); + virtual std::future pull_graph_list(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, int start, + int end, + std::vector &res); + virtual int32_t initialize(); + int get_shard_num() { return shard_num; } + void set_shard_num(int shard_num) { this->shard_num = shard_num; } + int get_server_index_by_id(uint64_t id); + + private: + int shard_num; + size_t server_size; }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 3239648ff1567..d8c0de5f476cd 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -1,14 +1,26 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/service/graph_brpc_server.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" #include // NOLINT +#include "butil/endpoint.h" #include "iomanip" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/platform/profiler.h" -#include "paddle/fluid/distributed/table/graph_node.h" -#include "Eigen/Dense" -#include "butil/endpoint.h" namespace paddle { namespace distributed { @@ -76,13 +88,15 @@ int32_t GraphBrpcService::initialize() { _service_handler_map[PS_LOAD_ONE_TABLE] = &GraphBrpcService::load_one_table; _service_handler_map[PS_LOAD_ALL_TABLE] = &GraphBrpcService::load_all_table; - _service_handler_map[PS_PRINT_TABLE_STAT] = &GraphBrpcService::print_table_stat; + _service_handler_map[PS_PRINT_TABLE_STAT] = + &GraphBrpcService::print_table_stat; _service_handler_map[PS_BARRIER] = &GraphBrpcService::barrier; _service_handler_map[PS_START_PROFILER] = &GraphBrpcService::start_profiler; _service_handler_map[PS_STOP_PROFILER] = &GraphBrpcService::stop_profiler; - _service_handler_map[PS_PULL_GRAPH_LIST] =&GraphBrpcService::pull_graph_list; - _service_handler_map[PS_GRAPH_SAMPLE] = &GraphBrpcService::graph_random_sample; + _service_handler_map[PS_PULL_GRAPH_LIST] = &GraphBrpcService::pull_graph_list; + _service_handler_map[PS_GRAPH_SAMPLE] = + &GraphBrpcService::graph_random_sample; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); @@ -115,9 +129,9 @@ int32_t GraphBrpcService::initialize_shard_info() { } void GraphBrpcService::service(google::protobuf::RpcController *cntl_base, - const PsRequestMessage *request, - PsResponseMessage *response, - google::protobuf::Closure *done) { + const PsRequestMessage *request, + PsResponseMessage *response, + google::protobuf::Closure *done) { brpc::ClosureGuard done_guard(done); std::string log_label("ReceiveCmd-"); if (!request->has_table_id()) { @@ -145,11 +159,9 @@ void GraphBrpcService::service(google::protobuf::RpcController *cntl_base, } } - - int32_t GraphBrpcService::barrier(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 1) { @@ -165,12 +177,10 @@ int32_t GraphBrpcService::barrier(Table *table, const PsRequestMessage &request, return 0; } - - int32_t GraphBrpcService::print_table_stat(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) std::pair ret = table->print_table_stat(); paddle::framework::BinaryArchive ar; @@ -182,9 +192,9 @@ int32_t GraphBrpcService::print_table_stat(Table *table, } int32_t GraphBrpcService::load_one_table(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 2) { set_response_code( @@ -200,9 +210,9 @@ int32_t GraphBrpcService::load_one_table(Table *table, } int32_t GraphBrpcService::load_all_table(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { auto &table_map = *(_server->table()); for (auto &itr : table_map) { if (load_one_table(itr.second.get(), request, response, cntl) != 0) { @@ -213,13 +223,10 @@ int32_t GraphBrpcService::load_all_table(Table *table, return 0; } - - - int32_t GraphBrpcService::stop_server(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { auto *p_server = _server; std::thread t_stop([p_server]() { p_server->stop(); @@ -230,46 +237,48 @@ int32_t GraphBrpcService::stop_server(Table *table, } int32_t GraphBrpcService::stop_profiler(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { platform::DisableProfiler(platform::EventSortingKey::kDefault, string::Sprintf("server_%s_profile", _rank)); return 0; } int32_t GraphBrpcService::start_profiler(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { platform::EnableProfiler(platform::ProfilerState::kCPU); return 0; } -int32_t GraphBrpcService::pull_graph_list(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, brpc::Controller *cntl){ +int32_t GraphBrpcService::pull_graph_list(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 2) { - set_response_code( - response, -1, - "pull_graph_list request requires at least 2 arguments"); + set_response_code(response, -1, + "pull_graph_list request requires at least 2 arguments"); return 0; } uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); - std::string type_str = request.params(1); - GraphNodeType type = GraphNode::get_graph_node_type(type_str); + std::string type_str = request.params(1); + GraphNodeType type = GraphNode::get_graph_node_type(type_str); int start = *(int *)(request.params(2).c_str()); int size = *(int *)(request.params(3).c_str()); std::vector res_data; char *buffer; int actual_size; - table->pull_graph_list(node_id, type, start, size, buffer,actual_size); - cntl->response_attachment().append(buffer, - actual_size); + table->pull_graph_list(node_id, type, start, size, buffer, actual_size); + cntl->response_attachment().append(buffer, actual_size); return 0; } -int32_t GraphBrpcService::graph_random_sample(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, brpc::Controller *cntl){ +int32_t GraphBrpcService::graph_random_sample(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 3) { set_response_code( @@ -279,17 +288,14 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, const PsRequestMessa } uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); std::string type_str = request.params(1); - int sample_size = *(uint64_t *)(request.params(2).c_str()); + int sample_size = *(uint64_t *)(request.params(2).c_str()); GraphNodeType type = GraphNode::get_graph_node_type(type_str); char *buffer; int actual_size; - table->random_sample(node_id,type,sample_size,buffer, actual_size); - cntl->response_attachment().append(buffer, - actual_size); - return 0; - } - - + table->random_sample(node_id, type, sample_size, buffer, actual_size); + cntl->response_attachment().append(buffer, actual_size); + return 0; +} } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index 54480e806036b..b4056692f2ccf 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -1,3 +1,17 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + #pragma once #include "brpc/channel.h" @@ -6,8 +20,8 @@ #include #include -#include "paddle/fluid/distributed/service/server.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" +#include "paddle/fluid/distributed/service/server.h" namespace paddle { namespace distributed { @@ -51,7 +65,6 @@ typedef int32_t (GraphBrpcService::*serviceFunc)( Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); - class GraphBrpcService : public PsBaseService { public: virtual int32_t initialize() override; @@ -62,12 +75,13 @@ class GraphBrpcService : public PsBaseService { ::google::protobuf::Closure *done) override; protected: - std::unordered_map _service_handler_map; + std::unordered_map _service_handler_map; int32_t initialize_shard_info(); - int32_t pull_graph_list(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, brpc::Controller *cntl); -int32_t graph_random_sample(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, brpc::Controller *cntl); + int32_t pull_graph_list(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t graph_random_sample(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, @@ -94,9 +108,11 @@ int32_t graph_random_sample(Table *table, const PsRequestMessage &request, // public: // virtual int32_t initialize() override; // int32_t pull_graph_list(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller *cntl); +// PsResponseMessage &response, brpc::Controller +// *cntl); // int32_t graph_random_sample(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller *cntl); +// PsResponseMessage &response, brpc::Controller +// *cntl); // }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc new file mode 100644 index 0000000000000..e8b0948fd834a --- /dev/null +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -0,0 +1,46 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/service/graph_py_service.h" +namespace paddle { +namespace distributed { +std::vector graph_service::split(std::string &str, + const char pattern) { + std::vector res; + std::stringstream input(str); + std::string temp; + while (std::getline(input, temp, pattern)) { + res.push_back(temp); + } + return res; +} +void graph_service::init(std::string ips_str, int shard_num) { + std::istringstream stream(ips_str); + std::string ip, port; + server_size = 0; + std::vector ips_list = split(ips_str, ';'); + int index = 0; + for (auto ips : ips_list) { + auto ip_and_port = split(ips, ':'); + server_list.push_back(ip_and_port[0]); + port_list.push_back(ip_and_port[1]); + // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], + // ip_and_port[1], index); + // host_sign_list_.push_back(ph_host.serialize_to_string()); + index++; + } + start_client(); +} +} +} \ No newline at end of file diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h new file mode 100644 index 0000000000000..538dc3f85f9f8 --- /dev/null +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -0,0 +1,148 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once +#include +#include // NOLINT +#include +#include +#include +#include +#include +#include // NOLINT +#include +#include "google/protobuf/text_format.h" + +#include "gtest/gtest.h" +#include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/tensor_util.h" +#include "paddle/fluid/framework/variable.h" + +#include "paddle/fluid/distributed/ps.pb.h" +#include "paddle/fluid/distributed/service/env.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_server.h" +#include "paddle/fluid/distributed/service/sendrecv.pb.h" +#include "paddle/fluid/distributed/service/service.h" +#include "paddle/fluid/framework/program_desc.h" +#include "paddle/fluid/operators/math/math_function.h" +#include "paddle/fluid/platform/place.h" +#include "paddle/fluid/string/printf.h" +namespace paddle { +namespace distributed { +class graph_service { + std::vector keys; + std::vector server_list, port_list, host_sign_list; + int server_size, shard_num; + GraphBrpcClient g_client; + GraphBrpcServer g_server; + + public: + // void GetDownpourSparseTableProto( + // ::paddle::distributed::TableParameter* sparse_table_proto) { + // sparse_table_proto->set_table_id(0); + // sparse_table_proto->set_table_class("GraphTable"); + // sparse_table_proto->set_shard_num(256); + // sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); + // ::paddle::distributed::TableAccessorParameter* accessor_proto = + // sparse_table_proto->mutable_accessor(); + // ::paddle::distributed::CommonAccessorParameter* common_proto = + // sparse_table_proto->mutable_common(); + + // accessor_proto->set_accessor_class("CommMergeAccessor"); + // accessor_proto->set_fea_dim(0); + // accessor_proto->set_embedx_dim(10); + + // common_proto->set_name("sgd"); + // common_proto->set_table_name("MergedDense"); + // common_proto->set_trainer_num(1); + // common_proto->set_sync(false); + // common_proto->add_params("Param"); + // common_proto->add_dims(10); + // common_proto->add_initializers("uniform_random&0&-1.0&1.0"); + // common_proto->add_params("LearningRate"); + // common_proto->add_dims(1); + // common_proto->add_initializers("fill_constant&1.0"); + // } + + // ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { + // ::paddle::distributed::PSParameter worker_fleet_desc; + // worker_fleet_desc.set_shard_num(shard_num); + // ::paddle::distributed::WorkerParameter* worker_proto = + // worker_fleet_desc.mutable_worker_param(); + + // ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = + // worker_proto->mutable_downpour_worker_param(); + + // ::paddle::distributed::TableParameter* worker_sparse_table_proto = + // downpour_worker_proto->add_downpour_table_param(); + // GetDownpourSparseTableProto(worker_sparse_table_proto); + + // ::paddle::distributed::ServerParameter* server_proto = + // worker_fleet_desc.mutable_server_param(); + // ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + // server_proto->mutable_downpour_server_param(); + // ::paddle::distributed::ServerServiceParameter* server_service_proto = + // downpour_server_proto->mutable_service_param(); + // server_service_proto->set_service_class("GraphBrpcService"); + // server_service_proto->set_server_class("GraphBrpcServer"); + // server_service_proto->set_client_class("GraphBrpcClient"); + // server_service_proto->set_start_server_port(0); + // server_service_proto->set_server_thread_num(12); + + // ::paddle::distributed::TableParameter* server_sparse_table_proto = + // downpour_server_proto->add_downpour_table_param(); + // GetDownpourSparseTableProto(server_sparse_table_proto); + + // return worker_fleet_desc; + // } + void set_server_size(int server_size) { this->server_size = server_size; } + int get_server_size(int server_size) { return server_size; } + std::vector split(std::string &str, const char pattern); + void start_client() { + // framework::Scope client_scope; + // platform::CPUPlace place; + // InitTensorsOnClient(&client_scope, &place, 100); + // std::map> + // dense_regions; + // dense_regions.insert( + // std::pair>(0, + // {})); + // auto regions = dense_regions[0]; + // framework::Variable* var = client_scope.FindVar("x"); + // framework::LoDTensor* tensor = var->GetMutable(); + + // RunClient(dense_regions); + // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + // paddle::distributed::PaddlePSEnvironment _ps_env; + // auto servers_ = host_sign_list_.size(); + // _ps_env = paddle::distributed::PaddlePSEnvironment(); + // _ps_env.set_ps_servers(&host_sign_list_, servers_); + // worker_ptr_ = std::shared_ptr( + // paddle::distributed::PSClientFactory::create(worker_proto)); + // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); + } + void init(std::string ips_str, int shard_num); + + public: + void set_keys(std::vector keys) { // just for test + this->keys = keys; + } + std::vector get_keys(int start, int size) { // just for test + return std::vector(keys.begin() + start, keys.begin() + start + size); + } +}; +} +} diff --git a/paddle/fluid/distributed/service/graph_service.cc b/paddle/fluid/distributed/service/graph_service.cc deleted file mode 100644 index f8ce4dc3c8504..0000000000000 --- a/paddle/fluid/distributed/service/graph_service.cc +++ /dev/null @@ -1,162 +0,0 @@ -#include -#include -#include -// #include -// #include // NOLINT -// #include // NOLINT -// #include -// #include -// #include -// #include "google/protobuf/text_format.h" - -// #include "gtest/gtest.h" -// #include "paddle/fluid/framework/lod_tensor.h" -// #include "paddle/fluid/framework/scope.h" -// #include "paddle/fluid/framework/tensor_util.h" -// #include "paddle/fluid/framework/variable.h" - -#include "paddle/fluid/distributed/ps.pb.h" -// #include "paddle/fluid/distributed/service/brpc_ps_client.h" -// #include "paddle/fluid/distributed/service/brpc_ps_server.h" -// #include "paddle/fluid/distributed/service/env.h" -// #include "paddle/fluid/distributed/service/ps_client.h" -// #include "paddle/fluid/distributed/service/sendrecv.pb.h" -// #include "paddle/fluid/distributed/service/service.h" -#include "paddle/fluid/distributed/table/graph_node.h" -// #include "paddle/fluid/framework/program_desc.h" -// #include "paddle/fluid/operators/math/math_function.h" -// #include "paddle/fluid/platform/place.h" -// #include "paddle/fluid/string/printf.h" -namespace paddle { -namespace distributed { -class graph_service{ - - std::vector keys; - std::vector server_list,port_list,host_sign_list; - int server_size,shard_num; - public: - // void GetDownpourSparseTableProto( -// ::paddle::distributed::TableParameter* sparse_table_proto) { -// sparse_table_proto->set_table_id(0); -// sparse_table_proto->set_table_class("GraphTable"); -// sparse_table_proto->set_shard_num(256); -// sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); -// ::paddle::distributed::TableAccessorParameter* accessor_proto = -// sparse_table_proto->mutable_accessor(); -// ::paddle::distributed::CommonAccessorParameter* common_proto = -// sparse_table_proto->mutable_common(); - -// accessor_proto->set_accessor_class("CommMergeAccessor"); -// accessor_proto->set_fea_dim(0); -// accessor_proto->set_embedx_dim(10); - -// common_proto->set_name("sgd"); -// common_proto->set_table_name("MergedDense"); -// common_proto->set_trainer_num(1); -// common_proto->set_sync(false); -// common_proto->add_params("Param"); -// common_proto->add_dims(10); -// common_proto->add_initializers("uniform_random&0&-1.0&1.0"); -// common_proto->add_params("LearningRate"); -// common_proto->add_dims(1); -// common_proto->add_initializers("fill_constant&1.0"); -// } - -// ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { -// ::paddle::distributed::PSParameter worker_fleet_desc; -// worker_fleet_desc.set_shard_num(shard_num); -// ::paddle::distributed::WorkerParameter* worker_proto = -// worker_fleet_desc.mutable_worker_param(); - -// ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = -// worker_proto->mutable_downpour_worker_param(); - -// ::paddle::distributed::TableParameter* worker_sparse_table_proto = -// downpour_worker_proto->add_downpour_table_param(); -// GetDownpourSparseTableProto(worker_sparse_table_proto); - -// ::paddle::distributed::ServerParameter* server_proto = -// worker_fleet_desc.mutable_server_param(); -// ::paddle::distributed::DownpourServerParameter* downpour_server_proto = -// server_proto->mutable_downpour_server_param(); -// ::paddle::distributed::ServerServiceParameter* server_service_proto = -// downpour_server_proto->mutable_service_param(); -// server_service_proto->set_service_class("GraphBrpcService"); -// server_service_proto->set_server_class("GraphBrpcServer"); -// server_service_proto->set_client_class("GraphBrpcClient"); -// server_service_proto->set_start_server_port(0); -// server_service_proto->set_server_thread_num(12); - -// ::paddle::distributed::TableParameter* server_sparse_table_proto = -// downpour_server_proto->add_downpour_table_param(); -// GetDownpourSparseTableProto(server_sparse_table_proto); - -// return worker_fleet_desc; -// } - void set_server_size(int server_size){ - this->server_size = server_size; - } - int get_server_size(int server_size){ - return server_size; - } - std::vector split(std::string &str, const char pattern) -{ - std::vector res; - std::stringstream input(str); - std::string temp; - while(std::getline(input, temp, pattern)) - { - res.push_back(temp); - } - return res; -} - void start_client(){ -// framework::Scope client_scope; -// platform::CPUPlace place; -// InitTensorsOnClient(&client_scope, &place, 100); -// std::map> dense_regions; -// dense_regions.insert( -// std::pair>(0, {})); -// auto regions = dense_regions[0]; -// framework::Variable* var = client_scope.FindVar("x"); -// framework::LoDTensor* tensor = var->GetMutable(); - -// RunClient(dense_regions); -// ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); -// paddle::distributed::PaddlePSEnvironment _ps_env; -// auto servers_ = host_sign_list_.size(); -// _ps_env = paddle::distributed::PaddlePSEnvironment(); -// _ps_env.set_ps_servers(&host_sign_list_, servers_); -// worker_ptr_ = std::shared_ptr( -// paddle::distributed::PSClientFactory::create(worker_proto)); -// worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); - } - void init(std::string ips_str,int shard_num){ - std::istringstream stream(ips_str); - std::string ip,port; - server_size = 0; - std::vector ips_list = split(ips_str, ';'); - int index = 0; - for(auto ips:ips_list){ - auto ip_and_port = split(ips,':'); - server_list.push_back(ip_and_port[0]); - port_list.push_back(ip_and_port[1]); - // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], ip_and_port[1], index); - // host_sign_list_.push_back(ph_host.serialize_to_string()); - index++; - } - start_client(); - } - - public: - void set_keys(std::vector keys){//just for test - this->keys = keys; - } - std::vector get_keys(int start,int size){//just for test - return std::vector(keys.begin() + start, keys.begin() + start + size); - } - - -}; - } -} \ No newline at end of file diff --git a/paddle/fluid/distributed/service/ps_client.cc b/paddle/fluid/distributed/service/ps_client.cc index 68b240e5f1ac5..5478ddfbef202 100644 --- a/paddle/fluid/distributed/service/ps_client.cc +++ b/paddle/fluid/distributed/service/ps_client.cc @@ -13,11 +13,10 @@ // limitations under the License. #include "paddle/fluid/distributed/service/ps_client.h" -#include -#include "brpc/server.h" #include "glog/logging.h" #include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { @@ -85,4 +84,4 @@ PSClient *PSClientFactory::create(const PSParameter &ps_config) { return client; } } // namespace distributed -} // namespace paddle +} // namespace paddle \ No newline at end of file diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index a104968493861..08a3881ad0bd2 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -70,7 +70,8 @@ class PSClient { int max_retry) = 0; // 触发table数据退场 - virtual std::future shrink(uint32_t table_id) = 0; + virtual std::future shrink(uint32_t table_id, + const std::string threshold) = 0; // 全量table进行数据load virtual std::future load(const std::string &epoch, @@ -154,20 +155,26 @@ class PSClient { promise.set_value(-1); return fut; } - // virtual std::future sample(uint32_t table_id, uint64_t node_id, GraphNodeType type, int sample_size,std::vector &res){ - // LOG(FATAL) << "Did not implement"; - // std::promise promise; - // std::future fut = promise.get_future(); - // promise.set_value(-1); - // return fut; - // } - // virtual std::future pull_graph_list(uint32_t table_id, uint64_t node_id, GraphNodeType type,int start, int size, std::vector &res){ - // LOG(FATAL) << "Did not implement"; - // std::promise promise; - // std::future fut = promise.get_future(); - // promise.set_value(-1); - // return fut; - // } + virtual std::future sample(uint32_t table_id, uint64_t node_id, + GraphNodeType type, int sample_size, + std::vector &res) { + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } + virtual std::future pull_graph_list(uint32_t table_id, + uint64_t node_id, + GraphNodeType type, int start, + int size, + std::vector &res) { + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 5c77a472c4e9f..52c2d7104b544 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -52,7 +52,7 @@ enum PsCmdID { PS_GRAPH_SAMPLE = 31; } -message PsRequestMessage{ +message PsRequestMessage { required uint32 cmd_id = 1; optional uint32 table_id = 2; repeated bytes params = 3; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 43b44cdac5bbe..d75fc028e4c23 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -1,59 +1,74 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/common_graph_table.h" #include #include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" -#include "paddle/fluid/distributed/table/common_graph_table.h" namespace paddle { namespace distributed { -size_t GraphShard::get_size(){ +size_t GraphShard::get_size() { size_t res = 0; - for(int i = 0;i < bucket_size;i++){ + for (int i = 0; i < bucket_size; i++) { res += bucket[i].size(); } return res; } -list::iterator GraphShard::add_node(GraphNode *node){ - if(node_location.find({node->get_id(),node->get_graph_node_type()}) != node_location.end()) - return node_location.find({node->get_id(),node->get_graph_node_type()})->second; +std::list::iterator GraphShard::add_node(GraphNode *node) { + if (node_location.find({node->get_id(), node->get_graph_node_type()}) != + node_location.end()) + return node_location.find({node->get_id(), node->get_graph_node_type()}) + ->second; int index = node->get_id() % shard_num % bucket_size; - list::iterator iter = bucket[index].insert(bucket[index].end(), node); - node_location[{node->get_id(),node->get_graph_node_type()}] = iter; + std::list::iterator iter = + bucket[index].insert(bucket[index].end(), node); + node_location[{node->get_id(), node->get_graph_node_type()}] = iter; return iter; } -void GraphShard:: add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge){ - - (*add_node(new GraphNode(id,type,std::string(""))))->add_edge(edge); +void GraphShard::add_neighboor(uint64_t id, GraphNodeType type, + GraphEdge *edge) { + (*add_node(new GraphNode(id, type, std::string(""))))->add_edge(edge); } -GraphNode * GraphShard::find_node(uint64_t id, GraphNodeType type){ - if(node_location.find({id,type}) == node_location.end()) - return NULL; - return *(node_location[{id,type}]); +GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { + if (node_location.find({id, type}) == node_location.end()) return NULL; + return *(node_location[{id, type}]); } -int32_t GraphTable::load(const std::string& path, const std::string& param){ +int32_t GraphTable::load(const std::string &path, const std::string ¶m) { rwlock_->WRLock(); auto paths = paddle::string::split_string(path, ";"); - for(auto path: paths){ + for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); - if(values.size() < 3) - continue; + if (values.size() < 3) continue; auto id = std::stoull(values[0]); size_t shard_id = id % _shard_num; if (shard_id >= shard_end || shard_id < shard_start) { VLOG(0) << "will not load " << id << " from " << path - << ", please check id distribution"; + << ", please check id distribution"; continue; } size_t index = shard_id - shard_start; GraphNodeType type = GraphNode::get_graph_node_type(values[1]); - shards[index].add_node(new GraphNode(id, type, values[2])); - for(size_t i = 3;i < values.size();i++){ - auto edge_arr = paddle::string::split_string(values[i], ";"); - if(edge_arr.size() < 3){ + shards[index].add_node(new GraphNode(id, type, values[2])); + for (size_t i = 3; i < values.size(); i++) { + auto edge_arr = + paddle::string::split_string(values[i], ";"); + if (edge_arr.size() < 3) { auto edge_id = std::stoull(edge_arr[0]); GraphNodeType edge_type = GraphNode::get_graph_node_type(edge_arr[1]); auto weight = std::stod(edge_arr[2]); @@ -62,130 +77,136 @@ int32_t GraphTable::load(const std::string& path, const std::string& param){ } } } - for(auto &shard:shards){ - list::iter = shard.begin(); - while(iter != shard.end()){ - auto node = *iter; - - iter++; + for (auto &shard : shards) { + auto bucket = shard.get_bucket(); + for (int i = 0; i < bucket.size(); i++) { + std::list::iterator iter = bucket[i].begin(); + while (iter != bucket[i].end()) { + auto node = *iter; + node->build_sampler(); + iter++; + } } - } + } } rwlock_->UNLock(); return 0; } -GraphNode * GraphTable::find_node(uint64_t id, GraphNodeType type){ - rwlock_->WRLock(); - size_t shard_id = id % _shard_num; - if (shard_id >= shard_end || shard_id < shard_start) { - return NULL; - } - size_t index = shard_id - shard_start; - - GraphNode *node = shards[index].find_node(id,type); - rwlock_->UNLock(); - return node; +GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { + rwlock_->WRLock(); + size_t shard_id = id % _shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + return NULL; + } + size_t index = shard_id - shard_start; + GraphNode *node = shards[index].find_node(id, type); + rwlock_->UNLock(); + return node; +} +int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, + int sample_size, char *&buffer, + int &actual_size) { + rwlock_->RDLock(); + GraphNode *node = find_node(node_id, type); + if (node == NULL) { + actual_size = 0; + rwlock_->UNLock(); + return 0; + } + std::vector res = node->sample_k(sample_size); + std::vector node_list; + int total_size = 0; + for (auto x : res) { + GraphNode temp; + temp.set_id(x->id); + temp.set_graph_node_type(x->type); + total_size += temp.get_size(); + node_list.push_back(temp); + } + buffer = new char[total_size]; + int index = 0; + for (auto x : node_list) { + x.to_buffer(buffer + index); + index += x.get_size(); } -int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type,int sample_size, char* &buffer, int &actual_size){ - rwlock_->RDLock(); - GraphNode * node = find_node(node_id,type); - if(node == NULL){ - actual_size = 0; - rwlock_->UNLock(); - return 0; - } - vector res = node->sample_k(sample_size); - vector node_list; - int total_size = 0; - for(auto x:res){ - GraphNode temp; - temp.set_id(x->id); - temp.set_graph_node_type(x->type); - total_size += temp.get_size(); - node_list.push_back(temp); - } - buffer = new char[total_size]; - int index = 0; - for(auto x:node_list){ - x.to_buffer(buffer + index); - index += x.get_size(); - } - actual_size = total_size; - rwlock_->UNLock(); - return 0; + actual_size = total_size; + rwlock_->UNLock(); + return 0; } -int32_t GraphTable::pull_graph_list(int start, int total_size, char* &buffer,int &actual_size){ - rwlock_->RDLock(); +int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, + int &actual_size) { + rwlock_->RDLock(); int thread_index = 0; - if(start < 0) - start = 0; - int size = 0, cur_size; - if(total_size <= 0){ - actual_size = 0; - return 0; - } - std::vector>> tasks; - for(size_t i = 0;i < shards.size();i++){ - cur_size = shards[i].get_size(); - if(size + cur_size <= start){ - size += cur_size; - continue; - } - if(size + cur_size - start >= total_size){ - - tasks.push_back(_shards_task_pool[thread_index]->enqueue( - [this,i,start,size,total_size]() -> vector { - return this->shards[i].get_batch(start - size,total_size); - })); - thread_index++; - break; - } else { - tasks.push_back(_shards_task_pool[thread_index]->enqueue( - [this,i,start,size,total_size,cur_size]() -> vector { - return this->shards[i].get_batch(start - size,size + cur_size - start); - })); - total_size -= size + cur_size - start; - size += cur_size; - start = size; - } - } + if (start < 0) start = 0; + int size = 0, cur_size; + if (total_size <= 0) { + actual_size = 0; + return 0; + } + std::vector>> tasks; + for (size_t i = 0; i < shards.size(); i++) { + cur_size = shards[i].get_size(); + if (size + cur_size <= start) { + size += cur_size; + continue; + } + if (size + cur_size - start >= total_size) { + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this, i, start, size, total_size]() -> std::vector { + return this->shards[i].get_batch(start - size, total_size); + })); + thread_index++; + break; + } else { + tasks.push_back(_shards_task_pool[thread_index]->enqueue( + [this, i, start, size, total_size, + cur_size]() -> std::vector { + return this->shards[i].get_batch(start - size, + size + cur_size - start); + })); + total_size -= size + cur_size - start; + size += cur_size; + start = size; + } + } for (size_t i = 0; i < tasks.size(); ++i) { tasks[i].wait(); } size = 0; - vector > res; - for(size_t i = 0;i < tasks.size();i++){ + std::vector> res; + for (size_t i = 0; i < tasks.size(); i++) { res.push_back(tasks[i].get()); - for(size_t j = 0;j < res.back().size();j++){ + for (size_t j = 0; j < res.back().size(); j++) { size += res.back()[j]->get_size(); } - } + } buffer = new char[size]; int index = 0; - for(size_t i = 0;i < res.size();i++){ - for(size_t j = 0;j < res[i].size();j++){ + for (size_t i = 0; i < res.size(); i++) { + for (size_t j = 0; j < res[i].size(); j++) { res[i][j]->to_buffer(buffer + index); index += res[i][j]->get_size(); } } actual_size = size; - rwlock_->UNLock(); - return 0; + rwlock_->UNLock(); + return 0; } -int32_t GraphTable::initialize(){ +int32_t GraphTable::initialize() { _shards_task_pool.resize(task_pool_size_); for (size_t i = 0; i < _shards_task_pool.size(); ++i) { _shards_task_pool[i].reset(new ::ThreadPool(1)); - } - server_num = _shard_num; + } + server_num = _shard_num; /* _shard_num is actually server number here - when a server initialize its tables, it sets tables' _shard_num to server_num, and _shard_idx to server + when a server initialize its tables, it sets tables' _shard_num to server_num, + and _shard_idx to server rank */ _shard_num = _config.shard_num(); - shard_num_per_table = sparse_local_shard_num(_shard_num,server_num); + shard_num_per_table = sparse_local_shard_num(_shard_num, server_num); shard_start = _shard_idx * shard_num_per_table; shards.resize(shard_num_per_table); return 0; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 0e4f2b29acf7b..028d724a9ffa4 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -1,17 +1,29 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + #pragma once #include #include #include +#include #include #include // NOLINT #include #include #include #include -#include "Eigen/Dense" -#include -#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/distributed/table/accessor.h" #include "paddle/fluid/distributed/table/common_table.h" #include "paddle/fluid/framework/rw_lock.h" @@ -19,92 +31,108 @@ namespace paddle { namespace distributed { struct pair_hash { - inline size_t operator()(const pair & p) const { - return p.first * 10007 + int(p.second); - } + inline size_t operator()(const std::pair &p) const { + return p.first * 10007 + int(p.second); + } }; class GraphShard { - public: + public: static int bucket_low_bound; - static int gcd(int s,int t){ - if(s % t == 0) - return t; + static int gcd(int s, int t) { + if (s % t == 0) return t; return gcd(t, s % t); } size_t get_size(); - GraphShard(){ - - } - GraphShard(int shard_num){ + GraphShard() {} + GraphShard(int shard_num) { this->shard_num = shard_num; bucket_size = init_bucket_size(shard_num); bucket.resize(bucket_size); } - vector get_batch(int start,int total_size){ - if(start < 0) - start = 0; - int size = 0, cur_size; - vector res; - if(total_size <= 0) - return res; - for(int i = 0;i < bucket_size;i++){ - cur_size = bucket[i].size(); - if(size + cur_size <= start){ - size += cur_size; - continue; - } - int read = 0; - list::iterator iter = bucket[i].begin(); - while(size + read < start){ - iter++; - read++; - } - read = 0; - while(iter != bucket[i].end() && read < total_size){ - res.push_back(*iter); - iter++; - read++; - } - if(read == total_size) - break; - size += cur_size; - start = size; - total_size -= read; + std::vector> &get_bucket() { return bucket; } + std::vector get_batch(int start, int total_size) { + if (start < 0) start = 0; + int size = 0, cur_size; + std::vector res; + if (total_size <= 0) return res; + for (int i = 0; i < bucket_size; i++) { + cur_size = bucket[i].size(); + if (size + cur_size <= start) { + size += cur_size; + continue; + } + int read = 0; + std::list::iterator iter = bucket[i].begin(); + while (size + read < start) { + iter++; + read++; } - return res; + read = 0; + while (iter != bucket[i].end() && read < total_size) { + res.push_back(*iter); + iter++; + read++; + } + if (read == total_size) break; + size += cur_size; + start = size; + total_size -= read; + } + return res; } - int init_bucket_size(int shard_num){ - for(int i = bucket_low_bound;;i++){ - if(gcd(i,shard_num) == 1) - return i; - } - return -1; + int init_bucket_size(int shard_num) { + for (int i = bucket_low_bound;; i++) { + if (gcd(i, shard_num) == 1) return i; + } + return -1; } - list::iterator add_node(GraphNode *node); - GraphNode * find_node(uint64_t id, GraphNodeType type); - void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); - private: - unordered_map,list::iterator , pair_hash> node_location; - int bucket_size, shard_num; - vector> bucket; + std::list::iterator add_node(GraphNode *node); + GraphNode *find_node(uint64_t id, GraphNodeType type); + void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); + private: + std::unordered_map, + std::list::iterator, pair_hash> + node_location; + int bucket_size, shard_num; + std::vector> bucket; }; class GraphTable : public SparseTable { public: GraphTable() {} - virtual ~GraphTable() {} - virtual int32_t pull_graph_list(int start, int size, char* &buffer,int &actual_size); - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, int sampe_size, char* &buffer, int &actual_size); + virtual ~GraphTable() {} + virtual int32_t pull_graph_list(int start, int size, char *&buffer, + int &actual_size); + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, + int sampe_size, char *&buffer, + int &actual_size); virtual int32_t initialize(); - int32_t load(const std::string& path, const std::string& param); + int32_t load(const std::string &path, const std::string ¶m); GraphNode *find_node(uint64_t id, GraphNodeType type); -protected: - vector shards; - unordered_set id_set; - size_t shard_start,shard_end, server_num,shard_num_per_table; + + virtual int32_t pull_sparse(float *values, const uint64_t *keys, size_t num) { + return 0; + } + virtual int32_t push_sparse(const uint64_t *keys, const float *values, + size_t num) { + return 0; + } + virtual void clear() {} + virtual int32_t flush() { return 0; } + virtual int32_t shrink(const std::string ¶m) { return 0; } + //指定保存路径 + virtual int32_t save(const std::string &path, const std::string &converter) { + return 0; + } + virtual int32_t initialize_shard() { return 0; } + + protected: + std::vector shards; + std::unordered_set id_set; + size_t shard_start, shard_end, server_num, shard_num_per_table; std::unique_ptr rwlock_{nullptr}; const int task_pool_size_ = 7; - std::vector> _shards_task_pool; + std::vector> _shards_task_pool; }; } -}; \ No newline at end of file +}; diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 929623656d90d..8263be586a717 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -1,40 +1,53 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/graph_node.h" -#include +#include namespace paddle { namespace distributed { -int GraphNode:: enum_size = sizeof(int); -int GraphNode:: id_size = sizeof(uint64_t); -int GraphNode:: double_size = sizeof(double); -int GraphNode:: int_size = sizeof(int); -int GraphNode::get_size(){ - return feature.size() + id_size + enum_size + int_size; +int GraphNode::enum_size = sizeof(int); +int GraphNode::id_size = sizeof(uint64_t); +int GraphNode::double_size = sizeof(double); +int GraphNode::int_size = sizeof(int); +int GraphNode::get_size() { + return feature.size() + id_size + enum_size + int_size; } -void GraphNode::build_sampler(){ - sampler = new WeightedSampler(); - GraphEdge** arr = edges.data(); - sampler->build((WeightedObject **)arr, 0, edges.size()); - +void GraphNode::build_sampler() { + sampler = new WeightedSampler(); + GraphEdge** arr = edges.data(); + sampler->build((WeightedObject**)arr, 0, edges.size()); } -void GraphNode:: to_buffer(char * buffer){ - int size = get_size(); - memcpy(buffer, &size, int_size); - memcpy(buffer + int_size,feature.c_str(),feature.size()); - memcpy(buffer + int_size + feature.size(),&id,id_size); - int int_state = type; - memcpy(buffer + int_size + feature.size() + id_size,&int_state,enum_size); +void GraphNode::to_buffer(char* buffer) { + int size = get_size(); + memcpy(buffer, &size, int_size); + memcpy(buffer + int_size, feature.c_str(), feature.size()); + memcpy(buffer + int_size + feature.size(), &id, id_size); + int int_state = type; + memcpy(buffer + int_size + feature.size() + id_size, &int_state, enum_size); } -void GraphNode:: recover_from_buffer(char* buffer){ - int size; - memcpy(&size,buffer,int_size); - int feature_size = size - id_size - enum_size - int_size; - char str[feature_size + 1]; - memcpy(str,buffer + int_size,feature_size); - str[feature_size] = '\0'; - feature = str; - memcpy(&id,buffer + int_size + feature_size,id_size); - int int_state; - memcpy(&int_state,buffer + int_size + feature_size + id_size,enum_size); - type = GraphNodeType(int_state); +void GraphNode::recover_from_buffer(char* buffer) { + int size; + memcpy(&size, buffer, int_size); + int feature_size = size - id_size - enum_size - int_size; + char str[feature_size + 1]; + memcpy(str, buffer + int_size, feature_size); + str[feature_size] = '\0'; + feature = str; + memcpy(&id, buffer + int_size + feature_size, id_size); + int int_state; + memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); + type = GraphNodeType(int_state); } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 4571e10c34fbe..9c30c6b1dfc35 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -1,84 +1,80 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + #pragma once +#include #include "paddle/fluid/distributed/table/weighted_sampler.h" -#include namespace paddle { namespace distributed { -enum GraphNodeType{ - user=0,item=1,query=2,unknown=3 +enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; +class GraphEdge : public WeightedObject { + public: + double weight; + uint64_t id; + GraphNodeType type; + GraphEdge() {} + GraphEdge(uint64_t id, GraphNodeType type, double weight) + : weight(weight), id(id), type(type) {} }; -class GraphEdge: public WeightedObject{ -public: - double weight; - uint64_t id; +class GraphNode { + public: + GraphNode() { sampler = NULL; } + GraphNode(uint64_t id, GraphNodeType type, std::string feature) + : id(id), type(type), feature(feature), sampler(NULL) {} + virtual ~GraphNode() {} + static int enum_size, id_size, int_size, double_size; + uint64_t get_id() { return id; } + void set_id(uint64_t id) { this->id = id; } + GraphNodeType get_graph_node_type() { return type; } + void set_graph_node_type(GraphNodeType type) { this->type = type; } + void set_feature(std::string feature) { this->feature = feature; } + std::string get_feature() { return feature; } + virtual int get_size(); + virtual void build_sampler(); + virtual void to_buffer(char *buffer); + virtual void recover_from_buffer(char *buffer); + virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } + static GraphNodeType get_graph_node_type(std::string &str) { GraphNodeType type; - GraphEdge(){ - } - GraphEdge(uint64_t id, GraphNodeType type,double weight):weight(weight),id(id),type(type){ - } -}; -class GraphNode{ -public: -GraphNode(){ - sampler = NULL; -} -GraphNode(uint64_t id,GraphNodeType type,std::string feature):id(id),type(type),feature(feature),sampler(NULL){ -} -virtual ~GraphNode() {} -static int enum_size,id_size,int_size,double_size; -uint64_t get_id(){ - return id; -} -void set_id(uint64_t id){ - this->id = id; -} -GraphNodeType get_graph_node_type(){ + if (str == "user") + type = GraphNodeType::user; + + else if (str == "item") + type = GraphNodeType::item; + else if (str == "query") + type = GraphNodeType::query; + else + type = GraphNodeType::unknown; return type; -} -void set_graph_node_type(GraphNodeType type){ - this->type = type; -} -void set_feature(std::string feature){ - this->feature = feature; -} -std::string get_feature(){ - return feature; -} -virtual int get_size(); -virtual void build_sampler(); -virtual void to_buffer(char* buffer); -virtual void recover_from_buffer(char* buffer); -virtual void add_edge(GraphEdge * edge){ - edges.push_back(edge); -} -static GraphNodeType get_graph_node_type(std::string &str){ - GraphNodeType type; - if(str == "user") - type = GraphNodeType::user; - else if(str == "item") - type = GraphNodeType::item; - else if(str == "query") - type = GraphNodeType:: query; - else - type = GraphNodeType::unknown; - return type; -} -std::vector sample_k(int k){ + } + std::vector sample_k(int k) { std::vector v; - if(sampler != NULL){ - auto res = sampler->sample_k(k); - for(auto x: res){ - v.push_back((GraphEdge *)x); - } + if (sampler != NULL) { + auto res = sampler->sample_k(k); + for (auto x : res) { + v.push_back((GraphEdge *)x); + } } return v; - -} -protected: -uint64_t id; -GraphNodeType type; -std::string feature; -WeightedSampler *sampler; -std::vector edges; + } + + protected: + uint64_t id; + GraphNodeType type; + std::string feature; + WeightedSampler *sampler; + std::vector edges; }; } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/table/table.cc b/paddle/fluid/distributed/table/table.cc index 7ee9925b45fbe..4ec4d9faaa4a7 100644 --- a/paddle/fluid/distributed/table/table.cc +++ b/paddle/fluid/distributed/table/table.cc @@ -20,15 +20,15 @@ #include "paddle/fluid/distributed/common/registerer.h" #include "paddle/fluid/distributed/table/common_dense_table.h" +#include "paddle/fluid/distributed/table/common_graph_table.h" #include "paddle/fluid/distributed/table/common_sparse_table.h" #include "paddle/fluid/distributed/table/sparse_geo_table.h" #include "paddle/fluid/distributed/table/tensor_accessor.h" #include "paddle/fluid/distributed/table/tensor_table.h" -#include "paddle/fluid/distributed/table/common_graph_table.h" namespace paddle { namespace distributed { - +REGISTER_PSCORE_CLASS(Table, GraphTable); REGISTER_PSCORE_CLASS(Table, CommonDenseTable); REGISTER_PSCORE_CLASS(Table, CommonSparseTable); REGISTER_PSCORE_CLASS(Table, SparseGeoTable); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index bf651c64d00e9..7161d2016004d 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -21,12 +21,12 @@ #include #include #include "paddle/fluid/distributed/table/accessor.h" +#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/place.h" #include "paddle/fluid/string/string_helper.h" -#include "paddle/fluid/distributed/table/graph_node.h" namespace paddle { namespace distributed { @@ -88,18 +88,22 @@ class Table { } // only for graph table - virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, int total_size, char* &buffer,int &actual_size){ + virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, + int total_size, char *&buffer, + int &actual_size) { return 0; } // only for graph table - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type,int sampe_size, char* &buffer, int &actual_size){ - return 0; + virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, + int sampe_size, char *&buffer, + int &actual_size) { + return 0; } virtual int32_t pour() { return 0; } virtual void clear() = 0; virtual int32_t flush() = 0; - virtual int32_t shrink() = 0; + virtual int32_t shrink(const std::string ¶m) = 0; //指定加载路径 virtual int32_t load(const std::string &path, diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index cc61c6caa0267..c93bc551f54f3 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -1,60 +1,80 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { - void WeightedSampler::build(WeightedObject** v,int start,int end){ - count = 0; - if(start + 1 == end){ - left = right = NULL; - weight = v[start]->get_weight(); - object = v[start]; - count = 1; - - } else { - left = new WeightedSampler(); - right = new WeightedSampler(); - left->build(v,start, start + (end - start)/2); - right->build(v,start + (end - start)/2, end); - weight = left->weight + right->weight; - count = left->count + right->count; - } - } - vector WeightedSampler::sample_k(int k){ - if(k > count){ - k = count; - } - vector sample_result; - double subtract; - unordered_map subtract_weight_map; - unordered_map subtract_count_map; - while(k--){ - double query_weight = rand() % 100000/100000.0; - query_weight *= weight - subtract_weight_map[this]; - sample_result.push_back(sample(query_weight, subtract_weight_map,subtract_count_map, subtract)); - } - return sample_result; +void WeightedSampler::build(WeightedObject **v, int start, int end) { + count = 0; + if (start + 1 == end) { + left = right = NULL; + weight = v[start]->get_weight(); + object = v[start]; + count = 1; - } - WeightedObject * WeightedSampler::sample(double query_weight, unordered_map &subtract_weight_map, unordered_map &subtract_count_map, double &subtract){ - if(left == NULL){ - subtract_weight_map[this] = weight; - subtract = weight; - subtract_count_map[this] = 1; - return object; - } - int left_count = left->count - subtract_count_map[left]; - int right_count = right->count - subtract_count_map[right]; - double left_subtract = subtract_weight_map[left]; - WeightedObject * return_id; - if(right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight){ - return_id = left->sample(query_weight, subtract_weight_map,subtract_count_map, subtract); - } else { - return_id = right->sample(query_weight - (left->weight - left_subtract),subtract_weight_map,subtract_count_map, subtract); - } - subtract_weight_map[this] += subtract; - subtract_count_map[this]++; - return return_id; - - } + } else { + left = new WeightedSampler(); + right = new WeightedSampler(); + left->build(v, start, start + (end - start) / 2); + right->build(v, start + (end - start) / 2, end); + weight = left->weight + right->weight; + count = left->count + right->count; + } +} +std::vector WeightedSampler::sample_k(int k) { + if (k > count) { + k = count; + } + std::vector sample_result; + double subtract; + std::unordered_map subtract_weight_map; + std::unordered_map subtract_count_map; + while (k--) { + double query_weight = rand() % 100000 / 100000.0; + query_weight *= weight - subtract_weight_map[this]; + sample_result.push_back(sample(query_weight, subtract_weight_map, + subtract_count_map, subtract)); + } + return sample_result; +} +WeightedObject *WeightedSampler::sample( + double query_weight, + std::unordered_map &subtract_weight_map, + std::unordered_map &subtract_count_map, + double &subtract) { + if (left == NULL) { + subtract_weight_map[this] = weight; + subtract = weight; + subtract_count_map[this] = 1; + return object; + } + int left_count = left->count - subtract_count_map[left]; + int right_count = right->count - subtract_count_map[right]; + double left_subtract = subtract_weight_map[left]; + WeightedObject *return_id; + if (right_count == 0 || + left_count > 0 && left->weight - left_subtract >= query_weight) { + return_id = left->sample(query_weight, subtract_weight_map, + subtract_count_map, subtract); + } else { + return_id = + right->sample(query_weight - (left->weight - left_subtract), + subtract_weight_map, subtract_count_map, subtract); + } + subtract_weight_map[this] += subtract; + subtract_count_map[this]++; + return return_id; +} } } - diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 838a034d2b4f6..53bfaa8d30119 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -1,47 +1,53 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + #pragma once -#include -#include -#include -using namespace std; +#include +#include +#include namespace paddle { namespace distributed { -class WeightedObject{ -public: - WeightedObject(){ - - } - virtual ~WeightedObject(){ +class WeightedObject { + public: + WeightedObject() {} + virtual ~WeightedObject() {} + virtual unsigned long long get_id() { return id; } + virtual double get_weight() { return weight; } - } - virtual unsigned long long get_id(){ - return id; - } - virtual double get_weight(){ - return weight; - } - - virtual void set_id(unsigned long long id){ - this->id = id; - } - virtual void set_weight(double weight){ - this->weight = weight; - } -private: - unsigned long long id; - double weight; + virtual void set_id(unsigned long long id) { this->id = id; } + virtual void set_weight(double weight) { this->weight = weight; } + private: + unsigned long long id; + double weight; }; -class WeightedSampler{ -public: - WeightedSampler* left,*right; - WeightedObject* object; - int count; - double weight; - void build(WeightedObject** v,int start,int end); - vector sample_k(int k); -private: - WeightedObject* sample(double query_weight, unordered_map &subtract_weight_map, unordered_map &subtract_count_map, double &subtract); +class WeightedSampler { + public: + WeightedSampler *left, *right; + WeightedObject *object; + int count; + double weight; + void build(WeightedObject **v, int start, int end); + std::vector sample_k(int k); + + private: + WeightedObject *sample( + double query_weight, + std::unordered_map &subtract_weight_map, + std::unordered_map &subtract_count_map, + double &subtract); }; } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/test/CMakeLists.txt b/paddle/fluid/distributed/test/CMakeLists.txt index 73cb33a2e0f30..2bf0eeebff3c3 100644 --- a/paddle/fluid/distributed/test/CMakeLists.txt +++ b/paddle/fluid/distributed/test/CMakeLists.txt @@ -1,3 +1,24 @@ +# set_source_files_properties(table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(table_test SRCS table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) + +# set_source_files_properties(dense_table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(dense_table_test SRCS dense_table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) + +# set_source_files_properties(barrier_table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(barrier_table_test SRCS barrier_table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) + +# set_source_files_properties(brpc_service_dense_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(brpc_service_dense_sgd_test SRCS brpc_service_dense_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) + +# set_source_files_properties(brpc_service_sparse_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) + +# set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) + +# set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +# cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) + set_source_files_properties(table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_test(table_test SRCS table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) @@ -11,10 +32,10 @@ set_source_files_properties(brpc_service_dense_sgd_test.cc PROPERTIES COMPILE_FL cc_test(brpc_service_dense_sgd_test SRCS brpc_service_dense_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) set_source_files_properties(brpc_service_sparse_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) +cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -#cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) \ No newline at end of file +cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index c335ce796878f..20c59c188b803 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -14,10 +14,10 @@ limitations under the License. */ #include #include // NOLINT +#include +#include #include #include // NOLINT -#include -#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -47,27 +47,29 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = {std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"),std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; +std::string nodes[] = { + std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), + std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; char file_name[] = "nodes.txt"; -void prepare_file(char file_name[]){ - ofstream ofile; - ofile.open(file_name); - for(auto x: nodes){ - ofile<mutable_common(); } -distributed::GraphNodeType get_graph_node_type(std::string str){ - distributed::GraphNodeType type; - if(str == "user") - type = distributed::GraphNodeType::user; - else if(str == "item") - type = distributed::GraphNodeType::item; - else if(str == "query") - type = distributed::GraphNodeType:: query; - else - type = distributed::GraphNodeType::unknown; - return type; +distributed::GraphNodeType get_graph_node_type(std::string str) { + distributed::GraphNodeType type; + if (str == "user") + type = distributed::GraphNodeType::user; + else if (str == "item") + type = distributed::GraphNodeType::item; + else if (str == "query") + type = distributed::GraphNodeType::query; + else + type = distributed::GraphNodeType::unknown; + return type; } ::paddle::distributed::PSParameter GetServerProto() { @@ -118,7 +120,7 @@ ::paddle::distributed::PSParameter GetServerProto() { ::paddle::distributed::PSParameter GetWorkerProto() { ::paddle::distributed::PSParameter worker_fleet_desc; - worker_fleet_desc.set_shard_num(127); + worker_fleet_desc.set_shard_num(127); ::paddle::distributed::WorkerParameter* worker_proto = worker_fleet_desc.mutable_worker_param(); @@ -150,7 +152,7 @@ ::paddle::distributed::PSParameter GetWorkerProto() { /*-------------------------------------------------------------------------*/ -std::string ip_ = "127.0.0.1",ip2 = "127.0.0.1"; +std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; uint32_t port_ = 4209, port2 = 4210; std::vector host_sign_list_; @@ -164,31 +166,31 @@ void RunServer() { ::paddle::distributed::PSParameter server_proto = GetServerProto(); auto _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&host_sign_list_, 2); //test + _ps_env.set_ps_servers(&host_sign_list_, 2); // test pserver_ptr_ = std::shared_ptr( paddle::distributed::PSServerFactory::create(server_proto)); std::vector empty_vec; framework::ProgramDesc empty_prog; empty_vec.push_back(empty_prog); pserver_ptr_->configure(server_proto, _ps_env, 0, empty_vec); -LOG(INFO) << "first server, run start(ip,port)"; + LOG(INFO) << "first server, run start(ip,port)"; pserver_ptr_->start(ip_, port_); LOG(INFO) << "init first server Done"; } -void RunServer2(){ +void RunServer2() { LOG(INFO) << "init second server"; - ::paddle::distributed::PSParameter server_proto2 = GetServerProto(); + ::paddle::distributed::PSParameter server_proto2 = GetServerProto(); auto _ps_env2 = paddle::distributed::PaddlePSEnvironment(); - _ps_env2.set_ps_servers(&host_sign_list_, 2); //test + _ps_env2.set_ps_servers(&host_sign_list_, 2); // test pserver_ptr2 = std::shared_ptr( paddle::distributed::PSServerFactory::create(server_proto2)); std::vector empty_vec2; framework::ProgramDesc empty_prog2; empty_vec2.push_back(empty_prog2); pserver_ptr2->configure(server_proto2, _ps_env2, 1, empty_vec2); - pserver_ptr2->start(ip2, port2); + pserver_ptr2->start(ip2, port2); } void RunClient(std::map>& @@ -210,13 +212,13 @@ void RunBrpcPushSparse() { auto ph_host = paddle::distributed::PSHost(ip_, port_, 0); host_sign_list_.push_back(ph_host.serialize_to_string()); - //test-start + // test-start auto ph_host2 = paddle::distributed::PSHost(ip2, port2, 1); host_sign_list_.push_back(ph_host2.serialize_to_string()); - //test-end + // test-end // Srart Server std::thread server_thread(RunServer); - std::thread server_thread2(RunServer2); + std::thread server_thread2(RunServer2); sleep(1); std::map> dense_regions; @@ -228,80 +230,86 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ LOG(INFO) << "Run pull_sparse_param"; - auto pull_status = worker_ptr_->load(0,std::string(file_name),std::string("")); + auto pull_status = + worker_ptr_->load(0, std::string(file_name), std::string("")); pull_status.wait(); - vector v; - pull_status = worker_ptr_->sample(0,37,get_graph_node_type(string("user")),v); + std::vector v; + pull_status = worker_ptr_->sample( + 0, 37, get_graph_node_type(std::string("user")), 4, v); pull_status.wait(); - for(auto g:v){ - std::cout<check_response( -// i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { -// ret = -1; -// break; -// } -// VLOG(0)<check_response( -// i, paddle::distributed::PS_PUSH_SPARSE_PARAM); -// } -// closure->set_promise_value(ret); -// }); -// VLOG(0)<<"begin to push_sparse_param"; -// auto push_status = worker_ptr_->push_sparse_param( -// 0, fea_keys.data(), (const float**)fea_value_ptr.data(), fea_keys.size(), -// closure_push_param); -// push_status.wait(); - -// auto pull_param_status = worker_ptr_->pull_sparse( -// fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); -// pull_param_status.wait(); - -// for (size_t idx = 0; idx < tensor->numel(); ++idx) { -// EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); -// } -// LOG(INFO) << "first stage done"; -// /*-----------------------Test Push Grad----------------------------------*/ - -// paddle::distributed::DownpourBrpcClosure* closure_push_grad = -// new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { -// int ret = 0; -// auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; -// for (size_t i = 0; i < 2; ++i) { -// if (closure->check_response( -// i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { -// ret = -1; -// break; -// } -// } -// closure->set_promise_value(ret); -// }); - -// LOG(INFO) << "Run pull_sparse_grad"; -// std::vector push_g_vec; -// for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { -// push_g_vec.push_back(tensor->data() + i * 10); -// } -// auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( -// 0, fea_keys.data(), (const float**)push_g_vec.data(), fea_keys.size(), -// closure_push_grad); -// push_grad_status.wait(); - -// auto pull_update_status = worker_ptr_->pull_sparse( -// fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); -// pull_update_status.wait(); - -// for (size_t idx = 0; idx < tensor->numel(); ++idx) { -// EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); -// } + // LOG(INFO) << "Run push_sparse_param"; + // paddle::distributed::DownpourBrpcClosure* closure_push_param = + // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { + // int ret = 0; + // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; + // for (size_t i = 0; i < 2; ++i) { + // LOG(INFO) << "check response" <check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { + // ret = -1; + // break; + // } + // VLOG(0)<check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_PARAM); + // } + // closure->set_promise_value(ret); + // }); + // VLOG(0)<<"begin to push_sparse_param"; + // auto push_status = worker_ptr_->push_sparse_param( + // 0, fea_keys.data(), (const float**)fea_value_ptr.data(), + // fea_keys.size(), + // closure_push_param); + // push_status.wait(); + + // auto pull_param_status = worker_ptr_->pull_sparse( + // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); + // pull_param_status.wait(); + + // for (size_t idx = 0; idx < tensor->numel(); ++idx) { + // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); + // } + // LOG(INFO) << "first stage done"; + // /*-----------------------Test Push + // Grad----------------------------------*/ + + // paddle::distributed::DownpourBrpcClosure* closure_push_grad = + // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { + // int ret = 0; + // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; + // for (size_t i = 0; i < 2; ++i) { + // if (closure->check_response( + // i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { + // ret = -1; + // break; + // } + // } + // closure->set_promise_value(ret); + // }); + + // LOG(INFO) << "Run pull_sparse_grad"; + // std::vector push_g_vec; + // for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { + // push_g_vec.push_back(tensor->data() + i * 10); + // } + // auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( + // 0, fea_keys.data(), (const float**)push_g_vec.data(), + // fea_keys.size(), + // closure_push_grad); + // push_grad_status.wait(); + + // auto pull_update_status = worker_ptr_->pull_sparse( + // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); + // pull_update_status.wait(); + + // for (size_t idx = 0; idx < tensor->numel(); ++idx) { + // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); + // } LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); @@ -312,8 +320,8 @@ void RunBrpcPushSparse() { testGraphToBuffer(); } -void testGraphToBuffer(){ - ::paddle::distributed::GraphNode s,s1; +void testGraphToBuffer() { + ::paddle::distributed::GraphNode s, s1; s.set_feature("hhhh"); s.set_id(65); s.set_graph_node_type(::paddle::distributed::GraphNodeType(0)); @@ -322,10 +330,8 @@ void testGraphToBuffer(){ s.to_buffer(str); s1.recover_from_buffer(str); ASSERT_EQ(s.get_id(), s1.get_id()); - ASSERT_EQ((int)s.get_graph_node_type(),(int)s1.get_graph_node_type()); - VLOG(0)< Date: Sun, 28 Feb 2021 15:01:12 +0000 Subject: [PATCH 004/106] fix shard_num problem --- .../distributed/service/graph_brpc_client.cc | 1 + .../distributed/service/graph_py_service.cc | 5 +- .../distributed/service/graph_py_service.h | 139 ++++++++---------- paddle/fluid/distributed/service/server.cc | 3 + .../distributed/table/common_graph_table.cc | 61 +++++++- .../distributed/table/common_graph_table.h | 36 +---- .../fluid/distributed/test/graph_node_test.cc | 8 +- paddle/fluid/pybind/CMakeLists.txt | 2 +- paddle/fluid/pybind/graph_service.cc | 15 +- paddle/fluid/pybind/pybind.cc | 4 +- 10 files changed, 149 insertions(+), 125 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index e179f55c49037..bc3be47c386de 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -74,6 +74,7 @@ std::future GraphBrpcClient::sample(uint32_t table_id, int type_int = (int)type; closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); closure->request(0)->add_params((char *)&type_int, sizeof(int)); + closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index e8b0948fd834a..8e0477d869ac0 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -25,7 +25,10 @@ std::vector graph_service::split(std::string &str, } return res; } -void graph_service::init(std::string ips_str, int shard_num) { +void graph_service::set_up(std::string ips_str, int shard_num, int rank, + int client_id) { + set_shard_num(shard_num); + set_client_Id(client_id); std::istringstream stream(ips_str); std::string ip, port; server_size = 0; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 538dc3f85f9f8..ab69e4bd4ed59 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -45,96 +45,87 @@ namespace distributed { class graph_service { std::vector keys; std::vector server_list, port_list, host_sign_list; - int server_size, shard_num; + int server_size, shard_num, rank, client_id; GraphBrpcClient g_client; GraphBrpcServer g_server; public: - // void GetDownpourSparseTableProto( - // ::paddle::distributed::TableParameter* sparse_table_proto) { - // sparse_table_proto->set_table_id(0); - // sparse_table_proto->set_table_class("GraphTable"); - // sparse_table_proto->set_shard_num(256); - // sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); - // ::paddle::distributed::TableAccessorParameter* accessor_proto = - // sparse_table_proto->mutable_accessor(); - // ::paddle::distributed::CommonAccessorParameter* common_proto = - // sparse_table_proto->mutable_common(); + int get_client_id() { return client_id; } + void set_client_Id(int client_Id) { this->client_id = client_id; } + int get_rank() { return rank; } + void set_rank(int rank) { this->rank = rank; } + int get_shard_num() { return shard_num; } + void set_shard_num(int shard_num) { this->shard_num = shard_num; } + void GetDownpourSparseTableProto( + ::paddle::distributed::TableParameter* sparse_table_proto) { + sparse_table_proto->set_table_id(0); + sparse_table_proto->set_table_class("GraphTable"); + sparse_table_proto->set_shard_num(256); + sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); + ::paddle::distributed::TableAccessorParameter* accessor_proto = + sparse_table_proto->mutable_accessor(); + ::paddle::distributed::CommonAccessorParameter* common_proto = + sparse_table_proto->mutable_common(); - // accessor_proto->set_accessor_class("CommMergeAccessor"); - // accessor_proto->set_fea_dim(0); - // accessor_proto->set_embedx_dim(10); - - // common_proto->set_name("sgd"); - // common_proto->set_table_name("MergedDense"); - // common_proto->set_trainer_num(1); - // common_proto->set_sync(false); - // common_proto->add_params("Param"); - // common_proto->add_dims(10); - // common_proto->add_initializers("uniform_random&0&-1.0&1.0"); - // common_proto->add_params("LearningRate"); - // common_proto->add_dims(1); - // common_proto->add_initializers("fill_constant&1.0"); - // } + accessor_proto->set_accessor_class("CommMergeAccessor"); + } - // ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { - // ::paddle::distributed::PSParameter worker_fleet_desc; - // worker_fleet_desc.set_shard_num(shard_num); - // ::paddle::distributed::WorkerParameter* worker_proto = - // worker_fleet_desc.mutable_worker_param(); + ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { + ::paddle::distributed::PSParameter worker_fleet_desc; + worker_fleet_desc.set_shard_num(shard_num); + ::paddle::distributed::WorkerParameter* worker_proto = + worker_fleet_desc.mutable_worker_param(); - // ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = - // worker_proto->mutable_downpour_worker_param(); + ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = + worker_proto->mutable_downpour_worker_param(); - // ::paddle::distributed::TableParameter* worker_sparse_table_proto = - // downpour_worker_proto->add_downpour_table_param(); - // GetDownpourSparseTableProto(worker_sparse_table_proto); + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto); - // ::paddle::distributed::ServerParameter* server_proto = - // worker_fleet_desc.mutable_server_param(); - // ::paddle::distributed::DownpourServerParameter* downpour_server_proto = - // server_proto->mutable_downpour_server_param(); - // ::paddle::distributed::ServerServiceParameter* server_service_proto = - // downpour_server_proto->mutable_service_param(); - // server_service_proto->set_service_class("GraphBrpcService"); - // server_service_proto->set_server_class("GraphBrpcServer"); - // server_service_proto->set_client_class("GraphBrpcClient"); - // server_service_proto->set_start_server_port(0); - // server_service_proto->set_server_thread_num(12); + ::paddle::distributed::ServerParameter* server_proto = + worker_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); - // ::paddle::distributed::TableParameter* server_sparse_table_proto = - // downpour_server_proto->add_downpour_table_param(); - // GetDownpourSparseTableProto(server_sparse_table_proto); + ::paddle::distributed::TableParameter* server_sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(server_sparse_table_proto); - // return worker_fleet_desc; - // } + return worker_fleet_desc; + } void set_server_size(int server_size) { this->server_size = server_size; } int get_server_size(int server_size) { return server_size; } - std::vector split(std::string &str, const char pattern); + std::vector split(std::string& str, const char pattern); void start_client() { - // framework::Scope client_scope; - // platform::CPUPlace place; - // InitTensorsOnClient(&client_scope, &place, 100); - // std::map> - // dense_regions; - // dense_regions.insert( - // std::pair>(0, - // {})); - // auto regions = dense_regions[0]; - // framework::Variable* var = client_scope.FindVar("x"); - // framework::LoDTensor* tensor = var->GetMutable(); + // framework::Scope client_scope; + // platform::CPUPlace place; + // InitTensorsOnClient(&client_scope, &place, 100); + // std::map> + // dense_regions; + // dense_regions.insert( + // std::pair>(0, + // {})); + // auto regions = dense_regions[0]; - // RunClient(dense_regions); - // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); - // paddle::distributed::PaddlePSEnvironment _ps_env; - // auto servers_ = host_sign_list_.size(); - // _ps_env = paddle::distributed::PaddlePSEnvironment(); - // _ps_env.set_ps_servers(&host_sign_list_, servers_); - // worker_ptr_ = std::shared_ptr( - // paddle::distributed::PSClientFactory::create(worker_proto)); - // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); + // RunClient(dense_regions); + // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + // paddle::distributed::PaddlePSEnvironment _ps_env; + // auto servers_ = host_sign_list_.size(); + // _ps_env = paddle::distributed::PaddlePSEnvironment(); + // _ps_env.set_ps_servers(&host_sign_list_, servers_); + // worker_ptr_ = std::shared_ptr( + // paddle::distributed::PSClientFactory::create(worker_proto)); + // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, client_Id); } - void init(std::string ips_str, int shard_num); + void set_up(std::string ips_str, int shard_num, int rank, int client_id); public: void set_keys(std::vector keys) { // just for test diff --git a/paddle/fluid/distributed/service/server.cc b/paddle/fluid/distributed/service/server.cc index fc230a0b9c92e..9324adad6979e 100644 --- a/paddle/fluid/distributed/service/server.cc +++ b/paddle/fluid/distributed/service/server.cc @@ -16,6 +16,7 @@ #include "glog/logging.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" +#include "paddle/fluid/distributed/service/graph_brpc_server.h" #include "paddle/fluid/distributed/table/table.h" namespace paddle { @@ -23,6 +24,8 @@ namespace distributed { REGISTER_PSCORE_CLASS(PSServer, BrpcPsServer); REGISTER_PSCORE_CLASS(PsBaseService, BrpcPsService); +REGISTER_PSCORE_CLASS(PSServer, GraphBrpcServer); +REGISTER_PSCORE_CLASS(PsBaseService, GraphBrpcService); PSServer *PSServerFactory::create(const PSParameter &ps_config) { const auto &config = ps_config.server_param(); diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index d75fc028e4c23..255b9491c5655 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -20,6 +20,37 @@ #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { +int GraphShard::bucket_low_bound = 10; +std::vector GraphShard::get_batch(int start, int total_size) { + if (start < 0) start = 0; + int size = 0, cur_size; + std::vector res; + if (total_size <= 0) return res; + for (int i = 0; i < bucket_size; i++) { + cur_size = bucket[i].size(); + if (size + cur_size <= start) { + size += cur_size; + continue; + } + int read = 0; + std::list::iterator iter = bucket[i].begin(); + while (size + read < start) { + iter++; + read++; + } + read = 0; + while (iter != bucket[i].end() && read < total_size) { + res.push_back(*iter); + iter++; + read++; + } + if (read == total_size) break; + size += cur_size; + start = size; + total_size -= read; + } + return res; +} size_t GraphShard::get_size() { size_t res = 0; for (int i = 0; i < bucket_size; i++) { @@ -47,19 +78,24 @@ GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { return *(node_location[{id, type}]); } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { + VLOG(0) << "in load graph table-->" << path; rwlock_->WRLock(); auto paths = paddle::string::split_string(path, ";"); + VLOG(0) << paths.size(); for (auto path : paths) { + VLOG(0) << "load single path " << path; std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); if (values.size() < 3) continue; auto id = std::stoull(values[0]); - size_t shard_id = id % _shard_num; + size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { VLOG(0) << "will not load " << id << " from " << path << ", please check id distribution"; + VLOG(0) << shard_start << " " << shard_end << " shard_num " + << shard_num; continue; } size_t index = shard_id - shard_start; @@ -79,22 +115,27 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { } for (auto &shard : shards) { auto bucket = shard.get_bucket(); + VLOG(0) << bucket.size() << " bucketsize " + << " shard_start" << shard_start << " shard end " << shard_end; for (int i = 0; i < bucket.size(); i++) { std::list::iterator iter = bucket[i].begin(); while (iter != bucket[i].end()) { auto node = *iter; node->build_sampler(); + VLOG(0) << node->get_id() << " bucket " << i << " shard_start" + << shard_start << " shard end " << shard_end; iter++; } } } } + VLOG(0) << "load data done"; rwlock_->UNLock(); return 0; } GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { - rwlock_->WRLock(); - size_t shard_id = id % _shard_num; + rwlock_->RDLock(); + size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { return NULL; } @@ -107,13 +148,16 @@ GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, int sample_size, char *&buffer, int &actual_size) { + VLOG(0) << "in table random_sample" << node_id << " " << sample_size; rwlock_->RDLock(); + VLOG(0) << "get read rock"; GraphNode *node = find_node(node_id, type); if (node == NULL) { actual_size = 0; rwlock_->UNLock(); return 0; } + VLOG(0) << "find node " << (uint64_t)(&node); std::vector res = node->sample_k(sample_size); std::vector node_list; int total_size = 0; @@ -194,20 +238,27 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, return 0; } int32_t GraphTable::initialize() { + VLOG(0) << " init graph table"; _shards_task_pool.resize(task_pool_size_); for (size_t i = 0; i < _shards_task_pool.size(); ++i) { _shards_task_pool[i].reset(new ::ThreadPool(1)); } server_num = _shard_num; + VLOG(0) << "in init graph table server num = " << server_num; /* _shard_num is actually server number here when a server initialize its tables, it sets tables' _shard_num to server_num, and _shard_idx to server rank */ - _shard_num = _config.shard_num(); - shard_num_per_table = sparse_local_shard_num(_shard_num, server_num); + shard_num = _config.shard_num(); + VLOG(0) << "in init graph table shard num = " << shard_num << " shard_idx" + << _shard_idx; + shard_num_per_table = sparse_local_shard_num(shard_num, server_num); shard_start = _shard_idx * shard_num_per_table; + shard_end = shard_start + shard_num_per_table; + VLOG(0) << "in init graph table shard idx = " << _shard_idx << " shard_start " + << shard_start << " shard_end " << shard_end; shards.resize(shard_num_per_table); return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 028d724a9ffa4..7cc428d5f1d85 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -50,36 +50,7 @@ class GraphShard { bucket.resize(bucket_size); } std::vector> &get_bucket() { return bucket; } - std::vector get_batch(int start, int total_size) { - if (start < 0) start = 0; - int size = 0, cur_size; - std::vector res; - if (total_size <= 0) return res; - for (int i = 0; i < bucket_size; i++) { - cur_size = bucket[i].size(); - if (size + cur_size <= start) { - size += cur_size; - continue; - } - int read = 0; - std::list::iterator iter = bucket[i].begin(); - while (size + read < start) { - iter++; - read++; - } - read = 0; - while (iter != bucket[i].end() && read < total_size) { - res.push_back(*iter); - iter++; - read++; - } - if (read == total_size) break; - size += cur_size; - start = size; - total_size -= read; - } - return res; - } + std::vector get_batch(int start, int total_size); int init_bucket_size(int shard_num) { for (int i = bucket_low_bound;; i++) { if (gcd(i, shard_num) == 1) return i; @@ -99,7 +70,7 @@ class GraphShard { }; class GraphTable : public SparseTable { public: - GraphTable() {} + GraphTable() { rwlock_.reset(new framework::RWLock); } virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); @@ -128,8 +99,7 @@ class GraphTable : public SparseTable { protected: std::vector shards; - std::unordered_set id_set; - size_t shard_start, shard_end, server_num, shard_num_per_table; + size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; std::unique_ptr rwlock_{nullptr}; const int task_pool_size_ = 7; std::vector> _shards_task_pool; diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 20c59c188b803..cfb143348bf1a 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -75,12 +75,13 @@ void GetDownpourSparseTableProto( ::paddle::distributed::TableParameter* sparse_table_proto) { sparse_table_proto->set_table_id(0); sparse_table_proto->set_table_class("GraphTable"); - sparse_table_proto->set_shard_num(256); + sparse_table_proto->set_shard_num(127); sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); ::paddle::distributed::TableAccessorParameter* accessor_proto = sparse_table_proto->mutable_accessor(); - ::paddle::distributed::CommonAccessorParameter* common_proto = - sparse_table_proto->mutable_common(); + // ::paddle::distributed::CommonAccessorParameter* common_proto = + // sparse_table_proto->mutable_common(); + accessor_proto->set_accessor_class("CommMergeAccessor"); } distributed::GraphNodeType get_graph_node_type(std::string str) { @@ -232,6 +233,7 @@ void RunBrpcPushSparse() { LOG(INFO) << "Run pull_sparse_param"; auto pull_status = worker_ptr_->load(0, std::string(file_name), std::string("")); + pull_status.wait(); std::vector v; pull_status = worker_ptr_->sample( diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 1877f9f971181..34a733d7ccb0a 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -1,7 +1,7 @@ set(PYBIND_DEPS pybind python proto_desc memory executor fleet_wrapper box_wrapper prune feed_fetch_method pass_builder parallel_executor profiler layer tracer engine scope_pool analysis_predictor imperative_profiler imperative_flag save_load_util dlpack_tensor device_context - gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper) + gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper ps_service graph_py_service) if (WITH_NCCL) set(PYBIND_DEPS ${PYBIND_DEPS} nccl_wrapper) diff --git a/paddle/fluid/pybind/graph_service.cc b/paddle/fluid/pybind/graph_service.cc index e7172773b4dae..1113f2e7ae94f 100644 --- a/paddle/fluid/pybind/graph_service.cc +++ b/paddle/fluid/pybind/graph_service.cc @@ -13,18 +13,21 @@ // limitations under the License. #include "paddle/fluid/pybind/graph_service.h" -#include "paddle/fluid/distributed/service/graph_service.cc" +// #include "paddle/fluid/distributed/service/graph_py_service.h" +// #include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "brpc/stream.h" namespace py = pybind11; -using paddle::distributed::graph_service; +// using paddle::distributed::graph_service; namespace paddle { namespace pybind { void BindGraphService(py::module* m) { - py::class_(*m, "graph_service") - .def(py::init<>()) - .def("set_keys", &graph_service::set_keys) - .def("get_keys", &graph_service::get_keys); + // py::class_(*m, "graph_service") + // .def( + // py::init<>()) + // .def("set_keys", &paddle::distributed::graph_service::set_keys) + // .def("get_keys",&paddle::distributed::graph_service::get_keys); } } // namespace pybind diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 5e14ee0bb8b81..baae2931dbd15 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -113,8 +113,8 @@ limitations under the License. */ #include "paddle/fluid/pybind/fleet_py.h" #endif -#include "paddle/fluid/pybind/graph_service.h" #include "pybind11/stl.h" +//#include "paddle/fluid/pybind/graph_ps_service.h" DECLARE_bool(use_mkldnn); @@ -2845,7 +2845,7 @@ All parameter, weight, gradient are variables in Paddle. BindDistFleetWrapper(&m); BindPSHost(&m); BindCommunicatorContext(&m); - BindGraphService(&m); + // BindGraphService(&m); BindDistCommunicator(&m); BindHeterClient(&m); #endif From acacce0f47db6293877b10ccab8209dfa689cbe9 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 2 Mar 2021 08:59:02 +0000 Subject: [PATCH 005/106] py client --- .../distributed/service/graph_brpc_client.cc | 15 +- .../distributed/service/graph_brpc_client.h | 5 +- .../distributed/service/graph_brpc_server.cc | 9 +- .../distributed/service/graph_py_service.cc | 23 ++-- .../distributed/service/graph_py_service.h | 129 ++++++++++++++---- paddle/fluid/distributed/service/ps_client.h | 3 +- .../distributed/table/common_graph_table.cc | 49 ++++--- .../distributed/table/common_graph_table.h | 5 + paddle/fluid/distributed/table/graph_node.h | 19 +++ paddle/fluid/distributed/table/table.h | 3 +- paddle/fluid/distributed/test/CMakeLists.txt | 2 +- .../fluid/distributed/test/graph_node_test.cc | 128 +++++++---------- 12 files changed, 236 insertions(+), 154 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index bc3be47c386de..a9801ea9a2162 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -71,9 +71,9 @@ std::future GraphBrpcClient::sample(uint32_t table_id, closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; + std::string type_str = GraphNode::node_type_to_string(type); closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, sizeof(int)); + closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); @@ -84,9 +84,8 @@ std::future GraphBrpcClient::sample(uint32_t table_id, } std::future GraphBrpcClient::pull_graph_list( - uint32_t table_id, uint64_t node_id, GraphNodeType type, int start, - int size, std::vector &res) { - int server_index = get_server_index_by_id(node_id); + uint32_t table_id, int server_index, int start, int size, + std::vector &res) { DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; @@ -117,9 +116,9 @@ std::future GraphBrpcClient::pull_graph_list( closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - int type_int = (int)type; - closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params((char *)&type_int, sizeof(int)); + // std::string type_str = GraphNode::node_type_to_string(type); + // closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); + // closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&start, sizeof(int)); closure->request(0)->add_params((char *)&size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 0efb6bbabaec1..c759e95c86bfc 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -39,9 +39,8 @@ class GraphBrpcClient : public BrpcPsClient { GraphNodeType type, int sample_size, std::vector &res); virtual std::future pull_graph_list(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, int start, - int end, + int server_index, int start, + int size, std::vector &res); virtual int32_t initialize(); int get_shard_num() { return shard_num; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index d8c0de5f476cd..3b1b3d74907f4 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -263,15 +263,12 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, "pull_graph_list request requires at least 2 arguments"); return 0; } - uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); - std::string type_str = request.params(1); - GraphNodeType type = GraphNode::get_graph_node_type(type_str); - int start = *(int *)(request.params(2).c_str()); - int size = *(int *)(request.params(3).c_str()); + int start = *(int *)(request.params(0).c_str()); + int size = *(int *)(request.params(1).c_str()); std::vector res_data; char *buffer; int actual_size; - table->pull_graph_list(node_id, type, start, size, buffer, actual_size); + table->pull_graph_list(start, size, buffer, actual_size); cntl->response_attachment().append(buffer, actual_size); return 0; } diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 8e0477d869ac0..86d2d54193bd9 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -15,8 +15,8 @@ #include "paddle/fluid/distributed/service/graph_py_service.h" namespace paddle { namespace distributed { -std::vector graph_service::split(std::string &str, - const char pattern) { +std::vector GraphPyService::split(std::string &str, + const char pattern) { std::vector res; std::stringstream input(str); std::string temp; @@ -25,12 +25,16 @@ std::vector graph_service::split(std::string &str, } return res; } -void graph_service::set_up(std::string ips_str, int shard_num, int rank, - int client_id) { + +void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, + int client_id, uint32_t table_id) { set_shard_num(shard_num); set_client_Id(client_id); + set_rank(rank); + this->table_id = table_id; + server_thread = client_thread = NULL; std::istringstream stream(ips_str); - std::string ip, port; + std::string ip; server_size = 0; std::vector ips_list = split(ips_str, ';'); int index = 0; @@ -38,12 +42,15 @@ void graph_service::set_up(std::string ips_str, int shard_num, int rank, auto ip_and_port = split(ips, ':'); server_list.push_back(ip_and_port[0]); port_list.push_back(ip_and_port[1]); - // auto ph_host = paddle::distributed::PSHost(ip_and_port[0], - // ip_and_port[1], index); - // host_sign_list_.push_back(ph_host.serialize_to_string()); + uint32_t port = stoul(ip_and_port[1]); + auto ph_host = paddle::distributed::PSHost(ip_and_port[0], port, index); + host_sign_list.push_back(ph_host.serialize_to_string()); index++; } + VLOG(0) << "IN set up rank = " << rank; start_client(); + start_server(server_list[rank], std::stoul(port_list[rank])); + sleep(1); } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index ab69e4bd4ed59..fc8d7bfad7ddd 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -42,14 +42,24 @@ #include "paddle/fluid/string/printf.h" namespace paddle { namespace distributed { -class graph_service { +class GraphPyService { std::vector keys; std::vector server_list, port_list, host_sign_list; int server_size, shard_num, rank, client_id; - GraphBrpcClient g_client; - GraphBrpcServer g_server; + uint32_t table_id; + std::thread *server_thread, *client_thread; + + std::shared_ptr pserver_ptr; + + std::shared_ptr worker_ptr; public: + std::shared_ptr get_ps_server() { + return pserver_ptr; + } + std::shared_ptr get_ps_client() { + return worker_ptr; + } int get_client_id() { return client_id; } void set_client_Id(int client_Id) { this->client_id = client_id; } int get_rank() { return rank; } @@ -58,9 +68,9 @@ class graph_service { void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( ::paddle::distributed::TableParameter* sparse_table_proto) { - sparse_table_proto->set_table_id(0); + sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); - sparse_table_proto->set_shard_num(256); + sparse_table_proto->set_shard_num(shard_num); sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); ::paddle::distributed::TableAccessorParameter* accessor_proto = sparse_table_proto->mutable_accessor(); @@ -70,7 +80,29 @@ class graph_service { accessor_proto->set_accessor_class("CommMergeAccessor"); } - ::paddle::distributed::PSParameter GetWorkerProto(int shard_num) { + ::paddle::distributed::PSParameter GetServerProto() { + // Generate server proto desc + ::paddle::distributed::PSParameter server_fleet_desc; + server_fleet_desc.set_shard_num(shard_num); + ::paddle::distributed::ServerParameter* server_proto = + server_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); + + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto); + return server_fleet_desc; + } + + ::paddle::distributed::PSParameter GetWorkerProto() { ::paddle::distributed::PSParameter worker_fleet_desc; worker_fleet_desc.set_shard_num(shard_num); ::paddle::distributed::WorkerParameter* worker_proto = @@ -104,30 +136,71 @@ class graph_service { void set_server_size(int server_size) { this->server_size = server_size; } int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); - void start_client() { - // framework::Scope client_scope; - // platform::CPUPlace place; - // InitTensorsOnClient(&client_scope, &place, 100); - // std::map> - // dense_regions; - // dense_regions.insert( - // std::pair>(0, - // {})); - // auto regions = dense_regions[0]; - - // RunClient(dense_regions); - // ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); - // paddle::distributed::PaddlePSEnvironment _ps_env; - // auto servers_ = host_sign_list_.size(); - // _ps_env = paddle::distributed::PaddlePSEnvironment(); - // _ps_env.set_ps_servers(&host_sign_list_, servers_); - // worker_ptr_ = std::shared_ptr( - // paddle::distributed::PSClientFactory::create(worker_proto)); - // worker_ptr_->configure(worker_proto, dense_regions, _ps_env, client_Id); + + void load_file(std::string filepath) { + auto status = + get_ps_client()->load(table_id, std::string(filepath), std::string("")); + status.wait(); } - void set_up(std::string ips_str, int shard_num, int rank, int client_id); - public: + std::vector sample_k(uint64_t node_id, std::string type, + int sample_size) { + std::vector v; + auto status = worker_ptr->sample(table_id, node_id, + GraphNode::get_graph_node_type(type), + sample_size, v); + status.wait(); + return v; + } + std::vector pull_graph_list(int server_index, int start, + int size) { + std::vector res; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); + return res; + } + void start_server(std::string ip, uint32_t port) { + server_thread = new std::thread([this, &ip, &port]() { + std::function func = [this, &ip, &port]() { + VLOG(0) << "enter inner function "; + ::paddle::distributed::PSParameter server_proto = + this->GetServerProto(); + + auto _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&this->host_sign_list, + this->host_sign_list.size()); // test + pserver_ptr = std::shared_ptr( + paddle::distributed::PSServerFactory::create(server_proto)); + VLOG(0) << "pserver-ptr created "; + std::vector empty_vec; + framework::ProgramDesc empty_prog; + empty_vec.push_back(empty_prog); + pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); + VLOG(0) << " starting server " << ip << " " << port; + pserver_ptr->start(ip, port); + }; + std::thread t1(func); + t1.join(); + }); + } + void start_client() { + VLOG(0) << "in start_client " << rank; + std::map> dense_regions; + dense_regions.insert( + std::pair>(0, {})); + auto regions = dense_regions[0]; + ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + paddle::distributed::PaddlePSEnvironment _ps_env; + auto servers_ = host_sign_list.size(); + _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&host_sign_list, servers_); + worker_ptr = std::shared_ptr( + paddle::distributed::PSClientFactory::create(worker_proto)); + worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); + } + void set_up(std::string ips_str, int shard_num, int rank, int client_id, + uint32_t table_id); void set_keys(std::vector keys) { // just for test this->keys = keys; } diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 08a3881ad0bd2..6e169b9a594f5 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -165,8 +165,7 @@ class PSClient { return fut; } virtual std::future pull_graph_list(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, int start, + int server_index, int start, int size, std::vector &res) { LOG(FATAL) << "Did not implement"; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 255b9491c5655..5001c48ceaf88 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -20,7 +20,7 @@ #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { -int GraphShard::bucket_low_bound = 10; +int GraphShard::bucket_low_bound = 11; std::vector GraphShard::get_batch(int start, int total_size) { if (start < 0) start = 0; int size = 0, cur_size; @@ -78,12 +78,10 @@ GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { return *(node_location[{id, type}]); } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { - VLOG(0) << "in load graph table-->" << path; rwlock_->WRLock(); auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); for (auto path : paths) { - VLOG(0) << "load single path " << path; std::ifstream file(path); std::string line; while (std::getline(file, line)) { @@ -94,20 +92,27 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { if (shard_id >= shard_end || shard_id < shard_start) { VLOG(0) << "will not load " << id << " from " << path << ", please check id distribution"; - VLOG(0) << shard_start << " " << shard_end << " shard_num " - << shard_num; continue; } size_t index = shard_id - shard_start; GraphNodeType type = GraphNode::get_graph_node_type(values[1]); - shards[index].add_node(new GraphNode(id, type, values[2])); - for (size_t i = 3; i < values.size(); i++) { + // VLOG(0)<<"shards's size = "<get_id(); + for (size_t i = 2; i < values.size(); i++) { auto edge_arr = paddle::string::split_string(values[i], ";"); - if (edge_arr.size() < 3) { + if (edge_arr.size() == 3) { + // VLOG(0)<<"edge content "<::iterator iter = bucket[i].begin(); while (iter != bucket[i].end()) { auto node = *iter; node->build_sampler(); - VLOG(0) << node->get_id() << " bucket " << i << " shard_start" - << shard_start << " shard end " << shard_end; + // VLOG(0) << node->get_id() << " bucket " << i << " shard_start" + // << shard_start << " shard end " << shard_end<<" shard idx + // "<<_shard_idx; + // for(auto edge:(*iter)->get_graph_edge()){ + // VLOG(0)<<(*iter)->get_id()<<" has a neighboor "<id; + // } + // auto p = + // shard.get_node_location()[{node->get_id(),node->get_graph_node_type()}]; + // VLOG(0)<<"check iter equal "<<(*p)->get_id()<<" --- "<<(*p == + // *iter); iter++; } } } } - VLOG(0) << "load data done"; rwlock_->UNLock(); return 0; } @@ -137,10 +147,12 @@ GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { rwlock_->RDLock(); size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { + rwlock_->UNLock(); return NULL; } size_t index = shard_id - shard_start; - + // VLOG(0)<<"try to find node-id "<UNLock(); return node; @@ -148,16 +160,13 @@ GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, int sample_size, char *&buffer, int &actual_size) { - VLOG(0) << "in table random_sample" << node_id << " " << sample_size; rwlock_->RDLock(); - VLOG(0) << "get read rock"; GraphNode *node = find_node(node_id, type); if (node == NULL) { actual_size = 0; rwlock_->UNLock(); return 0; } - VLOG(0) << "find node " << (uint64_t)(&node); std::vector res = node->sample_k(sample_size); std::vector node_list; int total_size = 0; @@ -238,13 +247,12 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, return 0; } int32_t GraphTable::initialize() { - VLOG(0) << " init graph table"; _shards_task_pool.resize(task_pool_size_); for (size_t i = 0; i < _shards_task_pool.size(); ++i) { _shards_task_pool[i].reset(new ::ThreadPool(1)); } server_num = _shard_num; - VLOG(0) << "in init graph table server num = " << server_num; + // VLOG(0) << "in init graph table server num = " << server_num; /* _shard_num is actually server number here when a server initialize its tables, it sets tables' _shard_num to server_num, @@ -259,7 +267,8 @@ int32_t GraphTable::initialize() { shard_end = shard_start + shard_num_per_table; VLOG(0) << "in init graph table shard idx = " << _shard_idx << " shard_start " << shard_start << " shard_end " << shard_end; - shards.resize(shard_num_per_table); + // shards.resize(shard_num_per_table); + shards = std::vector(shard_num_per_table, GraphShard(shard_num)); return 0; } } diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 7cc428d5f1d85..0dc1c7392c80e 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -60,6 +60,11 @@ class GraphShard { std::list::iterator add_node(GraphNode *node); GraphNode *find_node(uint64_t id, GraphNodeType type); void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); + std::unordered_map, + std::list::iterator, pair_hash> + get_node_location() { + return node_location; + } private: std::unordered_map, diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 9c30c6b1dfc35..25045019f4f9f 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -33,6 +33,7 @@ class GraphNode { GraphNode(uint64_t id, GraphNodeType type, std::string feature) : id(id), type(type), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} + std::vector get_graph_edge() { return edges; } static int enum_size, id_size, int_size, double_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } @@ -45,6 +46,7 @@ class GraphNode { virtual void to_buffer(char *buffer); virtual void recover_from_buffer(char *buffer); virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } + std::string get_type_string() { return node_type_to_string(this->type); } static GraphNodeType get_graph_node_type(std::string &str) { GraphNodeType type; if (str == "user") @@ -58,6 +60,23 @@ class GraphNode { type = GraphNodeType::unknown; return type; } + static std::string node_type_to_string(GraphNodeType type) { + std::string res; + switch (type) { + case GraphNodeType::user: + res = "user"; + break; + case GraphNodeType::item: + res = "item"; + break; + case GraphNodeType::query: + res = "query"; + break; + default: + res = "unKnown"; + } + return res; + } std::vector sample_k(int k) { std::vector v; if (sampler != NULL) { diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 7161d2016004d..f58fc02b8d27b 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -88,8 +88,7 @@ class Table { } // only for graph table - virtual int32_t pull_graph_list(uint64_t id, GraphNodeType type, int start, - int total_size, char *&buffer, + virtual int32_t pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { return 0; } diff --git a/paddle/fluid/distributed/test/CMakeLists.txt b/paddle/fluid/distributed/test/CMakeLists.txt index 2bf0eeebff3c3..69cb3bfc36d5f 100644 --- a/paddle/fluid/distributed/test/CMakeLists.txt +++ b/paddle/fluid/distributed/test/CMakeLists.txt @@ -38,4 +38,4 @@ set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRI cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) +cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_py_service scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index cfb143348bf1a..9a82137d0383a 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -30,6 +30,7 @@ limitations under the License. */ #include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" #include "paddle/fluid/distributed/service/env.h" +#include "paddle/fluid/distributed/service/graph_py_service.h" #include "paddle/fluid/distributed/service/ps_client.h" #include "paddle/fluid/distributed/service/sendrecv.pb.h" #include "paddle/fluid/distributed/service/service.h" @@ -49,7 +50,9 @@ namespace distributed = paddle::distributed; void testGraphToBuffer(); std::string nodes[] = { std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), - std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; + std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21"), + std::string("59\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), + std::string("97\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -79,16 +82,14 @@ void GetDownpourSparseTableProto( sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); ::paddle::distributed::TableAccessorParameter* accessor_proto = sparse_table_proto->mutable_accessor(); - // ::paddle::distributed::CommonAccessorParameter* common_proto = - // sparse_table_proto->mutable_common(); accessor_proto->set_accessor_class("CommMergeAccessor"); } distributed::GraphNodeType get_graph_node_type(std::string str) { distributed::GraphNodeType type; - if (str == "user") + if (str == "user") { type = distributed::GraphNodeType::user; - else if (str == "item") + } else if (str == "item") type = distributed::GraphNodeType::item; else if (str == "query") type = distributed::GraphNodeType::query; @@ -230,7 +231,6 @@ void RunBrpcPushSparse() { RunClient(dense_regions); /*-----------------------Test Server Init----------------------------------*/ - LOG(INFO) << "Run pull_sparse_param"; auto pull_status = worker_ptr_->load(0, std::string(file_name), std::string("")); @@ -239,80 +239,56 @@ void RunBrpcPushSparse() { pull_status = worker_ptr_->sample( 0, 37, get_graph_node_type(std::string("user")), 4, v); pull_status.wait(); + // for (auto g : v) { + // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + // } + ASSERT_EQ(v.size(), 3); + v.clear(); + pull_status = worker_ptr_->sample( + 0, 96, get_graph_node_type(std::string("user")), 4, v); + pull_status.wait(); + for (auto g : v) { + std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + } + // ASSERT_EQ(v.size(),3); + v.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, v); + pull_status.wait(); + ASSERT_EQ(v.size(), 1); + ASSERT_EQ(v[0].get_id(), 37); + // for (auto g : v) { + // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + // } + // ASSERT_EQ(v.size(),1); + v.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, v); + pull_status.wait(); + ASSERT_EQ(v.size(), 1); + ASSERT_EQ(v[0].get_id(), 59); for (auto g : v) { std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; } - /*-----------------------Test Push Param----------------------------------*/ - - // LOG(INFO) << "Run push_sparse_param"; - // paddle::distributed::DownpourBrpcClosure* closure_push_param = - // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { - // int ret = 0; - // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; - // for (size_t i = 0; i < 2; ++i) { - // LOG(INFO) << "check response" <check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_PARAM) != 0) { - // ret = -1; - // break; - // } - // VLOG(0)<check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_PARAM); - // } - // closure->set_promise_value(ret); - // }); - // VLOG(0)<<"begin to push_sparse_param"; - // auto push_status = worker_ptr_->push_sparse_param( - // 0, fea_keys.data(), (const float**)fea_value_ptr.data(), - // fea_keys.size(), - // closure_push_param); - // push_status.wait(); - - // auto pull_param_status = worker_ptr_->pull_sparse( - // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); - // pull_param_status.wait(); - - // for (size_t idx = 0; idx < tensor->numel(); ++idx) { - // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx]); - // } - // LOG(INFO) << "first stage done"; - // /*-----------------------Test Push - // Grad----------------------------------*/ - - // paddle::distributed::DownpourBrpcClosure* closure_push_grad = - // new paddle::distributed::DownpourBrpcClosure(2, [&](void* done) { - // int ret = 0; - // auto* closure = (paddle::distributed::DownpourBrpcClosure*)done; - // for (size_t i = 0; i < 2; ++i) { - // if (closure->check_response( - // i, paddle::distributed::PS_PUSH_SPARSE_TABLE) != 0) { - // ret = -1; - // break; - // } - // } - // closure->set_promise_value(ret); - // }); - - // LOG(INFO) << "Run pull_sparse_grad"; - // std::vector push_g_vec; - // for (auto i = 0; i < static_cast(fea_keys.size()); ++i) { - // push_g_vec.push_back(tensor->data() + i * 10); - // } - // auto push_grad_status = worker_ptr_->push_sparse_raw_gradient( - // 0, fea_keys.data(), (const float**)push_g_vec.data(), - // fea_keys.size(), - // closure_push_grad); - // push_grad_status.wait(); - - // auto pull_update_status = worker_ptr_->pull_sparse( - // fea_temp_value_ptr.data(), 0, fea_keys.data(), fea_keys.size()); - // pull_update_status.wait(); - - // for (size_t idx = 0; idx < tensor->numel(); ++idx) { - // EXPECT_FLOAT_EQ(fea_temp_values[idx], fea_values[idx] - 1.0); - // } + distributed::GraphPyService gps1, gps2; + std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; + gps1.set_up(ips_str, 127, 0, 0, 0); + gps2.set_up(ips_str, 127, 1, 1, 0); + gps1.load_file(std::string(file_name)); + v.clear(); + v = gps2.pull_graph_list(0, 1, 4); + ASSERT_EQ(v[0].get_id(), 59); + // for (auto g : v) { + // std::cout << g.get_id() << " service-test " << g.get_graph_node_type() + // << std::endl; + // } + v.clear(); + v = gps2.sample_k(96, std::string("user"), 4); + ASSERT_EQ(v.size(), 3); + // for (auto g : v) { + // std::cout << g.get_id() << " service-test--neighboor " << + // g.get_graph_node_type() << std::endl; + // } + std::remove(file_name); LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); LOG(INFO) << "Run finalize_worker"; From 3f32bf1b5e59a732d88653d12d0cb50548ccff57 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 9 Mar 2021 04:18:42 +0000 Subject: [PATCH 006/106] remove lock and graph-type --- .../distributed/service/graph_brpc_client.cc | 11 +- .../distributed/service/graph_brpc_client.h | 2 +- .../distributed/service/graph_brpc_server.cc | 10 +- .../distributed/service/graph_py_service.h | 7 +- paddle/fluid/distributed/service/ps_client.h | 2 +- .../distributed/table/common_graph_table.cc | 147 +++++++++--------- .../distributed/table/common_graph_table.h | 24 +-- paddle/fluid/distributed/table/graph_node.cc | 14 +- paddle/fluid/distributed/table/graph_node.h | 49 ++---- paddle/fluid/distributed/table/table.h | 3 +- .../fluid/distributed/test/graph_node_test.cc | 74 ++++----- paddle/fluid/pybind/CMakeLists.txt | 1 - paddle/fluid/pybind/fleet_py.cc | 19 +++ paddle/fluid/pybind/fleet_py.h | 2 + paddle/fluid/pybind/graph_service.cc | 34 ---- paddle/fluid/pybind/graph_service.h | 28 ---- paddle/fluid/pybind/pybind.cc | 4 +- 17 files changed, 159 insertions(+), 272 deletions(-) delete mode 100644 paddle/fluid/pybind/graph_service.cc delete mode 100644 paddle/fluid/pybind/graph_service.h diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index a9801ea9a2162..16014c8dbf23f 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -36,9 +36,7 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { } // char* &buffer,int &actual_size std::future GraphBrpcClient::sample(uint32_t table_id, - uint64_t node_id, - GraphNodeType type, - int sample_size, + uint64_t node_id, int sample_size, std::vector &res) { int server_index = get_server_index_by_id(node_id); DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { @@ -71,9 +69,9 @@ std::future GraphBrpcClient::sample(uint32_t table_id, closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - std::string type_str = GraphNode::node_type_to_string(type); + // std::string type_str = GraphNode::node_type_to_string(type); closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params(type_str.c_str(), type_str.size()); + // closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); @@ -116,9 +114,6 @@ std::future GraphBrpcClient::pull_graph_list( closure->request(0)->set_cmd_id(PS_PULL_GRAPH_LIST); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); - // closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - // closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&start, sizeof(int)); closure->request(0)->add_params((char *)&size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index c759e95c86bfc..8e472b96be94d 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -36,7 +36,7 @@ class GraphBrpcClient : public BrpcPsClient { GraphBrpcClient() {} virtual ~GraphBrpcClient() {} virtual std::future sample(uint32_t table_id, uint64_t node_id, - GraphNodeType type, int sample_size, + int sample_size, std::vector &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 3b1b3d74907f4..ecb078c0ad400 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -277,19 +277,17 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, PsResponseMessage &response, brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) - if (request.params_size() < 3) { + if (request.params_size() < 2) { set_response_code( response, -1, - "graph_random_sample request requires at least 3 arguments"); + "graph_random_sample request requires at least 2 arguments"); return 0; } uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); - std::string type_str = request.params(1); - int sample_size = *(uint64_t *)(request.params(2).c_str()); - GraphNodeType type = GraphNode::get_graph_node_type(type_str); + int sample_size = *(uint64_t *)(request.params(1).c_str()); char *buffer; int actual_size; - table->random_sample(node_id, type, sample_size, buffer, actual_size); + table->random_sample(node_id, sample_size, buffer, actual_size); cntl->response_attachment().append(buffer, actual_size); return 0; } diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index fc8d7bfad7ddd..be946dc44e57d 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -143,12 +143,9 @@ class GraphPyService { status.wait(); } - std::vector sample_k(uint64_t node_id, std::string type, - int sample_size) { + std::vector sample_k(uint64_t node_id, int sample_size) { std::vector v; - auto status = worker_ptr->sample(table_id, node_id, - GraphNode::get_graph_node_type(type), - sample_size, v); + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); status.wait(); return v; } diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 6e169b9a594f5..b6014b9aea139 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -156,7 +156,7 @@ class PSClient { return fut; } virtual std::future sample(uint32_t table_id, uint64_t node_id, - GraphNodeType type, int sample_size, + int sample_size, std::vector &res) { LOG(FATAL) << "Did not implement"; std::promise promise; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 5001c48ceaf88..5c97ba631d57d 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -59,26 +59,22 @@ size_t GraphShard::get_size() { return res; } std::list::iterator GraphShard::add_node(GraphNode *node) { - if (node_location.find({node->get_id(), node->get_graph_node_type()}) != - node_location.end()) - return node_location.find({node->get_id(), node->get_graph_node_type()}) - ->second; + if (node_location.find(node->get_id()) != node_location.end()) + return node_location.find(node->get_id())->second; int index = node->get_id() % shard_num % bucket_size; std::list::iterator iter = bucket[index].insert(bucket[index].end(), node); - node_location[{node->get_id(), node->get_graph_node_type()}] = iter; + node_location[node->get_id()] = iter; return iter; } -void GraphShard::add_neighboor(uint64_t id, GraphNodeType type, - GraphEdge *edge) { - (*add_node(new GraphNode(id, type, std::string(""))))->add_edge(edge); +void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { + (*add_node(new GraphNode(id, std::string(""))))->add_edge(edge); } -GraphNode *GraphShard::find_node(uint64_t id, GraphNodeType type) { - if (node_location.find({id, type}) == node_location.end()) return NULL; - return *(node_location[{id, type}]); +GraphNode *GraphShard::find_node(uint64_t id) { + if (node_location.find(id) == node_location.end()) return NULL; + return *(node_location[id]); } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { - rwlock_->WRLock(); auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); for (auto path : paths) { @@ -86,7 +82,7 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); - if (values.size() < 3) continue; + if (values.size() < 2) continue; auto id = std::stoull(values[0]); size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { @@ -95,26 +91,24 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { continue; } size_t index = shard_id - shard_start; - GraphNodeType type = GraphNode::get_graph_node_type(values[1]); + // GraphNodeType type = GraphNode::get_graph_node_type(values[1]); // VLOG(0)<<"shards's size = "<get_id(); for (size_t i = 2; i < values.size(); i++) { auto edge_arr = paddle::string::split_string(values[i], ";"); - if (edge_arr.size() == 3) { + if (edge_arr.size() == 2) { // VLOG(0)<<"edge content "<build_sampler(); - // VLOG(0) << node->get_id() << " bucket " << i << " shard_start" - // << shard_start << " shard end " << shard_end<<" shard idx - // "<<_shard_idx; - // for(auto edge:(*iter)->get_graph_edge()){ - // VLOG(0)<<(*iter)->get_id()<<" has a neighboor "<id; - // } - // auto p = - // shard.get_node_location()[{node->get_id(),node->get_graph_node_type()}]; - // VLOG(0)<<"check iter equal "<<(*p)->get_id()<<" --- "<<(*p == - // *iter); iter++; } } } } - rwlock_->UNLock(); return 0; } -GraphNode *GraphTable::find_node(uint64_t id, GraphNodeType type) { - rwlock_->RDLock(); +GraphNode *GraphTable::find_node(uint64_t id) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { - rwlock_->UNLock(); return NULL; } size_t index = shard_id - shard_start; // VLOG(0)<<"try to find node-id "<UNLock(); + GraphNode *node = shards[index].find_node(id); return node; } -int32_t GraphTable::random_sample(uint64_t node_id, GraphNodeType type, - int sample_size, char *&buffer, - int &actual_size) { - rwlock_->RDLock(); - GraphNode *node = find_node(node_id, type); - if (node == NULL) { - actual_size = 0; - rwlock_->UNLock(); - return 0; - } - std::vector res = node->sample_k(sample_size); - std::vector node_list; - int total_size = 0; - for (auto x : res) { - GraphNode temp; - temp.set_id(x->id); - temp.set_graph_node_type(x->type); - total_size += temp.get_size(); - node_list.push_back(temp); - } - buffer = new char[total_size]; - int index = 0; - for (auto x : node_list) { - x.to_buffer(buffer + index); - index += x.get_size(); - } - actual_size = total_size; - rwlock_->UNLock(); - return 0; +uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { + return node_id % shard_num_per_table % task_pool_size_; +} +int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, + char *&buffer, int &actual_size) { + return _shards_task_pool[get_thread_pool_index(node_id)] + ->enqueue([&]() -> int { + GraphNode *node = find_node(node_id); + if (node == NULL) { + actual_size = 0; + return 0; + } + std::vector res = node->sample_k(sample_size); + std::vector node_list; + int total_size = 0; + for (auto x : res) { + GraphNode temp; + temp.set_id(x->id); + total_size += temp.get_size(); + node_list.push_back(temp); + } + buffer = new char[total_size]; + int index = 0; + for (auto x : node_list) { + x.to_buffer(buffer + index); + index += x.get_size(); + } + actual_size = total_size; + return 0; + }) + .get(); + // GraphNode *node = find_node(node_id, type); + // if (node == NULL) { + // actual_size = 0; + // rwlock_->UNLock(); + // return 0; + // } + // std::vector res = node->sample_k(sample_size); + // std::vector node_list; + // int total_size = 0; + // for (auto x : res) { + // GraphNode temp; + // temp.set_id(x->id); + // temp.set_graph_node_type(x->type); + // total_size += temp.get_size(); + // node_list.push_back(temp); + // } + // buffer = new char[total_size]; + // int index = 0; + // for (auto x : node_list) { + // x.to_buffer(buffer + index); + // index += x.get_size(); + // } + // actual_size = total_size; + // rwlock_->UNLock(); + // return 0; } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { - rwlock_->RDLock(); - int thread_index = 0; if (start < 0) start = 0; int size = 0, cur_size; if (total_size <= 0) { @@ -205,14 +210,13 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, continue; } if (size + cur_size - start >= total_size) { - tasks.push_back(_shards_task_pool[thread_index]->enqueue( + tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( [this, i, start, size, total_size]() -> std::vector { return this->shards[i].get_batch(start - size, total_size); })); - thread_index++; break; } else { - tasks.push_back(_shards_task_pool[thread_index]->enqueue( + tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( [this, i, start, size, total_size, cur_size]() -> std::vector { return this->shards[i].get_batch(start - size, @@ -243,7 +247,6 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, } } actual_size = size; - rwlock_->UNLock(); return 0; } int32_t GraphTable::initialize() { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 0dc1c7392c80e..1f2b8c86d363b 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -30,11 +30,6 @@ #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { -struct pair_hash { - inline size_t operator()(const std::pair &p) const { - return p.first * 10007 + int(p.second); - } -}; class GraphShard { public: static int bucket_low_bound; @@ -58,18 +53,15 @@ class GraphShard { return -1; } std::list::iterator add_node(GraphNode *node); - GraphNode *find_node(uint64_t id, GraphNodeType type); - void add_neighboor(uint64_t id, GraphNodeType type, GraphEdge *edge); - std::unordered_map, - std::list::iterator, pair_hash> + GraphNode *find_node(uint64_t id); + void add_neighboor(uint64_t id, GraphEdge *edge); + std::unordered_map::iterator> get_node_location() { return node_location; } private: - std::unordered_map, - std::list::iterator, pair_hash> - node_location; + std::unordered_map::iterator> node_location; int bucket_size, shard_num; std::vector> bucket; }; @@ -79,12 +71,11 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, - int sampe_size, char *&buffer, + virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); - GraphNode *find_node(uint64_t id, GraphNodeType type); + GraphNode *find_node(uint64_t id); virtual int32_t pull_sparse(float *values, const uint64_t *keys, size_t num) { return 0; @@ -101,12 +92,13 @@ class GraphTable : public SparseTable { return 0; } virtual int32_t initialize_shard() { return 0; } + virtual uint32_t get_thread_pool_index(uint64_t node_id); protected: std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; std::unique_ptr rwlock_{nullptr}; - const int task_pool_size_ = 7; + const int task_pool_size_ = 11; std::vector> _shards_task_pool; }; } diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 8263be586a717..78a586d507ef4 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -20,9 +20,7 @@ int GraphNode::enum_size = sizeof(int); int GraphNode::id_size = sizeof(uint64_t); int GraphNode::double_size = sizeof(double); int GraphNode::int_size = sizeof(int); -int GraphNode::get_size() { - return feature.size() + id_size + enum_size + int_size; -} +int GraphNode::get_size() { return feature.size() + id_size + int_size; } void GraphNode::build_sampler() { sampler = new WeightedSampler(); GraphEdge** arr = edges.data(); @@ -33,21 +31,19 @@ void GraphNode::to_buffer(char* buffer) { memcpy(buffer, &size, int_size); memcpy(buffer + int_size, feature.c_str(), feature.size()); memcpy(buffer + int_size + feature.size(), &id, id_size); - int int_state = type; - memcpy(buffer + int_size + feature.size() + id_size, &int_state, enum_size); } void GraphNode::recover_from_buffer(char* buffer) { int size; memcpy(&size, buffer, int_size); - int feature_size = size - id_size - enum_size - int_size; + int feature_size = size - id_size - int_size; char str[feature_size + 1]; memcpy(str, buffer + int_size, feature_size); str[feature_size] = '\0'; feature = str; memcpy(&id, buffer + int_size + feature_size, id_size); - int int_state; - memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); - type = GraphNodeType(int_state); + // int int_state; + // memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); + // type = GraphNodeType(int_state); } } } \ No newline at end of file diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 25045019f4f9f..218d14e01edc1 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -17,28 +17,29 @@ #include "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { -enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; +// enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; class GraphEdge : public WeightedObject { public: double weight; uint64_t id; - GraphNodeType type; + // GraphNodeType type; GraphEdge() {} - GraphEdge(uint64_t id, GraphNodeType type, double weight) - : weight(weight), id(id), type(type) {} + GraphEdge(uint64_t id, double weight) : weight(weight), id(id) {} }; class GraphNode { public: GraphNode() { sampler = NULL; } - GraphNode(uint64_t id, GraphNodeType type, std::string feature) - : id(id), type(type), feature(feature), sampler(NULL) {} + // GraphNode(uint64_t id, GraphNodeType type, std::string feature) + // : id(id), type(type), feature(feature), sampler(NULL) {} + GraphNode(uint64_t id, std::string feature) + : id(id), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} std::vector get_graph_edge() { return edges; } static int enum_size, id_size, int_size, double_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } - GraphNodeType get_graph_node_type() { return type; } - void set_graph_node_type(GraphNodeType type) { this->type = type; } + // GraphNodeType get_graph_node_type() { return type; } + // void set_graph_node_type(GraphNodeType type) { this->type = type; } void set_feature(std::string feature) { this->feature = feature; } std::string get_feature() { return feature; } virtual int get_size(); @@ -46,37 +47,6 @@ class GraphNode { virtual void to_buffer(char *buffer); virtual void recover_from_buffer(char *buffer); virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } - std::string get_type_string() { return node_type_to_string(this->type); } - static GraphNodeType get_graph_node_type(std::string &str) { - GraphNodeType type; - if (str == "user") - type = GraphNodeType::user; - - else if (str == "item") - type = GraphNodeType::item; - else if (str == "query") - type = GraphNodeType::query; - else - type = GraphNodeType::unknown; - return type; - } - static std::string node_type_to_string(GraphNodeType type) { - std::string res; - switch (type) { - case GraphNodeType::user: - res = "user"; - break; - case GraphNodeType::item: - res = "item"; - break; - case GraphNodeType::query: - res = "query"; - break; - default: - res = "unKnown"; - } - return res; - } std::vector sample_k(int k) { std::vector v; if (sampler != NULL) { @@ -90,7 +60,6 @@ class GraphNode { protected: uint64_t id; - GraphNodeType type; std::string feature; WeightedSampler *sampler; std::vector edges; diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index f58fc02b8d27b..83da622f940fb 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -93,8 +93,7 @@ class Table { return 0; } // only for graph table - virtual int32_t random_sample(uint64_t node_id, GraphNodeType type, - int sampe_size, char *&buffer, + virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size) { return 0; } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 9a82137d0383a..2aa3ff5599068 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include // NOLINT +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -48,11 +49,10 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = { - std::string("37\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), - std::string("96\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21"), - std::string("59\tuser\t45;user;0.34\t145;user;0.31\t112;item;0.21"), - std::string("97\tuser\t48;user;1.4\t247;user;0.31\t111;item;1.21")}; +std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), + std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), + std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), + std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -62,18 +62,6 @@ void prepare_file(char file_name[]) { } ofile.close(); } -distributed::GraphNodeType get_graph_node_type(std::string& str) { - distributed::GraphNodeType type; - if (str == "user") - type = distributed::GraphNodeType::user; - else if (str == "item") - type = distributed::GraphNodeType::item; - else if (str == "query") - type = distributed::GraphNodeType::query; - else - type = distributed::GraphNodeType::unknown; - return type; -} void GetDownpourSparseTableProto( ::paddle::distributed::TableParameter* sparse_table_proto) { sparse_table_proto->set_table_id(0); @@ -85,19 +73,6 @@ void GetDownpourSparseTableProto( accessor_proto->set_accessor_class("CommMergeAccessor"); } -distributed::GraphNodeType get_graph_node_type(std::string str) { - distributed::GraphNodeType type; - if (str == "user") { - type = distributed::GraphNodeType::user; - } else if (str == "item") - type = distributed::GraphNodeType::item; - else if (str == "query") - type = distributed::GraphNodeType::query; - else - type = distributed::GraphNodeType::unknown; - return type; -} - ::paddle::distributed::PSParameter GetServerProto() { // Generate server proto desc ::paddle::distributed::PSParameter server_fleet_desc; @@ -236,19 +211,17 @@ void RunBrpcPushSparse() { pull_status.wait(); std::vector v; - pull_status = worker_ptr_->sample( - 0, 37, get_graph_node_type(std::string("user")), 4, v); + pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); // for (auto g : v) { // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; // } ASSERT_EQ(v.size(), 3); v.clear(); - pull_status = worker_ptr_->sample( - 0, 96, get_graph_node_type(std::string("user")), 4, v); + pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); for (auto g : v) { - std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + std::cout << g.get_id() << std::endl; } // ASSERT_EQ(v.size(),3); v.clear(); @@ -266,7 +239,7 @@ void RunBrpcPushSparse() { ASSERT_EQ(v.size(), 1); ASSERT_EQ(v[0].get_id(), 59); for (auto g : v) { - std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; + std::cout << g.get_id() << std::endl; } distributed::GraphPyService gps1, gps2; @@ -277,17 +250,26 @@ void RunBrpcPushSparse() { v.clear(); v = gps2.pull_graph_list(0, 1, 4); ASSERT_EQ(v[0].get_id(), 59); - // for (auto g : v) { - // std::cout << g.get_id() << " service-test " << g.get_graph_node_type() - // << std::endl; - // } v.clear(); - v = gps2.sample_k(96, std::string("user"), 4); + v = gps2.sample_k(96, 4); ASSERT_EQ(v.size(), 3); - // for (auto g : v) { - // std::cout << g.get_id() << " service-test--neighboor " << - // g.get_graph_node_type() << std::endl; - // } + // to test in python,try this: + // from paddle.fluid.core import GraphPyService + // ips_str = "127.0.0.1:4211;127.0.0.1:4212" + // gps1 = GraphPyService(); + // gps2 = GraphPyService(); + // gps1.set_up(ips_str, 127, 0, 0, 0); + // gps2.set_up(ips_str, 127, 1, 1, 0); + // gps1.load_file("input.txt"); + + // list = gps2.pull_graph_list(0,1,4) + // for x in list: + // print(x.get_id()) + + // list = gps2.sample_k(96, "user", 4); + // for x in list: + // print(x.get_id()) + std::remove(file_name); LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); @@ -302,13 +284,11 @@ void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; s.set_feature("hhhh"); s.set_id(65); - s.set_graph_node_type(::paddle::distributed::GraphNodeType(0)); int size = s.get_size(); char str[size]; s.to_buffer(str); s1.recover_from_buffer(str); ASSERT_EQ(s.get_id(), s1.get_id()); - ASSERT_EQ((int)s.get_graph_node_type(), (int)s1.get_graph_node_type()); VLOG(0) << s.get_feature(); VLOG(0) << s1.get_feature(); } diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 34a733d7ccb0a..af3745f510837 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -31,7 +31,6 @@ set(PYBIND_SRCS heter_wrapper_py.cc ps_gpu_wrapper_py.cc gloo_wrapper_py.cc - graph_service.cc box_helper_py.cc data_set_py.cc imperative.cc diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index ba716fb3b550a..68d8c7ca338fa 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -32,6 +32,8 @@ limitations under the License. */ #include "paddle/fluid/distributed/fleet.h" #include "paddle/fluid/distributed/service/communicator.h" #include "paddle/fluid/distributed/service/env.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "paddle/fluid/distributed/service/graph_py_service.h" #include "paddle/fluid/distributed/service/heter_client.h" namespace py = pybind11; @@ -39,6 +41,8 @@ using paddle::distributed::CommContext; using paddle::distributed::Communicator; using paddle::distributed::FleetWrapper; using paddle::distributed::HeterClient; +using paddle::distributed::GraphPyService; +using paddle::distributed::GraphNode; namespace paddle { namespace pybind { @@ -152,5 +156,20 @@ void BindHeterClient(py::module* m) { .def("stop", &HeterClient::Stop); } +void BindGraphNode(py::module* m) { + py::class_(*m, "GraphNode") + .def(py::init<>()) + .def("get_id", &GraphNode::get_id) + .def("get_feature", &GraphNode::get_feature); +} +void BindGraphService(py::module* m) { + py::class_(*m, "GraphPyService") + .def(py::init<>()) + .def("load_file", &GraphPyService::load_file) + .def("set_up", &GraphPyService::set_up) + .def("pull_graph_list", &GraphPyService::pull_graph_list) + .def("sample_k", &GraphPyService::sample_k); +} + } // end namespace pybind } // namespace paddle diff --git a/paddle/fluid/pybind/fleet_py.h b/paddle/fluid/pybind/fleet_py.h index 7f471598ad281..57861a8e2ed6e 100644 --- a/paddle/fluid/pybind/fleet_py.h +++ b/paddle/fluid/pybind/fleet_py.h @@ -27,6 +27,8 @@ void BindPSHost(py::module* m); void BindCommunicatorContext(py::module* m); void BindDistCommunicator(py::module* m); void BindHeterClient(py::module* m); +void BindGraphNode(py::module* m); +void BindGraphService(py::module* m); } // namespace pybind } // namespace paddle diff --git a/paddle/fluid/pybind/graph_service.cc b/paddle/fluid/pybind/graph_service.cc deleted file mode 100644 index 1113f2e7ae94f..0000000000000 --- a/paddle/fluid/pybind/graph_service.cc +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/pybind/graph_service.h" -// #include "paddle/fluid/distributed/service/graph_py_service.h" -// #include "paddle/fluid/distributed/service/graph_brpc_client.h" -#include "brpc/stream.h" -namespace py = pybind11; -// using paddle::distributed::graph_service; - -namespace paddle { -namespace pybind { - -void BindGraphService(py::module* m) { - // py::class_(*m, "graph_service") - // .def( - // py::init<>()) - // .def("set_keys", &paddle::distributed::graph_service::set_keys) - // .def("get_keys",&paddle::distributed::graph_service::get_keys); -} - -} // namespace pybind -} \ No newline at end of file diff --git a/paddle/fluid/pybind/graph_service.h b/paddle/fluid/pybind/graph_service.h deleted file mode 100644 index 560a01642a729..0000000000000 --- a/paddle/fluid/pybind/graph_service.h +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - -#pragma once - -#include "pybind11/pybind11.h" -#include "pybind11/stl.h" - -namespace py = pybind11; - -namespace paddle { -namespace pybind { - -void BindGraphService(py::module* m); - -} // namespace pybind -} // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index baae2931dbd15..34a44e74c1b34 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -114,7 +114,6 @@ limitations under the License. */ #endif #include "pybind11/stl.h" -//#include "paddle/fluid/pybind/graph_ps_service.h" DECLARE_bool(use_mkldnn); @@ -2845,9 +2844,10 @@ All parameter, weight, gradient are variables in Paddle. BindDistFleetWrapper(&m); BindPSHost(&m); BindCommunicatorContext(&m); - // BindGraphService(&m); BindDistCommunicator(&m); BindHeterClient(&m); + BindGraphNode(&m); + BindGraphService(&m); #endif } } // namespace pybind From 44df9ace56d98dcfa55dc305433dade984547c0f Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 11 Mar 2021 16:01:05 +0800 Subject: [PATCH 007/106] add load direct graph --- .../distributed/table/common_graph_table.cc | 62 ++++++++++--------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 5c97ba631d57d..dfd492c011160 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -15,12 +15,15 @@ #include "paddle/fluid/distributed/table/common_graph_table.h" #include #include +#include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { + int GraphShard::bucket_low_bound = 11; + std::vector GraphShard::get_batch(int start, int total_size) { if (start < 0) start = 0; int size = 0, cur_size; @@ -51,6 +54,7 @@ std::vector GraphShard::get_batch(int start, int total_size) { } return res; } + size_t GraphShard::get_size() { size_t res = 0; for (int i = 0; i < bucket_size; i++) { @@ -58,61 +62,63 @@ size_t GraphShard::get_size() { } return res; } + std::list::iterator GraphShard::add_node(GraphNode *node) { if (node_location.find(node->get_id()) != node_location.end()) return node_location.find(node->get_id())->second; + int index = node->get_id() % shard_num % bucket_size; + std::list::iterator iter = bucket[index].insert(bucket[index].end(), node); + node_location[node->get_id()] = iter; return iter; } + void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { (*add_node(new GraphNode(id, std::string(""))))->add_edge(edge); } + GraphNode *GraphShard::find_node(uint64_t id) { if (node_location.find(id) == node_location.end()) return NULL; return *(node_location[id]); } + int32_t GraphTable::load(const std::string &path, const std::string ¶m) { auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); + int count = 0; for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); + count ++; if (values.size() < 2) continue; - auto id = std::stoull(values[0]); - size_t shard_id = id % shard_num; - if (shard_id >= shard_end || shard_id < shard_start) { - VLOG(0) << "will not load " << id << " from " << path + auto src_id = std::stoull(values[0]); + auto dst_id = std::stoull(values[1]); + double weight = 0; + if (values.size() == 3) { + weight = std::stod(values[2]); + } + size_t src_shard_id = src_id % shard_num; + + if (src_shard_id >= shard_end || src_shard_id < shard_start) { + VLOG(0) << "will not load " << src_id << " from " << path << ", please check id distribution"; continue; + } - size_t index = shard_id - shard_start; - // GraphNodeType type = GraphNode::get_graph_node_type(values[1]); - // VLOG(0)<<"shards's size = "<get_id(); - for (size_t i = 2; i < values.size(); i++) { - auto edge_arr = - paddle::string::split_string(values[i], ";"); - if (edge_arr.size() == 2) { - // VLOG(0)<<"edge content "<::iterator iter = bucket[i].begin(); @@ -122,7 +128,6 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { iter++; } } - } } return 0; } @@ -144,6 +149,7 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, char *&buffer, int &actual_size) { return _shards_task_pool[get_thread_pool_index(node_id)] ->enqueue([&]() -> int { + GraphNode *node = find_node(node_id); if (node == NULL) { actual_size = 0; @@ -275,4 +281,4 @@ int32_t GraphTable::initialize() { return 0; } } -}; \ No newline at end of file +}; From c8e67b8f300b578151e06d737f65cd488207df92 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 11 Mar 2021 18:45:57 +0800 Subject: [PATCH 008/106] add load direct graph --- .../distributed/service/graph_py_service.cc | 13 +++- .../distributed/service/graph_py_service.h | 78 +++++++++++++------ .../distributed/table/common_graph_table.cc | 10 +++ .../fluid/distributed/test/graph_node_test.cc | 11 +-- paddle/fluid/pybind/fleet_py.cc | 3 +- 5 files changed, 82 insertions(+), 33 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 86d2d54193bd9..04c04930e43e8 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -27,11 +27,16 @@ std::vector GraphPyService::split(std::string &str, } void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, - int client_id, uint32_t table_id) { + int client_id, std::vector edge_types) { set_shard_num(shard_num); set_client_Id(client_id); set_rank(rank); - this->table_id = table_id; + + this -> table_id_map[std::string("")] = 0; + // Table 0 are for nodes + for(size_t table_id = 0; table_id < edge_types.size(); table_id ++ ) { + this -> table_id_map[edge_types[table_id]] = int(table_id + 1); + } server_thread = client_thread = NULL; std::istringstream stream(ips_str); std::string ip; @@ -47,10 +52,10 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, host_sign_list.push_back(ph_host.serialize_to_string()); index++; } - VLOG(0) << "IN set up rank = " << rank; + //VLOG(0) << "IN set up rank = " << rank; start_client(); start_server(server_list[rank], std::stoul(port_list[rank])); sleep(1); } } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index be946dc44e57d..25b70d11fe05b 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -22,6 +22,7 @@ #include #include // NOLINT #include +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -46,7 +47,7 @@ class GraphPyService { std::vector keys; std::vector server_list, port_list, host_sign_list; int server_size, shard_num, rank, client_id; - uint32_t table_id; + std::unordered_map table_id_map; std::thread *server_thread, *client_thread; std::shared_ptr pserver_ptr; @@ -67,7 +68,7 @@ class GraphPyService { int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( - ::paddle::distributed::TableParameter* sparse_table_proto) { + ::paddle::distributed::TableParameter* sparse_table_proto, uint32_t table_id) { sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); sparse_table_proto->set_shard_num(shard_num); @@ -96,10 +97,14 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto); + for(auto& tuple : this -> table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } + return server_fleet_desc; + } ::paddle::distributed::PSParameter GetWorkerProto() { @@ -111,9 +116,11 @@ class GraphPyService { ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = worker_proto->mutable_downpour_worker_param(); - ::paddle::distributed::TableParameter* worker_sparse_table_proto = - downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto); + for(auto& tuple : this -> table_id_map) { + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + } ::paddle::distributed::ServerParameter* server_proto = worker_fleet_desc.mutable_server_param(); @@ -127,9 +134,11 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - ::paddle::distributed::TableParameter* server_sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(server_sparse_table_proto); + for(auto& tuple : this -> table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } return worker_fleet_desc; } @@ -137,24 +146,47 @@ class GraphPyService { int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); - void load_file(std::string filepath) { - auto status = - get_ps_client()->load(table_id, std::string(filepath), std::string("")); - status.wait(); + void load_edge_file(std::string name, std::string filepath, bool reverse) { + std::string params = "edge"; + if(reverse) { + params += "|reverse"; + } + if (this -> table_id_map.count(name)) { + uint32_t table_id = this -> table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } + } + + void load_node_file(std::string name, std::string filepath) { + std::string params = "node"; + if (this -> table_id_map.count(name)) { + uint32_t table_id = this -> table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } } - std::vector sample_k(uint64_t node_id, int sample_size) { + std::vector sample_k(std::string name, uint64_t node_id, int sample_size) { std::vector v; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); + if (this -> table_id_map.count(name)) { + uint32_t table_id = this -> table_id_map[name]; + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); + status.wait(); + } return v; } - std::vector pull_graph_list(int server_index, int start, + std::vector pull_graph_list(std::string name, int server_index, int start, int size) { std::vector res; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); - status.wait(); + if (this -> table_id_map.count(name)) { + uint32_t table_id = this -> table_id_map[name]; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); + } return res; } void start_server(std::string ip, uint32_t port) { @@ -197,7 +229,7 @@ class GraphPyService { worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); } void set_up(std::string ips_str, int shard_num, int rank, int client_id, - uint32_t table_id); + std::vector edge_types); void set_keys(std::vector keys) { // just for test this->keys = keys; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index dfd492c011160..3f77720883e4d 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -16,6 +16,7 @@ #include #include #include +#include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" @@ -86,9 +87,15 @@ GraphNode *GraphShard::find_node(uint64_t id) { } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { + auto cmd = paddle::string::split_string(path, "|"); + std::set cmd_set(cmd.begin(), cmd.end()); + bool load_edge = cmd_set.count(std::string("edge")); + bool reverse_edge = cmd_set.count(std::string("reverse")); + auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); int count = 0; + for (auto path : paths) { std::ifstream file(path); std::string line; @@ -98,6 +105,9 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { if (values.size() < 2) continue; auto src_id = std::stoull(values[0]); auto dst_id = std::stoull(values[1]); + if(reverse_edge) { + std::swap(src_id, dst_id); + } double weight = 0; if (values.size() == 3) { weight = std::stod(values[2]); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 2aa3ff5599068..a2a87025aaed8 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -244,14 +244,15 @@ void RunBrpcPushSparse() { distributed::GraphPyService gps1, gps2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; - gps1.set_up(ips_str, 127, 0, 0, 0); - gps2.set_up(ips_str, 127, 1, 1, 0); - gps1.load_file(std::string(file_name)); + std::vector edge_types = { std::string("user2item")}; + gps1.set_up(ips_str, 127, 0, 0, edge_types); + gps2.set_up(ips_str, 127, 1, 1, edge_types); + gps1.load_edge_file(std::string("user2item"), std::string(file_name), 0); v.clear(); - v = gps2.pull_graph_list(0, 1, 4); + v = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); ASSERT_EQ(v[0].get_id(), 59); v.clear(); - v = gps2.sample_k(96, 4); + v = gps2.sample_k(std::string("user2item"), 96, 4); ASSERT_EQ(v.size(), 3); // to test in python,try this: // from paddle.fluid.core import GraphPyService diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 68d8c7ca338fa..e967acee77d20 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -165,7 +165,8 @@ void BindGraphNode(py::module* m) { void BindGraphService(py::module* m) { py::class_(*m, "GraphPyService") .def(py::init<>()) - .def("load_file", &GraphPyService::load_file) + .def("load_edge_file", &GraphPyService::load_edge_file) + .def("load_node_file", &GraphPyService::load_node_file) .def("set_up", &GraphPyService::set_up) .def("pull_graph_list", &GraphPyService::pull_graph_list) .def("sample_k", &GraphPyService::sample_k); From 732099437f069af02cd06acc378726ff40e8c929 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 11 Mar 2021 20:18:36 +0800 Subject: [PATCH 009/106] add load direct graph --- paddle/fluid/distributed/table/common_graph_table.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 3f77720883e4d..f4f235b114dba 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -87,10 +87,11 @@ GraphNode *GraphShard::find_node(uint64_t id) { } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { - auto cmd = paddle::string::split_string(path, "|"); + auto cmd = paddle::string::split_string(param, "|"); std::set cmd_set(cmd.begin(), cmd.end()); bool load_edge = cmd_set.count(std::string("edge")); bool reverse_edge = cmd_set.count(std::string("reverse")); + VLOG(0) << "Reverse Edge " << reverse_edge; auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); From f4ea97324c1a6fc426d32de3129f9aad1636e10d Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Thu, 11 Mar 2021 22:32:27 +0800 Subject: [PATCH 010/106] batch random_sample --- .../distributed/service/graph_brpc_client.cc | 35 ++++++++++++++----- .../distributed/service/graph_brpc_server.cc | 25 ++++++++++--- .../distributed/table/common_graph_table.cc | 8 ++--- .../distributed/table/common_graph_table.h | 2 +- paddle/fluid/distributed/table/table.h | 4 +-- 5 files changed, 53 insertions(+), 21 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 16014c8dbf23f..dc51da6cd15a0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -52,13 +52,27 @@ std::future GraphBrpcClient::sample(uint32_t table_id, size_t bytes_size = io_buffer_itr.bytes_left(); char *buffer = new char[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - int start = 0; - while (start < bytes_size) { - GraphNode node; - node.recover_from_buffer(buffer + start); - start += node.get_size(); - res.push_back(node); + + size_t num_nodes = *(size_t *)buffer; + int *actual_sizes = (int *)(buffer + sizeof(size_t)); + char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * num_nodes; + + std::vector > ress; + std::vector res_; + int offset = 0; + for (size_t idx = 0; idx < num_nodes; ++idx){ + int actual_size = actual_sizes[idx]; + int start = 0; + while (start < actual_size) { + GraphNode node; + node.recover_from_buffer(node_buffer + offset + start); + start += node.get_size(); + res_.push_back(node); + } + offset += actual_size; + ress.push_back(res_); } + res = ress[0]; } closure->set_promise_value(ret); }); @@ -70,8 +84,11 @@ std::future GraphBrpcClient::sample(uint32_t table_id, closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); // std::string type_str = GraphNode::node_type_to_string(type); - closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - // closure->request(0)->add_params(type_str.c_str(), type_str.size()); + std::vector node_ids; + node_ids.push_back(node_id); + size_t num_nodes = node_ids.size(); + + closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*num_nodes); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); @@ -129,4 +146,4 @@ int32_t GraphBrpcClient::initialize() { return 0; } } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index ecb078c0ad400..14748bce78664 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -283,12 +283,27 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, "graph_random_sample request requires at least 2 arguments"); return 0; } - uint64_t node_id = *(uint64_t *)(request.params(0).c_str()); + size_t num_nodes = request.params(0).size(); + uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); - char *buffer; - int actual_size; - table->random_sample(node_id, sample_size, buffer, actual_size); - cntl->response_attachment().append(buffer, actual_size); + + std::vector*> tasks; + std::vector buffers(num_nodes); + std::vector actual_sizes(num_nodes); + + for (size_t idx = 0; idx < num_nodes; ++idx){ + std::future task = table->random_sample(node_data[idx], sample_size, + buffers[idx], actual_sizes[idx]); + tasks.push_back(&task); + } + for (size_t idx = 0; idx < num_nodes; ++idx){ + tasks[idx]->get(); + } + cntl->response_attachment().append(&num_nodes, sizeof(size_t)); + cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*num_nodes); + for (size_t idx = 0; idx < num_nodes; ++idx){ + cntl->response_attachment().append(buffers[idx], actual_sizes[idx]); + } return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 5c97ba631d57d..4d126c3818af3 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -140,7 +140,7 @@ GraphNode *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num_per_table % task_pool_size_; } -int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, +std::future GraphTable::random_sample(uint64_t node_id, int sample_size, char *&buffer, int &actual_size) { return _shards_task_pool[get_thread_pool_index(node_id)] ->enqueue([&]() -> int { @@ -166,8 +166,8 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, } actual_size = total_size; return 0; - }) - .get(); + }); + //.get(); // GraphNode *node = find_node(node_id, type); // if (node == NULL) { // actual_size = 0; @@ -275,4 +275,4 @@ int32_t GraphTable::initialize() { return 0; } } -}; \ No newline at end of file +}; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 1f2b8c86d363b..8a806f8278877 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -71,7 +71,7 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); - virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 83da622f940fb..b389eb3acae98 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -93,9 +93,9 @@ class Table { return 0; } // only for graph table - virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size) { - return 0; + return std::future(); } virtual int32_t pour() { return 0; } From 492e5511d13a82317a9932d24a99cd7e1d47a482 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Thu, 11 Mar 2021 23:20:06 +0800 Subject: [PATCH 011/106] batch_sample_k --- .../distributed/service/graph_brpc_client.cc | 60 +++++++++++++++++++ .../distributed/service/graph_brpc_client.h | 3 + .../distributed/service/graph_py_service.h | 6 ++ paddle/fluid/distributed/service/ps_client.h | 9 +++ paddle/fluid/pybind/fleet_py.cc | 3 +- 5 files changed, 80 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index dc51da6cd15a0..da2e936819040 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -98,6 +98,66 @@ std::future GraphBrpcClient::sample(uint32_t table_id, return fut; } +std::future GraphBrpcClient::batch_sample(uint32_t table_id, + std::vector node_ids, int sample_size, + std::vector > &res) { + uint64_t node_id = node_ids[0]; + int server_index = get_server_index_by_id(node_id); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(0, PS_GRAPH_SAMPLE); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + + size_t num_nodes = *(size_t *)buffer; + int *actual_sizes = (int *)(buffer + sizeof(size_t)); + char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * num_nodes; + + std::vector res_; + int offset = 0; + for (size_t idx = 0; idx < num_nodes; ++idx){ + int actual_size = actual_sizes[idx]; + int start = 0; + while (start < actual_size) { + GraphNode node; + node.recover_from_buffer(node_buffer + offset + start); + start += node.get_size(); + res_.push_back(node); + } + offset += actual_size; + res.push_back(res_); + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + // std::string type_str = GraphNode::node_type_to_string(type); + size_t num_nodes = node_ids.size(); + + closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*num_nodes); + closure->request(0)->add_params((char *)&sample_size, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); + + return fut; +} + std::future GraphBrpcClient::pull_graph_list( uint32_t table_id, int server_index, int start, int size, std::vector &res) { diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 8e472b96be94d..d0653120e07ce 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -38,6 +38,9 @@ class GraphBrpcClient : public BrpcPsClient { virtual std::future sample(uint32_t table_id, uint64_t node_id, int sample_size, std::vector &res); + virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, + int sample_size, + std::vector > &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index be946dc44e57d..0975460003436 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -149,6 +149,12 @@ class GraphPyService { status.wait(); return v; } + std::vector > batch_sample_k(std::vector node_ids, int sample_size) { + std::vector > v; + auto status = worker_ptr->batch_sample(table_id, node_ids, sample_size, v); + status.wait(); + return v; + } std::vector pull_graph_list(int server_index, int start, int size) { std::vector res; diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index b6014b9aea139..2122743197052 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -164,6 +164,15 @@ class PSClient { promise.set_value(-1); return fut; } + virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, + int sample_size, + std::vector > &res) { + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 68d8c7ca338fa..6b86da9664897 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -168,7 +168,8 @@ void BindGraphService(py::module* m) { .def("load_file", &GraphPyService::load_file) .def("set_up", &GraphPyService::set_up) .def("pull_graph_list", &GraphPyService::pull_graph_list) - .def("sample_k", &GraphPyService::sample_k); + .def("sample_k", &GraphPyService::sample_k) + .def("batch_sample_k", &GraphPyService::batch_sample_k); } } // end namespace pybind From 193203f421ec192a4abf83e9442429cb5bab96c9 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 12 Mar 2021 00:40:47 +0800 Subject: [PATCH 012/106] fix num_nodes size --- .../fluid/distributed/service/graph_brpc_client.cc | 2 ++ .../fluid/distributed/service/graph_brpc_server.cc | 14 ++++++++------ .../fluid/distributed/table/common_graph_table.cc | 6 ++++-- .../fluid/distributed/table/common_graph_table.h | 4 +++- paddle/fluid/distributed/table/table.h | 8 ++++++-- 5 files changed, 23 insertions(+), 11 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index da2e936819040..2156065caba38 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -123,8 +123,10 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector res_; int offset = 0; + std::cout << "num_nodes: " << num_nodes << std::endl; for (size_t idx = 0; idx < num_nodes; ++idx){ int actual_size = actual_sizes[idx]; + std::cout << "actual_size: " << actual_size << std::endl; int start = 0; while (start < actual_size) { GraphNode node; diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 14748bce78664..bf926d7fbe52c 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -283,7 +283,7 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, "graph_random_sample request requires at least 2 arguments"); return 0; } - size_t num_nodes = request.params(0).size(); + size_t num_nodes = request.params(0).size() / sizeof(uint64_t); uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); @@ -292,13 +292,15 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, std::vector actual_sizes(num_nodes); for (size_t idx = 0; idx < num_nodes; ++idx){ - std::future task = table->random_sample(node_data[idx], sample_size, + //std::future task = table->random_sample(node_data[idx], sample_size, + //buffers[idx], actual_sizes[idx]); + table->random_sample(node_data[idx], sample_size, buffers[idx], actual_sizes[idx]); - tasks.push_back(&task); - } - for (size_t idx = 0; idx < num_nodes; ++idx){ - tasks[idx]->get(); + //tasks.push_back(&task); } + //for (size_t idx = 0; idx < num_nodes; ++idx){ + //tasks[idx]->get(); + //} cntl->response_attachment().append(&num_nodes, sizeof(size_t)); cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*num_nodes); for (size_t idx = 0; idx < num_nodes; ++idx){ diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 4d126c3818af3..9b9e9a88b84a7 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -140,7 +140,9 @@ GraphNode *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num_per_table % task_pool_size_; } -std::future GraphTable::random_sample(uint64_t node_id, int sample_size, +//std::future GraphTable::random_sample(uint64_t node_id, int sample_size, + //char *&buffer, int &actual_size) { +int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, char *&buffer, int &actual_size) { return _shards_task_pool[get_thread_pool_index(node_id)] ->enqueue([&]() -> int { @@ -166,7 +168,7 @@ std::future GraphTable::random_sample(uint64_t node_id, int sample_size, } actual_size = total_size; return 0; - }); + }).get(); //.get(); // GraphNode *node = find_node(node_id, type); // if (node == NULL) { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 8a806f8278877..18fdfdb5d8f66 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -71,7 +71,9 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); - virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, + //virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, + //int &actual_size); + int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index b389eb3acae98..d9b6835a31f29 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -93,10 +93,14 @@ class Table { return 0; } // only for graph table - virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size) { - return std::future(); + return 0; } + //virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, + //int &actual_size) { + //return std::future(); + //} virtual int32_t pour() { return 0; } virtual void clear() = 0; From b55bf426b73f8dc7fa8c69e19d32d919518ef14c Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 12 Mar 2021 01:59:43 +0800 Subject: [PATCH 013/106] batch brpc --- .../distributed/service/graph_brpc_client.cc | 119 +++++++++++------- 1 file changed, 71 insertions(+), 48 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 2156065caba38..1db5a30274dca 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -53,14 +53,14 @@ std::future GraphBrpcClient::sample(uint32_t table_id, char *buffer = new char[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - size_t num_nodes = *(size_t *)buffer; + size_t node_num = *(size_t *)buffer; int *actual_sizes = (int *)(buffer + sizeof(size_t)); - char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * num_nodes; + char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; std::vector > ress; std::vector res_; int offset = 0; - for (size_t idx = 0; idx < num_nodes; ++idx){ + for (size_t idx = 0; idx < node_num; ++idx){ int actual_size = actual_sizes[idx]; int start = 0; while (start < actual_size) { @@ -86,9 +86,9 @@ std::future GraphBrpcClient::sample(uint32_t table_id, // std::string type_str = GraphNode::node_type_to_string(type); std::vector node_ids; node_ids.push_back(node_id); - size_t num_nodes = node_ids.size(); + size_t node_num = node_ids.size(); - closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*num_nodes); + closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*node_num); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); @@ -101,61 +101,84 @@ std::future GraphBrpcClient::sample(uint32_t table_id, std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector > &res) { - uint64_t node_id = node_ids[0]; - int server_index = get_server_index_by_id(node_id); + + std::vector > node_id_buckets; + std::vector > query_idx_buckets; + std::vector request2server; + std::vector server2request(server_size, -1); + + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ + int server_index = get_server_index_by_id(node_ids[query_idx]); + if(server2request[server_index] == -1){ + server2request[server_index] = request2server.size(); + request2server.push_back(server_index); + node_id_buckets.push_back(std::vector ()); + query_idx_buckets.push_back(std::vector ()); + } + int request_idx = server2request[server_index]; + node_id_buckets[request_idx].push_back(node_ids[query_idx]); + query_idx_buckets[request_idx].push_back(query_idx); + res.push_back(std::vector()); + } + size_t request_call_num = request2server.size(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { - ret = -1; - } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_GRAPH_SAMPLE); - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ + if (closure->check_response(request_idx, PS_GRAPH_SAMPLE) != 0) { + ret = -1; + } else { + VLOG(0) << "check sample response: " + << " " << closure->check_response(request_idx, PS_GRAPH_SAMPLE); + auto &res_io_buffer = closure->cntl(request_idx)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char *buffer = new char[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - size_t num_nodes = *(size_t *)buffer; - int *actual_sizes = (int *)(buffer + sizeof(size_t)); - char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * num_nodes; - - std::vector res_; - int offset = 0; - std::cout << "num_nodes: " << num_nodes << std::endl; - for (size_t idx = 0; idx < num_nodes; ++idx){ - int actual_size = actual_sizes[idx]; - std::cout << "actual_size: " << actual_size << std::endl; - int start = 0; - while (start < actual_size) { - GraphNode node; - node.recover_from_buffer(node_buffer + offset + start); - start += node.get_size(); - res_.push_back(node); + size_t node_num = *(size_t *)buffer; + int *actual_sizes = (int *)(buffer + sizeof(size_t)); + char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; + + //std::vector res_; + int offset = 0; + for (size_t node_idx = 0; node_idx < node_num; ++node_idx){ + int query_idx = query_idx_buckets[request_idx][node_idx]; + int actual_size = actual_sizes[node_idx]; + int start = 0; + while (start < actual_size) { + GraphNode node; + node.recover_from_buffer(node_buffer + offset + start); + start += node.get_size(); + res[query_idx].push_back(node); + } + offset += actual_size; } - offset += actual_size; - res.push_back(res_); } } closure->set_promise_value(ret); }); + auto promise = std::make_shared>(); closure->add_promise(promise); std::future fut = promise->get_future(); - ; - closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); - size_t num_nodes = node_ids.size(); - - closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*num_nodes); - closure->request(0)->add_params((char *)&sample_size, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), - closure); + + for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ + int server_index = request2server[request_idx]; + closure->request(request_idx)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(request_idx)->set_table_id(table_id); + closure->request(request_idx)->set_client_id(_client_id); + // std::string type_str = GraphNode::node_type_to_string(type); + size_t node_num = node_id_buckets[request_idx].size(); + + closure->request(request_idx)->add_params((char *)node_id_buckets[request_idx].data(), sizeof(uint64_t)*node_num); + closure->request(request_idx)->add_params((char *)&sample_size, sizeof(int)); + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), closure->response(request_idx), + closure); + } return fut; } From 102ae95bdb324f3f573dc5dd4f5f9836a3db6268 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 12 Mar 2021 03:24:41 +0800 Subject: [PATCH 014/106] batch brpc --- .../distributed/service/graph_brpc_client.cc | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 1db5a30274dca..bb63487496f48 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -102,27 +102,31 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector > &res) { - std::vector > node_id_buckets; - std::vector > query_idx_buckets; std::vector request2server; std::vector server2request(server_size, -1); - for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ int server_index = get_server_index_by_id(node_ids[query_idx]); if(server2request[server_index] == -1){ server2request[server_index] = request2server.size(); request2server.push_back(server_index); - node_id_buckets.push_back(std::vector ()); - query_idx_buckets.push_back(std::vector ()); } + res.push_back(std::vector()); + } + size_t request_call_num = request2server.size(); + std::vector > node_id_buckets(request_call_num); + std::vector > query_idx_buckets(request_call_num); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ + int server_index = get_server_index_by_id(node_ids[query_idx]); int request_idx = server2request[server_index]; node_id_buckets[request_idx].push_back(node_ids[query_idx]); query_idx_buckets[request_idx].push_back(query_idx); - res.push_back(std::vector()); } - size_t request_call_num = request2server.size(); + + for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ + + } - DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + DownpourBrpcClosure *closure = new DownpourBrpcClosure(request_call_num, [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ @@ -130,7 +134,7 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, ret = -1; } else { VLOG(0) << "check sample response: " - << " " << closure->check_response(request_idx, PS_GRAPH_SAMPLE); + << " " << closure->check_response(request_idx, PS_GRAPH_SAMPLE); auto &res_io_buffer = closure->cntl(request_idx)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); @@ -141,10 +145,9 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, int *actual_sizes = (int *)(buffer + sizeof(size_t)); char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; - //std::vector res_; int offset = 0; for (size_t node_idx = 0; node_idx < node_num; ++node_idx){ - int query_idx = query_idx_buckets[request_idx][node_idx]; + int query_idx = query_idx_buckets.at(request_idx).at(node_idx); int actual_size = actual_sizes[node_idx]; int start = 0; while (start < actual_size) { From 3106b2a8bc0a1d63bb307ceda7a1adfe44c4a008 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 12 Mar 2021 03:34:33 +0800 Subject: [PATCH 015/106] add test --- paddle/fluid/distributed/service/graph_brpc_client.cc | 7 +------ paddle/fluid/distributed/test/graph_node_test.cc | 3 +++ 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index bb63487496f48..a5134c8614d47 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -57,8 +57,6 @@ std::future GraphBrpcClient::sample(uint32_t table_id, int *actual_sizes = (int *)(buffer + sizeof(size_t)); char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; - std::vector > ress; - std::vector res_; int offset = 0; for (size_t idx = 0; idx < node_num; ++idx){ int actual_size = actual_sizes[idx]; @@ -67,19 +65,16 @@ std::future GraphBrpcClient::sample(uint32_t table_id, GraphNode node; node.recover_from_buffer(node_buffer + offset + start); start += node.get_size(); - res_.push_back(node); + res.push_back(node); } offset += actual_size; - ress.push_back(res_); } - res = ress[0]; } closure->set_promise_value(ret); }); auto promise = std::make_shared>(); closure->add_promise(promise); std::future fut = promise->get_future(); - ; closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 2aa3ff5599068..5c3e62ae9d427 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -211,6 +211,7 @@ void RunBrpcPushSparse() { pull_status.wait(); std::vector v; + std::vector > vs; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); // for (auto g : v) { @@ -253,6 +254,8 @@ void RunBrpcPushSparse() { v.clear(); v = gps2.sample_k(96, 4); ASSERT_EQ(v.size(), 3); + vs = gps2.batch_sample_k([96, 37], 4); + ASSERT_EQ(vs.size(), 2); // to test in python,try this: // from paddle.fluid.core import GraphPyService // ips_str = "127.0.0.1:4211;127.0.0.1:4212" From 51ed7d8791e5d41fe1eab55cf465f782a20a1cb1 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 12 Mar 2021 03:39:58 +0800 Subject: [PATCH 016/106] add test --- paddle/fluid/distributed/test/graph_node_test.cc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 5c3e62ae9d427..e522af9910ec5 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -254,7 +254,10 @@ void RunBrpcPushSparse() { v.clear(); v = gps2.sample_k(96, 4); ASSERT_EQ(v.size(), 3); - vs = gps2.batch_sample_k([96, 37], 4); + std::vector node_ids; + node_ids.push_back(96); + node_ids.push_back(37); + vs = gps2.batch_sample_k(node_ids, 4); ASSERT_EQ(vs.size(), 2); // to test in python,try this: // from paddle.fluid.core import GraphPyService From e087485e45594d1b0c25456c902a0b0e04122d73 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Fri, 12 Mar 2021 15:54:45 +0800 Subject: [PATCH 017/106] add load_nodes; change add_node function --- .../distributed/table/common_graph_table.cc | 83 ++++++++++++++----- .../distributed/table/common_graph_table.h | 8 +- 2 files changed, 71 insertions(+), 20 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index f4f235b114dba..c96eb4680d5aa 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -64,21 +64,22 @@ size_t GraphShard::get_size() { return res; } -std::list::iterator GraphShard::add_node(GraphNode *node) { - if (node_location.find(node->get_id()) != node_location.end()) - return node_location.find(node->get_id())->second; +std::list::iterator GraphShard::add_node(uint64_t id, std::string feature) { + if (node_location.find(id) != node_location.end()) + return node_location.find(id)->second; - int index = node->get_id() % shard_num % bucket_size; + int index = id % shard_num % bucket_size; + GraphNode *node = new GraphNode(id, std::string("")); std::list::iterator iter = bucket[index].insert(bucket[index].end(), node); - node_location[node->get_id()] = iter; + node_location[id] = iter; return iter; } void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { - (*add_node(new GraphNode(id, std::string(""))))->add_edge(edge); + (*add_node(id, std::string("")))->add_edge(edge); } GraphNode *GraphShard::find_node(uint64_t id) { @@ -89,14 +90,54 @@ GraphNode *GraphShard::find_node(uint64_t id) { int32_t GraphTable::load(const std::string &path, const std::string ¶m) { auto cmd = paddle::string::split_string(param, "|"); std::set cmd_set(cmd.begin(), cmd.end()); - bool load_edge = cmd_set.count(std::string("edge")); bool reverse_edge = cmd_set.count(std::string("reverse")); - VLOG(0) << "Reverse Edge " << reverse_edge; - + bool load_edge = cmd_set.count(std::string("edge")); + if(load_edge) { + return this -> load_edges(path, reverse_edge); + } + else { + return this -> load_nodes(path); + } +} + +int32_t GraphTable::load_nodes(const std::string &path) { + auto paths = paddle::string::split_string(path, ";"); + for (auto path : paths) { + std::ifstream file(path); + std::string line; + while (std::getline(file, line)) { + auto values = paddle::string::split_string(line, "\t"); + if (values.size() < 2) continue; + auto id = std::stoull(values[1]); + + + size_t shard_id = id % shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + VLOG(0) << "will not load " << id << " from " << path + << ", please check id distribution"; + continue; + + } + + std::string node_type = values[0]; + std::vector feature; + feature.push_back(node_type); + for(size_t slice = 2; slice < values.size(); slice ++) { + feature.push_back(values[slice]); + } + auto feat = paddle::string::join_strings(feature, '\t'); + size_t index = shard_id - shard_start; + shards[index].add_node(id, std::string("")); + + } + } + return 0; +} + + +int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { auto paths = paddle::string::split_string(path, ";"); - VLOG(0) << paths.size(); int count = 0; - for (auto path : paths) { std::ifstream file(path); std::string line; @@ -113,6 +154,7 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { if (values.size() == 3) { weight = std::stod(values[2]); } + size_t src_shard_id = src_id % shard_num; if (src_shard_id >= shard_end || src_shard_id < shard_start) { @@ -121,6 +163,7 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { continue; } + size_t index = src_shard_id - shard_start; GraphEdge *edge = new GraphEdge(dst_id, weight); shards[index].add_neighboor(src_id, edge); @@ -129,19 +172,21 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { VLOG(0) << "Load Finished Total Edge Count " << count; // Build Sampler j + for (auto &shard : shards) { - auto bucket = shard.get_bucket(); - for (int i = 0; i < bucket.size(); i++) { - std::list::iterator iter = bucket[i].begin(); - while (iter != bucket[i].end()) { - auto node = *iter; - node->build_sampler(); - iter++; - } + auto bucket = shard.get_bucket(); + for (int i = 0; i < bucket.size(); i ++) { + std::list::iterator iter = bucket[i].begin(); + while (iter != bucket[i].end()) { + auto node = *iter; + node->build_sampler(); + iter++; } + } } return 0; } + GraphNode *GraphTable::find_node(uint64_t id) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 1f2b8c86d363b..decf5f1f20462 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -52,7 +52,7 @@ class GraphShard { } return -1; } - std::list::iterator add_node(GraphNode *node); + std::list::iterator add_node(uint64_t id, std::string feature); GraphNode *find_node(uint64_t id); void add_neighboor(uint64_t id, GraphEdge *edge); std::unordered_map::iterator> @@ -74,7 +74,13 @@ class GraphTable : public SparseTable { virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); + int32_t load(const std::string &path, const std::string ¶m); + + int32_t load_edges(const std::string &path, bool reverse); + + int32_t load_nodes(const std::string &path); + GraphNode *find_node(uint64_t id); virtual int32_t pull_sparse(float *values, const uint64_t *keys, size_t num) { From c588c9606d462a8105eff090648d6cb9c37d4dc9 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Fri, 12 Mar 2021 11:26:39 +0000 Subject: [PATCH 018/106] change sample return type to pair --- .../distributed/service/graph_brpc_client.cc | 25 ++--- .../distributed/service/graph_brpc_client.h | 7 +- .../distributed/service/graph_py_service.h | 84 ++++++++--------- paddle/fluid/distributed/service/ps_client.h | 6 +- .../distributed/table/common_graph_table.cc | 91 +++++++------------ paddle/fluid/distributed/table/graph_node.cc | 3 +- paddle/fluid/distributed/table/graph_node.h | 10 +- .../distributed/table/weighted_sampler.cc | 15 +-- .../distributed/table/weighted_sampler.h | 19 ++-- .../fluid/distributed/test/graph_node_test.cc | 55 +++++++---- 10 files changed, 147 insertions(+), 168 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 16014c8dbf23f..bc6a03b9eaf84 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -12,15 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include #include #include #include +#include #include #include "Eigen/Dense" - #include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/string/string_helper.h" @@ -35,9 +35,9 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::sample(uint32_t table_id, - uint64_t node_id, int sample_size, - std::vector &res) { +std::future GraphBrpcClient::sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { int server_index = get_server_index_by_id(node_id); DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; @@ -45,19 +45,16 @@ std::future GraphBrpcClient::sample(uint32_t table_id, if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { ret = -1; } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_GRAPH_SAMPLE); auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); char *buffer = new char[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - int start = 0; - while (start < bytes_size) { - GraphNode node; - node.recover_from_buffer(buffer + start); - start += node.get_size(); - res.push_back(node); + int offset = 0; + while (offset < bytes_size) { + res.push_back({*(uint64_t *)(buffer + offset), + *(float *)(buffer + offset + GraphNode::id_size)}); + offset += GraphNode::id_size + GraphNode::weight_size; } } closure->set_promise_value(ret); @@ -69,9 +66,7 @@ std::future GraphBrpcClient::sample(uint32_t table_id, closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - // closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 8e472b96be94d..84d4dbb78cba4 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -18,6 +18,7 @@ #include #include +#include #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" @@ -35,9 +36,9 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res); + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 25b70d11fe05b..82272deabf721 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -21,8 +21,8 @@ #include #include #include // NOLINT -#include #include +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -47,7 +47,7 @@ class GraphPyService { std::vector keys; std::vector server_list, port_list, host_sign_list; int server_size, shard_num, rank, client_id; - std::unordered_map table_id_map; + std::unordered_map table_id_map; std::thread *server_thread, *client_thread; std::shared_ptr pserver_ptr; @@ -68,7 +68,8 @@ class GraphPyService { int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( - ::paddle::distributed::TableParameter* sparse_table_proto, uint32_t table_id) { + ::paddle::distributed::TableParameter* sparse_table_proto, + uint32_t table_id) { sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); sparse_table_proto->set_shard_num(shard_num); @@ -97,14 +98,13 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); } return server_fleet_desc; - } ::paddle::distributed::PSParameter GetWorkerProto() { @@ -116,10 +116,10 @@ class GraphPyService { ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = worker_proto->mutable_downpour_worker_param(); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* worker_sparse_table_proto = - downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); } ::paddle::distributed::ServerParameter* server_proto = @@ -134,10 +134,10 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); } return worker_fleet_desc; @@ -148,44 +148,46 @@ class GraphPyService { void load_edge_file(std::string name, std::string filepath, bool reverse) { std::string params = "edge"; - if(reverse) { - params += "|reverse"; + if (reverse) { + params += "|reverse"; } - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); } } void load_node_file(std::string name, std::string filepath) { std::string params = "node"; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); } } - std::vector sample_k(std::string name, uint64_t node_id, int sample_size) { - std::vector v; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); + std::vector> sample_k(std::string name, + uint64_t node_id, + int sample_size) { + std::vector> v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); + status.wait(); } return v; } - std::vector pull_graph_list(std::string name, int server_index, int start, - int size) { + std::vector pull_graph_list(std::string name, int server_index, + int start, int size) { std::vector res; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); } return res; } diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index b6014b9aea139..8da749931cc2f 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,9 +155,9 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res) { + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index f4f235b114dba..8bcf77975afc5 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -13,10 +13,10 @@ // limitations under the License. #include "paddle/fluid/distributed/table/common_graph_table.h" -#include -#include #include +#include #include +#include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" @@ -88,30 +88,30 @@ GraphNode *GraphShard::find_node(uint64_t id) { int32_t GraphTable::load(const std::string &path, const std::string ¶m) { auto cmd = paddle::string::split_string(param, "|"); - std::set cmd_set(cmd.begin(), cmd.end()); + std::set cmd_set(cmd.begin(), cmd.end()); bool load_edge = cmd_set.count(std::string("edge")); bool reverse_edge = cmd_set.count(std::string("reverse")); VLOG(0) << "Reverse Edge " << reverse_edge; - + auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); int count = 0; - + for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); - count ++; + count++; if (values.size() < 2) continue; auto src_id = std::stoull(values[0]); auto dst_id = std::stoull(values[1]); - if(reverse_edge) { - std::swap(src_id, dst_id); + if (reverse_edge) { + std::swap(src_id, dst_id); } - double weight = 0; + float weight = 0; if (values.size() == 3) { - weight = std::stod(values[2]); + weight = std::stof(values[2]); } size_t src_shard_id = src_id % shard_num; @@ -119,7 +119,6 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { VLOG(0) << "will not load " << src_id << " from " << path << ", please check id distribution"; continue; - } size_t index = src_shard_id - shard_start; GraphEdge *edge = new GraphEdge(dst_id, weight); @@ -130,15 +129,15 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { // Build Sampler j for (auto &shard : shards) { - auto bucket = shard.get_bucket(); - for (int i = 0; i < bucket.size(); i++) { - std::list::iterator iter = bucket[i].begin(); - while (iter != bucket[i].end()) { - auto node = *iter; - node->build_sampler(); - iter++; - } + auto bucket = shard.get_bucket(); + for (int i = 0; i < bucket.size(); i++) { + std::list::iterator iter = bucket[i].begin(); + while (iter != bucket[i].end()) { + auto node = *iter; + node->build_sampler(); + iter++; } + } } return 0; } @@ -148,8 +147,6 @@ GraphNode *GraphTable::find_node(uint64_t id) { return NULL; } size_t index = shard_id - shard_start; - // VLOG(0)<<"try to find node-id "< res = node->sample_k(sample_size); std::vector node_list; - int total_size = 0; - for (auto x : res) { - GraphNode temp; - temp.set_id(x->id); - total_size += temp.get_size(); - node_list.push_back(temp); - } - buffer = new char[total_size]; - int index = 0; - for (auto x : node_list) { - x.to_buffer(buffer + index); - index += x.get_size(); + actual_size = + res.size() * (GraphNode::id_size + GraphNode::weight_size); + buffer = new char[actual_size]; + int offset = 0; + uint64_t id; + float weight; + for (auto &x : res) { + id = x->get_id(); + weight = x->get_weight(); + memcpy(buffer + offset, &id, GraphNode::id_size); + offset += GraphNode::id_size; + memcpy(buffer + offset, &weight, GraphNode::weight_size); + offset += GraphNode::weight_size; } - actual_size = total_size; - return 0; }) .get(); - // GraphNode *node = find_node(node_id, type); - // if (node == NULL) { - // actual_size = 0; - // rwlock_->UNLock(); - // return 0; - // } - // std::vector res = node->sample_k(sample_size); - // std::vector node_list; - // int total_size = 0; - // for (auto x : res) { - // GraphNode temp; - // temp.set_id(x->id); - // temp.set_graph_node_type(x->type); - // total_size += temp.get_size(); - // node_list.push_back(temp); - // } - // buffer = new char[total_size]; - // int index = 0; - // for (auto x : node_list) { - // x.to_buffer(buffer + index); - // index += x.get_size(); - // } - // actual_size = total_size; - // rwlock_->UNLock(); - // return 0; } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 78a586d507ef4..c63fff8883636 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -16,9 +16,8 @@ #include namespace paddle { namespace distributed { -int GraphNode::enum_size = sizeof(int); +int GraphNode::weight_size = sizeof(float); int GraphNode::id_size = sizeof(uint64_t); -int GraphNode::double_size = sizeof(double); int GraphNode::int_size = sizeof(int); int GraphNode::get_size() { return feature.size() + id_size + int_size; } void GraphNode::build_sampler() { diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 218d14e01edc1..a8fe5eca3e824 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -20,11 +20,13 @@ namespace distributed { // enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; class GraphEdge : public WeightedObject { public: - double weight; - uint64_t id; // GraphNodeType type; GraphEdge() {} - GraphEdge(uint64_t id, double weight) : weight(weight), id(id) {} + GraphEdge(uint64_t id, float weight) : id(id), weight(weight) {} + uint64_t get_id() { return id; } + float get_weight() { return weight; } + uint64_t id; + float weight; }; class GraphNode { public: @@ -35,7 +37,7 @@ class GraphNode { : id(id), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} std::vector get_graph_edge() { return edges; } - static int enum_size, id_size, int_size, double_size; + static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } // GraphNodeType get_graph_node_type() { return type; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index c93bc551f54f3..09ecdc2b642e4 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/distributed/table/weighted_sampler.h" +#include namespace paddle { namespace distributed { void WeightedSampler::build(WeightedObject **v, int start, int end) { @@ -37,11 +38,11 @@ std::vector WeightedSampler::sample_k(int k) { k = count; } std::vector sample_result; - double subtract; - std::unordered_map subtract_weight_map; + float subtract; + std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; while (k--) { - double query_weight = rand() % 100000 / 100000.0; + float query_weight = rand() % 100000 / 100000.0; query_weight *= weight - subtract_weight_map[this]; sample_result.push_back(sample(query_weight, subtract_weight_map, subtract_count_map, subtract)); @@ -49,10 +50,10 @@ std::vector WeightedSampler::sample_k(int k) { return sample_result; } WeightedObject *WeightedSampler::sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract) { + float &subtract) { if (left == NULL) { subtract_weight_map[this] = weight; subtract = weight; @@ -61,7 +62,7 @@ WeightedObject *WeightedSampler::sample( } int left_count = left->count - subtract_count_map[left]; int right_count = right->count - subtract_count_map[right]; - double left_subtract = subtract_weight_map[left]; + float left_subtract = subtract_weight_map[left]; WeightedObject *return_id; if (right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight) { diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 53bfaa8d30119..9ed2cc04649de 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -22,15 +22,8 @@ class WeightedObject { public: WeightedObject() {} virtual ~WeightedObject() {} - virtual unsigned long long get_id() { return id; } - virtual double get_weight() { return weight; } - - virtual void set_id(unsigned long long id) { this->id = id; } - virtual void set_weight(double weight) { this->weight = weight; } - - private: - unsigned long long id; - double weight; + virtual uint64_t get_id() = 0; + virtual float get_weight() = 0; }; class WeightedSampler { @@ -38,16 +31,16 @@ class WeightedSampler { WeightedSampler *left, *right; WeightedObject *object; int count; - double weight; + float weight; void build(WeightedObject **v, int start, int end); std::vector sample_k(int k); private: WeightedObject *sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract); + float &subtract); }; } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index a2a87025aaed8..fccca80a17302 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include // NOLINT +#include #include #include "google/protobuf/text_format.h" @@ -49,10 +50,18 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), - std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), - std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), - std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; +// std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), +// std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), +// std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), +// std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; + +std::string nodes[] = { + std::string("37\t45\t0.34"), std::string("37\t145\t0.31"), + std::string("37\t112\t0.21"), std::string("96\t48\t1.4"), + std::string("96\t247\t0.31"), std::string("96\t111\t1.21"), + std::string("59\t45\t0.34"), std::string("59\t145\t0.31"), + std::string("59\t122\t0.21"), std::string("97\t48\t0.34"), + std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -210,7 +219,7 @@ void RunBrpcPushSparse() { worker_ptr_->load(0, std::string(file_name), std::string("")); pull_status.wait(); - std::vector v; + std::vector> v; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); // for (auto g : v) { @@ -220,40 +229,46 @@ void RunBrpcPushSparse() { v.clear(); pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); + std::unordered_set s = { 111, 48, 247 } ASSERT_EQ(3, v.size()); for (auto g : v) { - std::cout << g.get_id() << std::endl; + // std::cout << g.first << std::endl; + ASSERT_EQ(true, s.find(g.first) != s.end()) } - // ASSERT_EQ(v.size(),3); v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, v); + std::vector nodes; + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 37); + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 37); // for (auto g : v) { // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; // } // ASSERT_EQ(v.size(),1); - v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, v); + nodes.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 59); - for (auto g : v) { + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 59); + for (auto g : nodes) { std::cout << g.get_id() << std::endl; } distributed::GraphPyService gps1, gps2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; - std::vector edge_types = { std::string("user2item")}; + std::vector edge_types = {std::string("user2item")}; gps1.set_up(ips_str, 127, 0, 0, edge_types); gps2.set_up(ips_str, 127, 1, 1, edge_types); gps1.load_edge_file(std::string("user2item"), std::string(file_name), 0); - v.clear(); - v = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); - ASSERT_EQ(v[0].get_id(), 59); - v.clear(); + nodes.clear(); + nodes = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); + ASSERT_EQ(nodes[0].get_id(), 59); + nodes.clear(); v = gps2.sample_k(std::string("user2item"), 96, 4); ASSERT_EQ(v.size(), 3); + std::cout << "sample result" << std::endl; + for (auto p : v) { + std::cout << p.first << " " << p.second << std::endl; + } // to test in python,try this: // from paddle.fluid.core import GraphPyService // ips_str = "127.0.0.1:4211;127.0.0.1:4212" From 2dc7ebfe0342f39fbbfdc16170f39fd5fb6aa7b3 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 15 Mar 2021 11:39:44 +0800 Subject: [PATCH 019/106] resolve conflict --- .../distributed/table/common_graph_table.cc | 75 +++++++------------ 1 file changed, 25 insertions(+), 50 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index c96eb4680d5aa..600f17d6d0246 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -13,10 +13,10 @@ // limitations under the License. #include "paddle/fluid/distributed/table/common_graph_table.h" -#include -#include #include +#include #include +#include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" @@ -136,23 +136,25 @@ int32_t GraphTable::load_nodes(const std::string &path) { int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { + auto paths = paddle::string::split_string(path, ";"); int count = 0; + for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); - count ++; + count++; if (values.size() < 2) continue; auto src_id = std::stoull(values[0]); auto dst_id = std::stoull(values[1]); - if(reverse_edge) { - std::swap(src_id, dst_id); + if (reverse_edge) { + std::swap(src_id, dst_id); } - double weight = 0; + float weight = 0; if (values.size() == 3) { - weight = std::stod(values[2]); + weight = std::stof(values[2]); } size_t src_shard_id = src_id % shard_num; @@ -161,7 +163,6 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { VLOG(0) << "will not load " << src_id << " from " << path << ", please check id distribution"; continue; - } size_t index = src_shard_id - shard_start; @@ -175,7 +176,8 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { for (auto &shard : shards) { auto bucket = shard.get_bucket(); - for (int i = 0; i < bucket.size(); i ++) { + + for (int i = 0; i < bucket.size(); i++) { std::list::iterator iter = bucket[i].begin(); while (iter != bucket[i].end()) { auto node = *iter; @@ -193,8 +195,6 @@ GraphNode *GraphTable::find_node(uint64_t id) { return NULL; } size_t index = shard_id - shard_start; - // VLOG(0)<<"try to find node-id "< res = node->sample_k(sample_size); std::vector node_list; - int total_size = 0; - for (auto x : res) { - GraphNode temp; - temp.set_id(x->id); - total_size += temp.get_size(); - node_list.push_back(temp); + actual_size = + res.size() * (GraphNode::id_size + GraphNode::weight_size); + buffer = new char[actual_size]; + int offset = 0; + uint64_t id; + float weight; + for (auto &x : res) { + id = x->get_id(); + weight = x->get_weight(); + memcpy(buffer + offset, &id, GraphNode::id_size); + offset += GraphNode::id_size; + memcpy(buffer + offset, &weight, GraphNode::weight_size); + offset += GraphNode::weight_size; } - buffer = new char[total_size]; - int index = 0; - for (auto x : node_list) { - x.to_buffer(buffer + index); - index += x.get_size(); - } - actual_size = total_size; - return 0; }) .get(); - // GraphNode *node = find_node(node_id, type); - // if (node == NULL) { - // actual_size = 0; - // rwlock_->UNLock(); - // return 0; - // } - // std::vector res = node->sample_k(sample_size); - // std::vector node_list; - // int total_size = 0; - // for (auto x : res) { - // GraphNode temp; - // temp.set_id(x->id); - // temp.set_graph_node_type(x->type); - // total_size += temp.get_size(); - // node_list.push_back(temp); - // } - // buffer = new char[total_size]; - // int index = 0; - // for (auto x : node_list) { - // x.to_buffer(buffer + index); - // index += x.get_size(); - // } - // actual_size = total_size; - // rwlock_->UNLock(); - // return 0; } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { @@ -338,3 +312,4 @@ int32_t GraphTable::initialize() { } } }; + From e35913568f27dd0bb4abe70242b59a4b860f3c6e Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 15 Mar 2021 11:45:11 +0800 Subject: [PATCH 020/106] resolved conflict --- .../distributed/service/graph_brpc_client.cc | 25 +++--- .../distributed/service/graph_brpc_client.h | 7 +- .../distributed/service/graph_py_service.h | 84 ++++++++++--------- paddle/fluid/distributed/service/ps_client.h | 6 +- paddle/fluid/distributed/table/graph_node.cc | 3 +- paddle/fluid/distributed/table/graph_node.h | 10 ++- .../distributed/table/weighted_sampler.cc | 15 ++-- .../distributed/table/weighted_sampler.h | 19 ++--- .../fluid/distributed/test/graph_node_test.cc | 55 +++++++----- 9 files changed, 116 insertions(+), 108 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 16014c8dbf23f..bc6a03b9eaf84 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -12,15 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include #include #include #include +#include #include #include "Eigen/Dense" - #include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/string/string_helper.h" @@ -35,9 +35,9 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::sample(uint32_t table_id, - uint64_t node_id, int sample_size, - std::vector &res) { +std::future GraphBrpcClient::sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { int server_index = get_server_index_by_id(node_id); DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; @@ -45,19 +45,16 @@ std::future GraphBrpcClient::sample(uint32_t table_id, if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { ret = -1; } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_GRAPH_SAMPLE); auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); char *buffer = new char[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - int start = 0; - while (start < bytes_size) { - GraphNode node; - node.recover_from_buffer(buffer + start); - start += node.get_size(); - res.push_back(node); + int offset = 0; + while (offset < bytes_size) { + res.push_back({*(uint64_t *)(buffer + offset), + *(float *)(buffer + offset + GraphNode::id_size)}); + offset += GraphNode::id_size + GraphNode::weight_size; } } closure->set_promise_value(ret); @@ -69,9 +66,7 @@ std::future GraphBrpcClient::sample(uint32_t table_id, closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(0)->set_table_id(table_id); closure->request(0)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - // closure->request(0)->add_params(type_str.c_str(), type_str.size()); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 8e472b96be94d..84d4dbb78cba4 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -18,6 +18,7 @@ #include #include +#include #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" @@ -35,9 +36,9 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res); + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 25b70d11fe05b..82272deabf721 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -21,8 +21,8 @@ #include #include #include // NOLINT -#include #include +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -47,7 +47,7 @@ class GraphPyService { std::vector keys; std::vector server_list, port_list, host_sign_list; int server_size, shard_num, rank, client_id; - std::unordered_map table_id_map; + std::unordered_map table_id_map; std::thread *server_thread, *client_thread; std::shared_ptr pserver_ptr; @@ -68,7 +68,8 @@ class GraphPyService { int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( - ::paddle::distributed::TableParameter* sparse_table_proto, uint32_t table_id) { + ::paddle::distributed::TableParameter* sparse_table_proto, + uint32_t table_id) { sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); sparse_table_proto->set_shard_num(shard_num); @@ -97,14 +98,13 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); } return server_fleet_desc; - } ::paddle::distributed::PSParameter GetWorkerProto() { @@ -116,10 +116,10 @@ class GraphPyService { ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = worker_proto->mutable_downpour_worker_param(); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* worker_sparse_table_proto = - downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); } ::paddle::distributed::ServerParameter* server_proto = @@ -134,10 +134,10 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - for(auto& tuple : this -> table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); } return worker_fleet_desc; @@ -148,44 +148,46 @@ class GraphPyService { void load_edge_file(std::string name, std::string filepath, bool reverse) { std::string params = "edge"; - if(reverse) { - params += "|reverse"; + if (reverse) { + params += "|reverse"; } - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); } } void load_node_file(std::string name, std::string filepath) { std::string params = "node"; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); } } - std::vector sample_k(std::string name, uint64_t node_id, int sample_size) { - std::vector v; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); + std::vector> sample_k(std::string name, + uint64_t node_id, + int sample_size) { + std::vector> v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); + status.wait(); } return v; } - std::vector pull_graph_list(std::string name, int server_index, int start, - int size) { + std::vector pull_graph_list(std::string name, int server_index, + int start, int size) { std::vector res; - if (this -> table_id_map.count(name)) { - uint32_t table_id = this -> table_id_map[name]; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); } return res; } diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index b6014b9aea139..8da749931cc2f 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,9 +155,9 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res) { + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 78a586d507ef4..c63fff8883636 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -16,9 +16,8 @@ #include namespace paddle { namespace distributed { -int GraphNode::enum_size = sizeof(int); +int GraphNode::weight_size = sizeof(float); int GraphNode::id_size = sizeof(uint64_t); -int GraphNode::double_size = sizeof(double); int GraphNode::int_size = sizeof(int); int GraphNode::get_size() { return feature.size() + id_size + int_size; } void GraphNode::build_sampler() { diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 218d14e01edc1..a8fe5eca3e824 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -20,11 +20,13 @@ namespace distributed { // enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; class GraphEdge : public WeightedObject { public: - double weight; - uint64_t id; // GraphNodeType type; GraphEdge() {} - GraphEdge(uint64_t id, double weight) : weight(weight), id(id) {} + GraphEdge(uint64_t id, float weight) : id(id), weight(weight) {} + uint64_t get_id() { return id; } + float get_weight() { return weight; } + uint64_t id; + float weight; }; class GraphNode { public: @@ -35,7 +37,7 @@ class GraphNode { : id(id), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} std::vector get_graph_edge() { return edges; } - static int enum_size, id_size, int_size, double_size; + static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } // GraphNodeType get_graph_node_type() { return type; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index c93bc551f54f3..09ecdc2b642e4 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/distributed/table/weighted_sampler.h" +#include namespace paddle { namespace distributed { void WeightedSampler::build(WeightedObject **v, int start, int end) { @@ -37,11 +38,11 @@ std::vector WeightedSampler::sample_k(int k) { k = count; } std::vector sample_result; - double subtract; - std::unordered_map subtract_weight_map; + float subtract; + std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; while (k--) { - double query_weight = rand() % 100000 / 100000.0; + float query_weight = rand() % 100000 / 100000.0; query_weight *= weight - subtract_weight_map[this]; sample_result.push_back(sample(query_weight, subtract_weight_map, subtract_count_map, subtract)); @@ -49,10 +50,10 @@ std::vector WeightedSampler::sample_k(int k) { return sample_result; } WeightedObject *WeightedSampler::sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract) { + float &subtract) { if (left == NULL) { subtract_weight_map[this] = weight; subtract = weight; @@ -61,7 +62,7 @@ WeightedObject *WeightedSampler::sample( } int left_count = left->count - subtract_count_map[left]; int right_count = right->count - subtract_count_map[right]; - double left_subtract = subtract_weight_map[left]; + float left_subtract = subtract_weight_map[left]; WeightedObject *return_id; if (right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight) { diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 53bfaa8d30119..9ed2cc04649de 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -22,15 +22,8 @@ class WeightedObject { public: WeightedObject() {} virtual ~WeightedObject() {} - virtual unsigned long long get_id() { return id; } - virtual double get_weight() { return weight; } - - virtual void set_id(unsigned long long id) { this->id = id; } - virtual void set_weight(double weight) { this->weight = weight; } - - private: - unsigned long long id; - double weight; + virtual uint64_t get_id() = 0; + virtual float get_weight() = 0; }; class WeightedSampler { @@ -38,16 +31,16 @@ class WeightedSampler { WeightedSampler *left, *right; WeightedObject *object; int count; - double weight; + float weight; void build(WeightedObject **v, int start, int end); std::vector sample_k(int k); private: WeightedObject *sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract); + float &subtract); }; } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index a2a87025aaed8..fccca80a17302 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include // NOLINT +#include #include #include "google/protobuf/text_format.h" @@ -49,10 +50,18 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), - std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), - std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), - std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; +// std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), +// std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), +// std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), +// std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; + +std::string nodes[] = { + std::string("37\t45\t0.34"), std::string("37\t145\t0.31"), + std::string("37\t112\t0.21"), std::string("96\t48\t1.4"), + std::string("96\t247\t0.31"), std::string("96\t111\t1.21"), + std::string("59\t45\t0.34"), std::string("59\t145\t0.31"), + std::string("59\t122\t0.21"), std::string("97\t48\t0.34"), + std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -210,7 +219,7 @@ void RunBrpcPushSparse() { worker_ptr_->load(0, std::string(file_name), std::string("")); pull_status.wait(); - std::vector v; + std::vector> v; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); // for (auto g : v) { @@ -220,40 +229,46 @@ void RunBrpcPushSparse() { v.clear(); pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); + std::unordered_set s = { 111, 48, 247 } ASSERT_EQ(3, v.size()); for (auto g : v) { - std::cout << g.get_id() << std::endl; + // std::cout << g.first << std::endl; + ASSERT_EQ(true, s.find(g.first) != s.end()) } - // ASSERT_EQ(v.size(),3); v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, v); + std::vector nodes; + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 37); + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 37); // for (auto g : v) { // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; // } // ASSERT_EQ(v.size(),1); - v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, v); + nodes.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 59); - for (auto g : v) { + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 59); + for (auto g : nodes) { std::cout << g.get_id() << std::endl; } distributed::GraphPyService gps1, gps2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; - std::vector edge_types = { std::string("user2item")}; + std::vector edge_types = {std::string("user2item")}; gps1.set_up(ips_str, 127, 0, 0, edge_types); gps2.set_up(ips_str, 127, 1, 1, edge_types); gps1.load_edge_file(std::string("user2item"), std::string(file_name), 0); - v.clear(); - v = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); - ASSERT_EQ(v[0].get_id(), 59); - v.clear(); + nodes.clear(); + nodes = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); + ASSERT_EQ(nodes[0].get_id(), 59); + nodes.clear(); v = gps2.sample_k(std::string("user2item"), 96, 4); ASSERT_EQ(v.size(), 3); + std::cout << "sample result" << std::endl; + for (auto p : v) { + std::cout << p.first << " " << p.second << std::endl; + } // to test in python,try this: // from paddle.fluid.core import GraphPyService // ips_str = "127.0.0.1:4211;127.0.0.1:4212" From 290840f47dc62b942acd09777b572da5a3bbd044 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 15 Mar 2021 11:59:14 +0800 Subject: [PATCH 021/106] resolved conflict --- paddle/fluid/distributed/test/graph_node_test.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index fccca80a17302..50b4c0eda9b6f 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -229,10 +229,11 @@ void RunBrpcPushSparse() { v.clear(); pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); - std::unordered_set s = { 111, 48, 247 } ASSERT_EQ(3, v.size()); + std::unordered_set s = { 111, 48, 247 }; + ASSERT_EQ(3, v.size()); for (auto g : v) { // std::cout << g.first << std::endl; - ASSERT_EQ(true, s.find(g.first) != s.end()) + ASSERT_EQ(true, s.find(g.first) != s.end()); } v.clear(); std::vector nodes; From d98be69971160359874419d32cb88a406108e731 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Mon, 15 Mar 2021 06:47:25 +0000 Subject: [PATCH 022/106] separate server and client --- .../distributed/service/graph_py_service.cc | 176 ++++++++++++-- .../distributed/service/graph_py_service.h | 214 +++++------------- .../fluid/distributed/test/graph_node_test.cc | 61 ++++- paddle/fluid/pybind/fleet_py.cc | 26 ++- paddle/fluid/pybind/fleet_py.h | 5 +- paddle/fluid/pybind/pybind.cc | 4 +- 6 files changed, 290 insertions(+), 196 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 04c04930e43e8..2df3843c96afe 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -13,9 +13,15 @@ // limitations under the License. #include "paddle/fluid/distributed/service/graph_py_service.h" +#include // NOLINT +#include "butil/endpoint.h" +#include "iomanip" +#include "paddle/fluid/distributed/table/table.h" +#include "paddle/fluid/framework/archive.h" +#include "paddle/fluid/platform/profiler.h" namespace paddle { namespace distributed { -std::vector GraphPyService::split(std::string &str, +std::vector GraphPyService::split(std::string& str, const char pattern) { std::vector res; std::stringstream input(str); @@ -26,18 +32,17 @@ std::vector GraphPyService::split(std::string &str, return res; } -void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, - int client_id, std::vector edge_types) { +void GraphPyService::set_up(std::string ips_str, int shard_num, + std::vector edge_types) { set_shard_num(shard_num); - set_client_Id(client_id); - set_rank(rank); + // set_client_Id(client_id); + // set_rank(rank); - this -> table_id_map[std::string("")] = 0; + this->table_id_map[std::string("")] = 0; // Table 0 are for nodes - for(size_t table_id = 0; table_id < edge_types.size(); table_id ++ ) { - this -> table_id_map[edge_types[table_id]] = int(table_id + 1); - } - server_thread = client_thread = NULL; + for (size_t table_id = 0; table_id < edge_types.size(); table_id++) { + this->table_id_map[edge_types[table_id]] = int(table_id + 1); + } std::istringstream stream(ips_str); std::string ip; server_size = 0; @@ -52,10 +57,153 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, host_sign_list.push_back(ph_host.serialize_to_string()); index++; } - //VLOG(0) << "IN set up rank = " << rank; - start_client(); - start_server(server_list[rank], std::stoul(port_list[rank])); - sleep(1); + // VLOG(0) << "IN set up rank = " << rank; + // start_client(); + // start_server(server_list[rank], std::stoul(port_list[rank])); +} +void GraphPyClient::start_client() { + std::map> dense_regions; + dense_regions.insert( + std::pair>(0, {})); + auto regions = dense_regions[0]; + ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); + paddle::distributed::PaddlePSEnvironment _ps_env; + auto servers_ = host_sign_list.size(); + _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&host_sign_list, servers_); + worker_ptr = std::shared_ptr( + paddle::distributed::PSClientFactory::create(worker_proto)); + worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); +} +void GraphPyServer::start_server() { + std::string ip = server_list[rank]; + uint32_t port = std::stoul(port_list[rank]); + server_thread = new std::thread([this, &ip, &port]() { + std::function func = [this, &ip, &port]() { + ::paddle::distributed::PSParameter server_proto = this->GetServerProto(); + + auto _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&this->host_sign_list, + this->host_sign_list.size()); // test + pserver_ptr = std::shared_ptr( + paddle::distributed::PSServerFactory::create(server_proto)); + VLOG(0) << "pserver-ptr created "; + std::vector empty_vec; + framework::ProgramDesc empty_prog; + empty_vec.push_back(empty_prog); + pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); + pserver_ptr->start(ip, port); + }; + std::thread t1(func); + t1.join(); + }); + sleep(3); +} +::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { + // Generate server proto desc + ::paddle::distributed::PSParameter server_fleet_desc; + server_fleet_desc.set_shard_num(get_shard_num()); + ::paddle::distributed::ServerParameter* server_proto = + server_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); + + for (auto& tuple : this->table_id_map) { + VLOG(0) << " make a new table " << tuple.second; + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } + + return server_fleet_desc; +} + +::paddle::distributed::PSParameter GraphPyClient::GetWorkerProto() { + ::paddle::distributed::PSParameter worker_fleet_desc; + worker_fleet_desc.set_shard_num(get_shard_num()); + ::paddle::distributed::WorkerParameter* worker_proto = + worker_fleet_desc.mutable_worker_param(); + + ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = + worker_proto->mutable_downpour_worker_param(); + + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + } + + ::paddle::distributed::ServerParameter* server_proto = + worker_fleet_desc.mutable_server_param(); + ::paddle::distributed::DownpourServerParameter* downpour_server_proto = + server_proto->mutable_downpour_server_param(); + ::paddle::distributed::ServerServiceParameter* server_service_proto = + downpour_server_proto->mutable_service_param(); + server_service_proto->set_service_class("GraphBrpcService"); + server_service_proto->set_server_class("GraphBrpcServer"); + server_service_proto->set_client_class("GraphBrpcClient"); + server_service_proto->set_start_server_port(0); + server_service_proto->set_server_thread_num(12); + + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } + + return worker_fleet_desc; +} +void GraphPyClient::load_edge_file(std::string name, std::string filepath, + bool reverse) { + std::string params = "edge"; + if (reverse) { + params += "|reverse"; + } + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } +} + +void GraphPyClient::load_node_file(std::string name, std::string filepath) { + std::string params = "node"; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } +} +std::vector> GraphPyClient::sample_k( + std::string name, uint64_t node_id, int sample_size) { + std::vector> v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); + status.wait(); + } + return v; +} +std::vector GraphPyClient::pull_graph_list(std::string name, + int server_index, + int start, int size) { + std::vector res; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); + } + return res; } } } diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 82272deabf721..56b47d71093f6 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -44,27 +44,23 @@ namespace paddle { namespace distributed { class GraphPyService { - std::vector keys; + protected: std::vector server_list, port_list, host_sign_list; - int server_size, shard_num, rank, client_id; + int server_size, shard_num; std::unordered_map table_id_map; - std::thread *server_thread, *client_thread; + // std::thread *server_thread, *client_thread; - std::shared_ptr pserver_ptr; + // std::shared_ptr pserver_ptr; - std::shared_ptr worker_ptr; + // std::shared_ptr worker_ptr; public: - std::shared_ptr get_ps_server() { - return pserver_ptr; - } - std::shared_ptr get_ps_client() { - return worker_ptr; - } - int get_client_id() { return client_id; } - void set_client_Id(int client_Id) { this->client_id = client_id; } - int get_rank() { return rank; } - void set_rank(int rank) { this->rank = rank; } + // std::shared_ptr get_ps_server() { + // return pserver_ptr; + // } + // std::shared_ptr get_ps_client() { + // return worker_ptr; + // } int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( @@ -82,162 +78,58 @@ class GraphPyService { accessor_proto->set_accessor_class("CommMergeAccessor"); } - ::paddle::distributed::PSParameter GetServerProto() { - // Generate server proto desc - ::paddle::distributed::PSParameter server_fleet_desc; - server_fleet_desc.set_shard_num(shard_num); - ::paddle::distributed::ServerParameter* server_proto = - server_fleet_desc.mutable_server_param(); - ::paddle::distributed::DownpourServerParameter* downpour_server_proto = - server_proto->mutable_downpour_server_param(); - ::paddle::distributed::ServerServiceParameter* server_service_proto = - downpour_server_proto->mutable_service_param(); - server_service_proto->set_service_class("GraphBrpcService"); - server_service_proto->set_server_class("GraphBrpcServer"); - server_service_proto->set_client_class("GraphBrpcClient"); - server_service_proto->set_start_server_port(0); - server_service_proto->set_server_thread_num(12); - - for (auto& tuple : this->table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); - } - - return server_fleet_desc; - } - - ::paddle::distributed::PSParameter GetWorkerProto() { - ::paddle::distributed::PSParameter worker_fleet_desc; - worker_fleet_desc.set_shard_num(shard_num); - ::paddle::distributed::WorkerParameter* worker_proto = - worker_fleet_desc.mutable_worker_param(); - - ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = - worker_proto->mutable_downpour_worker_param(); - - for (auto& tuple : this->table_id_map) { - ::paddle::distributed::TableParameter* worker_sparse_table_proto = - downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); - } - - ::paddle::distributed::ServerParameter* server_proto = - worker_fleet_desc.mutable_server_param(); - ::paddle::distributed::DownpourServerParameter* downpour_server_proto = - server_proto->mutable_downpour_server_param(); - ::paddle::distributed::ServerServiceParameter* server_service_proto = - downpour_server_proto->mutable_service_param(); - server_service_proto->set_service_class("GraphBrpcService"); - server_service_proto->set_server_class("GraphBrpcServer"); - server_service_proto->set_client_class("GraphBrpcClient"); - server_service_proto->set_start_server_port(0); - server_service_proto->set_server_thread_num(12); - - for (auto& tuple : this->table_id_map) { - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); - } - - return worker_fleet_desc; - } void set_server_size(int server_size) { this->server_size = server_size; } int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); - - void load_edge_file(std::string name, std::string filepath, bool reverse) { - std::string params = "edge"; - if (reverse) { - params += "|reverse"; - } - if (this->table_id_map.count(name)) { - uint32_t table_id = this->table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); - } + void set_up(std::string ips_str, int shard_num, + std::vector edge_types); +}; +class GraphPyServer : public GraphPyService { + public: + void set_up(std::string ips_str, int shard_num, + std::vector edge_types, int rank) { + set_rank(rank); + GraphPyService::set_up(ips_str, shard_num, edge_types); } - - void load_node_file(std::string name, std::string filepath) { - std::string params = "node"; - if (this->table_id_map.count(name)) { - uint32_t table_id = this->table_id_map[name]; - auto status = - get_ps_client()->load(table_id, std::string(filepath), params); - status.wait(); - } + int get_rank() { return rank; } + void set_rank(int rank) { this->rank = rank; } + void start_server(); + ::paddle::distributed::PSParameter GetServerProto(); + std::shared_ptr get_ps_server() { + return pserver_ptr; } + protected: + int rank; + std::shared_ptr pserver_ptr; + std::thread* server_thread; +}; +class GraphPyClient : public GraphPyService { + public: + void set_up(std::string ips_str, int shard_num, + std::vector edge_types, int client_id) { + set_client_id(client_id); + GraphPyService::set_up(ips_str, shard_num, edge_types); + } + std::shared_ptr get_ps_client() { + return worker_ptr; + } + void load_edge_file(std::string name, std::string filepath, bool reverse); + void load_node_file(std::string name, std::string filepath); + int get_client_id() { return client_id; } + void set_client_id(int client_id) { this->client_id = client_id; } + void start_client(); std::vector> sample_k(std::string name, uint64_t node_id, - int sample_size) { - std::vector> v; - if (this->table_id_map.count(name)) { - uint32_t table_id = this->table_id_map[name]; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); - } - return v; - } + int sample_size); std::vector pull_graph_list(std::string name, int server_index, - int start, int size) { - std::vector res; - if (this->table_id_map.count(name)) { - uint32_t table_id = this->table_id_map[name]; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); - status.wait(); - } - return res; - } - void start_server(std::string ip, uint32_t port) { - server_thread = new std::thread([this, &ip, &port]() { - std::function func = [this, &ip, &port]() { - VLOG(0) << "enter inner function "; - ::paddle::distributed::PSParameter server_proto = - this->GetServerProto(); + int start, int size); + ::paddle::distributed::PSParameter GetWorkerProto(); - auto _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&this->host_sign_list, - this->host_sign_list.size()); // test - pserver_ptr = std::shared_ptr( - paddle::distributed::PSServerFactory::create(server_proto)); - VLOG(0) << "pserver-ptr created "; - std::vector empty_vec; - framework::ProgramDesc empty_prog; - empty_vec.push_back(empty_prog); - pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); - VLOG(0) << " starting server " << ip << " " << port; - pserver_ptr->start(ip, port); - }; - std::thread t1(func); - t1.join(); - }); - } - void start_client() { - VLOG(0) << "in start_client " << rank; - std::map> dense_regions; - dense_regions.insert( - std::pair>(0, {})); - auto regions = dense_regions[0]; - ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); - paddle::distributed::PaddlePSEnvironment _ps_env; - auto servers_ = host_sign_list.size(); - _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&host_sign_list, servers_); - worker_ptr = std::shared_ptr( - paddle::distributed::PSClientFactory::create(worker_proto)); - worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); - } - void set_up(std::string ips_str, int shard_num, int rank, int client_id, - std::vector edge_types); - void set_keys(std::vector keys) { // just for test - this->keys = keys; - } - std::vector get_keys(int start, int size) { // just for test - return std::vector(keys.begin() + start, keys.begin() + start + size); - } + protected: + int client_id; + std::shared_ptr worker_ptr; + std::thread* client_thread; }; } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index fccca80a17302..9d0aa046db101 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -222,17 +222,15 @@ void RunBrpcPushSparse() { std::vector> v; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); - // for (auto g : v) { - // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; - // } ASSERT_EQ(v.size(), 3); v.clear(); pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); - std::unordered_set s = { 111, 48, 247 } ASSERT_EQ(3, v.size()); + std::unordered_set s = {111, 48, 247}; + ASSERT_EQ(3, v.size()); for (auto g : v) { // std::cout << g.first << std::endl; - ASSERT_EQ(true, s.find(g.first) != s.end()) + ASSERT_EQ(true, s.find(g.first) != s.end()); } v.clear(); std::vector nodes; @@ -253,22 +251,63 @@ void RunBrpcPushSparse() { std::cout << g.get_id() << std::endl; } - distributed::GraphPyService gps1, gps2; + // distributed::GraphPyService gps1, gps2; + distributed::GraphPyServer server1, server2; + distributed::GraphPyClient client1, client2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; std::vector edge_types = {std::string("user2item")}; - gps1.set_up(ips_str, 127, 0, 0, edge_types); - gps2.set_up(ips_str, 127, 1, 1, edge_types); - gps1.load_edge_file(std::string("user2item"), std::string(file_name), 0); + server1.set_up(ips_str, 127, edge_types, 0); + server2.set_up(ips_str, 127, edge_types, 1); + client1.set_up(ips_str, 127, edge_types, 0); + client2.set_up(ips_str, 127, edge_types, 1); + server1.start_server(); + std::cout << "first server done" << std::endl; + server2.start_server(); + std::cout << "second server done" << std::endl; + client1.start_client(); + std::cout << "first client done" << std::endl; + client2.start_client(); + std::cout << "first client done" << std::endl; + std::cout << "started" << std::endl; + client1.load_edge_file(std::string("user2item"), std::string(file_name), 0); + // client2.load_edge_file(std::string("user2item"), std::string(file_name), + // 0); nodes.clear(); - nodes = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); + nodes = client2.pull_graph_list(std::string("user2item"), 0, 1, 4); ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); - v = gps2.sample_k(std::string("user2item"), 96, 4); + v = client1.sample_k(std::string("user2item"), 96, 4); ASSERT_EQ(v.size(), 3); std::cout << "sample result" << std::endl; for (auto p : v) { std::cout << p.first << " " << p.second << std::endl; } + /* +from paddle.fluid.core import GraphPyService +ips_str = "127.0.0.1:4211;127.0.0.1:4212" +server1 = GraphPyServer() +server2 = GraphPyServer() +client1 = GraphPyClient() +client2 = GraphPyClient() +edge_types = ["user2item"] +server1.set_up(ips_str,127,edge_types,0); +server2.set_up(ips_str,127,edge_types,1); +client1.set_up(ips_str,127,edge_types,0); +client2.set_up(ips_str,127,edge_types,1); +server1.start_server(); +server2.start_server(); +client1.start_client(); +client2.start_client(); +client1.load_edge_file(user2item", "input.txt", 0); +list = client2.pull_graph_list("user2item",0,1,4) +for x in list: + print(x.get_id()) + +list = client1.sample_k("user2item",96, 4); +for x in list: + print(x.get_id()) + */ + // to test in python,try this: // from paddle.fluid.core import GraphPyService // ips_str = "127.0.0.1:4211;127.0.0.1:4212" diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index e967acee77d20..ac40a61147512 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -43,6 +43,8 @@ using paddle::distributed::FleetWrapper; using paddle::distributed::HeterClient; using paddle::distributed::GraphPyService; using paddle::distributed::GraphNode; +using paddle::distributed::GraphPyServer; +using paddle::distributed::GraphPyClient; namespace paddle { namespace pybind { @@ -162,14 +164,24 @@ void BindGraphNode(py::module* m) { .def("get_id", &GraphNode::get_id) .def("get_feature", &GraphNode::get_feature); } -void BindGraphService(py::module* m) { - py::class_(*m, "GraphPyService") +void BindGraphPyService(py::module* m) { + py::class_(*m, "GraphPyService").def(py::init<>()); +} +void BindGraphPyServer(py::module* m) { + py::class_(*m, "GraphPyServer") + .def(py::init<>()) + .def("start_server", &GraphPyServer::start_server) + .def("set_up", &GraphPyServer::set_up); +} +void BindGraphPyClient(py::module* m) { + py::class_(*m, "GraphPyClient") .def(py::init<>()) - .def("load_edge_file", &GraphPyService::load_edge_file) - .def("load_node_file", &GraphPyService::load_node_file) - .def("set_up", &GraphPyService::set_up) - .def("pull_graph_list", &GraphPyService::pull_graph_list) - .def("sample_k", &GraphPyService::sample_k); + .def("load_edge_file", &GraphPyClient::load_edge_file) + .def("load_node_file", &GraphPyClient::load_node_file) + .def("set_up", &GraphPyClient::set_up) + .def("pull_graph_list", &GraphPyClient::pull_graph_list) + .def("sample_k", &GraphPyClient::sample_k) + .def("start_client", &GraphPyClient::start_client); } } // end namespace pybind diff --git a/paddle/fluid/pybind/fleet_py.h b/paddle/fluid/pybind/fleet_py.h index 57861a8e2ed6e..aea5e69ca0df1 100644 --- a/paddle/fluid/pybind/fleet_py.h +++ b/paddle/fluid/pybind/fleet_py.h @@ -28,7 +28,8 @@ void BindCommunicatorContext(py::module* m); void BindDistCommunicator(py::module* m); void BindHeterClient(py::module* m); void BindGraphNode(py::module* m); -void BindGraphService(py::module* m); - +void BindGraphPyService(py::module* m); +void BindGraphPyServer(py::module* m); +void BindGraphPyClient(py::module* m); } // namespace pybind } // namespace paddle diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 34a44e74c1b34..9bba6ab74e3ca 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -2847,7 +2847,9 @@ All parameter, weight, gradient are variables in Paddle. BindDistCommunicator(&m); BindHeterClient(&m); BindGraphNode(&m); - BindGraphService(&m); + BindGraphPyService(&m); + BindGraphPyServer(&m); + BindGraphPyClient(&m); #endif } } // namespace pybind From db6ee862c56c9af33ad01ae720f6e114aa3e1ba9 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 14:48:39 +0800 Subject: [PATCH 023/106] merge pair type --- .../distributed/service/graph_brpc_client.cc | 24 +-- .../distributed/service/graph_brpc_client.h | 7 +- .../distributed/service/graph_py_service.cc | 13 +- .../distributed/service/graph_py_service.h | 93 ++++++++---- paddle/fluid/distributed/service/ps_client.h | 6 +- .../distributed/table/common_graph_table.cc | 142 ++++++++---------- paddle/fluid/distributed/table/graph_node.cc | 3 +- paddle/fluid/distributed/table/graph_node.h | 10 +- .../distributed/table/weighted_sampler.cc | 15 +- .../distributed/table/weighted_sampler.h | 19 +-- .../fluid/distributed/test/graph_node_test.cc | 65 +++++--- paddle/fluid/pybind/fleet_py.cc | 3 +- 12 files changed, 223 insertions(+), 177 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index a5134c8614d47..616d310f07fbc 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -12,15 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include #include #include #include +#include #include #include "Eigen/Dense" - #include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/service/graph_brpc_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/string/string_helper.h" @@ -35,9 +35,9 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::sample(uint32_t table_id, - uint64_t node_id, int sample_size, - std::vector &res) { +std::future GraphBrpcClient::sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { int server_index = get_server_index_by_id(node_id); DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; @@ -45,8 +45,6 @@ std::future GraphBrpcClient::sample(uint32_t table_id, if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { ret = -1; } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_GRAPH_SAMPLE); auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); @@ -62,10 +60,13 @@ std::future GraphBrpcClient::sample(uint32_t table_id, int actual_size = actual_sizes[idx]; int start = 0; while (start < actual_size) { - GraphNode node; - node.recover_from_buffer(node_buffer + offset + start); - start += node.get_size(); - res.push_back(node); + //GraphNode node; + //node.recover_from_buffer(node_buffer + offset + start); + //start += node.get_size(); + //res.push_back(node); + res.push_back({*(uint64_t *)(node_buffer + offset + start), + *(float *)(node_buffer + offset + start + GraphNode::id_size)}); + start += GraphNode::id_size + GraphNode::weight_size; } offset += actual_size; } @@ -84,6 +85,7 @@ std::future GraphBrpcClient::sample(uint32_t table_id, size_t node_num = node_ids.size(); closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*node_num); + //closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index d0653120e07ce..4778d5261219f 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -18,6 +18,7 @@ #include #include +#include #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" @@ -35,9 +36,9 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res); + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res); virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector > &res); diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 86d2d54193bd9..04c04930e43e8 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -27,11 +27,16 @@ std::vector GraphPyService::split(std::string &str, } void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, - int client_id, uint32_t table_id) { + int client_id, std::vector edge_types) { set_shard_num(shard_num); set_client_Id(client_id); set_rank(rank); - this->table_id = table_id; + + this -> table_id_map[std::string("")] = 0; + // Table 0 are for nodes + for(size_t table_id = 0; table_id < edge_types.size(); table_id ++ ) { + this -> table_id_map[edge_types[table_id]] = int(table_id + 1); + } server_thread = client_thread = NULL; std::istringstream stream(ips_str); std::string ip; @@ -47,10 +52,10 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, int rank, host_sign_list.push_back(ph_host.serialize_to_string()); index++; } - VLOG(0) << "IN set up rank = " << rank; + //VLOG(0) << "IN set up rank = " << rank; start_client(); start_server(server_list[rank], std::stoul(port_list[rank])); sleep(1); } } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 0975460003436..e4f049f2eb26a 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -21,6 +21,7 @@ #include #include #include // NOLINT +#include #include #include "google/protobuf/text_format.h" @@ -46,7 +47,7 @@ class GraphPyService { std::vector keys; std::vector server_list, port_list, host_sign_list; int server_size, shard_num, rank, client_id; - uint32_t table_id; + std::unordered_map table_id_map; std::thread *server_thread, *client_thread; std::shared_ptr pserver_ptr; @@ -67,7 +68,8 @@ class GraphPyService { int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( - ::paddle::distributed::TableParameter* sparse_table_proto) { + ::paddle::distributed::TableParameter* sparse_table_proto, + uint32_t table_id) { sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); sparse_table_proto->set_shard_num(shard_num); @@ -96,9 +98,12 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - ::paddle::distributed::TableParameter* sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } + return server_fleet_desc; } @@ -111,9 +116,11 @@ class GraphPyService { ::paddle::distributed::DownpourWorkerParameter* downpour_worker_proto = worker_proto->mutable_downpour_worker_param(); - ::paddle::distributed::TableParameter* worker_sparse_table_proto = - downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* worker_sparse_table_proto = + downpour_worker_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + } ::paddle::distributed::ServerParameter* server_proto = worker_fleet_desc.mutable_server_param(); @@ -127,9 +134,11 @@ class GraphPyService { server_service_proto->set_start_server_port(0); server_service_proto->set_server_thread_num(12); - ::paddle::distributed::TableParameter* server_sparse_table_proto = - downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(server_sparse_table_proto); + for (auto& tuple : this->table_id_map) { + ::paddle::distributed::TableParameter* sparse_table_proto = + downpour_server_proto->add_downpour_table_param(); + GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + } return worker_fleet_desc; } @@ -137,30 +146,58 @@ class GraphPyService { int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); - void load_file(std::string filepath) { - auto status = - get_ps_client()->load(table_id, std::string(filepath), std::string("")); - status.wait(); + void load_edge_file(std::string name, std::string filepath, bool reverse) { + std::string params = "edge"; + if (reverse) { + params += "|reverse"; + } + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } + } + + void load_node_file(std::string name, std::string filepath) { + std::string params = "node"; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->load(table_id, std::string(filepath), params); + status.wait(); + } } - std::vector sample_k(uint64_t node_id, int sample_size) { - std::vector v; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); + std::vector> sample_k(std::string name, + uint64_t node_id, + int sample_size) { + std::vector> v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->sample(table_id, node_id, sample_size, v); + status.wait(); + } return v; } - std::vector > batch_sample_k(std::vector node_ids, int sample_size) { + std::vector > batch_sample_k(std::string name, std::vector node_ids, int sample_size) { std::vector > v; - auto status = worker_ptr->batch_sample(table_id, node_ids, sample_size, v); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->batch_sample(table_id, node_ids, sample_size, v); + status.wait(); + } return v; } - std::vector pull_graph_list(int server_index, int start, - int size) { + std::vector pull_graph_list(std::string name, int server_index, + int start, int size) { std::vector res; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); - status.wait(); + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + status.wait(); + } return res; } void start_server(std::string ip, uint32_t port) { @@ -203,7 +240,7 @@ class GraphPyService { worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); } void set_up(std::string ips_str, int shard_num, int rank, int client_id, - uint32_t table_id); + std::vector edge_types); void set_keys(std::vector keys) { // just for test this->keys = keys; } diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 2122743197052..78ef708ffe8b4 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,9 +155,9 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future sample(uint32_t table_id, uint64_t node_id, - int sample_size, - std::vector &res) { + virtual std::future sample( + uint32_t table_id, uint64_t node_id, int sample_size, + std::vector> &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 9b9e9a88b84a7..4910ba125355a 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -13,14 +13,18 @@ // limitations under the License. #include "paddle/fluid/distributed/table/common_graph_table.h" +#include #include +#include #include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { + int GraphShard::bucket_low_bound = 11; + std::vector GraphShard::get_batch(int start, int total_size) { if (start < 0) start = 0; int size = 0, cur_size; @@ -51,6 +55,7 @@ std::vector GraphShard::get_batch(int start, int total_size) { } return res; } + size_t GraphShard::get_size() { size_t res = 0; for (int i = 0; i < bucket_size; i++) { @@ -58,69 +63,79 @@ size_t GraphShard::get_size() { } return res; } + std::list::iterator GraphShard::add_node(GraphNode *node) { if (node_location.find(node->get_id()) != node_location.end()) return node_location.find(node->get_id())->second; + int index = node->get_id() % shard_num % bucket_size; + std::list::iterator iter = bucket[index].insert(bucket[index].end(), node); + node_location[node->get_id()] = iter; return iter; } + void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { (*add_node(new GraphNode(id, std::string(""))))->add_edge(edge); } + GraphNode *GraphShard::find_node(uint64_t id) { if (node_location.find(id) == node_location.end()) return NULL; return *(node_location[id]); } + int32_t GraphTable::load(const std::string &path, const std::string ¶m) { + auto cmd = paddle::string::split_string(param, "|"); + std::set cmd_set(cmd.begin(), cmd.end()); + bool load_edge = cmd_set.count(std::string("edge")); + bool reverse_edge = cmd_set.count(std::string("reverse")); + VLOG(0) << "Reverse Edge " << reverse_edge; + auto paths = paddle::string::split_string(path, ";"); VLOG(0) << paths.size(); + int count = 0; + for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { auto values = paddle::string::split_string(line, "\t"); + count++; if (values.size() < 2) continue; - auto id = std::stoull(values[0]); - size_t shard_id = id % shard_num; - if (shard_id >= shard_end || shard_id < shard_start) { - VLOG(0) << "will not load " << id << " from " << path + auto src_id = std::stoull(values[0]); + auto dst_id = std::stoull(values[1]); + if (reverse_edge) { + std::swap(src_id, dst_id); + } + float weight = 0; + if (values.size() == 3) { + weight = std::stof(values[2]); + } + size_t src_shard_id = src_id % shard_num; + + if (src_shard_id >= shard_end || src_shard_id < shard_start) { + VLOG(0) << "will not load " << src_id << " from " << path << ", please check id distribution"; continue; } - size_t index = shard_id - shard_start; - // GraphNodeType type = GraphNode::get_graph_node_type(values[1]); - // VLOG(0)<<"shards's size = "<get_id(); - for (size_t i = 2; i < values.size(); i++) { - auto edge_arr = - paddle::string::split_string(values[i], ";"); - if (edge_arr.size() == 2) { - // VLOG(0)<<"edge content "<::iterator iter = bucket[i].begin(); - while (iter != bucket[i].end()) { - auto node = *iter; - node->build_sampler(); - iter++; - } + } + VLOG(0) << "Load Finished Total Edge Count " << count; + + // Build Sampler j + for (auto &shard : shards) { + auto bucket = shard.get_bucket(); + for (int i = 0; i < bucket.size(); i++) { + std::list::iterator iter = bucket[i].begin(); + while (iter != bucket[i].end()) { + auto node = *iter; + node->build_sampler(); + iter++; } } } @@ -132,8 +147,6 @@ GraphNode *GraphTable::find_node(uint64_t id) { return NULL; } size_t index = shard_id - shard_start; - // VLOG(0)<<"try to find node-id "<enqueue([&]() -> int { + GraphNode *node = find_node(node_id); if (node == NULL) { actual_size = 0; @@ -153,48 +167,22 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, } std::vector res = node->sample_k(sample_size); std::vector node_list; - int total_size = 0; - for (auto x : res) { - GraphNode temp; - temp.set_id(x->id); - total_size += temp.get_size(); - node_list.push_back(temp); - } - buffer = new char[total_size]; - int index = 0; - for (auto x : node_list) { - x.to_buffer(buffer + index); - index += x.get_size(); + actual_size = + res.size() * (GraphNode::id_size + GraphNode::weight_size); + buffer = new char[actual_size]; + int offset = 0; + uint64_t id; + float weight; + for (auto &x : res) { + id = x->get_id(); + weight = x->get_weight(); + memcpy(buffer + offset, &id, GraphNode::id_size); + offset += GraphNode::id_size; + memcpy(buffer + offset, &weight, GraphNode::weight_size); + offset += GraphNode::weight_size; } - actual_size = total_size; - return 0; - }).get(); - //.get(); - // GraphNode *node = find_node(node_id, type); - // if (node == NULL) { - // actual_size = 0; - // rwlock_->UNLock(); - // return 0; - // } - // std::vector res = node->sample_k(sample_size); - // std::vector node_list; - // int total_size = 0; - // for (auto x : res) { - // GraphNode temp; - // temp.set_id(x->id); - // temp.set_graph_node_type(x->type); - // total_size += temp.get_size(); - // node_list.push_back(temp); - // } - // buffer = new char[total_size]; - // int index = 0; - // for (auto x : node_list) { - // x.to_buffer(buffer + index); - // index += x.get_size(); - // } - // actual_size = total_size; - // rwlock_->UNLock(); - // return 0; + }) + .get(); } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 78a586d507ef4..c63fff8883636 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -16,9 +16,8 @@ #include namespace paddle { namespace distributed { -int GraphNode::enum_size = sizeof(int); +int GraphNode::weight_size = sizeof(float); int GraphNode::id_size = sizeof(uint64_t); -int GraphNode::double_size = sizeof(double); int GraphNode::int_size = sizeof(int); int GraphNode::get_size() { return feature.size() + id_size + int_size; } void GraphNode::build_sampler() { diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 218d14e01edc1..a8fe5eca3e824 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -20,11 +20,13 @@ namespace distributed { // enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; class GraphEdge : public WeightedObject { public: - double weight; - uint64_t id; // GraphNodeType type; GraphEdge() {} - GraphEdge(uint64_t id, double weight) : weight(weight), id(id) {} + GraphEdge(uint64_t id, float weight) : id(id), weight(weight) {} + uint64_t get_id() { return id; } + float get_weight() { return weight; } + uint64_t id; + float weight; }; class GraphNode { public: @@ -35,7 +37,7 @@ class GraphNode { : id(id), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} std::vector get_graph_edge() { return edges; } - static int enum_size, id_size, int_size, double_size; + static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } // GraphNodeType get_graph_node_type() { return type; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index c93bc551f54f3..09ecdc2b642e4 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -13,6 +13,7 @@ // limitations under the License. #include "paddle/fluid/distributed/table/weighted_sampler.h" +#include namespace paddle { namespace distributed { void WeightedSampler::build(WeightedObject **v, int start, int end) { @@ -37,11 +38,11 @@ std::vector WeightedSampler::sample_k(int k) { k = count; } std::vector sample_result; - double subtract; - std::unordered_map subtract_weight_map; + float subtract; + std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; while (k--) { - double query_weight = rand() % 100000 / 100000.0; + float query_weight = rand() % 100000 / 100000.0; query_weight *= weight - subtract_weight_map[this]; sample_result.push_back(sample(query_weight, subtract_weight_map, subtract_count_map, subtract)); @@ -49,10 +50,10 @@ std::vector WeightedSampler::sample_k(int k) { return sample_result; } WeightedObject *WeightedSampler::sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract) { + float &subtract) { if (left == NULL) { subtract_weight_map[this] = weight; subtract = weight; @@ -61,7 +62,7 @@ WeightedObject *WeightedSampler::sample( } int left_count = left->count - subtract_count_map[left]; int right_count = right->count - subtract_count_map[right]; - double left_subtract = subtract_weight_map[left]; + float left_subtract = subtract_weight_map[left]; WeightedObject *return_id; if (right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight) { diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 53bfaa8d30119..9ed2cc04649de 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -22,15 +22,8 @@ class WeightedObject { public: WeightedObject() {} virtual ~WeightedObject() {} - virtual unsigned long long get_id() { return id; } - virtual double get_weight() { return weight; } - - virtual void set_id(unsigned long long id) { this->id = id; } - virtual void set_weight(double weight) { this->weight = weight; } - - private: - unsigned long long id; - double weight; + virtual uint64_t get_id() = 0; + virtual float get_weight() = 0; }; class WeightedSampler { @@ -38,16 +31,16 @@ class WeightedSampler { WeightedSampler *left, *right; WeightedObject *object; int count; - double weight; + float weight; void build(WeightedObject **v, int start, int end); std::vector sample_k(int k); private: WeightedObject *sample( - double query_weight, - std::unordered_map &subtract_weight_map, + float query_weight, + std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, - double &subtract); + float &subtract); }; } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index e522af9910ec5..8921d35bb8ccb 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -18,6 +18,7 @@ limitations under the License. */ #include #include #include // NOLINT +#include #include #include "google/protobuf/text_format.h" @@ -49,10 +50,18 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testGraphToBuffer(); -std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), - std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), - std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), - std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; +// std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), +// std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), +// std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), +// std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; + +std::string nodes[] = { + std::string("37\t45\t0.34"), std::string("37\t145\t0.31"), + std::string("37\t112\t0.21"), std::string("96\t48\t1.4"), + std::string("96\t247\t0.31"), std::string("96\t111\t1.21"), + std::string("59\t45\t0.34"), std::string("59\t145\t0.31"), + std::string("59\t122\t0.21"), std::string("97\t48\t0.34"), + std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -210,8 +219,8 @@ void RunBrpcPushSparse() { worker_ptr_->load(0, std::string(file_name), std::string("")); pull_status.wait(); - std::vector v; std::vector > vs; + std::vector> v; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); // for (auto g : v) { @@ -221,43 +230,51 @@ void RunBrpcPushSparse() { v.clear(); pull_status = worker_ptr_->sample(0, 96, 4, v); pull_status.wait(); + std::unordered_set s = { 111, 48, 247 }; + ASSERT_EQ(3, v.size()); for (auto g : v) { - std::cout << g.get_id() << std::endl; + // std::cout << g.first << std::endl; + ASSERT_EQ(true, s.find(g.first) != s.end()); } - // ASSERT_EQ(v.size(),3); v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, v); + std::vector nodes; + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 37); + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 37); // for (auto g : v) { // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; // } // ASSERT_EQ(v.size(),1); - v.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, v); + nodes.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); pull_status.wait(); - ASSERT_EQ(v.size(), 1); - ASSERT_EQ(v[0].get_id(), 59); - for (auto g : v) { + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 59); + for (auto g : nodes) { std::cout << g.get_id() << std::endl; } distributed::GraphPyService gps1, gps2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; - gps1.set_up(ips_str, 127, 0, 0, 0); - gps2.set_up(ips_str, 127, 1, 1, 0); - gps1.load_file(std::string(file_name)); - v.clear(); - v = gps2.pull_graph_list(0, 1, 4); - ASSERT_EQ(v[0].get_id(), 59); - v.clear(); - v = gps2.sample_k(96, 4); + std::vector edge_types = {std::string("user2item")}; + gps1.set_up(ips_str, 127, 0, 0, edge_types); + gps2.set_up(ips_str, 127, 1, 1, edge_types); + gps1.load_edge_file(std::string("user2item"), std::string(file_name), 0); + nodes.clear(); + nodes = gps2.pull_graph_list(std::string("user2item"), 0, 1, 4); + ASSERT_EQ(nodes[0].get_id(), 59); + nodes.clear(); + v = gps2.sample_k(std::string("user2item"), 96, 4); ASSERT_EQ(v.size(), 3); + std::cout << "sample result" << std::endl; + for (auto p : v) { + std::cout << p.first << " " << p.second << std::endl; + } std::vector node_ids; node_ids.push_back(96); node_ids.push_back(37); - vs = gps2.batch_sample_k(node_ids, 4); + vs = gps2.batch_sample_k(std::string("user2item"), node_ids, 4); ASSERT_EQ(vs.size(), 2); // to test in python,try this: // from paddle.fluid.core import GraphPyService diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 6b86da9664897..ef0535293e438 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -165,7 +165,8 @@ void BindGraphNode(py::module* m) { void BindGraphService(py::module* m) { py::class_(*m, "GraphPyService") .def(py::init<>()) - .def("load_file", &GraphPyService::load_file) + .def("load_edge_file", &GraphPyService::load_edge_file) + .def("load_node_file", &GraphPyService::load_node_file) .def("set_up", &GraphPyService::set_up) .def("pull_graph_list", &GraphPyService::pull_graph_list) .def("sample_k", &GraphPyService::sample_k) From abb0ee0407312d39b7022367ddbeda06c8041b28 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 15:48:54 +0800 Subject: [PATCH 024/106] fix --- .../distributed/service/graph_brpc_client.cc | 16 ++++++++++------ .../distributed/service/graph_brpc_client.h | 3 ++- .../distributed/service/graph_py_service.cc | 10 ++++++++++ .../fluid/distributed/service/graph_py_service.h | 2 ++ paddle/fluid/distributed/service/ps_client.h | 2 +- .../fluid/distributed/table/common_graph_table.h | 2 +- paddle/fluid/distributed/test/graph_node_test.cc | 4 ++-- paddle/fluid/pybind/fleet_py.cc | 9 ++------- 8 files changed, 30 insertions(+), 18 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 616d310f07fbc..1c9583e440bb9 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -97,7 +97,7 @@ std::future GraphBrpcClient::sample( std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, - std::vector > &res) { + std::vector > > &res) { std::vector request2server; std::vector server2request(server_size, -1); @@ -107,7 +107,8 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, server2request[server_index] = request2server.size(); request2server.push_back(server_index); } - res.push_back(std::vector()); + //res.push_back(std::vector()); + res.push_back(std::vector>()); } size_t request_call_num = request2server.size(); std::vector > node_id_buckets(request_call_num); @@ -148,10 +149,13 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, int actual_size = actual_sizes[node_idx]; int start = 0; while (start < actual_size) { - GraphNode node; - node.recover_from_buffer(node_buffer + offset + start); - start += node.get_size(); - res[query_idx].push_back(node); + //GraphNode node; + //node.recover_from_buffer(node_buffer + offset + start); + //start += node.get_size(); + //res[query_idx].push_back(node); + res[query_idx].push_back({*(uint64_t *)(node_buffer + offset + start), + *(float *)(node_buffer + offset + start + GraphNode::id_size)}); + start += GraphNode::id_size + GraphNode::weight_size; } offset += actual_size; } diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 4778d5261219f..9bc7505a7009d 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -41,7 +41,8 @@ class GraphBrpcClient : public BrpcPsClient { std::vector> &res); virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, - std::vector > &res); + std::vector>> &res); + //std::vector > &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 2df3843c96afe..64a572677962a 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -193,6 +193,16 @@ std::vector> GraphPyClient::sample_k( } return v; } +std::vector > > GraphPyClient::batch_sample_k( + std::string name, std::vector node_ids, int sample_size) { + std::vector > > v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = worker_ptr->batch_sample(table_id, node_ids, sample_size, v); + status.wait(); + } + return v; +} std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, int start, int size) { diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 56b47d71093f6..8e3764fe62c46 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -122,6 +122,8 @@ class GraphPyClient : public GraphPyService { std::vector> sample_k(std::string name, uint64_t node_id, int sample_size); + std::vector > > batch_sample_k( + std::string name, std::vector node_ids, int sample_size); std::vector pull_graph_list(std::string name, int server_index, int start, int size); ::paddle::distributed::PSParameter GetWorkerProto(); diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 78ef708ffe8b4..21f0ef51aa76d 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -166,7 +166,7 @@ class PSClient { } virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, - std::vector > &res) { + std::vector>> &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 18fdfdb5d8f66..68f265685b9d9 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -73,7 +73,7 @@ class GraphTable : public SparseTable { int &actual_size); //virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, //int &actual_size); - int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 491d72655f762..aeaa32656bae1 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -219,7 +219,7 @@ void RunBrpcPushSparse() { worker_ptr_->load(0, std::string(file_name), std::string("")); pull_status.wait(); - std::vector > vs; + std::vector > > vs; std::vector> v; pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status.wait(); @@ -286,7 +286,7 @@ void RunBrpcPushSparse() { std::vector node_ids; node_ids.push_back(96); node_ids.push_back(37); - vs = gps2.batch_sample_k(std::string("user2item"), node_ids, 4); + vs = client1.batch_sample_k(std::string("user2item"), node_ids, 4); ASSERT_EQ(vs.size(), 2); // to test in python,try this: // from paddle.fluid.core import GraphPyService diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 086bb9ca161df..5dbb8cdd56d69 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -176,18 +176,13 @@ void BindGraphPyServer(py::module* m) { void BindGraphPyClient(py::module* m) { py::class_(*m, "GraphPyClient") .def(py::init<>()) - .def("load_edge_file", &GraphPyService::load_edge_file) - .def("load_node_file", &GraphPyService::load_node_file) - .def("set_up", &GraphPyService::set_up) - .def("pull_graph_list", &GraphPyService::pull_graph_list) - .def("sample_k", &GraphPyService::sample_k) .def("load_edge_file", &GraphPyClient::load_edge_file) .def("load_node_file", &GraphPyClient::load_node_file) .def("set_up", &GraphPyClient::set_up) .def("pull_graph_list", &GraphPyClient::pull_graph_list) .def("sample_k", &GraphPyClient::sample_k) - .def("start_client", &GraphPyClient::start_client); - .def("batch_sample_k", &GraphPyService::batch_sample_k); + .def("start_client", &GraphPyClient::start_client) + .def("batch_sample_k", &GraphPyClient::batch_sample_k); } } // end namespace pybind From 2feadfee4f06622af692e5f9a1c2da05bd748a46 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 15 Mar 2021 17:39:30 +0800 Subject: [PATCH 025/106] resolved conflict --- paddle/fluid/distributed/table/common_graph_table.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index a352c24c383df..107c619235ad1 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -69,7 +69,7 @@ std::list::iterator GraphShard::add_node(uint64_t id, std::string f return node_location.find(id)->second; int index = id % shard_num % bucket_size; - GraphNode *node = new GraphNode(id, std::string("")); + GraphNode *node = new GraphNode(id, feature); std::list::iterator iter = bucket[index].insert(bucket[index].end(), node); @@ -127,7 +127,7 @@ int32_t GraphTable::load_nodes(const std::string &path) { } auto feat = paddle::string::join_strings(feature, '\t'); size_t index = shard_id - shard_start; - shards[index].add_node(id, std::string("")); + shards[index].add_node(id, feat); } } From dbfa6ac2a72b43b88538f61caa58d09a39eeba32 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 15 Mar 2021 17:45:39 +0800 Subject: [PATCH 026/106] fixed segment fault; high-level VLOG for load edges and load nodes --- paddle/fluid/distributed/table/common_graph_table.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 107c619235ad1..520377944b609 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -113,7 +113,7 @@ int32_t GraphTable::load_nodes(const std::string &path) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { - VLOG(0) << "will not load " << id << " from " << path + VLOG(4) << "will not load " << id << " from " << path << ", please check id distribution"; continue; @@ -160,7 +160,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { size_t src_shard_id = src_id % shard_num; if (src_shard_id >= shard_end || src_shard_id < shard_start) { - VLOG(0) << "will not load " << src_id << " from " << path + VLOG(4) << "will not load " << src_id << " from " << path << ", please check id distribution"; continue; } @@ -226,6 +226,7 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, memcpy(buffer + offset, &weight, GraphNode::weight_size); offset += GraphNode::weight_size; } + return 0; }) .get(); } From 583e8de7e431f0512b0e4398f45db4eb713032f5 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 22:06:41 +0800 Subject: [PATCH 027/106] random_sample return 0 --- paddle/fluid/distributed/table/common_graph_table.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 7a848925d3a7a..aa011bbfcbf2e 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -228,6 +228,7 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, memcpy(buffer + offset, &weight, GraphNode::weight_size); offset += GraphNode::weight_size; } + return 0; }) .get(); } From 32558de1c1e40c7ff952f911802af0d1f748287d Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 22:11:19 +0800 Subject: [PATCH 028/106] rm useless loop --- paddle/fluid/distributed/service/graph_brpc_client.cc | 4 ---- 1 file changed, 4 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 1c9583e440bb9..65c012c060ddd 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -120,10 +120,6 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, query_idx_buckets[request_idx].push_back(query_idx); } - for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ - - } - DownpourBrpcClosure *closure = new DownpourBrpcClosure(request_call_num, [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; From 8643616c3b9be19a3ae5f0537d6b17176ae381c2 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 22:36:31 +0800 Subject: [PATCH 029/106] test:load edge --- paddle/fluid/distributed/test/graph_node_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index aeaa32656bae1..b419d3ae06356 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -216,7 +216,7 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ auto pull_status = - worker_ptr_->load(0, std::string(file_name), std::string("")); + worker_ptr_->load(0, std::string(file_name), std::string("edge")); pull_status.wait(); std::vector > > vs; From a21d599d24fad623b7568b25724471bf6c68bf6a Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 22:52:44 +0800 Subject: [PATCH 030/106] fix ret -1 --- paddle/fluid/distributed/service/graph_brpc_client.cc | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 65c012c060ddd..3cda11c6a36d8 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -123,9 +123,10 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, DownpourBrpcClosure *closure = new DownpourBrpcClosure(request_call_num, [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; + int fail_num = 0; for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ if (closure->check_response(request_idx, PS_GRAPH_SAMPLE) != 0) { - ret = -1; + ++fail_num; } else { VLOG(0) << "check sample response: " << " " << closure->check_response(request_idx, PS_GRAPH_SAMPLE); @@ -145,10 +146,6 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, int actual_size = actual_sizes[node_idx]; int start = 0; while (start < actual_size) { - //GraphNode node; - //node.recover_from_buffer(node_buffer + offset + start); - //start += node.get_size(); - //res[query_idx].push_back(node); res[query_idx].push_back({*(uint64_t *)(node_buffer + offset + start), *(float *)(node_buffer + offset + start + GraphNode::id_size)}); start += GraphNode::id_size + GraphNode::weight_size; @@ -156,6 +153,9 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, offset += actual_size; } } + if (fail_num == request_call_num){ + ret = -1; + } } closure->set_promise_value(ret); }); From dd14309a6ac7ff99526eebac14a3ab4a97ffef6e Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 23:06:56 +0800 Subject: [PATCH 031/106] test: rm sample --- .../distributed/service/graph_brpc_client.cc | 1 + .../distributed/service/graph_brpc_client.h | 1 - .../fluid/distributed/test/graph_node_test.cc | 26 ++++++++++--------- 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 3cda11c6a36d8..f30c8fad08c35 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -101,6 +101,7 @@ std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector request2server; std::vector server2request(server_size, -1); + res.clear(); for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ int server_index = get_server_index_by_id(node_ids[query_idx]); if(server2request[server_index] == -1){ diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 9bc7505a7009d..01eb11a393c1a 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -42,7 +42,6 @@ class GraphBrpcClient : public BrpcPsClient { virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector>> &res); - //std::vector > &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index b419d3ae06356..f10507c71147a 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -220,20 +220,22 @@ void RunBrpcPushSparse() { pull_status.wait(); std::vector > > vs; - std::vector> v; - pull_status = worker_ptr_->sample(0, 37, 4, v); + //std::vector> v; + //pull_status = worker_ptr_->sample(0, 37, 4, v); + pull_status = worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); pull_status.wait(); - ASSERT_EQ(v.size(), 3); - v.clear(); - pull_status = worker_ptr_->sample(0, 96, 4, v); + ASSERT_EQ(vs[0].size(), 3); + vs.clear(); + //pull_status = worker_ptr_->sample(0, 96, 4, v); + pull_status = worker_ptr_->batch_sample(0, std::vector(1, 96), 4, vs); pull_status.wait(); std::unordered_set s = {111, 48, 247}; - ASSERT_EQ(3, v.size()); - for (auto g : v) { + ASSERT_EQ(3, vs[0].size()); + for (auto g : vs[0]) { // std::cout << g.first << std::endl; ASSERT_EQ(true, s.find(g.first) != s.end()); } - v.clear(); + vs.clear(); std::vector nodes; pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); pull_status.wait(); @@ -277,10 +279,10 @@ void RunBrpcPushSparse() { nodes = client2.pull_graph_list(std::string("user2item"), 0, 1, 4); ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); - v = client1.sample_k(std::string("user2item"), 96, 4); - ASSERT_EQ(v.size(), 3); - std::cout << "sample result" << std::endl; - for (auto p : v) { + vs = client1.batch_sample_k(std::string("user2item"), std::vector(1, 96), 4); + ASSERT_EQ(vs[0].size(), 3); + std::cout << "batch sample result" << std::endl; + for (auto p : vs[0]) { std::cout << p.first << " " << p.second << std::endl; } std::vector node_ids; From 86ff4d9f1641b78ecefe872341a55e8049e63a84 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Mon, 15 Mar 2021 23:12:33 +0800 Subject: [PATCH 032/106] rm sample --- .../distributed/service/graph_brpc_client.cc | 60 ------------------- .../distributed/service/graph_brpc_client.h | 3 - .../distributed/service/graph_py_service.cc | 10 ---- .../distributed/service/graph_py_service.h | 3 - paddle/fluid/distributed/service/ps_client.h | 9 --- paddle/fluid/pybind/fleet_py.cc | 1 - 6 files changed, 86 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index f30c8fad08c35..b3c5540265a23 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -35,66 +35,6 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::sample( - uint32_t table_id, uint64_t node_id, int sample_size, - std::vector> &res) { - int server_index = get_server_index_by_id(node_id); - DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - if (closure->check_response(0, PS_GRAPH_SAMPLE) != 0) { - ret = -1; - } else { - auto &res_io_buffer = closure->cntl(0)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - - size_t node_num = *(size_t *)buffer; - int *actual_sizes = (int *)(buffer + sizeof(size_t)); - char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; - - int offset = 0; - for (size_t idx = 0; idx < node_num; ++idx){ - int actual_size = actual_sizes[idx]; - int start = 0; - while (start < actual_size) { - //GraphNode node; - //node.recover_from_buffer(node_buffer + offset + start); - //start += node.get_size(); - //res.push_back(node); - res.push_back({*(uint64_t *)(node_buffer + offset + start), - *(float *)(node_buffer + offset + start + GraphNode::id_size)}); - start += GraphNode::id_size + GraphNode::weight_size; - } - offset += actual_size; - } - } - closure->set_promise_value(ret); - }); - auto promise = std::make_shared>(); - closure->add_promise(promise); - std::future fut = promise->get_future(); - closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE); - closure->request(0)->set_table_id(table_id); - closure->request(0)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); - std::vector node_ids; - node_ids.push_back(node_id); - size_t node_num = node_ids.size(); - - closure->request(0)->add_params((char *)node_ids.data(), sizeof(uint64_t)*node_num); - //closure->request(0)->add_params((char *)&node_id, sizeof(uint64_t)); - closure->request(0)->add_params((char *)&sample_size, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); - closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), - closure); - - return fut; -} - std::future GraphBrpcClient::batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector > > &res) { diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 01eb11a393c1a..3cd1d3eb753e3 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -36,9 +36,6 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} - virtual std::future sample( - uint32_t table_id, uint64_t node_id, int sample_size, - std::vector> &res); virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector>> &res); diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 64a572677962a..3ba35fda0bf39 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -183,16 +183,6 @@ void GraphPyClient::load_node_file(std::string name, std::string filepath) { status.wait(); } } -std::vector> GraphPyClient::sample_k( - std::string name, uint64_t node_id, int sample_size) { - std::vector> v; - if (this->table_id_map.count(name)) { - uint32_t table_id = this->table_id_map[name]; - auto status = worker_ptr->sample(table_id, node_id, sample_size, v); - status.wait(); - } - return v; -} std::vector > > GraphPyClient::batch_sample_k( std::string name, std::vector node_ids, int sample_size) { std::vector > > v; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 8e3764fe62c46..8f6c9f0ad0b64 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -119,9 +119,6 @@ class GraphPyClient : public GraphPyService { int get_client_id() { return client_id; } void set_client_id(int client_id) { this->client_id = client_id; } void start_client(); - std::vector> sample_k(std::string name, - uint64_t node_id, - int sample_size); std::vector > > batch_sample_k( std::string name, std::vector node_ids, int sample_size); std::vector pull_graph_list(std::string name, int server_index, diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 21f0ef51aa76d..00ef4c5c8caef 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,15 +155,6 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future sample( - uint32_t table_id, uint64_t node_id, int sample_size, - std::vector> &res) { - LOG(FATAL) << "Did not implement"; - std::promise promise; - std::future fut = promise.get_future(); - promise.set_value(-1); - return fut; - } virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, int sample_size, std::vector>> &res) { diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 5dbb8cdd56d69..57e47461caf68 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -180,7 +180,6 @@ void BindGraphPyClient(py::module* m) { .def("load_node_file", &GraphPyClient::load_node_file) .def("set_up", &GraphPyClient::set_up) .def("pull_graph_list", &GraphPyClient::pull_graph_list) - .def("sample_k", &GraphPyClient::sample_k) .def("start_client", &GraphPyClient::start_client) .def("batch_sample_k", &GraphPyClient::batch_sample_k); } From 7a14737b912d7ced6d4ab64c6b7e0d5fb9727d7e Mon Sep 17 00:00:00 2001 From: suweiyue Date: Tue, 16 Mar 2021 11:02:44 +0800 Subject: [PATCH 033/106] random_sample return future --- .../distributed/service/graph_brpc_server.cc | 20 +++++++++---------- .../distributed/table/common_graph_table.cc | 7 ++----- .../distributed/table/common_graph_table.h | 4 +--- paddle/fluid/distributed/table/table.h | 8 ++------ 4 files changed, 15 insertions(+), 24 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index bf926d7fbe52c..4a5b5e648675a 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -287,24 +287,24 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); - std::vector*> tasks; - std::vector buffers(num_nodes); + std::vector> tasks; + std::vector buffers(num_nodes, nullptr); std::vector actual_sizes(num_nodes); for (size_t idx = 0; idx < num_nodes; ++idx){ - //std::future task = table->random_sample(node_data[idx], sample_size, - //buffers[idx], actual_sizes[idx]); - table->random_sample(node_data[idx], sample_size, - buffers[idx], actual_sizes[idx]); - //tasks.push_back(&task); + tasks.push_back(table->random_sample(node_data[idx], sample_size, + buffers[idx], actual_sizes[idx])); + } + for (size_t idx = 0; idx < num_nodes; ++idx){ + tasks[idx].get(); } - //for (size_t idx = 0; idx < num_nodes; ++idx){ - //tasks[idx]->get(); - //} cntl->response_attachment().append(&num_nodes, sizeof(size_t)); cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*num_nodes); for (size_t idx = 0; idx < num_nodes; ++idx){ cntl->response_attachment().append(buffers[idx], actual_sizes[idx]); + if (buffers[idx] != nullptr){ + delete buffers[idx]; + } } return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index aa011bbfcbf2e..dd460286ca8ed 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -200,9 +200,7 @@ GraphNode *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num_per_table % task_pool_size_; } -//std::future GraphTable::random_sample(uint64_t node_id, int sample_size, - //char *&buffer, int &actual_size) { -int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, +std::future GraphTable::random_sample(uint64_t node_id, int sample_size, char *&buffer, int &actual_size) { return _shards_task_pool[get_thread_pool_index(node_id)] ->enqueue([&]() -> int { @@ -229,8 +227,7 @@ int32_t GraphTable::random_sample(uint64_t node_id, int sample_size, offset += GraphNode::weight_size; } return 0; - }) - .get(); + }); } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index c1607a5125396..dd1deca745fa2 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -71,9 +71,7 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); - //virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, - //int &actual_size); - virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size); virtual int32_t initialize(); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index d9b6835a31f29..b389eb3acae98 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -93,14 +93,10 @@ class Table { return 0; } // only for graph table - virtual int32_t random_sample(uint64_t node_id, int sampe_size, char *&buffer, + virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, int &actual_size) { - return 0; + return std::future(); } - //virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, - //int &actual_size) { - //return std::future(); - //} virtual int32_t pour() { return 0; } virtual void clear() = 0; From b6cc9f0b66b3514d8e7f32144fd8a4384640136b Mon Sep 17 00:00:00 2001 From: suweiyue Date: Tue, 16 Mar 2021 13:12:28 +0800 Subject: [PATCH 034/106] random_sample return int --- .../distributed/service/graph_brpc_server.cc | 22 +++++++------------ .../distributed/table/common_graph_table.cc | 20 ++++++++++++----- .../distributed/table/common_graph_table.h | 4 ++-- paddle/fluid/distributed/table/table.h | 6 ++--- 4 files changed, 28 insertions(+), 24 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 4a5b5e648675a..abf1d02b6d41c 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -283,27 +283,21 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, "graph_random_sample request requires at least 2 arguments"); return 0; } - size_t num_nodes = request.params(0).size() / sizeof(uint64_t); + size_t node_num = request.params(0).size() / sizeof(uint64_t); uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); - std::vector> tasks; - std::vector buffers(num_nodes, nullptr); - std::vector actual_sizes(num_nodes); + std::vector buffers(node_num, nullptr); + std::vector actual_sizes(node_num, 0); + table->random_sample(node_data, sample_size, buffers, actual_sizes); - for (size_t idx = 0; idx < num_nodes; ++idx){ - tasks.push_back(table->random_sample(node_data[idx], sample_size, - buffers[idx], actual_sizes[idx])); - } - for (size_t idx = 0; idx < num_nodes; ++idx){ - tasks[idx].get(); - } - cntl->response_attachment().append(&num_nodes, sizeof(size_t)); - cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*num_nodes); - for (size_t idx = 0; idx < num_nodes; ++idx){ + cntl->response_attachment().append(&node_num, sizeof(size_t)); + cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*node_num); + for (size_t idx = 0; idx < node_num; ++idx){ cntl->response_attachment().append(buffers[idx], actual_sizes[idx]); if (buffers[idx] != nullptr){ delete buffers[idx]; + buffers[idx] = nullptr; } } return 0; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index dd460286ca8ed..d53b68df5d4a9 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -200,11 +200,16 @@ GraphNode *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num_per_table % task_pool_size_; } -std::future GraphTable::random_sample(uint64_t node_id, int sample_size, - char *&buffer, int &actual_size) { - return _shards_task_pool[get_thread_pool_index(node_id)] +int GraphTable::random_sample(uint64_t* node_ids, int sample_size, + std::vector& buffers, std::vector &actual_sizes) { + size_t node_num = buffers.size(); + std::vector> tasks; + for (size_t idx = 0; idx < node_num; ++idx){ + uint64_t node_id = node_ids[idx]; + char* & buffer = buffers[idx]; + int& actual_size = actual_sizes[idx]; + tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)] ->enqueue([&]() -> int { - GraphNode *node = find_node(node_id); if (node == NULL) { actual_size = 0; @@ -227,7 +232,12 @@ std::future GraphTable::random_sample(uint64_t node_id, int sample_size, offset += GraphNode::weight_size; } return 0; - }); + })); + } + for (size_t idx = 0; idx < node_num; ++idx){ + tasks[idx].get(); + } + return 0; } int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, int &actual_size) { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index dd1deca745fa2..4b1dc0a36e392 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -71,8 +71,8 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, char *&buffer, int &actual_size); - virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, - int &actual_size); + virtual int random_sample(uint64_t* node_ids, int sampe_size, std::vector&buffers, + std::vector &actual_sizes); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index b389eb3acae98..91ffc47eb183e 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -93,9 +93,9 @@ class Table { return 0; } // only for graph table - virtual std::future random_sample(uint64_t node_id, int sampe_size, char *&buffer, - int &actual_size) { - return std::future(); + virtual int random_sample(uint64_t* node_ids, int sampe_size, std::vector&buffers, + std::vector &actual_sizes) { + return 0; } virtual int32_t pour() { return 0; } From 2a70bd8c5781fef6a45d2a38f5e891949b0bf4a6 Mon Sep 17 00:00:00 2001 From: suweiyue Date: Tue, 16 Mar 2021 13:14:51 +0800 Subject: [PATCH 035/106] test fake node --- paddle/fluid/distributed/test/graph_node_test.cc | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index f10507c71147a..9fcb34810859f 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -236,6 +236,11 @@ void RunBrpcPushSparse() { ASSERT_EQ(true, s.find(g.first) != s.end()); } vs.clear(); + + pull_status = worker_ptr_->batch_sample(0, std::vector(1, 10240001024), 4, vs); + pull_status.wait(); + ASSERT_EQ(0, vs[0].size()); + std::vector nodes; pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); pull_status.wait(); From 6a28d023168719ad004428422ba4f0e8704bad22 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Tue, 16 Mar 2021 14:10:28 +0800 Subject: [PATCH 036/106] fixed here --- paddle/fluid/distributed/test/graph_node_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 9d0aa046db101..771314176e704 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -216,7 +216,7 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ auto pull_status = - worker_ptr_->load(0, std::string(file_name), std::string("")); + worker_ptr_->load(0, std::string(file_name), std::string("edge")); pull_status.wait(); std::vector> v; From 1dfc021f739b8a36e88bc5fd8f8fe7e3d8ca671a Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 16 Mar 2021 11:29:22 +0000 Subject: [PATCH 037/106] memory leak --- .../distributed/service/graph_brpc_client.cc | 121 ++++++++++-------- .../distributed/service/graph_brpc_server.cc | 28 ++-- .../distributed/table/common_graph_table.cc | 96 +++++++------- .../distributed/table/common_graph_table.h | 11 +- paddle/fluid/distributed/table/table.h | 6 +- .../fluid/distributed/test/graph_node_test.cc | 28 ++-- 6 files changed, 160 insertions(+), 130 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index b3c5540265a23..d83f675912d17 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -35,90 +35,99 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::batch_sample(uint32_t table_id, - std::vector node_ids, int sample_size, - std::vector > > &res) { - +std::future GraphBrpcClient::batch_sample( + uint32_t table_id, std::vector node_ids, int sample_size, + std::vector>> &res) { std::vector request2server; std::vector server2request(server_size, -1); res.clear(); - for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { int server_index = get_server_index_by_id(node_ids[query_idx]); - if(server2request[server_index] == -1){ + if (server2request[server_index] == -1) { server2request[server_index] = request2server.size(); request2server.push_back(server_index); } - //res.push_back(std::vector()); + // res.push_back(std::vector()); res.push_back(std::vector>()); } size_t request_call_num = request2server.size(); - std::vector > node_id_buckets(request_call_num); - std::vector > query_idx_buckets(request_call_num); - for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx){ + std::vector> node_id_buckets(request_call_num); + std::vector> query_idx_buckets(request_call_num); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { int server_index = get_server_index_by_id(node_ids[query_idx]); int request_idx = server2request[server_index]; node_id_buckets[request_idx].push_back(node_ids[query_idx]); query_idx_buckets[request_idx].push_back(query_idx); } - DownpourBrpcClosure *closure = new DownpourBrpcClosure(request_call_num, [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { - int ret = 0; - auto *closure = (DownpourBrpcClosure *)done; - int fail_num = 0; - for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ - if (closure->check_response(request_idx, PS_GRAPH_SAMPLE) != 0) { - ++fail_num; - } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(request_idx, PS_GRAPH_SAMPLE); - auto &res_io_buffer = closure->cntl(request_idx)->response_attachment(); - butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); - size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + request_call_num, + [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + int fail_num = 0; + for (int request_idx = 0; request_idx < request_call_num; + ++request_idx) { + if (closure->check_response(request_idx, PS_GRAPH_SAMPLE) != 0) { + ++fail_num; + } else { + auto &res_io_buffer = + closure->cntl(request_idx)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + // char buffer[bytes_size]; + std::unique_ptr buffer_wrapper(new char[bytes_size]); + char *buffer = buffer_wrapper.get(); + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); - size_t node_num = *(size_t *)buffer; - int *actual_sizes = (int *)(buffer + sizeof(size_t)); - char *node_buffer = buffer + sizeof(size_t) + sizeof(int) * node_num; - - int offset = 0; - for (size_t node_idx = 0; node_idx < node_num; ++node_idx){ - int query_idx = query_idx_buckets.at(request_idx).at(node_idx); - int actual_size = actual_sizes[node_idx]; - int start = 0; - while (start < actual_size) { - res[query_idx].push_back({*(uint64_t *)(node_buffer + offset + start), - *(float *)(node_buffer + offset + start + GraphNode::id_size)}); - start += GraphNode::id_size + GraphNode::weight_size; + size_t node_num = *(size_t *)buffer; + int *actual_sizes = (int *)(buffer + sizeof(size_t)); + char *node_buffer = + buffer + sizeof(size_t) + sizeof(int) * node_num; + + int offset = 0; + for (size_t node_idx = 0; node_idx < node_num; ++node_idx) { + int query_idx = query_idx_buckets.at(request_idx).at(node_idx); + int actual_size = actual_sizes[node_idx]; + int start = 0; + while (start < actual_size) { + res[query_idx].push_back( + {*(uint64_t *)(node_buffer + offset + start), + *(float *)(node_buffer + offset + start + + GraphNode::id_size)}); + start += GraphNode::id_size + GraphNode::weight_size; + } + offset += actual_size; + } + } + if (fail_num == request_call_num) { + ret = -1; } - offset += actual_size; } - } - if (fail_num == request_call_num){ - ret = -1; - } - } - closure->set_promise_value(ret); - }); + closure->set_promise_value(ret); + }); auto promise = std::make_shared>(); closure->add_promise(promise); std::future fut = promise->get_future(); - - for (int request_idx = 0; request_idx < request_call_num; ++request_idx){ + + for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { int server_index = request2server[request_idx]; closure->request(request_idx)->set_cmd_id(PS_GRAPH_SAMPLE); closure->request(request_idx)->set_table_id(table_id); closure->request(request_idx)->set_client_id(_client_id); // std::string type_str = GraphNode::node_type_to_string(type); size_t node_num = node_id_buckets[request_idx].size(); - - closure->request(request_idx)->add_params((char *)node_id_buckets[request_idx].data(), sizeof(uint64_t)*node_num); - closure->request(request_idx)->add_params((char *)&sample_size, sizeof(int)); + + closure->request(request_idx) + ->add_params((char *)node_id_buckets[request_idx].data(), + sizeof(uint64_t) * node_num); + closure->request(request_idx) + ->add_params((char *)&sample_size, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); - rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), closure->response(request_idx), - closure); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), + closure->response(request_idx), closure); } return fut; @@ -133,12 +142,12 @@ std::future GraphBrpcClient::pull_graph_list( if (closure->check_response(0, PS_PULL_GRAPH_LIST) != 0) { ret = -1; } else { - VLOG(0) << "check sample response: " - << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); + // VLOG(0) << "check sample response: " + // << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); - char *buffer = new char[bytes_size]; + char buffer[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); int index = 0; while (index < bytes_size) { diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index abf1d02b6d41c..cccd1c0d1b2ac 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -265,11 +265,10 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, } int start = *(int *)(request.params(0).c_str()); int size = *(int *)(request.params(1).c_str()); - std::vector res_data; - char *buffer; + std::unique_ptr buffer; int actual_size; table->pull_graph_list(start, size, buffer, actual_size); - cntl->response_attachment().append(buffer, actual_size); + cntl->response_attachment().append(buffer.get(), actual_size); return 0; } int32_t GraphBrpcService::graph_random_sample(Table *table, @@ -287,19 +286,26 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); - std::vector buffers(node_num, nullptr); + std::vector> buffers(node_num); std::vector actual_sizes(node_num, 0); table->random_sample(node_data, sample_size, buffers, actual_sizes); cntl->response_attachment().append(&node_num, sizeof(size_t)); - cntl->response_attachment().append(actual_sizes.data(), sizeof(int)*node_num); - for (size_t idx = 0; idx < node_num; ++idx){ - cntl->response_attachment().append(buffers[idx], actual_sizes[idx]); - if (buffers[idx] != nullptr){ - delete buffers[idx]; - buffers[idx] = nullptr; - } + cntl->response_attachment().append(actual_sizes.data(), + sizeof(int) * node_num); + for (size_t idx = 0; idx < node_num; ++idx) { + cntl->response_attachment().append(buffers[idx].get(), actual_sizes[idx]); + // if (buffers[idx] != nullptr){ + // delete buffers[idx]; + // buffers[idx] = nullptr; + // } } + // ======= + // std::unique_ptr buffer; + // int actual_size; + // table->random_sample(node_id, sample_size, buffer, actual_size); + // cntl->response_attachment().append(buffer.get(), actual_size); + // >>>>>>> Stashed changes return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index d53b68df5d4a9..7253d1cf94436 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -64,7 +64,8 @@ size_t GraphShard::get_size() { return res; } -std::list::iterator GraphShard::add_node(uint64_t id, std::string feature) { +std::list::iterator GraphShard::add_node(uint64_t id, + std::string feature) { if (node_location.find(id) != node_location.end()) return node_location.find(id)->second; @@ -89,14 +90,13 @@ GraphNode *GraphShard::find_node(uint64_t id) { int32_t GraphTable::load(const std::string &path, const std::string ¶m) { auto cmd = paddle::string::split_string(param, "|"); - std::set cmd_set(cmd.begin(), cmd.end()); + std::set cmd_set(cmd.begin(), cmd.end()); bool reverse_edge = cmd_set.count(std::string("reverse")); bool load_edge = cmd_set.count(std::string("edge")); - if(load_edge) { - return this -> load_edges(path, reverse_edge); - } - else { - return this -> load_nodes(path); + if (load_edge) { + return this->load_edges(path, reverse_edge); + } else { + return this->load_nodes(path); } } @@ -110,33 +110,28 @@ int32_t GraphTable::load_nodes(const std::string &path) { if (values.size() < 2) continue; auto id = std::stoull(values[1]); - size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { VLOG(0) << "will not load " << id << " from " << path << ", please check id distribution"; continue; - } std::string node_type = values[0]; - std::vector feature; + std::vector feature; feature.push_back(node_type); - for(size_t slice = 2; slice < values.size(); slice ++) { + for (size_t slice = 2; slice < values.size(); slice++) { feature.push_back(values[slice]); } auto feat = paddle::string::join_strings(feature, '\t'); size_t index = shard_id - shard_start; shards[index].add_node(id, feat); - } } return 0; } - int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { - auto paths = paddle::string::split_string(path, ";"); int count = 0; @@ -173,7 +168,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { VLOG(0) << "Load Finished Total Edge Count " << count; // Build Sampler j - + for (auto &shard : shards) { auto bucket = shard.get_bucket(); for (int i = 0; i < bucket.size(); i++) { @@ -200,46 +195,49 @@ GraphNode *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num_per_table % task_pool_size_; } -int GraphTable::random_sample(uint64_t* node_ids, int sample_size, - std::vector& buffers, std::vector &actual_sizes) { +int GraphTable::random_sample(uint64_t *node_ids, int sample_size, + std::vector> &buffers, + std::vector &actual_sizes) { size_t node_num = buffers.size(); std::vector> tasks; - for (size_t idx = 0; idx < node_num; ++idx){ + for (size_t idx = 0; idx < node_num; ++idx) { uint64_t node_id = node_ids[idx]; - char* & buffer = buffers[idx]; - int& actual_size = actual_sizes[idx]; - tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)] - ->enqueue([&]() -> int { - GraphNode *node = find_node(node_id); - if (node == NULL) { - actual_size = 0; + std::unique_ptr &buffer = buffers[idx]; + int &actual_size = actual_sizes[idx]; + tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( + [&]() -> int { + GraphNode *node = find_node(node_id); + if (node == NULL) { + actual_size = 0; + return 0; + } + std::vector res = node->sample_k(sample_size); + actual_size = + res.size() * (GraphNode::id_size + GraphNode::weight_size); + int offset = 0; + uint64_t id; + float weight; + char *buffer_addr = new char[actual_size]; + buffer.reset(buffer_addr); + for (auto &x : res) { + id = x->get_id(); + weight = x->get_weight(); + memcpy(buffer_addr + offset, &id, GraphNode::id_size); + offset += GraphNode::id_size; + memcpy(buffer_addr + offset, &weight, GraphNode::weight_size); + offset += GraphNode::weight_size; + return 0; + } return 0; - } - std::vector res = node->sample_k(sample_size); - std::vector node_list; - actual_size = - res.size() * (GraphNode::id_size + GraphNode::weight_size); - buffer = new char[actual_size]; - int offset = 0; - uint64_t id; - float weight; - for (auto &x : res) { - id = x->get_id(); - weight = x->get_weight(); - memcpy(buffer + offset, &id, GraphNode::id_size); - offset += GraphNode::id_size; - memcpy(buffer + offset, &weight, GraphNode::weight_size); - offset += GraphNode::weight_size; - } - return 0; - })); + })); } - for (size_t idx = 0; idx < node_num; ++idx){ + for (size_t idx = 0; idx < node_num; ++idx) { tasks[idx].get(); } return 0; } -int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, +int32_t GraphTable::pull_graph_list(int start, int total_size, + std::unique_ptr &buffer, int &actual_size) { if (start < 0) start = 0; int size = 0, cur_size; @@ -283,11 +281,12 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, char *&buffer, size += res.back()[j]->get_size(); } } - buffer = new char[size]; + char *buffer_addr = new char[size]; + buffer.reset(buffer_addr); int index = 0; for (size_t i = 0; i < res.size(); i++) { for (size_t j = 0; j < res[i].size(); j++) { - res[i][j]->to_buffer(buffer + index); + res[i][j]->to_buffer(buffer_addr + index); index += res[i][j]->get_size(); } } @@ -321,4 +320,3 @@ int32_t GraphTable::initialize() { } } }; - diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 4b1dc0a36e392..ab74e761d443c 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -67,12 +67,16 @@ class GraphShard { }; class GraphTable : public SparseTable { public: - GraphTable() { rwlock_.reset(new framework::RWLock); } + GraphTable() {} virtual ~GraphTable() {} - virtual int32_t pull_graph_list(int start, int size, char *&buffer, + virtual int32_t pull_graph_list(int start, int size, + std::unique_ptr &buffer, int &actual_size); - virtual int random_sample(uint64_t* node_ids, int sampe_size, std::vector&buffers, + + virtual int32_t random_sample(uint64_t *node_ids, int sample_size, + std::vector> &buffers, std::vector &actual_sizes); + virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); @@ -103,7 +107,6 @@ class GraphTable : public SparseTable { protected: std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; - std::unique_ptr rwlock_{nullptr}; const int task_pool_size_ = 11; std::vector> _shards_task_pool; }; diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 91ffc47eb183e..d3424c82da308 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -88,12 +88,14 @@ class Table { } // only for graph table - virtual int32_t pull_graph_list(int start, int total_size, char *&buffer, + virtual int32_t pull_graph_list(int start, int total_size, + std::unique_ptr &buffer, int &actual_size) { return 0; } // only for graph table - virtual int random_sample(uint64_t* node_ids, int sampe_size, std::vector&buffers, + virtual int32_t random_sample(uint64_t *node_ids, int sample_size, + std::vector> &buffers, std::vector &actual_sizes) { return 0; } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 9fcb34810859f..190b4ead50803 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -219,15 +219,25 @@ void RunBrpcPushSparse() { worker_ptr_->load(0, std::string(file_name), std::string("edge")); pull_status.wait(); - std::vector > > vs; - //std::vector> v; - //pull_status = worker_ptr_->sample(0, 37, 4, v); - pull_status = worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); + std::vector>> vs; + for (int i = 0; i < 100000000; i++) { + std::vector nodes; + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); + pull_status.wait(); + pull_status = + worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); + pull_status.wait(); + } + // std::vector> v; + // pull_status = worker_ptr_->sample(0, 37, 4, v); + pull_status = + worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); pull_status.wait(); ASSERT_EQ(vs[0].size(), 3); vs.clear(); - //pull_status = worker_ptr_->sample(0, 96, 4, v); - pull_status = worker_ptr_->batch_sample(0, std::vector(1, 96), 4, vs); + // pull_status = worker_ptr_->sample(0, 96, 4, v); + pull_status = + worker_ptr_->batch_sample(0, std::vector(1, 96), 4, vs); pull_status.wait(); std::unordered_set s = {111, 48, 247}; ASSERT_EQ(3, vs[0].size()); @@ -237,7 +247,8 @@ void RunBrpcPushSparse() { } vs.clear(); - pull_status = worker_ptr_->batch_sample(0, std::vector(1, 10240001024), 4, vs); + pull_status = worker_ptr_->batch_sample( + 0, std::vector(1, 10240001024), 4, vs); pull_status.wait(); ASSERT_EQ(0, vs[0].size()); @@ -284,7 +295,8 @@ void RunBrpcPushSparse() { nodes = client2.pull_graph_list(std::string("user2item"), 0, 1, 4); ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); - vs = client1.batch_sample_k(std::string("user2item"), std::vector(1, 96), 4); + vs = client1.batch_sample_k(std::string("user2item"), + std::vector(1, 96), 4); ASSERT_EQ(vs[0].size(), 3); std::cout << "batch sample result" << std::endl; for (auto p : vs[0]) { From 832cab847e260f0109e3b07cf6e3a15b8231a5ee Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 16 Mar 2021 11:33:12 +0000 Subject: [PATCH 038/106] remove test code --- paddle/fluid/distributed/test/graph_node_test.cc | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 190b4ead50803..192cc64027bef 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -220,14 +220,14 @@ void RunBrpcPushSparse() { pull_status.wait(); std::vector>> vs; - for (int i = 0; i < 100000000; i++) { - std::vector nodes; - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); - pull_status.wait(); - pull_status = - worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); - pull_status.wait(); - } + // for(int i = 0;i < 100000000;i++){ + // std::vector nodes; + // pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); + // pull_status.wait(); + // pull_status = worker_ptr_->batch_sample(0, std::vector(1, 37), 4, + // vs); + // pull_status.wait(); + // } // std::vector> v; // pull_status = worker_ptr_->sample(0, 37, 4, v); pull_status = From bf41ef9c1b00ef0e0b0ee0251a6dad0dec87112b Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 17 Mar 2021 08:25:31 +0000 Subject: [PATCH 039/106] fix return problem --- .../distributed/service/graph_brpc_client.cc | 2 +- .../distributed/service/graph_brpc_server.cc | 10 ---------- .../distributed/table/common_graph_table.cc | 7 +++---- paddle/fluid/distributed/table/graph_node.cc | 16 +++++++++++----- paddle/fluid/distributed/table/graph_node.h | 4 ++-- paddle/fluid/distributed/test/graph_node_test.cc | 4 ++-- 6 files changed, 19 insertions(+), 24 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index d83f675912d17..1191eda3de1e3 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -153,7 +153,7 @@ std::future GraphBrpcClient::pull_graph_list( while (index < bytes_size) { GraphNode node; node.recover_from_buffer(buffer + index); - index += node.get_size(); + index += node.get_size(true); res.push_back(node); } } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index cccd1c0d1b2ac..aace2975495b0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -295,17 +295,7 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, sizeof(int) * node_num); for (size_t idx = 0; idx < node_num; ++idx) { cntl->response_attachment().append(buffers[idx].get(), actual_sizes[idx]); - // if (buffers[idx] != nullptr){ - // delete buffers[idx]; - // buffers[idx] = nullptr; - // } } - // ======= - // std::unique_ptr buffer; - // int actual_size; - // table->random_sample(node_id, sample_size, buffer, actual_size); - // cntl->response_attachment().append(buffer.get(), actual_size); - // >>>>>>> Stashed changes return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 7253d1cf94436..b5bb1af26b1ec 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -226,7 +226,6 @@ int GraphTable::random_sample(uint64_t *node_ids, int sample_size, offset += GraphNode::id_size; memcpy(buffer_addr + offset, &weight, GraphNode::weight_size); offset += GraphNode::weight_size; - return 0; } return 0; })); @@ -278,7 +277,7 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, for (size_t i = 0; i < tasks.size(); i++) { res.push_back(tasks[i].get()); for (size_t j = 0; j < res.back().size(); j++) { - size += res.back()[j]->get_size(); + size += res.back()[j]->get_size(true); } } char *buffer_addr = new char[size]; @@ -286,8 +285,8 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, int index = 0; for (size_t i = 0; i < res.size(); i++) { for (size_t j = 0; j < res[i].size(); j++) { - res[i][j]->to_buffer(buffer_addr + index); - index += res[i][j]->get_size(); + res[i][j]->to_buffer(buffer_addr + index, true); + index += res[i][j]->get_size(true); } } actual_size = size; diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index c63fff8883636..8408ea7aeafed 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -19,17 +19,23 @@ namespace distributed { int GraphNode::weight_size = sizeof(float); int GraphNode::id_size = sizeof(uint64_t); int GraphNode::int_size = sizeof(int); -int GraphNode::get_size() { return feature.size() + id_size + int_size; } +int GraphNode::get_size(bool need_feature) { + return id_size + int_size + (need_feature ? feature.size() : 0); +} void GraphNode::build_sampler() { sampler = new WeightedSampler(); GraphEdge** arr = edges.data(); sampler->build((WeightedObject**)arr, 0, edges.size()); } -void GraphNode::to_buffer(char* buffer) { - int size = get_size(); +void GraphNode::to_buffer(char* buffer, bool need_feature) { + int size = get_size(need_feature); memcpy(buffer, &size, int_size); - memcpy(buffer + int_size, feature.c_str(), feature.size()); - memcpy(buffer + int_size + feature.size(), &id, id_size); + if (need_feature) { + memcpy(buffer + int_size, feature.c_str(), feature.size()); + memcpy(buffer + int_size + feature.size(), &id, id_size); + } else { + memcpy(buffer + int_size, &id, id_size); + } } void GraphNode::recover_from_buffer(char* buffer) { int size; diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index a8fe5eca3e824..24ee17fd698f9 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -44,9 +44,9 @@ class GraphNode { // void set_graph_node_type(GraphNodeType type) { this->type = type; } void set_feature(std::string feature) { this->feature = feature; } std::string get_feature() { return feature; } - virtual int get_size(); + virtual int get_size(bool need_feature); virtual void build_sampler(); - virtual void to_buffer(char *buffer); + virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } std::vector sample_k(int k) { diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 192cc64027bef..f9d790f6658cc 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -338,9 +338,9 @@ void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; s.set_feature("hhhh"); s.set_id(65); - int size = s.get_size(); + int size = s.get_size(true); char str[size]; - s.to_buffer(str); + s.to_buffer(str, true); s1.recover_from_buffer(str); ASSERT_EQ(s.get_id(), s1.get_id()); VLOG(0) << s.get_feature(); From ff54377573d314c27a6c31bd7796e06dbb307d12 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 18 Mar 2021 11:29:59 +0800 Subject: [PATCH 040/106] add common_graph_table --- paddle/fluid/distributed/table/common_graph_table.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 129a8d0cd9805..3a31d49190711 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -226,7 +226,6 @@ int GraphTable::random_sample(uint64_t *node_ids, int sample_size, offset += GraphNode::id_size; memcpy(buffer_addr + offset, &weight, GraphNode::weight_size); offset += GraphNode::weight_size; - return 0; } return 0; })); From efeded968ee086d7d199d83fe593d97c5e010657 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 18 Mar 2021 04:49:17 +0000 Subject: [PATCH 041/106] random sample node &test & change data-structure from linkedList to vector --- .../distributed/service/graph_brpc_client.cc | 51 +++++- .../distributed/service/graph_brpc_client.h | 10 +- .../distributed/service/graph_brpc_server.cc | 31 +++- .../distributed/service/graph_brpc_server.h | 12 +- .../distributed/service/graph_py_service.cc | 10 +- paddle/fluid/distributed/service/ps_client.h | 17 +- .../fluid/distributed/service/sendrecv.proto | 3 +- .../distributed/table/common_graph_table.cc | 173 +++++++++++------- .../distributed/table/common_graph_table.h | 59 +++--- paddle/fluid/distributed/table/graph_node.h | 5 - paddle/fluid/distributed/table/table.h | 15 +- .../fluid/distributed/test/graph_node_test.cc | 85 ++++++--- 12 files changed, 328 insertions(+), 143 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 1191eda3de1e3..0514c170f1575 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -35,7 +35,7 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { return id % shard_num / shard_per_server; } // char* &buffer,int &actual_size -std::future GraphBrpcClient::batch_sample( +std::future GraphBrpcClient::batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, std::vector>> &res) { std::vector request2server; @@ -68,7 +68,8 @@ std::future GraphBrpcClient::batch_sample( int fail_num = 0; for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { - if (closure->check_response(request_idx, PS_GRAPH_SAMPLE) != 0) { + if (closure->check_response(request_idx, + PS_GRAPH_SAMPLE_NEIGHBOORS) != 0) { ++fail_num; } else { auto &res_io_buffer = @@ -113,7 +114,7 @@ std::future GraphBrpcClient::batch_sample( for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { int server_index = request2server[request_idx]; - closure->request(request_idx)->set_cmd_id(PS_GRAPH_SAMPLE); + closure->request(request_idx)->set_cmd_id(PS_GRAPH_SAMPLE_NEIGHBOORS); closure->request(request_idx)->set_table_id(table_id); closure->request(request_idx)->set_client_id(_client_id); // std::string type_str = GraphNode::node_type_to_string(type); @@ -132,7 +133,49 @@ std::future GraphBrpcClient::batch_sample( return fut; } - +std::future GraphBrpcClient::random_sample_nodes( + uint32_t table_id, int server_index, int sample_size, + std::vector &ids) { + DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + if (closure->check_response(0, PS_GRAPH_SAMPLE_NODES) != 0) { + ret = -1; + } else { + // VLOG(0) << "check sample response: " + // << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); + auto &res_io_buffer = closure->cntl(0)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + char buffer[bytes_size]; + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + int index = 0; + while (index < bytes_size) { + // GraphNode node; + // node.recover_from_buffer(buffer + index); + // index += node.get_size(true); + // res.push_back(node); + ids.push_back(*(uint64_t *)(buffer + index)); + index += GraphNode::id_size; + } + } + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + ; + closure->request(0)->set_cmd_id(PS_GRAPH_SAMPLE_NODES); + closure->request(0)->set_table_id(table_id); + closure->request(0)->set_client_id(_client_id); + closure->request(0)->add_params((char *)&sample_size, sizeof(int)); + ; + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), + closure); + return fut; +} std::future GraphBrpcClient::pull_graph_list( uint32_t table_id, int server_index, int start, int size, std::vector &res) { diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 3cd1d3eb753e3..8f14e646050c3 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -36,13 +36,17 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} - virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, - int sample_size, - std::vector>> &res); + virtual std::future batch_sample_neighboors( + uint32_t table_id, std::vector node_ids, int sample_size, + std::vector>> &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, std::vector &res); + virtual std::future random_sample_nodes(uint32_t table_id, + int server_index, + int sample_size, + std::vector &ids); virtual int32_t initialize(); int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index aace2975495b0..cd1c2330a7b85 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -95,8 +95,10 @@ int32_t GraphBrpcService::initialize() { _service_handler_map[PS_STOP_PROFILER] = &GraphBrpcService::stop_profiler; _service_handler_map[PS_PULL_GRAPH_LIST] = &GraphBrpcService::pull_graph_list; - _service_handler_map[PS_GRAPH_SAMPLE] = - &GraphBrpcService::graph_random_sample; + _service_handler_map[PS_GRAPH_SAMPLE_NEIGHBOORS] = + &GraphBrpcService::graph_random_sample_neighboors; + _service_handler_map[PS_GRAPH_SAMPLE_NODES] = + &GraphBrpcService::graph_random_sample_nodes; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); @@ -267,14 +269,13 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, int size = *(int *)(request.params(1).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size); + table->pull_graph_list(start, size, buffer, actual_size, true); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } -int32_t GraphBrpcService::graph_random_sample(Table *table, - const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl) { +int32_t GraphBrpcService::graph_random_sample_neighboors( + Table *table, const PsRequestMessage &request, PsResponseMessage &response, + brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) if (request.params_size() < 2) { set_response_code( @@ -285,10 +286,10 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, size_t node_num = request.params(0).size() / sizeof(uint64_t); uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); int sample_size = *(uint64_t *)(request.params(1).c_str()); - std::vector> buffers(node_num); std::vector actual_sizes(node_num, 0); - table->random_sample(node_data, sample_size, buffers, actual_sizes); + table->random_sample_neighboors(node_data, sample_size, buffers, + actual_sizes); cntl->response_attachment().append(&node_num, sizeof(size_t)); cntl->response_attachment().append(actual_sizes.data(), @@ -298,6 +299,18 @@ int32_t GraphBrpcService::graph_random_sample(Table *table, } return 0; } +int32_t GraphBrpcService::graph_random_sample_nodes( + Table *table, const PsRequestMessage &request, PsResponseMessage &response, + brpc::Controller *cntl) { + size_t size = *(uint64_t *)(request.params(0).c_str()); + std::unique_ptr buffer; + int actual_size; + if (table->random_sample_nodes(size, buffer, actual_size) == 0) { + cntl->response_attachment().append(buffer.get(), actual_size); + } else + cntl->response_attachment().append(NULL, 0); + return 0; +} } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index b4056692f2ccf..bb054081257b4 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -79,9 +79,14 @@ class GraphBrpcService : public PsBaseService { int32_t initialize_shard_info(); int32_t pull_graph_list(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); - int32_t graph_random_sample(Table *table, const PsRequestMessage &request, - PsResponseMessage &response, - brpc::Controller *cntl); + int32_t graph_random_sample_neighboors(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); + int32_t graph_random_sample_nodes(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, @@ -103,6 +108,7 @@ class GraphBrpcService : public PsBaseService { std::mutex _initialize_shard_mutex; std::unordered_map _msg_handler_map; std::vector _ori_values; + const int sample_nodes_ranges = 3; }; // class GraphBrpcService : public BrpcPsService { // public: diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 3ba35fda0bf39..db5aaa947577a 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -183,12 +183,14 @@ void GraphPyClient::load_node_file(std::string name, std::string filepath) { status.wait(); } } -std::vector > > GraphPyClient::batch_sample_k( - std::string name, std::vector node_ids, int sample_size) { - std::vector > > v; +std::vector>> +GraphPyClient::batch_sample_k(std::string name, std::vector node_ids, + int sample_size) { + std::vector>> v; if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; - auto status = worker_ptr->batch_sample(table_id, node_ids, sample_size, v); + auto status = + worker_ptr->batch_sample_neighboors(table_id, node_ids, sample_size, v); status.wait(); } return v; diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 00ef4c5c8caef..d3274b4a57334 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,9 +155,9 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future batch_sample(uint32_t table_id, std::vector node_ids, - int sample_size, - std::vector>> &res) { + virtual std::future batch_sample_neighboors( + uint32_t table_id, std::vector node_ids, int sample_size, + std::vector>> &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); @@ -174,6 +174,17 @@ class PSClient { promise.set_value(-1); return fut; } + + virtual std::future random_sample_nodes(uint32_t table_id, + int server_index, + int sample_size, + std::vector &ids) { + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 52c2d7104b544..185024fe18721 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -49,7 +49,8 @@ enum PsCmdID { PS_STOP_PROFILER = 28; PS_PUSH_GLOBAL_STEP = 29; PS_PULL_GRAPH_LIST = 30; - PS_GRAPH_SAMPLE = 31; + PS_GRAPH_SAMPLE_NEIGHBOORS = 31; + PS_GRAPH_SAMPLE_NODES = 32; } message PsRequestMessage { diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index b5bb1af26b1ec..70c43e7099099 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -23,69 +23,32 @@ namespace paddle { namespace distributed { -int GraphShard::bucket_low_bound = 11; - std::vector GraphShard::get_batch(int start, int total_size) { if (start < 0) start = 0; - int size = 0, cur_size; std::vector res; - if (total_size <= 0) return res; - for (int i = 0; i < bucket_size; i++) { - cur_size = bucket[i].size(); - if (size + cur_size <= start) { - size += cur_size; - continue; - } - int read = 0; - std::list::iterator iter = bucket[i].begin(); - while (size + read < start) { - iter++; - read++; - } - read = 0; - while (iter != bucket[i].end() && read < total_size) { - res.push_back(*iter); - iter++; - read++; - } - if (read == total_size) break; - size += cur_size; - start = size; - total_size -= read; + for (int pos = start; pos < start + total_size; pos++) { + res.push_back(bucket[pos]); } return res; } -size_t GraphShard::get_size() { - size_t res = 0; - for (int i = 0; i < bucket_size; i++) { - res += bucket[i].size(); - } - return res; -} +size_t GraphShard::get_size() { return bucket.size(); } -std::list::iterator GraphShard::add_node(uint64_t id, - std::string feature) { +GraphNode *GraphShard::add_node(uint64_t id, std::string feature) { if (node_location.find(id) != node_location.end()) - return node_location.find(id)->second; - - int index = id % shard_num % bucket_size; - GraphNode *node = new GraphNode(id, feature); - - std::list::iterator iter = - bucket[index].insert(bucket[index].end(), node); - - node_location[id] = iter; - return iter; + return bucket[node_location[id]]; + node_location[id] = bucket.size(); + bucket.push_back(new GraphNode(id, feature)); + return bucket.back(); } void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { - (*add_node(id, std::string("")))->add_edge(edge); + add_node(id, std::string(""))->add_edge(edge); } GraphNode *GraphShard::find_node(uint64_t id) { - if (node_location.find(id) == node_location.end()) return NULL; - return *(node_location[id]); + auto iter = node_location.find(id); + return iter == node_location.end() ? NULL : bucket[iter->second]; } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { @@ -100,6 +63,42 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { } } +int32_t GraphTable::get_nodes_ids_by_ranges( + std::vector> ranges, std::vector res) { + int start = 0, end, index = 0, total_size = 0; + res.clear(); + std::vector>> tasks; + for (int i = 0; i < shards.size() && index < ranges.size(); i++) { + end = total_size + shards[i].get_size(); + start = total_size; + while (start < end && index < ranges.size()) { + if (ranges[index].second <= start) + index++; + else if (ranges[index].first >= end) { + break; + } else { + int first = std::max(ranges[index].first, start); + int second = std::min(ranges[index].second, end); + start = second; + first -= total_size; + second -= total_size; + index++; + tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( + [this, first, second, i]() -> std::vector { + return shards[i].get_ids_by_range(first, second); + })); + } + } + } + for (int i = 0; i < tasks.size(); i++) { + auto vec = tasks[i].get(); + for (auto &id : vec) { + res.push_back(id); + std::swap(res[rand() % res.size()], res[(int)res.size() - 1]); + } + } + return 0; +} int32_t GraphTable::load_nodes(const std::string &path) { auto paths = paddle::string::split_string(path, ";"); for (auto path : paths) { @@ -172,12 +171,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { for (auto &shard : shards) { auto bucket = shard.get_bucket(); for (int i = 0; i < bucket.size(); i++) { - std::list::iterator iter = bucket[i].begin(); - while (iter != bucket[i].end()) { - auto node = *iter; - node->build_sampler(); - iter++; - } + bucket[i]->build_sampler(); } } return 0; @@ -193,20 +187,73 @@ GraphNode *GraphTable::find_node(uint64_t id) { return node; } uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { - return node_id % shard_num_per_table % task_pool_size_; + return node_id % shard_num % shard_num_per_table % task_pool_size_; } -int GraphTable::random_sample(uint64_t *node_ids, int sample_size, - std::vector> &buffers, - std::vector &actual_sizes) { +int32_t GraphTable::random_sample_nodes(int sample_size, + std::unique_ptr &buffer, + int &actual_size, bool need_feature) { + int total_size = 0; + for (int i = 0; i < shards.size(); i++) { + total_size += shards[i].get_size(); + } + if (sample_size > total_size) sample_size = total_size; + int range_num = random_sample_nodes_ranges; + if (range_num > sample_size) range_num = sample_size; + std::vector ranges_len, ranges_pos; + int remain = sample_size, last_pos = -1, num; + std::set separator_set; + for (int i = 0; i < range_num - 1; i++) { + while (separator_set.find(num = rand() % (sample_size - 1)) != + separator_set.end()) + ; + separator_set.insert(num); + } + for (auto p : separator_set) { + ranges_len.push_back(p - last_pos); + last_pos = p; + } + ranges_len.push_back(sample_size - 1 - last_pos); + remain = total_size - sample_size + range_num; + separator_set.clear(); + for (int i = 0; i < range_num; i++) { + while (separator_set.find(num = rand() % remain) != separator_set.end()) + ; + separator_set.insert(num); + } + int used = 0, index = 0; + last_pos = -1; + for (auto p : separator_set) { + used += p - last_pos - 1; + last_pos = p; + ranges_pos.push_back(used); + used += ranges_len[index++]; + } + std::vector> vec; + for (int i = 0; i < ranges_len.size() && i < ranges_pos.size(); i++) { + vec.push_back({ranges_pos[i], ranges_len[i]}); + } + std::vector res; + get_nodes_ids_by_ranges(vec, res); + actual_size = res.size() * (GraphNode::id_size); + buffer.reset(new char[actual_size]); + char *pointer = buffer.get(); + memcpy(pointer, res.data(), actual_size); + return 0; +} +int GraphTable::random_sample_neighboors( + uint64_t *node_ids, int sample_size, + std::vector> &buffers, + std::vector &actual_sizes) { size_t node_num = buffers.size(); std::vector> tasks; for (size_t idx = 0; idx < node_num; ++idx) { - uint64_t node_id = node_ids[idx]; + uint64_t &node_id = node_ids[idx]; std::unique_ptr &buffer = buffers[idx]; int &actual_size = actual_sizes[idx]; tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( [&]() -> int { GraphNode *node = find_node(node_id); + if (node == NULL) { actual_size = 0; return 0; @@ -237,7 +284,7 @@ int GraphTable::random_sample(uint64_t *node_ids, int sample_size, } int32_t GraphTable::pull_graph_list(int start, int total_size, std::unique_ptr &buffer, - int &actual_size) { + int &actual_size, bool need_feature) { if (start < 0) start = 0; int size = 0, cur_size; if (total_size <= 0) { @@ -277,7 +324,7 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, for (size_t i = 0; i < tasks.size(); i++) { res.push_back(tasks[i].get()); for (size_t j = 0; j < res.back().size(); j++) { - size += res.back()[j]->get_size(true); + size += res.back()[j]->get_size(need_feature); } } char *buffer_addr = new char[size]; @@ -285,8 +332,8 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, int index = 0; for (size_t i = 0; i < res.size(); i++) { for (size_t j = 0; j < res[i].size(); j++) { - res[i][j]->to_buffer(buffer_addr + index, true); - index += res[i][j]->get_size(true); + res[i][j]->to_buffer(buffer_addr + index, need_feature); + index += res[i][j]->get_size(need_feature); } } actual_size = size; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index ab74e761d443c..9e39bbe842a0d 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -32,38 +32,45 @@ namespace paddle { namespace distributed { class GraphShard { public: - static int bucket_low_bound; - static int gcd(int s, int t) { - if (s % t == 0) return t; - return gcd(t, s % t); - } + // static int bucket_low_bound; + // static int gcd(int s, int t) { + // if (s % t == 0) return t; + // return gcd(t, s % t); + // } size_t get_size(); GraphShard() {} GraphShard(int shard_num) { this->shard_num = shard_num; - bucket_size = init_bucket_size(shard_num); - bucket.resize(bucket_size); + // bucket_size = init_bucket_size(shard_num); + // bucket.resize(bucket_size); } - std::vector> &get_bucket() { return bucket; } + std::vector &get_bucket() { return bucket; } std::vector get_batch(int start, int total_size); - int init_bucket_size(int shard_num) { - for (int i = bucket_low_bound;; i++) { - if (gcd(i, shard_num) == 1) return i; + // int init_bucket_size(int shard_num) { + // for (int i = bucket_low_bound;; i++) { + // if (gcd(i, shard_num) == 1) return i; + // } + // return -1; + // } + std::vector get_ids_by_range(int start, int end) { + std::vector res; + for (int i = start; i < end && i < bucket.size(); i++) { + res.push_back(bucket[i]->get_id()); } - return -1; + return res; } - std::list::iterator add_node(uint64_t id, std::string feature); + GraphNode *add_node(uint64_t id, std::string feature); GraphNode *find_node(uint64_t id); void add_neighboor(uint64_t id, GraphEdge *edge); - std::unordered_map::iterator> - get_node_location() { + // std::unordered_map::iterator> + std::unordered_map get_node_location() { return node_location; } private: - std::unordered_map::iterator> node_location; - int bucket_size, shard_num; - std::vector> bucket; + std::unordered_map node_location; + int shard_num; + std::vector bucket; }; class GraphTable : public SparseTable { public: @@ -71,12 +78,19 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, std::unique_ptr &buffer, - int &actual_size); + int &actual_size, bool need_feature); + + virtual int32_t random_sample_neighboors( + uint64_t *node_ids, int sample_size, + std::vector> &buffers, + std::vector &actual_sizes); - virtual int32_t random_sample(uint64_t *node_ids, int sample_size, - std::vector> &buffers, - std::vector &actual_sizes); + virtual int32_t random_sample_nodes(int sample_size, + std::unique_ptr &buffers, + int &actual_sizes, bool need_feature); + virtual int32_t get_nodes_ids_by_ranges( + std::vector> ranges, std::vector res); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); @@ -108,6 +122,7 @@ class GraphTable : public SparseTable { std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; const int task_pool_size_ = 11; + const int random_sample_nodes_ranges = 3; std::vector> _shards_task_pool; }; } diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 24ee17fd698f9..dbd387b2d4519 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -20,7 +20,6 @@ namespace distributed { // enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; class GraphEdge : public WeightedObject { public: - // GraphNodeType type; GraphEdge() {} GraphEdge(uint64_t id, float weight) : id(id), weight(weight) {} uint64_t get_id() { return id; } @@ -31,8 +30,6 @@ class GraphEdge : public WeightedObject { class GraphNode { public: GraphNode() { sampler = NULL; } - // GraphNode(uint64_t id, GraphNodeType type, std::string feature) - // : id(id), type(type), feature(feature), sampler(NULL) {} GraphNode(uint64_t id, std::string feature) : id(id), feature(feature), sampler(NULL) {} virtual ~GraphNode() {} @@ -40,8 +37,6 @@ class GraphNode { static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } - // GraphNodeType get_graph_node_type() { return type; } - // void set_graph_node_type(GraphNodeType type) { this->type = type; } void set_feature(std::string feature) { this->feature = feature; } std::string get_feature() { return feature; } virtual int get_size(bool need_feature); diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index d3424c82da308..65a3191c50255 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -90,13 +90,20 @@ class Table { // only for graph table virtual int32_t pull_graph_list(int start, int total_size, std::unique_ptr &buffer, - int &actual_size) { + int &actual_size, bool need_feature) { return 0; } // only for graph table - virtual int32_t random_sample(uint64_t *node_ids, int sample_size, - std::vector> &buffers, - std::vector &actual_sizes) { + virtual int32_t random_sample_neighboors( + uint64_t *node_ids, int sample_size, + std::vector> &buffers, + std::vector &actual_sizes) { + return 0; + } + + virtual int32_t random_sample_nodes(int sample_size, + std::unique_ptr &buffers, + int &actual_sizes) { return 0; } virtual int32_t pour() { return 0; } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index f9d790f6658cc..0b1a1b71c213e 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -49,6 +49,65 @@ namespace math = paddle::operators::math; namespace memory = paddle::memory; namespace distributed = paddle::distributed; +void testSingleSampleNeighboor( + std::shared_ptr& worker_ptr_) { + std::vector>> vs; + auto pull_status = worker_ptr_->batch_sample_neighboors( + 0, std::vector(1, 37), 4, vs); + pull_status.wait(); + + std::unordered_set s; + std::unordered_set s1 = {112, 45, 145}; + for (auto g : vs[0]) { + s.insert(g.first); + } + ASSERT_EQ(s.size(), 3); + for (auto g : s) { + ASSERT_EQ(true, s1.find(g) != s1.end()); + } + s.clear(); + s1.clear(); + vs.clear(); + pull_status = worker_ptr_->batch_sample_neighboors( + 0, std::vector(1, 96), 4, vs); + pull_status.wait(); + s1 = {111, 48, 247}; + for (auto g : vs[0]) { + s.insert(g.first); + } + ASSERT_EQ(s.size(), 3); + for (auto g : s) { + ASSERT_EQ(true, s1.find(g) != s1.end()); + } +} + +void testBatchSampleNeighboor( + std::shared_ptr& worker_ptr_) { + std::vector>> vs; + std::vector v = {37, 96}; + auto pull_status = worker_ptr_->batch_sample_neighboors(0, v, 4, vs); + pull_status.wait(); + std::unordered_set s; + std::unordered_set s1 = {112, 45, 145}; + for (auto g : vs[0]) { + s.insert(g.first); + } + ASSERT_EQ(s.size(), 3); + for (auto g : s) { + ASSERT_EQ(true, s1.find(g) != s1.end()); + } + s.clear(); + s1.clear(); + s1 = {111, 48, 247}; + for (auto g : vs[1]) { + s.insert(g.first); + } + ASSERT_EQ(s.size(), 3); + for (auto g : s) { + ASSERT_EQ(true, s1.find(g) != s1.end()); + } +} + void testGraphToBuffer(); // std::string nodes[] = {std::string("37\taa\t45;0.34\t145;0.31\t112;0.21"), // std::string("96\tfeature\t48;1.4\t247;0.31\t111;1.21"), @@ -230,24 +289,9 @@ void RunBrpcPushSparse() { // } // std::vector> v; // pull_status = worker_ptr_->sample(0, 37, 4, v); - pull_status = - worker_ptr_->batch_sample(0, std::vector(1, 37), 4, vs); - pull_status.wait(); - ASSERT_EQ(vs[0].size(), 3); - vs.clear(); - // pull_status = worker_ptr_->sample(0, 96, 4, v); - pull_status = - worker_ptr_->batch_sample(0, std::vector(1, 96), 4, vs); - pull_status.wait(); - std::unordered_set s = {111, 48, 247}; - ASSERT_EQ(3, vs[0].size()); - for (auto g : vs[0]) { - // std::cout << g.first << std::endl; - ASSERT_EQ(true, s.find(g.first) != s.end()); - } - vs.clear(); - - pull_status = worker_ptr_->batch_sample( + testSingleSampleNeighboor(worker_ptr_); + testBatchSampleNeighboor(worker_ptr_); + pull_status = worker_ptr_->batch_sample_neighboors( 0, std::vector(1, 10240001024), 4, vs); pull_status.wait(); ASSERT_EQ(0, vs[0].size()); @@ -257,10 +301,6 @@ void RunBrpcPushSparse() { pull_status.wait(); ASSERT_EQ(nodes.size(), 1); ASSERT_EQ(nodes[0].get_id(), 37); - // for (auto g : v) { - // std::cout << g.get_id() << " " << g.get_graph_node_type() << std::endl; - // } - // ASSERT_EQ(v.size(),1); nodes.clear(); pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); pull_status.wait(); @@ -346,4 +386,5 @@ void testGraphToBuffer() { VLOG(0) << s.get_feature(); VLOG(0) << s1.get_feature(); } + TEST(RunBrpcPushSparse, Run) { RunBrpcPushSparse(); } From 5d53dfe269d0be57a97850951c320bffaa312e1f Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 18 Mar 2021 12:56:47 +0800 Subject: [PATCH 042/106] add common_graph_table --- .../distributed/service/graph_py_service.cc | 7 ++- .../distributed/service/graph_py_service.h | 8 +-- .../distributed/table/common_graph_table.cc | 9 ++-- .../fluid/distributed/test/graph_node_test.cc | 53 ++++++++++++++----- 4 files changed, 57 insertions(+), 20 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 3ba35fda0bf39..ee355101993e9 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -33,15 +33,18 @@ std::vector GraphPyService::split(std::string& str, } void GraphPyService::set_up(std::string ips_str, int shard_num, + std::vector node_types, std::vector edge_types) { set_shard_num(shard_num); // set_client_Id(client_id); // set_rank(rank); - this->table_id_map[std::string("")] = 0; + for (size_t table_id = 0; table_id < node_types.size(); table_id++) { + this->table_id_map[node_types[table_id]] = this->table_id_map.size(); + } // Table 0 are for nodes for (size_t table_id = 0; table_id < edge_types.size(); table_id++) { - this->table_id_map[edge_types[table_id]] = int(table_id + 1); + this->table_id_map[edge_types[table_id]] = this->table_id_map.size(); } std::istringstream stream(ips_str); std::string ip; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 8f6c9f0ad0b64..137f854b39e11 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -82,14 +82,16 @@ class GraphPyService { int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); void set_up(std::string ips_str, int shard_num, + std::vector node_types, std::vector edge_types); }; class GraphPyServer : public GraphPyService { public: void set_up(std::string ips_str, int shard_num, + std::vector node_types, std::vector edge_types, int rank) { set_rank(rank); - GraphPyService::set_up(ips_str, shard_num, edge_types); + GraphPyService::set_up(ips_str, shard_num, node_types, edge_types); } int get_rank() { return rank; } void set_rank(int rank) { this->rank = rank; } @@ -107,9 +109,9 @@ class GraphPyServer : public GraphPyService { class GraphPyClient : public GraphPyService { public: void set_up(std::string ips_str, int shard_num, - std::vector edge_types, int client_id) { + std::vector node_types, std::vector edge_types, int client_id) { set_client_id(client_id); - GraphPyService::set_up(ips_str, shard_num, edge_types); + GraphPyService::set_up(ips_str, shard_num, node_types, edge_types); } std::shared_ptr get_ps_client() { return worker_ptr; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 3a31d49190711..3b42e151c2d7d 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -119,13 +119,16 @@ int32_t GraphTable::load_nodes(const std::string &path) { std::string node_type = values[0]; std::vector feature; - feature.push_back(node_type); for (size_t slice = 2; slice < values.size(); slice++) { feature.push_back(values[slice]); } - auto feat = paddle::string::join_strings(feature, '\t'); size_t index = shard_id - shard_start; - shards[index].add_node(id, feat); + if(feature.size() > 0) { + shards[index].add_node(id, paddle::string::join_strings(feature, '\t')); + } + else { + shards[index].add_node(id, std::string("")); + } } } return 0; diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 192cc64027bef..6770623eda23a 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -55,14 +55,35 @@ void testGraphToBuffer(); // std::string("59\ttreat\t45;0.34\t145;0.31\t112;0.21"), // std::string("97\tfood\t48;1.4\t247;0.31\t111;1.21")}; -std::string nodes[] = { +std::string edges[] = { std::string("37\t45\t0.34"), std::string("37\t145\t0.31"), std::string("37\t112\t0.21"), std::string("96\t48\t1.4"), std::string("96\t247\t0.31"), std::string("96\t111\t1.21"), std::string("59\t45\t0.34"), std::string("59\t145\t0.31"), std::string("59\t122\t0.21"), std::string("97\t48\t0.34"), std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; -char file_name[] = "nodes.txt"; +char edge_file_name[] = "edges.txt"; + +std::string nodes[] = { + std::string("user\t37\t0.34"), + std::string("user\t96\t0.31"), + std::string("user\t59\t0.11"), + std::string("user\t97\t0.11"), + std::string("item\t45\t0.21"), + std::string("item\t145\t0.21"), + std::string("item\t112\t0.21"), + std::string("item\t48\t0.21"), + std::string("item\t247\t0.21"), + std::string("item\t111\t0.21"), + std::string("item\t45\t0.21"), + std::string("item\t145\t0.21"), + std::string("item\t122\t0.21"), + std::string("item\t48\t0.21"), + std::string("item\t247\t0.21"), + std::string("item\t111\t0.21")}; +char node_file_name[] = "nodes.txt"; + + void prepare_file(char file_name[]) { std::ofstream ofile; ofile.open(file_name); @@ -139,7 +160,7 @@ ::paddle::distributed::PSParameter GetWorkerProto() { /*-------------------------------------------------------------------------*/ std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; -uint32_t port_ = 4209, port2 = 4210; +uint32_t port_ = 4250, port2 = 4251; std::vector host_sign_list_; @@ -194,7 +215,8 @@ void RunClient(std::map>& void RunBrpcPushSparse() { setenv("http_proxy", "", 1); setenv("https_proxy", "", 1); - prepare_file(file_name); + prepare_file(edge_file_name); + prepare_file(node_file_name); auto ph_host = paddle::distributed::PSHost(ip_, port_, 0); host_sign_list_.push_back(ph_host.serialize_to_string()); @@ -216,7 +238,8 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ auto pull_status = - worker_ptr_->load(0, std::string(file_name), std::string("edge")); + worker_ptr_->load(0, std::string(edge_file_name), std::string("edge")); + std::cout << "Fuck2" << std::endl; pull_status.wait(); std::vector>> vs; @@ -274,11 +297,13 @@ void RunBrpcPushSparse() { distributed::GraphPyServer server1, server2; distributed::GraphPyClient client1, client2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; + std::cout << "Fuck" << std::endl; std::vector edge_types = {std::string("user2item")}; - server1.set_up(ips_str, 127, edge_types, 0); - server2.set_up(ips_str, 127, edge_types, 1); - client1.set_up(ips_str, 127, edge_types, 0); - client2.set_up(ips_str, 127, edge_types, 1); + std::vector node_types = {std::string("user"), std::string("item")}; + server1.set_up(ips_str, 127, node_types, edge_types, 0); + server2.set_up(ips_str, 127, node_types, edge_types, 1); + client1.set_up(ips_str, 127, node_types, edge_types, 0); + client2.set_up(ips_str, 127, node_types, edge_types, 1); server1.start_server(); std::cout << "first server done" << std::endl; server2.start_server(); @@ -288,11 +313,14 @@ void RunBrpcPushSparse() { client2.start_client(); std::cout << "first client done" << std::endl; std::cout << "started" << std::endl; - client1.load_edge_file(std::string("user2item"), std::string(file_name), 0); + client1.load_node_file(std::string("user"), std::string(node_file_name)); + client1.load_node_file(std::string("item"), std::string(node_file_name)); + client1.load_edge_file(std::string("user2item"), std::string(edge_file_name), 0); // client2.load_edge_file(std::string("user2item"), std::string(file_name), // 0); nodes.clear(); - nodes = client2.pull_graph_list(std::string("user2item"), 0, 1, 4); + nodes = client2.pull_graph_list(std::string("user"), 0, 1, 4); + std::cout << "node_ids: " << nodes[0].get_id() << std::endl; ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); vs = client1.batch_sample_k(std::string("user2item"), @@ -324,7 +352,8 @@ void RunBrpcPushSparse() { // for x in list: // print(x.get_id()) - std::remove(file_name); + std::remove(edge_file_name); + std::remove(node_file_name); LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); LOG(INFO) << "Run finalize_worker"; From e30dfa5e1a7672f89ccdaa7cc5c8b61eb5d5a8ee Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Thu, 18 Mar 2021 16:42:35 +0800 Subject: [PATCH 043/106] sample with srand --- paddle/fluid/distributed/table/weighted_sampler.cc | 3 +++ 1 file changed, 3 insertions(+) diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index 09ecdc2b642e4..b2753399db5d6 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -41,6 +41,9 @@ std::vector WeightedSampler::sample_k(int k) { float subtract; std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; + struct timespec tn; + clock_gettime(CLOCK_REALTIME, &tn); + srand(tn.tv_nsec); while (k--) { float query_weight = rand() % 100000 / 100000.0; query_weight *= weight - subtract_weight_map[this]; From 77b23515d42e1f5be18d7f72b8501b474743df1f Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Thu, 18 Mar 2021 18:50:20 +0800 Subject: [PATCH 044/106] add node_types --- .../distributed/service/graph_py_service.cc | 14 +++++++--- .../distributed/table/common_graph_table.cc | 21 ++++++++------ .../distributed/table/common_graph_table.h | 2 +- .../fluid/distributed/test/graph_node_test.cc | 28 ++++++++++++------- 4 files changed, 42 insertions(+), 23 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 5d62e090c8687..4a7d2706fef39 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -42,7 +42,6 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, for (size_t table_id = 0; table_id < node_types.size(); table_id++) { this->table_id_map[node_types[table_id]] = this->table_id_map.size(); } - // Table 0 are for nodes for (size_t table_id = 0; table_id < edge_types.size(); table_id++) { this->table_id_map[edge_types[table_id]] = this->table_id_map.size(); } @@ -165,9 +164,15 @@ ::paddle::distributed::PSParameter GraphPyClient::GetWorkerProto() { } void GraphPyClient::load_edge_file(std::string name, std::string filepath, bool reverse) { - std::string params = "edge"; + // 'e' means load edge + std::string params = "e"; if (reverse) { - params += "|reverse"; + // 'e<' means load edges from $2 to $1 + params += "<"; + } + else { + // 'e>' means load edges from $1 to $2 + params += ">"; } if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; @@ -178,7 +183,8 @@ void GraphPyClient::load_edge_file(std::string name, std::string filepath, } void GraphPyClient::load_node_file(std::string name, std::string filepath) { - std::string params = "node"; + // 'n' means load nodes and 'node_type' follows + std::string params = "n" + name; if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; auto status = diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 3b57023ac1901..82a7d48d14856 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -52,14 +52,16 @@ GraphNode *GraphShard::find_node(uint64_t id) { } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { - auto cmd = paddle::string::split_string(param, "|"); - std::set cmd_set(cmd.begin(), cmd.end()); - bool reverse_edge = cmd_set.count(std::string("reverse")); - bool load_edge = cmd_set.count(std::string("edge")); + + bool load_edge = (param[0] == 'e'); + bool load_node = (param[0] == 'n'); if (load_edge) { + bool reverse_edge = (param[1] == '<'); return this->load_edges(path, reverse_edge); - } else { - return this->load_nodes(path); + } + if (load_node){ + std::string node_type = param.substr(1); + return this->load_nodes(path, node_type); } } @@ -99,7 +101,7 @@ int32_t GraphTable::get_nodes_ids_by_ranges( } return 0; } -int32_t GraphTable::load_nodes(const std::string &path) { +int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto paths = paddle::string::split_string(path, ";"); for (auto path : paths) { std::ifstream file(path); @@ -116,7 +118,10 @@ int32_t GraphTable::load_nodes(const std::string &path) { continue; } - std::string node_type = values[0]; + std::string nt = values[0]; + if (nt != node_type) { + continue; + } std::vector feature; for (size_t slice = 2; slice < values.size(); slice++) { feature.push_back(values[slice]); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 9e39bbe842a0d..7413409dd4257 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -97,7 +97,7 @@ class GraphTable : public SparseTable { int32_t load_edges(const std::string &path, bool reverse); - int32_t load_nodes(const std::string &path); + int32_t load_nodes(const std::string &path, std::string node_type); GraphNode *find_node(uint64_t id); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index dafccb52c2984..b270405969589 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -143,11 +143,17 @@ std::string nodes[] = { char node_file_name[] = "nodes.txt"; -void prepare_file(char file_name[]) { +void prepare_file(char file_name[], bool load_edge) { std::ofstream ofile; ofile.open(file_name); - for (auto x : nodes) { - ofile << x << std::endl; + if(load_edge) { + for (auto x : edges) { + ofile << x << std::endl; + } + } else { + for (auto x : nodes) { + ofile << x << std::endl; + } } ofile.close(); } @@ -219,7 +225,7 @@ ::paddle::distributed::PSParameter GetWorkerProto() { /*-------------------------------------------------------------------------*/ std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; -uint32_t port_ = 4250, port2 = 4251; +uint32_t port_ = 4209, port2 = 4210; std::vector host_sign_list_; @@ -274,8 +280,8 @@ void RunClient(std::map>& void RunBrpcPushSparse() { setenv("http_proxy", "", 1); setenv("https_proxy", "", 1); - prepare_file(edge_file_name); - prepare_file(node_file_name); + prepare_file(edge_file_name, 1); + prepare_file(node_file_name, 0); auto ph_host = paddle::distributed::PSHost(ip_, port_, 0); host_sign_list_.push_back(ph_host.serialize_to_string()); @@ -297,8 +303,7 @@ void RunBrpcPushSparse() { /*-----------------------Test Server Init----------------------------------*/ auto pull_status = - worker_ptr_->load(0, std::string(edge_file_name), std::string("edge")); - std::cout << "Fuck2" << std::endl; + worker_ptr_->load(0, std::string(edge_file_name), std::string("e>")); pull_status.wait(); std::vector>> vs; @@ -337,7 +342,6 @@ void RunBrpcPushSparse() { distributed::GraphPyServer server1, server2; distributed::GraphPyClient client1, client2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; - std::cout << "Fuck" << std::endl; std::vector edge_types = {std::string("user2item")}; std::vector node_types = {std::string("user"), std::string("item")}; server1.set_up(ips_str, 127, node_types, edge_types, 0); @@ -359,7 +363,11 @@ void RunBrpcPushSparse() { // client2.load_edge_file(std::string("user2item"), std::string(file_name), // 0); nodes.clear(); - nodes = client2.pull_graph_list(std::string("user"), 0, 1, 4); + nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4); + + for (auto g : nodes) { + std::cout << "node_ids: " << g.get_id() << std::endl; + } std::cout << "node_ids: " << nodes[0].get_id() << std::endl; ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); From 6fdb5c9b9bdddcdfb4b410c3e462b25f5557d37d Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 18 Mar 2021 16:27:06 +0000 Subject: [PATCH 045/106] optimize nodes sample --- .../distributed/service/graph_brpc_client.cc | 6 +- .../distributed/table/common_graph_table.cc | 34 ++++++++--- .../distributed/table/common_graph_table.h | 7 +-- .../fluid/distributed/test/graph_node_test.cc | 58 +++++++++++++------ 4 files changed, 70 insertions(+), 35 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 0514c170f1575..2c9ee2b9037b7 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -148,13 +148,9 @@ std::future GraphBrpcClient::random_sample_nodes( butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); char buffer[bytes_size]; - io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + auto size = io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); int index = 0; while (index < bytes_size) { - // GraphNode node; - // node.recover_from_buffer(buffer + index); - // index += node.get_size(true); - // res.push_back(node); ids.push_back(*(uint64_t *)(buffer + index)); index += GraphNode::id_size; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 70c43e7099099..5bd5ee268bbb5 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -64,10 +64,14 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { } int32_t GraphTable::get_nodes_ids_by_ranges( - std::vector> ranges, std::vector res) { + std::vector> ranges, std::vector &res) { int start = 0, end, index = 0, total_size = 0; res.clear(); std::vector>> tasks; + // std::string temp = ""; + // for(int i = 0;i < shards.size();i++) + // temp+= std::to_string((int)shards[i].get_size()) + " "; + // VLOG(0)<<"range distribution "<enqueue( [this, first, second, i]() -> std::vector { return shards[i].get_ids_by_range(first, second); })); } } + total_size += shards[i].get_size(); } for (int i = 0; i < tasks.size(); i++) { auto vec = tasks[i].get(); @@ -191,7 +196,8 @@ uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { } int32_t GraphTable::random_sample_nodes(int sample_size, std::unique_ptr &buffer, - int &actual_size, bool need_feature) { + int &actual_size) { + bool need_feature = false; int total_size = 0; for (int i = 0; i < shards.size(); i++) { total_size += shards[i].get_size(); @@ -199,6 +205,7 @@ int32_t GraphTable::random_sample_nodes(int sample_size, if (sample_size > total_size) sample_size = total_size; int range_num = random_sample_nodes_ranges; if (range_num > sample_size) range_num = sample_size; + if (sample_size == 0 || range_num == 0) return 0; std::vector ranges_len, ranges_pos; int remain = sample_size, last_pos = -1, num; std::set separator_set; @@ -228,13 +235,26 @@ int32_t GraphTable::random_sample_nodes(int sample_size, ranges_pos.push_back(used); used += ranges_len[index++]; } - std::vector> vec; + std::vector> first_half, second_half; + int start_index = rand() % total_size; for (int i = 0; i < ranges_len.size() && i < ranges_pos.size(); i++) { - vec.push_back({ranges_pos[i], ranges_len[i]}); + if (ranges_pos[i] + ranges_len[i] - 1 + start_index < total_size) + first_half.push_back({ranges_pos[i] + start_index, + ranges_pos[i] + ranges_len[i] + start_index}); + else if (ranges_pos[i] + start_index >= total_size) { + second_half.push_back( + {ranges_pos[i] + start_index - total_size, + ranges_pos[i] + ranges_len[i] + start_index - total_size}); + } else { + first_half.push_back({ranges_pos[i] + start_index, total_size}); + second_half.push_back( + {0, ranges_pos[i] + ranges_len[i] + start_index - total_size}); + } } + for (auto &pair : first_half) second_half.push_back(pair); std::vector res; - get_nodes_ids_by_ranges(vec, res); - actual_size = res.size() * (GraphNode::id_size); + get_nodes_ids_by_ranges(second_half, res); + actual_size = res.size() * sizeof(uint64_t); buffer.reset(new char[actual_size]); char *pointer = buffer.get(); memcpy(pointer, res.data(), actual_size); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 9e39bbe842a0d..7134f53c075b3 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -85,12 +85,11 @@ class GraphTable : public SparseTable { std::vector> &buffers, std::vector &actual_sizes); - virtual int32_t random_sample_nodes(int sample_size, - std::unique_ptr &buffers, - int &actual_sizes, bool need_feature); + int32_t random_sample_nodes(int sample_size, std::unique_ptr &buffers, + int &actual_sizes); virtual int32_t get_nodes_ids_by_ranges( - std::vector> ranges, std::vector res); + std::vector> ranges, std::vector &res); virtual int32_t initialize(); int32_t load(const std::string &path, const std::string ¶m); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 0b1a1b71c213e..79946bce014c0 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -49,6 +49,19 @@ namespace math = paddle::operators::math; namespace memory = paddle::memory; namespace distributed = paddle::distributed; +void testSampleNodes( + std::shared_ptr& worker_ptr_) { + std::vector ids; + auto pull_status = worker_ptr_->random_sample_nodes(0, 0, 6, ids); + std::unordered_set s; + std::unordered_set s1 = {37, 59}; + pull_status.wait(); + for (auto id : ids) s.insert(id); + ASSERT_EQ(true, s.size() == s1.size()); + for (auto id : s) { + ASSERT_EQ(true, s1.find(id) != s1.end()); + } +} void testSingleSampleNeighboor( std::shared_ptr& worker_ptr_) { std::vector>> vs; @@ -120,7 +133,8 @@ std::string nodes[] = { std::string("96\t247\t0.31"), std::string("96\t111\t1.21"), std::string("59\t45\t0.34"), std::string("59\t145\t0.31"), std::string("59\t122\t0.21"), std::string("97\t48\t0.34"), - std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; + std::string("97\t247\t0.31"), std::string("97\t111\t0.21"), +}; char file_name[] = "nodes.txt"; void prepare_file(char file_name[]) { std::ofstream ofile; @@ -128,6 +142,11 @@ void prepare_file(char file_name[]) { for (auto x : nodes) { ofile << x << std::endl; } + // for(int i = 0;i < 10;i++){ + // for(int j = 0;j < 10;j++){ + // ofile<load(0, std::string(file_name), std::string("edge")); - + srand(time(0)); pull_status.wait(); std::vector>> vs; // for(int i = 0;i < 100000000;i++){ @@ -289,27 +308,28 @@ void RunBrpcPushSparse() { // } // std::vector> v; // pull_status = worker_ptr_->sample(0, 37, 4, v); + testSampleNodes(worker_ptr_); testSingleSampleNeighboor(worker_ptr_); testBatchSampleNeighboor(worker_ptr_); - pull_status = worker_ptr_->batch_sample_neighboors( - 0, std::vector(1, 10240001024), 4, vs); - pull_status.wait(); - ASSERT_EQ(0, vs[0].size()); + // pull_status = worker_ptr_->batch_sample_neighboors( + // 0, std::vector(1, 10240001024), 4, vs); + // pull_status.wait(); + // ASSERT_EQ(0, vs[0].size()); std::vector nodes; - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); - pull_status.wait(); - ASSERT_EQ(nodes.size(), 1); - ASSERT_EQ(nodes[0].get_id(), 37); - nodes.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); - pull_status.wait(); - ASSERT_EQ(nodes.size(), 1); - ASSERT_EQ(nodes[0].get_id(), 59); - for (auto g : nodes) { - std::cout << g.get_id() << std::endl; - } - + // pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); + // pull_status.wait(); + // ASSERT_EQ(nodes.size(), 1); + // ASSERT_EQ(nodes[0].get_id(), 37); + // nodes.clear(); + // pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); + // pull_status.wait(); + // ASSERT_EQ(nodes.size(), 1); + // ASSERT_EQ(nodes[0].get_id(), 59); + // for (auto g : nodes) { + // std::cout << g.get_id() << std::endl; + // } + sleep(5); // distributed::GraphPyService gps1, gps2; distributed::GraphPyServer server1, server2; distributed::GraphPyClient client1, client2; From 217b17f199119a5a9e8a1648481f0ee2bb172fb0 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 18 Mar 2021 16:32:32 +0000 Subject: [PATCH 046/106] recover test --- .../fluid/distributed/test/graph_node_test.cc | 34 +++++++++---------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 79946bce014c0..7b7412298da87 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -311,26 +311,24 @@ void RunBrpcPushSparse() { testSampleNodes(worker_ptr_); testSingleSampleNeighboor(worker_ptr_); testBatchSampleNeighboor(worker_ptr_); - // pull_status = worker_ptr_->batch_sample_neighboors( - // 0, std::vector(1, 10240001024), 4, vs); - // pull_status.wait(); - // ASSERT_EQ(0, vs[0].size()); + pull_status = worker_ptr_->batch_sample_neighboors( + 0, std::vector(1, 10240001024), 4, vs); + pull_status.wait(); + ASSERT_EQ(0, vs[0].size()); std::vector nodes; - // pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); - // pull_status.wait(); - // ASSERT_EQ(nodes.size(), 1); - // ASSERT_EQ(nodes[0].get_id(), 37); - // nodes.clear(); - // pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); - // pull_status.wait(); - // ASSERT_EQ(nodes.size(), 1); - // ASSERT_EQ(nodes[0].get_id(), 59); - // for (auto g : nodes) { - // std::cout << g.get_id() << std::endl; - // } - sleep(5); - // distributed::GraphPyService gps1, gps2; + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); + pull_status.wait(); + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 37); + nodes.clear(); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); + pull_status.wait(); + ASSERT_EQ(nodes.size(), 1); + ASSERT_EQ(nodes[0].get_id(), 59); + for (auto g : nodes) { + std::cout << g.get_id() << std::endl; + } distributed::GraphPyServer server1, server2; distributed::GraphPyClient client1, client2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; From 90f30ce85db7b31abb9fe15c4ecd837af423ed64 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 19 Mar 2021 10:26:18 +0800 Subject: [PATCH 047/106] random sample --- .../distributed/table/common_graph_table.cc | 7 +-- paddle/fluid/distributed/table/graph_node.cc | 15 ++++-- paddle/fluid/distributed/table/graph_node.h | 4 +- .../distributed/table/weighted_sampler.cc | 47 +++++++++++++++++-- .../distributed/table/weighted_sampler.h | 24 ++++++++-- 5 files changed, 81 insertions(+), 16 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 70c43e7099099..2af7c7338cbee 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -133,6 +133,7 @@ int32_t GraphTable::load_nodes(const std::string &path) { int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { auto paths = paddle::string::split_string(path, ";"); int count = 0; + std::string sample_type = "random"; for (auto path : paths) { std::ifstream file(path); @@ -146,9 +147,10 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { if (reverse_edge) { std::swap(src_id, dst_id); } - float weight = 0; + float weight = 1; if (values.size() == 3) { weight = std::stof(values[2]); + sample_type = "weighted"; } size_t src_shard_id = src_id % shard_num; @@ -171,8 +173,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { for (auto &shard : shards) { auto bucket = shard.get_bucket(); for (int i = 0; i < bucket.size(); i++) { - bucket[i]->build_sampler(); - } + bucket[i]->build_sampler(sample_type); } } return 0; } diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 8408ea7aeafed..4d557ca14182b 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -22,10 +22,15 @@ int GraphNode::int_size = sizeof(int); int GraphNode::get_size(bool need_feature) { return id_size + int_size + (need_feature ? feature.size() : 0); } -void GraphNode::build_sampler() { - sampler = new WeightedSampler(); - GraphEdge** arr = edges.data(); - sampler->build((WeightedObject**)arr, 0, edges.size()); +void GraphNode::build_sampler(std::string sample_type) { + if (sample_type == "random"){ + sampler = new RandomSampler(); + } else if (sample_type == "weighted"){ + sampler = new WeightedSampler(); + } + //GraphEdge** arr = edges.data(); + //sampler->build((WeightedObject**)arr, 0, edges.size()); + sampler->build((std::vector*)&edges); } void GraphNode::to_buffer(char* buffer, bool need_feature) { int size = get_size(need_feature); @@ -51,4 +56,4 @@ void GraphNode::recover_from_buffer(char* buffer) { // type = GraphNodeType(int_state); } } -} \ No newline at end of file +} diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index dbd387b2d4519..f5169129d9f47 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -40,7 +40,7 @@ class GraphNode { void set_feature(std::string feature) { this->feature = feature; } std::string get_feature() { return feature; } virtual int get_size(bool need_feature); - virtual void build_sampler(); + virtual void build_sampler(std::string sample_type); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } @@ -58,7 +58,7 @@ class GraphNode { protected: uint64_t id; std::string feature; - WeightedSampler *sampler; + Sampler *sampler; std::vector edges; }; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index b2753399db5d6..52d44068a21dc 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -14,9 +14,50 @@ #include "paddle/fluid/distributed/table/weighted_sampler.h" #include +#include namespace paddle { namespace distributed { -void WeightedSampler::build(WeightedObject **v, int start, int end) { + +void RandomSampler::build(std::vector* edges) { + this->edges = edges; +} + +std::vector RandomSampler::sample_k(int k) { + int n = edges->size(); + if (k > n){ + k = n; + } + struct timespec tn; + clock_gettime(CLOCK_REALTIME, &tn); + srand(tn.tv_nsec); + std::vector sample_result; + std::unordered_map replace_map; + while(k--){ + int rand_int = rand() % n; + auto tmp = replace_map.find(rand_int); + if(tmp == replace_map.end()){ + sample_result.push_back(edges->at(rand_int)); + }else{ + sample_result.push_back(edges->at(tmp->second)); + } + + tmp = replace_map.find(n - 1); + if(tmp == replace_map.end()){ + replace_map[rand_int] = n - 1; + }else{ + replace_map[rand_int] = tmp->second; + } + --n; + } + return sample_result; +} + +void WeightedSampler::build(std::vector* edges) { + WeightedObject** v = edges->data(); + return build_one(v, 0, edges->size()); +} + +void WeightedSampler::build_one(WeightedObject **v, int start, int end) { count = 0; if (start + 1 == end) { left = right = NULL; @@ -27,8 +68,8 @@ void WeightedSampler::build(WeightedObject **v, int start, int end) { } else { left = new WeightedSampler(); right = new WeightedSampler(); - left->build(v, start, start + (end - start) / 2); - right->build(v, start + (end - start) / 2, end); + left->build_one(v, start, start + (end - start) / 2); + right->build_one(v, start + (end - start) / 2, end); weight = left->weight + right->weight; count = left->count + right->count; } diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 9ed2cc04649de..4d4640a33f574 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -18,6 +18,7 @@ #include namespace paddle { namespace distributed { + class WeightedObject { public: WeightedObject() {} @@ -26,14 +27,31 @@ class WeightedObject { virtual float get_weight() = 0; }; -class WeightedSampler { +class Sampler { +public: + virtual ~Sampler() {} + virtual void build(std::vector* edges) = 0; + virtual std::vector sample_k(int k) = 0; +}; + +class RandomSampler: public Sampler { +public: + virtual ~RandomSampler() {} + virtual void build(std::vector* edges); + virtual std::vector sample_k(int k); + std::vector* edges; +}; + +class WeightedSampler: public Sampler { public: + virtual ~WeightedSampler() {} WeightedSampler *left, *right; WeightedObject *object; int count; float weight; - void build(WeightedObject **v, int start, int end); - std::vector sample_k(int k); + virtual void build(std::vector* edges); + virtual void build_one(WeightedObject **v, int start, int end); + virtual std::vector sample_k(int k); private: WeightedObject *sample( From ec2555ab1229dc2569304cb02f9ee6efdf32627b Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Fri, 19 Mar 2021 12:07:50 +0800 Subject: [PATCH 048/106] destruct weighted sampler --- .../distributed/table/weighted_sampler.cc | 41 +++++++++++++++---- .../distributed/table/weighted_sampler.h | 3 +- 2 files changed, 35 insertions(+), 9 deletions(-) diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index 52d44068a21dc..707d7ab0ba7d4 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -34,25 +34,50 @@ std::vector RandomSampler::sample_k(int k) { std::unordered_map replace_map; while(k--){ int rand_int = rand() % n; - auto tmp = replace_map.find(rand_int); - if(tmp == replace_map.end()){ + auto iter = replace_map.find(rand_int); + if(iter == replace_map.end()){ sample_result.push_back(edges->at(rand_int)); }else{ - sample_result.push_back(edges->at(tmp->second)); + sample_result.push_back(edges->at(iter->second)); } - tmp = replace_map.find(n - 1); - if(tmp == replace_map.end()){ + iter = replace_map.find(n - 1); + if(iter == replace_map.end()){ replace_map[rand_int] = n - 1; }else{ - replace_map[rand_int] = tmp->second; + replace_map[rand_int] = iter->second; } --n; } return sample_result; } +WeightedSampler::WeightedSampler(){ + left = nullptr; + right = nullptr; + object = nullptr; +} + +WeightedSampler::~WeightedSampler() { + if(left != nullptr){ + delete left; + left = nullptr; + } + if(right != nullptr){ + delete right; + right = nullptr; + } +} + void WeightedSampler::build(std::vector* edges) { + if(left != nullptr){ + delete left; + left = nullptr; + } + if(right != nullptr){ + delete right; + right = nullptr; + } WeightedObject** v = edges->data(); return build_one(v, 0, edges->size()); } @@ -60,7 +85,7 @@ void WeightedSampler::build(std::vector* edges) { void WeightedSampler::build_one(WeightedObject **v, int start, int end) { count = 0; if (start + 1 == end) { - left = right = NULL; + left = right = nullptr; weight = v[start]->get_weight(); object = v[start]; count = 1; @@ -98,7 +123,7 @@ WeightedObject *WeightedSampler::sample( std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, float &subtract) { - if (left == NULL) { + if (left == nullptr) { subtract_weight_map[this] = weight; subtract = weight; subtract_count_map[this] = 1; diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 4d4640a33f574..4a7d08c1404a9 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -44,7 +44,8 @@ class RandomSampler: public Sampler { class WeightedSampler: public Sampler { public: - virtual ~WeightedSampler() {} + WeightedSampler(); + virtual ~WeightedSampler(); WeightedSampler *left, *right; WeightedObject *object; int count; From 31d411af41854d7efb39143db0602b85a3bb1f50 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Sat, 20 Mar 2021 02:18:21 +0800 Subject: [PATCH 049/106] GraphEdgeBlob --- paddle/fluid/distributed/table/CMakeLists.txt | 6 ++- .../distributed/table/common_graph_table.cc | 18 +++---- .../distributed/table/common_graph_table.h | 2 +- paddle/fluid/distributed/table/graph_edge.cc | 30 ++++++++++++ paddle/fluid/distributed/table/graph_edge.h | 47 +++++++++++++++++++ paddle/fluid/distributed/table/graph_node.cc | 26 ++++++++-- paddle/fluid/distributed/table/graph_node.h | 36 ++++---------- .../distributed/table/weighted_sampler.cc | 45 +++++++++--------- .../distributed/table/weighted_sampler.h | 32 +++++-------- 9 files changed, 161 insertions(+), 81 deletions(-) create mode 100644 paddle/fluid/distributed/table/graph_edge.cc create mode 100644 paddle/fluid/distributed/table/graph_edge.h diff --git a/paddle/fluid/distributed/table/CMakeLists.txt b/paddle/fluid/distributed/table/CMakeLists.txt index 779012ad1a8f2..ede1c88d757bc 100644 --- a/paddle/fluid/distributed/table/CMakeLists.txt +++ b/paddle/fluid/distributed/table/CMakeLists.txt @@ -1,8 +1,10 @@ set_property(GLOBAL PROPERTY TABLE_DEPS string_helper) get_property(TABLE_DEPS GLOBAL PROPERTY TABLE_DEPS) +set_source_files_properties(graph_edge.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(graph_edge SRCS graph_edge.cc) set_source_files_properties(weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(WeightedSampler SRCS weighted_sampler.cc) +cc_library(WeightedSampler SRCS weighted_sampler.cc DEPS graph_edge) set_source_files_properties(graph_node.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_library(graph_node SRCS graph_node.cc DEPS WeightedSampler) set_source_files_properties(common_dense_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) @@ -11,7 +13,7 @@ set_source_files_properties(sparse_geo_table.cc PROPERTIES COMPILE_FLAGS ${DISTR set_source_files_properties(barrier_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(common_graph_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(common_table SRCS common_sparse_table.cc common_dense_table.cc sparse_geo_table.cc barrier_table.cc common_graph_table.cc DEPS ${TABLE_DEPS} graph_node device_context string_helper simple_threadpool xxhash generator) +cc_library(common_table SRCS common_sparse_table.cc common_dense_table.cc sparse_geo_table.cc barrier_table.cc common_graph_table.cc DEPS ${TABLE_DEPS} graph_edge graph_node device_context string_helper simple_threadpool xxhash generator) set_source_files_properties(tensor_accessor.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(tensor_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index c209cb2f508a9..d2c5a095a305e 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -42,8 +42,8 @@ GraphNode *GraphShard::add_node(uint64_t id, std::string feature) { return bucket.back(); } -void GraphShard::add_neighboor(uint64_t id, GraphEdge *edge) { - add_node(id, std::string(""))->add_edge(edge); +void GraphShard::add_neighboor(uint64_t id, uint64_t dst_id, float weight) { + add_node(id, std::string(""))->add_edge(dst_id, weight); } GraphNode *GraphShard::find_node(uint64_t id) { @@ -147,6 +147,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { auto paths = paddle::string::split_string(path, ";"); int count = 0; std::string sample_type = "random"; + bool is_weighted = false; for (auto path : paths) { std::ifstream file(path); @@ -164,6 +165,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { if (values.size() == 3) { weight = std::stof(values[2]); sample_type = "weighted"; + is_weighted = true; } size_t src_shard_id = src_id % shard_num; @@ -175,8 +177,8 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { } size_t index = src_shard_id - shard_start; - GraphEdge *edge = new GraphEdge(dst_id, weight); - shards[index].add_neighboor(src_id, edge); + shards[index].add_node(src_id, std::string(""))->build_edges(is_weighted); + shards[index].add_neighboor(src_id, dst_id, weight); } } VLOG(0) << "Load Finished Total Edge Count " << count; @@ -287,7 +289,7 @@ int GraphTable::random_sample_neighboors( actual_size = 0; return 0; } - std::vector res = node->sample_k(sample_size); + std::vector res = node->sample_k(sample_size); actual_size = res.size() * (GraphNode::id_size + GraphNode::weight_size); int offset = 0; @@ -295,9 +297,9 @@ int GraphTable::random_sample_neighboors( float weight; char *buffer_addr = new char[actual_size]; buffer.reset(buffer_addr); - for (auto &x : res) { - id = x->get_id(); - weight = x->get_weight(); + for (int &x : res) { + id = node->get_neighbor_id(x); + weight = node->get_neighbor_weight(x); memcpy(buffer_addr + offset, &id, GraphNode::id_size); offset += GraphNode::id_size; memcpy(buffer_addr + offset, &weight, GraphNode::weight_size); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 0aa67f08b5a38..106dcaaac2b00 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -61,7 +61,7 @@ class GraphShard { } GraphNode *add_node(uint64_t id, std::string feature); GraphNode *find_node(uint64_t id); - void add_neighboor(uint64_t id, GraphEdge *edge); + void add_neighboor(uint64_t id, uint64_t dst_id, float weight); // std::unordered_map::iterator> std::unordered_map get_node_location() { return node_location; diff --git a/paddle/fluid/distributed/table/graph_edge.cc b/paddle/fluid/distributed/table/graph_edge.cc new file mode 100644 index 0000000000000..56ce7852484ec --- /dev/null +++ b/paddle/fluid/distributed/table/graph_edge.cc @@ -0,0 +1,30 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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 "paddle/fluid/distributed/table/graph_edge.h" +#include +namespace paddle { +namespace distributed { + +void GraphEdgeBlob::add_edge(uint64_t id, float weight=1){ + id_arr.push_back(id); +} + +void WeightedGraphEdgeBlob::add_edge(uint64_t id, float weight=1){ + id_arr.push_back(id); + weight_arr.push_back(weight); +} + +} +} diff --git a/paddle/fluid/distributed/table/graph_edge.h b/paddle/fluid/distributed/table/graph_edge.h new file mode 100644 index 0000000000000..1dbfa58dcbf1a --- /dev/null +++ b/paddle/fluid/distributed/table/graph_edge.h @@ -0,0 +1,47 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once +#include +#include +namespace paddle { +namespace distributed { + + +class GraphEdgeBlob { +public: + GraphEdgeBlob() {} + virtual ~GraphEdgeBlob() {} + const size_t size() {return id_arr.size();} + //virtual void add_edge(GraphEdge e); + virtual void add_edge(uint64_t id, float weight); + const uint64_t get_id(int idx) { return id_arr[idx]; } + virtual const float get_weight(int idx) { return 1; } +protected: + std::vector id_arr; +}; + +class WeightedGraphEdgeBlob: public GraphEdgeBlob{ +public: + WeightedGraphEdgeBlob() {} + virtual ~WeightedGraphEdgeBlob() {} + //virtual void add_edge(WeightedGraphEdge e); + virtual void add_edge(uint64_t id, float weight); + virtual const float get_weight(int idx) { return weight_arr[idx]; } +protected: + std::vector weight_arr; +}; + +} +} diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 4d557ca14182b..4e4e210cccec6 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -16,21 +16,41 @@ #include namespace paddle { namespace distributed { + + +GraphNode::~GraphNode() { + if (sampler != nullptr){ + delete sampler; + sampler = nullptr; + } + if (edges != nullptr){ + delete edges; + edges = nullptr; + } +} + int GraphNode::weight_size = sizeof(float); int GraphNode::id_size = sizeof(uint64_t); int GraphNode::int_size = sizeof(int); int GraphNode::get_size(bool need_feature) { return id_size + int_size + (need_feature ? feature.size() : 0); } +void GraphNode::build_edges(bool is_weighted) { + if (edges == nullptr){ + if (is_weighted == true){ + edges = new WeightedGraphEdgeBlob(); + } else { + edges = new GraphEdgeBlob(); + } + } +} void GraphNode::build_sampler(std::string sample_type) { if (sample_type == "random"){ sampler = new RandomSampler(); } else if (sample_type == "weighted"){ sampler = new WeightedSampler(); } - //GraphEdge** arr = edges.data(); - //sampler->build((WeightedObject**)arr, 0, edges.size()); - sampler->build((std::vector*)&edges); + sampler->build(edges); } void GraphNode::to_buffer(char* buffer, bool need_feature) { int size = get_size(need_feature); diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index f5169129d9f47..4d2b866d5d822 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -17,49 +17,33 @@ #include "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { -// enum GraphNodeType { user = 0, item = 1, query = 2, unknown = 3 }; -class GraphEdge : public WeightedObject { - public: - GraphEdge() {} - GraphEdge(uint64_t id, float weight) : id(id), weight(weight) {} - uint64_t get_id() { return id; } - float get_weight() { return weight; } - uint64_t id; - float weight; -}; + class GraphNode { public: - GraphNode() { sampler = NULL; } + GraphNode(): sampler(nullptr), edges(nullptr) { } GraphNode(uint64_t id, std::string feature) - : id(id), feature(feature), sampler(NULL) {} - virtual ~GraphNode() {} - std::vector get_graph_edge() { return edges; } + : id(id), feature(feature), sampler(nullptr), edges(nullptr) {} + virtual ~GraphNode(); static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } void set_feature(std::string feature) { this->feature = feature; } std::string get_feature() { return feature; } virtual int get_size(bool need_feature); + virtual void build_edges(bool is_weighted); virtual void build_sampler(std::string sample_type); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual void add_edge(GraphEdge *edge) { edges.push_back(edge); } - std::vector sample_k(int k) { - std::vector v; - if (sampler != NULL) { - auto res = sampler->sample_k(k); - for (auto x : res) { - v.push_back((GraphEdge *)x); - } - } - return v; - } + virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } + std::vector sample_k(int k) { return sampler->sample_k(k); } + uint64_t get_neighbor_id(int idx){return edges->get_id(idx);} + float get_neighbor_weight(int idx){return edges->get_weight(idx);} protected: uint64_t id; std::string feature; Sampler *sampler; - std::vector edges; + GraphEdgeBlob * edges; }; } } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index 707d7ab0ba7d4..9dc9064742dee 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -18,11 +18,11 @@ namespace paddle { namespace distributed { -void RandomSampler::build(std::vector* edges) { +void RandomSampler::build(GraphEdgeBlob* edges) { this->edges = edges; } -std::vector RandomSampler::sample_k(int k) { +std::vector RandomSampler::sample_k(int k) { int n = edges->size(); if (k > n){ k = n; @@ -30,15 +30,15 @@ std::vector RandomSampler::sample_k(int k) { struct timespec tn; clock_gettime(CLOCK_REALTIME, &tn); srand(tn.tv_nsec); - std::vector sample_result; + std::vector sample_result; std::unordered_map replace_map; while(k--){ int rand_int = rand() % n; auto iter = replace_map.find(rand_int); if(iter == replace_map.end()){ - sample_result.push_back(edges->at(rand_int)); + sample_result.push_back(rand_int); }else{ - sample_result.push_back(edges->at(iter->second)); + sample_result.push_back(iter->second); } iter = replace_map.find(n - 1); @@ -55,7 +55,7 @@ std::vector RandomSampler::sample_k(int k) { WeightedSampler::WeightedSampler(){ left = nullptr; right = nullptr; - object = nullptr; + edges = nullptr; } WeightedSampler::~WeightedSampler() { @@ -69,7 +69,7 @@ WeightedSampler::~WeightedSampler() { } } -void WeightedSampler::build(std::vector* edges) { +void WeightedSampler::build(GraphEdgeBlob* edges) { if(left != nullptr){ delete left; left = nullptr; @@ -78,32 +78,32 @@ void WeightedSampler::build(std::vector* edges) { delete right; right = nullptr; } - WeightedObject** v = edges->data(); - return build_one(v, 0, edges->size()); + return build_one((WeightedGraphEdgeBlob*)edges, 0, edges->size()); } -void WeightedSampler::build_one(WeightedObject **v, int start, int end) { +void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, int end) { count = 0; + this->edges = edges; if (start + 1 == end) { left = right = nullptr; - weight = v[start]->get_weight(); - object = v[start]; + idx = start; count = 1; + weight = edges->get_weight(idx); } else { left = new WeightedSampler(); right = new WeightedSampler(); - left->build_one(v, start, start + (end - start) / 2); - right->build_one(v, start + (end - start) / 2, end); + left->build_one(edges, start, start + (end - start) / 2); + right->build_one(edges, start + (end - start) / 2, end); weight = left->weight + right->weight; count = left->count + right->count; } } -std::vector WeightedSampler::sample_k(int k) { +std::vector WeightedSampler::sample_k(int k) { if (k > count) { k = count; } - std::vector sample_result; + std::vector sample_result; float subtract; std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; @@ -118,7 +118,8 @@ std::vector WeightedSampler::sample_k(int k) { } return sample_result; } -WeightedObject *WeightedSampler::sample( + +int WeightedSampler::sample( float query_weight, std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, @@ -127,24 +128,24 @@ WeightedObject *WeightedSampler::sample( subtract_weight_map[this] = weight; subtract = weight; subtract_count_map[this] = 1; - return object; + return idx; } int left_count = left->count - subtract_count_map[left]; int right_count = right->count - subtract_count_map[right]; float left_subtract = subtract_weight_map[left]; - WeightedObject *return_id; + int return_idx; if (right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight) { - return_id = left->sample(query_weight, subtract_weight_map, + return_idx = left->sample(query_weight, subtract_weight_map, subtract_count_map, subtract); } else { - return_id = + return_idx = right->sample(query_weight - (left->weight - left_subtract), subtract_weight_map, subtract_count_map, subtract); } subtract_weight_map[this] += subtract; subtract_count_map[this]++; - return return_id; + return return_idx; } } } diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 4a7d08c1404a9..52a0395202ba4 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -16,30 +16,23 @@ #include #include #include +#include "paddle/fluid/distributed/table/graph_edge.h" namespace paddle { namespace distributed { -class WeightedObject { - public: - WeightedObject() {} - virtual ~WeightedObject() {} - virtual uint64_t get_id() = 0; - virtual float get_weight() = 0; -}; - class Sampler { public: virtual ~Sampler() {} - virtual void build(std::vector* edges) = 0; - virtual std::vector sample_k(int k) = 0; + virtual void build(GraphEdgeBlob* edges) = 0; + virtual std::vector sample_k(int k) = 0; }; class RandomSampler: public Sampler { public: virtual ~RandomSampler() {} - virtual void build(std::vector* edges); - virtual std::vector sample_k(int k); - std::vector* edges; + virtual void build(GraphEdgeBlob* edges); + virtual std::vector sample_k(int k); + GraphEdgeBlob* edges; }; class WeightedSampler: public Sampler { @@ -47,15 +40,16 @@ class WeightedSampler: public Sampler { WeightedSampler(); virtual ~WeightedSampler(); WeightedSampler *left, *right; - WeightedObject *object; - int count; float weight; - virtual void build(std::vector* edges); - virtual void build_one(WeightedObject **v, int start, int end); - virtual std::vector sample_k(int k); + int count; + int idx; + WeightedGraphEdgeBlob * edges; + virtual void build(GraphEdgeBlob* edges); + virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); + virtual std::vector sample_k(int k); private: - WeightedObject *sample( + int sample( float query_weight, std::unordered_map &subtract_weight_map, std::unordered_map &subtract_count_map, From 3ace54598f52fd5254ec60b05485e4260b07ed42 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Sat, 20 Mar 2021 12:12:37 +0800 Subject: [PATCH 050/106] WeightedGraphEdgeBlob to GraphEdgeBlob --- paddle/fluid/distributed/table/graph_edge.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/paddle/fluid/distributed/table/graph_edge.h b/paddle/fluid/distributed/table/graph_edge.h index 1dbfa58dcbf1a..2a838f0b763e3 100644 --- a/paddle/fluid/distributed/table/graph_edge.h +++ b/paddle/fluid/distributed/table/graph_edge.h @@ -24,7 +24,6 @@ class GraphEdgeBlob { GraphEdgeBlob() {} virtual ~GraphEdgeBlob() {} const size_t size() {return id_arr.size();} - //virtual void add_edge(GraphEdge e); virtual void add_edge(uint64_t id, float weight); const uint64_t get_id(int idx) { return id_arr[idx]; } virtual const float get_weight(int idx) { return 1; } @@ -36,7 +35,6 @@ class WeightedGraphEdgeBlob: public GraphEdgeBlob{ public: WeightedGraphEdgeBlob() {} virtual ~WeightedGraphEdgeBlob() {} - //virtual void add_edge(WeightedGraphEdge e); virtual void add_edge(uint64_t id, float weight); virtual const float get_weight(int idx) { return weight_arr[idx]; } protected: From 9e3b8b924469054b76635c0a248bd218375488ec Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Sat, 20 Mar 2021 12:14:22 +0800 Subject: [PATCH 051/106] WeightedGraphEdgeBlob to GraphEdgeBlob --- paddle/fluid/distributed/table/weighted_sampler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 52a0395202ba4..82e5109e8f34b 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -43,7 +43,7 @@ class WeightedSampler: public Sampler { float weight; int count; int idx; - WeightedGraphEdgeBlob * edges; + GraphEdgeBlob * edges; virtual void build(GraphEdgeBlob* edges); virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); virtual std::vector sample_k(int k); From 7d2dd6e4ad2aa6b1ec51e30b5d19a819ca591f20 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sun, 21 Mar 2021 07:56:42 +0000 Subject: [PATCH 052/106] pybind sample nodes api --- .../distributed/service/graph_py_service.cc | 21 +++++++-- .../distributed/service/graph_py_service.h | 9 ++-- .../fluid/distributed/test/graph_node_test.cc | 46 +++++++++---------- paddle/fluid/pybind/fleet_py.cc | 3 +- 4 files changed, 47 insertions(+), 32 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 4a7d2706fef39..d37eb289f66ea 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -169,8 +169,7 @@ void GraphPyClient::load_edge_file(std::string name, std::string filepath, if (reverse) { // 'e<' means load edges from $2 to $1 params += "<"; - } - else { + } else { // 'e>' means load edges from $1 to $2 params += ">"; } @@ -193,8 +192,9 @@ void GraphPyClient::load_node_file(std::string name, std::string filepath) { } } std::vector>> -GraphPyClient::batch_sample_k(std::string name, std::vector node_ids, - int sample_size) { +GraphPyClient::batch_sample_neighboors(std::string name, + std::vector node_ids, + int sample_size) { std::vector>> v; if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; @@ -204,6 +204,19 @@ GraphPyClient::batch_sample_k(std::string name, std::vector node_ids, } return v; } + +std::vector GraphPyClient::random_sample_nodes(std::string name, + int server_index, + int sample_size) { + std::vector v; + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + worker_ptr->random_sample_nodes(table_id, server_index, sample_size, v); + status.wait(); + } + return v; +} std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, int start, int size) { diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 137f854b39e11..9283618dc9533 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -109,7 +109,8 @@ class GraphPyServer : public GraphPyService { class GraphPyClient : public GraphPyService { public: void set_up(std::string ips_str, int shard_num, - std::vector node_types, std::vector edge_types, int client_id) { + std::vector node_types, + std::vector edge_types, int client_id) { set_client_id(client_id); GraphPyService::set_up(ips_str, shard_num, node_types, edge_types); } @@ -121,8 +122,10 @@ class GraphPyClient : public GraphPyService { int get_client_id() { return client_id; } void set_client_id(int client_id) { this->client_id = client_id; } void start_client(); - std::vector > > batch_sample_k( - std::string name, std::vector node_ids, int sample_size); + std::vector>> batch_sample_neighboors( + std::string name, std::vector node_ids, int sample_size); + std::vector random_sample_nodes(std::string name, int server_index, + int sample_size); std::vector pull_graph_list(std::string name, int server_index, int start, int size); ::paddle::distributed::PSParameter GetWorkerProto(); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 56c3359a20857..e10630b016600 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -137,29 +137,20 @@ std::string edges[] = { char edge_file_name[] = "edges.txt"; std::string nodes[] = { - std::string("user\t37\t0.34"), - std::string("user\t96\t0.31"), - std::string("user\t59\t0.11"), - std::string("user\t97\t0.11"), - std::string("item\t45\t0.21"), - std::string("item\t145\t0.21"), - std::string("item\t112\t0.21"), - std::string("item\t48\t0.21"), - std::string("item\t247\t0.21"), - std::string("item\t111\t0.21"), - std::string("item\t45\t0.21"), - std::string("item\t145\t0.21"), - std::string("item\t122\t0.21"), - std::string("item\t48\t0.21"), - std::string("item\t247\t0.21"), - std::string("item\t111\t0.21")}; + std::string("user\t37\t0.34"), std::string("user\t96\t0.31"), + std::string("user\t59\t0.11"), std::string("user\t97\t0.11"), + std::string("item\t45\t0.21"), std::string("item\t145\t0.21"), + std::string("item\t112\t0.21"), std::string("item\t48\t0.21"), + std::string("item\t247\t0.21"), std::string("item\t111\t0.21"), + std::string("item\t45\t0.21"), std::string("item\t145\t0.21"), + std::string("item\t122\t0.21"), std::string("item\t48\t0.21"), + std::string("item\t247\t0.21"), std::string("item\t111\t0.21")}; char node_file_name[] = "nodes.txt"; - void prepare_file(char file_name[], bool load_edge) { std::ofstream ofile; ofile.open(file_name); - if(load_edge) { + if (load_edge) { for (auto x : edges) { ofile << x << std::endl; } @@ -360,7 +351,8 @@ void RunBrpcPushSparse() { distributed::GraphPyClient client1, client2; std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; std::vector edge_types = {std::string("user2item")}; - std::vector node_types = {std::string("user"), std::string("item")}; + std::vector node_types = {std::string("user"), + std::string("item")}; server1.set_up(ips_str, 127, node_types, edge_types, 0); server2.set_up(ips_str, 127, node_types, edge_types, 1); client1.set_up(ips_str, 127, node_types, edge_types, 0); @@ -376,7 +368,8 @@ void RunBrpcPushSparse() { std::cout << "started" << std::endl; client1.load_node_file(std::string("user"), std::string(node_file_name)); client1.load_node_file(std::string("item"), std::string(node_file_name)); - client1.load_edge_file(std::string("user2item"), std::string(edge_file_name), 0); + client1.load_edge_file(std::string("user2item"), std::string(edge_file_name), + 0); // client2.load_edge_file(std::string("user2item"), std::string(file_name), // 0); nodes.clear(); @@ -385,11 +378,11 @@ void RunBrpcPushSparse() { for (auto g : nodes) { std::cout << "node_ids: " << g.get_id() << std::endl; } - std::cout << "node_ids: " << nodes[0].get_id() << std::endl; + std::cout << "node_ids: " << nodes[0].get_id() << std::endl; ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); - vs = client1.batch_sample_k(std::string("user2item"), - std::vector(1, 96), 4); + vs = client1.batch_sample_neighboors(std::string("user2item"), + std::vector(1, 96), 4); ASSERT_EQ(vs[0].size(), 3); std::cout << "batch sample result" << std::endl; for (auto p : vs[0]) { @@ -398,8 +391,13 @@ void RunBrpcPushSparse() { std::vector node_ids; node_ids.push_back(96); node_ids.push_back(37); - vs = client1.batch_sample_k(std::string("user2item"), node_ids, 4); + vs = client1.batch_sample_neighboors(std::string("user2item"), node_ids, 4); + ASSERT_EQ(vs.size(), 2); + std::vector nodes_ids = client2.random_sample_nodes("user", 0, 6); + ASSERT_EQ(nodes_ids.size(), 2); + ASSERT_EQ(true, (nodes_ids[0] == 59 && nodes_ids[1] == 37) || + (nodes_ids[0] == 37 && nodes_ids[1] == 59)); // to test in python,try this: // from paddle.fluid.core import GraphPyService // ips_str = "127.0.0.1:4211;127.0.0.1:4212" diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 57e47461caf68..3132403195260 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -181,7 +181,8 @@ void BindGraphPyClient(py::module* m) { .def("set_up", &GraphPyClient::set_up) .def("pull_graph_list", &GraphPyClient::pull_graph_list) .def("start_client", &GraphPyClient::start_client) - .def("batch_sample_k", &GraphPyClient::batch_sample_k); + .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) + .def("random_sample_nodes", &GraphPyClient::random_sample_nodes); } } // end namespace pybind From 09667d1fe6d9fccf086f9e2541a5857f48ce0909 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Mon, 22 Mar 2021 05:56:35 +0000 Subject: [PATCH 053/106] pull nodes with step --- .../distributed/service/graph_brpc_client.cc | 3 +- .../distributed/service/graph_brpc_client.h | 2 +- .../distributed/service/graph_brpc_server.cc | 7 ++- .../distributed/service/graph_py_service.cc | 7 ++- .../distributed/service/graph_py_service.h | 2 +- paddle/fluid/distributed/service/ps_client.h | 2 +- .../distributed/table/common_graph_table.cc | 59 ++++++++----------- .../distributed/table/common_graph_table.h | 5 +- paddle/fluid/distributed/table/graph_edge.h | 25 ++++---- paddle/fluid/distributed/table/table.h | 3 +- .../fluid/distributed/test/graph_node_test.cc | 6 +- 11 files changed, 58 insertions(+), 63 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 2c9ee2b9037b7..1b54ce986d9c2 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -173,7 +173,7 @@ std::future GraphBrpcClient::random_sample_nodes( return fut; } std::future GraphBrpcClient::pull_graph_list( - uint32_t table_id, int server_index, int start, int size, + uint32_t table_id, int server_index, int start, int size, int step, std::vector &res) { DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; @@ -207,6 +207,7 @@ std::future GraphBrpcClient::pull_graph_list( closure->request(0)->set_client_id(_client_id); closure->request(0)->add_params((char *)&start, sizeof(int)); closure->request(0)->add_params((char *)&size, sizeof(int)); + closure->request(0)->add_params((char *)&step, sizeof(int)); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 8f14e646050c3..e762409fbeeda 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -41,7 +41,7 @@ class GraphBrpcClient : public BrpcPsClient { std::vector>> &res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, - int size, + int size, int step, std::vector &res); virtual std::future random_sample_nodes(uint32_t table_id, int server_index, diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index cd1c2330a7b85..60d6bc203a074 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -260,16 +260,17 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, PsResponseMessage &response, brpc::Controller *cntl) { CHECK_TABLE_EXIST(table, request, response) - if (request.params_size() < 2) { + if (request.params_size() < 3) { set_response_code(response, -1, - "pull_graph_list request requires at least 2 arguments"); + "pull_graph_list request requires at least 3 arguments"); return 0; } int start = *(int *)(request.params(0).c_str()); int size = *(int *)(request.params(1).c_str()); + int step = *(int *)(request.params(2).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size, true); + table->pull_graph_list(start, size, buffer, actual_size, step, true); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index d37eb289f66ea..752dd0bcba2f0 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -219,12 +219,13 @@ std::vector GraphPyClient::random_sample_nodes(std::string name, } std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, - int start, int size) { + int start, int size, + int step) { std::vector res; if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; - auto status = - worker_ptr->pull_graph_list(table_id, server_index, start, size, res); + auto status = worker_ptr->pull_graph_list(table_id, server_index, start, + size, step, res); status.wait(); } return res; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 9283618dc9533..7452c99c07f60 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -127,7 +127,7 @@ class GraphPyClient : public GraphPyService { std::vector random_sample_nodes(std::string name, int server_index, int sample_size); std::vector pull_graph_list(std::string name, int server_index, - int start, int size); + int start, int size, int step = 1); ::paddle::distributed::PSParameter GetWorkerProto(); protected: diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index d3274b4a57334..1b61e35641322 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -166,7 +166,7 @@ class PSClient { } virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, - int size, + int size, int step, std::vector &res) { LOG(FATAL) << "Did not implement"; std::promise promise; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index d2c5a095a305e..2e8cf6593c253 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -23,10 +23,10 @@ namespace paddle { namespace distributed { -std::vector GraphShard::get_batch(int start, int total_size) { +std::vector GraphShard::get_batch(int start, int end, int step) { if (start < 0) start = 0; std::vector res; - for (int pos = start; pos < start + total_size; pos++) { + for (int pos = start; pos < std::min(end, (int)bucket.size()); pos += step) { res.push_back(bucket[pos]); } return res; @@ -52,15 +52,14 @@ GraphNode *GraphShard::find_node(uint64_t id) { } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { - bool load_edge = (param[0] == 'e'); bool load_node = (param[0] == 'n'); if (load_edge) { bool reverse_edge = (param[1] == '<'); return this->load_edges(path, reverse_edge); } - if (load_node){ - std::string node_type = param.substr(1); + if (load_node) { + std::string node_type = param.substr(1); return this->load_nodes(path, node_type); } } @@ -125,18 +124,17 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { std::string nt = values[0]; if (nt != node_type) { - continue; + continue; } std::vector feature; for (size_t slice = 2; slice < values.size(); slice++) { feature.push_back(values[slice]); } size_t index = shard_id - shard_start; - if(feature.size() > 0) { - shards[index].add_node(id, paddle::string::join_strings(feature, '\t')); - } - else { - shards[index].add_node(id, std::string("")); + if (feature.size() > 0) { + shards[index].add_node(id, paddle::string::join_strings(feature, '\t')); + } else { + shards[index].add_node(id, std::string("")); } } } @@ -188,7 +186,8 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { for (auto &shard : shards) { auto bucket = shard.get_bucket(); for (int i = 0; i < bucket.size(); i++) { - bucket[i]->build_sampler(sample_type); } + bucket[i]->build_sampler(sample_type); + } } return 0; } @@ -315,37 +314,27 @@ int GraphTable::random_sample_neighboors( } int32_t GraphTable::pull_graph_list(int start, int total_size, std::unique_ptr &buffer, - int &actual_size, bool need_feature) { + int &actual_size, bool need_feature, + int step) { if (start < 0) start = 0; int size = 0, cur_size; - if (total_size <= 0) { - actual_size = 0; - return 0; - } std::vector>> tasks; - for (size_t i = 0; i < shards.size(); i++) { + for (size_t i = 0; i < shards.size() && total_size > 0; i++) { cur_size = shards[i].get_size(); if (size + cur_size <= start) { size += cur_size; continue; } - if (size + cur_size - start >= total_size) { - tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( - [this, i, start, size, total_size]() -> std::vector { - return this->shards[i].get_batch(start - size, total_size); - })); - break; - } else { - tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( - [this, i, start, size, total_size, - cur_size]() -> std::vector { - return this->shards[i].get_batch(start - size, - size + cur_size - start); - })); - total_size -= size + cur_size - start; - size += cur_size; - start = size; - } + int count = std::min(1 + (size + cur_size - start - 1) / step, total_size); + int end = start + (count - 1) * step + 1; + tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( + [this, i, start, end, step, size]() -> std::vector { + + return this->shards[i].get_batch(start - size, end - size, step); + })); + start += count * step; + total_size -= count; + size += cur_size; } for (size_t i = 0; i < tasks.size(); ++i) { tasks[i].wait(); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 106dcaaac2b00..d298dc963800e 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -45,7 +45,7 @@ class GraphShard { // bucket.resize(bucket_size); } std::vector &get_bucket() { return bucket; } - std::vector get_batch(int start, int total_size); + std::vector get_batch(int start, int end, int step); // int init_bucket_size(int shard_num) { // for (int i = bucket_low_bound;; i++) { // if (gcd(i, shard_num) == 1) return i; @@ -78,7 +78,8 @@ class GraphTable : public SparseTable { virtual ~GraphTable() {} virtual int32_t pull_graph_list(int start, int size, std::unique_ptr &buffer, - int &actual_size, bool need_feature); + int &actual_size, bool need_feature, + int step); virtual int32_t random_sample_neighboors( uint64_t *node_ids, int sample_size, diff --git a/paddle/fluid/distributed/table/graph_edge.h b/paddle/fluid/distributed/table/graph_edge.h index 2a838f0b763e3..3dfe5a6f357a7 100644 --- a/paddle/fluid/distributed/table/graph_edge.h +++ b/paddle/fluid/distributed/table/graph_edge.h @@ -13,33 +13,34 @@ // limitations under the License. #pragma once -#include +#include #include +#include namespace paddle { namespace distributed { - class GraphEdgeBlob { -public: + public: GraphEdgeBlob() {} virtual ~GraphEdgeBlob() {} - const size_t size() {return id_arr.size();} + size_t size() { return id_arr.size(); } virtual void add_edge(uint64_t id, float weight); - const uint64_t get_id(int idx) { return id_arr[idx]; } - virtual const float get_weight(int idx) { return 1; } -protected: + uint64_t get_id(int idx) { return id_arr[idx]; } + virtual float get_weight(int idx) { return 1; } + + protected: std::vector id_arr; }; -class WeightedGraphEdgeBlob: public GraphEdgeBlob{ -public: +class WeightedGraphEdgeBlob : public GraphEdgeBlob { + public: WeightedGraphEdgeBlob() {} virtual ~WeightedGraphEdgeBlob() {} virtual void add_edge(uint64_t id, float weight); - virtual const float get_weight(int idx) { return weight_arr[idx]; } -protected: + virtual float get_weight(int idx) { return weight_arr[idx]; } + + protected: std::vector weight_arr; }; - } } diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 65a3191c50255..40d2abb6c86ae 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -90,7 +90,8 @@ class Table { // only for graph table virtual int32_t pull_graph_list(int start, int total_size, std::unique_ptr &buffer, - int &actual_size, bool need_feature) { + int &actual_size, bool need_feature, + int step = 1) { return 0; } // only for graph table diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index e10630b016600..efee0d9441ef2 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -335,12 +335,12 @@ void RunBrpcPushSparse() { ASSERT_EQ(0, vs[0].size()); std::vector nodes; - pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); + pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, 1, nodes); pull_status.wait(); ASSERT_EQ(nodes.size(), 1); ASSERT_EQ(nodes[0].get_id(), 37); nodes.clear(); - pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, nodes); + pull_status = worker_ptr_->pull_graph_list(0, 0, 1, 4, 1, nodes); pull_status.wait(); ASSERT_EQ(nodes.size(), 1); ASSERT_EQ(nodes[0].get_id(), 59); @@ -373,7 +373,7 @@ void RunBrpcPushSparse() { // client2.load_edge_file(std::string("user2item"), std::string(file_name), // 0); nodes.clear(); - nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4); + nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4, 1); for (auto g : nodes) { std::cout << "node_ids: " << g.get_id() << std::endl; From f861faaadae582e6a2bb58bf55a4c0378820924b Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 22 Mar 2021 14:46:31 +0800 Subject: [PATCH 054/106] fixed pull_graph_list bug; add test for pull_graph_list by step --- .../distributed/service/graph_brpc_server.cc | 2 +- .../fluid/distributed/test/graph_node_test.cc | 28 +++++++++++++++++-- 2 files changed, 26 insertions(+), 4 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 60d6bc203a074..765c4e9254254 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -270,7 +270,7 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, int step = *(int *)(request.params(2).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size, step, true); + table->pull_graph_list(start, size, buffer, actual_size, true, step); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index efee0d9441ef2..2ba5946cc443f 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -142,9 +142,9 @@ std::string nodes[] = { std::string("item\t45\t0.21"), std::string("item\t145\t0.21"), std::string("item\t112\t0.21"), std::string("item\t48\t0.21"), std::string("item\t247\t0.21"), std::string("item\t111\t0.21"), - std::string("item\t45\t0.21"), std::string("item\t145\t0.21"), - std::string("item\t122\t0.21"), std::string("item\t48\t0.21"), - std::string("item\t247\t0.21"), std::string("item\t111\t0.21")}; + std::string("item\t46\t0.21"), std::string("item\t146\t0.21"), + std::string("item\t122\t0.21"), std::string("item\t49\t0.21"), + std::string("item\t248\t0.21"), std::string("item\t113\t0.21")}; char node_file_name[] = "nodes.txt"; void prepare_file(char file_name[], bool load_edge) { @@ -373,6 +373,7 @@ void RunBrpcPushSparse() { // client2.load_edge_file(std::string("user2item"), std::string(file_name), // 0); nodes.clear(); + nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4, 1); for (auto g : nodes) { @@ -381,6 +382,27 @@ void RunBrpcPushSparse() { std::cout << "node_ids: " << nodes[0].get_id() << std::endl; ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); + + // Test Pull by step + + + std::unordered_set count_item_nodes; + // pull by step 2 + for(int test_step=1; test_step < 4 ; test_step ++) { + count_item_nodes.clear(); + std::cout << "check pull graph list by step " << test_step << std::endl; + for(int server_id = 0; server_id < 2; server_id ++) { + for(int start_step = 0; start_step < test_step; start_step ++) { + nodes = client1.pull_graph_list(std::string("item"), server_id, start_step, 12, test_step); + for (auto g : nodes) { + count_item_nodes.insert(g.get_id()); + } + nodes.clear(); + } + } + ASSERT_EQ(count_item_nodes.size(), 12); + } + vs = client1.batch_sample_neighboors(std::string("user2item"), std::vector(1, 96), 4); ASSERT_EQ(vs[0].size(), 3); From 95676f74e52774754a6c03468021ea23454e436f Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 22 Mar 2021 20:03:31 +0800 Subject: [PATCH 055/106] add graph table;name --- .../distributed/service/graph_py_service.cc | 100 +++++++++++++++++- .../distributed/service/graph_py_service.h | 27 ++++- .../distributed/table/common_graph_table.cc | 16 +++ .../distributed/table/common_graph_table.h | 7 ++ 4 files changed, 146 insertions(+), 4 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 752dd0bcba2f0..2fb10bff7df51 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -32,10 +32,25 @@ std::vector GraphPyService::split(std::string& str, return res; } + +void GraphPyService::add_table_feat_conf(std::string table_name, + std::string feat_name, + std::string feat_dtype, + int32_t feat_shape) { + if(this->table_id_map.count(table_name)) { + this->table_feat_conf_table_name.push_back(table_name); + this->table_feat_conf_feat_name.push_back(feat_name); + this->table_feat_conf_feat_dtype.push_back(feat_dtype); + this->table_feat_conf_feat_shape.push_back(feat_shape); + } +} + + void GraphPyService::set_up(std::string ips_str, int shard_num, std::vector node_types, std::vector edge_types) { set_shard_num(shard_num); + set_num_node_types(node_types.size()); // set_client_Id(client_id); // set_rank(rank); @@ -121,7 +136,31 @@ ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { VLOG(0) << " make a new table " << tuple.second; ::paddle::distributed::TableParameter* sparse_table_proto = downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + std::vector feat_name; + std::vector feat_dtype; + std::vector feat_shape; + for(size_t i=0; itable_feat_conf_table_name.size(); i++) { + if(tuple.first == table_feat_conf_table_name[i]) { + feat_name.push_back(table_feat_conf_feat_name[i]); + feat_dtype.push_back(table_feat_conf_feat_dtype[i]); + feat_shape.push_back(table_feat_conf_feat_shape[i]); + } + } + std::string table_type; + if(tuple.second < this->num_node_types) { + table_type = "node"; + } + else { + table_type = "edge"; + } + + GetDownpourSparseTableProto(sparse_table_proto, + tuple.second, + tuple.first, + table_type, + feat_name, + feat_dtype, + feat_shape); } return server_fleet_desc; @@ -137,11 +176,38 @@ ::paddle::distributed::PSParameter GraphPyClient::GetWorkerProto() { worker_proto->mutable_downpour_worker_param(); for (auto& tuple : this->table_id_map) { + VLOG(0) << " make a new table " << tuple.second; ::paddle::distributed::TableParameter* worker_sparse_table_proto = downpour_worker_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(worker_sparse_table_proto, tuple.second); + std::vector feat_name; + std::vector feat_dtype; + std::vector feat_shape; + for(size_t i=0; itable_feat_conf_table_name.size(); i++) { + if(tuple.first == table_feat_conf_table_name[i]) { + feat_name.push_back(table_feat_conf_feat_name[i]); + feat_dtype.push_back(table_feat_conf_feat_dtype[i]); + feat_shape.push_back(table_feat_conf_feat_shape[i]); + } + } + std::string table_type; + if(tuple.second < this->num_node_types) { + table_type = "node"; + } + else { + table_type = "edge"; + } + + GetDownpourSparseTableProto(worker_sparse_table_proto, + tuple.second, + tuple.first, + table_type, + feat_name, + feat_dtype, + feat_shape); } + + ::paddle::distributed::ServerParameter* server_proto = worker_fleet_desc.mutable_server_param(); ::paddle::distributed::DownpourServerParameter* downpour_server_proto = @@ -155,11 +221,38 @@ ::paddle::distributed::PSParameter GraphPyClient::GetWorkerProto() { server_service_proto->set_server_thread_num(12); for (auto& tuple : this->table_id_map) { + VLOG(0) << " make a new table " << tuple.second; ::paddle::distributed::TableParameter* sparse_table_proto = downpour_server_proto->add_downpour_table_param(); - GetDownpourSparseTableProto(sparse_table_proto, tuple.second); + std::vector feat_name; + std::vector feat_dtype; + std::vector feat_shape; + for(size_t i=0; itable_feat_conf_table_name.size(); i++) { + if(tuple.first == table_feat_conf_table_name[i]) { + feat_name.push_back(table_feat_conf_feat_name[i]); + feat_dtype.push_back(table_feat_conf_feat_dtype[i]); + feat_shape.push_back(table_feat_conf_feat_shape[i]); + } + } + std::string table_type; + if(tuple.second < this->num_node_types) { + table_type = "node"; + } + else { + table_type = "edge"; + } + + GetDownpourSparseTableProto(sparse_table_proto, + tuple.second, + tuple.first, + table_type, + feat_name, + feat_dtype, + feat_shape); } + + return worker_fleet_desc; } void GraphPyClient::load_edge_file(std::string name, std::string filepath, @@ -232,3 +325,4 @@ std::vector GraphPyClient::pull_graph_list(std::string name, } } } + diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 7452c99c07f60..813846face348 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -23,6 +23,7 @@ #include // NOLINT #include #include +#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" @@ -47,7 +48,13 @@ class GraphPyService { protected: std::vector server_list, port_list, host_sign_list; int server_size, shard_num; + int num_node_types; std::unordered_map table_id_map; + std::vector table_feat_conf_table_name; + std::vector table_feat_conf_feat_name; + std::vector table_feat_conf_feat_dtype; + std::vector table_feat_conf_feat_shape; + // std::thread *server_thread, *client_thread; // std::shared_ptr pserver_ptr; @@ -65,25 +72,43 @@ class GraphPyService { void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( ::paddle::distributed::TableParameter* sparse_table_proto, - uint32_t table_id) { + uint32_t table_id, + std::string table_name, + std::string table_type, + std::vector feat_name, + std::vector feat_dtype, + std::vector feat_shape) { sparse_table_proto->set_table_id(table_id); sparse_table_proto->set_table_class("GraphTable"); sparse_table_proto->set_shard_num(shard_num); sparse_table_proto->set_type(::paddle::distributed::PS_SPARSE_TABLE); ::paddle::distributed::TableAccessorParameter* accessor_proto = sparse_table_proto->mutable_accessor(); + ::paddle::distributed::CommonAccessorParameter* common_proto = sparse_table_proto->mutable_common(); + // Set GraphTable Parameter + common_proto->set_table_name(table_name); + common_proto->set_name(table_type); + for(size_t i = 0;i < feat_name.size();i ++) { + common_proto->add_params(feat_dtype[i]); + common_proto->add_dims(feat_shape[i]); + common_proto->add_attributes(feat_name[i]); + } + accessor_proto->set_accessor_class("CommMergeAccessor"); } void set_server_size(int server_size) { this->server_size = server_size; } + void set_num_node_types(int num_node_types) { this->num_node_types = num_node_types; } int get_server_size(int server_size) { return server_size; } std::vector split(std::string& str, const char pattern); void set_up(std::string ips_str, int shard_num, std::vector node_types, std::vector edge_types); + + void add_table_feat_conf(std::string node_type, std::string feat_name, std::string feat_dtype, int32_t feat_shape); }; class GraphPyServer : public GraphPyService { public: diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 2e8cf6593c253..7d67775635542 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -372,6 +372,22 @@ int32_t GraphTable::initialize() { and _shard_idx to server rank */ + auto common = _config.common(); + + this->table_name = common.table_name(); + this->table_type = common.name(); + VLOG(0) << " init graph table type " << this->table_type << " table name " << this->table_name; + int feat_conf_size = static_cast(common.attributes().size()); + for(int i=0; ifeat_name.push_back(f_name); + this->feat_shape.push_back(f_shape); + this->feat_dtype.push_back(f_dtype); + VLOG(0) << "init graph table feat conf name:"<< f_name << " shape:" << f_shape << " dtype:" << f_dtype; + } + shard_num = _config.shard_num(); VLOG(0) << "in init graph table shard num = " << shard_num << " shard_idx" << _shard_idx; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index d298dc963800e..8d03b70f69adb 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -123,6 +123,13 @@ class GraphTable : public SparseTable { size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; const int task_pool_size_ = 11; const int random_sample_nodes_ranges = 3; + + std::vector feat_name; + std::vector feat_dtype; + std::vector feat_shape; + std::string table_name; + std::string table_type; + std::vector> _shards_task_pool; }; } From a41ee8b907e7d02b56d794d5d6b525ad0547b9f0 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 22 Mar 2021 20:05:49 +0800 Subject: [PATCH 056/106] add graph table;name --- paddle/fluid/distributed/service/graph_py_service.h | 1 - 1 file changed, 1 deletion(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 813846face348..44147924ba49d 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -23,7 +23,6 @@ #include // NOLINT #include #include -#include #include "google/protobuf/text_format.h" #include "gtest/gtest.h" From 2da46ad049fb67b947795b332a033ccd08c4b9b6 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 22 Mar 2021 20:09:52 +0800 Subject: [PATCH 057/106] add pybind --- paddle/fluid/pybind/fleet_py.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 3132403195260..df3b7ffb17017 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -172,6 +172,7 @@ void BindGraphPyServer(py::module* m) { .def(py::init<>()) .def("start_server", &GraphPyServer::start_server) .def("set_up", &GraphPyServer::set_up); + .def("add_table_feat_conf", &GraphPyServer::add_table_feat_conf); } void BindGraphPyClient(py::module* m) { py::class_(*m, "GraphPyClient") @@ -179,6 +180,7 @@ void BindGraphPyClient(py::module* m) { .def("load_edge_file", &GraphPyClient::load_edge_file) .def("load_node_file", &GraphPyClient::load_node_file) .def("set_up", &GraphPyClient::set_up) + .def("add_table_feat_conf", &GraphPyClient::add_table_feat_conf) .def("pull_graph_list", &GraphPyClient::pull_graph_list) .def("start_client", &GraphPyClient::start_client) .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) From 7d515203fa23d477634f34740f0cd83804a7a92a Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Mon, 22 Mar 2021 20:12:27 +0800 Subject: [PATCH 058/106] add pybind --- paddle/fluid/pybind/fleet_py.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index df3b7ffb17017..866fa5fc624ba 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -171,7 +171,7 @@ void BindGraphPyServer(py::module* m) { py::class_(*m, "GraphPyServer") .def(py::init<>()) .def("start_server", &GraphPyServer::start_server) - .def("set_up", &GraphPyServer::set_up); + .def("set_up", &GraphPyServer::set_up) .def("add_table_feat_conf", &GraphPyServer::add_table_feat_conf); } void BindGraphPyClient(py::module* m) { From c0284fe87d71c16a7599ddb324c9ff0dd94c2dd6 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Tue, 23 Mar 2021 02:30:48 +0800 Subject: [PATCH 059/106] add FeatureNode --- .../distributed/service/graph_brpc_client.cc | 6 +- .../distributed/service/graph_brpc_client.h | 2 +- .../distributed/service/graph_brpc_server.cc | 2 +- .../distributed/service/graph_py_service.cc | 4 +- .../distributed/service/graph_py_service.h | 2 +- paddle/fluid/distributed/service/ps_client.h | 2 +- .../distributed/table/common_graph_table.cc | 59 +++++----- .../distributed/table/common_graph_table.h | 15 +-- paddle/fluid/distributed/table/graph_node.cc | 104 ++++++++++++++---- paddle/fluid/distributed/table/graph_node.h | 64 ++++++++--- .../fluid/distributed/test/graph_node_test.cc | 8 +- 11 files changed, 187 insertions(+), 81 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 1b54ce986d9c2..47bd70ff5d980 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -174,7 +174,7 @@ std::future GraphBrpcClient::random_sample_nodes( } std::future GraphBrpcClient::pull_graph_list( uint32_t table_id, int server_index, int start, int size, int step, - std::vector &res) { + std::vector &res) { DownpourBrpcClosure *closure = new DownpourBrpcClosure(1, [&](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; @@ -190,9 +190,9 @@ std::future GraphBrpcClient::pull_graph_list( io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); int index = 0; while (index < bytes_size) { - GraphNode node; + FeatureNode node; node.recover_from_buffer(buffer + index); - index += node.get_size(true); + index += node.get_size(false); res.push_back(node); } } diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index e762409fbeeda..87e481d578a84 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -42,7 +42,7 @@ class GraphBrpcClient : public BrpcPsClient { virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, int step, - std::vector &res); + std::vector &res); virtual std::future random_sample_nodes(uint32_t table_id, int server_index, int sample_size, diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 765c4e9254254..2f619fdb2a661 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -270,7 +270,7 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, int step = *(int *)(request.params(2).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size, true, step); + table->pull_graph_list(start, size, buffer, actual_size, false, step); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 2fb10bff7df51..84ceb102a1e0f 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -310,11 +310,11 @@ std::vector GraphPyClient::random_sample_nodes(std::string name, } return v; } -std::vector GraphPyClient::pull_graph_list(std::string name, +std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, int start, int size, int step) { - std::vector res; + std::vector res; if (this->table_id_map.count(name)) { uint32_t table_id = this->table_id_map[name]; auto status = worker_ptr->pull_graph_list(table_id, server_index, start, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 44147924ba49d..beab26085058b 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -150,7 +150,7 @@ class GraphPyClient : public GraphPyService { std::string name, std::vector node_ids, int sample_size); std::vector random_sample_nodes(std::string name, int server_index, int sample_size); - std::vector pull_graph_list(std::string name, int server_index, + std::vector pull_graph_list(std::string name, int server_index, int start, int size, int step = 1); ::paddle::distributed::PSParameter GetWorkerProto(); diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 1b61e35641322..1b50a0c37098d 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -167,7 +167,7 @@ class PSClient { virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, int step, - std::vector &res) { + std::vector &res) { LOG(FATAL) << "Did not implement"; std::promise promise; std::future fut = promise.get_future(); diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 7d67775635542..7774a402cb49e 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -23,9 +23,9 @@ namespace paddle { namespace distributed { -std::vector GraphShard::get_batch(int start, int end, int step) { +std::vector GraphShard::get_batch(int start, int end, int step) { if (start < 0) start = 0; - std::vector res; + std::vector res; for (int pos = start; pos < std::min(end, (int)bucket.size()); pos += step) { res.push_back(bucket[pos]); } @@ -34,21 +34,29 @@ std::vector GraphShard::get_batch(int start, int end, int step) { size_t GraphShard::get_size() { return bucket.size(); } -GraphNode *GraphShard::add_node(uint64_t id, std::string feature) { - if (node_location.find(id) != node_location.end()) - return bucket[node_location[id]]; - node_location[id] = bucket.size(); - bucket.push_back(new GraphNode(id, feature)); - return bucket.back(); +GraphNode *GraphShard::add_graph_node(uint64_t id) { + if (node_location.find(id) == node_location.end()){ + node_location[id] = bucket.size(); + bucket.push_back(new GraphNode(id)); + } + return (GraphNode*)bucket[node_location[id]]; +} + +FeatureNode *GraphShard::add_feature_node(uint64_t id) { + if (node_location.find(id) == node_location.end()){ + node_location[id] = bucket.size(); + bucket.push_back(new FeatureNode(id)); + } + return (FeatureNode*)bucket[node_location[id]]; } void GraphShard::add_neighboor(uint64_t id, uint64_t dst_id, float weight) { - add_node(id, std::string(""))->add_edge(dst_id, weight); + find_node(id)->add_edge(dst_id, weight); } -GraphNode *GraphShard::find_node(uint64_t id) { +Node *GraphShard::find_node(uint64_t id) { auto iter = node_location.find(id); - return iter == node_location.end() ? NULL : bucket[iter->second]; + return iter == node_location.end() ? nullptr : bucket[iter->second]; } int32_t GraphTable::load(const std::string &path, const std::string ¶m) { @@ -132,9 +140,10 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { } size_t index = shard_id - shard_start; if (feature.size() > 0) { - shards[index].add_node(id, paddle::string::join_strings(feature, '\t')); + // TODO add feature + shards[index].add_feature_node(id); } else { - shards[index].add_node(id, std::string("")); + shards[index].add_feature_node(id); } } } @@ -175,7 +184,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { } size_t index = src_shard_id - shard_start; - shards[index].add_node(src_id, std::string(""))->build_edges(is_weighted); + shards[index].add_graph_node(src_id)->build_edges(is_weighted); shards[index].add_neighboor(src_id, dst_id, weight); } } @@ -192,13 +201,13 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { return 0; } -GraphNode *GraphTable::find_node(uint64_t id) { +Node *GraphTable::find_node(uint64_t id) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { return NULL; } size_t index = shard_id - shard_start; - GraphNode *node = shards[index].find_node(id); + Node *node = shards[index].find_node(id); return node; } uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { @@ -282,7 +291,7 @@ int GraphTable::random_sample_neighboors( int &actual_size = actual_sizes[idx]; tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( [&]() -> int { - GraphNode *node = find_node(node_id); + Node *node = find_node(node_id); if (node == NULL) { actual_size = 0; @@ -290,7 +299,7 @@ int GraphTable::random_sample_neighboors( } std::vector res = node->sample_k(sample_size); actual_size = - res.size() * (GraphNode::id_size + GraphNode::weight_size); + res.size() * (Node::id_size + Node::weight_size); int offset = 0; uint64_t id; float weight; @@ -299,10 +308,10 @@ int GraphTable::random_sample_neighboors( for (int &x : res) { id = node->get_neighbor_id(x); weight = node->get_neighbor_weight(x); - memcpy(buffer_addr + offset, &id, GraphNode::id_size); - offset += GraphNode::id_size; - memcpy(buffer_addr + offset, &weight, GraphNode::weight_size); - offset += GraphNode::weight_size; + memcpy(buffer_addr + offset, &id, Node::id_size); + offset += Node::id_size; + memcpy(buffer_addr + offset, &weight, Node::weight_size); + offset += Node::weight_size; } return 0; })); @@ -318,7 +327,7 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, int step) { if (start < 0) start = 0; int size = 0, cur_size; - std::vector>> tasks; + std::vector>> tasks; for (size_t i = 0; i < shards.size() && total_size > 0; i++) { cur_size = shards[i].get_size(); if (size + cur_size <= start) { @@ -328,7 +337,7 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, int count = std::min(1 + (size + cur_size - start - 1) / step, total_size); int end = start + (count - 1) * step + 1; tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( - [this, i, start, end, step, size]() -> std::vector { + [this, i, start, end, step, size]() -> std::vector { return this->shards[i].get_batch(start - size, end - size, step); })); @@ -340,7 +349,7 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, tasks[i].wait(); } size = 0; - std::vector> res; + std::vector> res; for (size_t i = 0; i < tasks.size(); i++) { res.push_back(tasks[i].get()); for (size_t j = 0; j < res.back().size(); j++) { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 8d03b70f69adb..b13cb69c47b7a 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -44,8 +44,8 @@ class GraphShard { // bucket_size = init_bucket_size(shard_num); // bucket.resize(bucket_size); } - std::vector &get_bucket() { return bucket; } - std::vector get_batch(int start, int end, int step); + std::vector &get_bucket() { return bucket; } + std::vector get_batch(int start, int end, int step); // int init_bucket_size(int shard_num) { // for (int i = bucket_low_bound;; i++) { // if (gcd(i, shard_num) == 1) return i; @@ -59,8 +59,9 @@ class GraphShard { } return res; } - GraphNode *add_node(uint64_t id, std::string feature); - GraphNode *find_node(uint64_t id); + GraphNode *add_graph_node(uint64_t id); + FeatureNode *add_feature_node(uint64_t id); + Node *find_node(uint64_t id); void add_neighboor(uint64_t id, uint64_t dst_id, float weight); // std::unordered_map::iterator> std::unordered_map get_node_location() { @@ -70,7 +71,7 @@ class GraphShard { private: std::unordered_map node_location; int shard_num; - std::vector bucket; + std::vector bucket; }; class GraphTable : public SparseTable { public: @@ -98,8 +99,8 @@ class GraphTable : public SparseTable { int32_t load_edges(const std::string &path, bool reverse); int32_t load_nodes(const std::string &path, std::string node_type); - - GraphNode *find_node(uint64_t id); + + Node *find_node(uint64_t id); virtual int32_t pull_sparse(float *values, const uint64_t *keys, size_t num) { return 0; diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 4e4e210cccec6..334e9b0750cce 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -29,12 +29,42 @@ GraphNode::~GraphNode() { } } -int GraphNode::weight_size = sizeof(float); -int GraphNode::id_size = sizeof(uint64_t); -int GraphNode::int_size = sizeof(int); -int GraphNode::get_size(bool need_feature) { - return id_size + int_size + (need_feature ? feature.size() : 0); +int Node::weight_size = sizeof(float); +int Node::id_size = sizeof(uint64_t); +int Node::int_size = sizeof(int); + +int Node::get_size(bool need_feature) { + return int_size + id_size + int_size; +} + +void Node::to_buffer(char* buffer, bool need_feature) { + int size = get_size(need_feature); + memcpy(buffer, &size, sizeof(int)); + buffer += sizeof(int); + + memcpy(buffer, &id, id_size); + buffer += id_size; + + int feat_num = 0; + memcpy(buffer, &feat_num, sizeof(int)); +} + +void Node::recover_from_buffer(char* buffer) { + buffer += sizeof(int); + memcpy(&id, buffer, id_size); +} + +int FeatureNode::get_size(bool need_feature) { + int size = int_size + id_size + int_size; // buffer_size, id, feat_num + if (need_feature){ + size += feature.size() * int_size; + for (const std::string& fea: feature){ + size += fea.size(); + } + } + return size; } + void GraphNode::build_edges(bool is_weighted) { if (edges == nullptr){ if (is_weighted == true){ @@ -52,28 +82,58 @@ void GraphNode::build_sampler(std::string sample_type) { } sampler->build(edges); } -void GraphNode::to_buffer(char* buffer, bool need_feature) { +void FeatureNode::to_buffer(char* buffer, bool need_feature) { int size = get_size(need_feature); - memcpy(buffer, &size, int_size); + memcpy(buffer, &size, sizeof(int)); + buffer += sizeof(int); + + memcpy(buffer, &id, id_size); + buffer += id_size; + + int feat_num = 0; + int feat_len; if (need_feature) { - memcpy(buffer + int_size, feature.c_str(), feature.size()); - memcpy(buffer + int_size + feature.size(), &id, id_size); + feat_num += feature.size(); + memcpy(buffer, &feat_num, sizeof(int)); + buffer += sizeof(int); + for (int i = 0; i < feat_num; ++i){ + feat_len = feature[i].size(); + memcpy(buffer, &feat_len, sizeof(int)); + buffer += sizeof(int); + memcpy(buffer, feature[i].c_str(), feature[i].size()); + buffer += feature[i].size(); + } + //memcpy(buffer + int_size, feature.c_str(), feature.size()); + //memcpy(buffer + int_size + feature.size(), &id, id_size); } else { - memcpy(buffer + int_size, &id, id_size); + memcpy(buffer, &feat_num, sizeof(int)); } } -void GraphNode::recover_from_buffer(char* buffer) { - int size; - memcpy(&size, buffer, int_size); - int feature_size = size - id_size - int_size; - char str[feature_size + 1]; - memcpy(str, buffer + int_size, feature_size); - str[feature_size] = '\0'; - feature = str; - memcpy(&id, buffer + int_size + feature_size, id_size); - // int int_state; - // memcpy(&int_state, buffer + int_size + feature_size + id_size, enum_size); - // type = GraphNodeType(int_state); +void FeatureNode::recover_from_buffer(char* buffer) { + + int size, feat_num, feat_len; + memcpy(&size, buffer, sizeof(int)); + + buffer += sizeof(int); + memcpy(&id, buffer, id_size); + buffer += id_size; + + memcpy(&feat_num, buffer, sizeof(int)); + buffer += sizeof(int); + + feature.clear(); + + for (int i = 0; i < feat_num; ++i) { + memcpy(&feat_len, buffer, sizeof(int)); + buffer += sizeof(int); + + char str[feat_len + 1]; + memcpy(str, buffer, feat_len); + buffer += feat_len; + str[feat_len] = '\0'; + feature.push_back(std::string(str)); + } + } } } diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 4d2b866d5d822..74fb110830321 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -18,32 +18,68 @@ namespace paddle { namespace distributed { -class GraphNode { +class Node { public: - GraphNode(): sampler(nullptr), edges(nullptr) { } - GraphNode(uint64_t id, std::string feature) - : id(id), feature(feature), sampler(nullptr), edges(nullptr) {} - virtual ~GraphNode(); + Node(){} + Node(uint64_t id) + : id(id) {} + virtual ~Node() {} static int id_size, int_size, weight_size; uint64_t get_id() { return id; } void set_id(uint64_t id) { this->id = id; } - void set_feature(std::string feature) { this->feature = feature; } - std::string get_feature() { return feature; } + + virtual void build_edges(bool is_weighted) {} + virtual void build_sampler(std::string sample_type) {} + virtual void add_edge(uint64_t id, float weight) {} + virtual std::vector sample_k(int k) { return std::vector(); } + virtual uint64_t get_neighbor_id(int idx){ return 0; } + virtual float get_neighbor_weight(int idx){ return 1.; } + virtual int get_size(bool need_feature); - virtual void build_edges(bool is_weighted); - virtual void build_sampler(std::string sample_type); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } - std::vector sample_k(int k) { return sampler->sample_k(k); } - uint64_t get_neighbor_id(int idx){return edges->get_id(idx);} - float get_neighbor_weight(int idx){return edges->get_weight(idx);} + virtual void add_feature(std::string feature) { } + virtual std::string get_feature(int idx) { return std::string(""); } protected: uint64_t id; - std::string feature; + +}; + +class GraphNode: public Node { + public: + GraphNode(): Node(), sampler(nullptr), edges(nullptr) { } + GraphNode(uint64_t id) + : Node(id), sampler(nullptr), edges(nullptr) {} + virtual ~GraphNode(); + virtual void build_edges(bool is_weighted); + virtual void build_sampler(std::string sample_type); + virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } + virtual std::vector sample_k(int k) { return sampler->sample_k(k); } + virtual uint64_t get_neighbor_id(int idx){return edges->get_id(idx);} + virtual float get_neighbor_weight(int idx){return edges->get_weight(idx);} + + protected: Sampler *sampler; GraphEdgeBlob * edges; }; + + +class FeatureNode: public Node{ + public: + FeatureNode(): Node() { } + FeatureNode(uint64_t id) : Node(id) {} + virtual ~FeatureNode() {} + virtual int get_size(bool need_feature); + virtual void to_buffer(char *buffer, bool need_feature); + virtual void recover_from_buffer(char *buffer); + virtual void add_feature(std::string feature) { this->feature.push_back(feature); } + virtual std::string get_feature(int idx) { return feature[idx]; } + + protected: + std::vector feature; +}; + + } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 2ba5946cc443f..508d9b63f3d5f 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -334,7 +334,7 @@ void RunBrpcPushSparse() { pull_status.wait(); ASSERT_EQ(0, vs[0].size()); - std::vector nodes; + std::vector nodes; pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, 1, nodes); pull_status.wait(); ASSERT_EQ(nodes.size(), 1); @@ -450,15 +450,15 @@ void RunBrpcPushSparse() { void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; - s.set_feature("hhhh"); + s.add_feature("hhhh"); s.set_id(65); int size = s.get_size(true); char str[size]; s.to_buffer(str, true); s1.recover_from_buffer(str); ASSERT_EQ(s.get_id(), s1.get_id()); - VLOG(0) << s.get_feature(); - VLOG(0) << s1.get_feature(); + VLOG(0) << s.get_feature(0); + VLOG(0) << s1.get_feature(0); } TEST(RunBrpcPushSparse, Run) { RunBrpcPushSparse(); } From fe4afed49a1a2b7325381eb0645b644e82758e8c Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Tue, 23 Mar 2021 02:37:26 +0800 Subject: [PATCH 060/106] add FeatureNode --- paddle/fluid/distributed/table/graph_node.cc | 21 +++----------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 334e9b0750cce..8c20fc302f8d7 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -34,14 +34,10 @@ int Node::id_size = sizeof(uint64_t); int Node::int_size = sizeof(int); int Node::get_size(bool need_feature) { - return int_size + id_size + int_size; + return id_size + int_size; } void Node::to_buffer(char* buffer, bool need_feature) { - int size = get_size(need_feature); - memcpy(buffer, &size, sizeof(int)); - buffer += sizeof(int); - memcpy(buffer, &id, id_size); buffer += id_size; @@ -50,12 +46,11 @@ void Node::to_buffer(char* buffer, bool need_feature) { } void Node::recover_from_buffer(char* buffer) { - buffer += sizeof(int); memcpy(&id, buffer, id_size); } int FeatureNode::get_size(bool need_feature) { - int size = int_size + id_size + int_size; // buffer_size, id, feat_num + int size = id_size + int_size; // id, feat_num if (need_feature){ size += feature.size() * int_size; for (const std::string& fea: feature){ @@ -83,10 +78,6 @@ void GraphNode::build_sampler(std::string sample_type) { sampler->build(edges); } void FeatureNode::to_buffer(char* buffer, bool need_feature) { - int size = get_size(need_feature); - memcpy(buffer, &size, sizeof(int)); - buffer += sizeof(int); - memcpy(buffer, &id, id_size); buffer += id_size; @@ -103,18 +94,13 @@ void FeatureNode::to_buffer(char* buffer, bool need_feature) { memcpy(buffer, feature[i].c_str(), feature[i].size()); buffer += feature[i].size(); } - //memcpy(buffer + int_size, feature.c_str(), feature.size()); - //memcpy(buffer + int_size + feature.size(), &id, id_size); } else { memcpy(buffer, &feat_num, sizeof(int)); } } void FeatureNode::recover_from_buffer(char* buffer) { - int size, feat_num, feat_len; - memcpy(&size, buffer, sizeof(int)); - - buffer += sizeof(int); + int feat_num, feat_len; memcpy(&id, buffer, id_size); buffer += id_size; @@ -122,7 +108,6 @@ void FeatureNode::recover_from_buffer(char* buffer) { buffer += sizeof(int); feature.clear(); - for (int i = 0; i < feat_num; ++i) { memcpy(&feat_len, buffer, sizeof(int)); buffer += sizeof(int); From 9d3188a4a8b01a56051d2dd3ed6a70cc1d1400c8 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Tue, 23 Mar 2021 17:48:09 +0800 Subject: [PATCH 061/106] add FeatureNode Serialize --- .../distributed/table/common_graph_table.cc | 75 ++++++++++++++++--- .../distributed/table/common_graph_table.h | 3 + paddle/fluid/distributed/table/graph_node.h | 35 ++++++++- .../fluid/distributed/test/graph_node_test.cc | 46 ++++++++++++ 4 files changed, 146 insertions(+), 13 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 7774a402cb49e..c1dd26b2db869 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/distributed/table/common_graph_table.h" #include #include #include @@ -20,6 +19,8 @@ #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" +#include "paddle/fluid/distributed/table/common_graph_table.h" +#include "paddle/fluid/distributed/table/graph_node.h" namespace paddle { namespace distributed { @@ -70,6 +71,7 @@ int32_t GraphTable::load(const std::string &path, const std::string ¶m) { std::string node_type = param.substr(1); return this->load_nodes(path, node_type); } + return 0; } int32_t GraphTable::get_nodes_ids_by_ranges( @@ -113,6 +115,7 @@ int32_t GraphTable::get_nodes_ids_by_ranges( } return 0; } + int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto paths = paddle::string::split_string(path, ";"); for (auto path : paths) { @@ -134,16 +137,21 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { if (nt != node_type) { continue; } - std::vector feature; - for (size_t slice = 2; slice < values.size(); slice++) { - feature.push_back(values[slice]); - } + size_t index = shard_id - shard_start; - if (feature.size() > 0) { - // TODO add feature - shards[index].add_feature_node(id); - } else { - shards[index].add_feature_node(id); + + auto node = shards[index].add_feature_node(id); + + auto mutable_feature = node->get_mutable_feature(); + + mutable_feature.clear(); + mutable_feature.resize(this->feat_name.size()); + + for (size_t slice = 2; slice < values.size(); slice++) { + auto feat = this->parse_feature(values[slice]); + if(feat.first > 0) { + mutable_feature[feat.first] = feat.second; + } } } } @@ -321,6 +329,50 @@ int GraphTable::random_sample_neighboors( } return 0; } + + +std::pair GraphTable::parse_feature(std::string feat_str) { + // Return (feat_id, btyes) if name are in this->feat_name, else return (-1, "") + auto fields = paddle::string::split_string(feat_str, " "); + if(this->feat_id_map.count(fields[0])) { + int32_t id = this->feat_id_map[fields[0]]; + std::string dtype = this->feat_dtype[id]; + int32_t shape = this->feat_shape[id]; + std::vector values(fields.begin() + 1, fields.end()); + if(dtype == "feasign"){ + return std::make_pair ( + int32_t(id), + paddle::string::join_strings(values, ' ')); + } + else if(dtype == "string") { + return std::make_pair ( + int32_t(id), + paddle::string::join_strings(values, ' ')); + } + else if(dtype == "float32") { + return std::make_pair ( + int32_t(id), + FeatureNode::parse_value_to_bytes(values)); + } + else if(dtype == "float64") { + return std::make_pair ( + int32_t(id), + FeatureNode::parse_value_to_bytes(values)); + } + else if(dtype == "int32") { + return std::make_pair ( + int32_t(id), + FeatureNode::parse_value_to_bytes(values)); + } + else if (dtype == "int64"){ + return std::make_pair ( + int32_t(id), + FeatureNode::parse_value_to_bytes(values)); + } + } + return std::make_pair(-1, ""); +} + int32_t GraphTable::pull_graph_list(int start, int total_size, std::unique_ptr &buffer, int &actual_size, bool need_feature, @@ -387,13 +439,14 @@ int32_t GraphTable::initialize() { this->table_type = common.name(); VLOG(0) << " init graph table type " << this->table_type << " table name " << this->table_name; int feat_conf_size = static_cast(common.attributes().size()); - for(int i=0; ifeat_name.push_back(f_name); this->feat_shape.push_back(f_shape); this->feat_dtype.push_back(f_dtype); + this->feat_id_map[f_name] = i; VLOG(0) << "init graph table feat conf name:"<< f_name << " shape:" << f_shape << " dtype:" << f_dtype; } diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index b13cb69c47b7a..4f1905ba9165f 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -26,6 +26,7 @@ #include #include "paddle/fluid/distributed/table/accessor.h" #include "paddle/fluid/distributed/table/common_table.h" +#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/framework/rw_lock.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { @@ -118,6 +119,7 @@ class GraphTable : public SparseTable { } virtual int32_t initialize_shard() { return 0; } virtual uint32_t get_thread_pool_index(uint64_t node_id); + virtual std::pair parse_feature(std::string feat_str); protected: std::vector shards; @@ -128,6 +130,7 @@ class GraphTable : public SparseTable { std::vector feat_name; std::vector feat_dtype; std::vector feat_shape; + std::unordered_map feat_id_map; std::string table_name; std::string table_type; diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 74fb110830321..9d58ce34bb70b 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -14,6 +14,9 @@ #pragma once #include +#include +#include +#include #include "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { @@ -58,6 +61,7 @@ class GraphNode: public Node { virtual std::vector sample_k(int k) { return sampler->sample_k(k); } virtual uint64_t get_neighbor_id(int idx){return edges->get_id(idx);} virtual float get_neighbor_weight(int idx){return edges->get_weight(idx);} + protected: Sampler *sampler; @@ -73,8 +77,35 @@ class FeatureNode: public Node{ virtual int get_size(bool need_feature); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual void add_feature(std::string feature) { this->feature.push_back(feature); } - virtual std::string get_feature(int idx) { return feature[idx]; } + virtual std::string get_feature(int idx) { return this->feature[idx]; } + virtual std::vector & get_mutable_feature() { return this->feature; } + + template + static std::string parse_value_to_bytes(std::vector feat_str) { + T v; + size_t Tsize = sizeof(T) * feat_str.size(); + char buffer[Tsize]; + for(size_t i = 0;i < feat_str.size();i ++) { + std::stringstream ss(feat_str[i]); + ss >> v; + std::memcpy(buffer + sizeof(T) * i, (char *)&v, sizeof(T)); + } + return std::string(buffer, Tsize); + } + + template + static std::vector parse_bytes_to_array(std::string feat_str) { + T v; + std::vector out; + size_t start = 0; + const char * buffer = feat_str.data(); + while(start < feat_str.size()) { + std::memcpy((char *)&v, buffer + start, sizeof(T)); + start += sizeof(T); + out.push_back(v); + } + return out; + } protected: std::vector feature; diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 508d9b63f3d5f..c337442aba606 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -62,6 +62,48 @@ void testSampleNodes( ASSERT_EQ(true, s1.find(id) != s1.end()); } } + +void testFeatureNodeSerializeInt() { + std::string out = distributed::FeatureNode::parse_value_to_bytes({"123", "345"}); + std::vector out2 = distributed::FeatureNode::parse_bytes_to_array(out); + ASSERT_EQ(out2[0] , 123); + ASSERT_EQ(out2[1] , 345); +} + +void testFeatureNodeSerializeInt64() { + std::string out = distributed::FeatureNode::parse_value_to_bytes({"123", "345"}); + std::vector out2 = distributed::FeatureNode::parse_bytes_to_array(out); + ASSERT_EQ(out2[0] , 123); + ASSERT_EQ(out2[1] , 345); +} + + +void testFeatureNodeSerializeFloat32() { + std::string out = distributed::FeatureNode::parse_value_to_bytes({"123.123", "345.123"}); + std::vector out2 = distributed::FeatureNode::parse_bytes_to_array(out); + float eps; + std::cout << "Float " << out2[0] << " " << 123.123 << std::endl; + eps = out2[0] - 123.123; + ASSERT_LE(eps * eps , 1e-5); + eps = out2[1] - 345.123; + ASSERT_LE(eps * eps , 1e-5); +} + + +void testFeatureNodeSerializeFloat64() { + std::string out = distributed::FeatureNode::parse_value_to_bytes({"123.123", "345.123"}); + std::vector out2 = distributed::FeatureNode::parse_bytes_to_array(out); + float eps; + eps = out2[0] - 123.123; + std::cout << "Float64 " << out2[0] << " " << 123.123 << std::endl; + ASSERT_LE(eps * eps , 1e-5); + eps = out2[1] - 345.123; + ASSERT_LE(eps * eps , 1e-5); +} + + + + void testSingleSampleNeighboor( std::shared_ptr& worker_ptr_) { std::vector>> vs; @@ -445,6 +487,10 @@ void RunBrpcPushSparse() { worker_ptr_->finalize_worker(); server_thread.join(); server_thread2.join(); + testFeatureNodeSerializeInt(); + testFeatureNodeSerializeInt64(); + testFeatureNodeSerializeFloat32(); + testFeatureNodeSerializeFloat64(); testGraphToBuffer(); } From c7e47f139205a845c3771afce70e21787b0fe248 Mon Sep 17 00:00:00 2001 From: Yelrose <270018958@qq.com> Date: Tue, 23 Mar 2021 17:59:13 +0800 Subject: [PATCH 062/106] add FeatureNode Serialize --- .../fluid/distributed/test/graph_node_test.cc | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index c337442aba606..3a97165963d6e 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -179,14 +179,14 @@ std::string edges[] = { char edge_file_name[] = "edges.txt"; std::string nodes[] = { - std::string("user\t37\t0.34"), std::string("user\t96\t0.31"), - std::string("user\t59\t0.11"), std::string("user\t97\t0.11"), - std::string("item\t45\t0.21"), std::string("item\t145\t0.21"), - std::string("item\t112\t0.21"), std::string("item\t48\t0.21"), - std::string("item\t247\t0.21"), std::string("item\t111\t0.21"), - std::string("item\t46\t0.21"), std::string("item\t146\t0.21"), - std::string("item\t122\t0.21"), std::string("item\t49\t0.21"), - std::string("item\t248\t0.21"), std::string("item\t113\t0.21")}; + std::string("user\t37\ta 0.34\tb 13 14\tc hello"), std::string("user\t96\ta 0.31\tb 15 10"), + std::string("user\t59\ta 0.11\tb 11 14"), std::string("user\t97\ta 0.11\tb 12 11"), + std::string("item\t45\ta 0.21"), std::string("item\t145\ta 0.21"), + std::string("item\t112\ta 0.21"), std::string("item\t48\ta 0.21"), + std::string("item\t247\ta 0.21"), std::string("item\t111\ta 0.21"), + std::string("item\t46\ta 0.21"), std::string("item\t146\ta 0.21"), + std::string("item\t122\ta 0.21"), std::string("item\t49\ta 0.21"), + std::string("item\t248\ta 0.21"), std::string("item\t113\ta 0.21")}; char node_file_name[] = "nodes.txt"; void prepare_file(char file_name[], bool load_edge) { @@ -397,8 +397,31 @@ void RunBrpcPushSparse() { std::string("item")}; server1.set_up(ips_str, 127, node_types, edge_types, 0); server2.set_up(ips_str, 127, node_types, edge_types, 1); + + server1.add_table_feat_conf("user", "a", "float32", 1); + server1.add_table_feat_conf("user", "b", "int32", 2); + server1.add_table_feat_conf("user", "c", "string", 1); + server1.add_table_feat_conf("item", "a", "float32", 1); + + server2.add_table_feat_conf("user", "a", "float32", 1); + server2.add_table_feat_conf("user", "b", "int32", 2); + server2.add_table_feat_conf("user", "c", "string", 1); + server2.add_table_feat_conf("item", "a", "float32", 1); + client1.set_up(ips_str, 127, node_types, edge_types, 0); + + client1.add_table_feat_conf("user", "a", "float32", 1); + client1.add_table_feat_conf("user", "b", "int32", 2); + client1.add_table_feat_conf("user", "c", "string", 1); + client1.add_table_feat_conf("item", "a", "float32", 1); + client2.set_up(ips_str, 127, node_types, edge_types, 1); + + client2.add_table_feat_conf("user", "a", "float32", 1); + client2.add_table_feat_conf("user", "b", "int32", 2); + client2.add_table_feat_conf("user", "c", "string", 1); + client2.add_table_feat_conf("item", "a", "float32", 1); + server1.start_server(); std::cout << "first server done" << std::endl; server2.start_server(); From a81a214ae5af375b1f34a0226cad39b142285b12 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Wed, 24 Mar 2021 14:23:54 +0800 Subject: [PATCH 063/106] get_feat_node --- .../distributed/service/graph_brpc_client.cc | 87 +++++++++++++++++++ .../distributed/service/graph_brpc_client.h | 5 ++ .../distributed/service/graph_brpc_server.cc | 36 ++++++++ .../distributed/service/graph_brpc_server.h | 3 + .../distributed/service/graph_py_service.cc | 18 ++++ .../distributed/service/graph_py_service.h | 3 + paddle/fluid/distributed/service/ps_client.h | 11 +++ .../fluid/distributed/service/sendrecv.proto | 3 +- .../distributed/table/common_graph_table.cc | 35 +++++++- .../distributed/table/common_graph_table.h | 5 ++ paddle/fluid/distributed/table/table.h | 6 ++ .../fluid/distributed/test/graph_node_test.cc | 23 ++--- paddle/fluid/pybind/fleet_py.cc | 1 + 13 files changed, 216 insertions(+), 20 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 47bd70ff5d980..69e873e82b4a7 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -34,6 +34,93 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { : shard_num / server_size + 1; return id % shard_num / shard_per_server; } + +std::future GraphBrpcClient::get_node_feat( + const uint32_t& table_id, const std::vector& node_ids, const std::vector& feature_names, + std::vector > &res) { + + std::vector request2server; + std::vector server2request(server_size, -1); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_ids[query_idx]); + if (server2request[server_index] == -1) { + server2request[server_index] = request2server.size(); + request2server.push_back(server_index); + } + } + size_t request_call_num = request2server.size(); + std::vector> node_id_buckets(request_call_num); + std::vector> query_idx_buckets(request_call_num); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_ids[query_idx]); + int request_idx = server2request[server_index]; + node_id_buckets[request_idx].push_back(node_ids[query_idx]); + query_idx_buckets[request_idx].push_back(query_idx); + } + + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + request_call_num, + [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + int fail_num = 0; + for (int request_idx = 0; request_idx < request_call_num; + ++request_idx) { + if (closure->check_response(request_idx, + PS_GRAPH_SAMPLE_NEIGHBOORS) != 0) { + ++fail_num; + } else { + auto &res_io_buffer = + closure->cntl(request_idx)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + std::unique_ptr buffer_wrapper(new char[bytes_size]); + char *buffer = buffer_wrapper.get(); + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + + for (size_t feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { + for (size_t node_idx = 0; node_idx < query_idx_buckets.at(request_idx).size(); ++node_idx) { + int query_idx = query_idx_buckets.at(request_idx).at(node_idx); + size_t feat_len = *(size_t *)(buffer); + buffer += sizeof(size_t); + res[feat_idx][query_idx] = std::string(buffer, feat_len); + buffer += feat_len; + } + } + } + if (fail_num == request_call_num) { + ret = -1; + } + } + closure->set_promise_value(ret); + }); + + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + + for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { + int server_index = request2server[request_idx]; + closure->request(request_idx)->set_cmd_id(PS_GRAPH_GET_NODE_FEAT); + closure->request(request_idx)->set_table_id(table_id); + closure->request(request_idx)->set_client_id(_client_id); + size_t node_num = node_id_buckets[request_idx].size(); + + closure->request(request_idx) + ->add_params((char *)node_id_buckets[request_idx].data(), + sizeof(uint64_t) * node_num); + std::string joint_feature_name = paddle::string::join_strings(feature_names, '\t'); + closure->request(request_idx) + ->add_params(joint_feature_name.c_str(), sizeof(joint_feature_name)); + + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), + closure->response(request_idx), closure); + } + + return fut; +} // char* &buffer,int &actual_size std::future GraphBrpcClient::batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 87e481d578a84..ea1c842a8f2b0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -47,6 +47,11 @@ class GraphBrpcClient : public BrpcPsClient { int server_index, int sample_size, std::vector &ids); + virtual std::future get_node_feat( + const uint32_t& table_id, + const std::vector& node_ids, + const std::vector& feature_names, + std::vector > &res); virtual int32_t initialize(); int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 2f619fdb2a661..29864e55fa0fb 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -99,6 +99,8 @@ int32_t GraphBrpcService::initialize() { &GraphBrpcService::graph_random_sample_neighboors; _service_handler_map[PS_GRAPH_SAMPLE_NODES] = &GraphBrpcService::graph_random_sample_nodes; + _service_handler_map[PS_GRAPH_GET_NODE_FEAT] = + &GraphBrpcService::graph_get_node_feat; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); @@ -313,5 +315,39 @@ int32_t GraphBrpcService::graph_random_sample_nodes( return 0; } + + +int32_t GraphBrpcService::graph_get_node_feat( + Table *table, const PsRequestMessage &request, PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 2) { + set_response_code( + response, -1, + "graph_get_node_feat request requires at least 2 arguments"); + return 0; + } + size_t node_num = request.params(0).size() / sizeof(uint64_t); + uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); + std::vector node_ids(node_data, node_data+node_num); + + std::vector feature_names = + paddle::string::split_string(request.params(1), "\t"); + + std::vector> feature + (feature_names.size(), std::vector(node_num)); + + table->get_node_feat(node_ids, feature_names, feature); + + for (size_t feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { + for (size_t node_idx = 0; node_idx < node_num; ++node_idx) { + size_t feat_len = feature[feat_idx][node_idx].size(); + cntl->response_attachment().append(&feat_len, sizeof(size_t)); + cntl->response_attachment().append(&feature[feat_idx][node_idx], feat_len); + } + } + + return 0; +} } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index bb054081257b4..3248058701bb9 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -87,6 +87,9 @@ class GraphBrpcService : public PsBaseService { const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); + int32_t graph_get_node_feat( + Table *table, const PsRequestMessage &request, PsResponseMessage &response, + brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 84ceb102a1e0f..f57eabd845cb5 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -310,6 +310,24 @@ std::vector GraphPyClient::random_sample_nodes(std::string name, } return v; } + +// (name, dtype, ndarray) +std::vector > + GraphPyClient::get_node_feat(std::string node_type, + std::vector node_ids, + std::vector feature_names){ + + std::vector > v(feature_names.size(), + std::vector(node_ids.size())); + if (this->table_id_map.count(node_type)) { + uint32_t table_id = this->table_id_map[node_type]; + auto status = + worker_ptr->get_node_feat(table_id, node_ids, feature_names, v); + status.wait(); + } + return v; +} + std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index beab26085058b..52bdef5d32464 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -150,6 +150,9 @@ class GraphPyClient : public GraphPyService { std::string name, std::vector node_ids, int sample_size); std::vector random_sample_nodes(std::string name, int server_index, int sample_size); + std::vector > + get_node_feat(std::string node_type, std::vector node_ids, + std::vector feature_names); std::vector pull_graph_list(std::string name, int server_index, int start, int size, int step = 1); ::paddle::distributed::PSParameter GetWorkerProto(); diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 1b50a0c37098d..50b8c01abe7c7 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -185,6 +185,17 @@ class PSClient { promise.set_value(-1); return fut; } + virtual std::future get_node_feat( + const uint32_t& table_id, + const std::vector& node_ids, + const std::vector& feature_names, + std::vector > &res){ + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 185024fe18721..d908c26da9870 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -51,6 +51,7 @@ enum PsCmdID { PS_PULL_GRAPH_LIST = 30; PS_GRAPH_SAMPLE_NEIGHBOORS = 31; PS_GRAPH_SAMPLE_NODES = 32; + PS_GRAPH_GET_NODE_FEAT = 33; } message PsRequestMessage { @@ -114,4 +115,4 @@ message MultiVariableMessage { service PsService { rpc service(PsRequestMessage) returns (PsResponseMessage); rpc SendAndRecvVariable(MultiVariableMessage) returns (MultiVariableMessage); -}; \ No newline at end of file +}; diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index c1dd26b2db869..84841efc6d11f 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -212,7 +212,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { Node *GraphTable::find_node(uint64_t id) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { - return NULL; + return nullptr; } size_t index = shard_id - shard_start; Node *node = shards[index].find_node(id); @@ -287,7 +287,7 @@ int32_t GraphTable::random_sample_nodes(int sample_size, memcpy(pointer, res.data(), actual_size); return 0; } -int GraphTable::random_sample_neighboors( +int32_t GraphTable::random_sample_neighboors( uint64_t *node_ids, int sample_size, std::vector> &buffers, std::vector &actual_sizes) { @@ -301,7 +301,7 @@ int GraphTable::random_sample_neighboors( [&]() -> int { Node *node = find_node(node_id); - if (node == NULL) { + if (node == nullptr) { actual_size = 0; return 0; } @@ -330,6 +330,35 @@ int GraphTable::random_sample_neighboors( return 0; } +int32_t GraphTable::get_node_feat( + const std::vector& node_ids, + const std::vector& feature_names, + std::vector > &res){ + size_t node_num = node_ids.size(); + std::vector> tasks; + for (size_t idx = 0; idx < node_num; ++idx) { + uint64_t node_id = node_ids[idx]; + tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( + [&, idx, node_id]() -> int { + Node *node = find_node(node_id); + + if (node == nullptr) { + return 0; + } + for (int feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx){ + const std::string &feature_name = feature_names[feat_idx]; + if (feat_id_map.find(feature_name) != feat_id_map.end()){ + res[feat_idx][idx] = node->get_feature(feat_id_map[feature_name]); + } + } + return 0; + })); + } + for (size_t idx = 0; idx < node_num; ++idx) { + tasks[idx].get(); + } + return 0; +} std::pair GraphTable::parse_feature(std::string feat_str) { // Return (feat_id, btyes) if name are in this->feat_name, else return (-1, "") diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 4f1905ba9165f..83ef390ed4feb 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -121,6 +121,11 @@ class GraphTable : public SparseTable { virtual uint32_t get_thread_pool_index(uint64_t node_id); virtual std::pair parse_feature(std::string feat_str); + virtual int32_t get_node_feat( + const std::vector& node_ids, + const std::vector& feature_names, + std::vector > &res); + protected: std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 40d2abb6c86ae..fcf9bc37bb05d 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -107,6 +107,12 @@ class Table { int &actual_sizes) { return 0; } + virtual int32_t get_node_feat( + const std::vector& node_ids, + const std::vector& feature_names, + std::vector > &res) { + return 0; + } virtual int32_t pour() { return 0; } virtual void clear() = 0; diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 3a97165963d6e..01d404e736630 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -448,6 +448,7 @@ void RunBrpcPushSparse() { ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); + // Test Pull by step @@ -485,22 +486,12 @@ void RunBrpcPushSparse() { ASSERT_EQ(nodes_ids.size(), 2); ASSERT_EQ(true, (nodes_ids[0] == 59 && nodes_ids[1] == 37) || (nodes_ids[0] == 37 && nodes_ids[1] == 59)); - // to test in python,try this: - // from paddle.fluid.core import GraphPyService - // ips_str = "127.0.0.1:4211;127.0.0.1:4212" - // gps1 = GraphPyService(); - // gps2 = GraphPyService(); - // gps1.set_up(ips_str, 127, 0, 0, 0); - // gps2.set_up(ips_str, 127, 1, 1, 0); - // gps1.load_file("input.txt"); - - // list = gps2.pull_graph_list(0,1,4) - // for x in list: - // print(x.get_id()) - - // list = gps2.sample_k(96, "user", 4); - // for x in list: - // print(x.get_id()) + + // Test get node feat + auto node_feat = client1.get_node_feat(std::string("user"), std::vector(1, 37), std::vector(1, std::string("c"))); + ASSERT_EQ(node_feat.size(), 1); + ASSERT_EQ(node_feat[0].size(), 1); + std::cout << "get_node_feat: " << node_feat[0][0] << std::endl; std::remove(edge_file_name); std::remove(node_file_name); diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 866fa5fc624ba..3ab3fa047cc9d 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -184,6 +184,7 @@ void BindGraphPyClient(py::module* m) { .def("pull_graph_list", &GraphPyClient::pull_graph_list) .def("start_client", &GraphPyClient::start_client) .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) + .def("get_node_feat", &GraphPyClient::get_node_feat) .def("random_sample_nodes", &GraphPyClient::random_sample_nodes); } From f4a18d842f195590b160785328dbdd98a16bdd74 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 24 Mar 2021 08:02:01 +0000 Subject: [PATCH 064/106] avoid local rpc --- .../distributed/service/graph_brpc_client.cc | 35 +++++++--- .../distributed/service/graph_brpc_client.h | 41 +++++++++++- .../distributed/service/graph_brpc_server.cc | 3 +- .../distributed/service/graph_brpc_server.h | 2 +- .../distributed/service/graph_py_service.cc | 10 +-- .../distributed/service/graph_py_service.h | 21 ++++-- .../fluid/distributed/test/graph_node_test.cc | 64 +++++++++++-------- paddle/fluid/pybind/fleet_py.cc | 3 +- 8 files changed, 130 insertions(+), 49 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 1b54ce986d9c2..1ce7767dafb65 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -26,6 +26,24 @@ #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { +int GraphPsService_Stub::thread_num = 3; + +void GraphPsService_Stub::service( + ::google::protobuf::RpcController *controller, + const ::paddle::distributed::PsRequestMessage *request, + ::paddle::distributed::PsResponseMessage *response, + ::google::protobuf::Closure *done) { + if (graph_service != NULL && local_channel == channel()) { + // VLOG(0)<<"use local"; + task_pool->enqueue([this, controller, request, response, done]() -> int { + this->graph_service->service(controller, request, response, done); + return 0; + }); + } else { + // VLOG(0)<<"use server"; + PsService_Stub::service(controller, request, response, done); + } +} int GraphBrpcClient::get_server_index_by_id(uint64_t id) { int shard_num = get_shard_num(); @@ -47,7 +65,6 @@ std::future GraphBrpcClient::batch_sample_neighboors( server2request[server_index] = request2server.size(); request2server.push_back(server_index); } - // res.push_back(std::vector()); res.push_back(std::vector>()); } size_t request_call_num = request2server.size(); @@ -76,7 +93,6 @@ std::future GraphBrpcClient::batch_sample_neighboors( closure->cntl(request_idx)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); - // char buffer[bytes_size]; std::unique_ptr buffer_wrapper(new char[bytes_size]); char *buffer = buffer_wrapper.get(); io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); @@ -117,7 +133,6 @@ std::future GraphBrpcClient::batch_sample_neighboors( closure->request(request_idx)->set_cmd_id(PS_GRAPH_SAMPLE_NEIGHBOORS); closure->request(request_idx)->set_table_id(table_id); closure->request(request_idx)->set_client_id(_client_id); - // std::string type_str = GraphNode::node_type_to_string(type); size_t node_num = node_id_buckets[request_idx].size(); closure->request(request_idx) @@ -125,7 +140,9 @@ std::future GraphBrpcClient::batch_sample_neighboors( sizeof(uint64_t) * node_num); closure->request(request_idx) ->add_params((char *)&sample_size, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); + // PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = + getServiceStub(get_cmd_channel(server_index)); closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), closure->response(request_idx), closure); @@ -142,8 +159,6 @@ std::future GraphBrpcClient::random_sample_nodes( if (closure->check_response(0, PS_GRAPH_SAMPLE_NODES) != 0) { ret = -1; } else { - // VLOG(0) << "check sample response: " - // << " " << closure->check_response(0, PS_PULL_GRAPH_LIST); auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); @@ -166,7 +181,8 @@ std::future GraphBrpcClient::random_sample_nodes( closure->request(0)->set_client_id(_client_id); closure->request(0)->add_params((char *)&sample_size, sizeof(int)); ; - PsService_Stub rpc_stub(get_cmd_channel(server_index)); + // PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = getServiceStub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), closure); @@ -208,7 +224,8 @@ std::future GraphBrpcClient::pull_graph_list( closure->request(0)->add_params((char *)&start, sizeof(int)); closure->request(0)->add_params((char *)&size, sizeof(int)); closure->request(0)->add_params((char *)&step, sizeof(int)); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); + // PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = getServiceStub(get_cmd_channel(server_index)); closure->cntl(0)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(0), closure->request(0), closure->response(0), closure); @@ -218,6 +235,8 @@ int32_t GraphBrpcClient::initialize() { set_shard_num(_config.shard_num()); BrpcPsClient::initialize(); server_size = get_server_nums(); + graph_service = NULL; + local_channel = NULL; return 0; } } diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index e762409fbeeda..6c5227f54e918 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -23,6 +23,7 @@ #include "brpc/controller.h" #include "brpc/server.h" #include "paddle/fluid/distributed/service/brpc_ps_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_server.h" #include "paddle/fluid/distributed/service/ps_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/lod_tensor.h" @@ -32,29 +33,63 @@ namespace paddle { namespace distributed { +class GraphPsService_Stub : public PsService_Stub { + public: + static int thread_num; + GraphPsService_Stub(::google::protobuf::RpcChannel* channel, + ::google::protobuf::RpcChannel* local_channel = NULL, + GraphBrpcService* service = NULL) + : PsService_Stub(channel) { + this->local_channel = local_channel; + this->graph_service = service; + task_pool.reset(new ::ThreadPool(thread_num)); + } + virtual ~GraphPsService_Stub() {} + + // implements PsService ------------------------------------------ + GraphBrpcService* graph_service; + std::shared_ptr<::ThreadPool> task_pool; + ::google::protobuf::RpcChannel* local_channel; + GOOGLE_DISALLOW_EVIL_CONSTRUCTORS(GraphPsService_Stub); + void service(::google::protobuf::RpcController* controller, + const ::paddle::distributed::PsRequestMessage* request, + ::paddle::distributed::PsResponseMessage* response, + ::google::protobuf::Closure* done); +}; class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} virtual std::future batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, - std::vector>> &res); + std::vector>>& res); virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, int step, - std::vector &res); + std::vector& res); virtual std::future random_sample_nodes(uint32_t table_id, int server_index, int sample_size, - std::vector &ids); + std::vector& ids); virtual int32_t initialize(); int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } int get_server_index_by_id(uint64_t id); + void set_local_channel(int index) { + this->local_channel = get_cmd_channel(index); + } + void set_local_graph_service(GraphBrpcService* graph_service) { + this->graph_service = graph_service; + } + GraphPsService_Stub getServiceStub(::google::protobuf::RpcChannel* channel) { + return GraphPsService_Stub(channel, local_channel, graph_service); + } private: int shard_num; size_t server_size; + ::google::protobuf::RpcChannel* local_channel; + GraphBrpcService* graph_service; }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 60d6bc203a074..95f8d82d6f81b 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -18,6 +18,7 @@ #include // NOLINT #include "butil/endpoint.h" #include "iomanip" +#include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/platform/profiler.h" @@ -270,7 +271,7 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, int step = *(int *)(request.params(2).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size, step, true); + table->pull_graph_list(start, size, buffer, actual_size, true, step); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index bb054081257b4..e2b57dff4d859 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -29,6 +29,7 @@ class GraphBrpcServer : public PSServer { public: GraphBrpcServer() {} virtual ~GraphBrpcServer() {} + PsBaseService *get_service() { return _service.get(); } virtual uint64_t start(const std::string &ip, uint32_t port); virtual int32_t stop() { std::unique_lock lock(mutex_); @@ -43,7 +44,6 @@ class GraphBrpcServer : public PSServer { private: virtual int32_t initialize(); - mutable std::mutex mutex_; std::condition_variable cv_; bool stoped_ = false; diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 752dd0bcba2f0..8329ddd0e6d70 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -73,8 +73,9 @@ void GraphPyClient::start_client() { auto servers_ = host_sign_list.size(); _ps_env = paddle::distributed::PaddlePSEnvironment(); _ps_env.set_ps_servers(&host_sign_list, servers_); - worker_ptr = std::shared_ptr( - paddle::distributed::PSClientFactory::create(worker_proto)); + worker_ptr = std::shared_ptr( + (paddle::distributed::GraphBrpcClient*) + paddle::distributed::PSClientFactory::create(worker_proto)); worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); } void GraphPyServer::start_server() { @@ -87,8 +88,9 @@ void GraphPyServer::start_server() { auto _ps_env = paddle::distributed::PaddlePSEnvironment(); _ps_env.set_ps_servers(&this->host_sign_list, this->host_sign_list.size()); // test - pserver_ptr = std::shared_ptr( - paddle::distributed::PSServerFactory::create(server_proto)); + pserver_ptr = std::shared_ptr( + (paddle::distributed::GraphBrpcServer*) + paddle::distributed::PSServerFactory::create(server_proto)); VLOG(0) << "pserver-ptr created "; std::vector empty_vec; framework::ProgramDesc empty_prog; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 7452c99c07f60..7d7d1b341a90b 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -87,6 +87,7 @@ class GraphPyService { }; class GraphPyServer : public GraphPyService { public: + GraphPyServer() { VLOG(0) << "construct a graphPyServer"; } void set_up(std::string ips_str, int shard_num, std::vector node_types, std::vector edge_types, int rank) { @@ -95,15 +96,18 @@ class GraphPyServer : public GraphPyService { } int get_rank() { return rank; } void set_rank(int rank) { this->rank = rank; } + // paddle::distributed::GraphBrpcService * get_service(){ + // return pserver_ptr->get_service(); + // } void start_server(); ::paddle::distributed::PSParameter GetServerProto(); - std::shared_ptr get_ps_server() { + std::shared_ptr get_ps_server() { return pserver_ptr; } protected: int rank; - std::shared_ptr pserver_ptr; + std::shared_ptr pserver_ptr; std::thread* server_thread; }; class GraphPyClient : public GraphPyService { @@ -114,9 +118,18 @@ class GraphPyClient : public GraphPyService { set_client_id(client_id); GraphPyService::set_up(ips_str, shard_num, node_types, edge_types); } - std::shared_ptr get_ps_client() { + std::shared_ptr get_ps_client() { return worker_ptr; } + void bind_local_server(int local_channel_index, GraphPyServer& server) { + worker_ptr->set_local_channel(local_channel_index); + worker_ptr->set_local_graph_service( + (paddle::distributed::GraphBrpcService*)server.get_ps_server() + ->get_service()); + } + // void set_local_graph_service(GraphBrpcService *service){ + // worker_ptr->set_local_graph_service(service); + // } void load_edge_file(std::string name, std::string filepath, bool reverse); void load_node_file(std::string name, std::string filepath); int get_client_id() { return client_id; } @@ -132,7 +145,7 @@ class GraphPyClient : public GraphPyService { protected: int client_id; - std::shared_ptr worker_ptr; + std::shared_ptr worker_ptr; std::thread* client_thread; }; } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index efee0d9441ef2..24487ee5fbaa7 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -23,21 +23,22 @@ limitations under the License. */ #include "google/protobuf/text_format.h" #include "gtest/gtest.h" -#include "paddle/fluid/framework/lod_tensor.h" -#include "paddle/fluid/framework/scope.h" -#include "paddle/fluid/framework/tensor_util.h" -#include "paddle/fluid/framework/variable.h" - #include "paddle/fluid/distributed/ps.pb.h" #include "paddle/fluid/distributed/service/brpc_ps_client.h" #include "paddle/fluid/distributed/service/brpc_ps_server.h" #include "paddle/fluid/distributed/service/env.h" +#include "paddle/fluid/distributed/service/graph_brpc_client.h" +#include "paddle/fluid/distributed/service/graph_brpc_server.h" #include "paddle/fluid/distributed/service/graph_py_service.h" #include "paddle/fluid/distributed/service/ps_client.h" #include "paddle/fluid/distributed/service/sendrecv.pb.h" #include "paddle/fluid/distributed/service/service.h" #include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/program_desc.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/framework/tensor_util.h" +#include "paddle/fluid/framework/variable.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/platform/place.h" #include "paddle/fluid/string/printf.h" @@ -50,7 +51,7 @@ namespace memory = paddle::memory; namespace distributed = paddle::distributed; void testSampleNodes( - std::shared_ptr& worker_ptr_) { + std::shared_ptr& worker_ptr_) { std::vector ids; auto pull_status = worker_ptr_->random_sample_nodes(0, 0, 6, ids); std::unordered_set s; @@ -63,7 +64,7 @@ void testSampleNodes( } } void testSingleSampleNeighboor( - std::shared_ptr& worker_ptr_) { + std::shared_ptr& worker_ptr_) { std::vector>> vs; auto pull_status = worker_ptr_->batch_sample_neighboors( 0, std::vector(1, 37), 4, vs); @@ -78,6 +79,7 @@ void testSingleSampleNeighboor( for (auto g : s) { ASSERT_EQ(true, s1.find(g) != s1.end()); } + VLOG(0) << "test single done"; s.clear(); s1.clear(); vs.clear(); @@ -95,7 +97,7 @@ void testSingleSampleNeighboor( } void testBatchSampleNeighboor( - std::shared_ptr& worker_ptr_) { + std::shared_ptr& worker_ptr_) { std::vector>> vs; std::vector v = {37, 96}; auto pull_status = worker_ptr_->batch_sample_neighboors(0, v, 4, vs); @@ -238,9 +240,10 @@ uint32_t port_ = 4209, port2 = 4210; std::vector host_sign_list_; -std::shared_ptr pserver_ptr_, pserver_ptr2; +std::shared_ptr pserver_ptr_, + pserver_ptr2; -std::shared_ptr worker_ptr_; +std::shared_ptr worker_ptr_; void RunServer() { LOG(INFO) << "init first server"; @@ -248,8 +251,9 @@ void RunServer() { auto _ps_env = paddle::distributed::PaddlePSEnvironment(); _ps_env.set_ps_servers(&host_sign_list_, 2); // test - pserver_ptr_ = std::shared_ptr( - paddle::distributed::PSServerFactory::create(server_proto)); + pserver_ptr_ = std::shared_ptr( + (paddle::distributed::GraphBrpcServer*) + paddle::distributed::PSServerFactory::create(server_proto)); std::vector empty_vec; framework::ProgramDesc empty_prog; empty_vec.push_back(empty_prog); @@ -265,8 +269,9 @@ void RunServer2() { auto _ps_env2 = paddle::distributed::PaddlePSEnvironment(); _ps_env2.set_ps_servers(&host_sign_list_, 2); // test - pserver_ptr2 = std::shared_ptr( - paddle::distributed::PSServerFactory::create(server_proto2)); + pserver_ptr2 = std::shared_ptr( + (paddle::distributed::GraphBrpcServer*) + paddle::distributed::PSServerFactory::create(server_proto2)); std::vector empty_vec2; framework::ProgramDesc empty_prog2; empty_vec2.push_back(empty_prog2); @@ -274,16 +279,21 @@ void RunServer2() { pserver_ptr2->start(ip2, port2); } -void RunClient(std::map>& - dense_regions) { +void RunClient( + std::map>& dense_regions, + int index, paddle::distributed::PsBaseService* service) { ::paddle::distributed::PSParameter worker_proto = GetWorkerProto(); paddle::distributed::PaddlePSEnvironment _ps_env; auto servers_ = host_sign_list_.size(); _ps_env = paddle::distributed::PaddlePSEnvironment(); _ps_env.set_ps_servers(&host_sign_list_, servers_); - worker_ptr_ = std::shared_ptr( - paddle::distributed::PSClientFactory::create(worker_proto)); + worker_ptr_ = std::shared_ptr( + (paddle::distributed::GraphBrpcClient*) + paddle::distributed::PSClientFactory::create(worker_proto)); worker_ptr_->configure(worker_proto, dense_regions, _ps_env, 0); + worker_ptr_->set_local_channel(index); + worker_ptr_->set_local_graph_service( + (paddle::distributed::GraphBrpcService*)service); } void RunBrpcPushSparse() { @@ -308,7 +318,7 @@ void RunBrpcPushSparse() { std::pair>(0, {})); auto regions = dense_regions[0]; - RunClient(dense_regions); + RunClient(dense_regions, 0, pserver_ptr_->get_service()); /*-----------------------Test Server Init----------------------------------*/ auto pull_status = @@ -326,7 +336,9 @@ void RunBrpcPushSparse() { // } // std::vector> v; // pull_status = worker_ptr_->sample(0, 37, 4, v); + std::cerr << "start to test sample nodes" << std::endl; testSampleNodes(worker_ptr_); + sleep(5); testSingleSampleNeighboor(worker_ptr_); testBatchSampleNeighboor(worker_ptr_); pull_status = worker_ptr_->batch_sample_neighboors( @@ -353,6 +365,7 @@ void RunBrpcPushSparse() { std::vector edge_types = {std::string("user2item")}; std::vector node_types = {std::string("user"), std::string("item")}; + VLOG(0) << "make 2 servers"; server1.set_up(ips_str, 127, node_types, edge_types, 0); server2.set_up(ips_str, 127, node_types, edge_types, 1); client1.set_up(ips_str, 127, node_types, edge_types, 0); @@ -366,6 +379,11 @@ void RunBrpcPushSparse() { client2.start_client(); std::cout << "first client done" << std::endl; std::cout << "started" << std::endl; + VLOG(0) << "come to set local server"; + client1.bind_local_server(0, server1); + VLOG(0) << "first bound"; + client2.bind_local_server(1, server2); + VLOG(0) << "second bound"; client1.load_node_file(std::string("user"), std::string(node_file_name)); client1.load_node_file(std::string("item"), std::string(node_file_name)); client1.load_edge_file(std::string("user2item"), std::string(edge_file_name), @@ -375,19 +393,11 @@ void RunBrpcPushSparse() { nodes.clear(); nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4, 1); - for (auto g : nodes) { - std::cout << "node_ids: " << g.get_id() << std::endl; - } - std::cout << "node_ids: " << nodes[0].get_id() << std::endl; ASSERT_EQ(nodes[0].get_id(), 59); nodes.clear(); vs = client1.batch_sample_neighboors(std::string("user2item"), std::vector(1, 96), 4); ASSERT_EQ(vs[0].size(), 3); - std::cout << "batch sample result" << std::endl; - for (auto p : vs[0]) { - std::cout << p.first << " " << p.second << std::endl; - } std::vector node_ids; node_ids.push_back(96); node_ids.push_back(37); diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 3132403195260..1a6d00ddda0ba 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -182,7 +182,8 @@ void BindGraphPyClient(py::module* m) { .def("pull_graph_list", &GraphPyClient::pull_graph_list) .def("start_client", &GraphPyClient::start_client) .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) - .def("random_sample_nodes", &GraphPyClient::random_sample_nodes); + .def("random_sample_nodes", &GraphPyClient::random_sample_nodes) + .def("bind_local_server", &GraphPyClient::bind_local_server); } } // end namespace pybind From 09fc14540326370da33b3ee581efa3a2576408a4 Mon Sep 17 00:00:00 2001 From: luobin06 Date: Wed, 24 Mar 2021 16:03:30 +0800 Subject: [PATCH 065/106] fix get_node_feat --- .../distributed/service/graph_brpc_client.cc | 5 ++-- .../distributed/service/graph_brpc_server.cc | 3 ++- .../distributed/table/common_graph_table.cc | 14 +++++++---- paddle/fluid/distributed/table/graph_node.h | 13 ++++++++-- .../fluid/distributed/test/graph_node_test.cc | 25 +++++++++++++++---- 5 files changed, 45 insertions(+), 15 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 69e873e82b4a7..73dae38a33e37 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -83,7 +83,8 @@ std::future GraphBrpcClient::get_node_feat( int query_idx = query_idx_buckets.at(request_idx).at(node_idx); size_t feat_len = *(size_t *)(buffer); buffer += sizeof(size_t); - res[feat_idx][query_idx] = std::string(buffer, feat_len); + auto feature = std::string(buffer, feat_len); + res[feat_idx][query_idx] = feature; buffer += feat_len; } } @@ -111,7 +112,7 @@ std::future GraphBrpcClient::get_node_feat( sizeof(uint64_t) * node_num); std::string joint_feature_name = paddle::string::join_strings(feature_names, '\t'); closure->request(request_idx) - ->add_params(joint_feature_name.c_str(), sizeof(joint_feature_name)); + ->add_params(joint_feature_name.c_str(), joint_feature_name.size()); PsService_Stub rpc_stub(get_cmd_channel(server_index)); closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 29864e55fa0fb..ea0e5eaa26214 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -334,6 +334,7 @@ int32_t GraphBrpcService::graph_get_node_feat( std::vector feature_names = paddle::string::split_string(request.params(1), "\t"); + std::vector> feature (feature_names.size(), std::vector(node_num)); @@ -343,7 +344,7 @@ int32_t GraphBrpcService::graph_get_node_feat( for (size_t node_idx = 0; node_idx < node_num; ++node_idx) { size_t feat_len = feature[feat_idx][node_idx].size(); cntl->response_attachment().append(&feat_len, sizeof(size_t)); - cntl->response_attachment().append(&feature[feat_idx][node_idx], feat_len); + cntl->response_attachment().append(feature[feat_idx][node_idx].data(), feat_len); } } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 84841efc6d11f..67137ad050443 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -142,15 +142,17 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto node = shards[index].add_feature_node(id); - auto mutable_feature = node->get_mutable_feature(); + //auto mutable_feature = node->get_mutable_feature(); - mutable_feature.clear(); - mutable_feature.resize(this->feat_name.size()); + //mutable_feature.clear(); + //mutable_feature.resize(this->feat_name.size()); + node->set_feature_size(feat_name.size()); for (size_t slice = 2; slice < values.size(); slice++) { auto feat = this->parse_feature(values[slice]); if(feat.first > 0) { - mutable_feature[feat.first] = feat.second; + //mutable_feature[feat.first] = feat.second; + node->set_feature(feat.first, feat.second); } } } @@ -348,7 +350,9 @@ int32_t GraphTable::get_node_feat( for (int feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx){ const std::string &feature_name = feature_names[feat_idx]; if (feat_id_map.find(feature_name) != feat_id_map.end()){ - res[feat_idx][idx] = node->get_feature(feat_id_map[feature_name]); + //res[feat_idx][idx] = node->get_feature(feat_id_map[feature_name]); + auto feat = node->get_feature(feat_id_map[feature_name]); + res[feat_idx][idx] = feat; } } return 0; diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 9d58ce34bb70b..d38b8a36d58ff 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -41,8 +41,10 @@ class Node { virtual int get_size(bool need_feature); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual void add_feature(std::string feature) { } virtual std::string get_feature(int idx) { return std::string(""); } + virtual void set_feature(int idx, std::string str) {} + virtual void set_feature_size(int size) {} + virtual int get_feature_size() {return 0;} protected: uint64_t id; @@ -78,7 +80,14 @@ class FeatureNode: public Node{ virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); virtual std::string get_feature(int idx) { return this->feature[idx]; } - virtual std::vector & get_mutable_feature() { return this->feature; } + virtual void set_feature(int idx, std::string str) { + //if (idx >= this->feature.size()){ + //this->feature.resize(idx+1); + //} + this->feature[idx] = str; + } + virtual void set_feature_size(int size) {this->feature.resize(size);} + virtual int get_feature_size() {return this->feature.size();} template static std::string parse_value_to_bytes(std::vector feat_str) { diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 01d404e736630..96c037eef2d76 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -179,7 +179,7 @@ std::string edges[] = { char edge_file_name[] = "edges.txt"; std::string nodes[] = { - std::string("user\t37\ta 0.34\tb 13 14\tc hello"), std::string("user\t96\ta 0.31\tb 15 10"), + std::string("user\t37\ta 0.34\tb 13 14\tc hello\td abc"), std::string("user\t96\ta 0.31\tb 15 10\tc 96hello\td abcd"), std::string("user\t59\ta 0.11\tb 11 14"), std::string("user\t97\ta 0.11\tb 12 11"), std::string("item\t45\ta 0.21"), std::string("item\t145\ta 0.21"), std::string("item\t112\ta 0.21"), std::string("item\t48\ta 0.21"), @@ -401,11 +401,13 @@ void RunBrpcPushSparse() { server1.add_table_feat_conf("user", "a", "float32", 1); server1.add_table_feat_conf("user", "b", "int32", 2); server1.add_table_feat_conf("user", "c", "string", 1); + server1.add_table_feat_conf("user", "d", "string", 1); server1.add_table_feat_conf("item", "a", "float32", 1); server2.add_table_feat_conf("user", "a", "float32", 1); server2.add_table_feat_conf("user", "b", "int32", 2); server2.add_table_feat_conf("user", "c", "string", 1); + server2.add_table_feat_conf("user", "d", "string", 1); server2.add_table_feat_conf("item", "a", "float32", 1); client1.set_up(ips_str, 127, node_types, edge_types, 0); @@ -413,6 +415,7 @@ void RunBrpcPushSparse() { client1.add_table_feat_conf("user", "a", "float32", 1); client1.add_table_feat_conf("user", "b", "int32", 2); client1.add_table_feat_conf("user", "c", "string", 1); + client1.add_table_feat_conf("user", "d", "string", 1); client1.add_table_feat_conf("item", "a", "float32", 1); client2.set_up(ips_str, 127, node_types, edge_types, 1); @@ -420,6 +423,7 @@ void RunBrpcPushSparse() { client2.add_table_feat_conf("user", "a", "float32", 1); client2.add_table_feat_conf("user", "b", "int32", 2); client2.add_table_feat_conf("user", "c", "string", 1); + client2.add_table_feat_conf("user", "d", "string", 1); client2.add_table_feat_conf("item", "a", "float32", 1); server1.start_server(); @@ -488,10 +492,19 @@ void RunBrpcPushSparse() { (nodes_ids[0] == 37 && nodes_ids[1] == 59)); // Test get node feat - auto node_feat = client1.get_node_feat(std::string("user"), std::vector(1, 37), std::vector(1, std::string("c"))); - ASSERT_EQ(node_feat.size(), 1); - ASSERT_EQ(node_feat[0].size(), 1); + node_ids.clear(); + node_ids.push_back(37); + node_ids.push_back(96); + std::vector feature_names; + feature_names.push_back(std::string("c")); + feature_names.push_back(std::string("d")); + auto node_feat = client1.get_node_feat(std::string("user"), node_ids, feature_names); + ASSERT_EQ(node_feat.size(), 2); + ASSERT_EQ(node_feat[0].size(), 2); std::cout << "get_node_feat: " << node_feat[0][0] << std::endl; + std::cout << "get_node_feat: " << node_feat[0][1] << std::endl; + std::cout << "get_node_feat: " << node_feat[1][0] << std::endl; + std::cout << "get_node_feat: " << node_feat[1][1] << std::endl; std::remove(edge_file_name); std::remove(node_file_name); @@ -510,7 +523,9 @@ void RunBrpcPushSparse() { void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; - s.add_feature("hhhh"); + //s.add_feature("hhhh"); + s.set_feature_size(1); + s.set_feature(0, std::string("hhhh")); s.set_id(65); int size = s.get_size(true); char str[size]; From ad30c1cdd63661ecc88207a94b2cc8da93cfd083 Mon Sep 17 00:00:00 2001 From: luobin06 Date: Wed, 24 Mar 2021 16:20:49 +0800 Subject: [PATCH 066/106] fix get_node_feat --- paddle/fluid/distributed/table/graph_node.h | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index d38b8a36d58ff..ccbd7cffb430b 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -79,11 +79,19 @@ class FeatureNode: public Node{ virtual int get_size(bool need_feature); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual std::string get_feature(int idx) { return this->feature[idx]; } + virtual std::string get_feature(int idx) { + if (idx < (int)this->feature.size()){ + return this->feature[idx]; + } + else{ + return std::string(""); + } + } + virtual void set_feature(int idx, std::string str) { - //if (idx >= this->feature.size()){ - //this->feature.resize(idx+1); - //} + if (idx >= (int)this->feature.size()){ + this->feature.resize(idx+1); + } this->feature[idx] = str; } virtual void set_feature_size(int size) {this->feature.resize(size);} From 4a32d643b9b5a7b52f11febfce2763de776458b9 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 24 Mar 2021 08:46:33 +0000 Subject: [PATCH 067/106] remove log --- paddle/fluid/distributed/service/graph_py_service.h | 2 +- paddle/fluid/distributed/test/graph_node_test.cc | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index f74e55bf2ff0f..18ebc57db4e6e 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -111,7 +111,7 @@ class GraphPyService { }; class GraphPyServer : public GraphPyService { public: - GraphPyServer() { VLOG(0) << "construct a graphPyServer"; } + GraphPyServer() {} void set_up(std::string ips_str, int shard_num, std::vector node_types, std::vector edge_types, int rank) { diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index f196b986c68f7..b410adc157104 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -388,7 +388,6 @@ void RunBrpcPushSparse() { // } // std::vector> v; // pull_status = worker_ptr_->sample(0, 37, 4, v); - std::cerr << "start to test sample nodes" << std::endl; testSampleNodes(worker_ptr_); sleep(5); testSingleSampleNeighboor(worker_ptr_); From 6f4223c2deea2efde7375852f319abf73513e3d8 Mon Sep 17 00:00:00 2001 From: Weiyue Su Date: Wed, 24 Mar 2021 23:37:42 +0800 Subject: [PATCH 068/106] get_node_feat return py:bytes --- .../distributed/table/common_graph_table.cc | 9 +++------ paddle/fluid/distributed/test/graph_node_test.cc | 16 ++++++++++++++++ paddle/fluid/pybind/fleet_py.cc | 12 +++++++++++- 3 files changed, 30 insertions(+), 7 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 67137ad050443..5ab5d7e2f1f23 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -142,17 +142,14 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto node = shards[index].add_feature_node(id); - //auto mutable_feature = node->get_mutable_feature(); - - //mutable_feature.clear(); - //mutable_feature.resize(this->feat_name.size()); node->set_feature_size(feat_name.size()); for (size_t slice = 2; slice < values.size(); slice++) { auto feat = this->parse_feature(values[slice]); - if(feat.first > 0) { - //mutable_feature[feat.first] = feat.second; + if (feat.first >= 0) { node->set_feature(feat.first, feat.second); + } else{ + VLOG(4) << "Node feature: " << values[slice] << " not in feature_map."; } } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index a8764e78d747d..f7727a07a8208 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -526,6 +526,22 @@ void RunBrpcPushSparse() { std::cout << "get_node_feat: " << node_feat[1][0] << std::endl; std::cout << "get_node_feat: " << node_feat[1][1] << std::endl; + // Test string + node_ids.clear(); + node_ids.push_back(37); + node_ids.push_back(96); + //std::vector feature_names; + feature_names.clear(); + feature_names.push_back(std::string("a")); + feature_names.push_back(std::string("b")); + node_feat = client1.get_node_feat(std::string("user"), node_ids, feature_names); + ASSERT_EQ(node_feat.size(), 2); + ASSERT_EQ(node_feat[0].size(), 2); + std::cout << "get_node_feat: " << node_feat[0][0].size() << std::endl; + std::cout << "get_node_feat: " << node_feat[0][1].size() << std::endl; + std::cout << "get_node_feat: " << node_feat[1][0].size() << std::endl; + std::cout << "get_node_feat: " << node_feat[1][1].size() << std::endl; + std::remove(edge_file_name); std::remove(node_file_name); LOG(INFO) << "Run stop_server"; diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index b370c43c22f25..97949c48ce908 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -191,7 +191,17 @@ void BindGraphPyClient(py::module* m) { .def("start_client", &GraphPyClient::start_client) .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) .def("random_sample_nodes", &GraphPyClient::random_sample_nodes) - .def("get_node_feat", &GraphPyClient::get_node_feat) + .def("get_node_feat", [](GraphPyClient& self, std::string node_type, std::vector node_ids, + std::vector feature_names){ + auto feats = self.get_node_feat(node_type, node_ids, feature_names); + std::vector > bytes_feats(feats.size()); + for (int i = 0; i < feats.size(); ++i ){ + for (int j = 0; j < feats[i].size(); ++j ){ + bytes_feats[i].push_back(py::bytes(feats[i][j])); + } + } + return bytes_feats; + }) .def("bind_local_server", &GraphPyClient::bind_local_server); } From 5be0c8ef6deedecf40b84a93dd8e7df82c33cd2b Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 25 Mar 2021 03:30:58 +0000 Subject: [PATCH 069/106] merge develop with graph_engine --- CMakeLists.txt | 14 +- README.md | 2 +- README_cn.md | 2 +- cmake/cuda.cmake | 12 +- cmake/external/cryptopp.cmake | 9 + cmake/external/mkldnn.cmake | 2 +- cmake/external/protobuf.cmake | 3 + cmake/external/pslib_brpc.cmake | 3 +- cmake/external/warpctc.cmake | 14 +- cmake/external/xpu.cmake | 7 +- cmake/generic.cmake | 16 +- cmake/hip.cmake | 1 + cmake/inference_lib.cmake | 23 +- cmake/init.cmake | 4 + cmake/paddle_win.props | 2 +- cmake/third_party.cmake | 5 +- paddle/extension.h | 2 +- paddle/fluid/distributed/fleet.cc | 2 +- .../distributed/service/brpc_ps_server.cc | 7 +- .../fluid/distributed/service/communicator.cc | 2 +- .../fluid/distributed/service/communicator.h | 10 +- .../distributed/service/graph_brpc_client.cc | 92 ++ .../distributed/service/graph_brpc_client.h | 4 + .../distributed/service/graph_brpc_server.cc | 37 + .../distributed/service/graph_brpc_server.h | 3 + .../distributed/service/graph_py_service.cc | 16 + .../distributed/service/graph_py_service.h | 3 + .../fluid/distributed/service/heter_client.cc | 8 +- .../fluid/distributed/service/heter_server.cc | 2 +- .../fluid/distributed/service/heter_server.h | 2 +- paddle/fluid/distributed/service/ps_client.h | 10 + .../fluid/distributed/service/sendrecv.proto | 3 +- .../distributed/table/common_dense_table.cc | 3 +- .../distributed/table/common_dense_table.h | 7 +- .../distributed/table/common_graph_table.cc | 137 +- .../distributed/table/common_graph_table.h | 14 +- .../distributed/table/common_sparse_table.cc | 16 +- paddle/fluid/distributed/table/common_table.h | 9 +- paddle/fluid/distributed/table/graph_node.h | 67 +- paddle/fluid/distributed/table/table.h | 5 + .../fluid/distributed/test/graph_node_test.cc | 73 +- .../extension/include/{all.h => ext_all.h} | 11 +- .../include/{dispatch.h => ext_dispatch.h} | 117 +- .../include/{dll_decl.h => ext_dll_decl.h} | 0 .../include/{dtype.h => ext_dtype.h} | 23 +- .../fluid/extension/include/ext_exception.h | 108 ++ .../extension/include/ext_op_meta_info.h | 539 +++++++ .../include/{place.h => ext_place.h} | 0 .../include/{tensor.h => ext_tensor.h} | 39 +- paddle/fluid/extension/include/op_meta_info.h | 379 ----- .../{op_meta_info.cc => ext_op_meta_info.cc} | 53 +- .../src/{tensor.cc => ext_tensor.cc} | 121 +- paddle/fluid/framework/CMakeLists.txt | 77 +- paddle/fluid/framework/array.h | 18 +- paddle/fluid/framework/conv_search_cache.h | 28 +- .../fluid/framework/copy_same_tensor_test.cc | 2 +- paddle/fluid/framework/custom_operator.cc | 396 ++++-- paddle/fluid/framework/custom_operator.h | 2 +- paddle/fluid/framework/custom_tensor_test.cc | 89 +- paddle/fluid/framework/custom_tensor_utils.h | 46 +- paddle/fluid/framework/data_feed.cc | 21 +- paddle/fluid/framework/data_feed.h | 2 +- paddle/fluid/framework/data_feed_factory.cc | 2 +- paddle/fluid/framework/data_type.h | 1 + paddle/fluid/framework/data_type_transform.cc | 2 +- .../framework/details/broadcast_op_handle.cc | 2 +- .../framework/details/broadcast_op_handle.h | 8 +- .../details/broadcast_op_handle_test.cc | 3 +- .../details/broadcast_op_handle_test.h | 10 +- .../framework/details/exception_holder.h | 2 +- .../details/fused_all_reduce_op_handle.cc | 10 +- .../framework/details/nan_inf_utils_detail.cu | 11 +- paddle/fluid/framework/device_worker.h | 67 +- .../fluid/framework/device_worker_factory.cc | 8 +- paddle/fluid/framework/dim_test.cu | 10 + paddle/fluid/framework/dist_multi_trainer.cc | 5 +- .../framework/distributed_strategy.proto | 6 +- paddle/fluid/framework/dlpack_tensor.cc | 4 +- paddle/fluid/framework/dlpack_tensor_test.cc | 2 +- paddle/fluid/framework/executor.cc | 2 +- paddle/fluid/framework/fleet/fleet_wrapper.cc | 2 - paddle/fluid/framework/fleet/heter_context.h | 9 +- .../cudf/concurrent_unordered_map.cuh.h | 6 +- .../framework/fleet/heter_ps/hashtable.h | 2 + .../framework/fleet/heter_ps/heter_comm.h | 13 +- .../framework/fleet/heter_ps/heter_comm_inl.h | 148 +- .../fluid/framework/fleet/ps_gpu_wrapper.cc | 15 +- paddle/fluid/framework/fleet/ps_gpu_wrapper.h | 2 +- paddle/fluid/framework/garbage_collector.cc | 15 +- paddle/fluid/framework/garbage_collector.h | 6 +- paddle/fluid/framework/generator.cc | 4 +- paddle/fluid/framework/heter_service.h | 2 +- paddle/fluid/framework/heterbox_trainer.cc | 29 +- paddle/fluid/framework/io/crypto/cipher.cc | 5 - .../framework/ir/graph_pattern_detector.cc | 14 +- .../ir/mkldnn/cpu_bfloat16_placement_pass.cc | 4 +- .../ir/mkldnn/cpu_bfloat16_placement_pass.h | 2 +- paddle/fluid/framework/lod_tensor.h | 2 +- paddle/fluid/framework/lod_tensor_test.cu | 14 +- paddle/fluid/framework/mixed_vector.h | 2 +- paddle/fluid/framework/mixed_vector_test.cu | 26 +- paddle/fluid/framework/op_meta_info_helper.h | 2 +- paddle/fluid/framework/op_registry.h | 2 +- paddle/fluid/framework/operator.cc | 28 +- paddle/fluid/framework/operator.h | 2 +- paddle/fluid/framework/parallel_executor.cc | 35 +- paddle/fluid/framework/parallel_executor.h | 2 +- paddle/fluid/framework/pipeline_trainer.cc | 12 +- paddle/fluid/framework/ps_gpu_trainer.cc | 12 +- paddle/fluid/framework/ps_gpu_worker.cc | 79 +- paddle/fluid/framework/pull_dense_worker.cc | 18 +- paddle/fluid/framework/save_load_util.cc | 2 +- paddle/fluid/framework/section_worker.cc | 169 ++- paddle/fluid/framework/tensor_test.cc | 6 +- paddle/fluid/framework/tensor_util.cc | 18 +- paddle/fluid/framework/tensor_util.h | 6 +- paddle/fluid/framework/tensor_util_test.cc | 10 +- paddle/fluid/framework/tensor_util_test.cu | 71 + paddle/fluid/framework/trainer.h | 33 +- paddle/fluid/framework/trainer_desc.proto | 3 + paddle/fluid/framework/trainer_factory.cc | 8 +- paddle/fluid/framework/var_type_traits.cc | 8 + paddle/fluid/framework/var_type_traits.h | 14 +- .../fluid/framework/var_type_traits_test.cc | 8 + paddle/fluid/imperative/data_loader.cc | 24 +- .../imperative/jit/program_desc_tracer.cc | 26 +- .../imperative/jit/program_desc_tracer.h | 2 +- paddle/fluid/imperative/prepared_operator.cc | 1 + paddle/fluid/imperative/reducer.cc | 96 +- paddle/fluid/imperative/reducer.h | 11 +- paddle/fluid/inference/CMakeLists.txt | 7 +- paddle/fluid/inference/analysis/helper.h | 4 +- .../inference/analysis/ir_pass_manager.cc | 22 +- .../ir_passes/tensorrt_subgraph_pass.cc | 51 +- .../analysis/passes/memory_optimize_pass.cc | 1 + paddle/fluid/inference/api/CMakeLists.txt | 9 +- .../fluid/inference/api/analysis_predictor.cc | 36 +- paddle/fluid/inference/api/api.cc | 6 + paddle/fluid/inference/api/api_tester.cc | 5 + .../inference/api/demo_ci/CMakeLists.txt | 6 +- paddle/fluid/inference/api/demo_ci/run.sh | 4 +- .../api/demo_ci/run_windows_demo.bat | 8 +- .../api/demo_ci/windows_inference.md | 2 +- .../inference/api/details/CMakeLists.txt | 2 + .../inference/api/details/zero_copy_tensor.cc | 262 ++-- .../api/details/zero_copy_tensor_dummy.cc | 30 +- .../api/details/zero_copy_tensor_test.cc | 138 ++ paddle/fluid/inference/api/helper.h | 20 + paddle/fluid/inference/api/paddle_api.h | 82 +- .../inference/api/paddle_inference_api.h | 114 +- paddle/fluid/inference/api/paddle_tensor.h | 111 ++ .../inference/tensorrt/convert/CMakeLists.txt | 3 + .../inference/tensorrt/convert/conv2d_op.cc | 7 + .../fluid/inference/tensorrt/convert/fc_op.cc | 70 +- .../inference/tensorrt/convert/gather_op.cc | 78 ++ .../tensorrt/convert/multihead_matmul_op.cc | 89 +- .../tensorrt/convert/nearest_interp_op.cc | 114 ++ .../inference/tensorrt/convert/prelu_op.cc | 25 + .../inference/tensorrt/convert/scale_op.cc | 33 +- .../inference/tensorrt/convert/softmax_op.cc | 13 +- .../inference/tensorrt/convert/split_op.cc | 2 +- paddle/fluid/inference/tensorrt/engine.cc | 9 +- paddle/fluid/inference/tensorrt/engine.h | 7 + paddle/fluid/inference/tensorrt/op_teller.cc | 52 +- .../inference/tensorrt/plugin/CMakeLists.txt | 3 + .../tensorrt/plugin/elementwise_op_plugin.cu | 9 +- .../tensorrt/plugin/elementwise_op_plugin.h | 47 +- .../plugin/emb_eltwise_layernorm_plugin.cu | 4 +- .../tensorrt/plugin/pool_op_plugin.cu | 8 +- .../tensorrt/plugin/qkv_to_context_plugin.cu | 4 +- .../plugin/skip_layernorm_op_plugin.cu | 5 - .../tensorrt/plugin/special_slice_plugin.cu | 7 + .../tensorrt/plugin/split_op_plugin.cu | 5 - .../tensorrt/plugin/split_op_plugin.h | 69 +- .../tensorrt/plugin/test_split_plugin.cc | 58 + .../inference/tensorrt/plugin/trt_plugin.cc | 78 +- .../inference/tensorrt/plugin/trt_plugin.h | 112 +- .../fluid/inference/tests/api/CMakeLists.txt | 16 +- .../api/full_ILSVRC2012_val_preprocess.py | 27 +- .../inference/tests/api/lite_resnet50_test.cc | 1 + .../tests/api/trt_dynamic_shape_test.cc | 29 +- .../inference/tests/api/trt_test_helper.h | 1 + paddle/fluid/memory/memcpy.cc | 2 +- paddle/fluid/operators/CMakeLists.txt | 13 +- paddle/fluid/operators/activation_cudnn.cu.cc | 4 + .../fluid/operators/activation_cudnn_op.cu.cc | 80 +- paddle/fluid/operators/activation_op.cc | 3 - paddle/fluid/operators/activation_op.cu | 284 +++- paddle/fluid/operators/activation_op.h | 6 +- paddle/fluid/operators/affine_channel_op.cu | 16 + .../operators/affine_grid_cudnn_op.cu.cc | 5 + paddle/fluid/operators/affine_grid_op.cc | 7 +- paddle/fluid/operators/allclose_op.cu | 5 +- .../fluid/operators/arg_min_max_op_base.cu.h | 10 +- paddle/fluid/operators/argsort_op.cu | 18 +- .../fluid/operators/array_to_lod_tensor_op.cc | 2 +- paddle/fluid/operators/assign_op.cc | 2 +- paddle/fluid/operators/batch_fc_op.cu | 5 +- paddle/fluid/operators/batch_norm_op.cu | 179 ++- paddle/fluid/operators/bce_loss_op.cu | 1 - paddle/fluid/operators/bmm_op.cu | 2 +- paddle/fluid/operators/cast_op.cc | 1 + paddle/fluid/operators/cholesky_op.cu | 5 + paddle/fluid/operators/clip_op.h | 4 +- paddle/fluid/operators/coalesce_tensor_op.cc | 12 +- .../operators/collective/c_comm_init_op.cc | 8 +- .../operators/collective/c_gen_nccl_id_op.cc | 3 +- .../operators/collective/c_wait_comm_op.cc | 91 ++ .../operators/collective/c_wait_compute_op.cc | 95 ++ .../operators/collective/gen_bkcl_id_op.cc | 2 +- .../fluid/operators/collective/recv_v2_op.cc | 8 + .../operators/collective/recv_v2_op.cu.cc | 29 +- .../operators/collective/send_v2_op.cu.cc | 15 - paddle/fluid/operators/conv_cudnn_op.cu | 256 +++- paddle/fluid/operators/conv_cudnn_op_cache.h | 11 +- paddle/fluid/operators/conv_miopen_helper.h | 308 ++++ paddle/fluid/operators/conv_op.cc | 23 +- .../operators/conv_transpose_cudnn_op.cu | 205 ++- paddle/fluid/operators/conv_transpose_op.cc | 6 +- paddle/fluid/operators/conv_transpose_op.h | 4 +- paddle/fluid/operators/correlation_op.cu | 5 + paddle/fluid/operators/cudnn_lstm_op.cu.cc | 73 +- paddle/fluid/operators/cumsum_op.cu | 8 +- paddle/fluid/operators/data_norm_op.cu | 8 +- .../fluid/operators/detection/bbox_util.cu.h | 35 +- .../detection/collect_fpn_proposals_op.cu | 48 +- .../detection/distribute_fpn_proposals_op.cu | 51 +- .../detection/generate_proposals_op.cu | 1 - paddle/fluid/operators/diag_embed_op.h | 2 +- paddle/fluid/operators/dist_op.cu | 9 + .../operators/distributed_ops/CMakeLists.txt | 2 +- .../operators/distributed_ops/allreduce_op.h | 8 +- .../distributed_ops/broadcast_op.cu.cc | 8 +- .../distributed_ops/ref_by_trainer_id_op.h | 2 +- paddle/fluid/operators/dot_op.h | 6 +- paddle/fluid/operators/dropout_op.cu | 54 +- paddle/fluid/operators/dropout_op.h | 4 +- .../elementwise/elementwise_add_op.h | 12 +- .../elementwise/elementwise_div_op.h | 2 +- .../elementwise/elementwise_floordiv_op.h | 12 +- .../elementwise/elementwise_mul_op.h | 2 +- .../operators/elementwise/elementwise_op.h | 10 +- .../elementwise/elementwise_op_function.cu.h | 18 +- .../elementwise/elementwise_op_function.h | 73 +- .../elementwise/elementwise_pow_op.h | 2 +- .../elementwise/elementwise_sub_op.h | 2 +- .../mkldnn/elementwise_add_mkldnn_op.cc | 42 +- .../mkldnn/elementwise_mkldnn_op.h | 1 - .../mkldnn/elementwise_mul_mkldnn_op.cc | 116 ++ .../test_elementwise_add_grad_grad.cc | 2 +- .../test_elementwise_add_op_inplace.cc | 4 +- .../test_elementwise_div_grad_grad.cc | 2 +- .../test_elementwise_op_grad_grad.h | 8 +- paddle/fluid/operators/fake_quantize_op.cu | 4 + .../fill_constant_batch_size_like_op.h | 2 +- paddle/fluid/operators/fill_constant_op.h | 4 +- paddle/fluid/operators/filter_by_instag_op.h | 2 +- paddle/fluid/operators/fused/CMakeLists.txt | 25 +- .../fused_embedding_eltwise_layernorm_op.cu | 14 +- .../fused_fc_elementwise_layernorm_op.cu | 7 + .../fluid/operators/fused/fusion_lstm_op.cc | 4 + .../fused/mkldnn/fusion_gru_mkldnn_op.cc | 64 +- .../fused/mkldnn/fusion_lstm_mkldnn_op.cc | 74 +- .../fused/mkldnn/fusion_rnn_mkldnn.h | 18 +- .../operators/fused/multihead_matmul_op.cu | 3 +- .../operators/fused/skip_layernorm_op.cu | 1 - paddle/fluid/operators/gelu_op.h | 6 +- .../get_tensor_from_selected_rows_op.cc | 2 +- .../operators/grid_sampler_cudnn_op.cu.cc | 5 + paddle/fluid/operators/grid_sampler_op.cc | 7 +- paddle/fluid/operators/group_norm_op.cu | 16 +- paddle/fluid/operators/gru_unit_op.h | 35 +- paddle/fluid/operators/index_select_op.cu | 16 + paddle/fluid/operators/inplace_abn_op.cu | 9 + paddle/fluid/operators/instance_norm_op.cu | 118 +- paddle/fluid/operators/kldiv_loss_op.h | 16 +- paddle/fluid/operators/kron_op.h | 8 +- paddle/fluid/operators/layer_norm_op.cu | 50 +- paddle/fluid/operators/layer_norm_op.h | 8 +- paddle/fluid/operators/linear_chain_crf_op.h | 23 +- paddle/fluid/operators/lite/CMakeLists.txt | 3 +- paddle/fluid/operators/lite/lite_engine_op.h | 4 +- .../operators/lite/lite_engine_op_test.cc | 10 +- paddle/fluid/operators/lite/ut_helper.h | 4 +- .../fluid/operators/lod_tensor_to_array_op.cc | 2 +- paddle/fluid/operators/lookup_table_op.cc | 7 +- paddle/fluid/operators/lookup_table_op.cu | 28 +- paddle/fluid/operators/lookup_table_op.h | 6 +- paddle/fluid/operators/math/CMakeLists.txt | 11 +- paddle/fluid/operators/math/algorithm.h | 8 +- .../fluid/operators/math/beam_search_test.cc | 2 +- .../operators/math/bert_encoder_functor.cu | 37 +- .../operators/math/bert_encoder_functor.h | 15 +- paddle/fluid/operators/math/blas.h | 22 +- paddle/fluid/operators/math/blas_impl.h | 22 +- paddle/fluid/operators/math/blas_impl.hip.h | 712 ++++++++++ paddle/fluid/operators/math/concat_test.cc | 2 +- paddle/fluid/operators/math/depthwise_conv.cu | 15 +- .../math/detail/activation_functions.h | 95 ++ .../operators/math/detail/gru_cpu_kernel.h | 5 +- .../fluid/operators/math/detail/gru_kernel.h | 20 +- .../operators/math/detail/lstm_cpu_kernel.h | 4 +- .../fluid/operators/math/detail/lstm_kernel.h | 8 +- paddle/fluid/operators/math/fc.cu | 2 +- paddle/fluid/operators/math/gru_compute.cc | 8 +- paddle/fluid/operators/math/im2col_test.cc | 2 +- paddle/fluid/operators/math/math_cuda_utils.h | 100 +- paddle/fluid/operators/math/math_function.cc | 2 +- paddle/fluid/operators/math/pooling.cc | 34 +- paddle/fluid/operators/math/pooling.cu | 66 +- paddle/fluid/operators/math/pooling.h | 44 +- paddle/fluid/operators/math/prelu.cu | 6 +- paddle/fluid/operators/math/prelu.h | 18 +- paddle/fluid/operators/math/sample_prob.cu | 14 + paddle/fluid/operators/math/sample_prob.h | 2 +- .../math/selected_rows_functor_test.cu.cc | 6 + .../operators/math/sequence_padding_test.cc | 2 +- .../operators/math/sequence_pooling_test.cc | 2 +- paddle/fluid/operators/math/sequence_scale.cu | 8 + paddle/fluid/operators/math/softmax.cu | 39 +- paddle/fluid/operators/math/softmax.h | 2 +- paddle/fluid/operators/math/vol2col_test.cc | 2 +- paddle/fluid/operators/matmul_op.cc | 20 +- paddle/fluid/operators/matmul_op_xpu.cc | 25 +- paddle/fluid/operators/matmul_v2_op.h | 4 +- paddle/fluid/operators/matmul_v2_op_xpu.cc | 24 +- paddle/fluid/operators/mean_op.cu | 6 + paddle/fluid/operators/merge_lod_tensor_op.cc | 2 +- paddle/fluid/operators/miopen_lstm_cache.h | 141 ++ paddle/fluid/operators/miopen_rnn_cache.h | 267 ++++ .../operators/mkldnn/concat_mkldnn_op.cc | 20 +- .../fluid/operators/mkldnn/conv_mkldnn_op.cc | 7 +- .../fluid/operators/modified_huber_loss_op.h | 4 +- paddle/fluid/operators/multinomial_op.cu | 21 +- paddle/fluid/operators/nccl/CMakeLists.txt | 14 +- paddle/fluid/operators/nccl/nccl_gpu_common.h | 4 + paddle/fluid/operators/nll_loss_op.cu | 9 +- paddle/fluid/operators/norm_op.cu | 15 +- paddle/fluid/operators/norm_utils.cu.h | 10 + .../fluid/operators/optimizers/lamb_op_xpu.cc | 125 ++ paddle/fluid/operators/p_norm_op.cu | 16 + paddle/fluid/operators/pool_cudnn_op.cu.cc | 165 ++- paddle/fluid/operators/pool_op.cc | 7 +- .../operators/{pool_op.cu.cc => pool_op.cu} | 0 paddle/fluid/operators/pool_op.h | 79 +- paddle/fluid/operators/prelu_op.cu | 2 +- paddle/fluid/operators/prroi_pool_op.cu | 27 +- paddle/fluid/operators/prroi_pool_op.h | 131 +- paddle/fluid/operators/pscore/CMakeLists.txt | 2 +- paddle/fluid/operators/pull_box_sparse_op.h | 6 +- paddle/fluid/operators/random_crop_op.h | 4 +- paddle/fluid/operators/range_op.cu | 28 +- paddle/fluid/operators/rank_attention.cu.h | 6 +- paddle/fluid/operators/rank_attention_op.cu | 1 - .../fluid/operators/reader/blocking_queue.h | 12 +- .../fluid/operators/reader/buffered_reader.cc | 21 +- .../fluid/operators/reader/buffered_reader.h | 6 +- paddle/fluid/operators/recurrent_op.cc | 25 +- .../fluid/operators/reduce_ops/CMakeLists.txt | 6 +- .../fluid/operators/reduce_ops/cub_reduce.h | 22 +- .../operators/reduce_ops/reduce_prod_op.cu | 12 + paddle/fluid/operators/reshape_op.cc | 2 +- paddle/fluid/operators/rnn_op.cu.cc | 180 ++- paddle/fluid/operators/roi_align_op.cu | 13 +- paddle/fluid/operators/roi_align_op.h | 12 +- paddle/fluid/operators/scale_op.cc | 2 + paddle/fluid/operators/seed_op.cu | 1 - paddle/fluid/operators/segment_pool_op.h | 8 +- paddle/fluid/operators/select_op_helper.h | 2 +- .../operators/sequence_ops/sequence_mask_op.h | 4 +- .../sequence_ops/sequence_reverse_op.h | 4 +- .../sequence_softmax_cudnn_op.cu.cc | 9 + .../sequence_ops/sequence_softmax_op.cc | 4 +- .../sequence_ops/sequence_softmax_op.cu | 11 +- paddle/fluid/operators/set_value_op.cc | 3 +- paddle/fluid/operators/set_value_op.h | 3 +- paddle/fluid/operators/shuffle_batch_op.h | 2 +- .../sigmoid_cross_entropy_with_logits_op.cu | 10 + paddle/fluid/operators/softmax_cudnn_op.cu | 43 +- paddle/fluid/operators/softmax_op.cc | 8 +- .../softmax_with_cross_entropy_op.cc | 12 + .../softmax_with_cross_entropy_op.cu | 452 +++++- .../operators/softmax_with_cross_entropy_op.h | 122 +- paddle/fluid/operators/split_lod_tensor_op.cc | 2 +- .../fluid/operators/split_selected_rows_op.h | 2 +- paddle/fluid/operators/spp_op.h | 10 +- paddle/fluid/operators/strided_memcpy.h | 2 +- paddle/fluid/operators/strided_memcpy_test.cc | 2 +- paddle/fluid/operators/sync_batch_norm_op.cu | 11 + .../fluid/operators/sync_batch_norm_op.cu.h | 13 +- paddle/fluid/operators/temporal_shift_op.cc | 19 +- paddle/fluid/operators/temporal_shift_op.cu | 192 ++- paddle/fluid/operators/temporal_shift_op.h | 211 ++- .../operators/tensorrt/tensorrt_engine_op.h | 12 +- .../tensorrt/tensorrt_engine_op_test.cc | 4 + .../test_leaky_relu_grad_grad_functor.h | 4 +- paddle/fluid/operators/top_k_function_cuda.h | 70 +- paddle/fluid/operators/top_k_op.cu | 5 + paddle/fluid/operators/top_k_op.h | 13 +- paddle/fluid/operators/top_k_v2_op.cu | 3 +- paddle/fluid/operators/top_k_v2_op.h | 13 +- paddle/fluid/operators/trace_op.h | 4 +- paddle/fluid/operators/unique_op.cu | 1 + paddle/fluid/operators/unity_build_rule.cmake | 46 +- paddle/fluid/operators/unstack_op.h | 8 +- paddle/fluid/operators/warpctc_op.cc | 3 + paddle/fluid/operators/warpctc_op.h | 1 + paddle/fluid/platform/bfloat16.h | 125 +- paddle/fluid/platform/bfloat16_test.cc | 1 + paddle/fluid/platform/collective_helper.cc | 28 + paddle/fluid/platform/collective_helper.h | 2 + paddle/fluid/platform/complex128.h | 122 +- paddle/fluid/platform/complex64.h | 125 +- paddle/fluid/platform/cuda_helper.h | 11 + paddle/fluid/platform/device_context.cc | 6 + paddle/fluid/platform/device_context.h | 6 +- paddle/fluid/platform/device_context_test.cu | 4 + .../fluid/platform/device_memory_aligment.cc | 3 + paddle/fluid/platform/dynload/miopen.h | 1 + paddle/fluid/platform/eigen_ext.h | 402 ++++++ paddle/fluid/platform/float16.h | 152 +- paddle/fluid/platform/float16_test.cc | 14 +- paddle/fluid/platform/float16_test.cu | 1 + paddle/fluid/platform/for_range.h | 5 + paddle/fluid/platform/gen_comm_id_helper.cc | 18 + paddle/fluid/platform/gen_comm_id_helper.h | 20 + paddle/fluid/platform/gpu_launch_config.h | 5 + paddle/fluid/platform/miopen_desc.h | 134 +- paddle/fluid/platform/miopen_helper.h | 17 - paddle/fluid/platform/mkldnn_reuse.h | 54 +- paddle/fluid/pybind/CMakeLists.txt | 67 +- paddle/fluid/pybind/fleet_py.cc | 1 + .../pybind/global_value_getter_setter.cc | 11 +- paddle/fluid/pybind/imperative.cc | 13 +- paddle/fluid/pybind/ps_gpu_wrapper_py.cc | 3 +- paddle/fluid/pybind/ps_gpu_wrapper_py.h | 3 +- paddle/fluid/pybind/pybind.cc | 63 +- paddle/fluid/pybind/reader_py.cc | 10 +- paddle/fluid/pybind/tensor_py.h | 17 +- paddle/fluid/string/tinyformat/tinyformat.h | 2 +- paddle/scripts/paddle_build.bat | 68 +- paddle/scripts/paddle_build.sh | 179 ++- paddle/scripts/windows_build/build.bat | 12 +- paddle/scripts/windows_build/config.ini | 2 +- patches/cryptopp/CMakeLists.txt | 1239 +++++++++++++++++ python/CMakeLists.txt | 1 + python/paddle/distributed/cloud_utils.py | 20 +- .../distributed/fleet/base/fleet_base.py | 5 + .../paddle/distributed/fleet/launch_utils.py | 2 +- .../meta_optimizers/pipeline_optimizer.py | 20 +- .../distributed/fleet/runtime/the_one_ps.py | 3 +- python/paddle/distributed/fleet/utils/fs.py | 3 - python/paddle/distributed/spawn.py | 141 +- python/paddle/distributed/utils.py | 26 +- python/paddle/fluid/__init__.py | 1 - python/paddle/fluid/compiler.py | 4 - .../fluid/contrib/mixed_precision/__init__.py | 3 + .../contrib/mixed_precision/bf16/__init__.py | 24 + .../contrib/mixed_precision/bf16/amp_lists.py | 97 ++ .../contrib/mixed_precision/bf16/amp_utils.py | 296 ++++ .../contrib/mixed_precision/fp16_lists.py | 2 +- .../slim/quantization/imperative/qat.py | 719 ++++++---- .../slim/quantization/imperative/quant_nn.py | 78 +- .../slim/quantization/imperative/utils.py | 87 ++ .../quantization/quant2_int8_mkldnn_pass.py | 50 +- .../slim/tests/test_imperative_out_scale.py | 135 +- .../test_imperative_qat_addquantdequant.py | 4 +- .../slim/tests/test_imperative_skip_op.py | 16 +- .../fluid/contrib/tests/test_bf16_utils.py | 144 ++ .../contrib/tests/test_model_cast_to_bf16.py | 138 ++ python/paddle/fluid/data_feeder.py | 23 +- python/paddle/fluid/dataloader/collate.py | 87 ++ .../fluid/dataloader/dataloader_iter.py | 342 +---- python/paddle/fluid/dataloader/dataset.py | 2 +- python/paddle/fluid/dataloader/flat.py | 150 ++ python/paddle/fluid/dataloader/worker.py | 253 ++++ python/paddle/fluid/debugger.py | 85 -- python/paddle/fluid/device_worker.py | 12 + .../dygraph_to_static/convert_operators.py | 14 +- .../dygraph_to_static/list_transformer.py | 8 +- .../dygraph_to_static/loop_transformer.py | 74 +- .../tensor_shape_transformer.py | 93 +- .../fluid/dygraph/dygraph_to_static/utils.py | 67 +- python/paddle/fluid/dygraph/jit.py | 7 +- python/paddle/fluid/framework.py | 11 +- .../fluid/incubate/fleet/base/role_maker.py | 74 +- .../fleet/parameter_server/ir/public.py | 2 +- .../fleet/parameter_server/ir/trainer_pass.py | 2 +- python/paddle/fluid/layers/control_flow.py | 4 + python/paddle/fluid/layers/nn.py | 38 +- python/paddle/fluid/layers/tensor.py | 6 +- python/paddle/fluid/layers/utils.py | 2 +- python/paddle/fluid/multiprocess_utils.py | 4 + python/paddle/fluid/optimizer.py | 17 +- python/paddle/fluid/tests/book/CMakeLists.txt | 2 +- .../fluid/tests/book/test_fit_a_line.py | 17 +- .../fluid/tests/book/test_word2vec_book.py | 29 +- .../fluid/tests/custom_op/CMakeLists.txt | 20 +- .../fluid/tests/custom_op/attr_test_op.cc | 190 ++- .../fluid/tests/custom_op/concat_and_split.h | 84 ++ .../fluid/tests/custom_op/custom_concat_op.cc | 236 ++++ .../fluid/tests/custom_op/custom_conj_op.cc | 94 ++ .../fluid/tests/custom_op/custom_relu_op.cc | 23 +- .../fluid/tests/custom_op/custom_relu_op.cu | 15 +- .../tests/custom_op/custom_relu_op_dup.cc | 13 +- .../fluid/tests/custom_op/dispatch_test_op.cc | 66 +- .../tests/custom_op/multi_out_test_op.cc | 2 +- .../fluid/tests/custom_op/test_check_abi.py | 89 +- .../fluid/tests/custom_op/test_check_error.cc | 169 +++ .../tests/custom_op/test_custom_attrs_jit.py | 43 +- .../tests/custom_op/test_custom_concat.py | 172 +++ .../fluid/tests/custom_op/test_custom_conj.py | 136 ++ .../tests/custom_op/test_custom_relu_model.py | 310 +++++ .../custom_op/test_custom_relu_op_jit.py | 51 +- .../custom_op/test_custom_relu_op_setup.py | 14 +- .../tests/custom_op/test_dispatch_jit.py | 10 +- .../fluid/tests/custom_op/test_jit_load.py | 7 +- .../tests/custom_op/test_multi_out_jit.py | 4 +- python/paddle/fluid/tests/custom_op/utils.py | 8 +- .../fluid/tests/unittests/CMakeLists.txt | 1 + .../unittests/collective_allreduce_op_wait.py | 114 ++ .../tests/unittests/collective_sendrecv_op.py | 5 +- .../paddle/fluid/tests/unittests/dist_test.sh | 35 +- .../test_convert_operators.py | 72 + .../dygraph_to_static/test_for_enumerate.py | 29 +- .../unittests/dygraph_to_static/test_loop.py | 20 +- .../dygraph_to_static/test_tensor_shape.py | 70 +- .../ir/inference/test_trt_conv_pass.py | 9 +- .../ir/inference/test_trt_gather_op.py | 70 + .../inference/test_trt_nearest_interp_op.py | 192 +++ .../ir/inference/test_trt_scale_op.py | 52 + .../ir/inference/test_trt_subgraph_pass.py | 52 + .../test_elementwise_add_bf16_mkldnn_op.py | 41 +- .../mkldnn/test_elementwise_add_mkldnn_op.py | 12 +- .../test_elementwise_mul_bf16_mkldnn_op.py | 66 +- .../mkldnn/test_elementwise_mul_mkldnn_op.py | 12 +- .../mkldnn/test_fusion_gru_bf16_mkldnn_op.py | 38 +- .../mkldnn/test_fusion_gru_int8_mkldnn_op.py | 2 + .../mkldnn/test_fusion_lstm_bf16_mkldnn_op.py | 159 +++ .../unittests/mkldnn/test_reshape_bf16_op.py | 7 +- .../paddle/fluid/tests/unittests/op_test.py | 46 +- .../unittests/parallel_executor_test_base.py | 1 - .../fluid/tests/unittests/pipeline_mnist.py | 22 +- .../unittests/pipeline_mnist_one_device.py | 4 + .../rnn/test_rnn_cudnn_params_packing.py | 53 + .../sequence/test_sequence_softmax_op.py | 6 +- .../tests/unittests/test_activation_op.py | 11 +- .../tests/unittests/test_batch_norm_op_v2.py | 14 +- .../fluid/tests/unittests/test_cholesky_op.py | 9 +- .../tests/unittests/test_collective_wait.py | 37 + .../fluid/tests/unittests/test_compare_op.py | 3 + .../fluid/tests/unittests/test_conv2d_op.py | 20 + .../unittests/test_conv2d_transpose_op.py | 2 +- .../fluid/tests/unittests/test_conv3d_op.py | 14 + .../tests/unittests/test_deform_conv2d.py | 94 +- .../tests/unittests/test_dist_fleet_base.py | 15 +- .../unittests/test_dist_fleet_grad_clip.py | 87 +- .../fluid/tests/unittests/test_dist_op.py | 22 +- .../test_fleet_distributed_strategy.py | 7 +- .../tests/unittests/test_fleet_launch_ps.sh | 42 +- .../test_fleet_pipeline_meta_optimizer.py | 5 +- .../tests/unittests/test_functional_conv2d.py | 15 + .../unittests/test_fuse_all_reduce_pass.py | 10 + .../unittests/test_gast_with_compatibility.py | 1 - .../test_imperative_ocr_attention_model.py | 34 +- .../tests/unittests/test_jit_save_load.py | 34 + .../unittests/test_lookup_table_bf16_op.py | 176 +++ .../unittests/test_math_op_patch_var_base.py | 7 +- .../tests/unittests/test_matmul_v2_op.py | 7 +- .../test_multiprocess_dataloader_dataset.py | 57 + ...ocess_dataloader_iterable_dataset_split.py | 4 +- .../fluid/tests/unittests/test_pad3d_op.py | 33 + .../fluid/tests/unittests/test_pipeline.py | 6 +- .../fluid/tests/unittests/test_pool2d_op.py | 6 +- .../fluid/tests/unittests/test_reduce_op.py | 22 +- .../fluid/tests/unittests/test_rnn_op.py | 27 +- .../tests/unittests/test_roi_align_op.py | 6 +- .../fluid/tests/unittests/test_softmax_op.py | 12 +- .../test_softmax_with_cross_entropy_op.py | 294 +++- .../test_spawn_and_init_parallel_env.py | 10 +- .../unittests/test_sync_batch_norm_op.py | 7 +- .../tests/unittests/test_temporal_shift_op.py | 56 +- .../fluid/tests/unittests/test_var_base.py | 25 +- .../tests/unittests/xpu/test_lamb_op_xpu.py | 121 ++ python/paddle/hapi/model_summary.py | 6 +- python/paddle/nn/functional/conv.py | 9 + python/paddle/nn/functional/norm.py | 9 +- python/paddle/nn/layer/common.py | 24 +- python/paddle/nn/layer/conv.py | 6 + python/paddle/nn/layer/pooling.py | 4 +- python/paddle/optimizer/adam.py | 2 +- python/paddle/optimizer/adamax.py | 2 +- python/paddle/optimizer/optimizer.py | 19 +- python/paddle/static/amp/__init__.py | 3 + .../utils/cpp_extension/cpp_extension.py | 133 +- .../utils/cpp_extension/extension_utils.py | 267 +++- python/paddle/utils/download.py | 38 - python/paddle/vision/ops.py | 13 +- python/paddle/vision/transforms/transforms.py | 7 +- python/setup.py.in | 29 +- tools/check_api_approvals.sh | 18 +- tools/check_file_diff_approvals.sh | 44 +- tools/dockerfile/Dockerfile.rocm | 55 +- tools/get_pr_ut.py | 57 +- tools/parallel_UT_rule.py | 1 + tools/remove_grad_op_and_kernel.py | 177 +++ tools/static_mode_white_list.py | 5 + tools/timeout_debug_help.sh | 27 + tools/windows/run_unittests.sh | 125 +- tools/wlist.json | 4 +- 610 files changed, 19588 insertions(+), 5156 deletions(-) rename paddle/fluid/extension/include/{all.h => ext_all.h} (76%) rename paddle/fluid/extension/include/{dispatch.h => ext_dispatch.h} (76%) rename paddle/fluid/extension/include/{dll_decl.h => ext_dll_decl.h} (100%) rename paddle/fluid/extension/include/{dtype.h => ext_dtype.h} (85%) create mode 100644 paddle/fluid/extension/include/ext_exception.h create mode 100644 paddle/fluid/extension/include/ext_op_meta_info.h rename paddle/fluid/extension/include/{place.h => ext_place.h} (100%) rename paddle/fluid/extension/include/{tensor.h => ext_tensor.h} (79%) delete mode 100644 paddle/fluid/extension/include/op_meta_info.h rename paddle/fluid/extension/src/{op_meta_info.cc => ext_op_meta_info.cc} (71%) rename paddle/fluid/extension/src/{tensor.cc => ext_tensor.cc} (91%) create mode 100644 paddle/fluid/inference/api/details/zero_copy_tensor_test.cc create mode 100644 paddle/fluid/inference/api/paddle_tensor.h create mode 100644 paddle/fluid/inference/tensorrt/convert/gather_op.cc create mode 100644 paddle/fluid/inference/tensorrt/convert/nearest_interp_op.cc create mode 100644 paddle/fluid/inference/tensorrt/plugin/test_split_plugin.cc create mode 100644 paddle/fluid/operators/collective/c_wait_comm_op.cc create mode 100644 paddle/fluid/operators/collective/c_wait_compute_op.cc create mode 100644 paddle/fluid/operators/conv_miopen_helper.h create mode 100644 paddle/fluid/operators/math/blas_impl.hip.h create mode 100644 paddle/fluid/operators/miopen_lstm_cache.h create mode 100644 paddle/fluid/operators/miopen_rnn_cache.h create mode 100644 paddle/fluid/operators/optimizers/lamb_op_xpu.cc rename paddle/fluid/operators/{pool_op.cu.cc => pool_op.cu} (100%) create mode 100644 paddle/fluid/platform/eigen_ext.h create mode 100644 patches/cryptopp/CMakeLists.txt create mode 100644 python/paddle/fluid/contrib/mixed_precision/bf16/__init__.py create mode 100644 python/paddle/fluid/contrib/mixed_precision/bf16/amp_lists.py create mode 100644 python/paddle/fluid/contrib/mixed_precision/bf16/amp_utils.py create mode 100644 python/paddle/fluid/contrib/slim/quantization/imperative/utils.py create mode 100644 python/paddle/fluid/contrib/tests/test_bf16_utils.py create mode 100644 python/paddle/fluid/contrib/tests/test_model_cast_to_bf16.py create mode 100644 python/paddle/fluid/dataloader/collate.py create mode 100644 python/paddle/fluid/dataloader/flat.py create mode 100644 python/paddle/fluid/dataloader/worker.py create mode 100644 python/paddle/fluid/tests/custom_op/concat_and_split.h create mode 100644 python/paddle/fluid/tests/custom_op/custom_concat_op.cc create mode 100644 python/paddle/fluid/tests/custom_op/custom_conj_op.cc create mode 100644 python/paddle/fluid/tests/custom_op/test_check_error.cc create mode 100644 python/paddle/fluid/tests/custom_op/test_custom_concat.py create mode 100644 python/paddle/fluid/tests/custom_op/test_custom_conj.py create mode 100644 python/paddle/fluid/tests/custom_op/test_custom_relu_model.py create mode 100644 python/paddle/fluid/tests/unittests/collective_allreduce_op_wait.py create mode 100644 python/paddle/fluid/tests/unittests/ir/inference/test_trt_gather_op.py create mode 100644 python/paddle/fluid/tests/unittests/ir/inference/test_trt_nearest_interp_op.py create mode 100644 python/paddle/fluid/tests/unittests/ir/inference/test_trt_scale_op.py create mode 100644 python/paddle/fluid/tests/unittests/mkldnn/test_fusion_lstm_bf16_mkldnn_op.py create mode 100644 python/paddle/fluid/tests/unittests/rnn/test_rnn_cudnn_params_packing.py create mode 100644 python/paddle/fluid/tests/unittests/test_collective_wait.py create mode 100644 python/paddle/fluid/tests/unittests/test_lookup_table_bf16_op.py create mode 100644 python/paddle/fluid/tests/unittests/xpu/test_lamb_op_xpu.py create mode 100644 tools/remove_grad_op_and_kernel.py create mode 100644 tools/timeout_debug_help.sh diff --git a/CMakeLists.txt b/CMakeLists.txt index f24513d605c49..765d8fc157856 100755 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -61,6 +61,7 @@ if(WITH_MUSL) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-error=deprecated-declarations -Wno-deprecated-declarations -Wno-error=pessimizing-move -Wno-error=deprecated-copy") endif() + if(WIN32) option(MSVC_STATIC_CRT "use static C Runtime library by default" ON) @@ -72,6 +73,13 @@ if(WIN32) set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} /bigobj") set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /bigobj") + if("${CMAKE_GENERATOR}" STREQUAL "Ninja") + set(CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} /Zc:inline") + set(CMAKE_C_FLAGS_RELEASE "${CMAKE_C_FLAGS_RELEASE} /Zc:inline") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} /Zc:inline") + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /Zc:inline") + endif() + if (MSVC_STATIC_CRT) message(STATUS "Use static C runtime time, refer to https://docs.microsoft.com/en-us/cpp/c-runtime-library/crt-library-features?view=vs-2019") set(CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} /MTd") @@ -88,7 +96,7 @@ if(WIN32) endif() endforeach(flag_var) endif() - + # NOTE(Avin0323): Less parallel count result in faster compilation. math(EXPR PROCESS_MAX "${CPU_CORES} * 2 / 3") # windows build turn off warnings, use parallel compiling. @@ -116,6 +124,10 @@ if(WIN32) set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /wd4068 /wd4129 /wd4244 /wd4267 /wd4297 /wd4530 /wd4577 /wd4819 /wd4838") set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /wd4068 /wd4129 /wd4244 /wd4267 /wd4297 /wd4530 /wd4577 /wd4819 /wd4838") + foreach(flag_var CMAKE_SHARED_LINKER_FLAGS CMAKE_STATIC_LINKER_FLAGS CMAKE_EXE_LINKER_FLAGS CMAKE_LINKER_FLAGS) + set(${flag_var} "${${flag_var}} /ignore:4049 /ignore:4217 /ignore:4006 /ignore:4221") + endforeach(flag_var) + if (WITH_WIN_DUMP_DBG) set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} /Zi") set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /Zi") diff --git a/README.md b/README.md index afb915506394f..e8a7013d0b443 100644 --- a/README.md +++ b/README.md @@ -36,7 +36,7 @@ pip install paddlepaddle-gpu ``` More infomation about installation, please view [Quick Install](https://www.paddlepaddle.org.cn/install/quick) -Now our developers can acquire Tesla V100 online computing resources for free. If you create a program by AI Studio, you will obtain 12 hours to train models online per day. If you can insist on that for five consecutive days, then you will receive an extra 48 hours. [Click here to start](https://ai.baidu.com/support/news?action=detail&id=981). +Now our developers can acquire Tesla V100 online computing resources for free. If you create a program by AI Studio, you will obtain 10 hours to train models online per day. [Click here to start](https://aistudio.baidu.com/aistudio/index). ## FOUR LEADING TECHNOLOGIES diff --git a/README_cn.md b/README_cn.md index ec68e81f2ed57..7a10cba284549 100644 --- a/README_cn.md +++ b/README_cn.md @@ -32,7 +32,7 @@ pip install paddlepaddle-gpu ``` 更多安装信息详见官网 [安装说明](https://www.paddlepaddle.org.cn/install/quick) -PaddlePaddle用户可领取**免费Tesla V100在线算力资源**,训练模型更高效。**每日登陆即送12小时**,**连续五天运行再加送48小时**,[前往使用免费算力](https://ai.baidu.com/support/news?action=detail&id=981)。 +PaddlePaddle用户可领取**免费Tesla V100在线算力资源**,训练模型更高效。**每日登陆即送10小时**,[前往使用免费算力](https://aistudio.baidu.com/aistudio/index)。 ## 四大领先技术 diff --git a/cmake/cuda.cmake b/cmake/cuda.cmake index 2f4f5449f482d..c4d1384312e3c 100644 --- a/cmake/cuda.cmake +++ b/cmake/cuda.cmake @@ -91,7 +91,7 @@ function(select_nvcc_arch_flags out_variable) if(${CUDA_ARCH_NAME} STREQUAL "Manual") set(CUDA_ARCH_BIN ${paddle_known_gpu_archs} CACHE STRING "Specify 'real' GPU architectures to build binaries for, BIN(PTX) format is supported") - set(CUDA_ARCH_PTX "50" CACHE STRING "Specify 'virtual' PTX architectures to build PTX intermediate code for") + set(CUDA_ARCH_PTX "" CACHE STRING "Specify 'virtual' PTX architectures to build PTX intermediate code for") mark_as_advanced(CUDA_ARCH_BIN CUDA_ARCH_PTX) else() unset(CUDA_ARCH_BIN CACHE) @@ -175,14 +175,22 @@ elseif (${CMAKE_CUDA_COMPILER_VERSION} LESS 10.0) # CUDA 9.x set(paddle_known_gpu_archs ${paddle_known_gpu_archs9}) set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D_MWAITXINTRIN_H_INCLUDED") set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D__STRICT_ANSI__") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -Wno-deprecated-gpu-targets") elseif (${CMAKE_CUDA_COMPILER_VERSION} LESS 11.0) # CUDA 10.x set(paddle_known_gpu_archs ${paddle_known_gpu_archs10}) set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D_MWAITXINTRIN_H_INCLUDED") set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D__STRICT_ANSI__") -elseif (${CMAKE_CUDA_COMPILER_VERSION} LESS 12.0) # CUDA 11.x + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -Wno-deprecated-gpu-targets") +elseif (${CMAKE_CUDA_COMPILER_VERSION} LESS 11.2) # CUDA 11.0/11.1 set(paddle_known_gpu_archs ${paddle_known_gpu_archs11}) set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D_MWAITXINTRIN_H_INCLUDED") set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D__STRICT_ANSI__") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -Wno-deprecated-gpu-targets") +elseif (${CMAKE_CUDA_COMPILER_VERSION} LESS 12.0) # CUDA 11.2+ + set(paddle_known_gpu_archs "${paddle_known_gpu_archs11} 86") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D_MWAITXINTRIN_H_INCLUDED") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -D__STRICT_ANSI__") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -Wno-deprecated-gpu-targets") endif() if (NOT ${CMAKE_CUDA_COMPILER_VERSION} LESS 10.0) diff --git a/cmake/external/cryptopp.cmake b/cmake/external/cryptopp.cmake index 3176e2a665c63..a30164ada2791 100644 --- a/cmake/external/cryptopp.cmake +++ b/cmake/external/cryptopp.cmake @@ -22,6 +22,13 @@ SET(CRYPTOPP_TAG CRYPTOPP_8_2_0) IF(WIN32) SET(CRYPTOPP_LIBRARIES "${CRYPTOPP_INSTALL_DIR}/lib/cryptopp-static.lib" CACHE FILEPATH "cryptopp library." FORCE) + # There is a compilation parameter 'FI\"winapifamily.h\"' can't be used correctly + # with Ninja on Windows. The only difference between the patch file and original + # file is that the compilation parameters are changed to 'FIwinapifamily.h'. This + # patch command can be removed when upgrading to a higher version. + if("${CMAKE_GENERATOR}" STREQUAL "Ninja") + set(CRYPTOPP_PATCH_COMMAND ${CMAKE_COMMAND} -E copy_if_different "${PADDLE_SOURCE_DIR}/patches/cryptopp/CMakeLists.txt" "/") + endif() ELSE(WIN32) SET(CRYPTOPP_LIBRARIES "${CRYPTOPP_INSTALL_DIR}/lib/libcryptopp.a" CACHE FILEPATH "cryptopp library." FORCE) ENDIF(WIN32) @@ -53,11 +60,13 @@ ExternalProject_Add( "${CRYPTOPP_DOWNLOAD_CMD}" PREFIX ${CRYPTOPP_PREFIX_DIR} SOURCE_DIR ${CRYPTOPP_SOURCE_DIR} + UPDATE_COMMAND "" PATCH_COMMAND COMMAND ${CMAKE_COMMAND} -E remove_directory "/cmake/" COMMAND git clone ${GIT_URL}/noloader/cryptopp-cmake "/cmake" COMMAND cd "/cmake" && git checkout tags/${CRYPTOPP_TAG} -b ${CRYPTOPP_TAG} COMMAND ${CMAKE_COMMAND} -E copy_directory "/cmake/" "/" + COMMAND ${CRYPTOPP_PATCH_COMMAND} INSTALL_DIR ${CRYPTOPP_INSTALL_DIR} CMAKE_ARGS ${CRYPTOPP_CMAKE_ARGS} CMAKE_CACHE_ARGS -DCMAKE_INSTALL_PREFIX:PATH=${CRYPTOPP_INSTALL_DIR} diff --git a/cmake/external/mkldnn.cmake b/cmake/external/mkldnn.cmake index e41d8fdb6daac..884219d8dd81f 100644 --- a/cmake/external/mkldnn.cmake +++ b/cmake/external/mkldnn.cmake @@ -20,7 +20,7 @@ SET(MKLDNN_SOURCE_DIR ${THIRD_PARTY_PATH}/mkldnn/src/extern_mkldnn) SET(MKLDNN_INSTALL_DIR ${THIRD_PARTY_PATH}/install/mkldnn) SET(MKLDNN_INC_DIR "${MKLDNN_INSTALL_DIR}/include" CACHE PATH "mkldnn include directory." FORCE) SET(MKLDNN_REPOSITORY ${GIT_URL}/oneapi-src/oneDNN.git) -SET(MKLDNN_TAG 3d53cd3f17ce7ca365c980f0e1e50359751ca038) +SET(MKLDNN_TAG 72efa005effb49595933e033cc732f215ef0445a) # Introduce variables: # * CMAKE_INSTALL_LIBDIR diff --git a/cmake/external/protobuf.cmake b/cmake/external/protobuf.cmake index 905c17b9304ae..40a27f506f307 100644 --- a/cmake/external/protobuf.cmake +++ b/cmake/external/protobuf.cmake @@ -250,5 +250,8 @@ IF(NOT PROTOBUF_FOUND) SET(PROTOBUF_PROTOC_EXECUTABLE ${extern_protobuf_PROTOC_EXECUTABLE} CACHE FILEPATH "protobuf executable." FORCE) + # `EXTERN_PROTOBUF_DEPEND` used in cmake function `proto_library` to ensure + # `protoc.exe` existed before calling it. + set(EXTERN_PROTOBUF_DEPEND extern_protobuf) PROMPT_PROTOBUF_LIB(extern_protobuf) ENDIF(NOT PROTOBUF_FOUND) diff --git a/cmake/external/pslib_brpc.cmake b/cmake/external/pslib_brpc.cmake index 7b00474a65070..e5a85a47a317b 100644 --- a/cmake/external/pslib_brpc.cmake +++ b/cmake/external/pslib_brpc.cmake @@ -41,7 +41,8 @@ FILE(WRITE ${PSLIB_BRPC_DOWNLOAD_DIR}/CMakeLists.txt "cmake_minimum_required(VERSION 3.0)\n" "install(DIRECTORY ${PSLIB_BRPC_NAME}/include ${PSLIB_BRPC_NAME}/lib \n" " DESTINATION ${PSLIB_BRPC_DST_DIR})\n") - +MESSAGE("DOWNLAODING PATH FOR BRPC ---------------",${PSLIB_BRPC_DOWNLOAD_DIR}) +MESSAGE("PREFIX----------",${PSLIB_BRPC_SOURCE_DIR}) ExternalProject_Add( ${PSLIB_BRPC_PROJECT} ${EXTERNAL_PROJECT_LOG_ARGS} diff --git a/cmake/external/warpctc.cmake b/cmake/external/warpctc.cmake index 0ee3e2116a94b..b0ef575f64323 100644 --- a/cmake/external/warpctc.cmake +++ b/cmake/external/warpctc.cmake @@ -18,7 +18,7 @@ SET(WARPCTC_PREFIX_DIR ${THIRD_PARTY_PATH}/warpctc) SET(WARPCTC_SOURCE_DIR ${THIRD_PARTY_PATH}/warpctc/src/extern_warpctc) SET(WARPCTC_INSTALL_DIR ${THIRD_PARTY_PATH}/install/warpctc) set(WARPCTC_REPOSITORY ${GIT_URL}/baidu-research/warp-ctc.git) -set(WARPCTC_TAG 95a461eddeabd51099ef059dcfada1117eb1bfb8) +set(WARPCTC_TAG cd828e5b6c3b953b82af73f7f44cddc393a20efa) SET(WARPCTC_INCLUDE_DIR "${WARPCTC_INSTALL_DIR}/include" CACHE PATH "Warp-ctc Directory" FORCE) @@ -49,12 +49,12 @@ ExternalProject_Add( BUILD_ALWAYS 1 CMAKE_ARGS -DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER} -DCMAKE_C_COMPILER=${CMAKE_C_COMPILER} - -DCMAKE_C_FLAGS=${CMAKE_C_FLAGS} - -DCMAKE_C_FLAGS_DEBUG=${CMAKE_C_FLAGS_DEBUG} - -DCMAKE_C_FLAGS_RELEASE=${CMAKE_C_FLAGS_RELEASE} - -DCMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS} - -DCMAKE_CXX_FLAGS_RELEASE=${CMAKE_CXX_FLAGS_RELEASE} - -DCMAKE_CXX_FLAGS_DEBUG=${CMAKE_CXX_FLAGS_DEBUG} + -DCMAKE_C_FLAGS=$ + -DCMAKE_C_FLAGS_DEBUG=$ + -DCMAKE_C_FLAGS_RELEASE=$ + -DCMAKE_CXX_FLAGS=$ + -DCMAKE_CXX_FLAGS_RELEASE=$ + -DCMAKE_CXX_FLAGS_DEBUG=$ -DCMAKE_INSTALL_PREFIX=${WARPCTC_INSTALL_DIR} -DWITH_GPU=${WITH_GPU} -DWITH_OMP=${USE_OMP} diff --git a/cmake/external/xpu.cmake b/cmake/external/xpu.cmake index 41b2907bbae4d..b5a3f0154745b 100644 --- a/cmake/external/xpu.cmake +++ b/cmake/external/xpu.cmake @@ -4,6 +4,8 @@ endif() INCLUDE(ExternalProject) SET(XPU_PROJECT "extern_xpu") +SET(XPU_API_LIB_NAME "libxpuapi.so") +SET(XPU_RT_LIB_NAME "libxpurt.so") if(NOT XPU_SDK_ROOT) if (WITH_AARCH64) @@ -11,7 +13,7 @@ if(NOT XPU_SDK_ROOT) elseif(WITH_SUNWAY) SET(XPU_URL "https://baidu-kunlun-public.su.bcebos.com/paddle_depence/sunway/xpu_2021_01_13.tar.gz" CACHE STRING "" FORCE) else() - SET(XPU_URL "https://baidu-kunlun-public.su.bcebos.com/paddle_depence/xpu_2021_02_19.tar.gz" CACHE STRING "" FORCE) + SET(XPU_URL "https://baidu-kunlun-public.su.bcebos.com/paddle_depence/xpu_2021_02_27.tar.gz" CACHE STRING "" FORCE) endif() SET(XPU_SOURCE_DIR "${THIRD_PARTY_PATH}/xpu") @@ -20,8 +22,6 @@ if(NOT XPU_SDK_ROOT) SET(XPU_API_INC_DIR "${THIRD_PARTY_PATH}/install/xpu/include") SET(XPU_LIB_DIR "${THIRD_PARTY_PATH}/install/xpu/lib") - SET(XPU_API_LIB_NAME "libxpuapi.so") - SET(XPU_RT_LIB_NAME "libxpurt.so") SET(XPU_API_LIB "${XPU_LIB_DIR}/${XPU_API_LIB_NAME}") SET(XPU_RT_LIB "${XPU_LIB_DIR}/${XPU_RT_LIB_NAME}") @@ -49,6 +49,7 @@ else() SET(XPU_API_INC_DIR "${XPU_SDK_ROOT}/XTDK/include/") SET(XPU_API_LIB "${XPU_SDK_ROOT}/XTDK/shlib/libxpuapi.so") SET(XPU_RT_LIB "${XPU_SDK_ROOT}/XTDK/runtime/shlib/libxpurt.so") + SET(XPU_LIB_DIR "${XPU_SDK_ROOT}/XTDK/shlib/") endif() INCLUDE_DIRECTORIES(${XPU_API_INC_DIR}) diff --git a/cmake/generic.cmake b/cmake/generic.cmake index cba338c2c49f6..c85654a5674a0 100644 --- a/cmake/generic.cmake +++ b/cmake/generic.cmake @@ -260,8 +260,8 @@ function(merge_static_libs TARGET_NAME) # msvc will put libarary in directory of "/Release/xxxlib" by default # COMMAND cmake -E remove "${CMAKE_CURRENT_BINARY_DIR}/${CMAKE_BUILD_TYPE}/${TARGET_NAME}.lib" add_custom_command(TARGET ${TARGET_NAME} POST_BUILD - COMMAND cmake -E make_directory "${CMAKE_CURRENT_BINARY_DIR}/${CMAKE_BUILD_TYPE}" - COMMAND lib /OUT:${CMAKE_CURRENT_BINARY_DIR}/${CMAKE_BUILD_TYPE}/lib${TARGET_NAME}.lib ${libfiles} + COMMAND cmake -E make_directory $ + COMMAND lib /OUT:$ ${libfiles} ) endif(WIN32) endfunction(merge_static_libs) @@ -492,7 +492,7 @@ function(nv_library TARGET_NAME) message(FATAL "Please specify source file or library in nv_library.") endif() endif(nv_library_SRCS) - if (WIN32 AND ${CMAKE_CUDA_COMPILER_VERSION} LESS 11.0) + if((CUDA_VERSION GREATER 9.2) AND (CUDA_VERSION LESS 11.0) AND (MSVC_VERSION LESS 1910)) set_target_properties(${TARGET_NAME} PROPERTIES VS_USER_PROPS ${WIN_PROPS}) endif() endif() @@ -510,7 +510,7 @@ function(nv_binary TARGET_NAME) add_dependencies(${TARGET_NAME} ${nv_binary_DEPS}) common_link(${TARGET_NAME}) endif() - if (WIN32 AND ${CMAKE_CUDA_COMPILER_VERSION} LESS 11.0) + if((CUDA_VERSION GREATER 9.2) AND (CUDA_VERSION LESS 11.0) AND (MSVC_VERSION LESS 1910)) set_target_properties(${TARGET_NAME} PROPERTIES VS_USER_PROPS ${WIN_PROPS}) endif() endif() @@ -537,7 +537,7 @@ function(nv_test TARGET_NAME) set_property(TEST ${TARGET_NAME} PROPERTY ENVIRONMENT FLAGS_cpu_deterministic=true) set_property(TEST ${TARGET_NAME} PROPERTY ENVIRONMENT FLAGS_init_allocated_mem=true) set_property(TEST ${TARGET_NAME} PROPERTY ENVIRONMENT FLAGS_cudnn_deterministic=true) - if (WIN32 AND ${CMAKE_CUDA_COMPILER_VERSION} LESS 11.0) + if((CUDA_VERSION GREATER 9.2) AND (CUDA_VERSION LESS 11.0) AND (MSVC_VERSION LESS 1910)) set_target_properties(${TARGET_NAME} PROPERTIES VS_USER_PROPS ${WIN_PROPS}) endif() endif() @@ -753,7 +753,8 @@ function(paddle_protobuf_generate_cpp SRCS HDRS) COMMAND ${PROTOBUF_PROTOC_EXECUTABLE} -I${CMAKE_CURRENT_SOURCE_DIR} --cpp_out "${CMAKE_CURRENT_BINARY_DIR}" ${ABS_FIL} - DEPENDS ${ABS_FIL} protoc + # Set `EXTERN_PROTOBUF_DEPEND` only if need to compile `protoc.exe`. + DEPENDS ${ABS_FIL} ${EXTERN_PROTOBUF_DEPEND} COMMENT "Running C++ protocol buffer compiler on ${FIL}" VERBATIM ) endforeach() @@ -794,7 +795,8 @@ function(py_test TARGET_NAME) if(WITH_COVERAGE AND NOT (WITH_INCREMENTAL_COVERAGE AND "$ENV{PADDLE_GIT_DIFF_PY_FILE}" STREQUAL "")) add_test(NAME ${TARGET_NAME} COMMAND ${CMAKE_COMMAND} -E env FLAGS_init_allocated_mem=true FLAGS_cudnn_deterministic=true - FLAGS_cpu_deterministic=true ${py_test_ENVS} + FLAGS_cpu_deterministic=true + PYTHONPATH=${PADDLE_BINARY_DIR}/python ${py_test_ENVS} COVERAGE_FILE=${PADDLE_BINARY_DIR}/python-coverage.data ${PYTHON_EXECUTABLE} -m coverage run --branch -p ${py_test_SRCS} ${py_test_ARGS} WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/cmake/hip.cmake b/cmake/hip.cmake index 523540c9794c1..4c492d7cc48f0 100644 --- a/cmake/hip.cmake +++ b/cmake/hip.cmake @@ -45,6 +45,7 @@ set(THRUST_DEVICE_SYSTEM THRUST_DEVICE_SYSTEM_HIP) # define HIP_CXX_FLAGS list(APPEND HIP_CXX_FLAGS -fPIC) list(APPEND HIP_CXX_FLAGS -D__HIP_PLATFORM_HCC__=1) +# Note(qili93): HIP has compile conflicts of float16.h as platform::float16 overload std::is_floating_point and std::is_integer list(APPEND HIP_CXX_FLAGS -D__HIP_NO_HALF_CONVERSIONS__=1) list(APPEND HIP_CXX_FLAGS -Wno-macro-redefined) list(APPEND HIP_CXX_FLAGS -Wno-inconsistent-missing-override) diff --git a/cmake/inference_lib.cmake b/cmake/inference_lib.cmake index 059c3a04487cc..4864e04fa0516 100644 --- a/cmake/inference_lib.cmake +++ b/cmake/inference_lib.cmake @@ -164,11 +164,11 @@ copy_part_of_thrid_party(inference_lib_dist ${PADDLE_INFERENCE_INSTALL_DIR}) set(src_dir "${PADDLE_SOURCE_DIR}/paddle/fluid") if(WIN32) if(WITH_STATIC_LIB) - set(paddle_inference_lib ${PADDLE_BINARY_DIR}/paddle/fluid/inference/${CMAKE_BUILD_TYPE}/libpaddle_inference.lib - ${PADDLE_BINARY_DIR}/paddle/fluid/inference/${CMAKE_BUILD_TYPE}/paddle_inference.*) + set(paddle_inference_lib $/libpaddle_inference.lib + $/paddle_inference.*) else() - set(paddle_inference_lib ${PADDLE_BINARY_DIR}/paddle/fluid/inference/${CMAKE_BUILD_TYPE}/paddle_inference.dll - ${PADDLE_BINARY_DIR}/paddle/fluid/inference/${CMAKE_BUILD_TYPE}/paddle_inference.lib) + set(paddle_inference_lib $/paddle_inference.dll + $/paddle_inference.lib) endif() copy(inference_lib_dist SRCS ${src_dir}/inference/api/paddle_*.h ${paddle_inference_lib} @@ -189,6 +189,19 @@ copy(inference_lib_dist DSTS ${PADDLE_INFERENCE_INSTALL_DIR}/paddle/include/crypto/) include_directories(${CMAKE_BINARY_DIR}/../paddle/fluid/framework/io) +copy(inference_lib_dist + SRCS ${PADDLE_SOURCE_DIR}/paddle/fluid/extension/include/* + DSTS ${PADDLE_INFERENCE_INSTALL_DIR}/paddle/include/experimental/) +copy(inference_lib_dist + SRCS ${PADDLE_SOURCE_DIR}/paddle/fluid/platform/complex64.h + DSTS ${PADDLE_INFERENCE_INSTALL_DIR}/paddle/include/experimental/) +copy(inference_lib_dist + SRCS ${PADDLE_SOURCE_DIR}/paddle/fluid/platform/complex128.h + DSTS ${PADDLE_INFERENCE_INSTALL_DIR}/paddle/include/experimental/) +copy(inference_lib_dist + SRCS ${PADDLE_SOURCE_DIR}/paddle/fluid/platform/float16.h + DSTS ${PADDLE_INFERENCE_INSTALL_DIR}/paddle/include/experimental/) + # CAPI inference library for only inference set(PADDLE_INFERENCE_C_INSTALL_DIR "${CMAKE_BINARY_DIR}/paddle_inference_c_install_dir" CACHE STRING "A path setting CAPI paddle inference shared") @@ -196,7 +209,7 @@ copy_part_of_thrid_party(inference_lib_dist ${PADDLE_INFERENCE_C_INSTALL_DIR}) set(src_dir "${PADDLE_SOURCE_DIR}/paddle/fluid") if(WIN32) - set(paddle_inference_c_lib ${PADDLE_BINARY_DIR}/paddle/fluid/inference/capi/${CMAKE_BUILD_TYPE}/paddle_inference_c.*) + set(paddle_inference_c_lib $/paddle_inference_c.*) else(WIN32) set(paddle_inference_c_lib ${PADDLE_BINARY_DIR}/paddle/fluid/inference/capi/libpaddle_inference_c.*) endif(WIN32) diff --git a/cmake/init.cmake b/cmake/init.cmake index aea02088750df..19fdb6c601a11 100644 --- a/cmake/init.cmake +++ b/cmake/init.cmake @@ -18,6 +18,10 @@ if(NOT WIN32) set(CMAKE_CXX_FLAGS_RELWITHDEBINFO "-O2 -g -DNDEBUG") set(CMAKE_CXX_FLAGS_MINSIZEREL "-Os -DNDEBUG") else() + # It has not been used now, it can specify CUDA compile flag manualy, + # its use is to remvoe /Zi to reduce GPU static library size. But it's dangerous + # because CUDA will update by nvidia, then error will occur. + # Now, it's used in CUDA:[10.0, 10.2] set(WIN_PROPS ${CMAKE_SOURCE_DIR}/cmake/paddle_win.props) endif() diff --git a/cmake/paddle_win.props b/cmake/paddle_win.props index 0115ad4b59fc4..296940dc3f50c 100644 --- a/cmake/paddle_win.props +++ b/cmake/paddle_win.props @@ -15,7 +15,7 @@ InheritFromHost -ccbin "%(VCBinDir)" -x cu [GenerateRelocatableDeviceCode] [Include] [RequiredIncludes] [InterleaveSourceInPTX] [GPUDebugInfo] [GenerateLineInfo] [Keep] [KeepDir] [MaxRegCount] [PtxAsOptionV] [TargetMachinePlatform] [NvccCompilation] [CudaRuntime] [AdditionalOptions] - --use-local-env --cl-version $(CudaClVersion) + --use-local-env $(CudaClVersion) [CodeGeneration] -clean diff --git a/cmake/third_party.cmake b/cmake/third_party.cmake index d576a299b866c..6488d29afc5f7 100644 --- a/cmake/third_party.cmake +++ b/cmake/third_party.cmake @@ -222,7 +222,7 @@ if(WITH_MKLDNN) endif() include(external/protobuf) # find first, then download, build, install protobuf -if(NOT PROTOBUF_FOUND OR WIN32) +if(TARGET extern_protobuf) list(APPEND third_party_deps extern_protobuf) endif() @@ -311,11 +311,14 @@ if(WITH_DGC) endif() if (WITH_LITE) + message(STATUS "Compile Paddle with Lite Engine.") include(external/lite) endif (WITH_LITE) if (WITH_CRYPTO) include(external/cryptopp) # download, build, install cryptopp + list(APPEND third_party_deps extern_cryptopp) + add_definitions(-DPADDLE_WITH_CRYPTO) endif (WITH_CRYPTO) add_custom_target(third_party ALL DEPENDS ${third_party_deps}) diff --git a/paddle/extension.h b/paddle/extension.h index 1c64b92c5a374..71469576853a3 100644 --- a/paddle/extension.h +++ b/paddle/extension.h @@ -15,4 +15,4 @@ limitations under the License. */ #pragma once // All paddle apis in C++ frontend -#include "paddle/fluid/extension/include/all.h" +#include "paddle/fluid/extension/include/ext_all.h" diff --git a/paddle/fluid/distributed/fleet.cc b/paddle/fluid/distributed/fleet.cc index f4fdf4880bcf5..b638af49730dd 100644 --- a/paddle/fluid/distributed/fleet.cc +++ b/paddle/fluid/distributed/fleet.cc @@ -501,7 +501,7 @@ void FleetWrapper::ShrinkDenseTable(int table_id, Scope* scope, if (name.find("batch_sum") != std::string::npos) { Variable* var = scope->FindVar(name); CHECK(var != nullptr) << "var[" << name << "] not found"; - VLOG(0) << "prepare shrink dense batch_sum"; + VLOG(3) << "prepare shrink dense batch_sum"; LoDTensor* tensor = var->GetMutable(); float* g = tensor->data(); diff --git a/paddle/fluid/distributed/service/brpc_ps_server.cc b/paddle/fluid/distributed/service/brpc_ps_server.cc index 32de11847387b..8400e669182d6 100644 --- a/paddle/fluid/distributed/service/brpc_ps_server.cc +++ b/paddle/fluid/distributed/service/brpc_ps_server.cc @@ -79,16 +79,13 @@ uint64_t BrpcPsServer::start(const std::string &ip, uint32_t port) { } } - VLOG(0) << "BrpcPsServer::start registe_ps_server"; _environment->registe_ps_server(ip, port, _rank); - VLOG(0) << "BrpcPsServer::start wait"; cv_.wait(lock, [&] { return stoped_; }); PSHost host; host.ip = ip; host.port = port; host.rank = _rank; - VLOG(0) << "BrpcPsServer::start return host.rank"; return host.rank; } @@ -464,7 +461,7 @@ int32_t BrpcPsService::save_one_table(Table *table, int32_t feasign_size = 0; - VLOG(0) << "save one table " << request.params(0) << " " << request.params(1); + VLOG(3) << "save table " << request.params(0) << " " << request.params(1); feasign_size = table->save(request.params(0), request.params(1)); if (feasign_size < 0) { set_response_code(response, -1, "table save failed"); @@ -507,7 +504,7 @@ int32_t BrpcPsService::shrink_table(Table *table, set_response_code(response, -1, "table shrink failed"); return -1; } - VLOG(0) << "Pserver Shrink Finished"; + VLOG(3) << "Pserver Shrink Finished"; return 0; } diff --git a/paddle/fluid/distributed/service/communicator.cc b/paddle/fluid/distributed/service/communicator.cc index aea758a717b2d..8699719e5cdcc 100644 --- a/paddle/fluid/distributed/service/communicator.cc +++ b/paddle/fluid/distributed/service/communicator.cc @@ -39,7 +39,7 @@ inline double GetCurrentUS() { Communicator::Communicator() {} void Communicator::init_gflag(const std::string &gflags) { - VLOG(0) << "Init With Gflags:" << gflags; + VLOG(3) << "Init With Gflags:" << gflags; std::vector flags = paddle::string::split_string(gflags); if (flags.size() < 1) { flags.push_back("-max_body_size=314217728"); diff --git a/paddle/fluid/distributed/service/communicator.h b/paddle/fluid/distributed/service/communicator.h index fd53e0e4f4a48..043fe9d83dfc5 100644 --- a/paddle/fluid/distributed/service/communicator.h +++ b/paddle/fluid/distributed/service/communicator.h @@ -199,10 +199,10 @@ class Communicator { Communicator(); explicit Communicator(const std::map &envs_) { - VLOG(0) << "Communicator Init Envs"; + VLOG(3) << "Communicator Init Envs"; for (auto &iter : envs_) { envs[iter.first] = iter.second; - VLOG(0) << iter.first << ": " << iter.second; + VLOG(3) << iter.first << ": " << iter.second; } barrier_table_id_ = std::stoi(envs.at("barrier_table_id")); trainer_id_ = std::stoi(envs.at("trainer_id")); @@ -436,7 +436,7 @@ class HalfAsyncCommunicator : public AsyncCommunicator { need_global_step_ = static_cast(std::stoi(envs.at("need_global_step"))); - VLOG(0) << "HalfAsyncCommunicator Initialized"; + VLOG(1) << "HalfAsyncCommunicator Initialized"; } void MainThread() override; @@ -481,7 +481,7 @@ class SyncCommunicator : public HalfAsyncCommunicator { need_global_step_ = static_cast(std::stoi(envs.at("need_global_step"))); - VLOG(0) << "SyncCommunicator Initialized"; + VLOG(1) << "SyncCommunicator Initialized"; } void BarrierSend(); @@ -525,7 +525,7 @@ class GeoCommunicator : public AsyncCommunicator { // id_queue's size max_merge_var_num_ = std::stoi(envs.at("communicator_max_merge_var_num")); send_queue_size_ = max_merge_var_num_; - VLOG(0) << "GeoCommunicator Initialized"; + VLOG(1) << "GeoCommunicator Initialized"; } void Send(const std::vector &var_names, diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 0b079b2170e43..14bf6a03763d8 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -52,6 +52,98 @@ int GraphBrpcClient::get_server_index_by_id(uint64_t id) { : shard_num / server_size + 1; return id % shard_num / shard_per_server; } + +std::future GraphBrpcClient::get_node_feat( + const uint32_t &table_id, const std::vector &node_ids, + const std::vector &feature_names, + std::vector> &res) { + std::vector request2server; + std::vector server2request(server_size, -1); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_ids[query_idx]); + if (server2request[server_index] == -1) { + server2request[server_index] = request2server.size(); + request2server.push_back(server_index); + } + } + size_t request_call_num = request2server.size(); + std::vector> node_id_buckets(request_call_num); + std::vector> query_idx_buckets(request_call_num); + for (int query_idx = 0; query_idx < node_ids.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_ids[query_idx]); + int request_idx = server2request[server_index]; + node_id_buckets[request_idx].push_back(node_ids[query_idx]); + query_idx_buckets[request_idx].push_back(query_idx); + } + + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + request_call_num, + [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + int fail_num = 0; + for (int request_idx = 0; request_idx < request_call_num; + ++request_idx) { + if (closure->check_response(request_idx, + PS_GRAPH_SAMPLE_NEIGHBOORS) != 0) { + ++fail_num; + } else { + auto &res_io_buffer = + closure->cntl(request_idx)->response_attachment(); + butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); + size_t bytes_size = io_buffer_itr.bytes_left(); + std::unique_ptr buffer_wrapper(new char[bytes_size]); + char *buffer = buffer_wrapper.get(); + io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); + + for (size_t feat_idx = 0; feat_idx < feature_names.size(); + ++feat_idx) { + for (size_t node_idx = 0; + node_idx < query_idx_buckets.at(request_idx).size(); + ++node_idx) { + int query_idx = query_idx_buckets.at(request_idx).at(node_idx); + size_t feat_len = *(size_t *)(buffer); + buffer += sizeof(size_t); + auto feature = std::string(buffer, feat_len); + res[feat_idx][query_idx] = feature; + buffer += feat_len; + } + } + } + if (fail_num == request_call_num) { + ret = -1; + } + } + closure->set_promise_value(ret); + }); + + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + + for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { + int server_index = request2server[request_idx]; + closure->request(request_idx)->set_cmd_id(PS_GRAPH_GET_NODE_FEAT); + closure->request(request_idx)->set_table_id(table_id); + closure->request(request_idx)->set_client_id(_client_id); + size_t node_num = node_id_buckets[request_idx].size(); + + closure->request(request_idx) + ->add_params((char *)node_id_buckets[request_idx].data(), + sizeof(uint64_t) * node_num); + std::string joint_feature_name = + paddle::string::join_strings(feature_names, '\t'); + closure->request(request_idx) + ->add_params(joint_feature_name.c_str(), joint_feature_name.size()); + + PsService_Stub rpc_stub(get_cmd_channel(server_index)); + closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), + closure->response(request_idx), closure); + } + + return fut; +} // char* &buffer,int &actual_size std::future GraphBrpcClient::batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 55183fb402d79..60bbe42d32ccd 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -71,6 +71,10 @@ class GraphBrpcClient : public BrpcPsClient { int server_index, int sample_size, std::vector& ids); + virtual std::future get_node_feat( + const uint32_t& table_id, const std::vector& node_ids, + const std::vector& feature_names, + std::vector>& res); virtual int32_t initialize(); int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index fd6032413a9b1..9585134d8637e 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -100,6 +100,8 @@ int32_t GraphBrpcService::initialize() { &GraphBrpcService::graph_random_sample_neighboors; _service_handler_map[PS_GRAPH_SAMPLE_NODES] = &GraphBrpcService::graph_random_sample_nodes; + _service_handler_map[PS_GRAPH_GET_NODE_FEAT] = + &GraphBrpcService::graph_get_node_feat; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); @@ -314,5 +316,40 @@ int32_t GraphBrpcService::graph_random_sample_nodes( return 0; } + +int32_t GraphBrpcService::graph_get_node_feat(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 2) { + set_response_code( + response, -1, + "graph_get_node_feat request requires at least 2 arguments"); + return 0; + } + size_t node_num = request.params(0).size() / sizeof(uint64_t); + uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); + std::vector node_ids(node_data, node_data + node_num); + + std::vector feature_names = + paddle::string::split_string(request.params(1), "\t"); + + std::vector> feature( + feature_names.size(), std::vector(node_num)); + + table->get_node_feat(node_ids, feature_names, feature); + + for (size_t feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { + for (size_t node_idx = 0; node_idx < node_num; ++node_idx) { + size_t feat_len = feature[feat_idx][node_idx].size(); + cntl->response_attachment().append(&feat_len, sizeof(size_t)); + cntl->response_attachment().append(feature[feat_idx][node_idx].data(), + feat_len); + } + } + + return 0; +} } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index e2b57dff4d859..0badfd4313eb0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -87,6 +87,9 @@ class GraphBrpcService : public PsBaseService { const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); + int32_t graph_get_node_feat(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 2b70ae8298ef4..f7295efe6560a 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -289,6 +289,22 @@ std::vector GraphPyClient::random_sample_nodes(std::string name, } return v; } + +// (name, dtype, ndarray) +std::vector> GraphPyClient::get_node_feat( + std::string node_type, std::vector node_ids, + std::vector feature_names) { + std::vector> v( + feature_names.size(), std::vector(node_ids.size())); + if (this->table_id_map.count(node_type)) { + uint32_t table_id = this->table_id_map[node_type]; + auto status = + worker_ptr->get_node_feat(table_id, node_ids, feature_names, v); + status.wait(); + } + return v; +} + std::vector GraphPyClient::pull_graph_list(std::string name, int server_index, int start, int size, diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 18ebc57db4e6e..375f8f4560ea8 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -163,6 +163,9 @@ class GraphPyClient : public GraphPyService { std::string name, std::vector node_ids, int sample_size); std::vector random_sample_nodes(std::string name, int server_index, int sample_size); + std::vector> get_node_feat( + std::string node_type, std::vector node_ids, + std::vector feature_names); std::vector pull_graph_list(std::string name, int server_index, int start, int size, int step = 1); ::paddle::distributed::PSParameter GetWorkerProto(); diff --git a/paddle/fluid/distributed/service/heter_client.cc b/paddle/fluid/distributed/service/heter_client.cc index b83549714952f..10fc8368a26a9 100644 --- a/paddle/fluid/distributed/service/heter_client.cc +++ b/paddle/fluid/distributed/service/heter_client.cc @@ -34,7 +34,7 @@ void HeterClient::MainThread() { void HeterClient::Stop() { running_ = false; if (!is_initialized_) { - VLOG(0) << "HeterClient is not inited, do nothing"; + VLOG(3) << "HeterClient is not inited, do nothing"; } else { if (main_thread_) { auto status = StopHeterWorker(); @@ -42,20 +42,20 @@ void HeterClient::Stop() { main_thread_->join(); main_thread_.reset(nullptr); } - VLOG(1) << "HeterClient Stop Done"; + VLOG(3) << "HeterClient Stop Done"; } } void HeterClient::FinalizeWorker() { running_ = false; if (!is_initialized_) { - VLOG(0) << "HeterClient is not inited, do nothing"; + VLOG(3) << "HeterClient is not inited, do nothing"; } else { if (main_thread_) { main_thread_->join(); main_thread_.reset(nullptr); } - VLOG(1) << "HeterClient Stop Done"; + VLOG(3) << "HeterClient Stop Done"; } } diff --git a/paddle/fluid/distributed/service/heter_server.cc b/paddle/fluid/distributed/service/heter_server.cc index 7e0ac8ecf3516..57a1a16a72383 100644 --- a/paddle/fluid/distributed/service/heter_server.cc +++ b/paddle/fluid/distributed/service/heter_server.cc @@ -89,7 +89,7 @@ int32_t HeterService::stop_heter_worker(const PsRequestMessage& request, stop_cpu_worker_set_.insert(client_id); if (stop_cpu_worker_set_.size() == fan_in_) { is_exit_ = true; - VLOG(0) << "Stop heter Service done."; + VLOG(3) << "Stop heter Service done."; } return 0; } diff --git a/paddle/fluid/distributed/service/heter_server.h b/paddle/fluid/distributed/service/heter_server.h index 5d967ae06d802..93fa37454a574 100644 --- a/paddle/fluid/distributed/service/heter_server.h +++ b/paddle/fluid/distributed/service/heter_server.h @@ -153,7 +153,7 @@ class HeterServer { virtual ~HeterServer() {} void Stop() { - VLOG(0) << "HeterServer Stop()"; + VLOG(3) << "HeterServer Stop()"; std::unique_lock lock(mutex_); stoped_ = true; cv_.notify_all(); diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 1b50a0c37098d..9fd0bedf23ee9 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -185,6 +185,16 @@ class PSClient { promise.set_value(-1); return fut; } + virtual std::future get_node_feat( + const uint32_t &table_id, const std::vector &node_ids, + const std::vector &feature_names, + std::vector> &res) { + LOG(FATAL) << "Did not implement"; + std::promise promise; + std::future fut = promise.get_future(); + promise.set_value(-1); + return fut; + } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index 185024fe18721..d908c26da9870 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -51,6 +51,7 @@ enum PsCmdID { PS_PULL_GRAPH_LIST = 30; PS_GRAPH_SAMPLE_NEIGHBOORS = 31; PS_GRAPH_SAMPLE_NODES = 32; + PS_GRAPH_GET_NODE_FEAT = 33; } message PsRequestMessage { @@ -114,4 +115,4 @@ message MultiVariableMessage { service PsService { rpc service(PsRequestMessage) returns (PsResponseMessage); rpc SendAndRecvVariable(MultiVariableMessage) returns (MultiVariableMessage); -}; \ No newline at end of file +}; diff --git a/paddle/fluid/distributed/table/common_dense_table.cc b/paddle/fluid/distributed/table/common_dense_table.cc index 4063e4f501d01..8d8b43b37403a 100644 --- a/paddle/fluid/distributed/table/common_dense_table.cc +++ b/paddle/fluid/distributed/table/common_dense_table.cc @@ -94,7 +94,7 @@ int32_t CommonDenseTable::initialize_optimizer() { } else { VLOG(0) << "init optimizer failed"; } - VLOG(0) << "init optimizer " << name << " done"; + VLOG(3) << "init optimizer " << name << " done"; return 0; } @@ -120,6 +120,7 @@ int32_t CommonDenseTable::push_dense_param(const float* values, size_t num) { } int32_t CommonDenseTable::pour() { + pull_reservoir_.avg(); _push_dense(pull_reservoir_.values.data(), pull_reservoir_.values.size()); pull_reservoir_.reset(); return 0; diff --git a/paddle/fluid/distributed/table/common_dense_table.h b/paddle/fluid/distributed/table/common_dense_table.h index e363afc45c54c..74366f0358890 100644 --- a/paddle/fluid/distributed/table/common_dense_table.h +++ b/paddle/fluid/distributed/table/common_dense_table.h @@ -47,15 +47,12 @@ class CommonDenseTable : public DenseTable { virtual int32_t set_global_lr(float* lr) override; int32_t load(const std::string& path, const std::string& param) override { - VLOG(0) << "Dense table may load by " - "paddle.distributed.fleet.init_server"; + VLOG(0) << "WARNING: dense variables will load on No.0 trainer"; return 0; } int32_t save(const std::string& path, const std::string& param) override { - VLOG(0) - << "Dense table may be saved by " - "paddle.distributed.fleet.save_persistables/save_inference_model"; + VLOG(0) << "WARNING: dense variables will save on No.0 trainer"; return 0; } diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index c1dd26b2db869..1540083b6934a 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -12,15 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include "paddle/fluid/distributed/table/common_graph_table.h" #include #include #include #include #include "paddle/fluid/distributed/common/utils.h" +#include "paddle/fluid/distributed/table/graph_node.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" -#include "paddle/fluid/distributed/table/common_graph_table.h" -#include "paddle/fluid/distributed/table/graph_node.h" namespace paddle { namespace distributed { @@ -36,19 +36,19 @@ std::vector GraphShard::get_batch(int start, int end, int step) { size_t GraphShard::get_size() { return bucket.size(); } GraphNode *GraphShard::add_graph_node(uint64_t id) { - if (node_location.find(id) == node_location.end()){ + if (node_location.find(id) == node_location.end()) { node_location[id] = bucket.size(); bucket.push_back(new GraphNode(id)); } - return (GraphNode*)bucket[node_location[id]]; + return (GraphNode *)bucket[node_location[id]]; } FeatureNode *GraphShard::add_feature_node(uint64_t id) { - if (node_location.find(id) == node_location.end()){ + if (node_location.find(id) == node_location.end()) { node_location[id] = bucket.size(); bucket.push_back(new FeatureNode(id)); } - return (FeatureNode*)bucket[node_location[id]]; + return (FeatureNode *)bucket[node_location[id]]; } void GraphShard::add_neighboor(uint64_t id, uint64_t dst_id, float weight) { @@ -142,15 +142,17 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto node = shards[index].add_feature_node(id); - auto mutable_feature = node->get_mutable_feature(); + // auto mutable_feature = node->get_mutable_feature(); - mutable_feature.clear(); - mutable_feature.resize(this->feat_name.size()); + // mutable_feature.clear(); + // mutable_feature.resize(this->feat_name.size()); + node->set_feature_size(feat_name.size()); for (size_t slice = 2; slice < values.size(); slice++) { auto feat = this->parse_feature(values[slice]); - if(feat.first > 0) { - mutable_feature[feat.first] = feat.second; + if (feat.first > 0) { + // mutable_feature[feat.first] = feat.second; + node->set_feature(feat.first, feat.second); } } } @@ -212,7 +214,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { Node *GraphTable::find_node(uint64_t id) { size_t shard_id = id % shard_num; if (shard_id >= shard_end || shard_id < shard_start) { - return NULL; + return nullptr; } size_t index = shard_id - shard_start; Node *node = shards[index].find_node(id); @@ -287,7 +289,7 @@ int32_t GraphTable::random_sample_nodes(int sample_size, memcpy(pointer, res.data(), actual_size); return 0; } -int GraphTable::random_sample_neighboors( +int32_t GraphTable::random_sample_neighboors( uint64_t *node_ids, int sample_size, std::vector> &buffers, std::vector &actual_sizes) { @@ -301,13 +303,12 @@ int GraphTable::random_sample_neighboors( [&]() -> int { Node *node = find_node(node_id); - if (node == NULL) { + if (node == nullptr) { actual_size = 0; return 0; } std::vector res = node->sample_k(sample_size); - actual_size = - res.size() * (Node::id_size + Node::weight_size); + actual_size = res.size() * (Node::id_size + Node::weight_size); int offset = 0; uint64_t id; float weight; @@ -330,44 +331,66 @@ int GraphTable::random_sample_neighboors( return 0; } +int32_t GraphTable::get_node_feat(const std::vector &node_ids, + const std::vector &feature_names, + std::vector> &res) { + size_t node_num = node_ids.size(); + std::vector> tasks; + for (size_t idx = 0; idx < node_num; ++idx) { + uint64_t node_id = node_ids[idx]; + tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( + [&, idx, node_id]() -> int { + Node *node = find_node(node_id); -std::pair GraphTable::parse_feature(std::string feat_str) { - // Return (feat_id, btyes) if name are in this->feat_name, else return (-1, "") + if (node == nullptr) { + return 0; + } + for (int feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { + const std::string &feature_name = feature_names[feat_idx]; + if (feat_id_map.find(feature_name) != feat_id_map.end()) { + // res[feat_idx][idx] = + // node->get_feature(feat_id_map[feature_name]); + auto feat = node->get_feature(feat_id_map[feature_name]); + res[feat_idx][idx] = feat; + } + } + return 0; + })); + } + for (size_t idx = 0; idx < node_num; ++idx) { + tasks[idx].get(); + } + return 0; +} + +std::pair GraphTable::parse_feature( + std::string feat_str) { + // Return (feat_id, btyes) if name are in this->feat_name, else return (-1, + // "") auto fields = paddle::string::split_string(feat_str, " "); - if(this->feat_id_map.count(fields[0])) { - int32_t id = this->feat_id_map[fields[0]]; + if (this->feat_id_map.count(fields[0])) { + int32_t id = this->feat_id_map[fields[0]]; std::string dtype = this->feat_dtype[id]; int32_t shape = this->feat_shape[id]; - std::vector values(fields.begin() + 1, fields.end()); - if(dtype == "feasign"){ - return std::make_pair ( - int32_t(id), - paddle::string::join_strings(values, ' ')); - } - else if(dtype == "string") { - return std::make_pair ( - int32_t(id), - paddle::string::join_strings(values, ' ')); - } - else if(dtype == "float32") { - return std::make_pair ( - int32_t(id), - FeatureNode::parse_value_to_bytes(values)); - } - else if(dtype == "float64") { - return std::make_pair ( - int32_t(id), - FeatureNode::parse_value_to_bytes(values)); - } - else if(dtype == "int32") { - return std::make_pair ( - int32_t(id), - FeatureNode::parse_value_to_bytes(values)); - } - else if (dtype == "int64"){ - return std::make_pair ( - int32_t(id), - FeatureNode::parse_value_to_bytes(values)); + std::vector values(fields.begin() + 1, fields.end()); + if (dtype == "feasign") { + return std::make_pair( + int32_t(id), paddle::string::join_strings(values, ' ')); + } else if (dtype == "string") { + return std::make_pair( + int32_t(id), paddle::string::join_strings(values, ' ')); + } else if (dtype == "float32") { + return std::make_pair( + int32_t(id), FeatureNode::parse_value_to_bytes(values)); + } else if (dtype == "float64") { + return std::make_pair( + int32_t(id), FeatureNode::parse_value_to_bytes(values)); + } else if (dtype == "int32") { + return std::make_pair( + int32_t(id), FeatureNode::parse_value_to_bytes(values)); + } else if (dtype == "int64") { + return std::make_pair( + int32_t(id), FeatureNode::parse_value_to_bytes(values)); } } return std::make_pair(-1, ""); @@ -437,17 +460,19 @@ int32_t GraphTable::initialize() { this->table_name = common.table_name(); this->table_type = common.name(); - VLOG(0) << " init graph table type " << this->table_type << " table name " << this->table_name; + VLOG(0) << " init graph table type " << this->table_type << " table name " + << this->table_name; int feat_conf_size = static_cast(common.attributes().size()); - for(int i=0; ifeat_name.push_back(f_name); this->feat_shape.push_back(f_shape); this->feat_dtype.push_back(f_dtype); this->feat_id_map[f_name] = i; - VLOG(0) << "init graph table feat conf name:"<< f_name << " shape:" << f_shape << " dtype:" << f_dtype; + VLOG(0) << "init graph table feat conf name:" << f_name + << " shape:" << f_shape << " dtype:" << f_dtype; } shard_num = _config.shard_num(); diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 4f1905ba9165f..de3cac134cd51 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -100,7 +100,7 @@ class GraphTable : public SparseTable { int32_t load_edges(const std::string &path, bool reverse); int32_t load_nodes(const std::string &path, std::string node_type); - + Node *find_node(uint64_t id); virtual int32_t pull_sparse(float *values, const uint64_t *keys, size_t num) { @@ -121,16 +121,20 @@ class GraphTable : public SparseTable { virtual uint32_t get_thread_pool_index(uint64_t node_id); virtual std::pair parse_feature(std::string feat_str); + virtual int32_t get_node_feat(const std::vector &node_ids, + const std::vector &feature_names, + std::vector> &res); + protected: std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; const int task_pool_size_ = 11; const int random_sample_nodes_ranges = 3; - std::vector feat_name; - std::vector feat_dtype; - std::vector feat_shape; - std::unordered_map feat_id_map; + std::vector feat_name; + std::vector feat_dtype; + std::vector feat_shape; + std::unordered_map feat_id_map; std::string table_name; std::string table_type; diff --git a/paddle/fluid/distributed/table/common_sparse_table.cc b/paddle/fluid/distributed/table/common_sparse_table.cc index 9155bb7c2067b..ffedbea14a029 100644 --- a/paddle/fluid/distributed/table/common_sparse_table.cc +++ b/paddle/fluid/distributed/table/common_sparse_table.cc @@ -170,7 +170,7 @@ int64_t LoadFromText(const std::string& valuepath, const std::string& metapath, auto id = std::stoull(values[0]); if (id % pserver_num != pserver_id) { - VLOG(0) << "will not load " << values[0] << " from " << valuepath + VLOG(3) << "will not load " << values[0] << " from " << valuepath << ", please check id distribution"; continue; } @@ -263,7 +263,7 @@ int32_t CommonSparseTable::initialize_value() { } } - VLOG(0) << "has " << feasigns.size() << " ids need to be pre inited"; + VLOG(3) << "has " << feasigns.size() << " ids need to be pre inited"; auto buckets = bucket(feasigns.size(), 10); for (int x = 0; x < 10; ++x) { @@ -295,10 +295,10 @@ int32_t CommonSparseTable::initialize_optimizer() { optimizer_ = std::make_shared(value_names_, value_dims_, value_offsets_, value_idx_); } else { - VLOG(0) << "init optimizer failed"; + VLOG(3) << "init optimizer failed"; } - VLOG(0) << "init optimizer " << name << " done"; + VLOG(3) << "init optimizer " << name << " done"; return 0; } @@ -311,7 +311,7 @@ int32_t CommonSparseTable::set_global_lr(float* lr) { int32_t CommonSparseTable::load(const std::string& path, const std::string& param) { rwlock_->WRLock(); - VLOG(0) << "sparse table load with " << path << " with meta " << param; + VLOG(3) << "sparse table load with " << path << " with meta " << param; LoadFromText(path, param, _shard_idx, _shard_num, task_pool_size_, &shard_values_); rwlock_->UNLock(); @@ -322,7 +322,7 @@ int32_t CommonSparseTable::save(const std::string& dirname, const std::string& param) { rwlock_->WRLock(); int mode = std::stoi(param); - VLOG(0) << "sparse table save: " << dirname << " mode: " << mode; + VLOG(3) << "sparse table save: " << dirname << " mode: " << mode; auto varname = _config.common().table_name(); std::string var_store = @@ -538,11 +538,11 @@ int32_t CommonSparseTable::flush() { return 0; } int32_t CommonSparseTable::shrink(const std::string& param) { rwlock_->WRLock(); int threshold = std::stoi(param); - VLOG(0) << "sparse table shrink: " << threshold; + VLOG(3) << "sparse table shrink: " << threshold; for (int shard_id = 0; shard_id < task_pool_size_; ++shard_id) { // shrink - VLOG(0) << shard_id << " " << task_pool_size_ << " begin shrink"; + VLOG(4) << shard_id << " " << task_pool_size_ << " begin shrink"; shard_values_[shard_id]->Shrink(threshold); } rwlock_->UNLock(); diff --git a/paddle/fluid/distributed/table/common_table.h b/paddle/fluid/distributed/table/common_table.h index 034769e021207..dc3cfa75ff689 100644 --- a/paddle/fluid/distributed/table/common_table.h +++ b/paddle/fluid/distributed/table/common_table.h @@ -55,12 +55,13 @@ struct ReservoirValue { } void avg() { + if (counter == 0) return; auto scale = 1 / static_cast(counter); GetBlas().SCAL(values.size(), scale, values.data()); } void reset() { - values.resize(dim, 0); + std::fill(values.begin(), values.end(), 0); counter = 0; } }; @@ -134,15 +135,15 @@ class BarrierTable : public Table { return 0; } int32_t shrink(const std::string ¶m) override { return 0; } - virtual void clear(){}; - virtual int32_t flush() { return 0; }; + virtual void clear() {} + virtual int32_t flush() { return 0; } virtual int32_t load(const std::string &path, const std::string ¶m) { return 0; } virtual int32_t save(const std::string &path, const std::string ¶m) { return 0; } - virtual int32_t initialize_shard() { return 0; }; + virtual int32_t initialize_shard() { return 0; } virtual int32_t initialize() override; // only for barrier diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 9d58ce34bb70b..2e1b84a2af5be 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -13,19 +13,18 @@ // limitations under the License. #pragma once -#include #include -#include #include +#include +#include #include "paddle/fluid/distributed/table/weighted_sampler.h" namespace paddle { namespace distributed { class Node { public: - Node(){} - Node(uint64_t id) - : id(id) {} + Node() {} + Node(uint64_t id) : id(id) {} virtual ~Node() {} static int id_size, int_size, weight_size; uint64_t get_id() { return id; } @@ -35,57 +34,71 @@ class Node { virtual void build_sampler(std::string sample_type) {} virtual void add_edge(uint64_t id, float weight) {} virtual std::vector sample_k(int k) { return std::vector(); } - virtual uint64_t get_neighbor_id(int idx){ return 0; } - virtual float get_neighbor_weight(int idx){ return 1.; } + virtual uint64_t get_neighbor_id(int idx) { return 0; } + virtual float get_neighbor_weight(int idx) { return 1.; } virtual int get_size(bool need_feature); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual void add_feature(std::string feature) { } virtual std::string get_feature(int idx) { return std::string(""); } + virtual void set_feature(int idx, std::string str) {} + virtual void set_feature_size(int size) {} + virtual int get_feature_size() { return 0; } protected: uint64_t id; - }; -class GraphNode: public Node { +class GraphNode : public Node { public: - GraphNode(): Node(), sampler(nullptr), edges(nullptr) { } - GraphNode(uint64_t id) - : Node(id), sampler(nullptr), edges(nullptr) {} + GraphNode() : Node(), sampler(nullptr), edges(nullptr) {} + GraphNode(uint64_t id) : Node(id), sampler(nullptr), edges(nullptr) {} virtual ~GraphNode(); virtual void build_edges(bool is_weighted); virtual void build_sampler(std::string sample_type); - virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } + virtual void add_edge(uint64_t id, float weight) { + edges->add_edge(id, weight); + } virtual std::vector sample_k(int k) { return sampler->sample_k(k); } - virtual uint64_t get_neighbor_id(int idx){return edges->get_id(idx);} - virtual float get_neighbor_weight(int idx){return edges->get_weight(idx);} - + virtual uint64_t get_neighbor_id(int idx) { return edges->get_id(idx); } + virtual float get_neighbor_weight(int idx) { return edges->get_weight(idx); } protected: Sampler *sampler; - GraphEdgeBlob * edges; + GraphEdgeBlob *edges; }; - -class FeatureNode: public Node{ +class FeatureNode : public Node { public: - FeatureNode(): Node() { } + FeatureNode() : Node() {} FeatureNode(uint64_t id) : Node(id) {} virtual ~FeatureNode() {} virtual int get_size(bool need_feature); virtual void to_buffer(char *buffer, bool need_feature); virtual void recover_from_buffer(char *buffer); - virtual std::string get_feature(int idx) { return this->feature[idx]; } - virtual std::vector & get_mutable_feature() { return this->feature; } + virtual std::string get_feature(int idx) { + if (idx < (int)this->feature.size()) { + return this->feature[idx]; + } else { + return std::string(""); + } + } + + virtual void set_feature(int idx, std::string str) { + if (idx >= (int)this->feature.size()) { + this->feature.resize(idx + 1); + } + this->feature[idx] = str; + } + virtual void set_feature_size(int size) { this->feature.resize(size); } + virtual int get_feature_size() { return this->feature.size(); } template static std::string parse_value_to_bytes(std::vector feat_str) { T v; size_t Tsize = sizeof(T) * feat_str.size(); char buffer[Tsize]; - for(size_t i = 0;i < feat_str.size();i ++) { + for (size_t i = 0; i < feat_str.size(); i++) { std::stringstream ss(feat_str[i]); ss >> v; std::memcpy(buffer + sizeof(T) * i, (char *)&v, sizeof(T)); @@ -98,8 +111,8 @@ class FeatureNode: public Node{ T v; std::vector out; size_t start = 0; - const char * buffer = feat_str.data(); - while(start < feat_str.size()) { + const char *buffer = feat_str.data(); + while (start < feat_str.size()) { std::memcpy((char *)&v, buffer + start, sizeof(T)); start += sizeof(T); out.push_back(v); @@ -110,7 +123,5 @@ class FeatureNode: public Node{ protected: std::vector feature; }; - - } } diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 40d2abb6c86ae..d64e805af40e6 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -107,6 +107,11 @@ class Table { int &actual_sizes) { return 0; } + virtual int32_t get_node_feat(const std::vector &node_ids, + const std::vector &feature_names, + std::vector> &res) { + return 0; + } virtual int32_t pour() { return 0; } virtual void clear() = 0; diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index b410adc157104..49f915e68f3d6 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -183,22 +183,23 @@ std::string edges[] = { std::string("97\t247\t0.31"), std::string("97\t111\t0.21")}; char edge_file_name[] = "edges.txt"; -std::string nodes[] = {std::string("user\t37\ta 0.34\tb 13 14\tc hello"), - std::string("user\t96\ta 0.31\tb 15 10"), - std::string("user\t59\ta 0.11\tb 11 14"), - std::string("user\t97\ta 0.11\tb 12 11"), - std::string("item\t45\ta 0.21"), - std::string("item\t145\ta 0.21"), - std::string("item\t112\ta 0.21"), - std::string("item\t48\ta 0.21"), - std::string("item\t247\ta 0.21"), - std::string("item\t111\ta 0.21"), - std::string("item\t46\ta 0.21"), - std::string("item\t146\ta 0.21"), - std::string("item\t122\ta 0.21"), - std::string("item\t49\ta 0.21"), - std::string("item\t248\ta 0.21"), - std::string("item\t113\ta 0.21")}; +std::string nodes[] = { + std::string("user\t37\ta 0.34\tb 13 14\tc hello\td abc"), + std::string("user\t96\ta 0.31\tb 15 10\tc 96hello\td abcd"), + std::string("user\t59\ta 0.11\tb 11 14"), + std::string("user\t97\ta 0.11\tb 12 11"), + std::string("item\t45\ta 0.21"), + std::string("item\t145\ta 0.21"), + std::string("item\t112\ta 0.21"), + std::string("item\t48\ta 0.21"), + std::string("item\t247\ta 0.21"), + std::string("item\t111\ta 0.21"), + std::string("item\t46\ta 0.21"), + std::string("item\t146\ta 0.21"), + std::string("item\t122\ta 0.21"), + std::string("item\t49\ta 0.21"), + std::string("item\t248\ta 0.21"), + std::string("item\t113\ta 0.21")}; char node_file_name[] = "nodes.txt"; void prepare_file(char file_name[], bool load_edge) { @@ -423,11 +424,13 @@ void RunBrpcPushSparse() { server1.add_table_feat_conf("user", "a", "float32", 1); server1.add_table_feat_conf("user", "b", "int32", 2); server1.add_table_feat_conf("user", "c", "string", 1); + server1.add_table_feat_conf("user", "d", "string", 1); server1.add_table_feat_conf("item", "a", "float32", 1); server2.add_table_feat_conf("user", "a", "float32", 1); server2.add_table_feat_conf("user", "b", "int32", 2); server2.add_table_feat_conf("user", "c", "string", 1); + server2.add_table_feat_conf("user", "d", "string", 1); server2.add_table_feat_conf("item", "a", "float32", 1); client1.set_up(ips_str, 127, node_types, edge_types, 0); @@ -435,6 +438,7 @@ void RunBrpcPushSparse() { client1.add_table_feat_conf("user", "a", "float32", 1); client1.add_table_feat_conf("user", "b", "int32", 2); client1.add_table_feat_conf("user", "c", "string", 1); + client1.add_table_feat_conf("user", "d", "string", 1); client1.add_table_feat_conf("item", "a", "float32", 1); client2.set_up(ips_str, 127, node_types, edge_types, 1); @@ -442,6 +446,7 @@ void RunBrpcPushSparse() { client2.add_table_feat_conf("user", "a", "float32", 1); client2.add_table_feat_conf("user", "b", "int32", 2); client2.add_table_feat_conf("user", "c", "string", 1); + client2.add_table_feat_conf("user", "d", "string", 1); client2.add_table_feat_conf("item", "a", "float32", 1); server1.start_server(); @@ -504,22 +509,22 @@ void RunBrpcPushSparse() { ASSERT_EQ(nodes_ids.size(), 2); ASSERT_EQ(true, (nodes_ids[0] == 59 && nodes_ids[1] == 37) || (nodes_ids[0] == 37 && nodes_ids[1] == 59)); - // to test in python,try this: - // from paddle.fluid.core import GraphPyService - // ips_str = "127.0.0.1:4211;127.0.0.1:4212" - // gps1 = GraphPyService(); - // gps2 = GraphPyService(); - // gps1.set_up(ips_str, 127, 0, 0, 0); - // gps2.set_up(ips_str, 127, 1, 1, 0); - // gps1.load_file("input.txt"); - - // list = gps2.pull_graph_list(0,1,4) - // for x in list: - // print(x.get_id()) - - // list = gps2.sample_k(96, "user", 4); - // for x in list: - // print(x.get_id()) + + // Test get node feat + node_ids.clear(); + node_ids.push_back(37); + node_ids.push_back(96); + std::vector feature_names; + feature_names.push_back(std::string("c")); + feature_names.push_back(std::string("d")); + auto node_feat = + client1.get_node_feat(std::string("user"), node_ids, feature_names); + ASSERT_EQ(node_feat.size(), 2); + ASSERT_EQ(node_feat[0].size(), 2); + std::cout << "get_node_feat: " << node_feat[0][0] << std::endl; + std::cout << "get_node_feat: " << node_feat[0][1] << std::endl; + std::cout << "get_node_feat: " << node_feat[1][0] << std::endl; + std::cout << "get_node_feat: " << node_feat[1][1] << std::endl; std::remove(edge_file_name); std::remove(node_file_name); @@ -538,7 +543,9 @@ void RunBrpcPushSparse() { void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; - s.add_feature("hhhh"); + // s.add_feature("hhhh"); + s.set_feature_size(1); + s.set_feature(0, std::string("hhhh")); s.set_id(65); int size = s.get_size(true); char str[size]; diff --git a/paddle/fluid/extension/include/all.h b/paddle/fluid/extension/include/ext_all.h similarity index 76% rename from paddle/fluid/extension/include/all.h rename to paddle/fluid/extension/include/ext_all.h index e2a3bc38c5f4a..f2b3bcf5191c3 100644 --- a/paddle/fluid/extension/include/all.h +++ b/paddle/fluid/extension/include/ext_all.h @@ -24,8 +24,9 @@ limitations under the License. */ #endif #endif -#include "paddle/fluid/extension/include/dispatch.h" -#include "paddle/fluid/extension/include/dtype.h" -#include "paddle/fluid/extension/include/op_meta_info.h" -#include "paddle/fluid/extension/include/place.h" -#include "paddle/fluid/extension/include/tensor.h" +#include "ext_dispatch.h" // NOLINT +#include "ext_dtype.h" // NOLINT +#include "ext_exception.h" // NOLINT +#include "ext_op_meta_info.h" // NOLINT +#include "ext_place.h" // NOLINT +#include "ext_tensor.h" // NOLINT diff --git a/paddle/fluid/extension/include/dispatch.h b/paddle/fluid/extension/include/ext_dispatch.h similarity index 76% rename from paddle/fluid/extension/include/dispatch.h rename to paddle/fluid/extension/include/ext_dispatch.h index c22971039521c..9b3e199708adc 100644 --- a/paddle/fluid/extension/include/dispatch.h +++ b/paddle/fluid/extension/include/ext_dispatch.h @@ -14,7 +14,8 @@ limitations under the License. */ #pragma once -#include "paddle/fluid/extension/include/dtype.h" +#include "ext_dtype.h" // NOLINT +#include "ext_exception.h" // NOLINT namespace paddle { @@ -32,19 +33,34 @@ namespace paddle { ///////// Floating Dispatch Marco /////////// -#define PD_DISPATCH_FLOATING_TYPES(TYPE, NAME, ...) \ - [&] { \ - const auto& __dtype__ = TYPE; \ - switch (__dtype__) { \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT32, float, \ - __VA_ARGS__) \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT64, double, \ - __VA_ARGS__) \ - default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ - } \ +#define PD_DISPATCH_FLOATING_TYPES(TYPE, NAME, ...) \ + [&] { \ + const auto& __dtype__ = TYPE; \ + switch (__dtype__) { \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT32, float, \ + __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT64, double, \ + __VA_ARGS__) \ + default: \ + PD_THROW("function " #NAME " is not implemented for data type `", \ + ::paddle::ToString(__dtype__), "`"); \ + } \ + }() + +#define PD_DISPATCH_FLOATING_AND_HALF_TYPES(TYPE, NAME, ...) \ + [&] { \ + const auto& __dtype__ = TYPE; \ + switch (__dtype__) { \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT32, float, \ + __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT64, double, \ + __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT16, paddle::float16, \ + __VA_ARGS__) \ + default: \ + PD_THROW("function " #NAME " is not implemented for data type `", \ + ::paddle::ToString(__dtype__), "`"); \ + } \ }() ///////// Integral Dispatch Marco /////////// @@ -63,27 +79,25 @@ namespace paddle { PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::INT16, int16_t, \ __VA_ARGS__) \ default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ + PD_THROW("function " #NAME " is not implemented for data type `" + \ + ::paddle::ToString(__dtype__) + "`"); \ } \ }() ///////// Complex Dispatch Marco /////////// -#define PD_DISPATCH_COMPLEX_TYPES(TYPE, NAME, ...) \ - [&] { \ - const auto& __dtype__ = TYPE; \ - switch (__dtype__) { \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX64, \ - ::paddle::complex64, __VA_ARGS__) \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX128, \ - ::paddle::complex128, __VA_ARGS__) \ - default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ - } \ +#define PD_DISPATCH_COMPLEX_TYPES(TYPE, NAME, ...) \ + [&] { \ + const auto& __dtype__ = TYPE; \ + switch (__dtype__) { \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX64, \ + ::paddle::complex64, __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX128, \ + ::paddle::complex128, __VA_ARGS__) \ + default: \ + PD_THROW("function " #NAME " is not implemented for data type `" + \ + ::paddle::ToString(__dtype__) + "`"); \ + } \ }() ///////// Floating and Integral Dispatch Marco /////////// @@ -106,31 +120,29 @@ namespace paddle { PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::INT16, int16_t, \ __VA_ARGS__) \ default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ + PD_THROW("function " #NAME " is not implemented for data type `" + \ + ::paddle::ToString(__dtype__) + "`"); \ } \ }() ///////// Floating and Complex Dispatch Marco /////////// -#define PD_DISPATCH_FLOATING_AND_COMPLEX_TYPES(TYPE, NAME, ...) \ - [&] { \ - const auto& __dtype__ = TYPE; \ - switch (__dtype__) { \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT32, float, \ - __VA_ARGS__) \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT64, double, \ - __VA_ARGS__) \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX64, \ - ::paddle::complex64, __VA_ARGS__) \ - PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX128, \ - ::paddle::complex128, __VA_ARGS__) \ - default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ - } \ +#define PD_DISPATCH_FLOATING_AND_COMPLEX_TYPES(TYPE, NAME, ...) \ + [&] { \ + const auto& __dtype__ = TYPE; \ + switch (__dtype__) { \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT32, float, \ + __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::FLOAT64, double, \ + __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX64, \ + ::paddle::complex64, __VA_ARGS__) \ + PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX128, \ + ::paddle::complex128, __VA_ARGS__) \ + default: \ + PD_THROW("function " #NAME " is not implemented for data type `" + \ + ::paddle::ToString(__dtype__) + "`"); \ + } \ }() ///////// Floating, Integral and Complex Dispatch Marco /////////// @@ -157,9 +169,8 @@ namespace paddle { PD_PRIVATE_CASE_TYPE(NAME, ::paddle::DataType::COMPLEX128, \ ::paddle::complex128, __VA_ARGS__) \ default: \ - throw std::runtime_error("function " #NAME \ - " not implemented for data type `" + \ - ::paddle::ToString(__dtype__) + "`"); \ + PD_THROW("function " #NAME " is not implemented for data type `" + \ + ::paddle::ToString(__dtype__) + "`"); \ } \ }() diff --git a/paddle/fluid/extension/include/dll_decl.h b/paddle/fluid/extension/include/ext_dll_decl.h similarity index 100% rename from paddle/fluid/extension/include/dll_decl.h rename to paddle/fluid/extension/include/ext_dll_decl.h diff --git a/paddle/fluid/extension/include/dtype.h b/paddle/fluid/extension/include/ext_dtype.h similarity index 85% rename from paddle/fluid/extension/include/dtype.h rename to paddle/fluid/extension/include/ext_dtype.h index c5d2e0f820555..3890631a6f8a9 100644 --- a/paddle/fluid/extension/include/dtype.h +++ b/paddle/fluid/extension/include/ext_dtype.h @@ -11,22 +11,23 @@ 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. */ - #pragma once -#include "paddle/fluid/platform/bfloat16.h" -#include "paddle/fluid/platform/complex128.h" -#include "paddle/fluid/platform/complex64.h" -#include "paddle/fluid/platform/float16.h" +#include +#include + +#include "complex128.h" // NOLINT +#include "complex64.h" // NOLINT +#include "ext_exception.h" // NOLINT +#include "float16.h" // NOLINT namespace paddle { -using float16 = paddle::platform::float16; -using bfloat16 = paddle::platform::bfloat16; using complex64 = paddle::platform::complex64; using complex128 = paddle::platform::complex128; +using float16 = paddle::platform::float16; -enum DataType { +enum class DataType { BOOL, INT8, UINT8, @@ -34,7 +35,6 @@ enum DataType { INT32, INT64, FLOAT16, - BFLOAT16, FLOAT32, FLOAT64, COMPLEX64, @@ -58,8 +58,6 @@ inline std::string ToString(DataType dtype) { return "int64_t"; case DataType::FLOAT16: return "float16"; - case DataType::BFLOAT16: - return "bfloat16"; case DataType::FLOAT32: return "float"; case DataType::FLOAT64: @@ -69,7 +67,7 @@ inline std::string ToString(DataType dtype) { case DataType::COMPLEX128: return "complex128"; default: - throw std::runtime_error("Unsupported paddle enum data type."); + PD_THROW("Unsupported paddle enum data type."); } } @@ -81,7 +79,6 @@ inline std::string ToString(DataType dtype) { _(int, DataType::INT32) \ _(int64_t, DataType::INT64) \ _(float16, DataType::FLOAT16) \ - _(bfloat16, DataType::BFLOAT16) \ _(float, DataType::FLOAT32) \ _(double, DataType::FLOAT64) \ _(complex64, DataType::COMPLEX64) \ diff --git a/paddle/fluid/extension/include/ext_exception.h b/paddle/fluid/extension/include/ext_exception.h new file mode 100644 index 0000000000000..632d91d5285c2 --- /dev/null +++ b/paddle/fluid/extension/include/ext_exception.h @@ -0,0 +1,108 @@ +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. + +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. */ + +#pragma once + +#include +#include +#include + +namespace paddle { + +//////////////// Exception handling and Error Message ///////////////// +#if !defined(_WIN32) +#define PD_UNLIKELY(expr) (__builtin_expect(static_cast(expr), 0)) +#define PD_LIKELY(expr) (__builtin_expect(static_cast(expr), 1)) +#else +#define PD_UNLIKELY(expr) (expr) +#define PD_LIKELY(expr) (expr) +#endif + +struct PD_Exception : public std::exception { + public: + template + explicit PD_Exception(const std::string& msg, const char* file, int line, + const char* default_msg) { + std::ostringstream sout; + if (msg.empty()) { + sout << default_msg << "\n [" << file << ":" << line << "]"; + } else { + sout << msg << "\n [" << file << ":" << line << "]"; + } + err_msg_ = sout.str(); + } + + const char* what() const noexcept override { return err_msg_.c_str(); } + + private: + std::string err_msg_; +}; + +class ErrorMessage { + public: + template + explicit ErrorMessage(const Args&... args) { + build_string(args...); + } + + void build_string() { oss << ""; } + + template + void build_string(const T& t) { + oss << t; + } + + template + void build_string(const T& t, const Args&... args) { + build_string(t); + build_string(args...); + } + + std::string to_string() { return oss.str(); } + + private: + std::ostringstream oss; +}; + +#if defined _WIN32 +#define HANDLE_THE_ERROR try { +#define END_HANDLE_THE_ERROR \ + } \ + catch (const std::exception& e) { \ + std::cerr << e.what() << std::endl; \ + throw e; \ + } +#else +#define HANDLE_THE_ERROR +#define END_HANDLE_THE_ERROR +#endif + +#define PD_CHECK(COND, ...) \ + do { \ + if (PD_UNLIKELY(!(COND))) { \ + auto __message__ = ::paddle::ErrorMessage(__VA_ARGS__).to_string(); \ + throw ::paddle::PD_Exception(__message__, __FILE__, __LINE__, \ + "Expected " #COND \ + ", but it's not satisfied."); \ + } \ + } while (0) + +#define PD_THROW(...) \ + do { \ + auto __message__ = ::paddle::ErrorMessage(__VA_ARGS__).to_string(); \ + throw ::paddle::PD_Exception(__message__, __FILE__, __LINE__, \ + "An error occurred."); \ + } while (0) + +} // namespace paddle diff --git a/paddle/fluid/extension/include/ext_op_meta_info.h b/paddle/fluid/extension/include/ext_op_meta_info.h new file mode 100644 index 0000000000000..c400164c7543d --- /dev/null +++ b/paddle/fluid/extension/include/ext_op_meta_info.h @@ -0,0 +1,539 @@ +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. + +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. */ + +#pragma once + +#include +#include +#include +#include + +#include + +#include "ext_dll_decl.h" // NOLINT +#include "ext_exception.h" // NOLINT +#include "ext_tensor.h" // NOLINT + +/** + * Op Meta Info Related Define. + * + * Used to maintain operator core information. + * + */ + +namespace paddle { +namespace framework { +class PD_DLL_DECL OpMetaInfoHelper; +} // namespace framework + +using Tensor = paddle::Tensor; + +///////////////// Util Marco Define //////////////// + +#define PD_DISABLE_COPY_AND_ASSIGN(classname) \ + private: \ + classname(const classname&) = delete; \ + classname(classname&&) = delete; \ + classname& operator=(const classname&) = delete; \ + classname& operator=(classname&&) = delete + +#define STATIC_ASSERT_GLOBAL_NAMESPACE(uniq_name, msg) \ + struct __test_global_namespace_##uniq_name##__ {}; \ + static_assert(std::is_same<::__test_global_namespace_##uniq_name##__, \ + __test_global_namespace_##uniq_name##__>::value, \ + msg) + +///////////////// Util Define and Function //////////////// + +constexpr char kGradTensorSuffix[] = "@GRAD"; +constexpr char kTensorVectorSuffix[] = "@VECTOR"; + +// Used for Construct Grad Tensor name +inline std::string Grad(const std::string& t_name) { + std::string result; + result.reserve(t_name.size() + 5U); + result += t_name; + result += kGradTensorSuffix; + return result; +} + +// Used for Construct std::vector name +inline std::string Vec(const std::string& t_name) { + std::string result; + result.reserve(t_name.size() + 7U); + result += t_name; + result += kTensorVectorSuffix; + return result; +} + +////////////////////// Kernel Function (PD_KERNEL) //////////////////////// + +// Record Op kernel core function +using KernelFunc = + std::vector (*)(const std::vector& inputs, + const std::vector>& vec_inputs, + const std::vector& attrs); + +#define PD_SPECIALIZE_ComputeCallHelper(attr_type) \ + template \ + struct ComputeCallHelper { \ + template \ + static Return Compute(const std::vector& inputs, \ + const std::vector>& vec_inputs, \ + const std::vector& attrs, \ + const PreviousArgs&... pargs) { \ + try { \ + attr_type arg = boost::any_cast(attrs[attr_idx]); \ + return ComputeCallHelper::template Compute< \ + in_idx, vec_in_idx, attr_idx + 1>(inputs, vec_inputs, attrs, \ + pargs..., arg); \ + } catch (boost::bad_any_cast&) { \ + PD_THROW( \ + "Attribute cast error in custom operator. Expected " #attr_type \ + " value."); \ + } \ + } \ + } + +template +struct TypeTag {}; + +template +struct KernelFuncImpl; + +template +struct KernelFuncImpl { + static Return Compute(const std::vector& inputs, + const std::vector>& vec_inputs, + const std::vector& attrs) { + return ComputeCallHelper>::template Compute<0, 0, 0>( + inputs, vec_inputs, attrs); + } + + private: + template + struct ComputeCallHelper; + + template + struct ComputeCallHelper { + template + static Return Compute(const std::vector& inputs, + const std::vector>& vec_inputs, + const std::vector& attrs, + const PreviousArgs&... pargs) { + const Tensor& arg = inputs[in_idx]; + return ComputeCallHelper::template Compute( + inputs, vec_inputs, attrs, pargs..., arg); + } + }; + + template + struct ComputeCallHelper&, Tail...> { + template + static Return Compute(const std::vector& inputs, + const std::vector>& vec_inputs, + const std::vector& attrs, + const PreviousArgs&... pargs) { + const std::vector& arg = vec_inputs[vec_in_idx]; + return ComputeCallHelper::template Compute< + in_idx, vec_in_idx + 1, attr_idx>(inputs, vec_inputs, attrs, pargs..., + arg); + } + }; + + PD_SPECIALIZE_ComputeCallHelper(const bool&); + PD_SPECIALIZE_ComputeCallHelper(const int&); + PD_SPECIALIZE_ComputeCallHelper(const float&); + PD_SPECIALIZE_ComputeCallHelper(const int64_t&); + PD_SPECIALIZE_ComputeCallHelper(const std::string&); + PD_SPECIALIZE_ComputeCallHelper(const std::vector&); + PD_SPECIALIZE_ComputeCallHelper(const std::vector&); + PD_SPECIALIZE_ComputeCallHelper(const std::vector&); + PD_SPECIALIZE_ComputeCallHelper(const std::vector&); + // TODO(chenweihang): support other attribute type if needed. + // Why not support other attribute type here? + // - boost::blank, std::vector and std::vector + // are not used in op + // - BlockDesc* and std::vector are used in framework + + // NOTE(chenweihang): Used to be compatible with the 2.0.1 released + // interface, and will be deprecated in the future + PD_SPECIALIZE_ComputeCallHelper(bool); + PD_SPECIALIZE_ComputeCallHelper(int); + PD_SPECIALIZE_ComputeCallHelper(float); + PD_SPECIALIZE_ComputeCallHelper(int64_t); + PD_SPECIALIZE_ComputeCallHelper(std::string); + PD_SPECIALIZE_ComputeCallHelper(std::vector); + PD_SPECIALIZE_ComputeCallHelper(std::vector); + PD_SPECIALIZE_ComputeCallHelper(std::vector); + PD_SPECIALIZE_ComputeCallHelper(std::vector); + + // end: base template + template + struct ComputeCallHelper> { + template + static Return Compute(const std::vector& inputs, + const std::vector>& vec_inputs, + const std::vector& attrs, + const Args&... args) { + return impl_fn(args...); + } + }; +}; + +#define PD_KERNEL(...) \ + ::paddle::KernelFuncImpl::Compute + +/////////////// InferShape Function (PD_INFER_SHAPE) /////////////// + +// Record Op infershape core function +using InferShapeFunc = std::vector> (*)( + const std::vector>& input_shapes, + const std::vector>>& vec_input_shapes, + const std::vector& attrs); + +#define PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPE(input_type) \ + template \ + struct InferShapeCallHelper { \ + template \ + static Return InferShape( \ + const std::vector>& input_shapes, \ + const std::vector>>& \ + vec_input_shapes, \ + const std::vector& attrs, const PreviousArgs&... pargs) { \ + input_type arg = input_shapes[in_idx]; \ + return InferShapeCallHelper::template InferShape< \ + in_idx + 1, vec_in_idx, attr_idx>(input_shapes, vec_input_shapes, \ + attrs, pargs..., arg); \ + } \ + } + +#define PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPES(input_type) \ + template \ + struct InferShapeCallHelper { \ + template \ + static Return InferShape( \ + const std::vector>& input_shapes, \ + const std::vector>>& \ + vec_input_shapes, \ + const std::vector& attrs, const PreviousArgs&... pargs) { \ + input_type arg = vec_input_shapes[vec_in_idx]; \ + return InferShapeCallHelper::template InferShape< \ + in_idx, vec_in_idx + 1, attr_idx>(input_shapes, vec_input_shapes, \ + attrs, pargs..., arg); \ + } \ + } + +#define PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(attr_type) \ + template \ + struct InferShapeCallHelper { \ + template \ + static Return InferShape( \ + const std::vector>& input_shapes, \ + const std::vector>>& \ + vec_input_shapes, \ + const std::vector& attrs, const PreviousArgs&... pargs) { \ + try { \ + attr_type arg = boost::any_cast(attrs[attr_idx]); \ + return InferShapeCallHelper::template InferShape< \ + in_idx, vec_in_idx, attr_idx + 1>(input_shapes, vec_input_shapes, \ + attrs, pargs..., arg); \ + } catch (boost::bad_any_cast&) { \ + PD_THROW( \ + "Attribute cast error in custom operator InferShapeFn. " \ + "Expected " #attr_type \ + " value. InferShapeFn's attribute list must be exactly same as " \ + "Forward " \ + "KernelFn's attribute list except std::vector " \ + "attribute."); \ + } \ + } \ + } + +template +struct InferShapeFuncImpl; + +template +struct InferShapeFuncImpl { + static Return InferShape( + const std::vector>& input_shapes, + const std::vector>>& vec_input_shapes, + const std::vector& attrs) { + return InferShapeCallHelper>::template InferShape< + 0, 0, 0>(input_shapes, vec_input_shapes, attrs); + } + + private: + template + struct InferShapeCallHelper; + + PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPE(const std::vector&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPES( + const std::vector>&); + + // NOTE(chenweihang): Used to be compatible with the 2.0.1 released + // interface, and will be deprecated in the future + PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPE(std::vector); + PD_SPECIALIZE_InferShapeCallHelper_FOR_SHAPES( + std::vector>); + + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const bool&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const int&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const float&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const int64_t&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const std::string&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const std::vector&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const std::vector&); + PD_SPECIALIZE_InferShapeCallHelper_FOR_ATTR(const std::vector&); + // NOTE(chenweihang): InferShape can't support std::vector attr type, + // because the input type is std::vector, only can use one rule to + // parse std::vector parameter + + // end: base template + template + struct InferShapeCallHelper> { + template + static Return InferShape( + const std::vector>& input_shapes, + const std::vector>>& vec_input_shapes, + const std::vector& attrs, const Args&... args) { + return impl_fn(args...); + } + }; +}; + +#define PD_INFER_SHAPE(...) \ + ::paddle::InferShapeFuncImpl::InferShape + +/////////////// InferDataType Function (PD_INFER_DTYPE) /////////////// + +// Record Op Infer dtype core function +using InferDtypeFunc = std::vector (*)( + const std::vector& input_dtypes, + const std::vector>& vec_input_dtypes); + +#define PD_SPECIALIZE_InferDtypeCallHelper_TO_DTYPE(input_type) \ + template \ + struct InferDtypeCallHelper { \ + template \ + static Return InferDtype( \ + const std::vector& input_dtypes, \ + const std::vector>& vec_input_dtypes, \ + const PreviousArgs&... pargs) { \ + input_type arg = input_dtypes[in_idx]; \ + return InferDtypeCallHelper::template InferDtype( \ + input_dtypes, vec_input_dtypes, pargs..., arg); \ + } \ + } + +#define PD_SPECIALIZE_InferDtypeCallHelper_FOR_DTYPES(input_type) \ + template \ + struct InferDtypeCallHelper { \ + template \ + static Return InferDtype( \ + const std::vector& input_dtypes, \ + const std::vector>& vec_input_dtypes, \ + const PreviousArgs&... pargs) { \ + input_type arg = vec_input_dtypes[vec_in_idx]; \ + return InferDtypeCallHelper::template InferDtype< \ + in_idx, vec_in_idx + 1>(input_dtypes, vec_input_dtypes, pargs..., \ + arg); \ + } \ + } + +template +struct InferDtypeFuncImpl; + +template +struct InferDtypeFuncImpl { + static Return InferDtype( + const std::vector& input_dtypes, + const std::vector>& vec_input_dtypes) { + return InferDtypeCallHelper>::template InferDtype<0, + 0>( + input_dtypes, vec_input_dtypes); + } + + private: + template + struct InferDtypeCallHelper; + + PD_SPECIALIZE_InferDtypeCallHelper_TO_DTYPE(const DataType&); + PD_SPECIALIZE_InferDtypeCallHelper_FOR_DTYPES(const std::vector&); + + // NOTE(chenweihang): Used to be compatible with the 2.0.1 released + // interface, and will be deprecated in the future + PD_SPECIALIZE_InferDtypeCallHelper_TO_DTYPE(DataType); + PD_SPECIALIZE_InferDtypeCallHelper_FOR_DTYPES(std::vector); + + // end: base template + template + struct InferDtypeCallHelper> { + template + static Return InferDtype( + const std::vector& input_dtypes, + const std::vector>& vec_input_dtypes, + const Args&... args) { + return impl_fn(args...); + } + }; +}; + +#define PD_INFER_DTYPE(...) \ + ::paddle::InferDtypeFuncImpl::InferDtype + +////////////////////// Op Meta Info ////////////////////// + +class PD_DLL_DECL OpMetaInfo { + public: + explicit OpMetaInfo(const std::string& op_name) : name_(op_name) {} + + // format: {"", "", ...} + OpMetaInfo& Inputs(std::vector&& inputs); + + // format: {"", "", ...} + OpMetaInfo& Outputs(std::vector&& outputs); + + // format: {":", ":", ...} + OpMetaInfo& Attrs(std::vector&& attrs); + + // format: PD_KERNEL(...) + OpMetaInfo& SetKernelFn(KernelFunc&& func); + + // format: PD_INFER_SHAPE(...) + OpMetaInfo& SetInferShapeFn(InferShapeFunc&& func); + + // format: PD_INFER_DTYPE(...) + OpMetaInfo& SetInferDtypeFn(InferDtypeFunc&& func); + + private: + friend class framework::OpMetaInfoHelper; + + // 1. desc info + std::string name_; + std::vector inputs_; + std::vector outputs_; + std::vector attrs_; + + // 2. func info + KernelFunc kernel_fn_{nullptr}; + InferShapeFunc infer_shape_fn_{nullptr}; + InferDtypeFunc infer_dtype_fn_{nullptr}; +}; + +//////////////// Op Meta Info Map ///////////////// + +class PD_DLL_DECL OpMetaInfoMap { + public: + // this function's impl should keep in header file. + // if move to cc file, meta info can not be added + // into map + static OpMetaInfoMap& Instance() { + static OpMetaInfoMap g_custom_op_meta_info_map; + return g_custom_op_meta_info_map; + } + + std::vector& operator[](const std::string& name); + + const std::unordered_map>& GetMap() + const; + + private: + OpMetaInfoMap() = default; + std::unordered_map> map_; + + PD_DISABLE_COPY_AND_ASSIGN(OpMetaInfoMap); +}; + +//////////////// Op Meta Info Builder ///////////////// + +class PD_DLL_DECL OpMetaInfoBuilder { + public: + explicit OpMetaInfoBuilder(std::string&& name, size_t index); + OpMetaInfoBuilder& Inputs(std::vector&& inputs); + OpMetaInfoBuilder& Outputs(std::vector&& outputs); + OpMetaInfoBuilder& Attrs(std::vector&& attrs); + OpMetaInfoBuilder& SetKernelFn(KernelFunc func); + OpMetaInfoBuilder& SetInferShapeFn(InferShapeFunc func); + OpMetaInfoBuilder& SetInferDtypeFn(InferDtypeFunc func); + + private: + // Forward Op name + std::string name_; + // ref current info ptr + OpMetaInfo* info_ptr_; + // The current op meta info index in vector + // - 0: op, 1: grad_op, 2: grad_grad_op + size_t index_; +}; + +/////////////////////// Op register API ///////////////////////// + +// For inference: compile directly with framework +// Call after PD_BUILD_OP(...) +void RegisterAllCustomOperator(); + +// Using this api to load compiled custom operator's dynamic library and +// register Custom +// Operator into it +void LoadCustomOperatorLib(const std::string& dso_name); + +/////////////////////// Op register Macro ///////////////////////// + +#define PD_BUILD_OP(op_name) \ + STATIC_ASSERT_GLOBAL_NAMESPACE( \ + __reg_op__##op_name, "PD_BUILD_OP must be called in global namespace."); \ + static ::paddle::OpMetaInfoBuilder __op_meta_info_##op_name##__ = \ + ::paddle::OpMetaInfoBuilder(#op_name, 0) + +#define PD_BUILD_GRAD_OP(op_name) \ + STATIC_ASSERT_GLOBAL_NAMESPACE( \ + __reg_grad_op__##op_name, \ + "PD_BUILD_GRAD_OP must be called in global namespace."); \ + static ::paddle::OpMetaInfoBuilder __grad_op_meta_info_##op_name##__ = \ + ::paddle::OpMetaInfoBuilder(#op_name, 1) + +#define PD_BUILD_DOUBLE_GRAD_OP(op_name) \ + STATIC_ASSERT_GLOBAL_NAMESPACE( \ + __reg_grad_grad_op__##op_name, \ + "PD_BUILD_DOUBLE_GRAD_OP must be called in global namespace."); \ + static ::paddle::OpMetaInfoBuilder __grad_grad_op_meta_info_##op_name##__ = \ + ::paddle::OpMetaInfoBuilder(#op_name, 2) + +} // namespace paddle + +///////////////////// C API /////////////////// + +#ifdef __cplusplus +extern "C" { +#endif + +#if defined(_WIN32) +// C-API to get global OpMetaInfoMap. +__declspec(dllexport) inline paddle::OpMetaInfoMap& PD_GetOpMetaInfoMap() { + return paddle::OpMetaInfoMap::Instance(); +} +#endif // _WIN32 + +#ifdef __cplusplus +} +#endif diff --git a/paddle/fluid/extension/include/place.h b/paddle/fluid/extension/include/ext_place.h similarity index 100% rename from paddle/fluid/extension/include/place.h rename to paddle/fluid/extension/include/ext_place.h diff --git a/paddle/fluid/extension/include/tensor.h b/paddle/fluid/extension/include/ext_tensor.h similarity index 79% rename from paddle/fluid/extension/include/tensor.h rename to paddle/fluid/extension/include/ext_tensor.h index 47af4dc70a15f..be492a6d5535d 100644 --- a/paddle/fluid/extension/include/tensor.h +++ b/paddle/fluid/extension/include/ext_tensor.h @@ -16,14 +16,37 @@ limitations under the License. */ #include #include -#include "paddle/fluid/extension/include/dll_decl.h" -#include "paddle/fluid/extension/include/dtype.h" -#include "paddle/fluid/extension/include/place.h" +#ifdef PADDLE_WITH_CUDA +#include +#endif + +#include "ext_dll_decl.h" // NOLINT +#include "ext_dtype.h" // NOLINT +#include "ext_place.h" // NOLINT namespace paddle { namespace framework { class CustomTensorUtils; } // namespace framework + +class StreamWrapper { + public: + StreamWrapper() : stream_(nullptr), is_stream_set_(false) {} + void SetStream(void* stream) { + stream_ = stream; + is_stream_set_ = true; + } + + void* GetStream() const { return stream_; } + + bool IsStreamSet() const { return is_stream_set_; } + + private: + // cudaStream_t stream_; + void* stream_; + bool is_stream_set_; +}; + class PD_DLL_DECL Tensor { public: /// \brief Construct a Tensor on target Place for CustomOp. @@ -34,7 +57,7 @@ class PD_DLL_DECL Tensor { /// Reshape must be called before calling /// mutable_data() or copy_to(const PlaceType& place) /// \param shape The shape to set. - void reshape(const std::vector& shape); + void reshape(const std::vector& shape); /// \brief Get the memory pointer in CPU or GPU with /// specific data type. @@ -67,7 +90,7 @@ class PD_DLL_DECL Tensor { Tensor copy_to(const PlaceType& place) const; /// \brief Return the shape of the Tensor. - std::vector shape() const; + std::vector shape() const; /// \brief Return the data type of the tensor. /// It's usually used to get the output tensor data type. @@ -87,10 +110,16 @@ class PD_DLL_DECL Tensor { /// \brief Cast datatype from one to another Tensor cast(const DataType& target_type) const; +#ifdef PADDLE_WITH_CUDA + /// \bref Get current stream of Tensor + cudaStream_t stream() const; +#endif + private: friend class framework::CustomTensorUtils; mutable std::shared_ptr tensor_; mutable PlaceType place_; + StreamWrapper stream_; }; } // namespace paddle diff --git a/paddle/fluid/extension/include/op_meta_info.h b/paddle/fluid/extension/include/op_meta_info.h deleted file mode 100644 index 1bc044f647fba..0000000000000 --- a/paddle/fluid/extension/include/op_meta_info.h +++ /dev/null @@ -1,379 +0,0 @@ -/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. - -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. */ - -#pragma once - -#include -#include -#include -#include - -#include - -#include "paddle/fluid/extension/include/dll_decl.h" -#include "paddle/fluid/extension/include/tensor.h" - -/** - * Op Meta Info Related Define. - * - * Used to maintain operator core information. - * - */ - -namespace paddle { -namespace framework { -class PD_DLL_DECL OpMetaInfoHelper; -} // namespace framework - -using Tensor = paddle::Tensor; - -#define PD_DISABLE_COPY_AND_ASSIGN(classname) \ - private: \ - classname(const classname&) = delete; \ - classname(classname&&) = delete; \ - classname& operator=(const classname&) = delete; \ - classname& operator=(classname&&) = delete - -#if defined _WIN32 -#define HANDLE_THE_ERROR try { -#define END_HANDLE_THE_ERROR \ - } \ - catch (const std::exception& e) { \ - std::cerr << e.what() << std::endl; \ - throw e; \ - } -#else -#define HANDLE_THE_ERROR -#define END_HANDLE_THE_ERROR -#endif - -#define PD_THROW(err_msg) \ - do { \ - HANDLE_THE_ERROR \ - throw std::runtime_error(err_msg); \ - END_HANDLE_THE_ERROR \ - } while (0) - -///////////////// Util Define and Function //////////////// - -inline std::string Grad(const std::string& var_name) { - std::string result; - result.reserve(var_name.size() + 5U); - result += var_name; - result += "@GRAD"; - return result; -} - -////////////////////// Kernel Function (PD_KERNEL) //////////////////////// - -// Record Op kernel core function -using KernelFunc = std::vector (*)(std::vector inputs, - std::vector attrs); - -#define PD_SPECIALIZE_ComputeCallHelper(attr_type) \ - template \ - struct ComputeCallHelper { \ - template \ - static Return Compute(std::vector inputs, \ - std::vector attrs, \ - const PreviousArgs&... pargs) { \ - try { \ - attr_type arg = boost::any_cast(attrs[attr_idx]); \ - return ComputeCallHelper::template Compute( \ - inputs, attrs, pargs..., arg); \ - } catch (boost::bad_any_cast&) { \ - PD_THROW( \ - "Attribute cast error in custom operator. Expected " #attr_type \ - " value."); \ - } \ - } \ - } - -template -struct TypeTag {}; - -template -struct KernelFuncImpl; - -template -struct KernelFuncImpl { - static Return Compute(std::vector inputs, - std::vector attrs) { - return ComputeCallHelper>::template Compute<0, 0>( - inputs, attrs); - } - - private: - template - struct ComputeCallHelper; - - // for Tensor input - template - struct ComputeCallHelper { - template - static Return Compute(std::vector inputs, - std::vector attrs, - const PreviousArgs&... pargs) { - static_assert(attr_idx == 0, - "Input tensor should appear before attributes."); - const Tensor& arg = inputs[in_idx]; - return ComputeCallHelper::template Compute( - inputs, attrs, pargs..., arg); - } - }; - - PD_SPECIALIZE_ComputeCallHelper(bool); - PD_SPECIALIZE_ComputeCallHelper(int); - PD_SPECIALIZE_ComputeCallHelper(float); - PD_SPECIALIZE_ComputeCallHelper(int64_t); - PD_SPECIALIZE_ComputeCallHelper(std::string); - PD_SPECIALIZE_ComputeCallHelper(std::vector); - PD_SPECIALIZE_ComputeCallHelper(std::vector); - PD_SPECIALIZE_ComputeCallHelper(std::vector); - PD_SPECIALIZE_ComputeCallHelper(std::vector); - // TODO(chenweihang): support other attribute type if needed. - // Why not support other attribute type here? - // - boost::blank, std::vector and std::vector - // are not used in op - // - BlockDesc* and std::vector are used in framework - // end: base template - template - struct ComputeCallHelper> { - template - static Return Compute(std::vector inputs, - std::vector attrs, const Args&... args) { - return impl_fn(args...); - } - }; -}; - -#define PD_KERNEL(...) \ - ::paddle::KernelFuncImpl::Compute - -/////////////// InferShape Function (PD_INFER_SHAPE) /////////////// - -// Record Op infershape core function -using InferShapeFunc = std::vector> (*)( - std::vector> input_shapes); - -template -struct InferShapeFuncImpl; - -template -struct InferShapeFuncImpl { - static Return InferShape(std::vector> input_shapes) { - return InferShapeCallHelper>::template InferShape<0>( - input_shapes); - } - - private: - template - struct InferShapeCallHelper; - - // only one type input: std::vector - template - struct InferShapeCallHelper, Tail...> { - template - static Return InferShape(std::vector> input_shapes, - const PreviousArgs&... pargs) { - std::vector arg = input_shapes[in_idx]; - return InferShapeCallHelper::template InferShape( - input_shapes, pargs..., arg); - } - }; - - // end: base template - template - struct InferShapeCallHelper> { - template - static Return InferShape(std::vector> input_shapes, - const Args&... args) { - return impl_fn(args...); - } - }; -}; - -#define PD_INFER_SHAPE(...) \ - ::paddle::InferShapeFuncImpl::InferShape - -/////////////// InferDataType Function (PD_INFER_DTYPE) /////////////// - -// Record Op Infer dtype core function -using InferDtypeFunc = - std::vector (*)(std::vector input_dtypes); - -template -struct InferDtypeFuncImpl; - -template -struct InferDtypeFuncImpl { - static Return InferDtype(std::vector input_dtypes) { - return InferDtypeCallHelper>::template InferDtype<0>( - input_dtypes); - } - - private: - template - struct InferDtypeCallHelper; - - // Only one type input now: DataType - template - struct InferDtypeCallHelper { - template - static Return InferDtype(std::vector input_dtypes, - const PreviousArgs&... pargs) { - DataType arg = input_dtypes[in_idx]; - return InferDtypeCallHelper::template InferDtype( - input_dtypes, pargs..., arg); - } - }; - - // end: base template - template - struct InferDtypeCallHelper> { - template - static Return InferDtype(std::vector input_dtypes, - const Args&... args) { - return impl_fn(args...); - } - }; -}; - -#define PD_INFER_DTYPE(...) \ - ::paddle::InferDtypeFuncImpl::InferDtype - -////////////////////// Op Meta Info ////////////////////// - -class PD_DLL_DECL OpMetaInfo { - public: - explicit OpMetaInfo(const std::string& op_name) : name_(op_name) {} - - // format: {"", "", ...} - OpMetaInfo& Inputs(std::vector&& inputs); - - // format: {"", "", ...} - OpMetaInfo& Outputs(std::vector&& outputs); - - // format: {":", ":", ...} - OpMetaInfo& Attrs(std::vector&& attrs); - - // format: PD_KERNEL(...) - OpMetaInfo& SetKernelFn(KernelFunc&& func); - - // format: PD_INFER_SHAPE(...) - OpMetaInfo& SetInferShapeFn(InferShapeFunc&& func); - - // format: PD_INFER_DTYPE(...) - OpMetaInfo& SetInferDtypeFn(InferDtypeFunc&& func); - - private: - friend class framework::OpMetaInfoHelper; - - // 1. desc info - std::string name_; - std::vector inputs_; - std::vector outputs_; - std::vector attrs_; - - // 2. func info - KernelFunc kernel_fn_; - InferShapeFunc infer_shape_fn_; - InferDtypeFunc infer_dtype_fn_; -}; - -//////////////// Op Meta Info Map ///////////////// - -class PD_DLL_DECL OpMetaInfoMap { - public: - // this function's impl should keep in header file. - // if move to cc file, meta info can not be added - // into map - static OpMetaInfoMap& Instance() { - static OpMetaInfoMap g_custom_op_meta_info_map; - return g_custom_op_meta_info_map; - } - - std::vector& operator[](const std::string& name); - - const std::unordered_map>& GetMap() - const; - - private: - OpMetaInfoMap() = default; - std::unordered_map> map_; - - PD_DISABLE_COPY_AND_ASSIGN(OpMetaInfoMap); -}; - -//////////////// Op Meta Info Builder ///////////////// - -class PD_DLL_DECL OpMetaInfoBuilder { - public: - explicit OpMetaInfoBuilder(std::string&& name); - OpMetaInfoBuilder& Inputs(std::vector&& inputs); - OpMetaInfoBuilder& Outputs(std::vector&& outputs); - OpMetaInfoBuilder& Attrs(std::vector&& attrs); - OpMetaInfoBuilder& SetKernelFn(KernelFunc func); - OpMetaInfoBuilder& SetInferShapeFn(InferShapeFunc func); - OpMetaInfoBuilder& SetInferDtypeFn(InferDtypeFunc func); - OpMetaInfoBuilder& SetBackwardOp(const std::string& bwd_op_name); - - private: - // Forward Op name - std::string name_; - // Point to the currently constructed op meta info - OpMetaInfo* info_ptr_; -}; - -/////////////////////// Op register API ///////////////////////// - -// For inference: compile directly with framework -// Call after PD_BUILD_OP(...) -void RegisterAllCustomOperator(); - -// Using this api to load compiled custom operator's dynamic library and -// register Custom -// Operator into it -void LoadCustomOperatorLib(const std::string& dso_name); - -/////////////////////// Op register Macro ///////////////////////// - -#define PD_BUILD_OP_WITH_COUNTER(op_name, counter) \ - static ::paddle::OpMetaInfoBuilder __op_meta_info_##counter##__ = \ - ::paddle::OpMetaInfoBuilder(op_name) - -#define PD_BUILD_OP_INNER(op_name, counter) \ - PD_BUILD_OP_WITH_COUNTER(op_name, counter) - -#define PD_BUILD_OP(op_name) PD_BUILD_OP_INNER(op_name, __COUNTER__) - -} // namespace paddle - -///////////////////// C API /////////////////// - -#ifdef __cplusplus -extern "C" { -#endif - -#if defined(_WIN32) -// C-API to get global OpMetaInfoMap. -__declspec(dllexport) inline paddle::OpMetaInfoMap& PD_GetOpMetaInfoMap() { - return paddle::OpMetaInfoMap::Instance(); -} -#endif // _WIN32 - -#ifdef __cplusplus -} -#endif diff --git a/paddle/fluid/extension/src/op_meta_info.cc b/paddle/fluid/extension/src/ext_op_meta_info.cc similarity index 71% rename from paddle/fluid/extension/src/op_meta_info.cc rename to paddle/fluid/extension/src/ext_op_meta_info.cc index d362282b8d9d2..40cad7a155226 100644 --- a/paddle/fluid/extension/src/op_meta_info.cc +++ b/paddle/fluid/extension/src/ext_op_meta_info.cc @@ -12,13 +12,14 @@ 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 "paddle/fluid/extension/include/op_meta_info.h" +#include "paddle/fluid/extension/include/ext_op_meta_info.h" #include #include #include #include "paddle/fluid/framework/custom_operator.h" +#include "paddle/fluid/platform/enforce.h" namespace paddle { @@ -62,11 +63,38 @@ OpMetaInfoMap::GetMap() const { //////////////// Op Meta Info Builder ///////////////// -OpMetaInfoBuilder::OpMetaInfoBuilder(std::string&& name) { +OpMetaInfoBuilder::OpMetaInfoBuilder(std::string&& name, size_t index) { + // 1. member assign name_ = std::forward(name); + index_ = index; + + // 2. check and meta info build auto& info_vector = OpMetaInfoMap::Instance()[name_]; + // index check + PADDLE_ENFORCE_EQ( + info_vector.size(), index_, + platform::errors::PreconditionNotMet( + "The operator %s's meta info register failed. " + "Please make sure you call marcos as order `PD_BUILD_OP`, " + "`PD_BUILD_GRAD_OP`, `PD_BUILD_DOUBLE_GRAD_OP`.", + name_)); + switch (index_) { + case 0: + break; + case 1: + name_ = name_ + "_grad"; + break; + case 2: + name_ = name_ + "_grad_grad"; + default: + PADDLE_THROW(platform::errors::InvalidArgument( + "Not support index `%d` when construct OpMetaInfoBuilder, " + "now only support `0, 1, 2`.", + index_)); + } auto op_meta = OpMetaInfo(name_); info_vector.emplace_back(std::move(op_meta)); + // 3. get current info ptr info_ptr_ = &(info_vector.back()); } @@ -93,24 +121,27 @@ OpMetaInfoBuilder& OpMetaInfoBuilder::SetKernelFn(KernelFunc func) { } OpMetaInfoBuilder& OpMetaInfoBuilder::SetInferShapeFn(InferShapeFunc func) { + PADDLE_ENFORCE_EQ( + index_, 0UL, + platform::errors::Unimplemented( + "Currently, the InferShapeFn setting of Grad Op is not supported, " + "And backward Tensor `X@GRAD` will use the shape of forward Tensor " + "`X` by default.")); info_ptr_->SetInferShapeFn(std::forward(func)); return *this; } OpMetaInfoBuilder& OpMetaInfoBuilder::SetInferDtypeFn(InferDtypeFunc func) { + PADDLE_ENFORCE_EQ( + index_, 0UL, + platform::errors::Unimplemented( + "Currently, the InferDtypeFn setting of Grad Op is not supported, " + "And backward Tensor `X@GRAD` will use the dtype of forward Tensor " + "`X` by default.")); info_ptr_->SetInferDtypeFn(std::forward(func)); return *this; } -OpMetaInfoBuilder& OpMetaInfoBuilder::SetBackwardOp( - const std::string& bwd_op_name) { - auto& info_vector = OpMetaInfoMap::Instance()[name_]; - auto op_meta = OpMetaInfo(bwd_op_name); - info_vector.emplace_back(std::move(op_meta)); - info_ptr_ = &(info_vector.back()); - return *this; -} - /////////////////////// Op register API ///////////////////////// void RegisterAllCustomOperator() { diff --git a/paddle/fluid/extension/src/tensor.cc b/paddle/fluid/extension/src/ext_tensor.cc similarity index 91% rename from paddle/fluid/extension/src/tensor.cc rename to paddle/fluid/extension/src/ext_tensor.cc index 39ed274864110..0cae8f4af7b97 100644 --- a/paddle/fluid/extension/src/tensor.cc +++ b/paddle/fluid/extension/src/ext_tensor.cc @@ -12,12 +12,17 @@ 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 "paddle/fluid/extension/include/tensor.h" +#include "paddle/fluid/extension/include/ext_tensor.h" + #include + #include "paddle/fluid/framework/custom_tensor_utils.h" #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/memory/memcpy.h" +#include "paddle/fluid/platform/complex128.h" +#include "paddle/fluid/platform/complex64.h" #include "paddle/fluid/platform/enforce.h" +#include "paddle/fluid/platform/float16.h" #include "paddle/fluid/platform/transform.h" namespace paddle { @@ -95,14 +100,15 @@ void GpuCopy(T *src, T *dst, PlaceType src_plc, PlaceType dst_plc, } \ auto *tensor = static_cast(tensor_.get()); -void Tensor::reshape(const std::vector &shape) { +void Tensor::reshape(const std::vector &shape) { GET_CASTED_TENSOR tensor->Resize(framework::make_ddim(shape)); } Tensor::Tensor(const PlaceType &place) - : tensor_(std::make_shared()), place_(place) {} - + : tensor_(std::make_shared()), + place_(place), + stream_(StreamWrapper()) {} template T *Tensor::mutable_data(const PlaceType &place) { place_ = place; @@ -159,17 +165,16 @@ DataType Tensor::type() const { return DataType::UINT8; } else if (type == framework::proto::VarType::FP64) { return DataType::FLOAT64; - } else if (type == framework::proto::VarType::BF16) { - return DataType::BFLOAT16; - } else if (type == framework::proto::VarType::FP16) { - return DataType::FLOAT16; + } else if (type == framework::proto::VarType::BOOL) { + return DataType::BOOL; } else if (type == framework::proto::VarType::COMPLEX64) { return DataType::COMPLEX64; } else if (type == framework::proto::VarType::COMPLEX128) { return DataType::COMPLEX128; - } else if (type == framework::proto::VarType::BOOL) { - return DataType::BOOL; + } else if (type == framework::proto::VarType::FP16) { + return DataType::FLOAT16; } + // TODO(JiabinYang) Support more dtype here return DataType::FLOAT32; } @@ -207,14 +212,6 @@ Tensor Tensor::copy_to(const PlaceType &target_place) const { return target; } -template PD_DLL_DECL Tensor -Tensor::copy_to(const PlaceType &target_place) const; -template PD_DLL_DECL Tensor Tensor::copy_to( - const PlaceType &target_place) const; -template PD_DLL_DECL Tensor Tensor::copy_to( - const PlaceType &target_place) const; -template PD_DLL_DECL Tensor Tensor::copy_to( - const PlaceType &target_place) const; template PD_DLL_DECL Tensor Tensor::copy_to(const PlaceType &target_place) const; template PD_DLL_DECL Tensor @@ -231,6 +228,12 @@ template PD_DLL_DECL Tensor Tensor::copy_to(const PlaceType &target_place) const; template PD_DLL_DECL Tensor Tensor::copy_to(const PlaceType &target_place) const; +template PD_DLL_DECL Tensor Tensor::copy_to( + const PlaceType &target_place) const; +template PD_DLL_DECL Tensor Tensor::copy_to( + const PlaceType &target_place) const; +template PD_DLL_DECL Tensor +Tensor::copy_to(const PlaceType &target_place) const; template PD_DLL_DECL float *Tensor::data() const; template PD_DLL_DECL double *Tensor::data() const; @@ -238,16 +241,14 @@ template PD_DLL_DECL int64_t *Tensor::data() const; template PD_DLL_DECL int32_t *Tensor::data() const; template PD_DLL_DECL uint8_t *Tensor::data() const; template PD_DLL_DECL int8_t *Tensor::data() const; -template PD_DLL_DECL paddle::platform::float16 * -Tensor::data() const; -template PD_DLL_DECL paddle::platform::bfloat16 * -Tensor::data() const; -template PD_DLL_DECL paddle::platform::complex128 * -Tensor::data() const; -template PD_DLL_DECL paddle::platform::complex64 * -Tensor::data() const; template PD_DLL_DECL int16_t *Tensor::data() const; template PD_DLL_DECL bool *Tensor::data() const; +template PD_DLL_DECL paddle::platform::complex64 * +Tensor::data() const; +template PD_DLL_DECL paddle::platform::complex128 * +Tensor::data() const; +template PD_DLL_DECL paddle::platform::float16 * +Tensor::data() const; template PD_DLL_DECL float *Tensor::mutable_data(); template PD_DLL_DECL double *Tensor::mutable_data(); @@ -255,16 +256,14 @@ template PD_DLL_DECL int64_t *Tensor::mutable_data(); template PD_DLL_DECL int32_t *Tensor::mutable_data(); template PD_DLL_DECL uint8_t *Tensor::mutable_data(); template PD_DLL_DECL int8_t *Tensor::mutable_data(); -template PD_DLL_DECL paddle::platform::float16 * -Tensor::mutable_data(); -template PD_DLL_DECL paddle::platform::bfloat16 * -Tensor::mutable_data(); -template PD_DLL_DECL paddle::platform::complex128 * -Tensor::mutable_data(); -template PD_DLL_DECL paddle::platform::complex64 * -Tensor::mutable_data(); template PD_DLL_DECL int16_t *Tensor::mutable_data(); template PD_DLL_DECL bool *Tensor::mutable_data(); +template PD_DLL_DECL paddle::platform::complex64 * +Tensor::mutable_data(); +template PD_DLL_DECL paddle::platform::complex128 * +Tensor::mutable_data(); +template PD_DLL_DECL paddle::platform::float16 * +Tensor::mutable_data(); template PD_DLL_DECL float *Tensor::mutable_data(const PlaceType &place); template PD_DLL_DECL double *Tensor::mutable_data( @@ -277,21 +276,19 @@ template PD_DLL_DECL uint8_t *Tensor::mutable_data( const PlaceType &place); template PD_DLL_DECL int8_t *Tensor::mutable_data( const PlaceType &place); -template PD_DLL_DECL paddle::platform::float16 * -Tensor::mutable_data(const PlaceType &place); -template PD_DLL_DECL paddle::platform::bfloat16 * -Tensor::mutable_data(const PlaceType &place); -template PD_DLL_DECL paddle::platform::complex128 * -Tensor::mutable_data(const PlaceType &place); -template PD_DLL_DECL paddle::platform::complex64 * -Tensor::mutable_data(const PlaceType &place); template PD_DLL_DECL int16_t *Tensor::mutable_data( const PlaceType &place); template PD_DLL_DECL bool *Tensor::mutable_data(const PlaceType &place); +template PD_DLL_DECL paddle::platform::complex64 * +Tensor::mutable_data(const PlaceType &place); +template PD_DLL_DECL paddle::platform::complex128 * +Tensor::mutable_data(const PlaceType &place); +template PD_DLL_DECL paddle::platform::float16 * +Tensor::mutable_data(const PlaceType &place); -std::vector Tensor::shape() const { +std::vector Tensor::shape() const { GET_CASTED_TENSOR - return framework::vectorize(tensor->dims()); + return framework::vectorize(tensor->dims()); } const PlaceType &Tensor::place() const { @@ -320,14 +317,6 @@ Tensor Tensor::cast(const DataType &target_type) const { auto dst_type = framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType(target_type); switch (src_type) { - case framework::proto::VarType::FP16: - framework::VisitDataType( - dst_type, CastDataType(*tensor, rlt_tensor_, ctx)); - break; - case framework::proto::VarType::BF16: - framework::VisitDataType(dst_type, CastDataType( - *tensor, rlt_tensor_, ctx)); - break; case framework::proto::VarType::FP32: framework::VisitDataType(dst_type, CastDataType(*tensor, rlt_tensor_, ctx)); @@ -357,13 +346,21 @@ Tensor Tensor::cast(const DataType &target_type) const { dst_type, CastDataType(*tensor, rlt_tensor_, ctx)); break; case framework::proto::VarType::COMPLEX64: - framework::VisitDataType(dst_type, CastDataType( - *tensor, rlt_tensor_, ctx)); + framework::VisitDataType( + dst_type, + CastDataType(*tensor, rlt_tensor_, ctx)); break; case framework::proto::VarType::COMPLEX128: - framework::VisitDataType(dst_type, CastDataType( - *tensor, rlt_tensor_, ctx)); + framework::VisitDataType(dst_type, + CastDataType( + *tensor, rlt_tensor_, ctx)); break; + case framework::proto::VarType::FP16: + framework::VisitDataType( + dst_type, + CastDataType(*tensor, rlt_tensor_, ctx)); + break; + // TODO(JiabinYang) Support more dtype here default: PADDLE_THROW(platform::errors::Unimplemented( "Data type (%s) is not supported when casting data type.", @@ -377,6 +374,18 @@ int64_t Tensor::size() const { return tensor->numel(); } +#ifdef PADDLE_WITH_CUDA +cudaStream_t Tensor::stream() const { + if (!stream_.IsStreamSet()) { + PADDLE_THROW(platform::errors::PreconditionNotMet( + "Stream is not Set, only input tensor will have " + "stream which is set by framework ")); + } else { + return reinterpret_cast(stream_.GetStream()); + } +} +#endif + namespace framework { void CustomTensorUtils::ShareDataTo(const paddle::Tensor &src, void *dst) { diff --git a/paddle/fluid/framework/CMakeLists.txt b/paddle/fluid/framework/CMakeLists.txt index 6e282a2e91c47..1fa4ce9b573a0 100644 --- a/paddle/fluid/framework/CMakeLists.txt +++ b/paddle/fluid/framework/CMakeLists.txt @@ -34,7 +34,11 @@ proto_library(trainer_desc_proto SRCS trainer_desc.proto DEPS framework_proto cc_library(ddim SRCS ddim.cc DEPS eigen3 boost enforce) cc_test(ddim_test SRCS ddim_test.cc DEPS ddim) -nv_test(dim_test SRCS dim_test.cu DEPS ddim) +if(WITH_GPU) + nv_test(dim_test SRCS dim_test.cu DEPS ddim) +elseif(WITH_ROCM) + hip_test(dim_test SRCS dim_test.cu DEPS ddim) +endif() cc_test(unroll_array_ops_test SRCS unroll_array_ops_test.cc) cc_library(data_type SRCS data_type.cc DEPS framework_proto ddim device_context) cc_test(data_type_test SRCS data_type_test.cc DEPS data_type place tensor) @@ -46,6 +50,8 @@ if(WITH_GPU) else() nv_library(tensor SRCS tensor.cc tensor_util.cu DEPS place memory data_type device_context profiler) endif(WIN32) +elseif(WITH_ROCM) + hip_library(tensor SRCS tensor.cc tensor_util.cu DEPS place memory data_type device_context profiler) else() cc_library(tensor SRCS tensor.cc tensor_util.cc DEPS place memory data_type device_context profiler) endif() @@ -53,6 +59,8 @@ endif() cc_test(tensor_test SRCS tensor_test.cc DEPS tensor) if(WITH_GPU) nv_test(tensor_util_test SRCS tensor_util_test.cc tensor_util_test.cu DEPS tensor dlpack_tensor) +elseif(WITH_ROCM) + hip_test(tensor_util_test SRCS tensor_util_test.cc tensor_util_test.cu DEPS tensor dlpack_tensor) else() cc_test(tensor_util_test SRCS tensor_util_test.cc DEPS tensor dlpack_tensor) endif() @@ -63,13 +71,20 @@ cc_test(eigen_test SRCS eigen_test.cc DEPS tensor) if(WITH_GPU) nv_test(mixed_vector_test SRCS mixed_vector_test.cc mixed_vector_test.cu DEPS place memory device_context tensor) +elseif(WITH_ROCM) + hip_test(mixed_vector_test SRCS mixed_vector_test.cc mixed_vector_test.cu DEPS place memory device_context tensor) else() cc_test(mixed_vector_test SRCS mixed_vector_test.cc DEPS place memory device_context tensor) endif() cc_library(lod_tensor SRCS lod_tensor.cc DEPS ddim place tensor framework_proto version) cc_test(lod_tensor_test SRCS lod_tensor_test.cc DEPS lod_tensor memory) -nv_test(lod_tensor_gpu_test SRCS lod_tensor_test.cu DEPS lod_tensor) + +if(WITH_GPU) + nv_test(lod_tensor_gpu_test SRCS lod_tensor_test.cu DEPS lod_tensor) +elseif(WITH_ROCM) + hip_test(lod_tensor_gpu_test SRCS lod_tensor_test.cu DEPS lod_tensor) +endif() cc_library(garbage_collector SRCS garbage_collector.cc DEPS device_context memory gflags glog) @@ -94,8 +109,13 @@ cc_test(scope_test SRCS scope_test.cc DEPS scope) cc_test(variable_test SRCS variable_test.cc DEPS tensor var_type_traits) cc_library(data_device_transform SRCS data_device_transform.cc DEPS tensor) -nv_test(data_device_transform_test SRCS data_device_transform_test.cu +if(WITH_GPU) + nv_test(data_device_transform_test SRCS data_device_transform_test.cu + DEPS operator op_registry device_context math_function scope) +elseif(WITH_ROCM) + hip_test(data_device_transform_test SRCS data_device_transform_test.cu DEPS operator op_registry device_context math_function scope) +endif() if(WITH_GPU) if (WIN32) @@ -108,6 +128,9 @@ if(WITH_GPU) nv_library(data_type_transform SRCS data_type_transform.cu DEPS tensor) endif(WIN32) nv_test(data_type_transform_test SRCS data_type_transform_test.cc data_type_transform_test.cu DEPS data_type_transform) +elseif(WITH_ROCM) + hip_library(data_type_transform SRCS data_type_transform.cu DEPS tensor) + hip_test(data_type_transform_test SRCS data_type_transform_test.cc data_type_transform_test.cu DEPS data_type_transform) else() cc_library(data_type_transform SRCS data_type_transform.cc DEPS tensor) cc_test(data_type_transform_test SRCS data_type_transform_test.cc DEPS data_type_transform) @@ -156,8 +179,11 @@ cc_library(op_registry SRCS op_registry.cc DEPS op_proto_maker op_info operator cc_library(op_call_stack SRCS op_call_stack.cc DEPS op_proto_maker enforce) cc_test(op_call_stack_test SRCS op_call_stack_test.cc DEPS op_call_stack) - -nv_test(op_registry_test SRCS op_registry_test.cc DEPS op_registry) +if(WITH_GPU) + nv_test(op_registry_test SRCS op_registry_test.cc DEPS op_registry) +elseif(WITH_ROCM) + hip_test(op_registry_test SRCS op_registry_test.cc DEPS op_registry) +endif() if(WITH_PYTHON) py_proto_compile(framework_py_proto SRCS framework.proto data_feed.proto) @@ -320,8 +346,13 @@ message(STATUS "branch: ${PADDLE_BRANCH}") configure_file(commit.h.in commit.h) -cc_library(custom_tensor SRCS ../extension/src/tensor.cc DEPS lod_tensor memory enforce) -cc_library(op_meta_info SRCS ../extension/src/op_meta_info.cc DEPS custom_tensor) +# Adapt to custom op mechanism: Include the header files related to the data type +# to avoid exposing the path of the underlying file +include_directories(${PADDLE_SOURCE_DIR}/paddle/fluid/platform) +include_directories(${CMAKE_CURRENT_SOURCE_DIR}/../extension/include) + +cc_library(custom_tensor SRCS ../extension/src/ext_tensor.cc DEPS lod_tensor memory enforce) +cc_library(op_meta_info SRCS ../extension/src/ext_op_meta_info.cc DEPS custom_tensor) cc_library(custom_operator SRCS custom_operator.cc DEPS tensor attribute framework_proto op_registry operator dynamic_loader string_helper custom_tensor op_meta_info) cc_test(custom_tensor_test SRCS custom_tensor_test.cc DEPS custom_tensor glog) @@ -346,11 +377,16 @@ if (LINUX) endif() if (WIN32) + if("${CMAKE_GENERATOR}" STREQUAL "Ninja") + set(paddle_framework_lib_path ${CMAKE_CURRENT_BINARY_DIR}) + else() + set(paddle_framework_lib_path ${CMAKE_CURRENT_BINARY_DIR}/${CMAKE_BUILD_TYPE}) + endif() set(FLUID_FRAMEWORK_IMPORT_LIB - ${PADDLE_BINARY_DIR}/paddle/fluid/framework/${CMAKE_BUILD_TYPE}/paddle_framework.lib - CACHE INTERNAL "Fluid framework lib") + ${paddle_framework_lib_path}/paddle_framework.lib + CACHE INTERNAL "Fluid framework lib") set(FLUID_FRAMEWORK_SHARED_LIB - ${PADDLE_BINARY_DIR}/paddle/fluid/framework/${CMAKE_BUILD_TYPE}/paddle_framework.dll + ${paddle_framework_lib_path}/paddle_framework.dll CACHE INTERNAL "Fluid framework dll") endif() @@ -368,10 +404,16 @@ endif() # if not deps `layer`, will cause: undefined symbol: _ZN6paddle10imperative7VarBase9name_set_ set(PADDLE_CUSTOM_OP_MODULES custom_tensor op_meta_info custom_operator layer) +set(PADDLE_CUSTOM_OP_SRCS + ${CMAKE_CURRENT_SOURCE_DIR}/custom_operator.cc + ${CMAKE_CURRENT_SOURCE_DIR}/../extension/src/ext_tensor.cc + ${CMAKE_CURRENT_SOURCE_DIR}/../extension/src/ext_op_meta_info.cc + ${CMAKE_SOURCE_DIR}/paddle/fluid/imperative/layer.cc) +set(PADDLE_CUSTOM_OP_SRCS ${PADDLE_CUSTOM_OP_SRCS} PARENT_SCOPE) + cc_library(paddle_custom_op_shared - SHARED SRCS custom_operator.cc ../extension/src/tensor.cc ../extension/src/op_meta_info.cc - ${CMAKE_SOURCE_DIR}/paddle/fluid/imperative/layer.cc - DEPS ${PADDLE_CUSTOM_OP_MODULES}) + SHARED SRCS ${PADDLE_CUSTOM_OP_SRCS} DEPS ${PADDLE_CUSTOM_OP_MODULES}) + get_property(os_dependency_modules GLOBAL PROPERTY OS_DEPENDENCY_MODULES) set_target_properties(paddle_custom_op_shared PROPERTIES OUTPUT_NAME paddle_custom_op) target_link_libraries(paddle_custom_op_shared ${os_dependency_modules}) @@ -383,11 +425,16 @@ if (LINUX) endif() if (WIN32) + if("${CMAKE_GENERATOR}" STREQUAL "Ninja") + set(paddle_custom_op_lib_path ${CMAKE_CURRENT_BINARY_DIR}) + else() + set(paddle_custom_op_lib_path ${CMAKE_CURRENT_BINARY_DIR}/${CMAKE_BUILD_TYPE}) + endif() set(PADDLE_CUSTOM_OP_IMPORT_LIB - ${PADDLE_BINARY_DIR}/paddle/fluid/framework/${CMAKE_BUILD_TYPE}/paddle_custom_op.lib + ${paddle_custom_op_lib_path}/paddle_custom_op.lib CACHE INTERNAL "Paddle custom op import lib") set(PADDLE_CUSTOM_OP_SHARED_LIB - ${PADDLE_BINARY_DIR}/paddle/fluid/framework/${CMAKE_BUILD_TYPE}/paddle_custom_op.dll + ${paddle_custom_op_lib_path}/paddle_custom_op.dll CACHE INTERNAL "Paddle custom op dll") endif() diff --git a/paddle/fluid/framework/array.h b/paddle/fluid/framework/array.h index 10abb83116624..0ec9cb81129c2 100644 --- a/paddle/fluid/framework/array.h +++ b/paddle/fluid/framework/array.h @@ -54,7 +54,7 @@ class Array { } HOSTDEVICE inline T &at(size_t i) { -#ifndef __CUDA_ARCH__ +#if !defined(__CUDA_ARCH__) && !defined(__HIPCC__) PADDLE_ENFORCE_LT( i, N, platform::errors::OutOfRange("Array index out of bounds.")); #endif @@ -62,7 +62,7 @@ class Array { } HOSTDEVICE inline const T &at(size_t i) const { -#ifndef __CUDA_ARCH__ +#if !defined(__CUDA_ARCH__) && !defined(__HIPCC__) PADDLE_ENFORCE_LT( i, N, platform::errors::OutOfRange("Array index out of bounds.")); #endif @@ -103,7 +103,12 @@ class Array { HOSTDEVICE inline T *GetMutable() { return nullptr; } HOSTDEVICE inline T &operator[](size_t) { -#ifdef __CUDA_ARCH__ +#if defined(__HIPCC__) + // HIP will have compile error, if use "obj()" + // function declared in block scope cannot have 'static' storage class + static T obj{}; + return obj; +#elif defined(__CUDA_ARCH__) static T obj(); return obj; #else @@ -112,7 +117,12 @@ class Array { } HOSTDEVICE inline const T &operator[](size_t) const { -#ifdef __CUDA_ARCH__ +#if defined(__HIPCC__) + // HIP will have compile error, if use "obj()" + // function declared in block scope cannot have 'static' storage class + static const T obj{}; + return obj; +#elif defined(__CUDA_ARCH__) static const T obj(); return obj; #else diff --git a/paddle/fluid/framework/conv_search_cache.h b/paddle/fluid/framework/conv_search_cache.h index 720467d6f1cda..db8dc22f68663 100644 --- a/paddle/fluid/framework/conv_search_cache.h +++ b/paddle/fluid/framework/conv_search_cache.h @@ -16,7 +16,12 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/operator_kernel_configs.h" + +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif namespace paddle { namespace framework { @@ -32,7 +37,20 @@ class ConvSearchCache { static ConvSearchCache instance; return instance; } - +#ifdef PADDLE_WITH_HIP + AlgorithmsCache* GetForward() { + return &forward_cache_; + } + AlgorithmsCache* GetBackwardData() { + return &backward_data_cache_; + } + AlgorithmsCache* GetBackwardFilter() { + return &backward_filter_cache_; + } + AlgorithmsCache* GetConvFusion() { + return &fusion_forward_cache_; + } +#else AlgorithmsCache* GetForward() { return &forward_cache_; } @@ -45,6 +63,7 @@ class ConvSearchCache { AlgorithmsCache* GetConvFusion() { return &fusion_forward_cache_; } +#endif private: ConvSearchCache() {} @@ -52,10 +71,17 @@ class ConvSearchCache { ConvSearchCache(const ConvSearchCache&) {} ConvSearchCache& operator=(const ConvSearchCache&) {} +#ifdef PADDLE_WITH_HIP + AlgorithmsCache forward_cache_; + AlgorithmsCache backward_data_cache_; + AlgorithmsCache backward_filter_cache_; + AlgorithmsCache fusion_forward_cache_; +#else AlgorithmsCache forward_cache_; AlgorithmsCache backward_data_cache_; AlgorithmsCache backward_filter_cache_; AlgorithmsCache fusion_forward_cache_; +#endif }; } // namespace framework diff --git a/paddle/fluid/framework/copy_same_tensor_test.cc b/paddle/fluid/framework/copy_same_tensor_test.cc index ad06473b519cd..0b1fdc3944689 100644 --- a/paddle/fluid/framework/copy_same_tensor_test.cc +++ b/paddle/fluid/framework/copy_same_tensor_test.cc @@ -31,7 +31,7 @@ namespace framework { static std::vector CreatePlaceList() { std::vector places; places.emplace_back(platform::CPUPlace()); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) places.emplace_back(platform::CUDAPlace(0)); #endif return places; diff --git a/paddle/fluid/framework/custom_operator.cc b/paddle/fluid/framework/custom_operator.cc index 03a8cc366e7f2..1ebb8998c854e 100644 --- a/paddle/fluid/framework/custom_operator.cc +++ b/paddle/fluid/framework/custom_operator.cc @@ -25,9 +25,8 @@ limitations under the License. */ #include #include -#include "paddle/fluid/extension/include/tensor.h" +#include "paddle/fluid/extension/include/ext_tensor.h" #include "paddle/fluid/framework/attribute.h" -#include "paddle/fluid/framework/c/c_api.h" #include "paddle/fluid/framework/custom_tensor_utils.h" #include "paddle/fluid/framework/framework.pb.h" #include "paddle/fluid/framework/op_meta_info_helper.h" @@ -63,6 +62,11 @@ inline bool IsGradVar(const std::string& var_name) { return var_name.rfind(suffix) != std::string::npos; } +inline bool IsDuplicableVar(const std::string& var_name) { + std::string suffix = kTensorVectorSuffix; + return var_name.rfind(suffix) != std::string::npos; +} + inline std::string NoGrad(const std::string& var_name) { std::string suffix = kGradVarSuffix; return var_name.substr(0, var_name.size() - kGradVarSuffixSize); @@ -103,18 +107,47 @@ static void RunKernelFunc(const framework::ExecutionContext& ctx, const std::vector& attrs) { VLOG(1) << "Custom Operator: Start run KernelFunc."; std::vector custom_ins; + std::vector> custom_vec_ins; for (auto& in_name : inputs) { VLOG(1) << "Custom Operator: input name - " << in_name; - auto* x = ctx.Input(in_name); - PADDLE_ENFORCE_NOT_NULL(x, platform::errors::NotFound( - "Input tensor (%s) is nullptr.", in_name)); - PADDLE_ENFORCE_EQ(x->IsInitialized(), true, - platform::errors::InvalidArgument( - "Input tensor (%s) is not initialized.")); - auto custom_in = paddle::Tensor( - CustomTensorUtils::ConvertInnerPlaceToEnumPlace(x->place())); - CustomTensorUtils::ShareDataFrom(static_cast(x), custom_in); - custom_ins.emplace_back(custom_in); + if (detail::IsDuplicableVar(in_name)) { + // return const std::vector + auto vec_x = ctx.MultiInput(in_name); + PADDLE_ENFORCE_NE(vec_x.empty(), true, + platform::errors::NotFound( + "Input vector (%s) is empty.", in_name)); + std::vector custom_vec_in; + for (size_t i = 0; i < vec_x.size(); ++i) { + auto* x = vec_x[i]; + PADDLE_ENFORCE_NOT_NULL( + x, platform::errors::NotFound( + "The %d-th tensor in input vector (%s) is nullptr.", + i, in_name)); + PADDLE_ENFORCE_EQ(x->IsInitialized(), true, + platform::errors::InvalidArgument( + "The %d-th tensor in input vector (%s) " + "is not initialized.", + i, in_name)); + auto custom_t = paddle::Tensor( + CustomTensorUtils::ConvertInnerPlaceToEnumPlace(x->place())); + CustomTensorUtils::ShareDataFrom(static_cast(x), custom_t); + CustomTensorUtils::SetTensorCurrentStream(&custom_t, ctx.GetPlace()); + custom_vec_in.emplace_back(custom_t); + } + custom_vec_ins.emplace_back(custom_vec_in); + } else { + auto* x = ctx.Input(in_name); + PADDLE_ENFORCE_NOT_NULL(x, platform::errors::NotFound( + "Input tensor (%s) is nullptr.", in_name)); + PADDLE_ENFORCE_EQ(x->IsInitialized(), true, + platform::errors::InvalidArgument( + "Input tensor (%s) is not initialized.", in_name)); + auto custom_in = paddle::Tensor( + CustomTensorUtils::ConvertInnerPlaceToEnumPlace(x->place())); + CustomTensorUtils::ShareDataFrom(static_cast(x), custom_in); + CustomTensorUtils::SetTensorCurrentStream(&custom_in, ctx.GetPlace()); + custom_ins.emplace_back(custom_in); + } } std::vector custom_attrs; @@ -145,20 +178,49 @@ static void RunKernelFunc(const framework::ExecutionContext& ctx, "Unsupported `%s` type value as custom attribute now. " "Supported data types include `bool`, `int`, `float`, " "`int64_t`, `std::string`, `std::vector`, " - "`std::vector`, `std::vector, " + "`std::vector`, `std::vector`, " "`std::vector`, Please check whether " "the attribute data type and data type string are matched.", attr_type_str)); } } - VLOG(1) << "Run ComputeFunc."; - auto outs = func(custom_ins, custom_attrs); - - VLOG(1) << "Custom Operator: Share outputs into ExecutionContext."; - for (size_t i = 0; i < outputs.size(); ++i) { - auto* true_out = ctx.Output(outputs[i]); - CustomTensorUtils::ShareDataTo(outs.at(i), true_out); + VLOG(1) << "Custom Operator: Run ComputeFunc."; + try { + auto outs = func(custom_ins, custom_vec_ins, custom_attrs); + + VLOG(1) << "Custom Operator: Share outputs into ExecutionContext."; + for (size_t i = 0; i < outputs.size(); ++i) { + auto out_name = outputs[i]; + if (detail::IsDuplicableVar(out_name)) { + PADDLE_ENFORCE(i == 0UL && outputs.size() == 1UL, + platform::errors::PreconditionNotMet( + "If custom operator's outputs contains `paddle::Vec(" + ")` type, " + "it only can hold one output.")); + auto vec_true_outs = ctx.MultiOutput(out_name); + PADDLE_ENFORCE_EQ( + vec_true_outs.size(), outs.size(), + platform::errors::InvalidArgument( + "The number of element in custom operator outputs is wrong, " + "expected contains %d Tensors, but actually contains %d " + "Tensors.", + vec_true_outs.size(), outs.size())); + for (size_t j = 0; j < vec_true_outs.size(); ++j) { + CustomTensorUtils::ShareDataTo(outs.at(j), vec_true_outs.at(j)); + } + } else { + auto* true_out = ctx.Output(out_name); + CustomTensorUtils::ShareDataTo(outs.at(i), true_out); + } + } + } catch (platform::EnforceNotMet& exception) { + throw std::move(exception); + } catch (std::exception& ex) { + PADDLE_THROW(platform::errors::External("%s", ex.what())); + } catch (...) { + PADDLE_THROW(platform::errors::Fatal( + "Custom operator raises an unknown exception in rumtime.")); } } @@ -211,10 +273,20 @@ class CustomOpMaker : public OpProtoAndCheckerMaker { void Make() override { for (auto& in_name : inputs_) { - AddInput(in_name, "The input " + in_name + "of Custom operator."); + if (detail::IsDuplicableVar(in_name)) { + AddInput(in_name, "The input " + in_name + "of Custom operator.") + .AsDuplicable(); + } else { + AddInput(in_name, "The input " + in_name + "of Custom operator."); + } } for (auto& out_name : outputs_) { - AddOutput(out_name, "The output " + out_name + "of Custom Operator."); + if (detail::IsDuplicableVar(out_name)) { + AddOutput(out_name, "The output " + out_name + "of Custom Operator.") + .AsDuplicable(); + } else { + AddOutput(out_name, "The output " + out_name + "of Custom Operator."); + } } for (auto& attr : attrs_) { auto attr_name_and_type = detail::ParseAttrStr(attr); @@ -255,7 +327,7 @@ class CustomOpMaker : public OpProtoAndCheckerMaker { "Unsupported `%s` type value as custom attribute now. " "Supported data types include `bool`, `int`, `float`, " "`int64_t`, `std::string`, `std::vector`, " - "`std::vector`, `std::vector, " + "`std::vector`, `std::vector`, " "`std::vector`, Please check whether " "the attribute data type and data type string are matched.", attr_type_str)); @@ -321,7 +393,13 @@ class CustomGradOpMaker : public SingleGradOpMaker { } for (auto& out_name : outputs_) { VLOG(1) << "Custom Operator: GradOpDescMaker - output: " << out_name; - grad_op->SetOutput(out_name, this->InputGrad(detail::NoGrad(out_name))); + if (detail::IsDuplicableVar(out_name)) { + grad_op->SetOutput(out_name, + this->InputGrad(detail::NoGrad(out_name), + /*drop_empty_grad=*/false)); + } else { + grad_op->SetOutput(out_name, this->InputGrad(detail::NoGrad(out_name))); + } } grad_op->SetAttrMap(this->Attrs()); } @@ -475,58 +553,195 @@ void RegisterOperatorWithMetaInfo( op_name, info.proto_->InitializationErrorString())); // InferShape - PADDLE_ENFORCE_NOT_NULL( - infer_shape_func, - platform::errors::PreconditionNotMet( - "InferShapeFn is nullptr. Need to set the InferShapeFn of custom " - "operator by .SetInferShapeFn(PD_INFER_SHAPE(...))")); - info.infer_shape_ = [op_inputs, op_outputs, - infer_shape_func](InferShapeContext* ctx) { - std::vector> input_shapes; - - VLOG(1) << "Custom Operator: InferShape - get input ddim."; - for (auto& in_name : op_inputs) { - OP_INOUT_CHECK(ctx->HasInput(in_name), "Input", in_name, "Custom"); - auto ddim = ctx->GetInputDim(in_name); - input_shapes.emplace_back(framework::vectorize(ddim)); - } + if (infer_shape_func == nullptr) { + // use default InferShape + info.infer_shape_ = [op_inputs, op_outputs](InferShapeContext* ctx) { + PADDLE_ENFORCE_EQ( + op_inputs.size(), 1UL, + platform::errors::Unavailable( + "Your custom operator contains multiple inputs. " + "We only allow a custom operator that contains only one input " + "and only one output without setting the InferShapeFn. " + "At this time, the input shape will be directly set to " + "the output shape.\n" + "Please set the InferShapeFn of custom " + "operator by .SetInferShapeFn(PD_INFER_SHAPE(...))")); + PADDLE_ENFORCE_EQ( + op_outputs.size(), 1UL, + platform::errors::Unavailable( + "Your custom operator contains multiple outputs. " + "We only allow a custom operator that contains only one input " + "and only one output without setting the InferShapeFn. " + "At this time, the input shape will be directly set to " + "the output shape.\n" + "Please set the InferShapeFn of custom " + "operator by .SetInferShapeFn(PD_INFER_SHAPE(...))")); + + VLOG(1) << "Custom Operator: Default InferShape - share ddim."; + ctx->ShareDim(op_inputs[0], op_outputs[0]); + }; + } else { + info.infer_shape_ = [op_inputs, op_outputs, op_attrs, + infer_shape_func](InferShapeContext* ctx) { + std::vector> input_shapes; + std::vector>> vec_input_shapes; + + VLOG(1) << "Custom Operator: InferShape - get input ddim."; + for (auto& in_name : op_inputs) { + if (detail::IsDuplicableVar(in_name)) { + OP_INOUT_CHECK(ctx->HasInputs(in_name), "Input", in_name, "Custom"); + auto vec_ddim = ctx->GetInputsDim(in_name); + std::vector> vec_shape; + vec_shape.reserve(vec_ddim.size()); + std::transform(vec_ddim.begin(), vec_ddim.end(), + std::back_inserter(vec_shape), + [&](const DDim& ddim) -> std::vector { + return framework::vectorize(ddim); + }); + vec_input_shapes.emplace_back(vec_shape); + } else { + OP_INOUT_CHECK(ctx->HasInput(in_name), "Input", in_name, "Custom"); + auto ddim = ctx->GetInputDim(in_name); + input_shapes.emplace_back(framework::vectorize(ddim)); + } + } - VLOG(1) << "Custom Operator: InferShape - calc output ddim."; - auto output_shapes = infer_shape_func(input_shapes); + std::vector custom_attrs; + for (auto& attr_str : op_attrs) { + auto attr_name_and_type = detail::ParseAttrStr(attr_str); + auto attr_name = attr_name_and_type[0]; + auto attr_type_str = attr_name_and_type[1]; + if (attr_type_str == "bool") { + custom_attrs.emplace_back(ctx->Attrs().Get(attr_name)); + } else if (attr_type_str == "int") { + custom_attrs.emplace_back(ctx->Attrs().Get(attr_name)); + } else if (attr_type_str == "float") { + custom_attrs.emplace_back(ctx->Attrs().Get(attr_name)); + } else if (attr_type_str == "int64_t") { + custom_attrs.emplace_back(ctx->Attrs().Get(attr_name)); + } else if (attr_type_str == "std::string") { + custom_attrs.emplace_back(ctx->Attrs().Get(attr_name)); + } else if (attr_type_str == "std::vector") { + custom_attrs.emplace_back( + ctx->Attrs().Get>(attr_name)); + } else if (attr_type_str == "std::vector") { + custom_attrs.emplace_back( + ctx->Attrs().Get>(attr_name)); + } else if (attr_type_str == "std::vector") { + // NOTE(chenweihang): InferShape can't support std::vector + // attr type, because the input type is std::vector, only + // can use one rule to parse std::vector parameter + continue; + } else if (attr_type_str == "std::vector") { + custom_attrs.emplace_back( + ctx->Attrs().Get>(attr_name)); + } else { + PADDLE_THROW(platform::errors::Unimplemented( + "Unsupported `%s` type value as custom attribute now. " + "Supported data types include `bool`, `int`, `float`, " + "`int64_t`, `std::string`, `std::vector`, " + "`std::vector`, `std::vector`, " + "Please check whether the attribute data type and " + "data type string are matched.", + attr_type_str)); + } + } - VLOG(1) << "Custom Operator: InferShape - set output ddim."; - for (size_t i = 0; i < op_outputs.size(); ++i) { - ctx->SetOutputDim(op_outputs[i], framework::make_ddim(output_shapes[i])); - } - }; + VLOG(1) << "Custom Operator: InferShape - calc output ddim."; + auto output_shapes = + infer_shape_func(input_shapes, vec_input_shapes, custom_attrs); + + VLOG(1) << "Custom Operator: InferShape - set output ddim."; + for (size_t i = 0; i < op_outputs.size(); ++i) { + auto out_name = op_outputs[i]; + if (detail::IsDuplicableVar(out_name)) { + std::vector vec_ddim; + vec_ddim.reserve(output_shapes.size()); + std::transform(output_shapes.begin(), output_shapes.end(), + std::back_inserter(vec_ddim), + [&](const std::vector& shape) -> DDim { + return framework::make_ddim(shape); + }); + ctx->SetOutputsDim(out_name, vec_ddim); + } else { + ctx->SetOutputDim(out_name, framework::make_ddim(output_shapes[i])); + } + } + }; + } // Infer Dtype - PADDLE_ENFORCE_NOT_NULL( - infer_dtype_func, - platform::errors::PreconditionNotMet( - "InferDtypeFn is nullptr. Need to set the InferDtypeFn of custom " - "operator by .SetInferDtypeFn(PD_INFER_DTYPE(...))")); - info.infer_var_type_ = [op_inputs, op_outputs, - infer_dtype_func](InferVarTypeContext* ctx) { - std::vector input_dtypes; - - VLOG(1) << "Custom Operator: InferDtype - get input dtype."; - for (auto& in_name : op_inputs) { - auto dtype = ctx->GetInputDataType(in_name); - input_dtypes.emplace_back( - CustomTensorUtils::ConvertInnerDTypeToEnumDType(dtype)); - } - - VLOG(1) << "Custom Operator: InferDtype - infer output dtype."; - auto output_dtypes = infer_dtype_func(input_dtypes); + if (infer_dtype_func == nullptr) { + // use defalut InferDtype + info.infer_var_type_ = [op_inputs, op_outputs](InferVarTypeContext* ctx) { + PADDLE_ENFORCE_EQ( + op_inputs.size(), 1UL, + platform::errors::Unavailable( + "Your custom operator contains multiple inputs. " + "We only allow a custom operator that contains only one input " + "and only one output without setting the InferDtypeFn. " + "At this time, the input dtype will be directly set to " + "the output dtype.\n" + "Please set the InferDtypeFn of custom " + "operator by .SetInferDtypeFn(PD_INFER_DTYPE(...))")); + PADDLE_ENFORCE_EQ( + op_outputs.size(), 1UL, + platform::errors::Unavailable( + "Your custom operator contains multiple outputs. " + "We only allow a custom operator that contains only one input " + "and only one output without setting the InferDtypeFn. " + "At this time, the input dtype will be directly set to " + "the output dtype.\n" + "Please set the InferDtypeFn of custom " + "operator by .SetInferDtypeFn(PD_INFER_DTYPE(...))")); + + VLOG(1) << "Custom Operator: InferDtype - share dtype."; + auto dtype = ctx->GetInputDataType(op_inputs[0]); + ctx->SetOutputDataType(op_outputs[0], dtype); + }; + } else { + info.infer_var_type_ = [op_inputs, op_outputs, + infer_dtype_func](InferVarTypeContext* ctx) { + std::vector input_dtypes; + std::vector> vec_input_dtypes; + + VLOG(1) << "Custom Operator: InferDtype - get input dtype."; + for (auto& in_name : op_inputs) { + if (detail::IsDuplicableVar(in_name)) { + std::vector vec_custom_dtype; + for (size_t i = 0; i < ctx->InputSize(in_name); ++i) { + auto dtype = ctx->GetInputDataType(in_name, i); + vec_custom_dtype.emplace_back( + CustomTensorUtils::ConvertInnerDTypeToEnumDType(dtype)); + } + vec_input_dtypes.emplace_back(vec_custom_dtype); + } else { + auto dtype = ctx->GetInputDataType(in_name); + input_dtypes.emplace_back( + CustomTensorUtils::ConvertInnerDTypeToEnumDType(dtype)); + } + } - VLOG(1) << "Custom Operator: InferDtype - set output dtype."; - for (size_t i = 0; i < op_outputs.size(); ++i) { - ctx->SetOutputDataType( - op_outputs[i], - CustomTensorUtils::ConvertEnumDTypeToInnerDType(output_dtypes[i])); - } - }; + VLOG(1) << "Custom Operator: InferDtype - infer output dtype."; + auto output_dtypes = infer_dtype_func(input_dtypes, vec_input_dtypes); + + VLOG(1) << "Custom Operator: InferDtype - set output dtype."; + for (size_t i = 0; i < op_outputs.size(); ++i) { + auto out_name = op_outputs[i]; + if (detail::IsDuplicableVar(out_name)) { + for (size_t j = 0; j < output_dtypes.size(); ++j) { + auto dtype = CustomTensorUtils::ConvertEnumDTypeToInnerDType( + output_dtypes[i]); + ctx->SetOutputDataType(out_name, dtype, j); + } + } else { + ctx->SetOutputDataType( + out_name, CustomTensorUtils::ConvertEnumDTypeToInnerDType( + output_dtypes[i])); + } + } + }; + } // Kernel func RegisterOperatorKernel(op_name, kernel_fn, op_inputs, op_outputs, op_attrs); @@ -584,10 +799,39 @@ void RegisterOperatorWithMetaInfo( return new CustomOperator(type, inputs, outputs, attrs); }; - // Grad InferShape (gradient's shape is same with forward input default) - grad_info.infer_shape_ = [grad_op_outputs](InferShapeContext* ctx) { + // Grad InferShape + grad_info.infer_shape_ = [grad_op_inputs, + grad_op_outputs](InferShapeContext* ctx) { + // 1. if forward input exists, gradient's shape is same with forward input + // default + // [Suitable for most situations] + // 2. if forward input not exists, and only contains one grad input and + // output, + // use grad input shape as grad output shape + // [Suitable for the situation that forward input is not used as + // backward input] + // TODO(chenweihang): support set grad op infershape func if needed for (auto& out_name : grad_op_outputs) { - ctx->ShareDim(detail::NoGrad(out_name), out_name); + auto fwd_name = detail::NoGrad(out_name); + if (detail::IsDuplicableVar(fwd_name)) { + // Duplicable forward var must as backward input + ctx->ShareDim(fwd_name, out_name); + } else { + if (ctx->HasInput(fwd_name)) { + ctx->ShareDim(fwd_name, out_name); + } else { + PADDLE_ENFORCE_EQ( + grad_op_inputs.size() == 1UL && grad_op_outputs.size() == 1UL, + true, + platform::errors::Unavailable( + "Custom grad operator infershape error. " + "If a custom grad operator contains only one input and " + "only one output, the input shape will be directly set to " + "the output shape. Otherwise, Please set the forward input " + "as the grad operator's input.")); + ctx->ShareDim(grad_op_inputs[0], out_name); + } + } } }; @@ -607,10 +851,6 @@ void RegisterOperatorWithMetaInfo( void RegisterOperatorWithMetaInfoMap( const paddle::OpMetaInfoMap& op_meta_info_map) { auto& meta_info_map = op_meta_info_map.GetMap(); - - PADDLE_ENFORCE_EQ(meta_info_map.empty(), false, - platform::errors::PreconditionNotMet( - "No custom operator that needs to be registered.")); VLOG(1) << "Custom Operator: size of op meta info map - " << meta_info_map.size(); // pair: {op_type, OpMetaInfo} diff --git a/paddle/fluid/framework/custom_operator.h b/paddle/fluid/framework/custom_operator.h index f2f97e5e5822a..117841f80cf47 100644 --- a/paddle/fluid/framework/custom_operator.h +++ b/paddle/fluid/framework/custom_operator.h @@ -16,7 +16,7 @@ limitations under the License. */ #include -#include "paddle/fluid/extension/include/op_meta_info.h" +#include "paddle/fluid/extension/include/ext_op_meta_info.h" namespace paddle { namespace framework { diff --git a/paddle/fluid/framework/custom_tensor_test.cc b/paddle/fluid/framework/custom_tensor_test.cc index 33b662454286f..8d6fd4efd5ae3 100644 --- a/paddle/fluid/framework/custom_tensor_test.cc +++ b/paddle/fluid/framework/custom_tensor_test.cc @@ -14,13 +14,13 @@ #include "glog/logging.h" #include "gtest/gtest.h" -#include "paddle/fluid/extension/include/all.h" +#include "paddle/fluid/extension/include/ext_all.h" #include "paddle/fluid/framework/custom_tensor_utils.h" #include "paddle/fluid/framework/lod_tensor.h" template paddle::Tensor InitCPUTensorForTest() { - std::vector tensor_shape{5, 5}; + std::vector tensor_shape{5, 5}; auto t1 = paddle::Tensor(paddle::PlaceType::kCPU); t1.reshape(tensor_shape); auto* p_data_ptr = t1.mutable_data(paddle::PlaceType::kCPU); @@ -54,7 +54,7 @@ void TestCopyTensor() { } void TestAPIPlace() { - std::vector tensor_shape = {5, 5}; + std::vector tensor_shape = {5, 5}; #ifdef PADDLE_WITH_CUDA auto t1 = paddle::Tensor(paddle::PlaceType::kGPU); t1.reshape(tensor_shape); @@ -68,7 +68,7 @@ void TestAPIPlace() { } void TestAPISizeAndShape() { - std::vector tensor_shape = {5, 5}; + std::vector tensor_shape = {5, 5}; auto t1 = paddle::Tensor(paddle::PlaceType::kCPU); t1.reshape(tensor_shape); CHECK_EQ(t1.size(), 25); @@ -77,7 +77,7 @@ void TestAPISizeAndShape() { template paddle::DataType TestDtype() { - std::vector tensor_shape = {5, 5}; + std::vector tensor_shape = {5, 5}; auto t1 = paddle::Tensor(paddle::PlaceType::kCPU); t1.reshape(tensor_shape); t1.template mutable_data(); @@ -86,12 +86,12 @@ paddle::DataType TestDtype() { template void TestCast(paddle::DataType data_type) { - std::vector tensor_shape = {5, 5}; + std::vector tensor_shape = {5, 5}; auto t1 = paddle::Tensor(paddle::PlaceType::kCPU); t1.reshape(tensor_shape); t1.template mutable_data(); auto t2 = t1.cast(data_type); - CHECK_EQ(t2.type(), data_type); + CHECK(t2.type() == data_type); } void GroupTestCopy() { @@ -99,14 +99,6 @@ void GroupTestCopy() { TestCopyTensor(); VLOG(2) << "Double cpu-cpu-gpu-gpu-cpu"; TestCopyTensor(); - VLOG(2) << "Fp16 cpu-cpu-gpu-gpu-cpu"; - TestCopyTensor(); - VLOG(2) << "BF16 cpu-cpu-gpu-gpu-cpu"; - TestCopyTensor(); - VLOG(2) << "complex128 cpu-cpu-gpu-gpu-cpu"; - TestCopyTensor(); - VLOG(2) << "complex64 cpu-cpu-gpu-gpu-cpu"; - TestCopyTensor(); VLOG(2) << "int cpu-cpu-gpu-gpu-cpu"; TestCopyTensor(); VLOG(2) << "int64 cpu-cpu-gpu-gpu-cpu"; @@ -117,6 +109,12 @@ void GroupTestCopy() { TestCopyTensor(); VLOG(2) << "uint8 cpu-cpu-gpu-gpu-cpu"; TestCopyTensor(); + VLOG(2) << "complex64 cpu-cpu-gpu-gpu-cpu"; + TestCopyTensor(); + VLOG(2) << "complex128 cpu-cpu-gpu-gpu-cpu"; + TestCopyTensor(); + VLOG(2) << "Fp16 cpu-cpu-gpu-gpu-cpu"; + TestCopyTensor(); } void GroupTestCast() { @@ -128,10 +126,6 @@ void GroupTestCast() { TestCast(paddle::DataType::FLOAT32); VLOG(2) << "double cast"; TestCast(paddle::DataType::FLOAT32); - VLOG(2) << "bfloat16 cast"; - TestCast(paddle::DataType::FLOAT32); - VLOG(2) << "float16 cast"; - TestCast(paddle::DataType::FLOAT32); VLOG(2) << "bool cast"; TestCast(paddle::DataType::FLOAT32); VLOG(2) << "uint8 cast"; @@ -139,47 +133,34 @@ void GroupTestCast() { VLOG(2) << "float cast"; TestCast(paddle::DataType::FLOAT32); VLOG(2) << "complex64 cast"; - TestCast(paddle::DataType::FLOAT32); + TestCast(paddle::DataType::FLOAT32); VLOG(2) << "complex128 cast"; - TestCast(paddle::DataType::FLOAT32); + TestCast(paddle::DataType::FLOAT32); + VLOG(2) << "float16 cast"; + TestCast(paddle::DataType::FLOAT16); } void GroupTestDtype() { CHECK(TestDtype() == paddle::DataType::FLOAT32); CHECK(TestDtype() == paddle::DataType::FLOAT64); - CHECK(TestDtype() == paddle::DataType::FLOAT16); - CHECK(TestDtype() == paddle::DataType::BFLOAT16); - CHECK(TestDtype() == - paddle::DataType::COMPLEX128); - CHECK(TestDtype() == - paddle::DataType::COMPLEX64); CHECK(TestDtype() == paddle::DataType::INT32); CHECK(TestDtype() == paddle::DataType::INT64); CHECK(TestDtype() == paddle::DataType::INT16); CHECK(TestDtype() == paddle::DataType::INT8); CHECK(TestDtype() == paddle::DataType::UINT8); + CHECK(TestDtype() == paddle::DataType::COMPLEX64); + CHECK(TestDtype() == paddle::DataType::COMPLEX128); + CHECK(TestDtype() == paddle::DataType::FLOAT16); } void GroupTestDtypeConvert() { // enum -> proto - CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( - paddle::DataType::COMPLEX128) == - paddle::framework::proto::VarType::COMPLEX128); - CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( - paddle::DataType::COMPLEX64) == - paddle::framework::proto::VarType::COMPLEX64); CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( paddle::DataType::FLOAT64) == paddle::framework::proto::VarType::FP64); CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( paddle::DataType::FLOAT32) == paddle::framework::proto::VarType::FP32); - CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( - paddle::DataType::FLOAT16) == - paddle::framework::proto::VarType::FP16); - CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( - paddle::DataType::BFLOAT16) == - paddle::framework::proto::VarType::BF16); CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( paddle::DataType::UINT8) == paddle::framework::proto::VarType::UINT8); @@ -196,25 +177,22 @@ void GroupTestDtypeConvert() { paddle::framework::proto::VarType::INT16); CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( paddle::DataType::BOOL) == paddle::framework::proto::VarType::BOOL); + CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( + paddle::DataType::COMPLEX64) == + paddle::framework::proto::VarType::COMPLEX64); + CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( + paddle::DataType::COMPLEX128) == + paddle::framework::proto::VarType::COMPLEX128); + CHECK(paddle::framework::CustomTensorUtils::ConvertEnumDTypeToInnerDType( + paddle::DataType::FLOAT16) == + paddle::framework::proto::VarType::FP16); // proto -> enum - CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( - paddle::framework::proto::VarType::COMPLEX128) == - paddle::DataType::COMPLEX128); - CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( - paddle::framework::proto::VarType::COMPLEX64) == - paddle::DataType::COMPLEX64); CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( paddle::framework::proto::VarType::FP64) == paddle::DataType::FLOAT64); CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( paddle::framework::proto::VarType::FP32) == paddle::DataType::FLOAT32); - CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( - paddle::framework::proto::VarType::FP16) == - paddle::DataType::FLOAT16); - CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( - paddle::framework::proto::VarType::BF16) == - paddle::DataType::BFLOAT16); CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( paddle::framework::proto::VarType::INT64) == paddle::DataType::INT64); @@ -231,6 +209,15 @@ void GroupTestDtypeConvert() { paddle::DataType::INT16); CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( paddle::framework::proto::VarType::BOOL) == paddle::DataType::BOOL); + CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( + paddle::framework::proto::VarType::COMPLEX64) == + paddle::DataType::COMPLEX64); + CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( + paddle::framework::proto::VarType::COMPLEX128) == + paddle::DataType::COMPLEX128); + CHECK(paddle::framework::CustomTensorUtils::ConvertInnerDTypeToEnumDType( + paddle::framework::proto::VarType::FP16) == + paddle::DataType::FLOAT16); } TEST(CustomTensor, copyTest) { diff --git a/paddle/fluid/framework/custom_tensor_utils.h b/paddle/fluid/framework/custom_tensor_utils.h index 4b465d3911df1..fad1e3ee3496c 100644 --- a/paddle/fluid/framework/custom_tensor_utils.h +++ b/paddle/fluid/framework/custom_tensor_utils.h @@ -16,10 +16,13 @@ limitations under the License. */ #include -#include "paddle/fluid/extension/include/tensor.h" +#include "paddle/fluid/extension/include/ext_tensor.h" #include "paddle/fluid/framework/data_type.h" #include "paddle/fluid/platform/gpu_info.h" #include "paddle/fluid/platform/place.h" +#ifdef PADDLE_WITH_CUDA +#endif +#include "paddle/fluid/platform/device_context.h" namespace paddle { namespace framework { @@ -39,18 +42,10 @@ class CustomTensorUtils { static framework::proto::VarType::Type ConvertEnumDTypeToInnerDType( const paddle::DataType& dtype) { switch (dtype) { - case paddle::DataType::COMPLEX128: - return framework::proto::VarType::COMPLEX128; - case paddle::DataType::COMPLEX64: - return framework::proto::VarType::COMPLEX64; case paddle::DataType::FLOAT64: return framework::proto::VarType::FP64; case paddle::DataType::FLOAT32: return framework::proto::VarType::FP32; - case paddle::DataType::FLOAT16: - return framework::proto::VarType::FP16; - case paddle::DataType::BFLOAT16: - return framework::proto::VarType::BF16; case paddle::DataType::UINT8: return framework::proto::VarType::UINT8; case paddle::DataType::INT8: @@ -61,6 +56,12 @@ class CustomTensorUtils { return framework::proto::VarType::INT64; case paddle::DataType::INT16: return framework::proto::VarType::INT16; + case paddle::DataType::COMPLEX64: + return framework::proto::VarType::COMPLEX64; + case paddle::DataType::COMPLEX128: + return framework::proto::VarType::COMPLEX128; + case paddle::DataType::FLOAT16: + return framework::proto::VarType::FP16; case paddle::DataType::BOOL: return framework::proto::VarType::BOOL; default: @@ -74,18 +75,10 @@ class CustomTensorUtils { static paddle::DataType ConvertInnerDTypeToEnumDType( const framework::proto::VarType::Type& dtype) { switch (dtype) { - case framework::proto::VarType::COMPLEX128: - return paddle::DataType::COMPLEX128; - case framework::proto::VarType::COMPLEX64: - return paddle::DataType::COMPLEX64; case framework::proto::VarType::FP64: return paddle::DataType::FLOAT64; case framework::proto::VarType::FP32: return paddle::DataType::FLOAT32; - case framework::proto::VarType::FP16: - return paddle::DataType::FLOAT16; - case framework::proto::VarType::BF16: - return paddle::DataType::BFLOAT16; case framework::proto::VarType::INT64: return paddle::DataType::INT64; case framework::proto::VarType::INT32: @@ -96,6 +89,12 @@ class CustomTensorUtils { return paddle::DataType::UINT8; case framework::proto::VarType::INT16: return paddle::DataType::INT16; + case framework::proto::VarType::COMPLEX64: + return paddle::DataType::COMPLEX64; + case framework::proto::VarType::COMPLEX128: + return paddle::DataType::COMPLEX128; + case framework::proto::VarType::FP16: + return paddle::DataType::FLOAT16; case framework::proto::VarType::BOOL: return paddle::DataType::BOOL; default: @@ -139,6 +138,19 @@ class CustomTensorUtils { } return PlaceType::kUNK; } + + static void SetTensorCurrentStream(paddle::Tensor* src, + const platform::Place& pc) { + if (platform::is_gpu_place(pc)) { +#ifdef PADDLE_WITH_CUDA + auto* dev_ctx = static_cast( + platform::DeviceContextPool::Instance().Get(pc)); + src->stream_.SetStream(reinterpret_cast(dev_ctx->stream())); +#endif + } else { + return; + } + } }; } // namespace framework diff --git a/paddle/fluid/framework/data_feed.cc b/paddle/fluid/framework/data_feed.cc index 2b70cdb9f13bc..6f244ee171359 100644 --- a/paddle/fluid/framework/data_feed.cc +++ b/paddle/fluid/framework/data_feed.cc @@ -151,9 +151,12 @@ void DataFeed::CopyToFeedTensor(void* dst, const void* src, size_t size) { } else { #ifdef PADDLE_WITH_CUDA cudaMemcpy(dst, src, size, cudaMemcpyHostToDevice); +#elif defined(PADDLE_WITH_HIP) + hipMemcpy(dst, src, size, hipMemcpyHostToDevice); #else PADDLE_THROW(platform::errors::Unimplemented( - "Not supported GPU, please compile with option WITH_GPU=ON.")); + "Not supported GPU/ROCM, please compile with option WITH_GPU=ON or " + "WITH_ROCM=ON.")); #endif } } @@ -516,7 +519,7 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { VLOG(0) << "error: the number of ids is a negative number: " << num; VLOG(0) << "please check line<" << instance_cout << "> in file<" << filename << ">"; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); return false; } else if (num == 0) { @@ -527,14 +530,14 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { "characters."; VLOG(0) << "please check line<" << instance_cout << "> in file<" << filename << ">"; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); return false; } else if (errno == ERANGE || num > INT_MAX) { VLOG(0) << "error: the number of ids greater than INT_MAX"; VLOG(0) << "please check line<" << instance_cout << "> in file<" << filename << ">"; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); return false; } @@ -546,7 +549,7 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { "representable values for float"; VLOG(0) << "please check line<" << instance_cout << "> in file<" << filename << ">"; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); VLOG(0) << "and in this slot: " << j @@ -555,7 +558,7 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { } if (j + 1 != num && endptr - str == len) { VLOG(0) << "error: there is a wrong with the number of ids."; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); VLOG(0) << "and in this slot: " << j @@ -571,7 +574,7 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { if (errno == ERANGE) { VLOG(0) << "error: the value is out of the range of " "representable values for uint64_t"; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); VLOG(0) << "and in this slot: " << j @@ -582,7 +585,7 @@ bool MultiSlotDataFeed::CheckFile(const char* filename) { } if (j + 1 != num && endptr - str == len) { VLOG(0) << "error: there is a wrong with the number of ids."; - VLOG(0) << "Error occured when parsing " << i + VLOG(0) << "Error occurred when parsing " << i << " th slot with total slots number: " << all_slots_.size(); VLOG(0) << "and in this slot: " << j @@ -1157,7 +1160,7 @@ void MultiSlotInMemoryDataFeed::PutToFeedVec( #endif } -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) +#if (defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)) && !defined(_WIN32) template void PrivateInstantDataFeed::PutToFeedVec() { for (size_t i = 0; i < use_slots_.size(); ++i) { diff --git a/paddle/fluid/framework/data_feed.h b/paddle/fluid/framework/data_feed.h index 1abca95b8bb73..ec79005dfecc1 100644 --- a/paddle/fluid/framework/data_feed.h +++ b/paddle/fluid/framework/data_feed.h @@ -716,7 +716,7 @@ class PaddleBoxDataFeed : public MultiSlotInMemoryDataFeed { int pv_batch_size_; }; -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) +#if (defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)) && !defined(_WIN32) template class PrivateInstantDataFeed : public DataFeed { public: diff --git a/paddle/fluid/framework/data_feed_factory.cc b/paddle/fluid/framework/data_feed_factory.cc index c967b0f0ca59d..ec1b8ec773fa6 100644 --- a/paddle/fluid/framework/data_feed_factory.cc +++ b/paddle/fluid/framework/data_feed_factory.cc @@ -68,7 +68,7 @@ std::shared_ptr DataFeedFactory::CreateDataFeed( REGISTER_DATAFEED_CLASS(MultiSlotDataFeed); REGISTER_DATAFEED_CLASS(MultiSlotInMemoryDataFeed); REGISTER_DATAFEED_CLASS(PaddleBoxDataFeed); -#if defined(PADDLE_WITH_CUDA) && !defined(_WIN32) +#if (defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)) && !defined(_WIN32) REGISTER_DATAFEED_CLASS(MultiSlotFileInstantDataFeed); #endif } // namespace framework diff --git a/paddle/fluid/framework/data_type.h b/paddle/fluid/framework/data_type.h index 7aa7b7b2d96cf..c8f73a5469ab3 100644 --- a/paddle/fluid/framework/data_type.h +++ b/paddle/fluid/framework/data_type.h @@ -20,6 +20,7 @@ limitations under the License. */ #include "paddle/fluid/platform/bfloat16.h" #include "paddle/fluid/platform/complex128.h" #include "paddle/fluid/platform/complex64.h" +#include "paddle/fluid/platform/eigen_ext.h" #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/float16.h" diff --git a/paddle/fluid/framework/data_type_transform.cc b/paddle/fluid/framework/data_type_transform.cc index 084c6e6816bd5..5a716eba8dbe8 100644 --- a/paddle/fluid/framework/data_type_transform.cc +++ b/paddle/fluid/framework/data_type_transform.cc @@ -47,7 +47,7 @@ struct CastDataType { auto* context = static_cast(ctx_); trans(*context, in_begin, in_end, out_begin, CastDataTypeFunctor()); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) } else if (platform::is_gpu_place(in_.place())) { platform::Transform trans; auto* context = static_cast(ctx_); diff --git a/paddle/fluid/framework/details/broadcast_op_handle.cc b/paddle/fluid/framework/details/broadcast_op_handle.cc index 34d800994f10d..36b840e4945a0 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle.cc +++ b/paddle/fluid/framework/details/broadcast_op_handle.cc @@ -81,7 +81,7 @@ void BroadcastOpHandle::BroadcastOneVar( }); } } else if (platform::is_gpu_place(in_tensor.place())) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) VarHandle *out_handle = nullptr; int root_id = BOOST_GET_CONST(platform::CUDAPlace, in_tensor.place()).device; diff --git a/paddle/fluid/framework/details/broadcast_op_handle.h b/paddle/fluid/framework/details/broadcast_op_handle.h index e15dd18467c72..8ca20da97416c 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle.h +++ b/paddle/fluid/framework/details/broadcast_op_handle.h @@ -34,7 +34,7 @@ class Node; } // namespace ir } // namespace framework namespace platform { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) struct NCCLContextMap; #endif #if defined(PADDLE_WITH_XPU_BKCL) @@ -43,7 +43,7 @@ struct BKCLContextMap; } // namespace platform } // namespace paddle -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/nccl_helper.h" #elif defined(PADDLE_WITH_XPU_BKCL) #include "paddle/fluid/platform/bkcl_helper.h" @@ -55,7 +55,7 @@ namespace details { struct BroadcastOpHandle : public OpHandleBase { public: -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) BroadcastOpHandle(ir::Node *node, const std::vector &local_scopes, const std::vector &places, const platform::NCCLContextMap *nccl_ctxs) @@ -106,7 +106,7 @@ struct BroadcastOpHandle : public OpHandleBase { std::vector local_scopes_; std::vector places_; -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) const platform::NCCLContextMap *nccl_ctxs_; #elif defined(PADDLE_WITH_XPU_BKCL) const platform::BKCLContextMap *bkcl_ctxs_; diff --git a/paddle/fluid/framework/details/broadcast_op_handle_test.cc b/paddle/fluid/framework/details/broadcast_op_handle_test.cc index cfd6b71aabdd2..d8fb1b05ed5b7 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle_test.cc +++ b/paddle/fluid/framework/details/broadcast_op_handle_test.cc @@ -36,7 +36,8 @@ TEST(BroadcastTester, TestCPUBroadcastTestSelectedRows) { test_op.TestBroadcastSelectedRows(input_scope_idx); } -#if defined(PADDLE_WITH_CUDA) && defined(PADDLE_WITH_NCCL) +#if (defined(PADDLE_WITH_CUDA) && defined(PADDLE_WITH_NCCL)) || \ + (defined(PADDLE_WITH_HIP) && defined(PADDLE_WITH_RCCL)) TEST(BroadcastTester, TestGPUBroadcastTestLodTensor) { TestBroadcastOpHandle test_op; size_t input_scope_idx = 0; diff --git a/paddle/fluid/framework/details/broadcast_op_handle_test.h b/paddle/fluid/framework/details/broadcast_op_handle_test.h index af053de4f6661..6ca4baa6d8b04 100644 --- a/paddle/fluid/framework/details/broadcast_op_handle_test.h +++ b/paddle/fluid/framework/details/broadcast_op_handle_test.h @@ -48,7 +48,7 @@ struct TestBroadcastOpHandle { std::vector> nodes_; std::vector place_list_; DeviceType use_device_; -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) std::unique_ptr nccl_ctxs_; #endif @@ -60,7 +60,7 @@ struct TestBroadcastOpHandle { for (size_t j = 0; j < ctxs_.size(); ++j) { ctxs_[j]->Wait(); } -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) if (nccl_ctxs_) { nccl_ctxs_->WaitAll(); } @@ -94,7 +94,7 @@ struct TestBroadcastOpHandle { platform::errors::PreconditionNotMet("Not compiled with BKCL.")); #endif } else if (use_device_ == p::kCUDA) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) int count = p::GetCUDADeviceCount(); if (count <= 1) { LOG(WARNING) << "Cannot test multi-gpu Broadcast, because the CUDA " @@ -122,7 +122,7 @@ struct TestBroadcastOpHandle { #if defined(PADDLE_WITH_XPU_BKCL) bkcl_ctxs_.reset(nullptr); #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) nccl_ctxs_.reset(nullptr); #endif } @@ -143,7 +143,7 @@ struct TestBroadcastOpHandle { nodes_.emplace_back( ir::CreateNodeForTest("node0", ir::Node::Type::kOperation)); if (use_device_ == p::kCUDA) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) op_handle_ = new BroadcastOpHandle(nodes_.back().get(), local_scopes_, place_list_, nccl_ctxs_.get()); #else diff --git a/paddle/fluid/framework/details/exception_holder.h b/paddle/fluid/framework/details/exception_holder.h index 66c490724c5e8..1fb802b3f651d 100644 --- a/paddle/fluid/framework/details/exception_holder.h +++ b/paddle/fluid/framework/details/exception_holder.h @@ -108,7 +108,7 @@ class ExceptionHolder { type_ = kNone; } - // NOTE: currently in PE, multiple exceptions may occured in multiple + // NOTE: currently in PE, multiple exceptions may occurred in multiple // threads, and the exception that occur later will overwrite that // occur earlier, but what we want should be the first triggered exception. // However, EOF exception is lower priority exception and can be overwritten, diff --git a/paddle/fluid/framework/details/fused_all_reduce_op_handle.cc b/paddle/fluid/framework/details/fused_all_reduce_op_handle.cc index f792f7f8963e0..8f45c364476a7 100644 --- a/paddle/fluid/framework/details/fused_all_reduce_op_handle.cc +++ b/paddle/fluid/framework/details/fused_all_reduce_op_handle.cc @@ -76,14 +76,10 @@ void FusedAllReduceOpHandle::RunImpl() { "handles is %d, and the number of output variable handles is %d.", in_var_handles.size(), out_var_handles.size())); -// Note: some gradient op doesn't have CUDAKernel, so the gradients of -// those op are in CPUPlace, in this case, the all reduce should not be fused. -#if defined(PADDLE_WITH_XPU_BKCL) - // TODO(liuyuhui): XPU don't support fuse all reduce for now - if (InputIsInDifferentPlace(in_var_handles) || true) { -#else + // Note: some gradient op doesn't have CUDAKernel or XPUKernel, so the + // gradients of those op are in CPUPlace, in this case, the all reduce + // should not be fused. if (InputIsInDifferentPlace(in_var_handles)) { -#endif for (size_t j = 0; j < num_of_all_reduce_; ++j) { std::vector dev_inputs; std::vector dev_outputs; diff --git a/paddle/fluid/framework/details/nan_inf_utils_detail.cu b/paddle/fluid/framework/details/nan_inf_utils_detail.cu index 55261cf7cde98..96d1a9fb94927 100644 --- a/paddle/fluid/framework/details/nan_inf_utils_detail.cu +++ b/paddle/fluid/framework/details/nan_inf_utils_detail.cu @@ -82,7 +82,7 @@ __device__ __forceinline__ void PrintNanInfKernel(const T* value, } __syncthreads; -#ifdef PADDLE_WITH_HIP +#ifdef __HIPCC__ if (true && hipThreadIdx_x == 0) { printf("In block %d, there has %u,%u,%u nan,inf,num\n", hipBlockIdx_x, nan_count, inf_count, num_count); @@ -156,7 +156,7 @@ void TensorCheckerVisitor::apply( "op_var2gpu_str, but now failed", op_var)); -#ifdef PADDLE_WITH_HIP +#ifdef __HIPCC__ PADDLE_ENFORCE_CUDA_SUCCESS( hipMemcpyAsync(gpu_str_ptr, iter->first.c_str(), op_var.length() + 1, hipMemcpyHostToDevice, dev_ctx->stream())); @@ -176,11 +176,16 @@ void TensorCheckerVisitor::apply( } } +#ifdef __HIPCC__ + // HIP will throw GPU memory access fault if threads > 256 + const size_t threads = 256; +#else const size_t threads = 1024; +#endif size_t blocks = std::min(static_cast(128), static_cast((tensor_.numel() + threads - 1) / threads)); -#ifdef PADDLE_WITH_HIP +#ifdef __HIPCC__ hipLaunchKernelGGL(CheckNanInfKernel, dim3(blocks), dim3(threads), 0, dev_ctx->stream(), tensor_.data(), tensor_.numel(), print_num, gpu_str_ptr); diff --git a/paddle/fluid/framework/device_worker.h b/paddle/fluid/framework/device_worker.h index 6ecc02bbae616..05c54a90f7eb0 100644 --- a/paddle/fluid/framework/device_worker.h +++ b/paddle/fluid/framework/device_worker.h @@ -28,6 +28,7 @@ limitations under the License. */ #include #include "paddle/fluid/framework/data_feed.h" +#include "paddle/fluid/framework/executor_gc_helper.h" #include "paddle/fluid/framework/heter_service.h" #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/op_registry.h" @@ -52,7 +53,7 @@ class DeviceContext; } // namespace platform } // namespace paddle -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/nccl_helper.h" #endif @@ -73,11 +74,12 @@ class PullDenseWorker { public: virtual ~PullDenseWorker() {} virtual void Initialize(const TrainerDesc& param); -#ifdef PADDLE_WITH_CUDA - void AddStream(const cudaStream_t stream) { copy_streams_.push_back(stream); } +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + void AddStream(const gpuStream_t stream) { copy_streams_.push_back(stream); } #endif -#if (defined PADDLE_WITH_CUDA) || (defined PADDLE_WITH_XPU) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) void AddPlace(const paddle::platform::Place place) { places_.push_back(place); } @@ -137,8 +139,8 @@ class PullDenseWorker { float total_batch_num_ = 0; std::unordered_map scope_to_thread_id_; -#ifdef PADDLE_WITH_CUDA - std::vector copy_streams_; +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + std::vector copy_streams_; #endif std::vector places_; std::vector thread_scopes_; @@ -167,9 +169,10 @@ class DeviceWorker { virtual void CacheProgram(const ProgramDesc& main_program) {} virtual void ProduceTasks() {} virtual void GetXpuOpIndex() {} -#ifdef PADDLE_WITH_CUDA - virtual void SetStream(const cudaStream_t stream) {} - virtual void SetEvent(const cudaEvent_t event) {} + virtual void Schedule(int taskid) {} +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + virtual void SetStream(const gpuStream_t stream) {} + virtual void SetEvent(const gpuEvent_t event) {} #endif virtual void SetNeedDumpField(bool need_dump_field) { need_dump_field_ = need_dump_field; @@ -437,7 +440,8 @@ class HeterCpuWorker : public HogwildWorker { }; #endif -#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU) && \ +#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_HIP || \ + defined PADDLE_WITH_XPU) && \ (defined PADDLE_WITH_PSLIB) class HeterBoxWorker : public HogwildWorker { public: @@ -451,9 +455,9 @@ class HeterBoxWorker : public HogwildWorker { virtual void CacheProgram(const ProgramDesc& main_program) { new (&program_) ProgramDesc(main_program); } - virtual void ProduceTasks() override; - virtual void SetStream(const cudaStream_t stream) { copy_stream_ = stream; } - virtual void SetEvent(const cudaEvent_t event) { event_ = event; } + void ProduceTasks() override; + virtual void SetStream(const gpuStream_t stream) { copy_stream_ = stream; } + virtual void SetEvent(const gpuEvent_t event) { event_ = event; } virtual void TrainFilesWithProfiler() {} void ResetStat(); @@ -515,8 +519,8 @@ class HeterBoxWorker : public HogwildWorker { std::unordered_map> feasign_set_; paddle::framework::Channel> pull_queue_; paddle::framework::Channel> push_queue_; - cudaEvent_t event_; - cudaStream_t copy_stream_; + gpuEvent_t event_; + gpuStream_t copy_stream_; int batch_cnt_{0}; std::atomic done_cnt_{0}; @@ -537,23 +541,24 @@ class HeterBoxWorker : public HogwildWorker { }; #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) class PSGPUWorker : public HogwildWorker { public: PSGPUWorker() {} virtual ~PSGPUWorker() {} virtual void Initialize(const TrainerDesc& desc); virtual void TrainFiles(); + virtual void TrainFilesWithProfiler(); virtual void SetNeedDump(bool need_dump_field); virtual void SetChannelWriter(ChannelObject* queue); virtual void SetWorkerNum(int num) { worker_num_ = num; } virtual void CacheProgram(const ProgramDesc& main_program) { new (&program_) ProgramDesc(main_program); } - virtual void ProduceTasks() override; - virtual void SetStream(const cudaStream_t stream) { copy_stream_ = stream; } - virtual void SetEvent(const cudaEvent_t event) { event_ = event; } - virtual void TrainFilesWithProfiler() {} + void ProduceTasks() override; + virtual void SetStream(const gpuStream_t stream) { copy_stream_ = stream; } + virtual void SetEvent(const gpuEvent_t event) { event_ = event; } void ResetStat(); protected: @@ -611,10 +616,11 @@ class PSGPUWorker : public HogwildWorker { std::unordered_map> feasign_set_; paddle::framework::Channel> pull_queue_; paddle::framework::Channel> push_queue_; - cudaEvent_t event_; - cudaStream_t copy_stream_; + gpuEvent_t event_; + gpuStream_t copy_stream_; int batch_cnt_{0}; std::atomic done_cnt_{0}; + platform::DeviceContext* dev_ctx_ = nullptr; double total_time_; double read_time_; @@ -633,7 +639,7 @@ class PSGPUWorker : public HogwildWorker { }; #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) class SectionWorker : public DeviceWorker { public: SectionWorker() {} @@ -654,6 +660,9 @@ class SectionWorker : public DeviceWorker { void SetDeviceIndex(int tid) override {} void SetThreadIndex(int thread_id) { thread_id_ = thread_id; } void SetMicrobatchNum(int num) { num_microbatches_ = num; } + void SetPipelineStageNum(int num) { num_pipeline_stages_ = num; } + void SetPipelineStage(int stage) { pipeline_stage_ = stage; } + void SetScheduleMode(int mode) { schedule_mode_ = mode; } void SetMicrobatchScopes(const std::vector& scope) { microbatch_scopes_ = scope; } @@ -661,11 +670,23 @@ class SectionWorker : public DeviceWorker { void SetSkipVars(const std::vector& skip_vars) { skip_vars_ = skip_vars; } + void RunBackward( + int micro_id, std::unique_ptr&, + std::unordered_map>&); + void RunForward( + int micro_id, std::unique_ptr&, + std::unordered_map>&); + void RunUpdate( + std::unique_ptr&, + std::unordered_map>&); protected: int section_id_; int thread_id_; int num_microbatches_; + int num_pipeline_stages_; + int pipeline_stage_; + int schedule_mode_; // 0 for F-then-B and 1 for 1F1B std::vector microbatch_scopes_; std::vector skip_vars_; const Scope* minibatch_scope_; diff --git a/paddle/fluid/framework/device_worker_factory.cc b/paddle/fluid/framework/device_worker_factory.cc index af1cf7804f49e..a539a5d5f96b5 100644 --- a/paddle/fluid/framework/device_worker_factory.cc +++ b/paddle/fluid/framework/device_worker_factory.cc @@ -69,15 +69,17 @@ REGISTER_DEVICE_WORKER_CLASS(DownpourWorkerOpt); REGISTER_DEVICE_WORKER_CLASS(HeterCpuWorker); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) REGISTER_DEVICE_WORKER_CLASS(HeterBoxWorker); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) REGISTER_DEVICE_WORKER_CLASS(PSGPUWorker); #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) REGISTER_DEVICE_WORKER_CLASS(SectionWorker); #endif } // namespace framework diff --git a/paddle/fluid/framework/dim_test.cu b/paddle/fluid/framework/dim_test.cu index 7add6d140c7e0..b3c26b10c6ffb 100644 --- a/paddle/fluid/framework/dim_test.cu +++ b/paddle/fluid/framework/dim_test.cu @@ -34,7 +34,12 @@ TEST(Dim, Equality) { // construct a Dim on the GPU thrust::device_vector> t(2); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(test, dim3(1), dim3(1), 0, 0, + thrust::raw_pointer_cast(t.data())); +#else test<<<1, 1>>>(thrust::raw_pointer_cast(t.data())); +#endif a = t[0]; EXPECT_EQ(a[0], 5); EXPECT_EQ(a[1], 6); @@ -55,7 +60,12 @@ TEST(Dim, Equality) { // dynamic access on GPU thrust::device_vector r(1); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(dyn_idx_gpu, dim3(1), dim3(1), 0, 0, + thrust::raw_pointer_cast(r.data())); +#else dyn_idx_gpu<<<1, 1>>>(thrust::raw_pointer_cast(r.data())); +#endif int64_t res = r[0]; EXPECT_EQ(res, 6); } diff --git a/paddle/fluid/framework/dist_multi_trainer.cc b/paddle/fluid/framework/dist_multi_trainer.cc index 7b3f03c6f5f13..4c8681aad2bb1 100644 --- a/paddle/fluid/framework/dist_multi_trainer.cc +++ b/paddle/fluid/framework/dist_multi_trainer.cc @@ -62,9 +62,8 @@ void DistMultiTrainer::Initialize(const TrainerDesc &trainer_desc, void DistMultiTrainer::RegisterHeterCallback() { auto fleet_ptr = FleetWrapper::GetInstance(); - fleet_ptr->RegisterHeterCallback([this](int worker, int taskid) { - // workers_[worker]->Schedule(taskid); - }); + fleet_ptr->RegisterHeterCallback( + [this](int worker, int taskid) { workers_[worker]->Schedule(taskid); }); } void DistMultiTrainer::InitDumpEnv() { diff --git a/paddle/fluid/framework/distributed_strategy.proto b/paddle/fluid/framework/distributed_strategy.proto index 208ab9a93c005..b36793507f54b 100644 --- a/paddle/fluid/framework/distributed_strategy.proto +++ b/paddle/fluid/framework/distributed_strategy.proto @@ -117,7 +117,11 @@ message AsyncConfig { optional int32 lr_decay_steps = 11 [ default = 10 ]; } -message PipelineConfig { optional int32 micro_batch = 1 [ default = 1 ]; } +message PipelineConfig { + optional int32 micro_batch_size = 1 [ default = 1 ]; + optional int32 accumulate_steps = 2 [ default = 1 ]; + optional string schedule_mode = 3 [ default = '1F1B' ]; +} message DistributedStrategy { // bool options diff --git a/paddle/fluid/framework/dlpack_tensor.cc b/paddle/fluid/framework/dlpack_tensor.cc index ac42edec688eb..a3fbb008fe4f4 100644 --- a/paddle/fluid/framework/dlpack_tensor.cc +++ b/paddle/fluid/framework/dlpack_tensor.cc @@ -83,7 +83,7 @@ struct DLContextVisitor : public boost::static_visitor<::DLContext> { } inline ::DLContext operator()(const platform::CUDAPlace &place) const { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) ::DLContext ctx; ctx.device_type = kDLGPU; ctx.device_id = place.device; @@ -95,7 +95,7 @@ struct DLContextVisitor : public boost::static_visitor<::DLContext> { } inline ::DLContext operator()(const platform::CUDAPinnedPlace &place) const { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) ::DLContext ctx; ctx.device_type = kDLCPUPinned; ctx.device_id = 0; diff --git a/paddle/fluid/framework/dlpack_tensor_test.cc b/paddle/fluid/framework/dlpack_tensor_test.cc index c0ab9d3aca0ac..d03437034d62a 100644 --- a/paddle/fluid/framework/dlpack_tensor_test.cc +++ b/paddle/fluid/framework/dlpack_tensor_test.cc @@ -103,7 +103,7 @@ void TestToCudfCompatibleDLManagedTensor(const platform::Place &place, template void TestMainLoop() { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) std::vector places{platform::CPUPlace(), platform::CUDAPlace(0), platform::CUDAPinnedPlace()}; diff --git a/paddle/fluid/framework/executor.cc b/paddle/fluid/framework/executor.cc index b4f7e5f518774..0acc8a55fa9f8 100644 --- a/paddle/fluid/framework/executor.cc +++ b/paddle/fluid/framework/executor.cc @@ -431,7 +431,7 @@ void Executor::RunPartialPreparedContext(ExecutorPrepareContext* ctx, std::unique_ptr gc; if (!ctx->force_disable_gc_ && max_memory_size >= 0) { if (platform::is_gpu_place(place_)) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (IsFastEagerDeletionModeEnabled()) { gc.reset(new UnsafeFastGPUGarbageCollector( BOOST_GET_CONST(platform::CUDAPlace, place_), max_memory_size)); diff --git a/paddle/fluid/framework/fleet/fleet_wrapper.cc b/paddle/fluid/framework/fleet/fleet_wrapper.cc index 0c0792a95cd70..3cd8b55026e51 100644 --- a/paddle/fluid/framework/fleet/fleet_wrapper.cc +++ b/paddle/fluid/framework/fleet/fleet_wrapper.cc @@ -193,7 +193,6 @@ void FleetWrapper::HeterPullSparseVars( for (auto& t : fea_values) { pull_result_ptr.push_back(t.data()); } - /* auto status = pslib_ptr_->_worker_ptr->heter_pull_sparse( workerid, pull_result_ptr.data(), table_id, fea_keys.data(), fea_keys.size(), task->taskid_); @@ -207,7 +206,6 @@ void FleetWrapper::HeterPullSparseVars( exit(-1); } } - */ } void FleetWrapper::HeterPushSparseVars( diff --git a/paddle/fluid/framework/fleet/heter_context.h b/paddle/fluid/framework/fleet/heter_context.h index fc987b523d559..a02931b3f5c28 100644 --- a/paddle/fluid/framework/fleet/heter_context.h +++ b/paddle/fluid/framework/fleet/heter_context.h @@ -66,18 +66,19 @@ class HeterContext { mutex_[i] = new std::mutex(); } } - void batch_add_keys(const std::vector>& thread_keys) { + void batch_add_keys( + const std::vector>& thread_keys) { assert(thread_keys.size() == feature_keys_.size()); for (uint32_t i = 0; i < shard_num_; i++) { int idx = 0; idx = feature_keys_[i].size(); feature_keys_[i].resize(feature_keys_[i].size() + thread_keys[i].size()); - for (uint64_t j = 0; j < thread_keys[i].size(); j++) { - feature_keys_[i][idx + j] = thread_keys[i][j]; - } + std::copy(thread_keys[i].begin(), thread_keys[i].end(), + feature_keys_[i].begin() + idx); } } + void UniqueKeys() { std::vector threads; auto unique_func = [this](int i) { diff --git a/paddle/fluid/framework/fleet/heter_ps/cudf/concurrent_unordered_map.cuh.h b/paddle/fluid/framework/fleet/heter_ps/cudf/concurrent_unordered_map.cuh.h index d14abd218c22a..c5647f2cdcffc 100644 --- a/paddle/fluid/framework/fleet/heter_ps/cudf/concurrent_unordered_map.cuh.h +++ b/paddle/fluid/framework/fleet/heter_ps/cudf/concurrent_unordered_map.cuh.h @@ -737,7 +737,7 @@ x.second ); } int assign_async(const concurrent_unordered_map& other, - gpuStream_t stream = 0) { + cudaStream_t stream = 0) { m_collisions = other.m_collisions; if (other.m_hashtbl_size <= m_hashtbl_capacity) { m_hashtbl_size = other.m_hashtbl_size; @@ -754,7 +754,7 @@ x.second ); return 0; } - void clear_async(gpuStream_t stream = 0) { + void clear_async(cudaStream_t stream = 0) { constexpr int block_size = 128; init_hashtbl<<<((m_hashtbl_size - 1) / block_size) + 1, block_size, 0, stream>>>(m_hashtbl_values, m_hashtbl_size, unused_key, @@ -771,7 +771,7 @@ x.second ); } } - int prefetch(const int dev_id, gpuStream_t stream = 0) { + int prefetch(const int dev_id, cudaStream_t stream = 0) { cudaPointerAttributes hashtbl_values_ptr_attributes; cudaError_t status = cudaPointerGetAttributes( &hashtbl_values_ptr_attributes, m_hashtbl_values); diff --git a/paddle/fluid/framework/fleet/heter_ps/hashtable.h b/paddle/fluid/framework/fleet/heter_ps/hashtable.h index 2aa00e84e1599..e5c0972763bed 100644 --- a/paddle/fluid/framework/fleet/heter_ps/hashtable.h +++ b/paddle/fluid/framework/fleet/heter_ps/hashtable.h @@ -55,6 +55,8 @@ class HashTable { void update(const KeyType* d_keys, const GradType* d_grads, size_t len, Sgd sgd, gpuStream_t stream); + int size() { return container_->size(); } + private: TableContainer* container_; int BLOCK_SIZE_{256}; diff --git a/paddle/fluid/framework/fleet/heter_ps/heter_comm.h b/paddle/fluid/framework/fleet/heter_ps/heter_comm.h index 77591c6df22a5..0e38ebbd7f4e7 100644 --- a/paddle/fluid/framework/fleet/heter_ps/heter_comm.h +++ b/paddle/fluid/framework/fleet/heter_ps/heter_comm.h @@ -118,6 +118,12 @@ class HeterComm { std::vector nodes_; }; + struct CopyTask { + Path* path; + int step; + CopyTask(Path* path_, int step_) : path(path_), step(step_) {} + }; + struct LocalStorage { LocalStorage() {} void init(int size, int dev_id) { @@ -160,9 +166,10 @@ class HeterComm { void create_storage( int start_index, int end_index, int keylen, int vallen, std::vector>& local_strorage); - void walk_to_src(int start_index, int end_index, char* src_val); - void walk_to_dest(int start_index, int end_index, char* src_key, - char* src_val); + void walk_to_dest(int start_index, int gpu_num, int* h_left, int* h_right, + KeyType* src_key, GradType* src_val); + void walk_to_src(int start_index, int gpu_num, int* h_left, int* h_right, + ValType* src_val); private: using Table = HashTable; diff --git a/paddle/fluid/framework/fleet/heter_ps/heter_comm_inl.h b/paddle/fluid/framework/fleet/heter_ps/heter_comm_inl.h index 4e4563daa19fa..2f1c809c01eaa 100644 --- a/paddle/fluid/framework/fleet/heter_ps/heter_comm_inl.h +++ b/paddle/fluid/framework/fleet/heter_ps/heter_comm_inl.h @@ -12,6 +12,8 @@ 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. */ #pragma once +#include + #ifdef PADDLE_WITH_PSLIB namespace paddle { namespace framework { @@ -182,53 +184,105 @@ void HeterComm::create_storage( } template -void HeterComm::walk_to_dest(int start_index, - int end_index, - char* src_key, - char* src_val) { +void HeterComm::walk_to_dest( + int start_index, int gpu_num, int* h_left, int* h_right, KeyType* src_key, + GradType* src_val) { int need_copy_val = 0; if (src_val) { need_copy_val = 1; } - auto& nodes = path_[start_index][end_index].nodes_; - for (size_t i = 0; i < nodes.size(); ++i) { - cudaMemcpyAsync(nodes[i].key_storage, src_key, nodes[i].key_bytes_len, - cudaMemcpyDefault, nodes[i].in_stream); + std::queue que; + for (int i = 0; i < gpu_num; i++) { + if (h_left[i] == -1 || h_right[i] == -1) { + continue; + } + int size = path_[start_index][i].nodes_.size(); + auto& node = path_[start_index][i].nodes_[0]; + CopyTask t(&path_[start_index][i], 0); + que.push(t); + cudaMemcpyAsync(node.key_storage, + reinterpret_cast(src_key + h_left[i]), + node.key_bytes_len, cudaMemcpyDefault, node.in_stream); if (need_copy_val) { - cudaMemcpyAsync(nodes[i].val_storage, src_val, nodes[i].val_bytes_len, - cudaMemcpyDefault, nodes[i].in_stream); + cudaMemcpyAsync(node.val_storage, + reinterpret_cast(src_val + h_left[i]), + node.val_bytes_len, cudaMemcpyDefault, node.in_stream); + } + } + while (!que.empty()) { + CopyTask& cur_task = que.front(); + que.pop(); + if (cur_task.path->nodes_[cur_task.step].sync) { + cudaStreamSynchronize(cur_task.path->nodes_[cur_task.step].in_stream); } - if (nodes[i].sync) { - cudaStreamSynchronize(nodes[i].in_stream); + if (cur_task.step != cur_task.path->nodes_.size() - 1) { + int cur_step = cur_task.step; + CopyTask c(cur_task.path, cur_step + 1); + que.push(c); + cudaMemcpyAsync(cur_task.path->nodes_[cur_step + 1].key_storage, + cur_task.path->nodes_[cur_step].key_storage, + cur_task.path->nodes_[cur_step + 1].key_bytes_len, + cudaMemcpyDefault, + cur_task.path->nodes_[cur_step + 1].in_stream); + if (need_copy_val) { + cudaMemcpyAsync(cur_task.path->nodes_[cur_step + 1].val_storage, + cur_task.path->nodes_[cur_step].val_storage, + cur_task.path->nodes_[cur_step + 1].val_bytes_len, + cudaMemcpyDefault, + cur_task.path->nodes_[cur_step + 1].in_stream); + } } - // cudaStreamSynchronize(nodes[i].in_stream); - src_key = nodes[i].key_storage; - src_val = nodes[i].val_storage; } } template -void HeterComm::walk_to_src(int start_index, - int end_index, - char* src_val) { - auto& nodes = path_[start_index][end_index].nodes_; - int len = nodes.size(); - char* start = NULL; - for (int i = len - 1; i >= 0; --i) { - if (start == NULL) { - start = nodes[i].val_storage; +void HeterComm::walk_to_src( + int start_index, int gpu_num, int* h_left, int* h_right, ValType* src_val) { + std::queue que; + for (int i = 0; i < gpu_num; i++) { + if (h_left[i] == -1 || h_right[i] == -1) { continue; } - cudaMemcpyAsync(nodes[i].val_storage, start, nodes[i].val_bytes_len, - cudaMemcpyDefault, nodes[i].out_stream); - if (nodes[i].sync) { - cudaStreamSynchronize(nodes[i].out_stream); + int cur_step = path_[start_index][i].nodes_.size() - 1; + auto& node = path_[start_index][i].nodes_[cur_step]; + if (cur_step == 0) { + cudaMemcpyAsync(reinterpret_cast(src_val + h_left[i]), + node.val_storage, node.val_bytes_len, cudaMemcpyDefault, + node.out_stream); + } else { + CopyTask t(&path_[start_index][i], cur_step - 1); + que.push(t); + cudaMemcpyAsync(path_[start_index][i].nodes_[cur_step - 1].val_storage, + node.val_storage, + path_[start_index][i].nodes_[cur_step - 1].val_bytes_len, + cudaMemcpyDefault, + path_[start_index][i].nodes_[cur_step - 1].out_stream); + } + } + while (!que.empty()) { + CopyTask& cur_task = que.front(); + que.pop(); + int cur_step = cur_task.step; + if (cur_task.path->nodes_[cur_step].sync) { + cudaStreamSynchronize(cur_task.path->nodes_[cur_step].out_stream); + } + if (cur_step > 0) { + CopyTask c(cur_task.path, cur_step - 1); + que.push(c); + cudaMemcpyAsync(cur_task.path->nodes_[cur_step - 1].val_storage, + cur_task.path->nodes_[cur_step].val_storage, + cur_task.path->nodes_[cur_step - 1].val_bytes_len, + cudaMemcpyDefault, + cur_task.path->nodes_[cur_step - 1].out_stream); + } else if (cur_step == 0) { + int end_index = cur_task.path->nodes_.back().gpu_num; + cudaMemcpyAsync(reinterpret_cast(src_val + h_left[end_index]), + cur_task.path->nodes_[cur_step].val_storage, + cur_task.path->nodes_[cur_step].val_bytes_len, + cudaMemcpyDefault, + cur_task.path->nodes_[cur_step].out_stream); } - start = nodes[i].val_storage; } - cudaMemcpyAsync(src_val, nodes[0].val_storage, nodes[0].val_bytes_len, - cudaMemcpyDefault, nodes[0].out_stream); - // cudaStreamSynchronize(nodes[0].out_stream); } template @@ -462,14 +516,7 @@ void HeterComm::pull_sparse(int num, shard_len * sizeof(ValType), local_storage); } - for (int i = 0; i < total_gpu; ++i) { - int shard_len = h_right[i] - h_left[i] + 1; - if (h_left[i] == -1 || h_right[i] == -1) { - continue; - } - walk_to_dest(num, i, reinterpret_cast(d_shard_keys_ptr + h_left[i]), - NULL); - } + walk_to_dest(num, total_gpu, h_left, h_right, d_shard_keys_ptr, NULL); for (int i = 0; i < total_gpu; ++i) { if (h_left[i] == -1) { @@ -486,14 +533,7 @@ void HeterComm::pull_sparse(int num, cudaStreamSynchronize(resource_->remote_stream(i)); } - for (int i = 0; i < total_gpu; ++i) { - int shard_len = h_right[i] - h_left[i] + 1; - if (h_left[i] == -1 || h_right[i] == -1) { - continue; - } - platform::CUDADeviceGuard guard(resource_->dev_id(i)); - walk_to_src(num, i, reinterpret_cast(d_shard_vals_ptr + h_left[i])); - } + walk_to_src(num, total_gpu, h_left, h_right, d_shard_vals_ptr); for (int i = 0; i < total_gpu; ++i) { auto& node = path_[num][i].nodes_.front(); @@ -561,7 +601,6 @@ void HeterComm::push_sparse(int gpu_num, cudaMemcpyDeviceToHost); std::vector> local_storage; - for (int i = 0; i < total_gpu; ++i) { int shard_len = h_right[i] - h_left[i] + 1; if (h_left[i] == -1 || h_right[i] == -1) { @@ -571,15 +610,8 @@ void HeterComm::push_sparse(int gpu_num, shard_len * sizeof(GradType), local_storage); } - for (int i = 0; i < total_gpu; ++i) { - int shard_len = h_right[i] - h_left[i] + 1; - if (h_left[i] == -1 || h_right[i] == -1) { - continue; - } - walk_to_dest(gpu_num, i, - reinterpret_cast(d_shard_keys_ptr + h_left[i]), - reinterpret_cast(d_shard_grads_ptr + h_left[i])); - } + walk_to_dest(gpu_num, total_gpu, h_left, h_right, d_shard_keys_ptr, + d_shard_grads_ptr); for (int i = 0; i < total_gpu; ++i) { if (h_left[i] == -1 || h_right[i] == -1) { diff --git a/paddle/fluid/framework/fleet/ps_gpu_wrapper.cc b/paddle/fluid/framework/fleet/ps_gpu_wrapper.cc index 516f09a9ef26e..4274876c9975e 100644 --- a/paddle/fluid/framework/fleet/ps_gpu_wrapper.cc +++ b/paddle/fluid/framework/fleet/ps_gpu_wrapper.cc @@ -65,9 +65,6 @@ void PSGPUWrapper::BuildTask(std::shared_ptr gpu_task, for (int i = 0; i < thread_keys_thread_num_; i++) { thread_keys_[i].resize(thread_keys_shard_num_); for (int j = 0; j < thread_keys_shard_num_; j++) { - thread_keys_[i][j].reserve(2 * max_fea_num_per_pass_ / - thread_keys_shard_num_ / - thread_keys_thread_num_); } } const std::deque& vec_data = input_channel->GetData(); @@ -84,7 +81,7 @@ void PSGPUWrapper::BuildTask(std::shared_ptr gpu_task, for (const auto feasign : feasign_v) { uint64_t cur_key = feasign.sign().uint64_feasign_; int shard_id = cur_key % thread_keys_shard_num_; - this->thread_keys_[i][shard_id].push_back(cur_key); + this->thread_keys_[i][shard_id].insert(cur_key); } } }; @@ -123,7 +120,7 @@ void PSGPUWrapper::BuildTask(std::shared_ptr gpu_task, VLOG(3) << "GpuPs shard: " << i << " key len: " << local_keys[i].size(); local_ptr[i].resize(local_keys[i].size()); } - + timeline.Start(); auto ptl_func = [this, &local_keys, &local_ptr, &table_id, &fleet_ptr](int i) { size_t key_size = local_keys[i].size(); @@ -149,7 +146,8 @@ void PSGPUWrapper::BuildTask(std::shared_ptr gpu_task, t.join(); } timeline.Pause(); - VLOG(1) << "GpuPs pull sparse cost " << timeline.ElapsedSec() << " seconds."; + VLOG(1) << "pull sparse from CpuPS into GpuPS cost " << timeline.ElapsedSec() + << " seconds."; timeline.Start(); auto build_func = [device_num, &local_keys, &local_ptr, &device_keys, @@ -225,6 +223,7 @@ void PSGPUWrapper::BuildGPUPS(uint64_t table_id, int feature_dim) { size_t size_max = 0; for (int i = 0; i < device_num; i++) { feature_keys_count[i] = gpu_task->device_keys_[i].size(); + VLOG(1) << i << " card contains feasign nums: " << feature_keys_count[i]; size_max = std::max(size_max, feature_keys_count[i]); } if (HeterPs_) { @@ -314,7 +313,7 @@ void PSGPUWrapper::PullSparse(const paddle::platform::Place& place, "GpuPs: PullSparse Only Support CUDAPlace Now.")); } all_timer.Pause(); - VLOG(1) << "GpuPs PullSparse total costs: " << all_timer.ElapsedSec() + VLOG(3) << "GpuPs PullSparse total costs: " << all_timer.ElapsedSec() << " s, of which GPUPS costs: " << pull_gpups_timer.ElapsedSec() << " s"; VLOG(3) << "End PullSparse"; @@ -360,7 +359,7 @@ void PSGPUWrapper::PushSparseGrad(const paddle::platform::Place& place, "GPUPS: PushSparseGrad Only Support CUDAPlace Now.")); } all_timer.Pause(); - VLOG(1) << "PushSparseGrad total cost: " << all_timer.ElapsedSec() + VLOG(3) << "PushSparseGrad total cost: " << all_timer.ElapsedSec() << " s, of which GPUPS cost: " << push_gpups_timer.ElapsedSec() << " s"; VLOG(3) << "End PushSparseGrad"; diff --git a/paddle/fluid/framework/fleet/ps_gpu_wrapper.h b/paddle/fluid/framework/fleet/ps_gpu_wrapper.h index fd3323d9d4764..ef586b41fe05d 100644 --- a/paddle/fluid/framework/fleet/ps_gpu_wrapper.h +++ b/paddle/fluid/framework/fleet/ps_gpu_wrapper.h @@ -227,7 +227,7 @@ class PSGPUWrapper { std::vector heter_devices_; std::unordered_set gpu_ps_config_keys_; HeterObjectPool gpu_task_pool_; - std::vector>> thread_keys_; + std::vector>> thread_keys_; int thread_keys_thread_num_ = 37; int thread_keys_shard_num_ = 37; uint64_t max_fea_num_per_pass_ = 5000000000; diff --git a/paddle/fluid/framework/garbage_collector.cc b/paddle/fluid/framework/garbage_collector.cc index 907b341390746..c8b6c76425517 100644 --- a/paddle/fluid/framework/garbage_collector.cc +++ b/paddle/fluid/framework/garbage_collector.cc @@ -13,7 +13,7 @@ // limitations under the License. #include -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include "paddle/fluid/platform/cuda_device_guard.h" #endif #include "gflags/gflags.h" @@ -53,7 +53,7 @@ void XPUGarbageCollector::ClearCallback(const std::function &callback) { } #endif -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) UnsafeFastGPUGarbageCollector::UnsafeFastGPUGarbageCollector( const platform::CUDAPlace &place, size_t max_memory_size) : GarbageCollector(place, max_memory_size) {} @@ -82,18 +82,27 @@ StreamGarbageCollector::StreamGarbageCollector(const platform::CUDAPlace &place, size_t max_memory_size) : GarbageCollector(place, max_memory_size) { platform::CUDADeviceGuard guard(place.device); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamCreate(&stream_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamCreate(&stream_)); +#endif callback_manager_.reset(new platform::StreamCallbackManager(stream_)); } StreamGarbageCollector::~StreamGarbageCollector() { auto place = BOOST_GET_CONST(platform::CUDAPlace, this->dev_ctx_->GetPlace()); platform::CUDADeviceGuard guard(place.device); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream_)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamDestroy(stream_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream_)); PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamDestroy(stream_)); +#endif } -cudaStream_t StreamGarbageCollector::stream() const { return stream_; } +gpuStream_t StreamGarbageCollector::stream() const { return stream_; } void StreamGarbageCollector::Wait() const { callback_manager_->Wait(); } diff --git a/paddle/fluid/framework/garbage_collector.h b/paddle/fluid/framework/garbage_collector.h index 9148d2f2520a2..97800865af861 100644 --- a/paddle/fluid/framework/garbage_collector.h +++ b/paddle/fluid/framework/garbage_collector.h @@ -80,7 +80,7 @@ class XPUGarbageCollector : public GarbageCollector { }; #endif -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) class UnsafeFastGPUGarbageCollector : public GarbageCollector { public: UnsafeFastGPUGarbageCollector(const platform::CUDAPlace &place, @@ -110,13 +110,13 @@ class StreamGarbageCollector : public GarbageCollector { void Wait() const override; - cudaStream_t stream() const; + gpuStream_t stream() const; protected: void ClearCallback(const std::function &callback) override; private: - cudaStream_t stream_; + gpuStream_t stream_; std::unique_ptr callback_manager_; }; diff --git a/paddle/fluid/framework/generator.cc b/paddle/fluid/framework/generator.cc index 478d10ee7a4c1..737dbafb64cb2 100644 --- a/paddle/fluid/framework/generator.cc +++ b/paddle/fluid/framework/generator.cc @@ -25,7 +25,7 @@ namespace paddle { namespace framework { const std::shared_ptr& GetDefaultCUDAGenerator(int64_t device_id) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) static int64_t num_cuda_devices = -1; static std::once_flag num_devices_init_flag; @@ -157,7 +157,7 @@ uint64_t Generator::Random64() { std::pair Generator::IncrementOffset( uint64_t increament_offset) { uint64_t cur_offset = this->state_.thread_offset; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) std::lock_guard lock(this->mu_); this->state_.thread_offset += increament_offset; diff --git a/paddle/fluid/framework/heter_service.h b/paddle/fluid/framework/heter_service.h index a6687f9a65014..8f52235c96244 100644 --- a/paddle/fluid/framework/heter_service.h +++ b/paddle/fluid/framework/heter_service.h @@ -152,7 +152,7 @@ class HeterObjectPool { std::lock_guard lock(mutex_); if (pool_.empty()) { num_ += 1; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) VLOG(0) << "pool construct size: " << num_; #endif return std::make_shared(); diff --git a/paddle/fluid/framework/heterbox_trainer.cc b/paddle/fluid/framework/heterbox_trainer.cc index 3e55576b846dc..1f6dc39ae851d 100644 --- a/paddle/fluid/framework/heterbox_trainer.cc +++ b/paddle/fluid/framework/heterbox_trainer.cc @@ -21,9 +21,10 @@ limitations under the License. */ #include "paddle/fluid/framework/device_worker_factory.h" #include "paddle/fluid/framework/fleet/fleet_wrapper.h" #include "paddle/fluid/framework/trainer.h" -#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU) && \ +#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_HIP || \ + defined PADDLE_WITH_XPU) && \ (defined PADDLE_WITH_PSLIB) -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include "paddle/fluid/platform/cuda_device_guard.h" #endif namespace paddle { @@ -48,16 +49,25 @@ void HeterBoxTrainer::Initialize(const TrainerDesc& trainer_desc, dataset->GetReaders(); for (int i = 0; i < place_num; ++i) { int num = trainer_desc.worker_places(i); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) platform::CUDAPlace place = platform::CUDAPlace(num); platform::CUDADeviceGuard guard(place.device); - cudaStream_t stream; + gpuStream_t stream; +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamCreate(&stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamCreate(&stream)); +#endif copy_streams_.push_back(stream); places_.push_back(place); - cudaEvent_t event; + gpuEvent_t event; +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + hipEventCreateWithFlags(&event, hipEventDisableTiming)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( cudaEventCreateWithFlags(&event, cudaEventDisableTiming)); +#endif events_.push_back(event); #endif #ifdef PADDLE_WITH_XPU @@ -140,8 +150,13 @@ void HeterBoxTrainer::InitTrainerEnv(const ProgramDesc& main_program, _ForEachDataType_(HeterMemcpyFunc); } } +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipEventRecord(event, stream)); + hipEventSynchronize(event); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaEventRecord(event, stream)); cudaEventSynchronize(event); +#endif } place_ = place; } @@ -150,7 +165,7 @@ template void HeterBoxTrainer::HeterMemCpy(LoDTensor* thread_tensor, LoDTensor* root_tensor, const paddle::platform::Place& thread_place, - cudaStream_t stream) { + gpuStream_t stream) { T* thread_ptr = thread_tensor->mutable_data(root_tensor->dims(), thread_place); T* root_ptr = root_tensor->data(); @@ -171,7 +186,7 @@ void HeterBoxTrainer::InitOtherEnv(const ProgramDesc& main_program) { for (size_t i = 0; i < places_.size(); ++i) { pull_dense_worker_->AddThreadScope(workers_[i]->GetThreadScope()); pull_dense_worker_->AddPlace(places_[i]); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) pull_dense_worker_->AddStream(copy_streams_[i]); #endif } diff --git a/paddle/fluid/framework/io/crypto/cipher.cc b/paddle/fluid/framework/io/crypto/cipher.cc index 28767a68341df..eca175c020cb6 100644 --- a/paddle/fluid/framework/io/crypto/cipher.cc +++ b/paddle/fluid/framework/io/crypto/cipher.cc @@ -57,9 +57,4 @@ std::shared_ptr CipherFactory::CreateCipher( } } // namespace framework - -std::shared_ptr MakeCipher(const std::string& config_file) { - return framework::CipherFactory::CreateCipher(config_file); -} - } // namespace paddle diff --git a/paddle/fluid/framework/ir/graph_pattern_detector.cc b/paddle/fluid/framework/ir/graph_pattern_detector.cc index a38f10ba40814..deb182c0fbe19 100644 --- a/paddle/fluid/framework/ir/graph_pattern_detector.cc +++ b/paddle/fluid/framework/ir/graph_pattern_detector.cc @@ -790,27 +790,31 @@ PDNode *patterns::ConvBN::operator()(paddle::framework::ir::PDNode *conv_input, auto *bn_scale_var = pattern->NewNode(bn_scale_repr()) ->AsInput() ->assert_is_persistable_var() - ->assert_is_op_input("batch_norm", "Scale"); + ->assert_is_op_input("batch_norm", "Scale") + ->assert_has_n_outputs(1); // BN Bias auto *bn_bias_var = pattern->NewNode(bn_bias_repr()) ->AsInput() ->assert_is_persistable_var() - ->assert_is_op_input("batch_norm", "Bias"); + ->assert_is_op_input("batch_norm", "Bias") + ->assert_has_n_outputs(1); // BN Mean auto *bn_mean_var = pattern->NewNode(bn_mean_repr()) ->AsInput() ->assert_is_persistable_var() - ->assert_is_op_input("batch_norm", "Mean"); + ->assert_is_op_input("batch_norm", "Mean") + ->assert_has_n_outputs(1); // BN Variance auto *bn_variance_var = pattern->NewNode(bn_variance_repr()) ->AsInput() ->assert_is_persistable_var() - ->assert_is_op_input("batch_norm", "Variance"); + ->assert_is_op_input("batch_norm", "Variance") + ->assert_has_n_outputs(1); // BN output auto *bn_out_var = pattern->NewNode(bn_out_repr()) ->AsOutput() - ->assert_is_op_output("batch_norm"); + ->assert_is_op_output("batch_norm", "Y"); auto *bn_mean_out_var = pattern->NewNode(bn_mean_out_repr()) ->AsOutput() diff --git a/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.cc b/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.cc index 3d7a9c1107bba..531a04e1a0d4c 100644 --- a/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.cc +++ b/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.cc @@ -53,7 +53,7 @@ void CPUBfloat16PlacementPass::SetMkldnnDataType( gpd(graph, handler); } -void CPUBfloat16PlacementPass::RemoveOrhanedOperators( +void CPUBfloat16PlacementPass::RemoveOrphanedOperators( ir::Graph* graph, int* bfloat16_operators) const { // find orphaned bfloat16 operator that is between two float32 operators // revert mkldnn_data_type attr to float32 @@ -74,7 +74,7 @@ void CPUBfloat16PlacementPass::RemoveOrhanedOperators( void CPUBfloat16PlacementPass::ApplyImpl(ir::Graph* graph) const { int bfloat16_operators = 0; SetMkldnnDataType(graph, &bfloat16_operators); - RemoveOrhanedOperators(graph, &bfloat16_operators); + RemoveOrphanedOperators(graph, &bfloat16_operators); PrettyLogDetail("--- marked %d operators to bfloat16 ", bfloat16_operators); } diff --git a/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.h b/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.h index 1911b1a3cb32a..53b97f0e9726a 100644 --- a/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.h +++ b/paddle/fluid/framework/ir/mkldnn/cpu_bfloat16_placement_pass.h @@ -28,7 +28,7 @@ class CPUBfloat16PlacementPass : public Pass { protected: void SetMkldnnDataType(ir::Graph* graph, int* bfloat16_operators) const; - void RemoveOrhanedOperators(ir::Graph* graph, int* bfloat16_operators) const; + void RemoveOrphanedOperators(ir::Graph* graph, int* bfloat16_operators) const; void ApplyImpl(ir::Graph* graph) const override; }; diff --git a/paddle/fluid/framework/lod_tensor.h b/paddle/fluid/framework/lod_tensor.h index e09a628f49160..b8911154e6bf7 100644 --- a/paddle/fluid/framework/lod_tensor.h +++ b/paddle/fluid/framework/lod_tensor.h @@ -18,7 +18,7 @@ limitations under the License. */ #include #include #include -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include #include #endif diff --git a/paddle/fluid/framework/lod_tensor_test.cu b/paddle/fluid/framework/lod_tensor_test.cu index d58cfe447e88a..ddda7231887ed 100644 --- a/paddle/fluid/framework/lod_tensor_test.cu +++ b/paddle/fluid/framework/lod_tensor_test.cu @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include #include "gtest/gtest.h" @@ -34,8 +32,14 @@ TEST(LoD, data) { auto& v = lod[0]; paddle::platform::CUDAPlace gpu(0); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(test, dim3(1), dim3(1), 0, 0, v.CUDAMutableData(gpu), + v.size()); + hipDeviceSynchronize(); +#else test<<<1, 1>>>(v.CUDAMutableData(gpu), v.size()); cudaDeviceSynchronize(); +#endif for (size_t i = 0; i < v.size(); ++i) { EXPECT_EQ(v[i], i * 2); } @@ -59,8 +63,14 @@ TEST(LoDTensor, LoDInGPU) { auto lod = lod_tensor.lod(); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(test, dim3(1), dim3(8), 0, 0, + lod[0].CUDAMutableData(place), lod[0].size()); + hipDeviceSynchronize(); +#else test<<<1, 8>>>(lod[0].CUDAMutableData(place), lod[0].size()); cudaDeviceSynchronize(); +#endif for (size_t i = 0; i < src_lod[0].size(); ++i) { EXPECT_EQ(lod[0].data()[i], src_lod[0].data()[i] * 2); diff --git a/paddle/fluid/framework/mixed_vector.h b/paddle/fluid/framework/mixed_vector.h index 3a6e80f718d18..1e9b498bb2bfb 100644 --- a/paddle/fluid/framework/mixed_vector.h +++ b/paddle/fluid/framework/mixed_vector.h @@ -31,7 +31,7 @@ limitations under the License. */ namespace paddle { namespace framework { -#if defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // Vector implements the std::vector interface, and can get Data or // MutableData from any place. The data will be synced implicitly inside. template diff --git a/paddle/fluid/framework/mixed_vector_test.cu b/paddle/fluid/framework/mixed_vector_test.cu index 4b0caa8d350dd..8fb59d682e40f 100644 --- a/paddle/fluid/framework/mixed_vector_test.cu +++ b/paddle/fluid/framework/mixed_vector_test.cu @@ -12,7 +12,13 @@ See the License for the specific language governing permissions and limitations under the License. */ +#ifdef PADDLE_WITH_CUDA #include +#endif +#ifdef PADDLE_WITH_HIP +#include +#endif + #include #include "glog/logging.h" @@ -22,6 +28,7 @@ template using vec = paddle::framework::Vector; +using gpuStream_t = paddle::gpuStream_t; static __global__ void multiply_10(int* ptr) { for (int i = 0; i < 10; ++i) { @@ -29,7 +36,7 @@ static __global__ void multiply_10(int* ptr) { } } -cudaStream_t GetCUDAStream(paddle::platform::CUDAPlace place) { +gpuStream_t GetCUDAStream(paddle::platform::CUDAPlace place) { return reinterpret_cast( paddle::platform::DeviceContextPool::Instance().Get(place)) ->stream(); @@ -43,7 +50,12 @@ TEST(mixed_vector, GPU_VECTOR) { ASSERT_EQ(tmp.size(), 10UL); paddle::platform::CUDAPlace gpu(0); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(multiply_10, dim3(1), dim3(1), 0, GetCUDAStream(gpu), + tmp.MutableData(gpu)); +#else multiply_10<<<1, 1, 0, GetCUDAStream(gpu)>>>(tmp.MutableData(gpu)); +#endif for (int i = 0; i < 10; ++i) { ASSERT_EQ(tmp[i], i * 10); @@ -64,11 +76,23 @@ TEST(mixed_vector, MultiGPU) { ASSERT_EQ(tmp.size(), 10UL); paddle::platform::CUDAPlace gpu0(0); paddle::platform::SetDeviceId(0); + +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(multiply_10, dim3(1), dim3(1), 0, GetCUDAStream(gpu0), + tmp.MutableData(gpu0)); +#else multiply_10<<<1, 1, 0, GetCUDAStream(gpu0)>>>(tmp.MutableData(gpu0)); +#endif paddle::platform::CUDAPlace gpu1(1); auto* gpu1_ptr = tmp.MutableData(gpu1); paddle::platform::SetDeviceId(1); + +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(multiply_10, dim3(1), dim3(1), 0, GetCUDAStream(gpu1), + gpu1_ptr); +#else multiply_10<<<1, 1, 0, GetCUDAStream(gpu1)>>>(gpu1_ptr); +#endif for (int i = 0; i < 10; ++i) { ASSERT_EQ(tmp[i], i * 100); } diff --git a/paddle/fluid/framework/op_meta_info_helper.h b/paddle/fluid/framework/op_meta_info_helper.h index 06d9c94172df9..c70fe2f38ab63 100644 --- a/paddle/fluid/framework/op_meta_info_helper.h +++ b/paddle/fluid/framework/op_meta_info_helper.h @@ -17,7 +17,7 @@ limitations under the License. */ #include #include -#include "paddle/fluid/extension/include/op_meta_info.h" +#include "paddle/fluid/extension/include/ext_op_meta_info.h" namespace paddle { namespace framework { diff --git a/paddle/fluid/framework/op_registry.h b/paddle/fluid/framework/op_registry.h index e32ab8c7442e8..472c6f408266a 100644 --- a/paddle/fluid/framework/op_registry.h +++ b/paddle/fluid/framework/op_registry.h @@ -369,7 +369,7 @@ struct OpKernelRegistrarFunctorExIsType()) { - CheckTensorNANOrInf(type_, vname, var->Get()); - } else if (var->IsType()) { - CheckTensorNANOrInf(type_, vname, - var->Get().value()); - } - } - } +#if defined(PADDLE_WITH_HIP) + PADDLE_ENFORCE_CUDA_SUCCESS(hipGetLastError()); + VLOG(4) << "Operator(" << Type() << "): context wait and get last error"; +#endif } if (FLAGS_check_nan_inf) { diff --git a/paddle/fluid/framework/operator.h b/paddle/fluid/framework/operator.h index 4ad9bbd9d16cd..e9ecf9b5a8397 100644 --- a/paddle/fluid/framework/operator.h +++ b/paddle/fluid/framework/operator.h @@ -384,7 +384,7 @@ class ExecutionContext { return device_context_; } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) const inline platform::CUDADeviceContext& cuda_device_context() const { PADDLE_ENFORCE_EQ(platform::is_gpu_place(device_context_.GetPlace()), true, platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/framework/parallel_executor.cc b/paddle/fluid/framework/parallel_executor.cc index 83e812de39cef..2f280d5cc4ae0 100644 --- a/paddle/fluid/framework/parallel_executor.cc +++ b/paddle/fluid/framework/parallel_executor.cc @@ -37,7 +37,7 @@ limitations under the License. */ #include "paddle/fluid/platform/event.h" #include "paddle/fluid/platform/profiler.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include "paddle/fluid/platform/cuda_device_guard.h" #endif @@ -60,7 +60,7 @@ static std::once_flag gProfileOnce; static bool gProfileStarted = false; #endif -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) std::once_flag p2p_init_flag; #endif @@ -132,7 +132,7 @@ class ParallelExecutorPrivate { } } -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) void InitNCCLCtxs(framework::Scope *scope, const BuildStrategy &bst) { VLOG(1) << "nccl comm num:" << bst.nccl_comm_num_ << ", nranks:" << nranks_ << ", num_trainers:" << bst.num_trainers_ @@ -371,7 +371,7 @@ class ParallelExecutorPrivate { std::unordered_map is_persistable_; -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) platform::NCCLCommunicator *nccl_ctxs_{nullptr}; #elif defined(PADDLE_WITH_XPU_BKCL) platform::BKCLCommunicator *bkcl_ctxs_{nullptr}; @@ -483,7 +483,7 @@ ir::Graph *ParallelExecutorPrivate::ApplyMemoryOptimizePass(ir::Graph *graph) { } std::unique_ptr gc; if (platform::is_gpu_place(place)) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (IsFastEagerDeletionModeEnabled()) { gc.reset(new UnsafeFastGPUGarbageCollector( BOOST_GET_CONST(platform::CUDAPlace, place), max_memory_size)); @@ -572,7 +572,7 @@ bool ParallelExecutor::NeedCreateLocalExeScope() { } void InitP2P(const std::vector &places) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) std::call_once(p2p_init_flag, [&]() { int count = places.size(); if (count <= 1) return; @@ -590,14 +590,24 @@ void InitP2P(const std::vector &places) { for (int j = 0; j < count; ++j) { if (devices[i] == devices[j]) continue; int can_acess = -1; +#ifdef PADDLE_WITH_HIP + hipError_t ret = + hipDeviceCanAccessPeer(&can_acess, devices[i], devices[j]); + if (ret != hipSuccess || can_acess != 1) { +#else cudaError_t ret = cudaDeviceCanAccessPeer(&can_acess, devices[i], devices[j]); if (ret != cudaSuccess || can_acess != 1) { +#endif LOG(WARNING) << "Cannot enable P2P access from " << devices[i] << " to " << devices[j]; } else { platform::CUDADeviceGuard guard(devices[i]); +#ifdef PADDLE_WITH_HIP + hipDeviceEnablePeerAccess(devices[j], 0); +#else cudaDeviceEnablePeerAccess(devices[j], 0); +#endif } } } @@ -630,7 +640,7 @@ ParallelExecutor::ParallelExecutor(const std::vector &places, BuildStrategy::ReduceStrategy::kAllReduce; member_->use_all_reduce_ = true; } -#if defined(PADDLE_WITH_CUDA) && defined(_WIN32) +#if (defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)) && defined(_WIN32) if (member_->IsUseCUDA(member_->use_device_)) { PADDLE_ENFORCE_EQ( places.size(), 1, @@ -638,7 +648,8 @@ ParallelExecutor::ParallelExecutor(const std::vector &places, } #endif -#if defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_NCCL) +#if (defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)) && \ + (!defined(PADDLE_WITH_NCCL) && !defined(PADDLE_WITH_RCCL)) if (member_->IsUseCUDA(member_->use_device_)) { PADDLE_ENFORCE_EQ( places.size(), 1, @@ -710,7 +721,7 @@ ParallelExecutor::ParallelExecutor(const std::vector &places, } if (member_->IsUseCUDA(member_->use_device_) && member_->nranks_ > 1) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) member_->InitOrGetNCCLCommunicator(scope, &member_->build_strategy_); // Initialize device context's nccl comm, will be used by normal @@ -774,7 +785,7 @@ ParallelExecutor::ParallelExecutor(const std::vector &places, // Step 2. Convert main_program to SSA form and dependency graph. Also, insert // ncclOp std::vector async_graphs(places.size()); -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) if (member_->build_strategy_.async_mode_) { VLOG(3) << "use local async mode"; graph = member_->build_strategy_.Apply( @@ -885,7 +896,7 @@ ParallelExecutor::ParallelExecutor(const std::vector &places, final_graphs = async_graphs; } else if (member_->build_strategy_.enable_parallel_graph_) { VLOG(3) << "use ParallelSSAGraphExecutor"; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // TODO(Yancey1989): Remove passing in the main_program when // allreduce_seq_pass doesn't need it as the attr. bool is_inference = details::IsDataParallelInferenceGraph(*graph); @@ -996,7 +1007,7 @@ void ParallelExecutor::BCastParamsToDevices( } auto &dims = main_tensor.dims(); if (paddle::platform::is_gpu_place(main_tensor.place())) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) std::vector buffers; buffers.reserve(member_->places_.size()); size_t numel = main_tensor.numel(); diff --git a/paddle/fluid/framework/parallel_executor.h b/paddle/fluid/framework/parallel_executor.h index 0a1df2f194605..47de7dc48f4f2 100644 --- a/paddle/fluid/framework/parallel_executor.h +++ b/paddle/fluid/framework/parallel_executor.h @@ -32,7 +32,7 @@ limitations under the License. */ #include "paddle/fluid/framework/tensor.h" #include "paddle/fluid/platform/device_context.h" -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/nccl_helper.h" #endif diff --git a/paddle/fluid/framework/pipeline_trainer.cc b/paddle/fluid/framework/pipeline_trainer.cc index 01ab494adef54..a97fc2e75aab1 100644 --- a/paddle/fluid/framework/pipeline_trainer.cc +++ b/paddle/fluid/framework/pipeline_trainer.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/framework/data_feed_factory.h" #include "paddle/fluid/framework/device_worker_factory.h" #include "paddle/fluid/framework/trainer.h" @@ -24,6 +24,9 @@ namespace framework { void PipelineTrainer::Initialize(const TrainerDesc& trainer_desc, Dataset* dataset) { const auto& section_params = trainer_desc.section_param(); + const int num_pipeline_stages_ = section_params.num_pipeline_stages(); + const int pipeline_stage_ = section_params.pipeline_stage(); + const int schedule_mode_ = section_params.schedule_mode(); num_microbatches_ = section_params.num_microbatches(); VLOG(3) << "Number of microbatches per minibatch: " << num_microbatches_; trainer_desc_ = trainer_desc; @@ -39,6 +42,9 @@ void PipelineTrainer::Initialize(const TrainerDesc& trainer_desc, this_worker->SetPlace(place_); this_worker->Initialize(trainer_desc); this_worker->SetMicrobatchNum(num_microbatches_); + this_worker->SetPipelineStageNum(num_pipeline_stages_); + this_worker->SetPipelineStage(pipeline_stage_); + this_worker->SetScheduleMode(schedule_mode_); } void PipelineTrainer::InitOtherEnv(const ProgramDesc& main_program) { @@ -75,7 +81,9 @@ void PipelineTrainer::CopyParameters(int microbatch_id, for (auto& var : global_block.AllVars()) { bool is_param_grad = false; size_t pos = 0; - if ((pos = var->Name().find(kGradVarSuffix)) != std::string::npos) { + // A magic suffix to indicate the merged gradient + std::string magicSuffix = std::string(kGradVarSuffix) + "@MERGED"; + if ((pos = var->Name().find(magicSuffix)) != std::string::npos) { auto prefix_name = var->Name().substr(0, pos); if (param_map.find(prefix_name) != param_map.end()) { is_param_grad = true; diff --git a/paddle/fluid/framework/ps_gpu_trainer.cc b/paddle/fluid/framework/ps_gpu_trainer.cc index bca1843dd8f23..e77932fa5f226 100644 --- a/paddle/fluid/framework/ps_gpu_trainer.cc +++ b/paddle/fluid/framework/ps_gpu_trainer.cc @@ -24,7 +24,8 @@ limitations under the License. */ #include "paddle/fluid/framework/fleet/heter_ps/feature_value.h" #include "paddle/fluid/framework/fleet/ps_gpu_wrapper.h" #include "paddle/fluid/framework/trainer.h" -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) #include "paddle/fluid/platform/cuda_device_guard.h" namespace paddle { @@ -130,8 +131,13 @@ void PSGPUTrainer::InitOtherEnv(const ProgramDesc& main_program) { void PSGPUTrainer::Run() { for (size_t thidx = 0; thidx < places_.size(); ++thidx) { - threads_.push_back( - std::thread(&DeviceWorker::TrainFiles, workers_[thidx].get())); + if (!debug_) { + threads_.push_back( + std::thread(&DeviceWorker::TrainFiles, workers_[thidx].get())); + } else { + threads_.push_back(std::thread(&DeviceWorker::TrainFilesWithProfiler, + workers_[thidx].get())); + } } } diff --git a/paddle/fluid/framework/ps_gpu_worker.cc b/paddle/fluid/framework/ps_gpu_worker.cc index d75a32a88028e..2597901d91f36 100644 --- a/paddle/fluid/framework/ps_gpu_worker.cc +++ b/paddle/fluid/framework/ps_gpu_worker.cc @@ -19,7 +19,8 @@ limitations under the License. */ #include "paddle/fluid/platform/cpu_helper.h" #include "paddle/fluid/string/string_helper.h" -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) #include "paddle/fluid/platform/cuda_device_guard.h" #if defined _WIN32 || defined __APPLE__ @@ -32,6 +33,7 @@ namespace framework { void PSGPUWorker::Initialize(const TrainerDesc& desc) { param_ = desc.downpour_param(); + dev_ctx_ = platform::DeviceContextPool::Instance().Get(place_); mpi_rank_ = desc.mpi_rank(); trainer_desc_ = desc; /* @@ -176,6 +178,81 @@ void PSGPUWorker::TrainFiles() { return; } +void PSGPUWorker::TrainFilesWithProfiler() { + platform::SetNumThreads(1); + VLOG(1) << "Begin to train files with profiler"; + device_reader_->Start(); + std::vector op_total_time; + std::vector op_name; + for (auto& op : ops_) { + bool need_skip = false; + for (auto t = 0u; t < skip_ops_.size(); ++t) { + if (op->Type().find(skip_ops_[t]) != std::string::npos) { + need_skip = true; + break; + } + } + if (!need_skip) { + op_name.push_back(op->Type()); + } + } + + VLOG(3) << "op name size: " << op_name.size(); + op_total_time.resize(op_name.size()); + for (size_t i = 0; i < op_total_time.size(); ++i) { + op_total_time[i] = 0.0; + } + platform::Timer timeline; + double total_time = 0.0; + double read_time = 0.0; + int total_ins_num = 0; + int cur_batch; + timeline.Start(); + while ((cur_batch = device_reader_->Next()) > 0) { + total_ins_num += cur_batch; + timeline.Pause(); + read_time += timeline.ElapsedSec(); + total_time += timeline.ElapsedSec(); + + int run_op_idx = 0; + dev_ctx_->Wait(); + for (auto& op : ops_) { + bool need_skip = false; + for (auto t = 0u; t < skip_ops_.size(); ++t) { + if (op->Type().find(skip_ops_[t]) != std::string::npos) { + need_skip = true; + break; + } + } + if (!need_skip) { + timeline.Start(); + VLOG(3) << "Going to run op " << op_name[run_op_idx]; + op->Run(*thread_scope_, place_); + dev_ctx_->Wait(); + VLOG(3) << "Op " << op_name[run_op_idx] << " Finished"; + timeline.Pause(); + op_total_time[run_op_idx++] += timeline.ElapsedSec(); + total_time += timeline.ElapsedSec(); + } + } + timeline.Start(); + PrintFetchVars(); + thread_scope_->DropKids(); + dev_ctx_->Wait(); + timeline.Pause(); + total_time += timeline.ElapsedSec(); + timeline.Start(); + } + VLOG(1) << "GpuPs worker " << thread_id_ << " train cost " << total_time + << " seconds, ins_num: " << total_ins_num; + for (size_t i = 0; i < op_name.size(); ++i) { + VLOG(1) << "card:" << thread_id_ << ", op: " << op_name[i] + << ", mean time: " << op_total_time[i] / total_ins_num + << "s, totol time:" << op_total_time[i] << "sec"; + } + return; +} + void PSGPUWorker::ResetStat() { total_time_ = 0; read_time_ = 0; diff --git a/paddle/fluid/framework/pull_dense_worker.cc b/paddle/fluid/framework/pull_dense_worker.cc index fb268e4b6cf02..77d8abcd26e9e 100644 --- a/paddle/fluid/framework/pull_dense_worker.cc +++ b/paddle/fluid/framework/pull_dense_worker.cc @@ -59,17 +59,19 @@ void PullDenseWorker::Initialize(const TrainerDesc& param) { current_version_[tid] = 0; } fleet_ptr_ = FleetWrapper::GetInstance(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) copy_streams_.clear(); #endif -#if (defined PADDLE_WITH_CUDA) || (defined PADDLE_WITH_XPU) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) places_.clear(); thread_scopes_.clear(); #endif } void PullDenseWorker::CreatePinVar() { -#if (defined PADDLE_WITH_CUDA) || (defined PADDLE_WITH_XPU) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) // for (auto& v : dense_value_names_) { // for (auto& name : v.second) { for (int i = 0; i < dwp_param_.program_config(0).pull_dense_table_id_size(); @@ -84,7 +86,7 @@ void PullDenseWorker::CreatePinVar() { auto* ptr = root_scope_->Var(name + "pin"); InitializeVariable(ptr, proto::VarType::LOD_TENSOR); LoDTensor* pin_tensor = ptr->GetMutable(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) pin_tensor->mutable_data(tensor->dims(), platform::CUDAPinnedPlace()); #endif @@ -113,7 +115,8 @@ void PullDenseWorker::Wait(std::vector<::std::future>* status_vec) { exit(-1); } status_vec->resize(0); -#if (defined PADDLE_WITH_CUDA) || (defined PADDLE_WITH_XPU) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) for (size_t i = 0; i < places_.size(); ++i) { // for (auto& v : dense_value_names_) { @@ -131,7 +134,7 @@ void PullDenseWorker::Wait(std::vector<::std::future>* status_vec) { Variable* var = thread_scopes_[i]->FindVar(name); LoDTensor* tensor = var->GetMutable(); float* w = tensor->data(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) memory::Copy(BOOST_GET_CONST(platform::CUDAPlace, places_[i]), w, platform::CUDAPinnedPlace(), pin_w, sizeof(float) * tensor->numel(), copy_streams_[i]); @@ -161,7 +164,8 @@ void PullDenseWorker::PullDense(bool force_update) { uint64_t tid = static_cast( dwp_param_.program_config(0).pull_dense_table_id(i)); if (force_update || CheckUpdateParam(tid)) { -#if (defined PADDLE_WITH_CUDA) || (defined PADDLE_WITH_XPU) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) VLOG(3) << "pull dense " << force_update << " " << tid; fleet_ptr_->PullDenseVarsAsync(*root_scope_, tid, dense_value_names_[tid], &pull_dense_status_, false); diff --git a/paddle/fluid/framework/save_load_util.cc b/paddle/fluid/framework/save_load_util.cc index bd5725f49c0e5..1731a974b71d8 100644 --- a/paddle/fluid/framework/save_load_util.cc +++ b/paddle/fluid/framework/save_load_util.cc @@ -297,7 +297,7 @@ bool SaveTensorToDisk(const std::string& file_name, tensor->numel() * framework::SizeOfType(tensor->type()); auto* data_ptr = tensor->data(); if (platform::is_gpu_place(tensor->place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) framework::Tensor temp; TensorCopySync(*tensor, platform::CPUPlace(), &temp); data_ptr = temp.data(); diff --git a/paddle/fluid/framework/section_worker.cc b/paddle/fluid/framework/section_worker.cc index 6e17551818c4d..90a371e474756 100644 --- a/paddle/fluid/framework/section_worker.cc +++ b/paddle/fluid/framework/section_worker.cc @@ -9,7 +9,7 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include #include "paddle/fluid/framework/device_worker.h" #include "paddle/fluid/framework/executor_gc_helper.h" @@ -22,15 +22,79 @@ class TrainerDesc; uint64_t SectionWorker::batch_id_(0); -void SectionWorker::Initialize(const TrainerDesc& desc) { +void SectionWorker::Initialize(const TrainerDesc &desc) { dev_ctx_ = platform::DeviceContextPool::Instance().Get(place_); program_.reset( new ProgramDesc(desc.section_param().section_config().program_desc())); - for (auto& op_desc : program_->Block(0).AllOps()) { + for (auto &op_desc : program_->Block(0).AllOps()) { ops_.push_back(OpRegistry::CreateOp(*op_desc)); } } +void SectionWorker::RunForward( + int micro_id, std::unique_ptr &gc, + std::unordered_map> + &unused_vars_) { + for (auto &op : ops_) { + int op_role = op->Attr(std::string("op_role")); + // We run op with op_role = kLRSched only for the first microbatch + // to avoid increasing the @LR_DECAY_STEP@ multiple times. + bool run_first_mbatch = op_role == static_cast(OpRole::kForward) || + op_role == (static_cast(OpRole::kForward) | + static_cast(OpRole::kLoss)) || + op_role == static_cast(OpRole::kLRSched); + bool run_others = op_role == static_cast(OpRole::kForward) || + op_role == (static_cast(OpRole::kForward) | + static_cast(OpRole::kLoss)); + if ((micro_id == 0 && run_first_mbatch) || (micro_id != 0 && run_others)) { + VLOG(3) << "Forward: running op " << op->Type() << " for micro-batch " + << micro_id; + op->Run(*microbatch_scopes_[micro_id], place_); + if (gc) { + DeleteUnusedTensors(*microbatch_scopes_[micro_id], op.get(), + unused_vars_, gc.get()); + } + } + } +} + +void SectionWorker::RunBackward( + int micro_id, std::unique_ptr &gc, + std::unordered_map> + &unused_vars_) { + for (auto &op : ops_) { + int op_role = op->Attr(std::string("op_role")); + if (op_role == static_cast(OpRole::kBackward) || + op_role == (static_cast(OpRole::kBackward) | + static_cast(OpRole::kLoss))) { + VLOG(3) << "Backward: running op " << op->Type() << " for micro-batch " + << micro_id; + op->Run(*microbatch_scopes_[micro_id], place_); + if (gc) { + DeleteUnusedTensors(*microbatch_scopes_[micro_id], op.get(), + unused_vars_, gc.get()); + } + } + } +} + +void SectionWorker::RunUpdate( + std::unique_ptr &gc, + std::unordered_map> + &unused_vars_) { + for (auto &op : ops_) { + int op_role = op->Attr(std::string("op_role")); + if (op_role == static_cast(OpRole::kOptimize)) { + VLOG(3) << "Update: running op " << op->Type(); + op->Run(*microbatch_scopes_[num_microbatches_ - 1], place_); + if (gc) { + DeleteUnusedTensors(*microbatch_scopes_[num_microbatches_ - 1], + op.get(), unused_vars_, gc.get()); + } + } + } +} + void SectionWorker::TrainFiles() { VLOG(5) << "begin section_worker TrainFiles"; @@ -38,7 +102,7 @@ void SectionWorker::TrainFiles() { std::unique_ptr gc; auto unused_vars_ = GetUnusedVars(program_->Block(0), ops_, skip_vars_); if (max_memory_size >= 0) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(place_)) { if (IsFastEagerDeletionModeEnabled()) { gc.reset(new UnsafeFastGPUGarbageCollector( @@ -48,61 +112,56 @@ void SectionWorker::TrainFiles() { #endif } - for (int i = 0; i < num_microbatches_; ++i) { - for (auto& op : ops_) { - int op_role = op->Attr(std::string("op_role")); - // We run op with op_role = kLRSched only for the first microbatch - // to avoid increasing the @LR_DECAY_STEP@ multiple times. - bool run_first_mbatch = op_role == static_cast(OpRole::kForward) || - op_role == (static_cast(OpRole::kForward) | - static_cast(OpRole::kLoss)) || - op_role == static_cast(OpRole::kLRSched); - bool run_others = op_role == static_cast(OpRole::kForward) || - op_role == (static_cast(OpRole::kForward) | - static_cast(OpRole::kLoss)); - if ((i == 0 && run_first_mbatch) || (i != 0 && run_others)) { - VLOG(3) << "Forward: running op " << op->Type() << " for micro-batch " - << i; - op->Run(*microbatch_scopes_[i], place_); - if (gc) { - DeleteUnusedTensors(*microbatch_scopes_[i], op.get(), unused_vars_, - gc.get()); - } - } + if (schedule_mode_ == 0) { + // F-then-B scheduler which runs Forward phase for all microbatches, + // then runs Backward phase for all microbatches. + // step1: run forward + for (int i = 0; i < num_microbatches_; ++i) { + RunForward(i, gc, unused_vars_); } - cudaDeviceSynchronize(); - } - - // backward pass - for (int i = 0; i < num_microbatches_; ++i) { - for (auto& op : ops_) { - int op_role = op->Attr(std::string("op_role")); - if (op_role == static_cast(OpRole::kBackward) || - op_role == (static_cast(OpRole::kBackward) | - static_cast(OpRole::kLoss))) { - VLOG(3) << "Backward: running op " << op->Type() << " for micro-batch " - << i; - op->Run(*microbatch_scopes_[i], place_); - if (gc) { - DeleteUnusedTensors(*microbatch_scopes_[i], op.get(), unused_vars_, - gc.get()); - } - } + // step2: run backward + for (int i = 0; i < num_microbatches_; ++i) { + RunBackward(i, gc, unused_vars_); + } + // step3: run update + RunUpdate(gc, unused_vars_); + } else { + // 1F1B scheduler, which runs forward phase and backward phase altertively + // after startup phase. For a stage, the number of microbatches for + // startup is num_pipeline_stages_ - pipeline_stage_ - 1, where + // num_pipeline_stages_ is the total number of pipeline stages and + // pipeline_stage_ is the pipeline stage of the current device. + auto startup_steps = num_pipeline_stages_ - pipeline_stage_ - 1; + VLOG(3) << "startup_steps:" << startup_steps + << ", num_stages: " << num_pipeline_stages_ + << ", stage:" << pipeline_stage_; + PADDLE_ENFORCE_GT( + num_microbatches_, startup_steps, + platform::errors::InvalidArgument( + "To use pipeline with 1F1B scheduler, please make sure number of " + "microbatches (%d) is than startup steps (%d).", + num_microbatches_, startup_steps)); + int fw_step = 0; + int bw_step = 0; + // startup phase + while (fw_step < startup_steps) { + RunForward(fw_step, gc, unused_vars_); + fw_step += 1; } - cudaDeviceSynchronize(); - } - // update pass - for (auto& op : ops_) { - int op_role = op->Attr(std::string("op_role")); - if (op_role == static_cast(OpRole::kOptimize)) { - VLOG(3) << "Update: running op " << op->Type(); - op->Run(*microbatch_scopes_[0], place_); - if (gc) { - DeleteUnusedTensors(*microbatch_scopes_[0], op.get(), unused_vars_, - gc.get()); - } + // 1f1b phase + while (fw_step < num_microbatches_) { + RunForward(fw_step, gc, unused_vars_); + fw_step += 1; + RunBackward(bw_step, gc, unused_vars_); + bw_step += 1; + } + // backward phase + while (bw_step < num_microbatches_) { + RunBackward(bw_step, gc, unused_vars_); + bw_step += 1; } + RunUpdate(gc, unused_vars_); } dev_ctx_->Wait(); ++batch_id_; diff --git a/paddle/fluid/framework/tensor_test.cc b/paddle/fluid/framework/tensor_test.cc index 92a29d5165ce7..54f7798130633 100644 --- a/paddle/fluid/framework/tensor_test.cc +++ b/paddle/fluid/framework/tensor_test.cc @@ -118,7 +118,7 @@ TEST(Tensor, MutableData) { EXPECT_EQ(static_cast(p2[0]), 1); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { framework::Tensor src_tensor; float* p1 = nullptr; @@ -174,7 +174,7 @@ TEST(Tensor, ShareDataWith) { ASSERT_EQ(src_tensor.data(), dst_tensor.data()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { framework::Tensor src_tensor; framework::Tensor dst_tensor; @@ -212,7 +212,7 @@ TEST(Tensor, Slice) { EXPECT_EQ(src_data_address + 3 * 4 * 1 * sizeof(int), slice_data_address); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { framework::Tensor src_tensor; src_tensor.mutable_data(framework::make_ddim({6, 9}), diff --git a/paddle/fluid/framework/tensor_util.cc b/paddle/fluid/framework/tensor_util.cc index 1ad321df216fe..c6ac30a369859 100644 --- a/paddle/fluid/framework/tensor_util.cc +++ b/paddle/fluid/framework/tensor_util.cc @@ -97,7 +97,7 @@ void TensorCopy(const Tensor& src, const platform::Place& dst_place, "Copy from %s to %s is not supported.", src_place, dst_place)); } #endif -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) else if (platform::is_cuda_pinned_place(src_place) && // NOLINT platform::is_cuda_pinned_place(dst_place)) { memory::Copy(BOOST_GET_CONST(platform::CUDAPinnedPlace, dst_place), dst_ptr, @@ -304,7 +304,7 @@ void TensorCopySync(const Tensor& src, const platform::Place& dst_place, "Copy from %s to %s is not supported.", src_place, dst_place)); } #endif -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) else if (platform::is_cuda_pinned_place(src_place) && // NOLINT platform::is_cuda_pinned_place(dst_place)) { memory::Copy(BOOST_GET_CONST(platform::CUDAPinnedPlace, dst_place), dst_ptr, @@ -595,7 +595,7 @@ bool TensorIsfinite(const framework::Tensor& tensor) { return !Any(tensor, pred_inf) && !Any(tensor, pred_nan); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template static inline void __global__ BothFalse(const T* cmp, T* out, int element_num) { CUDA_KERNEL_LOOP(i, element_num) { out[i] = (!cmp[i]) && (!out[i]); } @@ -618,7 +618,7 @@ struct BothFalseVisitor : public boost::static_visitor<> { } void VisitorImpl(const platform::CUDAPlace& gpu) const { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto* ctx = platform::DeviceContextPool::Instance().GetByPlace(gpu); constexpr int MAX_BLOCK_DIM = 512; const int MAX_GRID_DIM = ctx->GetMaxPhysicalThreadCount() / MAX_BLOCK_DIM; @@ -703,7 +703,7 @@ void TensorToStream(std::ostream& os, const Tensor& tensor, platform::errors::ResourceExhausted( "tensor size %d overflow when writing tensor", size)); if (platform::is_gpu_place(tensor.place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) constexpr size_t kBufSize = 1024 * 1024 * 64; // 64MB std::unique_ptr buf(new char[kBufSize]); auto& gpu_dev_ctx = @@ -802,7 +802,8 @@ void TensorFromStream(std::istream& is, Tensor* tensor, size_t size = tensor->numel() * framework::SizeOfType(desc.data_type()); if (platform::is_gpu_place(dev_ctx.GetPlace()) || platform::is_xpu_place(dev_ctx.GetPlace())) { -#if defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) Tensor cpu_tensor; cpu_tensor.Resize(framework::make_ddim(shape)); framework::VisitDataType( @@ -859,7 +860,8 @@ void TensorFromStream(std::istream& is, Tensor* tensor, size_t size = tensor->numel() * framework::SizeOfType(desc.data_type()); if (platform::is_gpu_place(dev_ctx.GetPlace()) || platform::is_xpu_place(dev_ctx.GetPlace())) { -#if defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) || \ + defined(PADDLE_WITH_XPU) Tensor cpu_tensor; cpu_tensor.Resize(framework::make_ddim(dims)); framework::VisitDataType( @@ -954,7 +956,7 @@ void TensorFromDLPack(const ::DLTensor& dl_tensor, framework::Tensor* dst) { if (dl_tensor.ctx.device_type == kDLCPU) { memory::Copy(dst_place, dst_ptr, src_place, src_ptr, size); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (dl_tensor.ctx.device_type == kDLGPU) { platform::CUDAPlace dst_place = platform::CUDAPlace(dl_tensor.ctx.device_id); diff --git a/paddle/fluid/framework/tensor_util.h b/paddle/fluid/framework/tensor_util.h index 50644370bc6b6..8a127e0ed5929 100644 --- a/paddle/fluid/framework/tensor_util.h +++ b/paddle/fluid/framework/tensor_util.h @@ -127,7 +127,7 @@ void TensorFromArray(const T* src, const size_t& array_size, memory::Copy(BOOST_GET_CONST(platform::CPUPlace, dst_place), dst_ptr, src_place, src_ptr, size); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) else if (platform::is_gpu_place(dst_place)) { // NOLINT memory::Copy( BOOST_GET_CONST(platform::CUDAPlace, dst_place), dst_ptr, src_place, @@ -150,7 +150,7 @@ void TensorFromVector(const std::vector& src, memory::Copy(BOOST_GET_CONST(platform::CPUPlace, dst_place), dst_ptr, src_place, src_ptr, size); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) else if (platform::is_gpu_place(dst_place)) { // NOLINT memory::Copy( BOOST_GET_CONST(platform::CUDAPlace, dst_place), dst_ptr, src_place, @@ -187,7 +187,7 @@ void TensorToVector(const Tensor& src, const platform::DeviceContext& ctx, BOOST_GET_CONST(platform::CPUPlace, src.place()), src_ptr, size); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) else if (platform::is_gpu_place(src.place())) { // NOLINT memory::Copy( dst_place, dst_ptr, BOOST_GET_CONST(platform::CUDAPlace, src.place()), diff --git a/paddle/fluid/framework/tensor_util_test.cc b/paddle/fluid/framework/tensor_util_test.cc index e389cb34679a2..c32efd0a470be 100644 --- a/paddle/fluid/framework/tensor_util_test.cc +++ b/paddle/fluid/framework/tensor_util_test.cc @@ -58,7 +58,7 @@ TEST(TensorCopy, Tensor) { } EXPECT_TRUE(dst_tensor.layout() == src_tensor.layout()); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { Tensor src_tensor; Tensor gpu_tensor; @@ -149,7 +149,7 @@ TEST(TensorFromVector, Tensor) { delete cpu_place; } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { std::vector src_vec = {1, 2, 3, 4, 5, 6, 7, 8, 9}; paddle::framework::Tensor cpu_tensor; @@ -224,7 +224,7 @@ TEST(TensorToVector, Tensor) { EXPECT_EQ(src_ptr[i], dst[i]); } } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { std::vector src_vec = {1, 2, 3, 4, 5, 6, 7, 8, 9}; paddle::framework::Tensor gpu_tensor; @@ -264,7 +264,7 @@ TEST(TensorFromDLPack, Tensor) { } } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { std::vector src_vec = {1, 2, 3, 4, 5, 6, 7, 8, 9}; paddle::framework::Tensor cpu_tensor; @@ -430,7 +430,7 @@ TEST(Tensor, FromAndToStream) { EXPECT_EQ(dst_tensor.dims(), src_tensor.dims()); delete place; } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) { Tensor gpu_tensor; gpu_tensor.Resize({2, 3}); diff --git a/paddle/fluid/framework/tensor_util_test.cu b/paddle/fluid/framework/tensor_util_test.cu index a51f74199e714..4517726a5c09d 100644 --- a/paddle/fluid/framework/tensor_util_test.cu +++ b/paddle/fluid/framework/tensor_util_test.cu @@ -63,7 +63,11 @@ TEST(TensorContainsNAN, GPU) { { Tensor tensor; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); ASSERT_TRUE(TensorContainsNAN(tensor)); } @@ -71,7 +75,11 @@ TEST(TensorContainsNAN, GPU) { Tensor tensor; paddle::platform::float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); ASSERT_TRUE(TensorContainsNAN(tensor)); } @@ -84,7 +92,11 @@ TEST(TensorContainsInf, GPU) { { Tensor tensor; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); ASSERT_TRUE(TensorContainsInf(tensor)); } @@ -92,7 +104,11 @@ TEST(TensorContainsInf, GPU) { Tensor tensor; paddle::platform::float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); ASSERT_TRUE(TensorContainsInf(tensor)); } @@ -107,14 +123,22 @@ TEST(TensorIsfinite, GPU) { { Tensor tensor; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(!TensorIsfinite(tensor)); } { Tensor tensor; float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(!TensorIsfinite(tensor)); } @@ -123,14 +147,22 @@ TEST(TensorIsfinite, GPU) { { Tensor tensor; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(!TensorIsfinite(tensor)); } { Tensor tensor; float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(!TensorIsfinite(tensor)); } @@ -139,14 +171,24 @@ TEST(TensorIsfinite, GPU) { { Tensor tensor; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillFinite, dim3(1), dim3(1), 0, cuda_ctx->stream(), + buf); +#else FillFinite<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(TensorIsfinite(tensor)); } { Tensor tensor; float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillFinite, dim3(1), dim3(1), 0, cuda_ctx->stream(), + buf); +#else FillFinite<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); EXPECT_TRUE(TensorIsfinite(tensor)); } @@ -159,7 +201,11 @@ TEST(TensorContainsInf, GPUWithoutWait) { { Tensor tensor, out; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorContainsInf(tensor, &out); platform::CPUPlace cpu; @@ -172,7 +218,11 @@ TEST(TensorContainsInf, GPUWithoutWait) { Tensor tensor, out; paddle::platform::float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorContainsInf(tensor, &out); platform::CPUPlace cpu; @@ -190,7 +240,11 @@ TEST(TensorContainsNAN, GPUWithoutWait) { { Tensor tensor, out; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorContainsNAN(tensor, &out); platform::CPUPlace cpu; @@ -203,7 +257,11 @@ TEST(TensorContainsNAN, GPUWithoutWait) { Tensor tensor, out; paddle::platform::float16* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorContainsNAN(tensor, &out); platform::CPUPlace cpu; @@ -221,7 +279,11 @@ TEST(TensorIsfinite, GPUWithoutWait) { { Tensor tensor, out; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillInf, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillInf<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorIsfinite(tensor, &out); platform::CPUPlace cpu; @@ -233,7 +295,11 @@ TEST(TensorIsfinite, GPUWithoutWait) { { Tensor tensor, out; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillNAN, dim3(1), dim3(1), 0, cuda_ctx->stream(), buf); +#else FillNAN<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorIsfinite(tensor, &out); platform::CPUPlace cpu; @@ -245,7 +311,12 @@ TEST(TensorIsfinite, GPUWithoutWait) { { Tensor tensor, out; float* buf = tensor.mutable_data({3}, gpu); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(FillFinite, dim3(1), dim3(1), 0, cuda_ctx->stream(), + buf); +#else FillFinite<<<1, 1, 0, cuda_ctx->stream()>>>(buf); +#endif cuda_ctx->Wait(); TensorIsfinite(tensor, &out); platform::CPUPlace cpu; diff --git a/paddle/fluid/framework/trainer.h b/paddle/fluid/framework/trainer.h index d949ba2bffe6c..ca290a50b42fe 100644 --- a/paddle/fluid/framework/trainer.h +++ b/paddle/fluid/framework/trainer.h @@ -141,7 +141,8 @@ class DistMultiTrainer : public MultiTrainer { std::shared_ptr pull_dense_worker_; }; -#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU) && \ +#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_HIP || \ + defined PADDLE_WITH_XPU) && \ (defined PADDLE_WITH_PSLIB) class HeterServiceContext { public: @@ -155,8 +156,9 @@ class HeterServiceContext { void Reset() { push_dense_status_.clear(); } int place_num_; Scope* scope_{nullptr}; -#ifdef PADDLE_WITH_CUDA - cudaEvent_t event_; + +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + gpuEvent_t event_; #endif std::vector ops_; std::vector<::std::future> push_dense_status_; @@ -187,10 +189,10 @@ class HeterXpuTrainer : public TrainerBase { virtual std::string GetDumpPath(int tid) { return ""; } virtual void InitDumpEnv() {} template -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) void HeterMemCpy(LoDTensor* tensor, LoDTensor* root_tensor, const paddle::platform::Place& thread_place, - cudaStream_t stream); + gpuStream_t stream); #endif #ifdef PADDLE_WITH_XPU void HeterMemCpy(LoDTensor* thread_tensor, LoDTensor* root_tensor, @@ -222,9 +224,9 @@ class HeterXpuTrainer : public TrainerBase { std::vector place_scopes_; BtObjectPool object_pool_; std::vector places_; -#ifdef PADDLE_WITH_CUDA - std::vector copy_streams_; - std::vector events_; +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + std::vector copy_streams_; + std::vector events_; #endif }; @@ -247,10 +249,10 @@ class HeterBoxTrainer : public TrainerBase { virtual std::string GetDumpPath(int tid) { return ""; } virtual void InitDumpEnv() {} template -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) void HeterMemCpy(LoDTensor* tensor, LoDTensor* root_tensor, const paddle::platform::Place& thread_place, - cudaStream_t stream); + gpuStream_t stream); #endif void CreateThreadParam(const ProgramDesc& program, int num); template @@ -272,14 +274,15 @@ class HeterBoxTrainer : public TrainerBase { std::vector threads_; int use_ps_gpu_; int thread_num_; -#ifdef PADDLE_WITH_CUDA - std::vector copy_streams_; - std::vector events_; +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + std::vector copy_streams_; + std::vector events_; #endif }; #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) class PSGPUTrainer : public TrainerBase { public: PSGPUTrainer() {} @@ -321,7 +324,7 @@ class PSGPUTrainer : public TrainerBase { }; #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) class PipelineTrainer : public TrainerBase { public: PipelineTrainer() {} diff --git a/paddle/fluid/framework/trainer_desc.proto b/paddle/fluid/framework/trainer_desc.proto index 70481cf372701..504885ff5ccbc 100644 --- a/paddle/fluid/framework/trainer_desc.proto +++ b/paddle/fluid/framework/trainer_desc.proto @@ -93,6 +93,9 @@ message SectionWorkerParameter { optional int32 start_cpu_core_id = 4 [ default = 1 ]; repeated string param_need_sync = 5; optional int32 num_microbatches = 6; + optional int32 num_pipeline_stages = 7 [ default = 1 ]; + optional int32 pipeline_stage = 8 [ default = 1 ]; + optional int32 schedule_mode = 9 [ default = 0 ]; } message SectionConfig { diff --git a/paddle/fluid/framework/trainer_factory.cc b/paddle/fluid/framework/trainer_factory.cc index 764338a8cc671..6b9dbece8974c 100644 --- a/paddle/fluid/framework/trainer_factory.cc +++ b/paddle/fluid/framework/trainer_factory.cc @@ -66,15 +66,17 @@ std::shared_ptr TrainerFactory::CreateTrainer( REGISTER_TRAINER_CLASS(MultiTrainer); REGISTER_TRAINER_CLASS(DistMultiTrainer); -#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_XPU) && \ +#if (defined PADDLE_WITH_CUDA || defined PADDLE_WITH_HIP || \ + defined PADDLE_WITH_XPU) && \ (defined PADDLE_WITH_PSLIB) REGISTER_TRAINER_CLASS(HeterXpuTrainer); REGISTER_TRAINER_CLASS(HeterBoxTrainer); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) REGISTER_TRAINER_CLASS(PSGPUTrainer); #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) REGISTER_TRAINER_CLASS(PipelineTrainer); #endif } // namespace framework diff --git a/paddle/fluid/framework/var_type_traits.cc b/paddle/fluid/framework/var_type_traits.cc index 81c7d0d0c8840..886d00e562bff 100644 --- a/paddle/fluid/framework/var_type_traits.cc +++ b/paddle/fluid/framework/var_type_traits.cc @@ -28,6 +28,14 @@ #include "paddle/fluid/operators/conv_cudnn_op_cache.h" #include "paddle/fluid/operators/cudnn_rnn_cache.h" #endif +#ifdef PADDLE_WITH_HIP +#if defined(PADDLE_WITH_RCCL) +#include "paddle/fluid/operators/nccl/nccl_gpu_common.h" // NOLINT +#include "paddle/fluid/platform/nccl_helper.h" // NOLINT +#endif +#include "paddle/fluid/operators/conv_cudnn_op_cache.h" // NOLINT +#include "paddle/fluid/operators/miopen_rnn_cache.h" +#endif #if defined(PADDLE_WITH_XPU_BKCL) #include "paddle/fluid/platform/bkcl_helper.h" diff --git a/paddle/fluid/framework/var_type_traits.h b/paddle/fluid/framework/var_type_traits.h index 2fd4de5cfcba4..b0d8f43a90f35 100644 --- a/paddle/fluid/framework/var_type_traits.h +++ b/paddle/fluid/framework/var_type_traits.h @@ -30,6 +30,12 @@ #include #endif #endif +#ifdef PADDLE_WITH_HIP +#include +#ifdef PADDLE_WITH_RCCL +#include +#endif +#endif #if defined(PADDLE_WITH_XPU_BKCL) #include "xpu/bkcl.h" @@ -39,8 +45,8 @@ namespace paddle { namespace platform { -#ifdef PADDLE_WITH_CUDA -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) class Communicator; class NCCLCommunicator; #endif @@ -151,8 +157,8 @@ using VarTypeRegistry = detail::VarTypeRegistryImpl< LoDTensorArray, platform::PlaceList, ReaderHolder, std::string, Scope *, operators::reader::LoDTensorBlockingQueueHolder, FetchList, operators::reader::OrderedMultiDeviceLoDTensorBlockingQueueHolder, -#ifdef PADDLE_WITH_CUDA -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) ncclUniqueId, platform::Communicator, platform::NCCLCommunicator, #endif operators::CudnnRNNCache, diff --git a/paddle/fluid/framework/var_type_traits_test.cc b/paddle/fluid/framework/var_type_traits_test.cc index 9d1bd77ebdf69..2a6635c4b6050 100644 --- a/paddle/fluid/framework/var_type_traits_test.cc +++ b/paddle/fluid/framework/var_type_traits_test.cc @@ -28,6 +28,14 @@ #include "paddle/fluid/operators/conv_cudnn_op_cache.h" #include "paddle/fluid/operators/cudnn_rnn_cache.h" #endif +#ifdef PADDLE_WITH_HIP +#if defined(PADDLE_WITH_RCCL) +#include "paddle/fluid/operators/nccl/nccl_gpu_common.h" // NOLINT +#include "paddle/fluid/platform/nccl_helper.h" // NOLINT +#endif +#include "paddle/fluid/operators/conv_cudnn_op_cache.h" // NOLINT +#include "paddle/fluid/operators/miopen_rnn_cache.h" +#endif #if defined(PADDLE_WITH_XPU_BKCL) #include "paddle/fluid/platform/bkcl_helper.h" #endif diff --git a/paddle/fluid/imperative/data_loader.cc b/paddle/fluid/imperative/data_loader.cc index 71ea82e9a19e8..c43149c9b563e 100644 --- a/paddle/fluid/imperative/data_loader.cc +++ b/paddle/fluid/imperative/data_loader.cc @@ -71,9 +71,12 @@ void EraseLoadProcessPIDs(int64_t key) { } \ } while (0) -#define REGISTER_SIGNAL_HANDLER(SIGNAL, HANDLER_NAME) \ - static void HANDLER_NAME(int sig, siginfo_t *info, void *ctx) { \ - SIGNAL_HANDLE(SIGNAL); \ +#define REGISTER_SIGNAL_HANDLER(SIGNAL, HANDLER_NAME, ERROR_MSG) \ + static void HANDLER_NAME(int sig, siginfo_t *info, void *ctx) { \ + auto _w = \ + write(STDERR_FILENO, ERROR_MSG, sizeof(ERROR_MSG) / sizeof(char)); \ + (void)_w; \ + SIGNAL_HANDLE(SIGNAL); \ } #define REGISTER_SPEC_SIGNAL_HANDLER(SIGNAL, HANDLER_NAME) \ @@ -84,8 +87,18 @@ void EraseLoadProcessPIDs(int64_t key) { SIGNAL_HANDLE(SIGNAL); \ } -REGISTER_SIGNAL_HANDLER(SIGSEGV, SIGSEGV_handler); -REGISTER_SIGNAL_HANDLER(SIGBUS, SIGBUS_handler); +REGISTER_SIGNAL_HANDLER(SIGSEGV, SIGSEGV_handler, + "ERROR: Unexpected segmentation fault encountered in " + "DataLoader workers.\n"); +REGISTER_SIGNAL_HANDLER( + SIGBUS, SIGBUS_handler, + "ERROR: Unexpected BUS error encountered in DataLoader worker. " + "This might be caused by insufficient shared memory (shm), " + "please check whether use_shared_memory is set and storage space " + "in /dev/shm is enough\n"); +REGISTER_SIGNAL_HANDLER(SIGFPE, SIGFPE_handler, + "ERROR: Unexpected floating-point exception " + "encountered in DataLoader worker.\n") REGISTER_SPEC_SIGNAL_HANDLER(SIGTERM, SIGTERM_handler); static inline void setSignalHandler(int signal, @@ -105,6 +118,7 @@ static inline void setSignalHandler(int signal, void SetLoadProcessSignalHandler() { setSignalHandler(SIGSEGV, &SIGSEGV_handler, nullptr); setSignalHandler(SIGBUS, &SIGBUS_handler, nullptr); + setSignalHandler(SIGFPE, &SIGFPE_handler, nullptr); setSignalHandler(SIGTERM, &SIGTERM_handler, nullptr); } diff --git a/paddle/fluid/imperative/jit/program_desc_tracer.cc b/paddle/fluid/imperative/jit/program_desc_tracer.cc index 53750f7bf02be..1a44f50275ef8 100644 --- a/paddle/fluid/imperative/jit/program_desc_tracer.cc +++ b/paddle/fluid/imperative/jit/program_desc_tracer.cc @@ -69,6 +69,7 @@ UniqueBlockVarGenerator::UniqueBlockVarGenerator( std::string UniqueBlockVarGenerator::NameOf(const std::weak_ptr &var, const std::string &prefix) { + VLOG(3) << "Finding: " << var.lock()->Name(); auto all_vars_iter = all_vars_.find(var); PADDLE_ENFORCE_EQ(all_vars_iter != all_vars_.end(), true, platform::errors::NotFound( @@ -111,6 +112,15 @@ void UniqueBlockVarGenerator::InsertNewVarInBlock( } } +bool ProgramDescTracer::ContainVar(const std::weak_ptr &var) const { + auto vars_iter = vars_.find(var); + bool ret = (vars_iter != vars_.end()); + if (!ret) { + VLOG(5) << "Can't found variable: " << var.lock()->Name(); + } + return ret; +} + void ProgramDescTracer::InsertOp(const std::string &type, const NameVarBaseMap &inputs, const NameVarBaseMap &outputs, @@ -147,12 +157,16 @@ TracedProgramTuple ProgramDescTracer::CreateProgramDesc( std::vector feed_var_names; for (auto &feed_var : feed_vars) { - feed_var_names.emplace_back(generator.NameOf(feed_var, feed_prefix)); + if (ContainVar(feed_var)) { + feed_var_names.emplace_back(generator.NameOf(feed_var, feed_prefix)); + } } std::vector fetch_var_names; for (auto &fetch_var : fetch_vars) { - fetch_var_names.emplace_back(generator.NameOf(fetch_var, fetch_prefix)); + if (ContainVar(fetch_var)) { + fetch_var_names.emplace_back(generator.NameOf(fetch_var, fetch_prefix)); + } } for (auto &op : ops_) { @@ -164,7 +178,9 @@ TracedProgramTuple ProgramDescTracer::CreateProgramDesc( std::vector names; names.reserve(pair.second.size()); for (auto &var : pair.second) { - names.emplace_back(generator.NameOf(var, tmp_prefix)); + if (ContainVar(var)) { + names.emplace_back(generator.NameOf(var, tmp_prefix)); + } } op_desc->SetInput(pair.first, std::move(names)); @@ -174,7 +190,9 @@ TracedProgramTuple ProgramDescTracer::CreateProgramDesc( std::vector names; names.reserve(pair.second.size()); for (auto &var : pair.second) { - names.emplace_back(generator.NameOf(var, tmp_prefix)); + if (ContainVar(var)) { + names.emplace_back(generator.NameOf(var, tmp_prefix)); + } } op_desc->SetOutput(pair.first, std::move(names)); diff --git a/paddle/fluid/imperative/jit/program_desc_tracer.h b/paddle/fluid/imperative/jit/program_desc_tracer.h index 8e2e59a49ed7b..b231efb0e53a5 100644 --- a/paddle/fluid/imperative/jit/program_desc_tracer.h +++ b/paddle/fluid/imperative/jit/program_desc_tracer.h @@ -66,7 +66,7 @@ class ProgramDescTracer { const std::string &feed_prefix, const std::vector> &fetch_vars, const std::string &fetch_prefix, const std::string &tmp_prefix) const; - + bool ContainVar(const std::weak_ptr &var) const; void Reset(); private: diff --git a/paddle/fluid/imperative/prepared_operator.cc b/paddle/fluid/imperative/prepared_operator.cc index e6e5135316aba..2a3b6424d4a14 100644 --- a/paddle/fluid/imperative/prepared_operator.cc +++ b/paddle/fluid/imperative/prepared_operator.cc @@ -91,6 +91,7 @@ PreparedOp PrepareImpl(const NameVarMap& ins, const framework::AttributeMap& attrs) { platform::DeviceContextPool& pool = platform::DeviceContextPool::Instance(); auto* dev_ctx = pool.Get(place); + framework::RuntimeContext ctx({}, {}); #ifdef PADDLE_WITH_MKLDNN diff --git a/paddle/fluid/imperative/reducer.cc b/paddle/fluid/imperative/reducer.cc index f8740940d041a..e8b531d35cabf 100644 --- a/paddle/fluid/imperative/reducer.cc +++ b/paddle/fluid/imperative/reducer.cc @@ -301,6 +301,10 @@ Reducer::Reducer(const std::vector> &vars, VLOG(3) << "Start construct the Reducer ..."; nrings_ = parallel_ctx->GetNRings(); nranks_ = parallel_ctx->GetNRanks(); +#ifdef PADDLE_WITH_XPU_BKCL + comm_pool_.reset(new ::ThreadPool(1)); + comm_op_count_ = 0; +#endif // initialize groups InitializeGroups(group_indices); for (size_t global_var_index = 0; global_var_index < vars_.size(); @@ -634,6 +638,8 @@ void Reducer::MarkVarReady(const size_t var_index, const bool is_used_var) { } } +// TODO(liuyuhui): If BKCL support non-blocking communication, it should be +// fixed as same as multi gpus card trainging. void Reducer::MarkGroupReady(size_t group_index) { if (group_index > next_group_) { VLOG(3) << "It will adjust the order of group in next batch automatically"; @@ -651,45 +657,71 @@ void Reducer::MarkGroupReady(size_t group_index) { // so we expose WaitCompute() interface and call // it here. parallel_ctx_->WaitCompute(run_order); - - if (group.is_sparse_) { - if (group.sparse_contents_ != nullptr) { - VLOG(3) << "sparse group [" << next_group_ - << "] start allreduce in ring[" << run_order << "]"; - group.DivNRanks(*parallel_ctx_->GetDeviceContext(run_order), nranks_); - parallel_ctx_->AllReduceByStream( - *group.sparse_contents_, group.sparse_contents_, run_order, false); - } else { - VLOG(3) << "The sparse group[" << next_group_ - << "] has no var to allreduce"; +#ifdef PADDLE_WITH_XPU_BKCL + { + std::lock_guard lock(mutex_); + comm_op_count_ += 1; // lock + } + // TODO(liuyuhui): Add try catch to deal with exception later, + // otherwise the main thread will continue to run when an exception is + // thrown in comm_pool_. + comm_pool_->enqueue([&] { + auto dev_id = BOOST_GET_CONST(platform::XPUPlace, place_).device; + platform::SetXPUDeviceId(dev_id); + FusedAllReduceSchedule(run_order, group); + { + std::lock_guard lock(mutex_); + comm_op_count_ -= 1; // lock + cv_.notify_all(); } - } else { - VLOG(3) << "dense group [" << next_group_ << "] start allreduce in ring[" + }); +#elif defined(PADDLE_WITH_RCCL) || defined(PADDLE_WITH_NCCL) + FusedAllReduceSchedule(run_order, group); +#else + PADDLE_THROW(platform::errors::PreconditionNotMet( + "Not compiled with BKCL or NCCL.")); +#endif + } +} + +void Reducer::FusedAllReduceSchedule(int run_order, Group &group) { + if (group.is_sparse_) { + if (group.sparse_contents_ != nullptr) { + VLOG(3) << "sparse group [" << next_group_ << "] start allreduce in ring[" << run_order << "]"; - // Select common commstream to concat tensors - // group.dense_tensors ---> group.dense_contents_ - group.ConcatTensors(*parallel_ctx_->GetDeviceContext(run_order)); + group.DivNRanks(*parallel_ctx_->GetDeviceContext(run_order), nranks_); + parallel_ctx_->AllReduceByStream( + *group.sparse_contents_, group.sparse_contents_, run_order, false); + } else { + VLOG(3) << "The sparse group[" << next_group_ + << "] has no var to allreduce"; + } + } else { + VLOG(3) << "dense group [" << next_group_ << "] start allreduce in ring[" + << run_order << "]"; + // Select common commstream to concat tensors + // group.dense_tensors ---> group.dense_contents_ + group.ConcatTensors(*parallel_ctx_->GetDeviceContext(run_order)); // NOTE(liuyuhui): ConcatTensors use communication stream, but BKCL only support // default stream for communicating, so there exist some problems in // synchronization. And need to add a WaitComm there. -// TODO(liuyuhui): If BKCL support events, it should be fixed as non-blocking -// communication. +// TODO(liuyuhui): If BKCL support non-blocking communication, it should be +// fixed as multi gpus card trainging. #ifdef PADDLE_WITH_XPU_BKCL - if (platform::is_xpu_place(group.dense_tensors_[0].place())) { - parallel_ctx_->WaitComm(run_order); - } + if (platform::is_xpu_place(group.dense_tensors_[0].place())) { + parallel_ctx_->WaitComm(run_order); + } #endif - group.DivNRanks(*parallel_ctx_->GetDeviceContext(run_order), nranks_); + group.DivNRanks(*parallel_ctx_->GetDeviceContext(run_order), nranks_); - // Start allreduce - parallel_ctx_->AllReduceByStream( - group.dense_contents_, &(group.dense_contents_), run_order, false); + // Start allreduce + parallel_ctx_->AllReduceByStream( + group.dense_contents_, &(group.dense_contents_), run_order, false); - // Select common commstream to split tensors - // group.dense_contents_ ---> group.dense_tensors - group.SplitTensors(*parallel_ctx_->GetDeviceContext(run_order)); - } + // Select common commstream to split tensors + // group.dense_contents_ ---> group.dense_tensors + group.SplitTensors(*parallel_ctx_->GetDeviceContext(run_order)); } } @@ -717,6 +749,12 @@ std::vector> Reducer::RebuildGruops() { void Reducer::FinalizeBackward() { all_group_ready_ = false; +#ifdef PADDLE_WITH_XPU_BKCL + { + std::unique_lock lock(mutex_); + cv_.wait(lock, [&] { return comm_op_count_ == 0; }); + } +#endif // Must prevent compute_stream_ starting until all comm streams have finished for (int i = 0; i < nrings_; ++i) { parallel_ctx_->WaitComm(i); diff --git a/paddle/fluid/imperative/reducer.h b/paddle/fluid/imperative/reducer.h index f352ad17fda5d..b2680d0dea71a 100644 --- a/paddle/fluid/imperative/reducer.h +++ b/paddle/fluid/imperative/reducer.h @@ -13,7 +13,7 @@ // limitations under the License. #pragma once - +#include #include #include #include @@ -153,6 +153,8 @@ class Reducer { void MarkGroupReady(size_t group_index); + void FusedAllReduceSchedule(int run_order, Group& group); // NOLINT + void FinalizeBackward(); std::vector> RebuildGruops(); @@ -187,6 +189,13 @@ class Reducer { bool has_marked_unused_vars_{false}; bool find_unused_vars_{false}; bool all_group_ready_{false}; +#ifdef PADDLE_WITH_XPU_BKCL + // comm_pool_ is used for scheduling allreduce in multi Kunlun cards training. + std::unique_ptr<::ThreadPool> comm_pool_{nullptr}; + uint32_t comm_op_count_; + std::mutex mutex_; + std::condition_variable cv_; +#endif }; std::vector> AssignGroupBySize( diff --git a/paddle/fluid/inference/CMakeLists.txt b/paddle/fluid/inference/CMakeLists.txt index 8ef6bcd8600c8..93fd85f13cbf0 100644 --- a/paddle/fluid/inference/CMakeLists.txt +++ b/paddle/fluid/inference/CMakeLists.txt @@ -36,6 +36,10 @@ endif() # fluid_modules exclude API-interface of inference/api and inference/capi get_property(fluid_modules GLOBAL PROPERTY FLUID_MODULES) +# Adapt to custom op mechanism: Include the header files related to the data type +# to avoid exposing the path of the underlying file +include_directories(${PADDLE_SOURCE_DIR}/paddle/fluid/platform) + add_subdirectory(api) # Create static inference library if needed @@ -73,7 +77,8 @@ set(SHARED_INFERENCE_SRCS ${CMAKE_CURRENT_SOURCE_DIR}/api/analysis_predictor.cc ${CMAKE_CURRENT_SOURCE_DIR}/api/details/zero_copy_tensor.cc ${CMAKE_CURRENT_SOURCE_DIR}/utils/io_utils.cc - ${mkldnn_quantizer_src_file}) + ${mkldnn_quantizer_src_file} + ${PADDLE_CUSTOM_OP_SRCS}) # shared inference library deps set(SHARED_INFERENCE_DEPS ${fluid_modules} analysis_predictor) diff --git a/paddle/fluid/inference/analysis/helper.h b/paddle/fluid/inference/analysis/helper.h index 730fe35853a96..ab4949935140c 100644 --- a/paddle/fluid/inference/analysis/helper.h +++ b/paddle/fluid/inference/analysis/helper.h @@ -244,7 +244,7 @@ static std::string GetTrtEngineSerializedData( if (FileExists(trt_serialized_path)) { VLOG(3) << "Trt serialized file: " << trt_serialized_path << "is found here"; - std::ifstream infile(trt_serialized_path, std::ios::in); + std::ifstream infile(trt_serialized_path, std::ios::binary); std::stringstream buffer; buffer << infile.rdbuf(); std::string trt_engine_serialized_data(buffer.str()); @@ -256,7 +256,7 @@ static std::string GetTrtEngineSerializedData( static void SaveTrtEngineSerializedDataToFile( const std::string &trt_serialized_path, const std::string &engine_serialized_data) { - std::ofstream outfile(trt_serialized_path); + std::ofstream outfile(trt_serialized_path, std::ios::binary); outfile << engine_serialized_data; outfile.close(); } diff --git a/paddle/fluid/inference/analysis/ir_pass_manager.cc b/paddle/fluid/inference/analysis/ir_pass_manager.cc index 26bca9b1e54ec..a4e263e2f464c 100644 --- a/paddle/fluid/inference/analysis/ir_pass_manager.cc +++ b/paddle/fluid/inference/analysis/ir_pass_manager.cc @@ -114,13 +114,25 @@ void IRPassManager::CreatePasses(Argument *argument, "When you are in TRT INT8 mode, and load model from " "memory, you should set optim_cache_dir using " "config.SetOptimCacheDir()")); - PADDLE_ENFORCE_EQ( - !(model_from_memory && use_static_engine), true, - platform::errors::PreconditionNotMet( - "When you are using Paddle-TRT, and also using load model " - "from memory, you should set the use_static to false.")); + if (model_from_memory && use_static_engine) { + PADDLE_ENFORCE_EQ( + optim_cache_dir.empty(), false, + platform::errors::PreconditionNotMet( + "When you are using Paddle-TRT, and using load model " + "from memory, and also set the use_static to true. " + "you must set optim_cache_dir using " + "config.SetOptimCacheDir().")); + } if (!optim_cache_dir.empty()) { + if (!PathExists(optim_cache_dir)) { + PADDLE_ENFORCE_NE( + MKDIR(optim_cache_dir.c_str()), -1, + platform::errors::PreconditionNotMet( + "Can not create optimize cache directory: %s, Make sure you " + "have permission to write", + optim_cache_dir)); + } pass->Set("model_opt_cache_dir", new std::string(optim_cache_dir)); } else if (use_static_engine || enable_int8) { std::string model_opt_cache_dir = diff --git a/paddle/fluid/inference/analysis/ir_passes/tensorrt_subgraph_pass.cc b/paddle/fluid/inference/analysis/ir_passes/tensorrt_subgraph_pass.cc index a450ebdf89196..60de4234b41a8 100644 --- a/paddle/fluid/inference/analysis/ir_passes/tensorrt_subgraph_pass.cc +++ b/paddle/fluid/inference/analysis/ir_passes/tensorrt_subgraph_pass.cc @@ -83,16 +83,30 @@ void analysis::TensorRtSubgraphPass::ApplyImpl( std::string GenerateEngineKey(const std::set &engine_inputs, const std::set &engine_outputs, - const std::string &predictor_id) { + const std::string &predictor_id, + const std::string &max_batch_size, + const std::string &precision, + const bool for_calibration) { std::string engine_hash_key = ""; for (auto name : engine_inputs) { engine_hash_key += name; + engine_hash_key += "#"; } for (auto name : engine_outputs) { engine_hash_key += name; + engine_hash_key += "#"; } engine_hash_key += predictor_id; + if (!for_calibration) { + engine_hash_key += "#"; + engine_hash_key += max_batch_size; + } + engine_hash_key += "#"; + engine_hash_key += precision; + auto engine_key = std::to_string(std::hash()(engine_hash_key)); + VLOG(2) << "TRT engine hash key: " << engine_hash_key; + VLOG(2) << "TRT engine key: " << engine_key; return engine_key; } @@ -154,11 +168,11 @@ void TensorRtSubgraphPass::CreateTensorRTOp( std::set output_names; std::set output_names_with_id; - std::vector origin_output_dims; + std::map origin_name_output_dims; for (auto *x : node->outputs) { output_names.insert(x->Name()); output_names_with_id.insert(x->Name() + std::to_string(x->id())); - origin_output_dims.push_back(x->Var()->GetShape().size()); + origin_name_output_dims[x->Name()] = x->Var()->GetShape().size(); } std::unordered_map output_name_map; @@ -202,11 +216,13 @@ void TensorRtSubgraphPass::CreateTensorRTOp( // output_mapping help us copy the data from the renamed ITensor // to Tensor. std::vector output_mapping; + std::vector renamed_output_dims; for (auto name : output_names) { PADDLE_ENFORCE_NE(output_name_map.count(name), 0, platform::errors::PreconditionNotMet( "The output_name_map should have %s", name)); output_mapping.push_back(output_name_map[name]); + renamed_output_dims.push_back(origin_name_output_dims[name]); } PADDLE_ENFORCE_EQ(output_mapping.empty(), false, platform::errors::PreconditionNotMet( @@ -229,7 +245,7 @@ void TensorRtSubgraphPass::CreateTensorRTOp( op_desc->SetAttr("workspace_size", Get("workspace_size")); op_desc->SetAttr("gpu_id", Get("gpu_device_id")); op_desc->SetAttr("output_name_mapping", output_mapping); - op_desc->SetAttr("origin_output_dims", origin_output_dims); + op_desc->SetAttr("origin_output_dims", renamed_output_dims); op_desc->SetAttr("parameters", params); // we record all inputs' shapes in attr to check if they are consistent @@ -245,22 +261,31 @@ void TensorRtSubgraphPass::CreateTensorRTOp( // TODO(NHZlX) // There are models with the same structure but the different parameters, // when running in the 'use_serialize' mode, there is a bug. - auto engine_key = GenerateEngineKey(input_names_with_id, output_names_with_id, - std::to_string(0)); + // serialization is affected by max_batch_size, but calibration is not. + // So we use seperate engine keys in serialization and calibration. + auto engine_key = GenerateEngineKey( + input_names_with_id, output_names_with_id, std::to_string(0), + std::to_string(Get("max_batch_size")), + std::to_string(static_cast(precision_mode)), false); + auto calibration_engine_key = GenerateEngineKey( + input_names_with_id, output_names_with_id, std::to_string(0), + std::to_string(Get("max_batch_size")), + std::to_string(static_cast(precision_mode)), true); auto predictor_id = Get("predictor_id"); // Get "" when there is no cached calibration table data. - bool load_from_memory = Get("model_from_memory"); std::string calibration_data = ""; if (enable_int8 && use_calib_mode) { - calibration_data = GetTrtCalibTableData( - Get("model_opt_cache_dir"), engine_key, enable_int8); + calibration_data = + GetTrtCalibTableData(Get("model_opt_cache_dir"), + calibration_engine_key, enable_int8); } op_desc->SetAttr("calibration_data", calibration_data); op_desc->SetAttr("enable_int8", enable_int8); op_desc->SetAttr("enable_fp16", enable_fp16); op_desc->SetAttr("use_calib_mode", use_calib_mode); op_desc->SetAttr("engine_key", engine_key); + op_desc->SetAttr("calibration_engine_key", calibration_engine_key); op_desc->SetAttr("predictor_id", predictor_id); std::string trt_engine_serialized_data = ""; @@ -323,8 +348,7 @@ void TensorRtSubgraphPass::CreateTensorRTOp( graph->Has(framework::ir::kEmbEltwiseLayernormPass) && graph->Has(framework::ir::kMultiheadMatmulPass)); - bool need_serialize = (use_static_engine && !load_from_memory); - if (need_serialize) { + if (use_static_engine) { trt_engine_serialized_data = GetTrtEngineSerializedData( Get("model_opt_cache_dir"), engine_key); // we can load the engine info serialized before from the disk. @@ -352,7 +376,7 @@ void TensorRtSubgraphPass::CreateTensorRTOp( std::vector(input_names.begin(), input_names.end()), param_set, output_mapping, trt_engine); - if (need_serialize) { + if (use_static_engine) { nvinfer1::IHostMemory *serialized_engine_data = trt_engine->Serialize(); trt_engine_serialized_data = std::string((const char *)serialized_engine_data->data(), @@ -361,6 +385,9 @@ void TensorRtSubgraphPass::CreateTensorRTOp( GetTrtEngineSerializedPath(Get("model_opt_cache_dir"), engine_key), trt_engine_serialized_data); + LOG(INFO) << "Save TRT Optimized Info to " + << GetTrtEngineSerializedPath( + Get("model_opt_cache_dir"), engine_key); } } diff --git a/paddle/fluid/inference/analysis/passes/memory_optimize_pass.cc b/paddle/fluid/inference/analysis/passes/memory_optimize_pass.cc index 5e6960c4c7e8c..fdfd2c60af0c1 100644 --- a/paddle/fluid/inference/analysis/passes/memory_optimize_pass.cc +++ b/paddle/fluid/inference/analysis/passes/memory_optimize_pass.cc @@ -103,6 +103,7 @@ void MemoryOptimizePass::CollectVarMemorySize( "merge_lod_tensor", "equal", "sequence_pool", + "recurrent", "lod_reset"}; for (auto* tmp : node->inputs) { CHECK(tmp->IsOp()); diff --git a/paddle/fluid/inference/api/CMakeLists.txt b/paddle/fluid/inference/api/CMakeLists.txt index 22aa210c97ef8..9a4637306bb35 100755 --- a/paddle/fluid/inference/api/CMakeLists.txt +++ b/paddle/fluid/inference/api/CMakeLists.txt @@ -30,8 +30,13 @@ endif() cc_library(analysis_config SRCS analysis_config.cc DEPS ${mkldnn_quantizer_cfg} lod_tensor paddle_pass_builder) cc_library(paddle_pass_builder SRCS paddle_pass_builder.cc) -cc_library(paddle_inference_api SRCS api.cc api_impl.cc helper.cc DEPS lod_tensor scope reset_tensor_array - analysis_config zero_copy_tensor trainer_desc_proto) +if(WITH_CRYPTO) + cc_library(paddle_inference_api SRCS api.cc api_impl.cc helper.cc DEPS lod_tensor scope reset_tensor_array + analysis_config zero_copy_tensor trainer_desc_proto paddle_crypto) +else() + cc_library(paddle_inference_api SRCS api.cc api_impl.cc helper.cc DEPS lod_tensor scope reset_tensor_array + analysis_config zero_copy_tensor trainer_desc_proto) +endif() if(WIN32) target_link_libraries(paddle_inference_api gflags) diff --git a/paddle/fluid/inference/api/analysis_predictor.cc b/paddle/fluid/inference/api/analysis_predictor.cc index 2ee8bb6073972..8f2b217a2fde0 100644 --- a/paddle/fluid/inference/api/analysis_predictor.cc +++ b/paddle/fluid/inference/api/analysis_predictor.cc @@ -21,6 +21,7 @@ #include #include #include +#include "paddle/fluid/extension/include/ext_op_meta_info.h" #include "paddle/fluid/framework/feed_fetch_method.h" #include "paddle/fluid/framework/feed_fetch_type.h" #include "paddle/fluid/framework/ir/fuse_pass_base.h" @@ -612,6 +613,12 @@ std::unique_ptr CreatePaddlePredictor< platform::errors::InvalidArgument( "Note: Each config can only be used for one predictor.")); + // Register custom operators compiled by the user. + // This function can only be executed once per process. + static std::once_flag custom_operators_registered; + std::call_once(custom_operators_registered, + []() { paddle::RegisterAllCustomOperator(); }); + if (config.use_gpu()) { static std::once_flag gflags_initialized; static bool process_level_allocator_enabled; @@ -1010,8 +1017,8 @@ bool AnalysisPredictor::SaveTrtCalibToDisk() { auto &block = inference_program_->Block(0); for (auto &op_desc : block.AllOps()) { if (op_desc->Type() == "tensorrt_engine") { - std::string engine_name = - BOOST_GET_CONST(std::string, op_desc->GetAttr("engine_key")); + std::string engine_name = BOOST_GET_CONST( + std::string, op_desc->GetAttr("calibration_engine_key")); if (!Singleton::Global().Has(engine_name)) { LOG(ERROR) << "You should run the predictor(with trt) on the real data " "to generate calibration info"; @@ -1184,24 +1191,13 @@ USE_TRT_CONVERTER(slice); USE_TRT_CONVERTER(scale); USE_TRT_CONVERTER(stack); USE_TRT_CONVERTER(clip); +USE_TRT_CONVERTER(gather); + +USE_TRT_CONVERTER(nearest_interp); #endif namespace paddle_infer { -void Tensor::Reshape(const std::vector &shape) { tensor_->Reshape(shape); } - -std::vector Tensor::shape() const { return tensor_->shape(); } - -void Tensor::SetLoD(const std::vector> &x) { - return tensor_->SetLoD(x); -} - -std::vector> Tensor::lod() const { return tensor_->lod(); } - -const std::string &Tensor::name() const { return tensor_->name(); } - -DataType Tensor::type() const { return tensor_->type(); } - Predictor::Predictor(const Config &config) { const_cast(&config)->SwitchUseFeedFetchOps(false); // The second parameter indicates that the discard log is not printed @@ -1214,9 +1210,7 @@ std::vector Predictor::GetInputNames() { } std::unique_ptr Predictor::GetInputHandle(const std::string &name) { - auto zero_copy_tensor = predictor_->GetInputTensor(name); - std::unique_ptr tensor(new Tensor(std::move(zero_copy_tensor))); - return tensor; + return predictor_->GetInputTensor(name); } std::vector Predictor::GetOutputNames() { @@ -1224,9 +1218,7 @@ std::vector Predictor::GetOutputNames() { } std::unique_ptr Predictor::GetOutputHandle(const std::string &name) { - auto zero_copy_tensor = predictor_->GetOutputTensor(name); - std::unique_ptr tensor(new Tensor(std::move(zero_copy_tensor))); - return tensor; + return predictor_->GetOutputTensor(name); } bool Predictor::Run() { return predictor_->ZeroCopyRun(); } diff --git a/paddle/fluid/inference/api/api.cc b/paddle/fluid/inference/api/api.cc index f103eb7674bc6..e2befadf0a89b 100644 --- a/paddle/fluid/inference/api/api.cc +++ b/paddle/fluid/inference/api/api.cc @@ -146,4 +146,10 @@ std::string UpdateDllFlag(const char *name, const char *value) { return ret; } +#ifdef PADDLE_WITH_CRYPTO +std::shared_ptr MakeCipher(const std::string &config_file) { + return framework::CipherFactory::CreateCipher(config_file); +} +#endif + } // namespace paddle diff --git a/paddle/fluid/inference/api/api_tester.cc b/paddle/fluid/inference/api/api_tester.cc index 3b489616af9c2..46724fa6b1aca 100644 --- a/paddle/fluid/inference/api/api_tester.cc +++ b/paddle/fluid/inference/api/api_tester.cc @@ -94,4 +94,9 @@ TEST(paddle_inference_api, AnalysisConfigCopyCtor) { CHECK_NE(ps, delete_pass); } } + +#ifdef PADDLE_WITH_CRYPTO +TEST(paddle_inference_api, crypto) { paddle::MakeCipher(""); } +#endif + } // namespace paddle diff --git a/paddle/fluid/inference/api/demo_ci/CMakeLists.txt b/paddle/fluid/inference/api/demo_ci/CMakeLists.txt index e24d83af2f368..0a09b062803f6 100644 --- a/paddle/fluid/inference/api/demo_ci/CMakeLists.txt +++ b/paddle/fluid/inference/api/demo_ci/CMakeLists.txt @@ -36,11 +36,13 @@ include_directories("${PADDLE_LIB_THIRD_PARTY_PATH}protobuf/include") include_directories("${PADDLE_LIB_THIRD_PARTY_PATH}glog/include") include_directories("${PADDLE_LIB_THIRD_PARTY_PATH}gflags/include") include_directories("${PADDLE_LIB_THIRD_PARTY_PATH}xxhash/include") +include_directories("${PADDLE_LIB_THIRD_PARTY_PATH}cryptopp/include") link_directories("${PADDLE_LIB_THIRD_PARTY_PATH}protobuf/lib") link_directories("${PADDLE_LIB_THIRD_PARTY_PATH}glog/lib") link_directories("${PADDLE_LIB_THIRD_PARTY_PATH}gflags/lib") link_directories("${PADDLE_LIB_THIRD_PARTY_PATH}xxhash/lib") +link_directories("${PADDLE_LIB_THIRD_PARTY_PATH}cryptopp/lib") link_directories("${PADDLE_LIB}/paddle/lib") if (WIN32) @@ -145,12 +147,12 @@ if (NOT WIN32) set(EXTERNAL_LIB "-lrt -ldl -lpthread") set(DEPS ${DEPS} ${MATH_LIB} ${MKLDNN_LIB} - glog gflags protobuf xxhash + glog gflags protobuf xxhash cryptopp ${EXTERNAL_LIB}) else() set(DEPS ${DEPS} ${MATH_LIB} ${MKLDNN_LIB} - glog gflags_static libprotobuf xxhash ${EXTERNAL_LIB}) + glog gflags_static libprotobuf xxhash cryptopp-static ${EXTERNAL_LIB}) set(DEPS ${DEPS} shlwapi.lib) endif(NOT WIN32) diff --git a/paddle/fluid/inference/api/demo_ci/run.sh b/paddle/fluid/inference/api/demo_ci/run.sh index e11a5b9c3372a..53f9259666626 100755 --- a/paddle/fluid/inference/api/demo_ci/run.sh +++ b/paddle/fluid/inference/api/demo_ci/run.sh @@ -88,7 +88,7 @@ for WITH_STATIC_LIB in ON OFF; do return 0 fi # -----simple_on_word2vec on windows----- - cmake .. -G "Visual Studio 14 2015" -A x64 -DPADDLE_LIB=${inference_install_dir} \ + cmake .. -G "Visual Studio 15 2017" -A x64 -T host=x64 -DPADDLE_LIB=${inference_install_dir} \ -DWITH_MKL=$TURN_ON_MKL \ -DDEMO_NAME=simple_on_word2vec \ -DWITH_GPU=$TEST_GPU_CPU \ @@ -107,7 +107,7 @@ for WITH_STATIC_LIB in ON OFF; do # -----vis_demo on windows----- rm -rf * - cmake .. -G "Visual Studio 14 2015" -A x64 -DPADDLE_LIB=${inference_install_dir} \ + cmake .. -G "Visual Studio 15 2017" -A x64 -T host=x64 -DPADDLE_LIB=${inference_install_dir} \ -DWITH_MKL=$TURN_ON_MKL \ -DDEMO_NAME=vis_demo \ -DWITH_GPU=$TEST_GPU_CPU \ diff --git a/paddle/fluid/inference/api/demo_ci/run_windows_demo.bat b/paddle/fluid/inference/api/demo_ci/run_windows_demo.bat index 523dafa6649b9..d17f516fcca5e 100644 --- a/paddle/fluid/inference/api/demo_ci/run_windows_demo.bat +++ b/paddle/fluid/inference/api/demo_ci/run_windows_demo.bat @@ -67,7 +67,7 @@ if /i "%use_gpu%"=="Y" ( rem set_path_vs_command_prompt :set_vcvarsall_dir -SET /P vcvarsall_dir="Please input the path of visual studio command Prompt, such as C:\Program Files (x86)\Microsoft Visual Studio 14.0\VC\vcvarsall.bat =======>" +SET /P vcvarsall_dir="Please input the path of visual studio command Prompt, such as C:\Program Files (x86)\Microsoft Visual Studio\2017\Community\VC\Auxiliary\Build\vcvarsall.bat =======>" set tmp_var=!vcvarsall_dir! call:remove_space set vcvarsall_dir=!tmp_var! @@ -177,16 +177,16 @@ if /i "%use_mkl%"=="N" ( if /i "%gpu_inference%"=="Y" ( if "%demo_name%"=="trt_mobilenet_demo" ( - cmake .. -G "Visual Studio 14 2015 Win64" -T host=x64 -DWITH_GPU=ON ^ + cmake .. -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=ON ^ -DWITH_MKL=%use_mkl% -DWITH_STATIC_LIB=ON -DCMAKE_BUILD_TYPE=Release -DDEMO_NAME=%demo_name% ^ -DPADDLE_LIB="%paddle_infernece_lib%" -DMSVC_STATIC_CRT=ON -DCUDA_LIB="%cuda_lib_dir%" -DUSE_TENSORRT=ON ) else ( - cmake .. -G "Visual Studio 14 2015 Win64" -T host=x64 -DWITH_GPU=ON ^ + cmake .. -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=ON ^ -DWITH_MKL=%use_mkl% -DWITH_STATIC_LIB=ON -DCMAKE_BUILD_TYPE=Release -DDEMO_NAME=%demo_name% ^ -DPADDLE_LIB="%paddle_infernece_lib%" -DMSVC_STATIC_CRT=ON -DCUDA_LIB="%cuda_lib_dir%" ) ) else ( - cmake .. -G "Visual Studio 14 2015 Win64" -T host=x64 -DWITH_GPU=OFF ^ + cmake .. -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=OFF ^ -DWITH_MKL=%use_mkl% -DWITH_STATIC_LIB=ON -DCMAKE_BUILD_TYPE=Release -DDEMO_NAME=%demo_name% ^ -DPADDLE_LIB="%paddle_infernece_lib%" -DMSVC_STATIC_CRT=ON ) diff --git a/paddle/fluid/inference/api/demo_ci/windows_inference.md b/paddle/fluid/inference/api/demo_ci/windows_inference.md index 73938cb995f17..c646c351462d4 100644 --- a/paddle/fluid/inference/api/demo_ci/windows_inference.md +++ b/paddle/fluid/inference/api/demo_ci/windows_inference.md @@ -8,7 +8,7 @@ 3. 进入Paddle/paddle/fluid/inference/api/demo_ci目录,新建build目录,然后使用cmake生成vs2015的solution文件。 其中PADDLE_LIB是前面的paddle_inference.lib对应文件夹, CUDA_LIB指定为x64格式下的cuda系统库目录文件夹。 ```shell - cmake .. -G "Visual Studio 14 2015 Win64" -DWITH_GPU=ON -DWITH_MKL=OFF -DWITH_STATIC_LIB=ON -DCMAKE_BUILD_TYPE=Release -DDEMO_NAME=inference_icnet -DPADDLE_LIB=D:\to_the_paddle_inference.lib -DCUDA_LIB=D:\tools\v8.0\lib\x64 + cmake .. -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=ON -DWITH_MKL=OFF -DWITH_STATIC_LIB=ON -DCMAKE_BUILD_TYPE=Release -DDEMO_NAME=inference_icnet -DPADDLE_LIB=D:\to_the_paddle_inference.lib -DCUDA_LIB=D:\tools\v8.0\lib\x64 ``` 然后用vs2015打开对应的项目文件,注意使用静态链接 "/MT",生成对应的exe。将openblas.dll放到exe所在目录。 diff --git a/paddle/fluid/inference/api/details/CMakeLists.txt b/paddle/fluid/inference/api/details/CMakeLists.txt index 80b53b32a8607..4341fb0a9ccd8 100644 --- a/paddle/fluid/inference/api/details/CMakeLists.txt +++ b/paddle/fluid/inference/api/details/CMakeLists.txt @@ -16,3 +16,5 @@ cc_library(reset_tensor_array SRCS reset_tensor_array.cc DEPS lod_tensor scope) cc_library(zero_copy_tensor SRCS zero_copy_tensor.cc DEPS scope lod_tensor enforce) cc_library(zero_copy_tensor_dummy SRCS zero_copy_tensor_dummy.cc) + +cc_test(zero_copy_tensor_test SRCS zero_copy_tensor_test.cc DEPS paddle_inference_api) diff --git a/paddle/fluid/inference/api/details/zero_copy_tensor.cc b/paddle/fluid/inference/api/details/zero_copy_tensor.cc index 0ed7476bb61fe..f7dbfd39cd26e 100644 --- a/paddle/fluid/inference/api/details/zero_copy_tensor.cc +++ b/paddle/fluid/inference/api/details/zero_copy_tensor.cc @@ -18,126 +18,135 @@ #include "paddle/fluid/memory/memcpy.h" #include "paddle/fluid/platform/enforce.h" -namespace paddle { +namespace paddle_infer { -void ZeroCopyTensor::Reshape(const std::vector &shape) { +void Tensor::Reshape(const std::vector &shape) { PADDLE_ENFORCE_EQ( name_.empty(), false, - platform::errors::PreconditionNotMet( + paddle::platform::errors::PreconditionNotMet( "Need to SetName first, so that the corresponding tensor can " "be retrieved.")); PADDLE_ENFORCE_EQ(input_or_output_, true, - platform::errors::PermissionDenied( + paddle::platform::errors::PermissionDenied( "Can't reshape the output tensor, it is readonly")); - PADDLE_ENFORCE_NOT_NULL(scope_, platform::errors::PreconditionNotMet( - "The scope should not be nullptr.")); - auto *scope = static_cast(scope_); + auto *scope = static_cast(scope_); auto *var = scope->FindVar(name_); PADDLE_ENFORCE_NOT_NULL( - var, platform::errors::PreconditionNotMet( + var, paddle::platform::errors::PreconditionNotMet( "No tensor called [%s] in the runtime scope", name_)); - auto *tensor = var->GetMutable(); - tensor->Resize(framework::make_ddim(shape)); + auto *tensor = var->GetMutable(); + tensor->Resize(paddle::framework::make_ddim(shape)); } #define EAGER_GET_TENSOR \ if (!tensor_) { \ tensor_ = FindTensor(); \ } \ - auto *tensor = static_cast(tensor_); + auto *tensor = static_cast(tensor_); template -T *ZeroCopyTensor::mutable_data(PaddlePlace place) { +T *Tensor::mutable_data(PlaceType place) { EAGER_GET_TENSOR; PADDLE_ENFORCE_GT( tensor->numel(), 0, - platform::errors::PreconditionNotMet( - "You should call ZeroCopyTensor::Reshape(const std::vector " + paddle::platform::errors::PreconditionNotMet( + "You should call Tensor::Reshape(const std::vector " "&shape)" "function before retrieving mutable_data from input tensor.")); switch (static_cast(place)) { - case static_cast(PaddlePlace::kCPU): { - return tensor->mutable_data(platform::CPUPlace()); + case static_cast(PlaceType::kCPU): { + return tensor->mutable_data(paddle::platform::CPUPlace()); } - case static_cast(PaddlePlace::kGPU): { - return tensor->mutable_data(platform::CUDAPlace(device_)); + case static_cast(PlaceType::kGPU): { + return tensor->mutable_data(paddle::platform::CUDAPlace(device_)); + } + case static_cast(PlaceType::kXPU): { + return tensor->mutable_data(paddle::platform::XPUPlace(device_)); } default: - PADDLE_THROW(platform::errors::Unavailable("Unsupported place: %d", - static_cast(place))); + PADDLE_THROW(paddle::platform::errors::Unavailable( + "Only CPU / CUDA / XPU places is supported. The place `%d` is not " + "supported.", + static_cast(place))); break; } return nullptr; } template -T *ZeroCopyTensor::data(PaddlePlace *place, int *size) const { +T *Tensor::data(PlaceType *place, int *size) const { EAGER_GET_TENSOR; auto *res = tensor->data(); - if (platform::is_cpu_place(tensor->place())) { - *place = PaddlePlace::kCPU; - } else if (platform::is_gpu_place(tensor->place())) { - *place = PaddlePlace::kGPU; + if (paddle::platform::is_cpu_place(tensor->place())) { + *place = PlaceType::kCPU; + } else if (paddle::platform::is_gpu_place(tensor->place())) { + *place = PlaceType::kGPU; + } else if (paddle::platform::is_xpu_place(tensor->place())) { + *place = PlaceType::kXPU; } else { - *place = PaddlePlace::kUNK; + *place = PlaceType::kUNK; } *size = tensor->numel(); return res; } -PaddleDType ZeroCopyTensor::type() const { +DataType Tensor::type() const { EAGER_GET_TENSOR; auto type = tensor->type(); - if (type == framework::proto::VarType::FP32) { - return PaddleDType::FLOAT32; - } else if (type == framework::proto::VarType::INT64) { - return PaddleDType::INT64; - } else if (type == framework::proto::VarType::INT32) { - return PaddleDType::INT32; - } else if (type == framework::proto::VarType::UINT8) { - return PaddleDType::UINT8; + if (type == paddle::framework::proto::VarType::FP32) { + return DataType::FLOAT32; + } else if (type == paddle::framework::proto::VarType::INT64) { + return DataType::INT64; + } else if (type == paddle::framework::proto::VarType::INT32) { + return DataType::INT32; + } else if (type == paddle::framework::proto::VarType::UINT8) { + return DataType::UINT8; } - return PaddleDType::FLOAT32; + return DataType::FLOAT32; } template -void ZeroCopyTensor::copy_from_cpu(const T *data) { +void Tensor::CopyFromCpu(const T *data) { EAGER_GET_TENSOR; PADDLE_ENFORCE_GE(tensor->numel(), 0, - platform::errors::PreconditionNotMet( - "You should call ZeroCopyTensor::Reshape(const " + paddle::platform::errors::PreconditionNotMet( + "You should call Tensor::Reshape(const " "std::vector &shape)" "function before copying data from cpu.")); size_t ele_size = tensor->numel() * sizeof(T); - if (place_ == PaddlePlace::kCPU) { - auto *t_data = tensor->mutable_data(platform::CPUPlace()); + if (place_ == PlaceType::kCPU) { + auto *t_data = tensor->mutable_data(paddle::platform::CPUPlace()); std::memcpy(static_cast(t_data), data, ele_size); - } else if (place_ == PaddlePlace::kGPU) { + } else if (place_ == PlaceType::kGPU) { #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - platform::CUDAPlace gpu_place(device_); + paddle::platform::DeviceContextPool &pool = + paddle::platform::DeviceContextPool::Instance(); + paddle::platform::CUDAPlace gpu_place(device_); auto *t_data = tensor->mutable_data(gpu_place); - auto *dev_ctx = - static_cast(pool.Get(gpu_place)); + auto *dev_ctx = static_cast( + pool.Get(gpu_place)); - memory::Copy(gpu_place, static_cast(t_data), platform::CPUPlace(), - data, ele_size, dev_ctx->stream()); + paddle::memory::Copy(gpu_place, static_cast(t_data), + paddle::platform::CPUPlace(), data, ele_size, + dev_ctx->stream()); #else - PADDLE_THROW(platform::errors::Unavailable( - "Not compiled with CUDA, should not reach here.")); + PADDLE_THROW(paddle::platform::errors::Unavailable( + "Can not create tensor with CUDA place because paddle is not compiled " + "with CUDA.")); #endif - } else if (place_ == PaddlePlace::kXPU) { + } else if (place_ == PlaceType::kXPU) { #ifdef PADDLE_WITH_XPU - platform::XPUPlace xpu_place(device_); + paddle::platform::XPUPlace xpu_place(device_); auto *t_data = tensor->mutable_data(xpu_place); - memory::Copy(xpu_place, static_cast(t_data), platform::CPUPlace(), - data, ele_size); + paddle::memory::Copy(xpu_place, static_cast(t_data), + paddle::platform::CPUPlace(), data, ele_size); #else - PADDLE_THROW(platform::errors::Unavailable( - "Not compiled with XPU, should not reach here.")); + PADDLE_THROW(paddle::platform::errors::Unavailable( + "Can not create tensor with XPU place because paddle is not compiled " + "with XPU.")); #endif } else { PADDLE_THROW(paddle::platform::errors::InvalidArgument( @@ -146,119 +155,119 @@ void ZeroCopyTensor::copy_from_cpu(const T *data) { } template -void ZeroCopyTensor::copy_to_cpu(T *data) { +void Tensor::CopyToCpu(T *data) { EAGER_GET_TENSOR; auto ele_num = tensor->numel(); auto *t_data = tensor->data(); auto t_place = tensor->place(); - if (platform::is_cpu_place(t_place)) { + if (paddle::platform::is_cpu_place(t_place)) { std::memcpy(static_cast(data), t_data, ele_num * sizeof(T)); - } else if (place_ == PaddlePlace::kGPU) { + } else if (place_ == PlaceType::kGPU) { #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) - platform::DeviceContextPool &pool = platform::DeviceContextPool::Instance(); - auto gpu_place = BOOST_GET_CONST(platform::CUDAPlace, t_place); - auto *dev_ctx = - static_cast(pool.Get(gpu_place)); - memory::Copy(platform::CPUPlace(), static_cast(data), gpu_place, - t_data, ele_num * sizeof(T), dev_ctx->stream()); + paddle::platform::DeviceContextPool &pool = + paddle::platform::DeviceContextPool::Instance(); + auto gpu_place = BOOST_GET_CONST(paddle::platform::CUDAPlace, t_place); + auto *dev_ctx = static_cast( + pool.Get(gpu_place)); + paddle::memory::Copy(paddle::platform::CPUPlace(), + static_cast(data), gpu_place, t_data, + ele_num * sizeof(T), dev_ctx->stream()); #ifdef PADDLE_WITH_HIP hipStreamSynchronize(dev_ctx->stream()); #else cudaStreamSynchronize(dev_ctx->stream()); #endif #else - PADDLE_THROW(platform::errors::Unavailable( - "Not compile with CUDA, should not reach here.")); + PADDLE_THROW(paddle::platform::errors::Unavailable( + "Can not create tensor with CUDA place because paddle is not compiled " + "with CUDA.")); #endif - } else if (place_ == PaddlePlace::kXPU) { + } else if (place_ == PlaceType::kXPU) { #ifdef PADDLE_WITH_XPU - auto xpu_place = BOOST_GET_CONST(platform::XPUPlace, t_place); - memory::Copy(platform::CPUPlace(), static_cast(data), xpu_place, - t_data, ele_num * sizeof(T)); + auto xpu_place = BOOST_GET_CONST(paddle::platform::XPUPlace, t_place); + paddle::memory::Copy(paddle::platform::CPUPlace(), + static_cast(data), xpu_place, t_data, + ele_num * sizeof(T)); #else - PADDLE_THROW(platform::errors::Unavailable( - "Not compile with XPU, should not reach here.")); + PADDLE_THROW(paddle::platform::errors::Unavailable( + "Can not create tensor with XPU place because paddle is not compiled " + "with XPU.")); #endif } else { PADDLE_THROW(paddle::platform::errors::InvalidArgument( "The analysis predictor supports CPU, GPU and XPU now.")); } } -template PD_INFER_DECL void ZeroCopyTensor::copy_from_cpu( - const float *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_from_cpu( - const int64_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_from_cpu( - const int32_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_from_cpu( - const uint8_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_from_cpu( - const int8_t *data); +template PD_INFER_DECL void Tensor::CopyFromCpu(const float *data); +template PD_INFER_DECL void Tensor::CopyFromCpu(const int64_t *data); +template PD_INFER_DECL void Tensor::CopyFromCpu(const int32_t *data); +template PD_INFER_DECL void Tensor::CopyFromCpu(const uint8_t *data); +template PD_INFER_DECL void Tensor::CopyFromCpu(const int8_t *data); + +template PD_INFER_DECL void Tensor::CopyToCpu(float *data); +template PD_INFER_DECL void Tensor::CopyToCpu(int64_t *data); +template PD_INFER_DECL void Tensor::CopyToCpu(int32_t *data); +template PD_INFER_DECL void Tensor::CopyToCpu(uint8_t *data); +template PD_INFER_DECL void Tensor::CopyToCpu(int8_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_to_cpu(float *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_to_cpu(int64_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_to_cpu(int32_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_to_cpu(uint8_t *data); -template PD_INFER_DECL void ZeroCopyTensor::copy_to_cpu(int8_t *data); +template PD_INFER_DECL float *Tensor::data(PlaceType *place, + int *size) const; +template PD_INFER_DECL int64_t *Tensor::data(PlaceType *place, + int *size) const; +template PD_INFER_DECL int32_t *Tensor::data(PlaceType *place, + int *size) const; +template PD_INFER_DECL uint8_t *Tensor::data(PlaceType *place, + int *size) const; +template PD_INFER_DECL int8_t *Tensor::data(PlaceType *place, + int *size) const; -template PD_INFER_DECL float *ZeroCopyTensor::data(PaddlePlace *place, - int *size) const; -template PD_INFER_DECL int64_t *ZeroCopyTensor::data( - PaddlePlace *place, int *size) const; -template PD_INFER_DECL int32_t *ZeroCopyTensor::data( - PaddlePlace *place, int *size) const; -template PD_INFER_DECL uint8_t *ZeroCopyTensor::data( - PaddlePlace *place, int *size) const; -template PD_INFER_DECL int8_t *ZeroCopyTensor::data(PaddlePlace *place, - int *size) const; +template PD_INFER_DECL float *Tensor::mutable_data(PlaceType place); +template PD_INFER_DECL int64_t *Tensor::mutable_data(PlaceType place); +template PD_INFER_DECL int32_t *Tensor::mutable_data(PlaceType place); +template PD_INFER_DECL uint8_t *Tensor::mutable_data(PlaceType place); +template PD_INFER_DECL int8_t *Tensor::mutable_data(PlaceType place); -template PD_INFER_DECL float *ZeroCopyTensor::mutable_data( - PaddlePlace place); -template PD_INFER_DECL int64_t *ZeroCopyTensor::mutable_data( - PaddlePlace place); -template PD_INFER_DECL int32_t *ZeroCopyTensor::mutable_data( - PaddlePlace place); -template PD_INFER_DECL uint8_t *ZeroCopyTensor::mutable_data( - PaddlePlace place); -template PD_INFER_DECL int8_t *ZeroCopyTensor::mutable_data( - PaddlePlace place); +Tensor::Tensor(void *scope) : scope_{scope} { + PADDLE_ENFORCE_NOT_NULL(scope_, + paddle::platform::errors::PreconditionNotMet( + "The `scope` can not be nullptr. It should be " + "set to the pointer of scope.")); +} -void *ZeroCopyTensor::FindTensor() const { +void *Tensor::FindTensor() const { PADDLE_ENFORCE_EQ( name_.empty(), false, - platform::errors::PreconditionNotMet( + paddle::platform::errors::PreconditionNotMet( "Need to SetName first, so that the corresponding tensor can " "be retrieved.")); - PADDLE_ENFORCE_NOT_NULL(scope_, platform::errors::PreconditionNotMet( - "The scope should not be nullptr.")); - auto *scope = static_cast(scope_); + auto *scope = static_cast(scope_); auto *var = scope->FindVar(name_); PADDLE_ENFORCE_NOT_NULL( - var, platform::errors::PreconditionNotMet( + var, paddle::platform::errors::PreconditionNotMet( "No tensor called [%s] in the runtime scope", name_)); - auto *tensor = var->GetMutable(); + auto *tensor = var->GetMutable(); return tensor; } -std::vector ZeroCopyTensor::shape() const { +std::vector Tensor::shape() const { EAGER_GET_TENSOR; PADDLE_ENFORCE_NOT_NULL( - tensor_, platform::errors::PreconditionNotMet( + tensor_, paddle::platform::errors::PreconditionNotMet( "Not found tensor called %s in the scope", name_)); - return framework::vectorize(tensor->dims()); + return paddle::framework::vectorize(tensor->dims()); } -void ZeroCopyTensor::SetLoD(const std::vector> &x) { +void Tensor::SetLoD(const std::vector> &x) { EAGER_GET_TENSOR; - framework::LoD lod; + paddle::framework::LoD lod; for (auto &level : x) { lod.emplace_back(level); } tensor->set_lod(lod); } -std::vector> ZeroCopyTensor::lod() const { +std::vector> Tensor::lod() const { EAGER_GET_TENSOR; std::vector> res; for (auto &level : tensor->lod()) { @@ -267,4 +276,13 @@ std::vector> ZeroCopyTensor::lod() const { return res; } -} // namespace paddle +void Tensor::SetName(const std::string &name) { name_ = name; } + +const std::string &Tensor::name() const { return name_; } + +void Tensor::SetPlace(PlaceType place, int device) { + place_ = place; + device_ = device; +} + +} // namespace paddle_infer diff --git a/paddle/fluid/inference/api/details/zero_copy_tensor_dummy.cc b/paddle/fluid/inference/api/details/zero_copy_tensor_dummy.cc index ea90bc74533a3..1f1be13610379 100644 --- a/paddle/fluid/inference/api/details/zero_copy_tensor_dummy.cc +++ b/paddle/fluid/inference/api/details/zero_copy_tensor_dummy.cc @@ -15,35 +15,35 @@ #include "paddle/fluid/inference/api/paddle_api.h" #include "paddle/fluid/inference/api/paddle_infer_declare.h" -namespace paddle { +namespace paddle_infer { -void ZeroCopyTensor::Reshape(const std::vector &shape) {} +void Tensor::Reshape(const std::vector &shape) {} template -T *ZeroCopyTensor::mutable_data(PaddlePlace place) { +T *Tensor::mutable_data(PlaceType place) { return nullptr; } template -T *ZeroCopyTensor::data(PaddlePlace *place, int *size) const { +T *Tensor::data(PlaceType *place, int *size) const { return nullptr; } -template PD_INFER_DECL float *ZeroCopyTensor::data(PaddlePlace *place, - int *size) const; -template PD_INFER_DECL int64_t *ZeroCopyTensor::data( - PaddlePlace *place, int *size) const; -template float *ZeroCopyTensor::mutable_data(PaddlePlace place); -template int64_t *ZeroCopyTensor::mutable_data(PaddlePlace place); +template PD_INFER_DECL float *Tensor::data(PlaceType *place, + int *size) const; +template PD_INFER_DECL int64_t *Tensor::data(PlaceType *place, + int *size) const; +template float *Tensor::mutable_data(PlaceType place); +template int64_t *Tensor::mutable_data(PlaceType place); -void *ZeroCopyTensor::FindTensor() const { return nullptr; } +void *Tensor::FindTensor() const { return nullptr; } -std::vector ZeroCopyTensor::shape() const { return {}; } +std::vector Tensor::shape() const { return {}; } -void ZeroCopyTensor::SetLoD(const std::vector> &x) {} +void Tensor::SetLoD(const std::vector> &x) {} -std::vector> ZeroCopyTensor::lod() const { +std::vector> Tensor::lod() const { return std::vector>(); } -} // namespace paddle +} // namespace paddle_infer diff --git a/paddle/fluid/inference/api/details/zero_copy_tensor_test.cc b/paddle/fluid/inference/api/details/zero_copy_tensor_test.cc new file mode 100644 index 0000000000000..7e709924e91f9 --- /dev/null +++ b/paddle/fluid/inference/api/details/zero_copy_tensor_test.cc @@ -0,0 +1,138 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include +#include +#include +#include +#include + +#include "paddle/fluid/framework/data_type.h" +#include "paddle/fluid/framework/scope.h" +#include "paddle/fluid/inference/api/helper.h" +#include "paddle/fluid/inference/api/paddle_tensor.h" +#include "paddle/fluid/platform/place.h" + +namespace paddle_infer { + +struct TensorWrapper : public Tensor { + TensorWrapper(paddle_infer::PlaceType place, paddle::framework::Scope* scope, + const std::string& name) + : Tensor{static_cast(scope)} { + SetPlace(place, 0 /*device_id*/); + SetName(name); + input_or_output_ = true; + } +}; + +std::unique_ptr CreateTensor(paddle_infer::PlaceType place, + paddle::framework::Scope* scope, + const std::string& name) { + return std::unique_ptr(new TensorWrapper{place, scope, name}); +} + +template +struct RandomGenerator { + RandomGenerator(double min = (std::numeric_limits::min)(), + double max = (std::numeric_limits::max)()) + : dist_{static_cast(min), static_cast(max)} {} + T operator()() { return static_cast(dist_(random_engine_)); } + + private: + std::mt19937_64 random_engine_{std::random_device()()}; + std::uniform_real_distribution dist_; +}; + +template class G> +bool FillRandomDataAndCheck(PlaceType place, size_t length, G&& generator, + float threshold = 10e-5) { + std::vector data_in(length); + std::generate(data_in.begin(), data_in.end(), std::forward>(generator)); + paddle::framework::Scope scope; + const std::string name{"name"}; + scope.Var(name); + auto tensor = CreateTensor(place, &scope, name); + tensor->CopyFromCpu(data_in.data()); + if (tensor->type() != paddle::inference::ConvertToPaddleDType( + paddle::framework::DataTypeTrait::DataType())) { + return false; + } + std::vector data_out(length); + tensor->CopyToCpu(data_out.data()); + for (size_t i = 0; i < length; ++i) { + if (std::abs(data_out[i] - data_out[i]) > threshold) { + return false; + } + } + return true; +} + +template +bool SetPlaceAndCheck(PlaceType place, size_t length) { + paddle::framework::Scope scope; + const std::string name{"name"}; + const std::vector> lod{{0, length}}; + scope.Var(name); + auto tensor = CreateTensor(place, &scope, name); + tensor->Reshape({static_cast(length)}); + tensor->mutable_data(place); + tensor->SetLoD(lod); + + PlaceType place_out{PlaceType::kUNK}; + int length_out{-1}; + tensor->data(&place_out, &length_out); + if (length_out != static_cast(length) || place_out != place) { + return false; + } + if (tensor->name() != name || tensor->lod() != lod) { + return false; + } + return true; +} + +bool FillRandomDataAndCheck(PlaceType place) { + const size_t length{RandomGenerator{1, 1000}()}; + VLOG(3) << "FillRandomDataAndCheck: length = " << length; + return FillRandomDataAndCheck(place, length, + RandomGenerator{}) && + FillRandomDataAndCheck(place, length, + RandomGenerator{}) && + FillRandomDataAndCheck(place, length, + RandomGenerator{}) && + FillRandomDataAndCheck(place, length, + RandomGenerator{}); +} + +bool SetPlaceAndCheck(PlaceType place) { + const size_t length{RandomGenerator{1, 1000}()}; + VLOG(3) << "SetPlaceAndCheck: length = " << length; + return SetPlaceAndCheck(place, length) && + SetPlaceAndCheck(place, length) && + SetPlaceAndCheck(place, length) && + SetPlaceAndCheck(place, length); +} + +TEST(Tensor, FillRandomDataAndCheck) { + ASSERT_TRUE(FillRandomDataAndCheck(PlaceType::kCPU)); + ASSERT_TRUE(SetPlaceAndCheck(PlaceType::kCPU)); +#ifdef PADDLE_WITH_CUDA + ASSERT_TRUE(FillRandomDataAndCheck(PlaceType::kGPU)); + ASSERT_TRUE(SetPlaceAndCheck(PlaceType::kGPU)); +#endif +} + +} // namespace paddle_infer diff --git a/paddle/fluid/inference/api/helper.h b/paddle/fluid/inference/api/helper.h index 061b83e1d1e4a..14b968f5834da 100644 --- a/paddle/fluid/inference/api/helper.h +++ b/paddle/fluid/inference/api/helper.h @@ -58,6 +58,26 @@ constexpr PaddleDType PaddleTensorGetDType() { return PaddleDType::FLOAT32; } +inline PaddleDType ConvertToPaddleDType( + paddle::framework::proto::VarType::Type type) { + if (type == paddle::framework::proto::VarType::FP32) { + return PaddleDType::FLOAT32; + } else if (type == paddle::framework::proto::VarType::INT64) { + return PaddleDType::INT64; + } else if (type == paddle::framework::proto::VarType::INT32) { + return PaddleDType::INT32; + } else if (type == paddle::framework::proto::VarType::UINT8) { + return PaddleDType::UINT8; + } else { + PADDLE_THROW(paddle::platform::errors::Unimplemented( + "The paddle dtype convert function only supports FLOAT32, INT64, INT32 " + "and UINT8 now. But " + "we get %d here.", + static_cast(type))); + return PaddleDType::FLOAT32; + } +} + using paddle::framework::DataTypeToString; // Timer for timer diff --git a/paddle/fluid/inference/api/paddle_api.h b/paddle/fluid/inference/api/paddle_api.h index c5893a23a4960..3e92ffaf9dcbc 100644 --- a/paddle/fluid/inference/api/paddle_api.h +++ b/paddle/fluid/inference/api/paddle_api.h @@ -29,19 +29,13 @@ #include #include "crypto/cipher.h" #include "paddle_infer_declare.h" // NOLINT +#include "paddle_tensor.h" // NOLINT /*! \namespace paddle */ namespace paddle { -/// \brief Paddle data type. -enum PaddleDType { - FLOAT32, - INT64, - INT32, - UINT8, - INT8, - // TODO(Superjomn) support more data types if needed. -}; +using PaddleDType = paddle_infer::DataType; +using PaddlePlace = paddle_infer::PlaceType; /// \brief Memory manager for PaddleTensor. /// @@ -162,8 +156,6 @@ struct PD_INFER_DECL PaddleTensor { std::vector> lod; ///< Tensor+LoD equals LoDTensor }; -enum class PaddlePlace { kUNK = -1, kCPU, kGPU, kXPU }; - /// \brief Represents an n-dimensional array of values. /// The ZeroCopyTensor is used to store the input or output of the network. /// Zero copy means that the tensor supports direct copy of host or device data @@ -172,79 +164,27 @@ enum class PaddlePlace { kUNK = -1, kCPU, kGPU, kXPU }; /// AnalysisPredictor. /// It is obtained through PaddlePredictor::GetinputTensor() /// and PaddlePredictor::GetOutputTensor() interface. -class PD_INFER_DECL ZeroCopyTensor { - public: - /// \brief Reset the shape of the tensor. - /// Generally it's only used for the input tensor. - /// Reshape must be called before calling mutable_data() or copy_from_cpu() - /// \param shape The shape to set. - void Reshape(const std::vector& shape); - - /// \brief Get the memory pointer in CPU or GPU with specific data type. - /// Please Reshape the tensor first before call this. - /// It's usually used to get input data pointer. - /// \param place The place of the tensor. - template - T* mutable_data(PaddlePlace place); - - /// \brief Get the memory pointer directly. - /// It's usually used to get the output data pointer. - /// \param[out] place To get the device type of the tensor. - /// \param[out] size To get the data size of the tensor. - /// \return The tensor data buffer pointer. - template - T* data(PaddlePlace* place, int* size) const; +class PD_INFER_DECL ZeroCopyTensor : public paddle_infer::Tensor { + public: /// \brief Copy the host memory to tensor data. /// It's usually used to set the input tensor data. /// \param data The pointer of the data, from which the tensor will copy. template - void copy_from_cpu(const T* data); - + void copy_from_cpu(const T* data) { + return CopyFromCpu(data); + } /// \brief Copy the tensor data to the host memory. /// It's usually used to get the output tensor data. /// \param[out] data The tensor will copy the data to the address. template - void copy_to_cpu(T* data); - - /// \brief Return the shape of the Tensor. - std::vector shape() const; - - /// \brief Set lod info of the tensor. - /// More about LOD can be seen here: - /// https://www.paddlepaddle.org.cn/documentation/docs/zh/beginners_guide/basic_concept/lod_tensor.html#lodtensor - /// \param x the lod info. - void SetLoD(const std::vector>& x); - /// \brief Return the lod info of the tensor. - std::vector> lod() const; - /// \brief Return the name of the tensor. - const std::string& name() const { return name_; } - void SetPlace(PaddlePlace place, int device = -1) { - place_ = place; - device_ = device; + void copy_to_cpu(T* data) { + return CopyToCpu(data); } - /// \brief Return the data type of the tensor. - /// It's usually used to get the output tensor data type. - /// \return The data type of the tensor. - PaddleDType type() const; - - protected: - explicit ZeroCopyTensor(void* scope) : scope_{scope} {} - void SetName(const std::string& name) { name_ = name; } - void* FindTensor() const; - private: - std::string name_; - bool input_or_output_; friend class AnalysisPredictor; - void* scope_{nullptr}; - // The corresponding tensor pointer inside Paddle workspace is cached for - // performance. - mutable void* tensor_{nullptr}; - PaddlePlace place_; - PaddleDType dtype_; - int device_; + explicit ZeroCopyTensor(void* scope) : paddle_infer::Tensor{scope} {} }; /// \brief A Predictor for executing inference on a model. diff --git a/paddle/fluid/inference/api/paddle_inference_api.h b/paddle/fluid/inference/api/paddle_inference_api.h index 2e1e3b822d164..a516abb1432ca 100644 --- a/paddle/fluid/inference/api/paddle_inference_api.h +++ b/paddle/fluid/inference/api/paddle_inference_api.h @@ -42,97 +42,10 @@ limitations under the License. */ /// namespace paddle_infer { -using DataType = paddle::PaddleDType; -using PlaceType = paddle::PaddlePlace; + using PrecisionType = paddle::AnalysisConfig::Precision; using Config = paddle::AnalysisConfig; -/// -/// \class Tensor -/// -/// \brief Represents an n-dimensional array of values. -/// The Tensor is used to store the input or output of the network. -/// It is obtained through Predictor::GetinputHandle() -/// and Predictor::GetOutputHandle() interface. -/// -class PD_INFER_DECL Tensor { - public: - // Can only be created by predictor->GetInputHandle(cosnt std::string& name) - // or predictor->GetOutputHandle(cosnt std::string& name) - Tensor() = delete; - explicit Tensor(std::unique_ptr&& tensor) - : tensor_(std::move(tensor)) {} - - /// - /// \brief Reset the shape of the tensor. - /// Generally it's only used for the input tensor. - /// Reshape must be called before calling mutable_data() or CopyFromCpu() - /// \param shape The shape to set. - /// - void Reshape(const std::vector& shape); - - /// - /// \brief Copy the host memory to tensor data. - /// It's usually used to set the input tensor data. - /// \param data The pointer of the data, from which the tensor will copy. - /// - template - void CopyFromCpu(const T* data); - - /// - /// \brief Get the memory pointer in CPU or GPU with specific data type. - /// Please Reshape the tensor first before call this. - /// It's usually used to get input data pointer. - /// \param place The place of the tensor. - /// \return The tensor data buffer pointer. - /// - template - T* mutable_data(PlaceType place); - - /// - /// \brief Copy the tensor data to the host memory. - /// It's usually used to get the output tensor data. - /// \param[out] data The tensor will copy the data to the address. - /// - template - void CopyToCpu(T* data); - - /// - /// \brief Get the memory pointer directly. - /// It's usually used to get the output data pointer. - /// \param[out] place To get the device type of the tensor. - /// \param[out] size To get the data size of the tensor. - /// \return The tensor data buffer pointer. - /// - template - T* data(PlaceType* place, int* size) const; - - /// - /// \brief Set lod info of the tensor. - /// More about LOD can be seen here: - /// https://www.paddlepaddle.org.cn/documentation/docs/zh/beginners_guide/basic_concept/lod_tensor.html#lodtensor - /// \param x the lod info. - /// - void SetLoD(const std::vector>& x); - - /// \brief Return the lod info of the tensor. - std::vector> lod() const; - - /// \brief Return the data type of the tensor. - /// It's usually used to get the output tensor data type. - /// \return The data type of the tensor. - DataType type() const; - - /// \brief Return the shape of the Tensor. - std::vector shape() const; - - /// \brief Return the name of the tensor. - const std::string& name() const; - - private: - std::unique_ptr tensor_; -}; - /// /// \class Predictor /// @@ -258,31 +171,7 @@ PD_INFER_DECL int GetNumBytesOfDataType(DataType dtype); PD_INFER_DECL std::string GetVersion(); PD_INFER_DECL std::string UpdateDllFlag(const char* name, const char* value); -template -void Tensor::CopyFromCpu(const T* data) { - tensor_->copy_from_cpu(data); -} - -template -void Tensor::CopyToCpu(T* data) { - return tensor_->copy_to_cpu(data); -} - -template -T* Tensor::mutable_data(PlaceType place) { - return tensor_->mutable_data(place); -} - -template -T* Tensor::data(PlaceType* place, int* size) const { - return tensor_->data(place, size); -} - -} // namespace paddle_infer - -namespace paddle_infer { namespace services { - /// /// \class PredictorPool /// @@ -308,4 +197,5 @@ class PD_INFER_DECL PredictorPool { std::vector> preds_; }; } // namespace services + } // namespace paddle_infer diff --git a/paddle/fluid/inference/api/paddle_tensor.h b/paddle/fluid/inference/api/paddle_tensor.h new file mode 100644 index 0000000000000..9c4e5858af3ad --- /dev/null +++ b/paddle/fluid/inference/api/paddle_tensor.h @@ -0,0 +1,111 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include "paddle_infer_declare.h" // NOLINT + +namespace paddle_infer { + +/// \brief Paddle data type. +enum DataType { + FLOAT32, + INT64, + INT32, + UINT8, + INT8, + // TODO(Superjomn) support more data types if needed. +}; + +enum class PlaceType { kUNK = -1, kCPU, kGPU, kXPU }; + +/// \brief Represents an n-dimensional array of values. +/// The Tensor is used to store the input or output of the network. +/// Zero copy means that the tensor supports direct copy of host or device data +/// to device, +/// eliminating additional CPU copy. Tensor is only used in the +/// AnalysisPredictor. +/// It is obtained through PaddlePredictor::GetinputTensor() +/// and PaddlePredictor::GetOutputTensor() interface. +class PD_INFER_DECL Tensor { + public: + /// \brief Reset the shape of the tensor. + /// Generally it's only used for the input tensor. + /// Reshape must be called before calling mutable_data() or copy_from_cpu() + /// \param shape The shape to set. + void Reshape(const std::vector& shape); + + /// \brief Get the memory pointer in CPU or GPU with specific data type. + /// Please Reshape the tensor first before call this. + /// It's usually used to get input data pointer. + /// \param place The place of the tensor. + template + T* mutable_data(PlaceType place); + + /// \brief Get the memory pointer directly. + /// It's usually used to get the output data pointer. + /// \param[out] place To get the device type of the tensor. + /// \param[out] size To get the data size of the tensor. + /// \return The tensor data buffer pointer. + template + T* data(PlaceType* place, int* size) const; + + /// \brief Copy the host memory to tensor data. + /// It's usually used to set the input tensor data. + /// \param data The pointer of the data, from which the tensor will copy. + template + void CopyFromCpu(const T* data); + + /// \brief Copy the tensor data to the host memory. + /// It's usually used to get the output tensor data. + /// \param[out] data The tensor will copy the data to the address. + template + void CopyToCpu(T* data); + + /// \brief Return the shape of the Tensor. + std::vector shape() const; + + /// \brief Set lod info of the tensor. + /// More about LOD can be seen here: + /// https://www.paddlepaddle.org.cn/documentation/docs/zh/beginners_guide/basic_concept/lod_tensor.html#lodtensor + /// \param x the lod info. + void SetLoD(const std::vector>& x); + /// \brief Return the lod info of the tensor. + std::vector> lod() const; + /// \brief Return the name of the tensor. + const std::string& name() const; + + /// \brief Return the data type of the tensor. + /// It's usually used to get the output tensor data type. + /// \return The data type of the tensor. + DataType type() const; + + protected: + explicit Tensor(void* scope); + void* FindTensor() const; + void SetPlace(PlaceType place, int device = -1); + void SetName(const std::string& name); + + std::string name_; + // The corresponding tensor pointer inside Paddle workspace is cached for + // performance. + mutable void* tensor_{nullptr}; + DataType dtype_; + bool input_or_output_; + void* scope_{nullptr}; + PlaceType place_; + int device_; +}; + +} // namespace paddle_infer diff --git a/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt b/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt index f9586ca1701f7..b0d0229ec0531 100644 --- a/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt +++ b/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt @@ -5,6 +5,9 @@ nv_library(tensorrt_converter pad_op.cc split_op.cc prelu_op.cc leaky_relu_op.cc gelu_op.cc layer_norm_op.cc multihead_matmul_op.cc shuffle_channel_op.cc swish_op.cc instance_norm_op.cc stack_op.cc transpose_op.cc flatten_op.cc emb_eltwise_layernorm.cc skip_layernorm.cc scale_op.cc slice_op.cc hard_sigmoid_op.cc hard_swish_op.cc clip_op.cc + gather_op.cc + + nearest_interp_op.cc DEPS tensorrt_engine tensorrt_plugin operator scope framework_proto op_registry) nv_test(test_op_converter SRCS test_op_converter.cc DEPS diff --git a/paddle/fluid/inference/tensorrt/convert/conv2d_op.cc b/paddle/fluid/inference/tensorrt/convert/conv2d_op.cc index 6871d53f42ccd..5515cd35daedc 100644 --- a/paddle/fluid/inference/tensorrt/convert/conv2d_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/conv2d_op.cc @@ -97,6 +97,10 @@ void ConvertConv2d(TensorRTEngine* engine, const framework::proto::OpDesc& op, BOOST_GET_CONST(std::vector, op_desc.GetAttr("strides")); const std::vector paddings = BOOST_GET_CONST(std::vector, op_desc.GetAttr("paddings")); + std::string padding_algorithm = "EXPLICIT"; + if (op_desc.HasAttr("padding_algorithm")) + padding_algorithm = + BOOST_GET_CONST(std::string, op_desc.GetAttr("padding_algorithm")); nvinfer1::DimsHW nv_ksize(filter_h, filter_w); nvinfer1::DimsHW nv_dilations(dilations[0], dilations[1]); @@ -126,6 +130,9 @@ void ConvertConv2d(TensorRTEngine* engine, const framework::proto::OpDesc& op, layer->setStride(nv_strides); layer->setPadding(nv_paddings); layer->setNbGroups(groups); + if (padding_algorithm == "SAME") { + layer->setPaddingMode(nvinfer1::PaddingMode::kSAME_UPPER); + } // set dilations fset_dilation(layer, nv_dilations); diff --git a/paddle/fluid/inference/tensorrt/convert/fc_op.cc b/paddle/fluid/inference/tensorrt/convert/fc_op.cc index 41fbbb557d647..527d0ee208578 100644 --- a/paddle/fluid/inference/tensorrt/convert/fc_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/fc_op.cc @@ -144,7 +144,69 @@ class FcOpConverter : public OpConverter { static_cast(bias_num)}; if (engine_->with_dynamic_shape()) { - regist_fc(X, n_output, weight, bias); + // not NCHW layout, but NLP layout with added 'x 1 x 1' + auto x_dim = X->getDimensions(); + if (x_dim.nbDims == 3 || x_dim.nbDims == 2) { + auto output_name = op_desc.Output("Out").front(); + // add shuffle before fc + nvinfer1::Dims reshape_before_fc_dim; + reshape_before_fc_dim.nbDims = x_dim.nbDims + 2; + for (int i = 0; i < x_dim.nbDims; i++) { + reshape_before_fc_dim.d[i] = 0; + } + reshape_before_fc_dim.d[x_dim.nbDims] = 1; + reshape_before_fc_dim.d[x_dim.nbDims + 1] = 1; + auto* reshape_before_fc_layer = + TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *X); + reshape_before_fc_layer->setReshapeDimensions(reshape_before_fc_dim); + reshape_before_fc_layer->setName( + ("shuffle_before_fc(Output: " + output_name + ")").c_str()); + + // add fc layer + auto* fc_layer = TRT_ENGINE_ADD_LAYER( + engine_, FullyConnected, *reshape_before_fc_layer->getOutput(0), + n_output, weight.get(), bias.get()); + fc_layer->setName(("fc_layer(Output: " + output_name + ")").c_str()); + + // add shuffle after fc + nvinfer1::Dims reshape_after_fc_dim; + if (x_dim.nbDims == 3) { + if (x_num_col_dims == 2) { + reshape_after_fc_dim.nbDims = 3; + reshape_after_fc_dim.d[0] = 0; + reshape_after_fc_dim.d[1] = 0; + reshape_after_fc_dim.d[2] = 0; + } else { + reshape_after_fc_dim.nbDims = 2; + reshape_after_fc_dim.d[0] = 0; + auto dim = fc_layer->getOutput(0)->getDimensions(); + reshape_after_fc_dim.d[1] = dim.d[1] * dim.d[2]; + } + // x_dim.nbDims == 2 + } else { + reshape_after_fc_dim.nbDims = 2; + reshape_after_fc_dim.d[0] = 0; + reshape_after_fc_dim.d[1] = 0; + } + auto* reshape_after_fc_layer = + TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *fc_layer->getOutput(0)); + reshape_after_fc_layer->setReshapeDimensions(reshape_after_fc_dim); + + if (activation_type == "relu") { + reshape_after_fc_layer->setName( + ("shuffle_after_fc(Output: " + output_name + ")").c_str()); + nvinfer1::IActivationLayer* relu_layer = TRT_ENGINE_ADD_LAYER( + engine_, Activation, *(reshape_after_fc_layer->getOutput(0)), + nvinfer1::ActivationType::kRELU); + RreplenishLayerAndOutput(relu_layer, "relu_after_fc_shuffle", + {output_name}, test_mode); + } else { + RreplenishLayerAndOutput(reshape_after_fc_layer, "shuffle_after_fc", + {output_name}, test_mode); + } + } else { + regist_fc(X, n_output, weight, bias); + } return; } // in order to handle situations in NLP models(input dims < 3, @@ -154,12 +216,6 @@ class FcOpConverter : public OpConverter { auto input_d = X->getDimensions().d; int reshape_dim3[3] = {0}; int reshape_dim4[4] = {0}; - PADDLE_ENFORCE_EQ( - x_num_col_dims == 1 || x_num_col_dims == 2, true, - platform::errors::InvalidArgument( - "Wrong x_num_col_dims param of op mul. Paddle-TRT FC converter " - "expects x_num_col_dims is either 1 or 2, but got %d", - x_num_col_dims)); PADDLE_ENFORCE_LE(x_num_col_dims, input_dims, platform::errors::InvalidArgument( "Params and input dims mismatch. Paddle-TRT FC " diff --git a/paddle/fluid/inference/tensorrt/convert/gather_op.cc b/paddle/fluid/inference/tensorrt/convert/gather_op.cc new file mode 100644 index 0000000000000..346a8bffa00e3 --- /dev/null +++ b/paddle/fluid/inference/tensorrt/convert/gather_op.cc @@ -0,0 +1,78 @@ +/* Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. + +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 "paddle/fluid/inference/tensorrt/convert/op_converter.h" + +namespace paddle { +namespace framework { +class Scope; + +namespace proto { +class OpDesc; +} // namespace proto +} // namespace framework +} // namespace paddle + +namespace paddle { +namespace inference { +namespace tensorrt { + +/* + * Gather Op + */ +class GatherOpConverter : public OpConverter { + public: + void operator()(const framework::proto::OpDesc& op, + const framework::Scope& scope, bool test_mode) override { + VLOG(3) << "convert a fluid gather op to tensorrt gather layer"; + + framework::OpDesc op_desc(op, nullptr); + std::string input_name = op_desc.Input("X").front(); + std::string index_name = op_desc.Input("Index").front(); + std::string output_name = op_desc.Output("Out").front(); + + const auto input_tensor = engine_->GetITensor(input_name); + const auto index_tensor = engine_->GetITensor(index_name); + + const int axis = 0; + + auto layer = TRT_ENGINE_ADD_LAYER(engine_, Gather, *input_tensor, + *index_tensor, axis); + + auto odim = layer->getOutput(0)->getDimensions(); + + auto reshape_layer = + TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *layer->getOutput(0)); + + nvinfer1::Dims target_shape{}; + target_shape.nbDims = odim.nbDims - 1; + for (int i = 0; i < axis; ++i) { + target_shape.d[i] = odim.d[i]; + } + target_shape.d[axis] = 0; + for (int i = axis + 1; i < target_shape.nbDims; ++i) { + target_shape.d[i] = odim.d[i + 1]; + } + + reshape_layer->setReshapeDimensions(target_shape); + + RreplenishLayerAndOutput(reshape_layer, "gather", {output_name}, test_mode); + } +}; + +} // namespace tensorrt +} // namespace inference +} // namespace paddle + +REGISTER_TRT_OP_CONVERTER(gather, GatherOpConverter); diff --git a/paddle/fluid/inference/tensorrt/convert/multihead_matmul_op.cc b/paddle/fluid/inference/tensorrt/convert/multihead_matmul_op.cc index 736315d3b53e1..8ce46a19d4b06 100644 --- a/paddle/fluid/inference/tensorrt/convert/multihead_matmul_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/multihead_matmul_op.cc @@ -8,8 +8,8 @@ You may obtain a copy of the License at 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 +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 "paddle/fluid/inference/tensorrt/convert/op_converter.h" @@ -28,7 +28,6 @@ class MultiheadMatMulOpConverter : public OpConverter { "network structure"; framework::OpDesc op_desc(op, nullptr); // Declare inputs - // Shouble be a 5 dims tensor. auto* input = engine_->GetITensor(op_desc.Input("Input").front()); // fc weights and fc bias @@ -49,14 +48,14 @@ class MultiheadMatMulOpConverter : public OpConverter { memcpy(weight_data_tmp.data(), weight_data, weight_t->numel() * sizeof(float)); - // (hidden, 3, all_head_size) + // (hidden_in, 3, hidden_out) auto weight_dims = weight_t->dims(); - int hidden = weight_dims[0]; // channels_in - int three = weight_dims[1]; // channels_out - int all_head_size = weight_dims[2]; // channels_out - int m = hidden; - int n = three * all_head_size; + int hidden_in = weight_dims[0]; // channels_in + int three = weight_dims[1]; // channels_out + int hidden_out = weight_dims[2]; // channels_out + int m = hidden_in; + int n = three * hidden_out; auto tranpose_weight = [](const float* src, float* dst, int m, int n) { for (int i = 0; i < m; i++) { for (int j = 0; j < n; j++) { @@ -69,24 +68,27 @@ class MultiheadMatMulOpConverter : public OpConverter { int head_number = BOOST_GET_CONST(int, op_desc.GetAttr("head_number")); nvinfer1::ILayer* layer = nullptr; + auto output_name = op_desc.Output("Out")[0]; if (engine_->with_dynamic_shape()) { if (engine_->use_oss()) { - int head_size = hidden / head_number; - // [3, Nout, Hout, Nin, Hin] -> [Nout, 3, Hout, Nin, Hin] - auto transpose_weight_v2 = [](const float* src, float* dst, int N, - int H) { - const int HNH = H * N * H; - for (int i = 0; i < 3; ++i) { - for (int n = 0; n < N; ++n) { - for (int hnh = 0; hnh < HNH; ++hnh) { - dst[n * 3 * HNH + i * HNH + hnh] = - src[i * N * HNH + n * HNH + hnh]; + int head_size = hidden_out / head_number; + // [3, head_number, head_size, hidden_in] -> [head_number, 3, head_size, + // hidden_in] + auto transpose_weight_v2 = [](const float* src, float* dst, int three, + int head_number, int head_size, + int hidden_in) { + const int HH = head_size * hidden_in; + for (int i = 0; i < three; ++i) { + for (int n = 0; n < head_number; ++n) { + for (int hh = 0; hh < HH; ++hh) { + dst[n * three * HH + i * HH + hh] = + src[i * head_number * HH + n * HH + hh]; } } } }; - // [3, N, H] -> [N, 3, H] + // [3, head_number, head_size] -> [head_number, 3, head_size] auto transpose_bias_v2 = [](const float* src, float* dst, int N, int H) { for (int i = 0; i < 3; ++i) { @@ -99,8 +101,8 @@ class MultiheadMatMulOpConverter : public OpConverter { }; memcpy(weight_data_tmp.data(), weight_data, weight_t->numel() * sizeof(float)); - transpose_weight_v2(weight_data_tmp.data(), weight_data, head_number, - head_size); + transpose_weight_v2(weight_data_tmp.data(), weight_data, three, + head_number, head_size, hidden_in); nvinfer1::Weights weight{nvinfer1::DataType::kFLOAT, static_cast(weight_data), static_cast(weight_t->numel())}; @@ -130,7 +132,7 @@ class MultiheadMatMulOpConverter : public OpConverter { int var_seqlen = 1; const std::vector fields{ {"type_id", &type, nvinfer1::PluginFieldType::kINT32, 1}, - {"hidden_size", &hidden, nvinfer1::PluginFieldType::kINT32, 1}, + {"hidden_size", &hidden_out, nvinfer1::PluginFieldType::kINT32, 1}, {"num_heads", &head_number, nvinfer1::PluginFieldType::kINT32, 1}, {"has_mask", &has_mask, nvinfer1::PluginFieldType::kINT32, 1}, {"var_seqlen", &var_seqlen, nvinfer1::PluginFieldType::kINT32, 1}, @@ -169,6 +171,12 @@ class MultiheadMatMulOpConverter : public OpConverter { plugin_inputs.data(), plugin_inputs.size(), *plugin); layer = plugin_layer; } else { + PADDLE_ENFORCE_EQ( + input->getDimensions().nbDims, 3, + platform::errors::InvalidArgument( + "The Input dim of the MultiheadMatMul should be 3, " + "but it's (%d) now.", + input->getDimensions().nbDims)); // transpose weight_data from m * n to n * m auto* input_bias_qk = engine_->GetITensor(op_desc.Input("BiasQK").front()); @@ -182,20 +190,42 @@ class MultiheadMatMulOpConverter : public OpConverter { static_cast(bias_data), static_cast(bias_t->numel())}; - auto* fc_layer = TRT_ENGINE_ADD_LAYER(engine_, FullyConnected, *input, - n, weight.get(), bias.get()); - auto* fc_out = fc_layer->getOutput(0); + // add shuffle before fc + nvinfer1::Dims reshape_before_fc_dim; + reshape_before_fc_dim.nbDims = 5; + reshape_before_fc_dim.d[0] = 0; + reshape_before_fc_dim.d[1] = 0; + reshape_before_fc_dim.d[2] = 0; + reshape_before_fc_dim.d[3] = 1; + reshape_before_fc_dim.d[4] = 1; + auto* reshape_before_fc_layer = + TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *input); + reshape_before_fc_layer->setReshapeDimensions(reshape_before_fc_dim); + reshape_before_fc_layer->setName( + ("shuffle_before_multihead_mamul(Output: " + output_name + ")") + .c_str()); + + // add layer fc + auto* fc_layer = TRT_ENGINE_ADD_LAYER( + engine_, FullyConnected, *reshape_before_fc_layer->getOutput(0), n, + weight.get(), bias.get()); + fc_layer->setName( + ("multihead_mamul_fc(Output: " + output_name + ")").c_str()); + + // no need to add shuffle after fc, just change it in + // QkvToContextPluginDynamic + // add qkv to context - int head_size = all_head_size / head_number; + int head_size = hidden_out / head_number; float scale = BOOST_GET_CONST(float, op_desc.GetAttr("alpha")); std::vector plugin_inputs; - plugin_inputs.push_back(fc_out); + plugin_inputs.push_back(fc_layer->getOutput(0)); plugin_inputs.push_back(input_bias_qk); bool with_fp16 = engine_->WithFp16() && !engine_->disable_trt_plugin_fp16(); plugin::DynamicPluginTensorRT* plugin = - new plugin::QkvToContextPluginDynamic(hidden, head_number, + new plugin::QkvToContextPluginDynamic(hidden_in, head_number, head_size, scale, with_fp16); layer = engine_->AddPluginV2(plugin_inputs.data(), 2, plugin); } @@ -206,7 +236,6 @@ class MultiheadMatMulOpConverter : public OpConverter { "You can use the config.SetTRTDynamicShapeInfo(...) interface to set " "the shape information to run the dynamic shape mode.")); } - auto output_name = op_desc.Output("Out")[0]; RreplenishLayerAndOutput(layer, "multihead_matmul", {output_name}, test_mode); #else diff --git a/paddle/fluid/inference/tensorrt/convert/nearest_interp_op.cc b/paddle/fluid/inference/tensorrt/convert/nearest_interp_op.cc new file mode 100644 index 0000000000000..e91a2ee13f4c2 --- /dev/null +++ b/paddle/fluid/inference/tensorrt/convert/nearest_interp_op.cc @@ -0,0 +1,114 @@ +/* Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. +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 "paddle/fluid/framework/data_layout.h" +#include "paddle/fluid/inference/tensorrt/convert/op_converter.h" + +namespace paddle { +namespace framework { +class Scope; +namespace proto { +class OpDesc; +} // namespace proto +} // namespace framework +} // namespace paddle + +namespace paddle { +namespace inference { +namespace tensorrt { + +class NearestInterpolateOpConverter : public OpConverter { + public: + void operator()(const framework::proto::OpDesc& op, + const framework::Scope& scope, bool test_mode) override { + VLOG(3) << "convert a fluid nearest_interp op"; + + framework::OpDesc op_desc(op, nullptr); + + std::string input_name = op_desc.Input("X").front(); + std::string output_name = op_desc.Output("Out").front(); + + auto input = engine_->GetITensor(input_name); + + auto data_layout = framework::StringToDataLayout( + BOOST_GET_CONST(std::string, op_desc.GetAttr("data_layout"))); + auto interp_method = + BOOST_GET_CONST(std::string, op_desc.GetAttr("interp_method")); + bool align_corners = + BOOST_GET_CONST(bool, op_desc.GetAttr("align_corners")); + + auto input_names = op_desc.Input("X"); + auto scale = BOOST_GET_CONST(float, op_desc.GetAttr("scale")); + auto out_h = BOOST_GET_CONST(int, op_desc.GetAttr("out_h")); + auto out_w = BOOST_GET_CONST(int, op_desc.GetAttr("out_w")); + + auto layer = TRT_ENGINE_ADD_LAYER(engine_, Resize, *input); + layer->setAlignCorners(align_corners); + + auto in_dim = input->getDimensions(); + + float scale_h = 1.f; + float scale_w = 1.f; + + std::vector scales; + + if (scale > 0.f && (out_h <= 0 && out_w <= 0)) { + scale_h = scale; + scale_w = scale; + } else { + // axis are different in static/dynamic mode + PADDLE_ENFORCE_GT( + out_h, 0, platform::errors::InvalidArgument( + "out_h must be greater than 0 if scale is not set.")); + PADDLE_ENFORCE_GT( + out_w, 0, platform::errors::InvalidArgument( + "out_w must be greater than 0 if scale is not set.")); + + bool with_dynamic = engine_->with_dynamic_shape(); + + int h_axis = (data_layout == framework::DataLayout::kNCHW) + with_dynamic; + int w_axis = + (data_layout == framework::DataLayout::kNCHW) + 1 + with_dynamic; + + scale_h = + static_cast(out_h) / static_cast(in_dim.d[h_axis]); + scale_w = + static_cast(out_w) / static_cast(in_dim.d[w_axis]); + } + + if (engine_->with_dynamic_shape()) { + scales.push_back(1.f); + } + + if (data_layout == framework::DataLayout::kNCHW) { + scales.push_back(1.f); + scales.push_back(scale_h); + scales.push_back(scale_w); + } else if (data_layout == framework::DataLayout::kNHWC) { + // NHWC + scales.push_back(scale_h); + scales.push_back(scale_w); + scales.push_back(1.f); + } else { + PADDLE_THROW(platform::errors::InvalidArgument( + "Data layout must be NCHW or NHWC.")); + } + layer->setScales(scales.data(), scales.size()); + + RreplenishLayerAndOutput(layer, "nearest_interp", {output_name}, test_mode); + } +}; + +} // namespace tensorrt +} // namespace inference +} // namespace paddle + +REGISTER_TRT_OP_CONVERTER(nearest_interp, NearestInterpolateOpConverter); diff --git a/paddle/fluid/inference/tensorrt/convert/prelu_op.cc b/paddle/fluid/inference/tensorrt/convert/prelu_op.cc index 0de93624f1758..5e881ecbbc4e2 100644 --- a/paddle/fluid/inference/tensorrt/convert/prelu_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/prelu_op.cc @@ -72,9 +72,34 @@ class PReluOpConverter : public OpConverter { "your TRT version is no less than 6.0")); #endif } else { +#if IS_TRT_VERSION_GE(7000) + float* alpha_weight_data = engine_->GetWeightCPUData( + op_desc.Input("Alpha")[0], alpha_tensor, false); + TensorRTEngine::Weight alpha_weight{ + nvinfer1::DataType::kFLOAT, static_cast(alpha_weight_data), + static_cast(alpha_tensor->numel())}; + + nvinfer1::Dims dims; + dims.nbDims = 0; + // jump batch dim + for (int i = 1; i < alpha_tensor->dims().size(); i++) { + dims.d[dims.nbDims++] = alpha_tensor->dims()[i]; + } + for (; dims.nbDims < input->getDimensions().nbDims; dims.nbDims++) { + dims.d[dims.nbDims] = 1; + } + + auto alpha_layer = + TRT_ENGINE_ADD_LAYER(engine_, Constant, dims, alpha_weight.get()); + auto alpha_layer_output = alpha_layer->getOutput(0); + + layer = TRT_ENGINE_ADD_LAYER(engine_, ParametricReLU, *input, + *alpha_layer_output); +#else plugin::PReluPlugin* plugin = new plugin::PReluPlugin(alpha_data, alpha_tensor_temp->numel(), mode); layer = engine_->AddPlugin(&input, input_num, plugin); +#endif } // keep alpha tensor to avoid release it's memory engine_->SetWeights(op_desc.Input("Alpha")[0], diff --git a/paddle/fluid/inference/tensorrt/convert/scale_op.cc b/paddle/fluid/inference/tensorrt/convert/scale_op.cc index 1cc0bd30c7bbc..b527f2db53808 100644 --- a/paddle/fluid/inference/tensorrt/convert/scale_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/scale_op.cc @@ -58,6 +58,8 @@ class ScaleOpConverter : public OpConverter { return tmp_data; }; + int dynamic_shape_offset = engine_->with_dynamic_shape() ? 1 : 0; + float* bias_ptr = create_weights(bias, "bias"); float* scale_ptr = create_weights(scale, "scale"); @@ -70,19 +72,22 @@ class ScaleOpConverter : public OpConverter { nvinfer1::ILayer* layer = nullptr; auto input_dim = input->getDimensions(); - PADDLE_ENFORCE_GE(input_dim.nbDims, 3, - platform::errors::Fatal( - "Paddle-TRT scale mode only support dimension >= 3")); nvinfer1::IShuffleLayer* expand_layer = nullptr; nvinfer1::IShuffleLayer* squeeze_layer = nullptr; - if (input_dim.nbDims == 3) { - // TensorRT scale layer is not supporting input dims < 4 when using - // explicit batch + if (input_dim.nbDims < 3 + dynamic_shape_offset) { + nvinfer1::Dims expand_shape; + expand_shape.nbDims = 3 + dynamic_shape_offset; + for (int i = 0; i < 3 + dynamic_shape_offset; i++) { + if (i < input_dim.nbDims) { + expand_shape.d[i] = input_dim.d[i] < 0 ? 0 : input_dim.d[i]; + } else { + expand_shape.d[i] = 1; + } + } expand_layer = TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *input); - nvinfer1::Dims4 target_shape(0, 0, 0, 1); // expand 1 dims - expand_layer->setReshapeDimensions(target_shape); + expand_layer->setReshapeDimensions(expand_shape); input = expand_layer->getOutput(0); } @@ -104,13 +109,15 @@ class ScaleOpConverter : public OpConverter { PADDLE_ENFORCE_EQ(layer != nullptr, true, platform::errors::Fatal("Create scale layer failed.")); - if (input_dim.nbDims == 3) { - // TensorRT scale layer is not supporting input dims < 4 when using - // explicit batch + if (input_dim.nbDims < 3 + dynamic_shape_offset) { + nvinfer1::Dims squeeze_shape; + squeeze_shape.nbDims = input_dim.nbDims; + for (int i = 0; i < squeeze_shape.nbDims; i++) { + squeeze_shape.d[i] = input_dim.d[i] < 0 ? 0 : input_dim.d[i]; + } squeeze_layer = TRT_ENGINE_ADD_LAYER(engine_, Shuffle, *(layer->getOutput(0))); - nvinfer1::Dims3 target_shape(0, 0, 0); // expand 1 dims - squeeze_layer->setReshapeDimensions(target_shape); + squeeze_layer->setReshapeDimensions(squeeze_shape); layer = static_cast(squeeze_layer); } RreplenishLayerAndOutput(layer, "scale", {out_name}, test_mode); diff --git a/paddle/fluid/inference/tensorrt/convert/softmax_op.cc b/paddle/fluid/inference/tensorrt/convert/softmax_op.cc index 79992065a2240..9cefb24751e18 100644 --- a/paddle/fluid/inference/tensorrt/convert/softmax_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/softmax_op.cc @@ -51,6 +51,7 @@ class SoftMaxOpConverter : public OpConverter { uint32_t axes = std::max(0, input_dims - 3); // TODO(cryoco): Poor workaround. Fix padded dims problem when TRT layers // support Nd. + // Tips: Dynammic shape alreay fixes. int padded_dims = 0; int explicit_batch = 0; if (engine_->with_dynamic_shape()) explicit_batch = 1; @@ -62,16 +63,16 @@ class SoftMaxOpConverter : public OpConverter { } } if (!engine_->with_dynamic_shape()) { - if (axis == -1) { - axes = input_dims - 1 - padded_dims; + if (axis < 0) { + axes = input_dims + axis - padded_dims; } else { - axes = axis; + axes = axis - 1; } } else { - if (axis == -1) { - axes = input_dims - 1 - padded_dims; + if (axis < 0) { + axes = input_dims + axis; } else { - axes = axis + 1; + axes = axis; } } layer->setAxes(1 << axes); diff --git a/paddle/fluid/inference/tensorrt/convert/split_op.cc b/paddle/fluid/inference/tensorrt/convert/split_op.cc index 768c6efaa6bd4..5d494c2093b2a 100644 --- a/paddle/fluid/inference/tensorrt/convert/split_op.cc +++ b/paddle/fluid/inference/tensorrt/convert/split_op.cc @@ -101,7 +101,7 @@ class SplitOpConverter : public OpConverter { engine_->WithFp16() && !engine_->disable_trt_plugin_fp16(); plugin::SplitPlugin* plugin = new plugin::SplitPlugin(axis, output_lengths, with_fp16); - layer = engine_->AddPlugin(&input, input_num, plugin); + layer = engine_->AddPluginV2Ext(&input, input_num, plugin); } std::string layer_name = "split (Output: "; diff --git a/paddle/fluid/inference/tensorrt/engine.cc b/paddle/fluid/inference/tensorrt/engine.cc index 0bba4581ff90f..99549fd6b5cbf 100644 --- a/paddle/fluid/inference/tensorrt/engine.cc +++ b/paddle/fluid/inference/tensorrt/engine.cc @@ -18,7 +18,7 @@ limitations under the License. */ #include #include -#include "cuda_runtime_api.h" +#include "cuda_runtime_api.h" // NOLINT #include "paddle/fluid/inference/tensorrt/helper.h" #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/gpu_info.h" @@ -353,6 +353,13 @@ nvinfer1::IPluginLayer *TensorRTEngine::AddPlugin( return network()->addPluginExt(inputs, num_inputs, *plugin); } +nvinfer1::IPluginV2Layer *TensorRTEngine::AddPluginV2Ext( + nvinfer1::ITensor *const *inputs, int num_inputs, + plugin::PluginTensorRTV2Ext *plugin) { + owned_plugin_v2ext_.emplace_back(plugin); + return network()->addPluginV2(inputs, num_inputs, *plugin); +} + void TensorRTEngine::freshDeviceId() { int count; cudaGetDeviceCount(&count); diff --git a/paddle/fluid/inference/tensorrt/engine.h b/paddle/fluid/inference/tensorrt/engine.h index 0e399578fa446..de2924824f09d 100644 --- a/paddle/fluid/inference/tensorrt/engine.h +++ b/paddle/fluid/inference/tensorrt/engine.h @@ -305,8 +305,14 @@ class TensorRTEngine { } int GetDeviceId() { return device_id_; } + nvinfer1::IPluginLayer* AddPlugin(nvinfer1::ITensor* const* inputs, int num_inputs, plugin::PluginTensorRT*); + + nvinfer1::IPluginV2Layer* AddPluginV2Ext(nvinfer1::ITensor* const* inputs, + int num_inputs, + plugin::PluginTensorRTV2Ext* plugin); + void SetTensorDynamicRange(nvinfer1::ITensor* tensor, float range) { quant_dynamic_range_[tensor] = range; } @@ -414,6 +420,7 @@ class TensorRTEngine { itensor_map_; std::vector> owned_plugin_; + std::vector> owned_plugin_v2ext_; // TensorRT related internal members template diff --git a/paddle/fluid/inference/tensorrt/op_teller.cc b/paddle/fluid/inference/tensorrt/op_teller.cc index 4eac38a04f88b..11752d71a45e1 100644 --- a/paddle/fluid/inference/tensorrt/op_teller.cc +++ b/paddle/fluid/inference/tensorrt/op_teller.cc @@ -14,6 +14,7 @@ #include "paddle/fluid/inference/tensorrt/op_teller.h" #include "paddle/fluid/framework/block_desc.h" +#include "paddle/fluid/framework/data_layout.h" namespace paddle { namespace framework { @@ -109,6 +110,9 @@ struct SimpleOpTypeSetTeller : public Teller { "transpose", "flatten2", "flatten", + "gather", + + "nearest_interp", }; }; @@ -129,13 +133,7 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, std::vector paddings = BOOST_GET_CONST(std::vector, desc.GetAttr("paddings")); - std::string padding_algorithm = "EXPLICIT"; - if (desc.HasAttr("padding_algorithm")) - padding_algorithm = - BOOST_GET_CONST(std::string, desc.GetAttr("padding_algorithm")); - if (paddings.size() > 2 || - (padding_algorithm == "SAME" && op_type != "pool2d")) - return false; + if (paddings.size() > 2) return false; } if (op_type == "matmul") { auto* block = desc.Block(); @@ -153,6 +151,7 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, } } if (op_type == "group_norm") { + if (!with_dynamic_shape) return false; bool has_attrs = (desc.HasAttr("epsilon") && desc.HasAttr("groups")); if (has_attrs == false) return false; @@ -164,7 +163,11 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, return false; } else { int axis = BOOST_GET_CONST(int, desc.GetAttr("axis")); - if (axis <= 0) return false; + if (with_dynamic_shape) { + if (axis < 0) return false; + } else { + if (axis <= 0) return false; + } } } if (op_type == "transpose2" || op_type == "transpose") { @@ -187,6 +190,39 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, if (axis != 1) return false; } } + + if (op_type == "gather") { + // current not support axis from input, use default 0 + if (!with_dynamic_shape || desc.Input("Axis").size() > 0) return false; + } + if (op_type == "fc" || op_type == "mul") { + const int x_num_col_dims = + desc.HasAttr("x_num_col_dims") + ? BOOST_GET_CONST(int, desc.GetAttr("x_num_col_dims")) + : (desc.HasAttr("in_num_col_dims") + ? BOOST_GET_CONST(int, desc.GetAttr("in_num_col_dims")) + : 1); + if (x_num_col_dims != 1 && x_num_col_dims != 2) { + return false; + } + } + if (op_type == "nearest_interp") { + std::vector attrs{"data_layout", "interp_method", + "align_corners", "scale", + "out_h", "out_w"}; + for (auto const attr : attrs) { + if (!desc.HasAttr(attr)) return false; + } + auto data_layout = framework::StringToDataLayout( + BOOST_GET_CONST(std::string, desc.GetAttr("data_layout"))); + if (data_layout != framework::DataLayout::kNCHW && + data_layout != framework::DataLayout::kNHWC) + return false; + auto interp_method = + BOOST_GET_CONST(std::string, desc.GetAttr("interp_method")); + if (interp_method != "nearest") return false; + } + if ((*teller)(op_type, desc, use_no_calib_int8)) return true; } return false; diff --git a/paddle/fluid/inference/tensorrt/plugin/CMakeLists.txt b/paddle/fluid/inference/tensorrt/plugin/CMakeLists.txt index e37beb3b8e5c3..7ee16a598d2d0 100644 --- a/paddle/fluid/inference/tensorrt/plugin/CMakeLists.txt +++ b/paddle/fluid/inference/tensorrt/plugin/CMakeLists.txt @@ -6,3 +6,6 @@ nv_library(tensorrt_plugin qkv_to_context_plugin.cu skip_layernorm_op_plugin.cu slice_op_plugin.cu hard_swish_op_plugin.cu stack_op_plugin.cu special_slice_plugin.cu DEPS enforce tensorrt_engine prelu tensor bert_encoder_functor) + +nv_test(test_split_plugin SRCS test_split_plugin.cc DEPS + paddle_framework ${GLOB_OPERATOR_DEPS} tensorrt_plugin) diff --git a/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.cu index 457d9dd873754..cc17f8aa24817 100644 --- a/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.cu @@ -152,9 +152,14 @@ int ElementWisePlugin::enqueue(int batch_size, const void *const *inputs, int ElementwisePluginDynamic::initialize() { return 0; } -size_t ElementwisePluginDynamic::getSerializationSize() const { return 0; } +size_t ElementwisePluginDynamic::getSerializationSize() const { + return SerializedSize(type_.c_str()) + SerializedSize(axis_); +} -void ElementwisePluginDynamic::serialize(void *buffer) const {} +void ElementwisePluginDynamic::serialize(void *buffer) const { + SerializeValue(&buffer, type_.c_str()); + SerializeValue(&buffer, axis_); +} nvinfer1::DimsExprs ElementwisePluginDynamic::getOutputDimensions( int output_index, const nvinfer1::DimsExprs *inputs, int nb_inputs, diff --git a/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.h b/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.h index e37511868d88f..49212aae9aa90 100644 --- a/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.h +++ b/paddle/fluid/inference/tensorrt/plugin/elementwise_op_plugin.h @@ -92,7 +92,12 @@ class ElementwisePluginDynamic : public DynamicPluginTensorRT { public: explicit ElementwisePluginDynamic(const std::string& type, int axis) : type_(type), axis_(axis) {} - ElementwisePluginDynamic(void const* serialData, size_t serialLength) {} + ElementwisePluginDynamic(void const* serialData, size_t serialLength) { + const char* elementwise_type; + DeserializeValue(&serialData, &serialLength, &elementwise_type); + type_ = std::string(elementwise_type); + DeserializeValue(&serialData, &serialLength, &axis_); + } nvinfer1::IPluginV2DynamicExt* clone() const override { return new ElementwisePluginDynamic(type_, axis_); } @@ -138,6 +143,46 @@ class ElementwisePluginDynamic : public DynamicPluginTensorRT { std::string type_; int axis_; }; + +class ElementwisePluginV2Creator : public nvinfer1::IPluginCreator { + public: + ElementwisePluginV2Creator() {} + const char* getPluginName() const override { return "elementwise_plugin"; } + + const char* getPluginVersion() const override { return "1"; } + + const nvinfer1::PluginFieldCollection* getFieldNames() override { + return &field_collection_; + } + + nvinfer1::IPluginV2* createPlugin( + const char* name, const nvinfer1::PluginFieldCollection* fc) override { + return nullptr; + } + + nvinfer1::IPluginV2* deserializePlugin(const char* name, + const void* serial_data, + size_t serial_length) override { + auto plugin = new ElementwisePluginDynamic(serial_data, serial_length); + return plugin; + } + + void setPluginNamespace(const char* lib_namespace) override { + plugin_namespace_ = lib_namespace; + } + + const char* getPluginNamespace() const override { + return plugin_namespace_.c_str(); + } + + private: + std::string plugin_namespace_; + std::string plugin_name_; + nvinfer1::PluginFieldCollection field_collection_{0, nullptr}; + std::vector plugin_attributes_; +}; + +REGISTER_TRT_PLUGIN_V2(ElementwisePluginV2Creator); #endif } // namespace plugin diff --git a/paddle/fluid/inference/tensorrt/plugin/emb_eltwise_layernorm_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/emb_eltwise_layernorm_plugin.cu index 238daa4a886a4..6d3872aaeb8a7 100644 --- a/paddle/fluid/inference/tensorrt/plugin/emb_eltwise_layernorm_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/emb_eltwise_layernorm_plugin.cu @@ -200,12 +200,10 @@ nvinfer1::DimsExprs EmbEltwiseLayernormPluginDynamic::getOutputDimensions( "but it's (%d)", output_index)); nvinfer1::DimsExprs ret; - ret.nbDims = 5; + ret.nbDims = 3; ret.d[0] = inputs[0].d[0]; ret.d[1] = inputs[0].d[1]; ret.d[2] = expr_builder.constant(hidden_size_); - ret.d[3] = expr_builder.constant(1); - ret.d[4] = expr_builder.constant(1); return ret; } diff --git a/paddle/fluid/inference/tensorrt/plugin/pool_op_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/pool_op_plugin.cu index 1fa5b3228e115..154f61a2b7cd3 100644 --- a/paddle/fluid/inference/tensorrt/plugin/pool_op_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/pool_op_plugin.cu @@ -59,14 +59,14 @@ int PoolPlugin::enqueue(int batchSize, const void *const *inputs, paddle::operators::math::MaxPool, float> pool2d_forward; pool2d_forward(idata, input_shape, output_shape, ksize_, strides_, - paddings_, pool_process, true, adaptive_, odatas[0], stream); + paddings_, true, adaptive_, odatas[0], stream, pool_process); } else if (pool_type_ == PoolType::avg) { paddle::operators::math::AvgPool pool_process; paddle::operators::math::Pool2dDirectCUDAFunctor< paddle::operators::math::AvgPool, float> pool2d_forward; pool2d_forward(idata, input_shape, output_shape, ksize_, strides_, - paddings_, pool_process, true, adaptive_, odatas[0], stream); + paddings_, true, adaptive_, odatas[0], stream, pool_process); } return cudaGetLastError() != cudaSuccess; @@ -224,14 +224,14 @@ int PoolPluginDynamic::enqueue(const nvinfer1::PluginTensorDesc *input_desc, paddle::operators::math::MaxPool, float> pool2d_forward; pool2d_forward(input, input_shape, output_shape, ksize, strides_, paddings, - pool_process, true, adaptive_, output, stream); + true, adaptive_, output, stream, pool_process); } else if (pool_type_ == "avg") { paddle::operators::math::AvgPool pool_process; paddle::operators::math::Pool2dDirectCUDAFunctor< paddle::operators::math::AvgPool, float> pool2d_forward; pool2d_forward(input, input_shape, output_shape, ksize, strides_, paddings, - pool_process, true, adaptive_, output, stream); + true, adaptive_, output, stream, pool_process); } return cudaGetLastError() != cudaSuccess; diff --git a/paddle/fluid/inference/tensorrt/plugin/qkv_to_context_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/qkv_to_context_plugin.cu index 1e7c83f4c60fb..a5fc9e73c5f27 100644 --- a/paddle/fluid/inference/tensorrt/plugin/qkv_to_context_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/qkv_to_context_plugin.cu @@ -169,12 +169,10 @@ nvinfer1::DimsExprs QkvToContextPluginDynamic::getOutputDimensions( "it has (%d) inputs", nb_inputs)); nvinfer1::DimsExprs ret; - ret.nbDims = 5; + ret.nbDims = 3; ret.d[0] = inputs[0].d[0]; ret.d[1] = inputs[0].d[1]; ret.d[2] = expr_builder.constant(head_size_ * head_number_); - ret.d[3] = expr_builder.constant(1); - ret.d[4] = expr_builder.constant(1); return ret; } diff --git a/paddle/fluid/inference/tensorrt/plugin/skip_layernorm_op_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/skip_layernorm_op_plugin.cu index 3b9eea22199d7..7be9e3a740ab1 100644 --- a/paddle/fluid/inference/tensorrt/plugin/skip_layernorm_op_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/skip_layernorm_op_plugin.cu @@ -54,11 +54,6 @@ void SkipLayerNormPluginDynamic::terminate() { nvinfer1::DimsExprs SkipLayerNormPluginDynamic::getOutputDimensions( int output_index, const nvinfer1::DimsExprs *inputs, int nb_inputs, nvinfer1::IExprBuilder &expr_builder) { - PADDLE_ENFORCE_EQ( - inputs[0].nbDims, 5, - platform::errors::InvalidArgument( - "The Input dim of the SkipLayernorm should be 5, but it's (%d) now.", - inputs[0].nbDims)); return inputs[0]; } diff --git a/paddle/fluid/inference/tensorrt/plugin/special_slice_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/special_slice_plugin.cu index ed0a530439f0a..fdb14f9ceaf29 100644 --- a/paddle/fluid/inference/tensorrt/plugin/special_slice_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/special_slice_plugin.cu @@ -54,9 +54,16 @@ nvinfer1::DimsExprs SpecialSlicePluginDynamic::getOutputDimensions( int output_index, const nvinfer1::DimsExprs* inputs, int nb_inputs, nvinfer1::IExprBuilder& expr_builder) { nvinfer1::DimsExprs output(inputs[0]); + output.nbDims++; + for (int i = output.nbDims - 1; i > 1; i--) { + output.d[i] = inputs[0].d[i - 1]; + } auto one = expr_builder.constant(1); + output.d[1] = one; output.d[0] = expr_builder.operation(nvinfer1::DimensionOperation::kSUB, *inputs[1].d[0], *one); + // remove padding 1 + output.nbDims -= 2; return output; } diff --git a/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.cu b/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.cu index 256aa28206ad1..1b5c39f8fff85 100644 --- a/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.cu +++ b/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.cu @@ -22,11 +22,6 @@ namespace inference { namespace tensorrt { namespace plugin { -SplitPlugin* CreateSplitPluginDeserialize(const void* buffer, size_t length) { - return new SplitPlugin(buffer, length); -} -REGISTER_TRT_PLUGIN("split_plugin", CreateSplitPluginDeserialize); - template __device__ int upper_bound(T const* vals, int n, T const& key) { int i = 0; diff --git a/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.h b/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.h index 5c47ec3a990f5..e43b57357fb64 100644 --- a/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.h +++ b/paddle/fluid/inference/tensorrt/plugin/split_op_plugin.h @@ -25,7 +25,7 @@ namespace inference { namespace tensorrt { namespace plugin { -class SplitPlugin : public PluginTensorRT { +class SplitPlugin : public PluginTensorRTV2Ext { public: SplitPlugin() {} SplitPlugin(int axis, std::vector const& output_lengths, bool with_fp16) @@ -39,13 +39,20 @@ class SplitPlugin : public PluginTensorRT { DeserializeValue(&serial_data, &serial_length, &output_length_); } - SplitPlugin* clone() const override { - auto* ptr = new SplitPlugin(axis_, output_length_, with_fp16_); + nvinfer1::IPluginV2Ext* clone() const override { + SplitPlugin* ptr = new SplitPlugin(axis_, output_length_, with_fp16_); + ptr->setPluginNamespace(this->getPluginNamespace()); ptr->shareData(this); return ptr; } - const char* getPluginType() const override { return "split_plugin"; } + nvinfer1::DataType getOutputDataType(int index, + const nvinfer1::DataType* input_types, + int nb_inputs) const override { + return input_types[0]; + } + + const char* getPluginType() const override { return "split_plugin_v2ext"; } int getNbOutputs() const override { return output_length_.size(); } nvinfer1::Dims getOutputDimensions(int index, const nvinfer1::Dims* input_dims, @@ -53,17 +60,18 @@ class SplitPlugin : public PluginTensorRT { int initialize() override; void terminate() override; - int enqueue(int batchSize, const void* const* inputs, void** outputs, + int enqueue(int batch_size, const void* const* inputs, void** outputs, void* workspace, cudaStream_t stream) override; + void destroy() override { delete this; } + protected: - size_t getSerializationSize() override { - return SerializedSize(getPluginType()) + SerializedSize(axis_) + - SerializedSize(output_length_) + getBaseSerializationSize(); + size_t getSerializationSize() const override { + return SerializedSize(axis_) + SerializedSize(output_length_) + + getBaseSerializationSize(); } - void serialize(void* buffer) override { - SerializeValue(&buffer, getPluginType()); + void serialize(void* buffer) const override { serializeBase(buffer); SerializeValue(&buffer, axis_); SerializeValue(&buffer, output_length_); @@ -83,6 +91,47 @@ class SplitPlugin : public PluginTensorRT { void shareData(const SplitPlugin* another); }; +class SplitPluginCreator : public nvinfer1::IPluginCreator { + public: + SplitPluginCreator() {} + const char* getPluginName() const override { return "split_plugin_v2ext"; } + + const char* getPluginVersion() const override { return "1"; } + + const nvinfer1::PluginFieldCollection* getFieldNames() override { + return &field_collection_; + } + + nvinfer1::IPluginV2* createPlugin( + const char* name, const nvinfer1::PluginFieldCollection* fc) override { + // not implemented + return nullptr; + } + + nvinfer1::IPluginV2* deserializePlugin(const char* name, + const void* serial_data, + size_t serial_length) override { + auto plugin = new SplitPlugin(serial_data, serial_length); + return plugin; + } + + void setPluginNamespace(const char* lib_namespace) override { + plugin_namespace_ = lib_namespace; + } + + const char* getPluginNamespace() const override { + return plugin_namespace_.c_str(); + } + + private: + std::string plugin_namespace_; + std::string plugin_name_; + nvinfer1::PluginFieldCollection field_collection_{0, nullptr}; + std::vector plugin_attributes_; +}; + +REGISTER_TRT_PLUGIN_V2(SplitPluginCreator); + #if IS_TRT_VERSION_GE(6000) class SplitPluginDynamic : public DynamicPluginTensorRT { public: diff --git a/paddle/fluid/inference/tensorrt/plugin/test_split_plugin.cc b/paddle/fluid/inference/tensorrt/plugin/test_split_plugin.cc new file mode 100644 index 0000000000000..6636513a555f9 --- /dev/null +++ b/paddle/fluid/inference/tensorrt/plugin/test_split_plugin.cc @@ -0,0 +1,58 @@ +/* Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + +#include +#include "paddle/fluid/inference/tensorrt/plugin/split_op_plugin.h" + +namespace paddle { +namespace inference { +namespace tensorrt { +namespace plugin { + +TEST(split_op_plugin, test_plugin) { + int axis = 1; + std::vector output_lengths{1, 1}; + bool with_fp16 = false; + std::vector input_types{nvinfer1::DataType::kFLOAT}; + std::vector input_dims; + + SplitPlugin sp_plugin(axis, output_lengths, with_fp16); + nvinfer1::Dims in_dims; + in_dims.nbDims = 4; + input_dims.push_back(in_dims); + sp_plugin.configurePlugin(input_dims.data(), 1, nullptr, 2, + input_types.data(), nullptr, nullptr, nullptr, + nvinfer1::PluginFormat::kNCHW, 4); + sp_plugin.initialize(); + sp_plugin.getPluginType(); + sp_plugin.canBroadcastInputAcrossBatch(0); + sp_plugin.getNbOutputs(); + auto clone_plugin = sp_plugin.clone(); + clone_plugin->setPluginNamespace("test"); + clone_plugin->destroy(); + sp_plugin.getOutputDataType(0, input_types.data(), 1); + sp_plugin.terminate(); +} + +TEST(split_op_plugin, test_plugin_creater) { + SplitPluginCreator creator; + creator.getFieldNames(); + creator.createPlugin("test", nullptr); + creator.setPluginNamespace("test"); +} + +} // namespace plugin +} // namespace tensorrt +} // namespace inference +} // namespace paddle diff --git a/paddle/fluid/inference/tensorrt/plugin/trt_plugin.cc b/paddle/fluid/inference/tensorrt/plugin/trt_plugin.cc index fd721b161450d..55bc786746bea 100644 --- a/paddle/fluid/inference/tensorrt/plugin/trt_plugin.cc +++ b/paddle/fluid/inference/tensorrt/plugin/trt_plugin.cc @@ -19,27 +19,50 @@ namespace inference { namespace tensorrt { namespace plugin { +inline void Seria(void*& buffer, // NOLINT + const std::vector& input_dims, + size_t max_batch_size, nvinfer1::DataType data_type, + nvinfer1::PluginFormat data_format, bool with_fp16) { + SerializeValue(&buffer, input_dims); + SerializeValue(&buffer, max_batch_size); + SerializeValue(&buffer, data_type); + SerializeValue(&buffer, data_format); + SerializeValue(&buffer, with_fp16); +} + +inline void Deseria(void const*& serial_data, size_t& serial_length, // NOLINT + std::vector* input_dims, + size_t* max_batch_size, nvinfer1::DataType* data_type, + nvinfer1::PluginFormat* data_format, bool* with_fp16) { + DeserializeValue(&serial_data, &serial_length, input_dims); + DeserializeValue(&serial_data, &serial_length, max_batch_size); + DeserializeValue(&serial_data, &serial_length, data_type); + DeserializeValue(&serial_data, &serial_length, data_format); + DeserializeValue(&serial_data, &serial_length, with_fp16); +} + +inline size_t SeriaSize(const std::vector& input_dims, + size_t max_batch_size, nvinfer1::DataType data_type, + nvinfer1::PluginFormat data_format, bool with_fp16) { + return (SerializedSize(input_dims) + SerializedSize(max_batch_size) + + SerializedSize(data_type) + SerializedSize(data_format) + + SerializedSize(with_fp16)); +} + void PluginTensorRT::serializeBase(void*& buffer) { - SerializeValue(&buffer, input_dims_); - SerializeValue(&buffer, max_batch_size_); - SerializeValue(&buffer, data_type_); - SerializeValue(&buffer, data_format_); - SerializeValue(&buffer, with_fp16_); + Seria(buffer, input_dims_, max_batch_size_, data_type_, data_format_, + with_fp16_); } void PluginTensorRT::deserializeBase(void const*& serial_data, size_t& serial_length) { - DeserializeValue(&serial_data, &serial_length, &input_dims_); - DeserializeValue(&serial_data, &serial_length, &max_batch_size_); - DeserializeValue(&serial_data, &serial_length, &data_type_); - DeserializeValue(&serial_data, &serial_length, &data_format_); - DeserializeValue(&serial_data, &serial_length, &with_fp16_); + Deseria(serial_data, serial_length, &input_dims_, &max_batch_size_, + &data_type_, &data_format_, &with_fp16_); } size_t PluginTensorRT::getBaseSerializationSize() { - return (SerializedSize(input_dims_) + SerializedSize(max_batch_size_) + - SerializedSize(data_type_) + SerializedSize(data_format_) + - SerializedSize(with_fp16_)); + return SeriaSize(input_dims_, max_batch_size_, data_type_, data_format_, + with_fp16_); } bool PluginTensorRT::supportsFormat(nvinfer1::DataType type, @@ -58,6 +81,35 @@ void PluginTensorRT::configureWithFormat( max_batch_size_ = max_batch_size; } +void PluginTensorRTV2Ext::serializeBase(void*& buffer) const { + Seria(buffer, input_dims_, max_batch_size_, data_type_, data_format_, + with_fp16_); +} + +void PluginTensorRTV2Ext::deserializeBase(void const*& serial_data, + size_t& serial_length) { + Deseria(serial_data, serial_length, &input_dims_, &max_batch_size_, + &data_type_, &data_format_, &with_fp16_); +} + +size_t PluginTensorRTV2Ext::getBaseSerializationSize() const { + return SeriaSize(input_dims_, max_batch_size_, data_type_, data_format_, + with_fp16_); +} + +void PluginTensorRTV2Ext::configurePlugin( + const nvinfer1::Dims* input_dims, int32_t nb_inputs, + const nvinfer1::Dims* output_dims, int32_t nb_outputs, + const nvinfer1::DataType* input_types, + const nvinfer1::DataType* output_types, const bool* input_is_broadcast, + const bool* output_is_broadcast, nvinfer1::PluginFormat float_format, + int32_t max_batch_size) { + input_dims_.assign(input_dims, input_dims + nb_inputs); + max_batch_size_ = max_batch_size; + data_format_ = float_format; + data_type_ = input_types[0]; +} + } // namespace plugin } // namespace tensorrt } // namespace inference diff --git a/paddle/fluid/inference/tensorrt/plugin/trt_plugin.h b/paddle/fluid/inference/tensorrt/plugin/trt_plugin.h index b3a3abe5d01fc..ce3133ae99e94 100644 --- a/paddle/fluid/inference/tensorrt/plugin/trt_plugin.h +++ b/paddle/fluid/inference/tensorrt/plugin/trt_plugin.h @@ -44,6 +44,7 @@ typedef std::function typedef std::function PluginConstructFunc; +// Deprecated. Do not inherit this class, please refer to PluginTensorRTV2Ext class PluginTensorRT : public nvinfer1::IPluginExt { public: PluginTensorRT() : with_fp16_(false) {} @@ -119,6 +120,114 @@ class PluginTensorRT : public nvinfer1::IPluginExt { bool with_fp16_; }; +// TensorRT introduced IPluginV2Ext after 5.1, Paddle no longer supports +// versions before 5.1 +class PluginTensorRTV2Ext : public nvinfer1::IPluginV2Ext { + public: + PluginTensorRTV2Ext() : with_fp16_(false) {} + PluginTensorRTV2Ext(const void* serialized_data, size_t length) {} + + nvinfer1::Dims const& getInputDims(int index) const { + return input_dims_.at(index); + } + size_t getMaxBatchSize() const { return max_batch_size_; } + nvinfer1::DataType getDataType() const { return data_type_; } + nvinfer1::PluginFormat getDataFormat() const { return data_format_; } + + // The Func in IPluginV2Ext + virtual nvinfer1::DataType getOutputDataType( + int index, const nvinfer1::DataType* input_types, + int nb_inputs) const = 0; + + virtual bool isOutputBroadcastAcrossBatch(int32_t output_index, + const bool* input_is_broadcasted, + int32_t nb_inputs) const { + return false; + } + + virtual bool canBroadcastInputAcrossBatch(int32_t input_index) const { + return false; + } + + void configurePlugin(const nvinfer1::Dims* input_dims, int32_t nb_inputs, + const nvinfer1::Dims* output_dims, int32_t nb_outputs, + const nvinfer1::DataType* input_types, + const nvinfer1::DataType* output_types, + const bool* input_is_broadcast, + const bool* output_is_broadcast, + nvinfer1::PluginFormat float_format, + int32_t max_batch_size) override; + + virtual IPluginV2Ext* clone() const = 0; + + void attachToContext(cudnnContext*, cublasContext*, + nvinfer1::IGpuAllocator*) override {} + + void detachFromContext() override {} + + // The Func in IPluginV2 + virtual const char* getPluginType() const = 0; + const char* getPluginVersion() const override { return "1"; } + virtual int32_t getNbOutputs() const { return 1; } + virtual nvinfer1::Dims getOutputDimensions(int32_t index, + const nvinfer1::Dims* inputs, + int32_t nb_input) = 0; + // Check format support. The default is FLOAT32 and NCHW. + bool supportsFormat(nvinfer1::DataType type, + nvinfer1::PluginFormat format) const override { + return ((type == nvinfer1::DataType::kFLOAT) && + (format == nvinfer1::PluginFormat::kNCHW)); + } + // Initialize the layer for execution. + // This is called when the engine is created. + int initialize() override { return 0; } + + // Shutdown the layer. This is called when the engine is destroyed + void terminate() override {} + + // Find the workspace size required by the layer + size_t getWorkspaceSize(int) const override { return 0; } + + // Execute the layer + virtual int enqueue(int batch_size, const void* const* inputs, void** outputs, + void* workspace, cudaStream_t stream) = 0; + + // Find the size of the serialization buffer required + virtual size_t getSerializationSize() const = 0; + + // Serialize the layer config to buffer. + // TensorRT will call this func to serialize the configuration of TensorRT + // engine. It should not be called by users. + virtual void serialize(void* buffer) const = 0; + + virtual void destroy() = 0; + + void setPluginNamespace(const char* plugin_namespace) override { + name_space_ = plugin_namespace; + } + + const char* getPluginNamespace() const override { + return name_space_.c_str(); + } + + protected: + void deserializeBase(void const*& serial_data, // NOLINT + size_t& serial_length); // NOLINT + size_t getBaseSerializationSize() const; + void serializeBase(void*& buffer) const; // NOLINT + + protected: + std::vector input_dims_; + size_t max_batch_size_; + nvinfer1::DataType data_type_; + nvinfer1::PluginFormat data_format_; + std::vector inputs_; + bool with_fp16_; + + private: + std::string name_space_; +}; + #if IS_TRT_VERSION_GE(6000) class DynamicPluginTensorRT : public nvinfer1::IPluginV2DynamicExt { public: @@ -184,6 +293,7 @@ class DynamicPluginTensorRT : public nvinfer1::IPluginV2DynamicExt { std::string name_space_; std::string plugin_base_; }; +#endif template class TrtPluginRegistrarV2 { @@ -203,8 +313,6 @@ class TrtPluginRegistrarV2 { static paddle::inference::tensorrt::plugin::TrtPluginRegistrarV2 \ plugin_registrar_##name {} -#endif - } // namespace plugin } // namespace tensorrt } // namespace inference diff --git a/paddle/fluid/inference/tests/api/CMakeLists.txt b/paddle/fluid/inference/tests/api/CMakeLists.txt index bb8faf30fdd87..92f9c20a369d7 100644 --- a/paddle/fluid/inference/tests/api/CMakeLists.txt +++ b/paddle/fluid/inference/tests/api/CMakeLists.txt @@ -599,13 +599,12 @@ endif() set(LITE_MODEL_INSTALL_DIR "${INFERENCE_DEMO_INSTALL_DIR}/lite") download_data(${LITE_MODEL_INSTALL_DIR} "mul_model_fp32.tgz") -#TODO(wilber): tmp disable ut. -#inference_analysis_test(lite_mul_model_test SRCS lite_mul_model_test.cc -# EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} -# ARGS --infer_model=${LITE_MODEL_INSTALL_DIR}) -#inference_analysis_test(lite_resnet50_test SRCS lite_resnet50_test.cc -# EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} -# ARGS --infer_model=${RESNET50_MODEL_DIR}) +inference_analysis_test(lite_mul_model_test SRCS lite_mul_model_test.cc + EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} + ARGS --infer_model=${LITE_MODEL_INSTALL_DIR}) +inference_analysis_test(lite_resnet50_test SRCS lite_resnet50_test.cc + EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} + ARGS --infer_model=${RESNET50_MODEL_DIR}) inference_analysis_test(test_analyzer_capi SRCS analyzer_capi_tester.cc EXTRA_DEPS ${INFERENCE_EXTRA_DEPS} paddle_inference_c @@ -657,8 +656,7 @@ if(WITH_MKLDNN) set_tests_properties(test_analyzer_int8_mobilenetv1 PROPERTIES TIMEOUT 120) endif() -#TODO(wilber): tmp disable ut -#set_tests_properties(lite_resnet50_test PROPERTIES TIMEOUT 120) +set_tests_properties(lite_resnet50_test PROPERTIES TIMEOUT 120) set_tests_properties(test_analyzer_mobilenet_transpose PROPERTIES TIMEOUT 120) set_tests_properties(test_analyzer_resnet50 PROPERTIES TIMEOUT 120) set_tests_properties(test_analyzer_ner PROPERTIES TIMEOUT 120) diff --git a/paddle/fluid/inference/tests/api/full_ILSVRC2012_val_preprocess.py b/paddle/fluid/inference/tests/api/full_ILSVRC2012_val_preprocess.py index f25ce8be9eeb7..e911c94208711 100644 --- a/paddle/fluid/inference/tests/api/full_ILSVRC2012_val_preprocess.py +++ b/paddle/fluid/inference/tests/api/full_ILSVRC2012_val_preprocess.py @@ -34,10 +34,8 @@ SIZE_INT64 = 8 FULL_SIZE_BYTES = 30106000008 FULL_IMAGES = 50000 -TARGET_HASH = '0be07c2c23296b97dad83c626682c66a' FOLDER_NAME = "ILSVRC2012/" VALLIST_TAR_NAME = "ILSVRC2012/val_list.txt" -CHUNK_SIZE = 8192 img_mean = np.array([0.485, 0.456, 0.406]).reshape((3, 1, 1)) img_std = np.array([0.229, 0.224, 0.225]).reshape((3, 1, 1)) @@ -108,28 +106,6 @@ def print_processbar(done_percentage): sys.stdout.flush() -def check_integrity(filename, target_hash): - print('\nThe binary file exists. Checking file integrity...\n') - md = hashlib.md5() - count = 0 - onepart = FULL_SIZE_BYTES // CHUNK_SIZE // 100 - with open(filename, 'rb') as ifs: - while True: - buf = ifs.read(CHUNK_SIZE) - if count % onepart == 0: - done = count // onepart - print_processbar(done) - count = count + 1 - if not buf: - break - md.update(buf) - hash1 = md.hexdigest() - if hash1 == target_hash: - return True - else: - return False - - def convert_Imagenet_tar2bin(tar_file, output_file): print('Converting 50000 images to binary file ...\n') tar = tarfile.open(name=tar_file, mode='r:gz') @@ -188,8 +164,7 @@ def run_convert(): try_limit = 3 while not (os.path.exists(output_file) and - os.path.getsize(output_file) == FULL_SIZE_BYTES and - check_integrity(output_file, TARGET_HASH)): + os.path.getsize(output_file) == FULL_SIZE_BYTES): if os.path.exists(output_file): sys.stderr.write( "\n\nThe existing binary file is broken. Start to generate new one...\n\n". diff --git a/paddle/fluid/inference/tests/api/lite_resnet50_test.cc b/paddle/fluid/inference/tests/api/lite_resnet50_test.cc index 99c2c2f6f3d9c..59bbaa2b78fb0 100644 --- a/paddle/fluid/inference/tests/api/lite_resnet50_test.cc +++ b/paddle/fluid/inference/tests/api/lite_resnet50_test.cc @@ -81,6 +81,7 @@ TEST(Predictor, use_gpu) { config.EnableLiteEngine(PrecisionType::kFloat32); auto predictor = CreatePredictor(config); + const int batch = 1; const int channel = 3; const int height = 318; diff --git a/paddle/fluid/inference/tests/api/trt_dynamic_shape_test.cc b/paddle/fluid/inference/tests/api/trt_dynamic_shape_test.cc index 552aefac9b6da..55ee2082e6959 100644 --- a/paddle/fluid/inference/tests/api/trt_dynamic_shape_test.cc +++ b/paddle/fluid/inference/tests/api/trt_dynamic_shape_test.cc @@ -21,17 +21,32 @@ limitations under the License. */ namespace paddle { namespace inference { -void TestDynamic(bool with_dynamic = true) { +void TestDynamic(bool with_dynamic = true, bool delete_cache = true, + bool delete_conv_bn = false) { std::string model_dir = FLAGS_infer_model + "/conv_bn_swish_split_gelu/conv_bn_swish_split_gelu"; + + std::string opt_cache_dir = model_dir + "/my_cache"; + if (delete_cache) { + delete_cache_files(opt_cache_dir); + } + AnalysisConfig config; config.EnableUseGpu(100, 0); - config.SetModel(model_dir + "/model", model_dir + "/params"); + std::string buffer_prog, buffer_param; + ReadBinaryFile(model_dir + "/model", &buffer_prog); + ReadBinaryFile(model_dir + "/params", &buffer_param); + config.SetModelBuffer(&buffer_prog[0], buffer_prog.size(), &buffer_param[0], + buffer_param.size()); + config.SetOptimCacheDir(opt_cache_dir); + config.SwitchUseFeedFetchOps(false); // Set the input's min, max, opt shape - config.EnableTensorRtEngine(1 << 30, 1, 1, - AnalysisConfig::Precision::kFloat32, false, true); + AnalysisConfig::Precision::kFloat32, true, true); + if (delete_conv_bn) { + config.pass_builder()->DeletePass("conv_bn_fuse_pass"); + } if (with_dynamic) { std::map> min_input_shape = { {"image", {1, 1, 3, 3}}}; @@ -130,6 +145,12 @@ void TestDynamic2() { TEST(AnalysisPredictor, trt_dynamic) { TestDynamic(true); } TEST(AnalysisPredictor, trt_static) { TestDynamic(false); } +TEST(AnalysisPredictor, trt_memory_serialize) { + // serailize + TestDynamic(false, true, true); + // deserailize + TestDynamic(false, false, true); +} TEST(AnalysisPredictor, trt_dynamic2) { TestDynamic2(); } } // namespace inference diff --git a/paddle/fluid/inference/tests/api/trt_test_helper.h b/paddle/fluid/inference/tests/api/trt_test_helper.h index 1abde73358121..aaa285b2fc2c9 100644 --- a/paddle/fluid/inference/tests/api/trt_test_helper.h +++ b/paddle/fluid/inference/tests/api/trt_test_helper.h @@ -148,6 +148,7 @@ void delete_cache_files(std::string path) { remove(file_rm.c_str()); } } + remove(path.c_str()); } } // namespace inference diff --git a/paddle/fluid/memory/memcpy.cc b/paddle/fluid/memory/memcpy.cc index 6a1d44f6cfe1e..7f871fab5a147 100644 --- a/paddle/fluid/memory/memcpy.cc +++ b/paddle/fluid/memory/memcpy.cc @@ -239,7 +239,7 @@ void Copy( platform::SetDeviceId(src_place.device); VLOG(4) << "memory::Copy " << num << " Bytes from " << src_place << " to " - << dst_place << " by thream(" << stream << ")"; + << dst_place << " by stream(" << stream << ")"; if (stream) { platform::RecordEvent record_event("GpuMemcpyAsync:GPU->CPU"); #ifdef PADDLE_WITH_HIP diff --git a/paddle/fluid/operators/CMakeLists.txt b/paddle/fluid/operators/CMakeLists.txt index 598e417526f97..467a5ff9063a6 100644 --- a/paddle/fluid/operators/CMakeLists.txt +++ b/paddle/fluid/operators/CMakeLists.txt @@ -73,9 +73,11 @@ register_operators(EXCLUDES py_func_op warpctc_op dgc_op lstm_op run_program_op op_library(run_program_op SRCS run_program_op.cc run_program_op.cu.cc DEPS executor_cache ${OP_HEADER_DEPS}) -if (WITH_GPU) +if (WITH_GPU OR WITH_ROCM) + if(WITH_ROCM) + op_library(warpctc_op DEPS dynload_warpctc sequence_padding sequence_scale SRCS warpctc_op.cc warpctc_op.cu.cc) # warpctc_op needs cudnn 7 above - if (${CUDNN_MAJOR_VERSION} VERSION_LESS 7) + elseif(${CUDNN_MAJOR_VERSION} VERSION_LESS 7) op_library(warpctc_op DEPS dynload_warpctc sequence_padding sequence_scale SRCS warpctc_op.cc warpctc_op.cu.cc) else() op_library(warpctc_op DEPS dynload_warpctc sequence_padding sequence_scale) @@ -108,7 +110,7 @@ set(COMMON_OP_DEPS ${COMMON_OP_DEPS} sequence_padding sequence_scale cos_sim_fun set(COMMON_OP_DEPS ${COMMON_OP_DEPS} sequence2batch lstm_compute matrix_bit_code gru_compute activation_functions beam_search fc matrix_inverse) set(COMMON_OP_DEPS ${COMMON_OP_DEPS} box_wrapper boost ps_gpu_wrapper) set(COMMON_OP_DEPS ${COMMON_OP_DEPS} common_infer_shape_functions) -if (WITH_GPU) +if (WITH_GPU OR WITH_ROCM) set(COMMON_OP_DEPS ${COMMON_OP_DEPS} depthwise_conv prelu bert_encoder_functor) endif() set(COMMON_OP_DEPS ${COMMON_OP_DEPS} device_memory_aligment) @@ -139,9 +141,12 @@ cc_test(beam_search_decode_op_test SRCS beam_search_decode_op_test.cc DEPS lod_t cc_test(strided_memcpy_test SRCS strided_memcpy_test.cc DEPS tensor memory) cc_test(save_load_op_test SRCS save_load_op_test.cc DEPS save_op load_op) cc_test(save_load_combine_op_test SRCS save_load_combine_op_test.cc DEPS save_combine_op load_combine_op) -nv_test(dropout_op_test SRCS dropout_op_test.cc DEPS dropout_op tensor generator) if (WITH_GPU) + nv_test(dropout_op_test SRCS dropout_op_test.cc DEPS dropout_op tensor generator) nv_test(test_leaky_relu_grad_grad_functor SRCS test_leaky_relu_grad_grad_functor.cc test_leaky_relu_grad_grad_functor.cu DEPS tensor device_context eigen3) +elseif(WITH_ROCM) + hip_test(dropout_op_test SRCS dropout_op_test.cc DEPS dropout_op tensor generator) + hip_test(test_leaky_relu_grad_grad_functor SRCS test_leaky_relu_grad_grad_functor.cc test_leaky_relu_grad_grad_functor.cu DEPS tensor device_context eigen3) else() cc_test(test_leaky_relu_grad_grad_functor SRCS test_leaky_relu_grad_grad_functor.cc DEPS tensor device_context eigen3) endif() diff --git a/paddle/fluid/operators/activation_cudnn.cu.cc b/paddle/fluid/operators/activation_cudnn.cu.cc index 7f8ecc1df0734..38499783eb492 100644 --- a/paddle/fluid/operators/activation_cudnn.cu.cc +++ b/paddle/fluid/operators/activation_cudnn.cu.cc @@ -14,7 +14,11 @@ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/activation_op.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_desc.h" +#else #include "paddle/fluid/platform/cudnn_desc.h" +#endif namespace paddle { namespace operators { diff --git a/paddle/fluid/operators/activation_cudnn_op.cu.cc b/paddle/fluid/operators/activation_cudnn_op.cu.cc index 26ad09cc265f1..b197d3511f96b 100644 --- a/paddle/fluid/operators/activation_cudnn_op.cu.cc +++ b/paddle/fluid/operators/activation_cudnn_op.cu.cc @@ -14,7 +14,11 @@ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/activation_op.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_desc.h" +#else #include "paddle/fluid/platform/cudnn_desc.h" +#endif namespace paddle { namespace platform { @@ -29,35 +33,71 @@ using platform::ActivationDescriptor; using platform::TensorDescriptor; using platform::CUDADeviceContext; +#ifdef PADDLE_WITH_HIP +#define GPUDNN_ACTIVATION_RELU miopenActivationRELU +#define GPUDNN_ACTIVATION_CLIPPED_RELU miopenActivationCLIPPEDRELU +#define GPUDNN_ACTIVATION_SIGMOID miopenActivationLOGISTIC +#define GPUDNN_ACTIVATION_TANH miopenActivationTANH +#else +#define GPUDNN_ACTIVATION_RELU CUDNN_ACTIVATION_RELU +#define GPUDNN_ACTIVATION_CLIPPED_RELU CUDNN_ACTIVATION_CLIPPED_RELU +#define GPUDNN_ACTIVATION_SIGMOID CUDNN_ACTIVATION_SIGMOID +#define GPUDNN_ACTIVATION_TANH CUDNN_ACTIVATION_TANH +#endif + template struct CudnnActivationFunctor { using ELEMENT_TYPE = T; +#ifdef PADDLE_WITH_HIP + CudnnActivationFunctor(const CUDADeviceContext& ctx, const T& c, + const miopenActivationMode_t& m) + : ctx_(ctx), coef_(c), mode_(m) {} +#else CudnnActivationFunctor(const CUDADeviceContext& ctx, const T& c, const cudnnActivationMode_t& m) : ctx_(ctx), coef_(c), mode_(m) {} +#endif void operator()(const Tensor& x, Tensor* out) { ActivationDescriptor act_desc; act_desc.set(mode_, coef_); TensorDescriptor x_desc, out_desc; x_desc.set(x); out_desc.set(GET_DATA_SAFELY(out, "Output", "Out", "CudnnActivation")); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenActivationForward( + ctx_.cudnn_handle(), act_desc.desc(), + platform::CudnnDataType::kOne(), x_desc.desc(), x.data(), + platform::CudnnDataType::kZero(), out_desc.desc(), + out->mutable_data(ctx_.GetPlace()))); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnActivationForward( ctx_.cudnn_handle(), act_desc.desc(), platform::CudnnDataType::kOne(), x_desc.desc(), x.data(), platform::CudnnDataType::kZero(), out_desc.desc(), out->mutable_data(ctx_.GetPlace()))); +#endif } const CUDADeviceContext& ctx_; const T coef_; +#ifdef PADDLE_WITH_HIP + const miopenActivationMode_t mode_; +#else const cudnnActivationMode_t mode_; +#endif }; template struct CudnnActivationGradFunctor { using ELEMENT_TYPE = T; +#ifdef PADDLE_WITH_HIP + CudnnActivationGradFunctor(const CUDADeviceContext& ctx, const T& c, + const miopenActivationMode_t& m) + : ctx_(ctx), coef_(c), mode_(m) {} +#else CudnnActivationGradFunctor(const CUDADeviceContext& ctx, const T& c, const cudnnActivationMode_t& m) : ctx_(ctx), coef_(c), mode_(m) {} +#endif void operator()(const Tensor& x, const Tensor& out, const Tensor dout, Tensor* dx) { ActivationDescriptor act_desc; @@ -67,27 +107,40 @@ struct CudnnActivationGradFunctor { out_desc.set(out); dout_desc.set(dout); dx_desc.set(GET_DATA_SAFELY(dx, "Output", "X@GRAD", "CudnnActivationGrad")); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenActivationBackward( + ctx_.cudnn_handle(), act_desc.desc(), + platform::CudnnDataType::kOne(), out_desc.desc(), out.data(), + dout_desc.desc(), dout.data(), x_desc.desc(), x.data(), + platform::CudnnDataType::kZero(), dx_desc.desc(), + dx->mutable_data(ctx_.GetPlace()))); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnActivationBackward( ctx_.cudnn_handle(), act_desc.desc(), platform::CudnnDataType::kOne(), out_desc.desc(), out.data(), dout_desc.desc(), dout.data(), x_desc.desc(), x.data(), platform::CudnnDataType::kZero(), dx_desc.desc(), dx->mutable_data(ctx_.GetPlace()))); +#endif } const CUDADeviceContext& ctx_; const T coef_; +#ifdef PADDLE_WITH_HIP + const miopenActivationMode_t mode_; +#else const cudnnActivationMode_t mode_; +#endif }; template struct CudnnReluFunctor : public CudnnActivationFunctor { explicit CudnnReluFunctor(const CUDADeviceContext& ctx) - : CudnnActivationFunctor(ctx, 0.0, CUDNN_ACTIVATION_RELU) {} + : CudnnActivationFunctor(ctx, 0.0, GPUDNN_ACTIVATION_RELU) {} }; template struct CudnnReluGradFunctor : public CudnnActivationGradFunctor { explicit CudnnReluGradFunctor(const CUDADeviceContext& ctx) - : CudnnActivationGradFunctor(ctx, 0.0, CUDNN_ACTIVATION_RELU) {} + : CudnnActivationGradFunctor(ctx, 0.0, GPUDNN_ACTIVATION_RELU) {} static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } }; @@ -95,13 +148,13 @@ struct CudnnReluGradFunctor : public CudnnActivationGradFunctor { template struct CudnnRelu6Functor : public CudnnActivationFunctor { explicit CudnnRelu6Functor(const CUDADeviceContext& ctx) - : CudnnActivationFunctor(ctx, 6.0, CUDNN_ACTIVATION_CLIPPED_RELU) {} + : CudnnActivationFunctor(ctx, 6.0, GPUDNN_ACTIVATION_CLIPPED_RELU) {} }; template struct CudnnRelu6GradFunctor : public CudnnActivationGradFunctor { explicit CudnnRelu6GradFunctor(const CUDADeviceContext& ctx) - : CudnnActivationGradFunctor(ctx, 6.0, CUDNN_ACTIVATION_CLIPPED_RELU) { - } + : CudnnActivationGradFunctor(ctx, 6.0, + GPUDNN_ACTIVATION_CLIPPED_RELU) {} static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } }; @@ -109,12 +162,12 @@ struct CudnnRelu6GradFunctor : public CudnnActivationGradFunctor { template struct CudnnSigmoidFunctor : public CudnnActivationFunctor { explicit CudnnSigmoidFunctor(const CUDADeviceContext& ctx) - : CudnnActivationFunctor(ctx, 0.0, CUDNN_ACTIVATION_SIGMOID) {} + : CudnnActivationFunctor(ctx, 0.0, GPUDNN_ACTIVATION_SIGMOID) {} }; template struct CudnnSigmoidGradFunctor : public CudnnActivationGradFunctor { explicit CudnnSigmoidGradFunctor(const CUDADeviceContext& ctx) - : CudnnActivationGradFunctor(ctx, 0.0, CUDNN_ACTIVATION_SIGMOID) {} + : CudnnActivationGradFunctor(ctx, 0.0, GPUDNN_ACTIVATION_SIGMOID) {} static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } }; @@ -122,12 +175,12 @@ struct CudnnSigmoidGradFunctor : public CudnnActivationGradFunctor { template struct CudnnTanhFunctor : public CudnnActivationFunctor { explicit CudnnTanhFunctor(const CUDADeviceContext& ctx) - : CudnnActivationFunctor(ctx, 0.0, CUDNN_ACTIVATION_TANH) {} + : CudnnActivationFunctor(ctx, 0.0, GPUDNN_ACTIVATION_TANH) {} }; template struct CudnnTanhGradFunctor : public CudnnActivationGradFunctor { explicit CudnnTanhGradFunctor(const CUDADeviceContext& ctx) - : CudnnActivationGradFunctor(ctx, 0.0, CUDNN_ACTIVATION_TANH) {} + : CudnnActivationGradFunctor(ctx, 0.0, GPUDNN_ACTIVATION_TANH) {} static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } }; @@ -183,6 +236,14 @@ namespace ops = paddle::operators; __macro(sigmoid, CudnnSigmoidFunctor, CudnnSigmoidGradFunctor); \ __macro(tanh, CudnnTanhFunctor, CudnnTanhGradFunctor) +#ifdef PADDLE_WITH_HIP +#define REGISTER_ACTIVATION_CUDNN_KERNEL(act_type, functor, grad_functor) \ + REGISTER_OP_KERNEL(act_type, CUDNN, plat::CUDAPlace, \ + ops::CudnnActivationKernel>); \ + REGISTER_OP_KERNEL( \ + act_type##_grad, CUDNN, plat::CUDAPlace, \ + ops::CudnnActivationGradKernel>); +#else #define REGISTER_ACTIVATION_CUDNN_KERNEL(act_type, functor, grad_functor) \ REGISTER_OP_KERNEL(act_type, CUDNN, plat::CUDAPlace, \ ops::CudnnActivationKernel>, \ @@ -191,5 +252,6 @@ namespace ops = paddle::operators; act_type##_grad, CUDNN, plat::CUDAPlace, \ ops::CudnnActivationGradKernel>, \ ops::CudnnActivationGradKernel>); +#endif FOR_EACH_CUDNN_OP_FUNCTOR(REGISTER_ACTIVATION_CUDNN_KERNEL); diff --git a/paddle/fluid/operators/activation_op.cc b/paddle/fluid/operators/activation_op.cc index 785d6daaecdd2..94f2eb3672bd5 100644 --- a/paddle/fluid/operators/activation_op.cc +++ b/paddle/fluid/operators/activation_op.cc @@ -24,9 +24,6 @@ limitations under the License. */ #include "paddle/fluid/operators/common_infer_shape_functions.h" #include "paddle/fluid/operators/mkldnn/mkldnn_activation_op.h" #include "paddle/fluid/platform/port.h" -#ifdef PADDLE_WITH_CUDA -#include "paddle/fluid/platform/cudnn_helper.h" -#endif DECLARE_bool(use_mkldnn); diff --git a/paddle/fluid/operators/activation_op.cu b/paddle/fluid/operators/activation_op.cu index 2033081af224a..29498da0f026f 100644 --- a/paddle/fluid/operators/activation_op.cu +++ b/paddle/fluid/operators/activation_op.cu @@ -10,8 +10,276 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/operators/activation_op.h" +#include "paddle/fluid/operators/math/math_cuda_utils.h" +#include "paddle/fluid/platform/cuda_device_function.h" #include "paddle/fluid/platform/float16.h" +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; +using float16 = paddle::platform::float16; + +template +struct CudaVecType { + using type = T; + static constexpr int vecsize = 1; +}; + +template <> +struct CudaVecType { + using type = __half2; + static constexpr int vecsize = 2; +}; + +template <> +struct CudaVecType { + using type = float4; + static constexpr int vecsize = 4; +}; + +template +class BaseGPUFunctor { + public: + using ELEMENT_TYPE = T; +}; + +/* ========================================================================== */ + +/* =========================== relu forward ============================ */ +template +class ReluGPUFuctor : public BaseGPUFunctor { + private: + T zero_; + + public: + ReluGPUFuctor() { zero_ = static_cast(0.0f); } + + // for relu forward when T is double + __device__ __forceinline__ typename CudaVecType::type Compute( + const typename CudaVecType::type* x); + + // when num % vecsize != 0 this func will be used + __device__ __forceinline__ T ComputeRemainder(const T x) { + return x > zero_ ? x : zero_; + } +}; + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGPUFuctor::Compute(const CudaVecType::type* x) { +// relu forward : out = max(x, 0) +#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 + return __ldg(x) > zero_ ? __ldg(x) : zero_; +#else + return (*x) > zero_ ? (*x) : zero_; +#endif +} + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGPUFuctor::Compute(const CudaVecType::type* xx) { + // relu forward : out = max(xx, 0) + return make_float4((xx->x > zero_) * (xx->x), (xx->y > zero_) * (xx->y), + (xx->z > zero_) * (xx->z), (xx->w > zero_) * (xx->w)); +} + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGPUFuctor::Compute(const CudaVecType::type* in) { +// relu forward : out = max(in, 0) +#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 + const half2 kzero = __float2half2_rn(0.0f); + return __hmul2(__hgt2(__ldg(in), kzero), __ldg(in)); +#else + const float2 xx = __half22float2(*in); + return __floats2half2_rn((xx.x > 0.0f) * static_cast(xx.x), + (xx.y > 0.0f) * static_cast(xx.y)); +#endif +} +/* ========================================================================== */ + +/* =========================== relu backward ============================ + */ + +template +class ReluGradGPUFunctor : public BaseGPUFunctor { + private: + T zero_; + + public: + ReluGradGPUFunctor() { zero_ = static_cast(0.0f); } + + // for relu backward when T is double + __device__ __forceinline__ typename CudaVecType::type Compute( + const typename CudaVecType::type* out, + const typename CudaVecType::type* dout); + + // when num % vecsize != 0 this func will be used + __device__ __forceinline__ T ComputeRemainder(const T out, const T dout) { + // relu backward : dx = out > 0 ? dout : 0; + return out > zero_ ? dout : zero_; + } + + static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } +}; + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGradGPUFunctor::Compute(const CudaVecType::type* out, + const CudaVecType::type* dout) { +// relu backward : dx = out > 0 ? dout : 0; +#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 + return __ldg(out) > zero_ ? __ldg(dout) : zero_; +#else + return (*out) > zero_ ? (*dout) : zero_; +#endif +} + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGradGPUFunctor::Compute(const CudaVecType::type* out, + const CudaVecType::type* dout) { + // relu backward : dx = out > 0 ? dout : 0; + return make_float4((out->x > zero_) * (dout->x), (out->y > zero_) * (dout->y), + (out->z > zero_) * (dout->z), + (out->w > zero_) * (dout->w)); +} + +template <> +__device__ __forceinline__ CudaVecType::type +ReluGradGPUFunctor::Compute(const CudaVecType::type* out, + const CudaVecType::type* dout) { +// relu backward : dx = out > 0 ? dout : 0; +#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 + const half2 kzero = __float2half2_rn(0.0f); + return __hmul2(__hgt2(__ldg(out), kzero), __ldg(dout)); +#else + const float2 xx = __half22float2(*out); + const float2 yy = __half22float2(*dout); + return __floats2half2_rn((xx.x > 0.0f) * static_cast(yy.x), + (xx.y > 0.0f) * static_cast(yy.y)); +#endif +} + +/* ========================================================================== */ + +template +__global__ void ActivationGradKernelVec(const T* forward_data, const T* dout, + T* dx, int num, Functor functor) { + using VecType = typename CudaVecType::type; + constexpr int vecsize = CudaVecType::vecsize; + int idx = threadIdx.x + blockIdx.x * blockDim.x; + int stride = blockDim.x * gridDim.x; + int loop = num / vecsize; + int tail = num % vecsize; + const VecType* in_forward = reinterpret_cast(forward_data); + const VecType* in_dout = reinterpret_cast(dout); + VecType* out = reinterpret_cast(dx); + + for (int i = idx; i < loop; i += stride) { + out[i] = functor.Compute((in_forward + i), (in_dout + i)); + } + + while (idx == loop && tail) { + dx[num - tail] = + functor.ComputeRemainder(forward_data[num - tail], dout[num - tail]); + --tail; + } +} + +template +__global__ void ActivationkernelVec(const T* src, T* dst, int num, + Functor functor) { + constexpr int vecsize = CudaVecType::vecsize; + using VecType = typename CudaVecType::type; + int idx = threadIdx.x + blockIdx.x * blockDim.x; + int stride = blockDim.x * gridDim.x; + int loop = num / vecsize; + int tail = num % vecsize; + const VecType* in = reinterpret_cast(src); + VecType* out = reinterpret_cast(dst); + + for (int i = idx; i < loop; i += stride) { + out[i] = functor.Compute((in + i)); + } + + while (idx == loop && tail) { + dst[num - tail] = functor.ComputeRemainder(src[num - tail]); + --tail; + } +} + +template +class ActivationGPUKernel + : public framework::OpKernel { + public: + using T = typename Functor::ELEMENT_TYPE; + void Compute(const framework::ExecutionContext& context) const override { + const framework::Tensor* in_x = nullptr; + framework::Tensor* out = nullptr; + ExtractActivationTensor(context, &in_x, &out); + auto& dev_ctx = context.template device_context(); + + int num = in_x->numel(); + const T* input_data = in_x->data(); + T* output_data = out->mutable_data(dev_ctx.GetPlace(), + static_cast(num * sizeof(T))); + + int block = 512; +#ifdef __HIPCC__ + block = 256; +#endif + Functor functor; + constexpr int vecsize = CudaVecType::vecsize; + int grid = max((num / vecsize + block - 1) / block, 1); + ActivationkernelVec<<>>(input_data, output_data, + num, functor); + } +}; + +template +class ActivationGradGPUKernel + : public framework::OpKernel { + public: + using T = typename Functor::ELEMENT_TYPE; + void Compute(const framework::ExecutionContext& context) const override { + const framework::Tensor *x, *out, *d_out; + framework::Tensor* d_x = nullptr; + x = out = d_out = nullptr; + ExtractActivationGradTensor(context, &x, &out, &d_out, + &d_x); + int numel = d_out->numel(); + auto& dev_ctx = context.template device_context(); + auto* dx_data = d_x->mutable_data( + dev_ctx.GetPlace(), static_cast(numel * sizeof(T))); + auto* dout_data = d_out->data(); + + auto* forward_data = dout_data; + if (static_cast(Functor::FwdDeps()) == static_cast(kDepOut)) { + // Only need forward output Out + forward_data = out->data(); + } else if (static_cast(Functor::FwdDeps()) == + static_cast(kDepX)) { + // Only need forward input X + forward_data = x->data(); + } + + int block = 512; +#ifdef __HIPCC__ + block = 256; +#endif + Functor functor; + constexpr int vecsize = CudaVecType::vecsize; + int grid = max((numel / vecsize + block - 1) / block, 1); + ActivationGradKernelVec<<>>( + forward_data, dout_data, dx_data, numel, functor); + } +}; + +} // namespace operators +} // namespace paddle + namespace ops = paddle::operators; namespace plat = paddle::platform; @@ -60,7 +328,21 @@ REGISTER_OP_CUDA_KERNEL( /* ========================================================================== */ /* =========================== relu register ============================ */ -REGISTER_ACTIVATION_CUDA_KERNEL(relu, Relu, ReluCUDAFunctor, ReluGradFunctor); +REGISTER_OP_CUDA_KERNEL( + relu, ops::ActivationGPUKernel>, + ops::ActivationGPUKernel>, + ops::ActivationGPUKernel>); + +REGISTER_OP_CUDA_KERNEL( + relu_grad, ops::ActivationGradGPUKernel>, + ops::ActivationGradGPUKernel>, + ops::ActivationGradGPUKernel>); REGISTER_OP_CUDA_KERNEL( relu_grad_grad, diff --git a/paddle/fluid/operators/activation_op.h b/paddle/fluid/operators/activation_op.h index 289cc70392a3f..bc7def61b2e24 100644 --- a/paddle/fluid/operators/activation_op.h +++ b/paddle/fluid/operators/activation_op.h @@ -1184,9 +1184,9 @@ struct ELUFunctor : public BaseActivationFunctor { template void operator()(Device d, X x, Out out) const { - out.device(d) = x.cwiseMax(static_cast(0)) + - (static_cast(alpha) * (x.exp() - static_cast(1))) - .cwiseMin(static_cast(0)); + out.device(d) = + (x < static_cast(0)) + .select(static_cast(alpha) * (x.exp() - static_cast(1)), x); } }; diff --git a/paddle/fluid/operators/affine_channel_op.cu b/paddle/fluid/operators/affine_channel_op.cu index 5e598071216ae..5fa1e18553bd5 100644 --- a/paddle/fluid/operators/affine_channel_op.cu +++ b/paddle/fluid/operators/affine_channel_op.cu @@ -12,7 +12,15 @@ 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. */ +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif + +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/cuda_primitives.h" @@ -63,7 +71,11 @@ class AffineChannelCUDAKernel : public framework::OpKernel { const T* bias_d = bias->data(); T* y_d = y->data(); +#ifdef PADDLE_WITH_HIP + int block = 256; +#else int block = 1024; +#endif // PADDLE_WITH_HIP int grid = (num + block - 1) / block; int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); @@ -145,7 +157,11 @@ class AffineChannelGradCUDAKernel : public framework::OpKernel { T* ds_d = dscale ? dscale->mutable_data(ctx.GetPlace()) : nullptr; T* db_d = dbias ? dbias->mutable_data(ctx.GetPlace()) : nullptr; +#ifdef PADDLE_WITH_HIP + const int block = 256; +#else const int block = 1024; +#endif // PADDLE_WITH_HIP int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); const int max_blocks = std::max(max_threads / block, 1); int grid1 = (num + block - 1) / block; diff --git a/paddle/fluid/operators/affine_grid_cudnn_op.cu.cc b/paddle/fluid/operators/affine_grid_cudnn_op.cu.cc index c09f71f46c81c..b8ce52387b959 100644 --- a/paddle/fluid/operators/affine_grid_cudnn_op.cu.cc +++ b/paddle/fluid/operators/affine_grid_cudnn_op.cu.cc @@ -12,6 +12,9 @@ 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 PADDLE_WITH_HIP +// HIP not support cudnnSpatialTfGridGeneratorForward + #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/cudnn_helper.h" @@ -121,3 +124,5 @@ REGISTER_OP_KERNEL(affine_grid, CUDNN, plat::CUDAPlace, REGISTER_OP_KERNEL(affine_grid_grad, CUDNN, plat::CUDAPlace, paddle::operators::CUDNNAffineGridGradOpKernel, paddle::operators::CUDNNAffineGridGradOpKernel); + +#endif // not PADDLE_WITH_HIP diff --git a/paddle/fluid/operators/affine_grid_op.cc b/paddle/fluid/operators/affine_grid_op.cc index 675baa67682d4..7be9bced133c2 100644 --- a/paddle/fluid/operators/affine_grid_op.cc +++ b/paddle/fluid/operators/affine_grid_op.cc @@ -21,6 +21,9 @@ limitations under the License. */ #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" #endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -109,7 +112,7 @@ class AffineGridOp : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext& ctx) const override { framework::LibraryType library{framework::LibraryType::kPlain}; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library = framework::LibraryType::kCUDNN; } @@ -226,7 +229,7 @@ class AffineGridOpGrad : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext& ctx) const override { framework::LibraryType library_{framework::LibraryType::kPlain}; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } diff --git a/paddle/fluid/operators/allclose_op.cu b/paddle/fluid/operators/allclose_op.cu index f98fe75cd681a..173e24b2f1450 100644 --- a/paddle/fluid/operators/allclose_op.cu +++ b/paddle/fluid/operators/allclose_op.cu @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/operator.h" #include "paddle/fluid/operators/allclose_op.h" @@ -67,7 +66,11 @@ struct AllcloseFunctor { int block = 1024; int grid = (block - 1 + num) / block; grid = (grid > block) ? block : grid; +#ifdef PADDLE_WITH_HIP + hipMemset(out_data, true, sizeof(bool)); +#else cudaMemset(out_data, true, sizeof(bool)); +#endif AllcloseCUDAKernel<<>>( in_data, other_data, rtol, atol, equal_nan, num, out_data); } diff --git a/paddle/fluid/operators/arg_min_max_op_base.cu.h b/paddle/fluid/operators/arg_min_max_op_base.cu.h index 3e549428b0418..b19ba1e1590fe 100644 --- a/paddle/fluid/operators/arg_min_max_op_base.cu.h +++ b/paddle/fluid/operators/arg_min_max_op_base.cu.h @@ -14,9 +14,15 @@ limitations under the License. */ #pragma once -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) -#include +#ifdef __NVCC__ +#include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include #include #include diff --git a/paddle/fluid/operators/argsort_op.cu b/paddle/fluid/operators/argsort_op.cu index 7fc2a92b7d912..f50d5e619ebea 100644 --- a/paddle/fluid/operators/argsort_op.cu +++ b/paddle/fluid/operators/argsort_op.cu @@ -16,13 +16,28 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/argsort_op.h" #include "paddle/fluid/operators/transpose_op.h" #include "paddle/fluid/platform/cuda_device_function.h" #include "paddle/fluid/platform/cuda_primitives.h" +#ifdef __HIPCC__ +namespace rocprim { +namespace detail { +template <> +struct radix_key_codec_base + : radix_key_codec_integral {}; +} // namespace detail +} // namespace rocprim +#else // set cub base traits in order to handle float16 namespace cub { template <> @@ -30,6 +45,7 @@ struct NumericTraits : BaseTraits {}; } // namespace cub +#endif namespace paddle { namespace operators { @@ -139,7 +155,7 @@ void ArgFullSort(const platform::CUDADeviceContext& ctx, const Tensor* input, cub::CountingInputIterator> segment_offsets_t(counting_iter, SegmentOffsetIter(num_cols)); - cudaError_t err; + gpuError_t err; if (descending) { err = cub::DeviceSegmentedRadixSort::SortPairsDescending( nullptr, temp_storage_bytes, inp, sorted_out_ptr, diff --git a/paddle/fluid/operators/array_to_lod_tensor_op.cc b/paddle/fluid/operators/array_to_lod_tensor_op.cc index 30ac662c5679c..1680ad528abf9 100644 --- a/paddle/fluid/operators/array_to_lod_tensor_op.cc +++ b/paddle/fluid/operators/array_to_lod_tensor_op.cc @@ -51,7 +51,7 @@ struct ArrayToLoDFunctor : public boost::static_visitor { if (std::is_same::value) { Apply(static_cast(pool.Get(place))); } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) Apply(static_cast(pool.Get(place))); #else PADDLE_THROW( diff --git a/paddle/fluid/operators/assign_op.cc b/paddle/fluid/operators/assign_op.cc index e5bceae1c9520..add533bafcb0a 100644 --- a/paddle/fluid/operators/assign_op.cc +++ b/paddle/fluid/operators/assign_op.cc @@ -164,7 +164,7 @@ REGISTER_OP_CPU_KERNEL_FUNCTOR(assign, float, ops::AssignKernel, double, ops::AssignKernel, plat::float16, ops::AssignKernel); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) REGISTER_OP_CUDA_KERNEL_FUNCTOR(assign, float, ops::AssignKernel, double, ops::AssignKernel, int, ops::AssignKernel, int64_t, ops::AssignKernel, bool, diff --git a/paddle/fluid/operators/batch_fc_op.cu b/paddle/fluid/operators/batch_fc_op.cu index 9a39306ccad6a..b686c766e0f8b 100644 --- a/paddle/fluid/operators/batch_fc_op.cu +++ b/paddle/fluid/operators/batch_fc_op.cu @@ -12,7 +12,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include #include #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/operators/batch_fc_op.h" @@ -42,7 +41,7 @@ __global__ void add_bias_kernel(T* data, int slot_pairs_num, int ins_num, } template -void add_bias(cudaStream_t stream, T* data, int slot_pairs_num, int ins_num, +void add_bias(gpuStream_t stream, T* data, int slot_pairs_num, int ins_num, int out_dim, const T* bias) { add_bias_kernel<<>>(data, slot_pairs_num, @@ -65,7 +64,7 @@ __global__ void add_bias_grad_kernel(const T* dout_data, int slot_pairs_num, } template -void add_bias_grad(cudaStream_t stream, const T* dout_data, int slot_pairs_num, +void add_bias_grad(gpuStream_t stream, const T* dout_data, int slot_pairs_num, int ins_num, int out_dim, T* db_data) { add_bias_grad_kernel<<>>(dout_data, slot_pairs_num, ins_num, diff --git a/paddle/fluid/operators/batch_norm_op.cu b/paddle/fluid/operators/batch_norm_op.cu index ae9cf2838b961..444c24b826b1b 100644 --- a/paddle/fluid/operators/batch_norm_op.cu +++ b/paddle/fluid/operators/batch_norm_op.cu @@ -16,12 +16,17 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/operators/batch_norm_op.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/norm_utils.cu.h" -#include "paddle/fluid/platform/cudnn_helper.h" #include "paddle/fluid/platform/float16.h" DECLARE_bool(cudnn_batchnorm_spatial_persistent); @@ -73,6 +78,11 @@ class BatchNormKernel ExtractNCWHD(x_dims, data_layout, &N, &C, &H, &W, &D); auto dtype = platform::CudnnDataType::type; + +#ifdef PADDLE_WITH_HIP + // HIP do not support compute format of NHWC + auto compute_format = DataLayout::kNCHW; +#else const bool fast_nhwc_batch_norm = test_mode || (dtype == CUDNN_DATA_HALF && FLAGS_cudnn_batchnorm_spatial_persistent); @@ -81,6 +91,7 @@ class BatchNormKernel fast_nhwc_batch_norm && data_layout == DataLayout::kNHWC ? DataLayout::kNHWC : DataLayout::kNCHW; +#endif Tensor transformed_x(x->type()); Tensor transformed_y(y->type()); @@ -98,7 +109,17 @@ class BatchNormKernel transformed_y.ShareDataWith(*y); } - // ------------------- cudnn descriptors --------------------- +// ------------------- cudnn descriptors --------------------- +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t data_desc_; + miopenTensorDescriptor_t bn_param_desc_; + miopenBatchNormMode_t mode_; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&bn_param_desc_)); +#else cudnnTensorDescriptor_t data_desc_; cudnnTensorDescriptor_t bn_param_desc_; cudnnBatchNormMode_t mode_; @@ -107,6 +128,7 @@ class BatchNormKernel platform::dynload::cudnnCreateTensorDescriptor(&data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnCreateTensorDescriptor(&bn_param_desc_)); +#endif if (epsilon <= CUDNN_BN_MIN_EPSILON - FLT_EPSILON) { LOG(ERROR) << "Provided epsilon is smaller than " @@ -114,7 +136,10 @@ class BatchNormKernel << "CUDNN_BN_MIN_EPSILON instead."; } epsilon = std::max(epsilon, CUDNN_BN_MIN_EPSILON); -#if CUDNN_VERSION_MIN(7, 0, 1) + +#ifdef PADDLE_WITH_HIP + mode_ = miopenBNSpatial; +#elif CUDNN_VERSION_MIN(7, 0, 1) if (FLAGS_cudnn_batchnorm_spatial_persistent) { mode_ = CUDNN_BATCHNORM_SPATIAL_PERSISTENT; } else { @@ -134,6 +159,17 @@ class BatchNormKernel dims = {N, C, H, W, D}; strides = {H * W * D * C, 1, W * D * C, D * C, C}; } + +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + data_desc_, CudnnDataType::type, + x_dims.size() > 3 ? x_dims.size() : 4, const_cast(dims.data()), + const_cast(strides.data()))); + // Note: PERSISTENT not implemented for inference + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDeriveBNTensorDescriptor( + bn_param_desc_, data_desc_, test_mode ? miopenBNSpatial : mode_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetTensorNdDescriptor( data_desc_, CudnnDataType::type, x_dims.size() > 3 ? x_dims.size() : 4, dims.data(), strides.data())); @@ -142,6 +178,7 @@ class BatchNormKernel platform::dynload::cudnnDeriveBNTensorDescriptor( bn_param_desc_, data_desc_, test_mode ? CUDNN_BATCHNORM_SPATIAL : mode_)); +#endif const auto *scale = ctx.Input("Scale"); const auto *bias = ctx.Input("Bias"); @@ -188,6 +225,30 @@ class BatchNormKernel "variance is [%d], the dimensions of variance is [%s].", C, est_var->dims()[0], est_var->dims())); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenBatchNormalizationForwardInference( + handle, miopenBNSpatial, + const_cast( + static_cast(CudnnDataType::kOne())), + const_cast( + static_cast(CudnnDataType::kZero())), + data_desc_, + static_cast(transformed_x.template data()), + data_desc_, + static_cast( + transformed_y.template mutable_data(ctx.GetPlace())), + bn_param_desc_, + const_cast(static_cast( + scale->template data>())), + const_cast(static_cast( + bias->template data>())), + const_cast(static_cast( + est_mean->template data>())), + const_cast(static_cast( + est_var->template data>())), + epsilon)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnBatchNormalizationForwardInference( handle, @@ -200,6 +261,7 @@ class BatchNormKernel bias->template data>(), est_mean->template data>(), est_var->template data>(), epsilon)); +#endif } else { // if MomentumTensor is set, use MomentumTensor value, momentum // is only used in this training branch @@ -302,6 +364,36 @@ class BatchNormKernel reserve_space_size)); #endif // CUDNN_VERSION_MIN(7, 4, 1) if (!called) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenBatchNormalizationForwardTraining( + handle, mode_, const_cast(static_cast( + CudnnDataType::kOne())), + const_cast( + static_cast(CudnnDataType::kZero())), + data_desc_, + static_cast(transformed_x.template data()), + data_desc_, + static_cast( + transformed_y.template mutable_data(ctx.GetPlace())), + bn_param_desc_, + const_cast(static_cast( + scale->template data>())), + const_cast(static_cast( + bias->template data>())), + this_factor, + static_cast( + mean_out->template mutable_data>( + ctx.GetPlace())), + static_cast(variance_out->template mutable_data< + BatchNormParamType>(ctx.GetPlace())), + epsilon, + static_cast( + saved_mean->template mutable_data>( + ctx.GetPlace())), + static_cast(saved_variance->template mutable_data< + BatchNormParamType>(ctx.GetPlace())))); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnBatchNormalizationForwardTraining( handle, mode_, CudnnDataType::kOne(), @@ -319,6 +411,7 @@ class BatchNormKernel ctx.GetPlace()), saved_variance->template mutable_data>( ctx.GetPlace()))); +#endif } } } @@ -329,11 +422,19 @@ class BatchNormKernel TransToChannelLast( ctx, &transformed_y, y); } +#ifdef PADDLE_WITH_HIP + // clean when exit. + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(bn_param_desc_)); +#else // clean when exit. PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(bn_param_desc_)); +#endif } }; @@ -416,7 +517,7 @@ class InplaceHelper { const BatchNormParamType *mean, const BatchNormParamType *variance, double epsilon, int C, int M, const int num, const T *y, int grid2, const int block, - const cudaStream_t &stream) { + const gpuStream_t &stream) { PADDLE_ENFORCE_EQ(x, y, platform::errors::InvalidArgument( "X and Y should be inplaced in inplace mode")); KeBNRestoreData<<>>( @@ -566,6 +667,10 @@ class BatchNormGradKernel auto dtype = platform::CudnnDataType::type; const auto *reserve_space = ctx.Input("ReserveSpace"); +#ifdef PADDLE_WITH_HIP + // HIP do not support compute format of NHWC + auto compute_format = DataLayout::kNCHW; +#else const bool fast_nhwc_batch_norm = dtype == CUDNN_DATA_HALF && FLAGS_cudnn_batchnorm_spatial_persistent && reserve_space != nullptr; @@ -573,6 +678,7 @@ class BatchNormGradKernel fast_nhwc_batch_norm && data_layout == DataLayout::kNHWC ? DataLayout::kNHWC : DataLayout::kNCHW; +#endif Tensor transformed_x(x->type()); Tensor transformed_d_y(d_y->type()); @@ -626,7 +732,17 @@ class BatchNormGradKernel return; } - // ------------------- cudnn descriptors --------------------- +// ------------------- cudnn descriptors --------------------- +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t data_desc_; + miopenTensorDescriptor_t bn_param_desc_; + miopenBatchNormMode_t mode_; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&bn_param_desc_)); +#else cudnnTensorDescriptor_t data_desc_; cudnnTensorDescriptor_t bn_param_desc_; cudnnBatchNormMode_t mode_; @@ -635,13 +751,16 @@ class BatchNormGradKernel platform::dynload::cudnnCreateTensorDescriptor(&data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnCreateTensorDescriptor(&bn_param_desc_)); +#endif if (epsilon <= CUDNN_BN_MIN_EPSILON - FLT_EPSILON) { LOG(ERROR) << "Provided epsilon is smaller than " << "CUDNN_BN_MIN_EPSILON. Setting it to " << "CUDNN_BN_MIN_EPSILON instead."; } epsilon = std::max(epsilon, CUDNN_BN_MIN_EPSILON); -#if CUDNN_VERSION_MIN(7, 0, 1) +#ifdef PADDLE_WITH_HIP + mode_ = miopenBNSpatial; +#elif CUDNN_VERSION_MIN(7, 0, 1) if (FLAGS_cudnn_batchnorm_spatial_persistent) { mode_ = CUDNN_BATCHNORM_SPATIAL_PERSISTENT; } else { @@ -651,12 +770,22 @@ class BatchNormGradKernel mode_ = CUDNN_BATCHNORM_SPATIAL; #endif // CUDNN_VERSION_MIN(7, 0, 1) +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + data_desc_, CudnnDataType::type, + x_dims.size() > 3 ? x_dims.size() : 4, const_cast(dims.data()), + const_cast(strides.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDeriveBNTensorDescriptor(bn_param_desc_, + data_desc_, mode_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetTensorNdDescriptor( data_desc_, CudnnDataType::type, x_dims.size() > 3 ? x_dims.size() : 4, dims.data(), strides.data())); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDeriveBNTensorDescriptor(bn_param_desc_, data_desc_, mode_)); +#endif const auto *saved_mean = ctx.Input("SavedMean"); const auto *saved_var = ctx.Input("SavedVariance"); @@ -741,6 +870,22 @@ class BatchNormGradKernel /*reserveSpaceSizeInBytes=*/reserve_space_size)); #endif // CUDNN_VERSION_MIN(7, 4, 1) if (!called) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenBatchNormalizationBackward( + dev_ctx.cudnn_handle(), mode_, CudnnDataType::kOne(), + CudnnDataType::kZero(), CudnnDataType::kOne(), + CudnnDataType::kZero(), data_desc_, + transformed_x.template data(), data_desc_, + transformed_d_y.template data(), data_desc_, + transformed_d_x.template mutable_data(ctx.GetPlace()), + bn_param_desc_, scale->template data>(), + d_scale->template mutable_data>( + ctx.GetPlace()), + d_bias->template mutable_data>( + ctx.GetPlace()), + epsilon, saved_mean_data, saved_var_data)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnBatchNormalizationBackward( dev_ctx.cudnn_handle(), mode_, CudnnDataType::kOne(), @@ -755,6 +900,7 @@ class BatchNormGradKernel d_bias->template mutable_data>( ctx.GetPlace()), epsilon, saved_mean_data, saved_var_data)); +#endif } if (data_layout == DataLayout::kNHWC && @@ -784,11 +930,19 @@ class BatchNormGradKernel } } +#ifdef PADDLE_WITH_HIP + // clean when exit. + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(bn_param_desc_)); +#else // clean when exit. PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(bn_param_desc_)); +#endif } else { const auto *running_mean = ctx.Input("Mean"); const auto *running_var = ctx.Input("Variance"); @@ -886,6 +1040,18 @@ class BatchNormDoubleGradKernel namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL( + batch_norm, ops::BatchNormKernel, + ops::BatchNormKernel); +REGISTER_OP_CUDA_KERNEL( + batch_norm_grad, ops::BatchNormGradKernel, + ops::BatchNormGradKernel); +REGISTER_OP_CUDA_KERNEL( + batch_norm_grad_grad, + ops::BatchNormDoubleGradKernel); +#else REGISTER_OP_CUDA_KERNEL( batch_norm, ops::BatchNormKernel, ops::BatchNormKernel, @@ -898,3 +1064,4 @@ REGISTER_OP_CUDA_KERNEL( batch_norm_grad_grad, ops::BatchNormDoubleGradKernel, ops::BatchNormDoubleGradKernel); +#endif diff --git a/paddle/fluid/operators/bce_loss_op.cu b/paddle/fluid/operators/bce_loss_op.cu index 1a967c57385a0..99153101fc326 100644 --- a/paddle/fluid/operators/bce_loss_op.cu +++ b/paddle/fluid/operators/bce_loss_op.cu @@ -12,7 +12,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ #include -#include "cub/cub.cuh" #include "paddle/fluid/operators/bce_loss_op.h" #include "paddle/fluid/operators/math.h" #include "paddle/fluid/platform/cuda_primitives.h" diff --git a/paddle/fluid/operators/bmm_op.cu b/paddle/fluid/operators/bmm_op.cu index 961d74b7ad42a..15a7506a8f5af 100644 --- a/paddle/fluid/operators/bmm_op.cu +++ b/paddle/fluid/operators/bmm_op.cu @@ -11,7 +11,7 @@ #include "paddle/fluid/operators/bmm_op.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) namespace ops = paddle::operators; REGISTER_OP_CUDA_KERNEL( bmm, ops::BmmKernel, diff --git a/paddle/fluid/operators/cast_op.cc b/paddle/fluid/operators/cast_op.cc index c5cfa7a3bafce..40f4b969ec060 100644 --- a/paddle/fluid/operators/cast_op.cc +++ b/paddle/fluid/operators/cast_op.cc @@ -97,5 +97,6 @@ REGISTER_OP_CPU_KERNEL(cast, ops::CastOpKernel, ops::CastOpKernel, ops::CastOpKernel, ops::CastOpKernel, + ops::CastOpKernel, ops::CastOpKernel, ops::CastOpKernel); diff --git a/paddle/fluid/operators/cholesky_op.cu b/paddle/fluid/operators/cholesky_op.cu index 530147609fe1e..4426057305249 100644 --- a/paddle/fluid/operators/cholesky_op.cu +++ b/paddle/fluid/operators/cholesky_op.cu @@ -12,6 +12,9 @@ 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 PADDLE_WITH_HIP +// HIP not support cusolver + #include #include #include @@ -164,3 +167,5 @@ REGISTER_OP_CUDA_KERNEL( cholesky_grad, ops::CholeskyGradKernel, ops::CholeskyGradKernel); + +#endif // not PADDLE_WITH_HIP diff --git a/paddle/fluid/operators/clip_op.h b/paddle/fluid/operators/clip_op.h index 097b5e4863d6f..93157ed9d47bb 100644 --- a/paddle/fluid/operators/clip_op.h +++ b/paddle/fluid/operators/clip_op.h @@ -25,7 +25,7 @@ namespace operators { using framework::Tensor; using platform::Transform; -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template __global__ void ClipCudaKernel(const T* input, T* out, int num, UnaryOperation op) { @@ -105,7 +105,7 @@ class ClipKernel : public framework::OpKernel { const T* x_data = x->data(); int64_t numel = x->numel(); if (platform::is_gpu_place(context.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) int threads = 256; int blocks = (numel + threads - 1) / threads; ClipCudaKernel><<< diff --git a/paddle/fluid/operators/coalesce_tensor_op.cc b/paddle/fluid/operators/coalesce_tensor_op.cc index 464d8c8d56f5c..153fa529f96a5 100644 --- a/paddle/fluid/operators/coalesce_tensor_op.cc +++ b/paddle/fluid/operators/coalesce_tensor_op.cc @@ -289,7 +289,7 @@ REGISTER_OP_CPU_KERNEL( ops::CoalesceTensorOpKernel, ops::CoalesceTensorOpKernel); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) REGISTER_OP_CUDA_KERNEL( coalesce_tensor, ops::CoalesceTensorOpKernel); #endif +#ifdef PADDLE_WITH_XPU +REGISTER_OP_XPU_KERNEL( + coalesce_tensor, + ops::CoalesceTensorOpKernel, + ops::CoalesceTensorOpKernel, + ops::CoalesceTensorOpKernel, + ops::CoalesceTensorOpKernel); +#endif + REGISTER_OP_VERSION(coalesce_tensor) .AddCheckpoint( R"ROC( diff --git a/paddle/fluid/operators/collective/c_comm_init_op.cc b/paddle/fluid/operators/collective/c_comm_init_op.cc index 3464bff486ae2..f4510861672ca 100644 --- a/paddle/fluid/operators/collective/c_comm_init_op.cc +++ b/paddle/fluid/operators/collective/c_comm_init_op.cc @@ -68,10 +68,10 @@ class CCommInitOp : public framework::OperatorBase { nccl_id, nranks, rank_id, device_id, rid); #else PADDLE_THROW(platform::errors::PreconditionNotMet( - "PaddlePaddle should compile with GPU.")); + "PaddlePaddle should be compiled with GPU.")); #endif } else if (is_xpu_place(place)) { -#if defined(PADDLE_WITH_BKCL) +#if defined(PADDLE_WITH_XPU_BKCL) BKCLUniqueId* bkcl_id = var->GetMutable(); int nranks = Attr("nranks"); @@ -81,7 +81,7 @@ class CCommInitOp : public framework::OperatorBase { rid, 0, platform::errors::OutOfRange( "Ring id must equal 0 in multi Kunlun cards training, but got %d", - ring_id)); + rid)); int device_id = BOOST_GET_CONST(platform::XPUPlace, place).device; if (Attr("device_id") >= 0) { device_id = Attr("device_id"); @@ -90,7 +90,7 @@ class CCommInitOp : public framework::OperatorBase { bkcl_id, nranks, rank_id, device_id, rid); #else PADDLE_THROW(platform::errors::PreconditionNotMet( - "PaddlePaddle should compile with XPU.")); + "PaddlePaddle should be compiled with XPU.")); #endif } else { PADDLE_THROW(platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/operators/collective/c_gen_nccl_id_op.cc b/paddle/fluid/operators/collective/c_gen_nccl_id_op.cc index 485a6d7ec4ed3..1592d809f91e2 100644 --- a/paddle/fluid/operators/collective/c_gen_nccl_id_op.cc +++ b/paddle/fluid/operators/collective/c_gen_nccl_id_op.cc @@ -75,7 +75,8 @@ class CGenNCCLIdOp : public framework::OperatorBase { platform::SendBroadCastCommID(endpoint_list, &nccl_ids); } else { std::string endpoint = Attr("endpoint"); - platform::RecvBroadCastCommID(endpoint, &nccl_ids); + int server_fd = platform::SocketServer::GetInstance(endpoint).socket(); + platform::RecvBroadCastCommID(server_fd, endpoint, &nccl_ids); } CopyNCCLIDToVar(nccl_ids, func, scope); diff --git a/paddle/fluid/operators/collective/c_wait_comm_op.cc b/paddle/fluid/operators/collective/c_wait_comm_op.cc new file mode 100644 index 0000000000000..d0dfc3bb1c2e5 --- /dev/null +++ b/paddle/fluid/operators/collective/c_wait_comm_op.cc @@ -0,0 +1,91 @@ +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include + +#include "paddle/fluid/framework/op_registry.h" +namespace paddle { +namespace framework { +class Scope; +} // namespace framework +} // namespace paddle +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) +#include "paddle/fluid/platform/collective_helper.h" +#endif + +namespace paddle { +namespace operators { + +class CWaitCommOp : public framework::OperatorBase { + public: + CWaitCommOp(const std::string& type, const framework::VariableNameMap& inputs, + const framework::VariableNameMap& outputs, + const framework::AttributeMap& attrs) + : OperatorBase(type, inputs, outputs, attrs) {} + + void RunImpl(const framework::Scope& scope, + const platform::Place& place) const override { + PADDLE_ENFORCE_EQ(is_gpu_place(place), true, + platform::errors::PreconditionNotMet( + "wait_comm op can run on gpu place only for now.")); + +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) + int ring_id = Attr("ring_id"); + + auto compute_stream = + static_cast( + platform::DeviceContextPool::Instance().Get(place)) + ->stream(); + auto comm_stream = + platform::NCCLCommContext::Instance().Get(ring_id, place)->stream(); + + auto event = + platform::NCCLCommContext::Instance().Get(ring_id, place)->comm_event(); + +// comm_stream-->event-->compute_stream +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipEventRecord(event, comm_stream)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamWaitEvent(compute_stream, event, 0)); +#else + PADDLE_ENFORCE_CUDA_SUCCESS(cudaEventRecord(event, comm_stream)); + PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamWaitEvent(compute_stream, event, 0)); +#endif +#else + PADDLE_THROW(platform::errors::PreconditionNotMet( + "PaddlePaddle should compile with GPU.")); +#endif + } +}; + +class CWaitCommOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() { + AddInput("X", "(Tensor) Dependency of the variable need to sync") + .AsDuplicable(); + AddOutput("Out", "(Tensor) Dependency of the variable need to sync") + .AsDuplicable(); + AddAttr("ring_id", "(int default 0) ring id.").SetDefault(0); + AddComment(R"DOC( +CWaitComm Operator + +Compute stream wait Comm Stream with async event. +)DOC"); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; + +REGISTER_OPERATOR(c_wait_comm, ops::CWaitCommOp, ops::CWaitCommOpMaker); diff --git a/paddle/fluid/operators/collective/c_wait_compute_op.cc b/paddle/fluid/operators/collective/c_wait_compute_op.cc new file mode 100644 index 0000000000000..12a28040ef1c5 --- /dev/null +++ b/paddle/fluid/operators/collective/c_wait_compute_op.cc @@ -0,0 +1,95 @@ +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include + +#include "paddle/fluid/framework/op_registry.h" +namespace paddle { +namespace framework { +class Scope; +} // namespace framework +} // namespace paddle +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) +#include "paddle/fluid/platform/collective_helper.h" +#endif + +namespace paddle { +namespace operators { + +class CWaitComputeOp : public framework::OperatorBase { + public: + CWaitComputeOp(const std::string& type, + const framework::VariableNameMap& inputs, + const framework::VariableNameMap& outputs, + const framework::AttributeMap& attrs) + : OperatorBase(type, inputs, outputs, attrs) {} + + void RunImpl(const framework::Scope& scope, + const platform::Place& place) const override { + PADDLE_ENFORCE_EQ( + is_gpu_place(place), true, + platform::errors::PreconditionNotMet( + "wait_compute op can run on gpu place only for now.")); + +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) + int ring_id = Attr("ring_id"); + + auto compute_stream = + static_cast( + platform::DeviceContextPool::Instance().Get(place)) + ->stream(); + auto comm_stream = + platform::NCCLCommContext::Instance().Get(ring_id, place)->stream(); + + auto event = platform::NCCLCommContext::Instance() + .Get(ring_id, place) + ->compute_event(); + +// compute_stream-->event-->comm_stream +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipEventRecord(event, compute_stream)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamWaitEvent(comm_stream, event, 0)); +#else + PADDLE_ENFORCE_CUDA_SUCCESS(cudaEventRecord(event, compute_stream)); + PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamWaitEvent(comm_stream, event, 0)); +#endif +#else + PADDLE_THROW(platform::errors::PreconditionNotMet( + "PaddlePaddle should compile with GPU.")); +#endif + } +}; + +class CWaitComputeOpMaker : public framework::OpProtoAndCheckerMaker { + public: + void Make() { + AddInput("X", "(Tensor) Dependency of the variable need to sync") + .AsDuplicable(); + AddOutput("Out", "(Tensor) Dependency of the variable need to sync") + .AsDuplicable(); + AddAttr("ring_id", "(int default 0) ring id.").SetDefault(0); + AddComment(R"DOC( +CWaitCompute Operator + +Comm stream wait Compute Stream with async event. +)DOC"); + } +}; + +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; + +REGISTER_OPERATOR(c_wait_compute, ops::CWaitComputeOp, + ops::CWaitComputeOpMaker); diff --git a/paddle/fluid/operators/collective/gen_bkcl_id_op.cc b/paddle/fluid/operators/collective/gen_bkcl_id_op.cc index f14271e367d1b..7067bfb314485 100644 --- a/paddle/fluid/operators/collective/gen_bkcl_id_op.cc +++ b/paddle/fluid/operators/collective/gen_bkcl_id_op.cc @@ -1,4 +1,4 @@ -/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. diff --git a/paddle/fluid/operators/collective/recv_v2_op.cc b/paddle/fluid/operators/collective/recv_v2_op.cc index 10408820387b7..0ae7b821617f9 100644 --- a/paddle/fluid/operators/collective/recv_v2_op.cc +++ b/paddle/fluid/operators/collective/recv_v2_op.cc @@ -40,6 +40,14 @@ class RecvOpV2 : public framework::OperatorWithKernel { "The size of the output shape must be greater than 0 " "but the value given is %d.", out_shape.size())); + for (size_t i = 0; i < out_shape.size(); ++i) { + PADDLE_ENFORCE_GE(out_shape[i], 1, + platform::errors::InvalidArgument( + "The shape attribute for recv_v2 must be set " + "explicitly, but the %dth element is %d which " + "is less than 1.", + i, out_shape[i])); + } ctx->SetOutputDim("Out", framework::make_ddim(out_shape)); } diff --git a/paddle/fluid/operators/collective/recv_v2_op.cu.cc b/paddle/fluid/operators/collective/recv_v2_op.cu.cc index 5b846598b892f..7912733fa50cc 100644 --- a/paddle/fluid/operators/collective/recv_v2_op.cu.cc +++ b/paddle/fluid/operators/collective/recv_v2_op.cu.cc @@ -42,6 +42,7 @@ class RecvOpV2CUDAKernel : public framework::OpKernel { auto out = ctx.Output("Out"); auto out_dims = out->dims(); + auto numel = out->numel(); int data_type = ctx.Attr("dtype"); framework::proto::VarType::Type type = framework::proto::VarType::Type(data_type); @@ -61,34 +62,8 @@ class RecvOpV2CUDAKernel : public framework::OpKernel { platform::errors::InvalidArgument("The value of peer (%d) you set must " "be less than comm->nranks (%d).", peer, comm->nranks())); - ncclDataType_t dtype = platform::ToNCCLDataType(type); - - // Recv the number of elements to receive first - int numel = 0; - int *numel_ptr = nullptr; -#ifdef PADDLE_WITH_RCCL - PADDLE_ENFORCE_CUDA_SUCCESS(hipMalloc(&numel_ptr, sizeof(int))); -#else - PADDLE_ENFORCE_CUDA_SUCCESS(cudaMalloc(&numel_ptr, sizeof(int))); -#endif - PADDLE_ENFORCE_CUDA_SUCCESS( - platform::dynload::ncclRecv(static_cast(numel_ptr), 1, ncclInt, - peer, comm->comm(), stream)); -#ifdef PADDLE_WITH_RCCL - PADDLE_ENFORCE_CUDA_SUCCESS( - hipMemcpy(&numel, numel_ptr, sizeof(int), hipMemcpyDeviceToHost)); -#else - PADDLE_ENFORCE_CUDA_SUCCESS( - cudaMemcpy(&numel, numel_ptr, sizeof(int), cudaMemcpyDeviceToHost)); -#endif - - int rest_numel = 1; - for (int i = 1; i < out_dims.size(); ++i) { - rest_numel = rest_numel * out_dims[i]; - } - out_dims[0] = numel / rest_numel; out->mutable_data(out_dims, place); - + ncclDataType_t dtype = platform::ToNCCLDataType(type); PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::ncclRecv( out->data(), numel, dtype, peer, comm->comm(), stream)); VLOG(3) << "rank " << comm->rank() << " recv " diff --git a/paddle/fluid/operators/collective/send_v2_op.cu.cc b/paddle/fluid/operators/collective/send_v2_op.cu.cc index b70124a7bf8dd..c4f5d05e68fa8 100644 --- a/paddle/fluid/operators/collective/send_v2_op.cu.cc +++ b/paddle/fluid/operators/collective/send_v2_op.cu.cc @@ -57,21 +57,6 @@ class SendOpV2CUDAKernel : public framework::OpKernel { "be less than comm->nranks (%d).", peer, comm->nranks())); ncclDataType_t dtype = platform::ToNCCLDataType(x->type()); - // Send number of elements to the receiver, as the receiver may have - // no information of the Tensor size. - int* numel_ptr = nullptr; -#ifdef PADDLE_WITH_RCCL - PADDLE_ENFORCE_CUDA_SUCCESS(hipMalloc(&numel_ptr, sizeof(int))); - PADDLE_ENFORCE_CUDA_SUCCESS( - hipMemcpy(numel_ptr, &numel, sizeof(int), hipMemcpyHostToDevice)); -#else - PADDLE_ENFORCE_CUDA_SUCCESS(cudaMalloc(&numel_ptr, sizeof(int))); - PADDLE_ENFORCE_CUDA_SUCCESS( - cudaMemcpy(numel_ptr, &numel, sizeof(int), cudaMemcpyHostToDevice)); -#endif - - PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::ncclSend( - numel_ptr, 1, ncclInt, peer, comm->comm(), stream)); PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::ncclSend( x->data(), numel, dtype, peer, comm->comm(), stream)); VLOG(3) << "rank " << comm->rank() << " send " diff --git a/paddle/fluid/operators/conv_cudnn_op.cu b/paddle/fluid/operators/conv_cudnn_op.cu index 5ef22b81869f6..39e9d37ddc6c7 100644 --- a/paddle/fluid/operators/conv_cudnn_op.cu +++ b/paddle/fluid/operators/conv_cudnn_op.cu @@ -19,11 +19,13 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/tensor.h" #include "paddle/fluid/memory/memory.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/operators/conv_miopen_helper.h" +#else #include "paddle/fluid/operators/conv_cudnn_helper.h" -#include "paddle/fluid/operators/conv_cudnn_op_cache.h" +#endif #include "paddle/fluid/operators/conv_op.h" #include "paddle/fluid/operators/math/padding.h" -#include "paddle/fluid/platform/cudnn_helper.h" #include "paddle/fluid/platform/cudnn_workspace_helper.h" #include "paddle/fluid/platform/float16.h" #include "paddle/fluid/platform/profiler.h" @@ -78,6 +80,10 @@ class CUDNNConvOpKernel : public framework::OpKernel { auto dtype = platform::CudnnDataType::type; +#ifdef PADDLE_WITH_HIP + // HIP MIOPEN ONLY SUPPORT NCHW format + auto compute_format = DataLayout::kNCHW; +#else // Tensor Core introduced from Volta GPUs supports more faster conv op // with FP16 in NHWC data format. const bool compute_in_nhwc = @@ -86,6 +92,7 @@ class CUDNNConvOpKernel : public framework::OpKernel { // cudnn will convert NCHW to NHWC automatically on Tensor Core. auto compute_format = compute_in_nhwc && channel_last ? DataLayout::kNHWC : DataLayout::kNCHW; +#endif VLOG(3) << "Compute ConvOp with cuDNN:" << " data_format=" << data_format << " compute_format=" << (compute_format == DataLayout::kNHWC ? "NHWC" : "NCHW"); @@ -240,10 +247,17 @@ class CUDNNConvOpKernel : public framework::OpKernel { auto layout_format = GetCudnnTensorFormat(layout); args.handle = handle; + +#ifdef PADDLE_WITH_HIP + // MIOPEN need to set groups in cdesc in miopen_desc.h + args.cdesc.set(dtype, padding_common, strides, dilations, + platform::AllowTF32Cudnn(), groups); +#else args.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn()); +#endif -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION_MIN(7, 0, 1) // cudnn 7 can support groups, no need to do it manually // FIXME(typhoonzero): find a better way to disable groups // rather than setting it to 1. @@ -251,6 +265,10 @@ class CUDNNConvOpKernel : public framework::OpKernel { platform::dynload::cudnnSetConvolutionGroupCount(args.cdesc.desc(), groups)); groups = 1; +#endif +#ifdef PADDLE_WITH_HIP + // MIOPEN do not set groups in wdesc after set groups in cdesc + groups = 1; #endif args.idesc.set(transformed_input, layout_format); args.wdesc.set(transformed_filter_channel, layout_format, groups); @@ -275,14 +293,20 @@ class CUDNNConvOpKernel : public framework::OpKernel { int group_offset_filter = transformed_filter_channel.numel() / groups; // ------------------- cudnn conv workspace --------------------- size_t workspace_size = 0; // final workspace to allocate. - // ------------------- cudnn conv algorithm --------------------- +// ------------------- cudnn conv algorithm --------------------- +#ifdef PADDLE_WITH_HIP + miopenConvFwdAlgorithm_t algo{}; + using search = SearchAlgorithm; + workspace_size = search::GetWorkspaceSize(args); + algo = search::Find(args, exhaustive_search, false, workspace_size, ctx); +#else cudnnConvolutionFwdAlgo_t algo{}; - using search = SearchAlgorithm; algo = search::Find(args, exhaustive_search, false, ctx); workspace_size = search::GetWorkspaceSize(args, algo); +#endif -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION_MIN(7, 0, 1) // when groups > 1, SearchAlgorithm find algo is CUDNN_CONVOLUTION_\ // FWD_ALGO_WINOGRAD_NONFUSED, but this kind of algorithm is unstable // in forward computation, so change the algorithm to CUDNN_CONVOLUTION_\ @@ -296,10 +320,22 @@ class CUDNNConvOpKernel : public framework::OpKernel { ScalingParamType alpha = 1.0f; ScalingParamType beta = 0.0f; - // NOTE(zhiqiu): inplace addto is not supportted in double grad yet. - // ScalingParamType beta = ctx.Attr("use_addto") ? 1.0f : 0.0f; - // VLOG(4) << "Conv: use_addto = " << ctx.Attr("use_addto"); - +// NOTE(zhiqiu): inplace addto is not supportted in double grad yet. +// ScalingParamType beta = ctx.Attr("use_addto") ? 1.0f : 0.0f; +// VLOG(4) << "Conv: use_addto = " << ctx.Attr("use_addto"); + +#ifdef PADDLE_WITH_HIP + workspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForward( + handle, &alpha, args.idesc.desc(), input_data, + args.wdesc.desc(), filter_data, args.cdesc.desc(), algo, + &beta, args.odesc.desc(), output_data, workspace_ptr, + workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { workspace_handle.RunFunc( [&](void* workspace_ptr) { @@ -313,6 +349,7 @@ class CUDNNConvOpKernel : public framework::OpKernel { }, workspace_size); } +#endif if (channel_last && compute_format == DataLayout::kNCHW) { TransToChannelLast( @@ -361,10 +398,16 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { const bool channel_last = (data_format == "NHWC" || data_format == "NDHWC"); auto dtype = platform::CudnnDataType::type; + +#ifdef PADDLE_WITH_HIP + // HIP MIOPEN ONLY SUPPORT NCHW format + auto compute_format = DataLayout::kNCHW; +#else const bool compute_in_nhwc = dtype == CUDNN_DATA_HALF && IsVoltaOrLater(dev_ctx); auto compute_format = compute_in_nhwc && channel_last ? DataLayout::kNHWC : DataLayout::kNCHW; +#endif VLOG(3) << "Compute ConvGradOp with cuDNN:" << " data_format=" << data_format << " compute_format=" << (compute_format == DataLayout::kNHWC ? "NHWC" : "NCHW"); @@ -581,16 +624,23 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { int group_offset_in = i_c / groups * i_h * i_w * i_d; int group_offset_out = o_c / groups * o_h * o_w * o_d; int group_offset_filter = transformed_filter_channel.numel() / groups; - // ------------------- cudnn backward algorithm --------------------- +// ------------------- cudnn backward algorithm --------------------- +#ifdef PADDLE_WITH_HIP + miopenConvBwdDataAlgorithm_t data_algo = + static_cast(0); + miopenConvBwdWeightsAlgorithm_t filter_algo = + static_cast(0); +#else cudnnConvolutionBwdDataAlgo_t data_algo = static_cast(0); cudnnConvolutionBwdFilterAlgo_t filter_algo = static_cast(0); +#endif size_t workspace_size = 0; int iwo_groups = groups; int c_groups = 1; -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_HIP) || CUDNN_VERSION_MIN(7, 0, 1) iwo_groups = 1; c_groups = groups; groups = 1; @@ -607,11 +657,19 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { args1.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_groups); +#ifdef PADDLE_WITH_HIP + using search1 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search1::GetWorkspaceSize(args1)); + data_algo = search1::Find(args1, exhaustive_search, deterministic, + workspace_size, ctx); +#else using search1 = SearchAlgorithm; data_algo = search1::Find(args1, exhaustive_search, deterministic, ctx); workspace_size = std::max(workspace_size, search1::GetWorkspaceSize(args1, data_algo)); +#endif } if (filter_grad) { @@ -624,12 +682,19 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { args2.odesc.set(transformed_output_grad_channel, layout_tensor); args2.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_groups); - +#ifdef PADDLE_WITH_HIP + using search2 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search2::GetWorkspaceSize(args2)); + filter_algo = search2::Find(args2, exhaustive_search, deterministic, + workspace_size, ctx); +#else using search2 = SearchAlgorithm; filter_algo = search2::Find(args2, exhaustive_search, deterministic, ctx); workspace_size = std::max(workspace_size, search2::GetWorkspaceSize(args2, filter_algo)); +#endif } // ------------------- cudnn conv backward data --------------------- @@ -638,8 +703,21 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { VLOG(4) << "Conv_grad: use_addto = " << ctx.Attr("use_addto"); if (input_grad) { - // When beta is 0, it is unnecessary to reset input_grad. - // When beta is 1, the output cannot be reset since addt strategy used. +// When beta is 0, it is unnecessary to reset input_grad. +// When beta is 1, the output cannot be reset since addt strategy used. +#ifdef PADDLE_WITH_HIP + workspace_handle.RunFunc( + [&](void* cudnn_workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardData( + handle, &alpha, args1.odesc.desc(), output_grad_data, + args1.wdesc.desc(), filter_data, args1.cdesc.desc(), + data_algo, &beta, args1.idesc.desc(), + transformed_input_grad_data, cudnn_workspace_ptr, + workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { workspace_handle.RunFunc( [&](void* cudnn_workspace_ptr) { @@ -654,7 +732,7 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { }, workspace_size); } - +#endif if (!is_sys_pad) { std::vector starts(transformed_input_channel.dims().size(), 0); std::vector axes(transformed_input_channel.dims().size(), 0); @@ -686,7 +764,19 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { ScalingParamType beta_filter = 0.0f; // ------------------- cudnn conv backward filter --------------------- if (filter_grad) { - // Because beta is zero, it is unnecessary to reset filter_grad. +// Because beta is zero, it is unnecessary to reset filter_grad. +#ifdef PADDLE_WITH_HIP + workspace_handle.RunFunc( + [&](void* cudnn_workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardWeights( + handle, &alpha, args2.odesc.desc(), output_grad_data, + args2.idesc.desc(), input_data, args2.cdesc.desc(), + filter_algo, &beta, args2.wdesc.desc(), filter_grad_data, + cudnn_workspace_ptr, workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { workspace_handle.RunFunc( [&](void* cudnn_workspace_ptr) { @@ -701,6 +791,7 @@ class CUDNNConvGradOpKernel : public framework::OpKernel { }, workspace_size); } +#endif if (compute_format == DataLayout::kNHWC) { TransToChannelFirst( @@ -930,7 +1021,7 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { int iwo_group = groups; int c_group = 1; -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_HIP) || CUDNN_VERSION_MIN(7, 0, 1) iwo_group = 1; c_group = groups; groups = 1; @@ -960,6 +1051,16 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { &transformed_dX, ddW, &transformed_dO_channel, strides, padding_common, dilations, dtype}; +#ifdef PADDLE_WITH_HIP + miopenConvFwdAlgorithm_t fwd_algo1 = + static_cast(0); + miopenConvFwdAlgorithm_t fwd_algo2 = + static_cast(0); + miopenConvBwdDataAlgorithm_t data_algo = + static_cast(0); + miopenConvBwdWeightsAlgorithm_t filter_algo = + static_cast(0); +#else cudnnConvolutionFwdAlgo_t fwd_algo1 = static_cast(0); cudnnConvolutionFwdAlgo_t fwd_algo2 = @@ -968,6 +1069,7 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { static_cast(0); cudnnConvolutionBwdFilterAlgo_t filter_algo = static_cast(0); +#endif auto layout = GetCudnnTensorFormat(DataLayout::kNCHW); @@ -986,26 +1088,39 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { args1.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_group); +#ifdef PADDLE_WITH_HIP + using search1 = SearchAlgorithm; + workspace_size = search1::GetWorkspaceSize(args1); + fwd_algo1 = search1::Find(args1, exhaustive_search, false, + workspace_size, ctx); +#else using search1 = SearchAlgorithm; fwd_algo1 = search1::Find(args1, exhaustive_search, false, ctx); workspace_size = search1::GetWorkspaceSize(args1, fwd_algo1); +#endif } if (ddW) { ddw = ddW->data(); args2.handle = handle; args2.idesc.set(transformed_X, iwo_group); - args2.wdesc.set(*ddW, layout, iwo_group); - args2.odesc.set(transformed_ddO_channel, iwo_group); args2.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_group); +#ifdef PADDLE_WITH_HIP + using search2 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search2::GetWorkspaceSize(args2)); + fwd_algo2 = search2::Find(args2, exhaustive_search, false, + workspace_size, ctx); +#else using search2 = SearchAlgorithm; fwd_algo2 = search2::Find(args2, exhaustive_search, false, ctx); workspace_size = std::max(workspace_size, search2::GetWorkspaceSize(args2, fwd_algo2)); +#endif } } @@ -1014,17 +1129,23 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { args3.handle = handle; args3.idesc.set(transformed_ddX, iwo_group); args3.wdesc.set(*dW, layout, iwo_group); - args3.odesc.set(transformed_dO_channel, iwo_group); - args3.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_group); +#ifdef PADDLE_WITH_HIP + using search3 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search3::GetWorkspaceSize(args3)); + filter_algo = search3::Find(args3, exhaustive_search, deterministic, + workspace_size, ctx); +#else using search3 = SearchAlgorithm; filter_algo = search3::Find(args3, exhaustive_search, deterministic, ctx); workspace_size = std::max(workspace_size, search3::GetWorkspaceSize(args3, filter_algo)); +#endif } if (ddW && dX) { @@ -1037,11 +1158,19 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { args4.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_group); +#ifdef PADDLE_WITH_HIP + using search4 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search4::GetWorkspaceSize(args4)); + data_algo = search4::Find(args4, exhaustive_search, deterministic, + workspace_size, ctx); +#else using search4 = SearchAlgorithm; data_algo = search4::Find(args4, exhaustive_search, deterministic, ctx); workspace_size = std::max(workspace_size, search4::GetWorkspaceSize(args4, data_algo)); +#endif } int i_n, i_c, i_d, i_h, i_w; @@ -1063,12 +1192,23 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { // ScalingParamType beta = ctx.Attr("use_addto") ? 1.0f : // 0.0f; // VLOG(4) << "Conv_grad_grad: use_addto = " << ctx.Attr("use_addto"); - auto wkspace_handle = dev_ctx.cudnn_workspace_handle(); if (ddO) { if (ddX) { ddx = transformed_ddX.data(); +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForward( + handle, &alpha, args1.idesc.desc(), ddx, + args1.wdesc.desc(), w, args1.cdesc.desc(), fwd_algo1, + &beta, args1.odesc.desc(), transformed_ddy_channel, + workspace_ptr, workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { wkspace_handle.RunFunc( [&](void* workspace_ptr) { @@ -1083,8 +1223,22 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { }, workspace_size); } +#endif } if (ddW) { +#ifdef PADDLE_WITH_HIP + // MIOPEN ONLY support beta to be 0.0f + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForward( + handle, &alpha, args2.idesc.desc(), x, args2.wdesc.desc(), + ddw, args2.cdesc.desc(), fwd_algo2, &beta, + args2.odesc.desc(), transformed_ddy_channel, + workspace_ptr, workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { wkspace_handle.RunFunc( [&](void* workspace_ptr) { @@ -1099,6 +1253,7 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { }, workspace_size); } +#endif } if (channel_last) { TransToChannelLast( @@ -1108,6 +1263,18 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { T* transformed_dy_channel = transformed_dO_channel.data(); if (dW && ddX) { ddx = transformed_ddX.data(); +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardWeights( + handle, &alpha, args3.odesc.desc(), transformed_dy_channel, + args3.idesc.desc(), ddx, args3.cdesc.desc(), filter_algo, + &beta, args3.wdesc.desc(), dw, workspace_ptr, + workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { wkspace_handle.RunFunc( [&](void* workspace_ptr) { @@ -1122,10 +1289,23 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { }, workspace_size); } +#endif } if (dX && ddW) { ddw = ddW->data(); +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardData( + handle, &alpha, args4.odesc.desc(), transformed_dy_channel, + args4.wdesc.desc(), ddw, args4.cdesc.desc(), data_algo, + &beta, args4.idesc.desc(), transformed_dx, workspace_ptr, + workspace_size)); + }, + workspace_size); +#else for (int i = 0; i < groups; i++) { wkspace_handle.RunFunc( [&](void* workspace_ptr) { @@ -1140,6 +1320,7 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { }, workspace_size); } +#endif if (!is_sys_pad) { // reverse padded input @@ -1170,6 +1351,34 @@ class CUDNNConvDoubleGradOpKernel : public framework::OpKernel { } // namespace paddle namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_KERNEL(conv2d, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvOpKernel, + paddle::operators::CUDNNConvOpKernel); +REGISTER_OP_KERNEL(conv2d_grad, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvGradOpKernel, + paddle::operators::CUDNNConvGradOpKernel); +REGISTER_OP_KERNEL( + conv2d_grad_grad, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvDoubleGradOpKernel, + paddle::operators::CUDNNConvDoubleGradOpKernel); + +REGISTER_OP_CUDA_KERNEL( + depthwise_conv2d_grad_grad, + paddle::operators::CUDNNConvDoubleGradOpKernel, + paddle::operators::CUDNNConvDoubleGradOpKernel); + +REGISTER_OP_KERNEL(conv3d, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvOpKernel, + paddle::operators::CUDNNConvOpKernel); +REGISTER_OP_KERNEL(conv3d_grad, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvGradOpKernel); +REGISTER_OP_KERNEL( + conv3d_grad_grad, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvDoubleGradOpKernel, + paddle::operators::CUDNNConvDoubleGradOpKernel); +#else REGISTER_OP_KERNEL(conv2d, CUDNN, plat::CUDAPlace, paddle::operators::CUDNNConvOpKernel, paddle::operators::CUDNNConvOpKernel, @@ -1202,3 +1411,4 @@ REGISTER_OP_KERNEL( paddle::operators::CUDNNConvDoubleGradOpKernel, paddle::operators::CUDNNConvDoubleGradOpKernel, paddle::operators::CUDNNConvDoubleGradOpKernel); +#endif diff --git a/paddle/fluid/operators/conv_cudnn_op_cache.h b/paddle/fluid/operators/conv_cudnn_op_cache.h index de883580dc026..ddddb7f8641ba 100644 --- a/paddle/fluid/operators/conv_cudnn_op_cache.h +++ b/paddle/fluid/operators/conv_cudnn_op_cache.h @@ -18,7 +18,11 @@ limitations under the License. */ #include #include #include "paddle/fluid/framework/operator.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif DECLARE_uint64(conv_workspace_size_limit); DECLARE_bool(cudnn_exhaustive_search); @@ -26,8 +30,11 @@ DECLARE_int64(cudnn_exhaustive_search_times); namespace paddle { namespace operators { - -#if CUDNN_VERSION_MIN(6, 0, 5) +#ifdef PADDLE_WITH_HIP +static constexpr size_t kNUM_CUDNN_FWD_ALGS = 1; +static constexpr size_t kNUM_CUDNN_BWD_FILTER_ALGS = 1; +static constexpr size_t kNUM_CUDNN_BWD_DATA_ALGS = 1; +#elif CUDNN_VERSION_MIN(6, 0, 5) static constexpr size_t kNUM_CUDNN_FWD_ALGS = CUDNN_CONVOLUTION_FWD_ALGO_COUNT; static constexpr size_t kNUM_CUDNN_BWD_FILTER_ALGS = CUDNN_CONVOLUTION_BWD_FILTER_ALGO_COUNT; diff --git a/paddle/fluid/operators/conv_miopen_helper.h b/paddle/fluid/operators/conv_miopen_helper.h new file mode 100644 index 0000000000000..3ab27e1ec4f4f --- /dev/null +++ b/paddle/fluid/operators/conv_miopen_helper.h @@ -0,0 +1,308 @@ +/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. + +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. */ + +#pragma once + +#include +#include +#include +#include +#include + +#include "paddle/fluid/framework/conv_search_cache.h" +#include "paddle/fluid/framework/operator_kernel_configs.h" +#include "paddle/fluid/operators/conv_cudnn_op_cache.h" +#include "paddle/fluid/platform/miopen_desc.h" + +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; +using DataLayout = platform::DataLayout; +template +using ScalingParamType = typename platform::CudnnDataType::ScalingParamType; +using framework::AlgorithmsCache; +static inline void GetNCDHW(const framework::DDim& dims, + const DataLayout& layout, int* N, int* C, int* D, + int* H, int* W) { + *N = dims[0]; + *C = layout == DataLayout::kNCHW ? dims[1] : dims[dims.size() - 1]; + int i = layout == DataLayout::kNCHW ? 0 : 1; + if (dims.size() == 5) { + *D = dims[2 - i]; + *H = dims[3 - i]; + *W = dims[4 - i]; + } else { + *D = 1; + *H = dims[2 - i]; + *W = dims[3 - i]; + } +} + +template +static void RemovePaddingSlice(const framework::ExecutionContext& context, + const Tensor* input, Tensor* out, + const std::vector& starts, + const std::vector& axes) { + auto& place = + *context.template device_context().eigen_device(); + auto in_dims = input->dims(); + auto new_out_dims = out->dims(); + auto offsets = Eigen::array(); + auto extents = Eigen::array(); + for (size_t i = 0; i < D; ++i) { + offsets[i] = 0; + extents[i] = new_out_dims[i]; + } + + int start; + for (size_t i = 0; i < axes.size(); ++i) { + start = starts[i]; + if (start < 0) { + start = (start + in_dims[axes[i]]); + } + start = std::max(start, 0); + offsets[axes[i]] = start; + } + auto in_t = + framework::EigenTensor::From( + *input); + + auto out_t = + framework::EigenTensor::From( + *out, new_out_dims); + out_t.device(place) = in_t.slice(offsets, extents); +} + +template +std::ostream& operator<<(std::ostream& out, const std::vector& v) { + out << "["; + for (auto const& tmp : v) out << tmp << ","; + out << "]"; + return out; +} + +using framework::ConvSearchCache; + +struct ConvArgs { + miopenHandle_t handle; + platform::TensorDescriptor idesc, odesc; + platform::FilterDescriptor wdesc; + platform::ConvolutionDescriptor cdesc; + const framework::Tensor *x, *w, *o; + miopenDataType_t cudnn_dtype; + + // strides + std::vector s; + // paddings + std::vector p; + // dilations + std::vector d; + + ConvArgs(const framework::Tensor* x, const framework::Tensor* w, + const framework::Tensor* o, const std::vector s, + const std::vector p, const std::vector d, + miopenDataType_t dtype) + : x(x), w(w), o(o), s(s), p(p), d(d), cudnn_dtype(dtype) {} +}; + +template +struct SearchAlgorithm {}; + +template <> +struct SearchAlgorithm { + using perf_t = miopenConvAlgoPerf_t; + using algo_t = miopenConvFwdAlgorithm_t; + + template + static algo_t Find(const ConvArgs& args, bool exhaustive_search, + bool deterministic, size_t workspace_size, + const framework::ExecutionContext& ctx) { + algo_t algo; + + auto& dev_ctx = ctx.template device_context(); + auto workspace_handle = dev_ctx.cudnn_workspace_handle(); + + int find_count; + miopenConvAlgoPerf_t find_result; + auto cudnn_find_func = [&](void* cudnn_workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenFindConvolutionForwardAlgorithm( + args.handle, args.idesc.desc(), args.x->data(), + args.wdesc.desc(), args.w->data(), args.cdesc.desc(), + args.odesc.desc(), const_cast(args.o->data()), + kNUM_CUDNN_FWD_ALGS, &find_count, &find_result, + cudnn_workspace_ptr, workspace_size, false)); + }; + + if (!exhaustive_search && !deterministic) { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + algo = find_result.fwd_algo; + } else { + auto& temp = ctx.cuda_device_context(); + AlgorithmsCache& algo_cache = + *(framework::ConvSearchCache::Instance().GetForward()); + + auto x_dims = framework::vectorize(args.x->dims()); + auto w_dims = framework::vectorize(args.w->dims()); + + VLOG(10) << "miopenConvolutionFwdAlgoPerf_t:" + << ", x_dims:" << x_dims << ", w_dims:" << w_dims << ", args.s" + << args.s << ", args.p" << args.p << ", args.d" << args.d; + + algo = algo_cache.GetAlgorithm( + x_dims, w_dims, args.s, args.p, args.d, 0, + static_cast(args.cudnn_dtype), [&]() { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + return find_result.fwd_algo; + }); + } + VLOG(3) << "choose algo " << algo; + return algo; + } + + static size_t GetWorkspaceSize(const ConvArgs& args) { + size_t workspace_size = 0; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForwardGetWorkSpaceSize( + args.handle, args.wdesc.desc(), args.idesc.desc(), + args.cdesc.desc(), args.odesc.desc(), &workspace_size)); + return workspace_size; + } +}; + +template <> +struct SearchAlgorithm { + using perf_t = miopenConvAlgoPerf_t; + using algo_t = miopenConvBwdDataAlgorithm_t; + + template + static algo_t Find(const ConvArgs& args, bool exhaustive_search, + bool deterministic, size_t workspace_size, + const framework::ExecutionContext& ctx) { + algo_t algo; + + auto& dev_ctx = ctx.template device_context(); + auto workspace_handle = dev_ctx.cudnn_workspace_handle(); + + int find_count; + miopenConvAlgoPerf_t find_result; + auto cudnn_find_func = [&](void* cudnn_workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenFindConvolutionBackwardDataAlgorithm( + args.handle, args.odesc.desc(), args.o->data(), + args.wdesc.desc(), args.w->data(), args.cdesc.desc(), + args.idesc.desc(), const_cast(args.x->data()), + kNUM_CUDNN_BWD_DATA_ALGS, &find_count, &find_result, + cudnn_workspace_ptr, workspace_size, false)); + }; + + if (!exhaustive_search && !deterministic) { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + algo = find_result.bwd_data_algo; + } else { + AlgorithmsCache& algo_cache = + *(framework::ConvSearchCache::Instance().GetBackwardData()); + + auto x_dims = framework::vectorize(args.x->dims()); + auto w_dims = framework::vectorize(args.w->dims()); + + VLOG(10) << "miopenConvolutionFwdAlgoPerf_t" + << ", x_dims:" << x_dims << ", w_dims:" << w_dims << ", args.s" + << args.s << ", args.p" << args.p << ", args.d" << args.d; + + algo = algo_cache.GetAlgorithm( + x_dims, w_dims, args.s, args.p, args.d, 0, + static_cast(args.cudnn_dtype), [&]() { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + return find_result.bwd_data_algo; + }); + } + VLOG(3) << "choose algo " << algo; + return algo; + } + + static size_t GetWorkspaceSize(const ConvArgs& args) { + size_t workspace_size = 0; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardDataGetWorkSpaceSize( + args.handle, args.odesc.desc(), args.wdesc.desc(), + args.cdesc.desc(), args.idesc.desc(), &workspace_size)); + return workspace_size; + } +}; + +template <> +struct SearchAlgorithm { + using perf_t = miopenConvAlgoPerf_t; + using algo_t = miopenConvBwdWeightsAlgorithm_t; + + template + static algo_t Find(const ConvArgs& args, bool exhaustive_search, + bool deterministic, size_t workspace_size, + const framework::ExecutionContext& ctx) { + algo_t algo; + + auto& dev_ctx = ctx.template device_context(); + auto workspace_handle = dev_ctx.cudnn_workspace_handle(); + + int find_count; + miopenConvAlgoPerf_t find_result; + auto cudnn_find_func = [&](void* cudnn_workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenFindConvolutionBackwardWeightsAlgorithm( + args.handle, args.odesc.desc(), args.o->data(), + args.idesc.desc(), args.x->data(), args.cdesc.desc(), + args.wdesc.desc(), const_cast(args.w->data()), + kNUM_CUDNN_BWD_FILTER_ALGS, &find_count, &find_result, + cudnn_workspace_ptr, workspace_size, false)); + }; + + if (!exhaustive_search && !deterministic) { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + algo = find_result.bwd_weights_algo; + } else { + AlgorithmsCache& algo_cache = + *(framework::ConvSearchCache::Instance().GetBackwardFilter()); + + auto x_dims = framework::vectorize(args.x->dims()); + auto w_dims = framework::vectorize(args.w->dims()); + + VLOG(10) << "miopenConvolutionFwdAlgoPerf_t:" + << ", x_dims:" << x_dims << ", w_dims:" << w_dims << ", args.s" + << args.s << ", args.p" << args.p << ", args.d" << args.d; + + algo = algo_cache.GetAlgorithm( + x_dims, w_dims, args.s, args.p, args.d, 0, + static_cast(args.cudnn_dtype), [&]() { + workspace_handle.RunFuncSync(cudnn_find_func, workspace_size); + return find_result.bwd_weights_algo; + }); + } + VLOG(3) << "choose algo " << algo; + return algo; + } + + static size_t GetWorkspaceSize(const ConvArgs& args) { + size_t workspace_size = 0; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardWeightsGetWorkSpaceSize( + args.handle, args.odesc.desc(), args.idesc.desc(), + args.cdesc.desc(), args.wdesc.desc(), &workspace_size)); + return workspace_size; + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/conv_op.cc b/paddle/fluid/operators/conv_op.cc index dd7bfbdaefeb2..85bb4e5baa058 100644 --- a/paddle/fluid/operators/conv_op.cc +++ b/paddle/fluid/operators/conv_op.cc @@ -21,9 +21,13 @@ limitations under the License. */ #include "paddle/fluid/framework/op_version_registry.h" #ifdef PADDLE_WITH_CUDA -#include "paddle/fluid/operators/conv_cudnn_op_cache.h" #include "paddle/fluid/platform/cudnn_helper.h" #endif + +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif + #ifdef PADDLE_WITH_MKLDNN #include "paddle/fluid/platform/mkldnn_helper.h" #endif @@ -149,7 +153,7 @@ framework::OpKernelType ConvOp::GetExpectedKernelType( "AnyLayout"; // todo enable data layout when it's ready framework::DataLayout layout = framework::StringToDataLayout(data_format); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library = framework::LibraryType::kCUDNN; } @@ -171,9 +175,14 @@ framework::OpKernelType ConvOp::GetExpectedKernelType( input_data_type != framework::proto::VarType::UINT8 && input_data_type != framework::proto::VarType::BF16) { auto filter_data_type = ctx.Input("Filter")->type(); - PADDLE_ENFORCE_EQ(input_data_type, filter_data_type, - platform::errors::InvalidArgument( - "input and filter data type should be consistent")); + PADDLE_ENFORCE_EQ( + input_data_type, filter_data_type, + platform::errors::InvalidArgument( + "input and filter data type should be consistent, " + "but received input data type is %s and filter type " + "is %s", + paddle::framework::DataTypeToString(input_data_type), + paddle::framework::DataTypeToString(filter_data_type))); } if (input_data_type == framework::proto::VarType::FP16) { PADDLE_ENFORCE_EQ(library, framework::LibraryType::kCUDNN, @@ -559,7 +568,7 @@ framework::OpKernelType ConvOpGrad::GetExpectedKernelType( framework::DataLayout layout_ = framework::StringToDataLayout(data_format); auto data_type = OperatorWithKernel::IndicateVarDataType(ctx, "Input"); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } @@ -744,7 +753,7 @@ framework::OpKernelType ConvOpDoubleGrad::GetExpectedKernelType( std::string data_format = "AnyLayout"; framework::DataLayout layout_ = framework::StringToDataLayout(data_format); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } diff --git a/paddle/fluid/operators/conv_transpose_cudnn_op.cu b/paddle/fluid/operators/conv_transpose_cudnn_op.cu index edf00eb2ba9a7..a712d31cf7e2c 100644 --- a/paddle/fluid/operators/conv_transpose_cudnn_op.cu +++ b/paddle/fluid/operators/conv_transpose_cudnn_op.cu @@ -15,11 +15,14 @@ limitations under the License. */ #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/memory/memory.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/operators/conv_miopen_helper.h" +#else #include "paddle/fluid/operators/conv_cudnn_helper.h" +#endif #include "paddle/fluid/operators/conv_transpose_op.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/math/padding.h" -#include "paddle/fluid/platform/cudnn_helper.h" namespace paddle { namespace operators { @@ -199,7 +202,7 @@ class CUDNNConvTransposeOpKernel : public framework::OpKernel { int iwo_groups = groups; int c_groups = 1; -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_HIP) || CUDNN_VERSION_MIN(7, 0, 1) iwo_groups = 1; c_groups = groups; groups = 1; @@ -212,7 +215,11 @@ class CUDNNConvTransposeOpKernel : public framework::OpKernel { } size_t workspace_size = 0; +#ifdef PADDLE_WITH_HIP + miopenConvBwdDataAlgorithm_t algo{}; +#else cudnnConvolutionBwdDataAlgo_t algo{}; +#endif // ------------------- cudnn conv algorithm --------------------- auto& dev_ctx = ctx.template device_context(); auto handle = dev_ctx.cudnn_handle(); @@ -235,10 +242,16 @@ class CUDNNConvTransposeOpKernel : public framework::OpKernel { args.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_groups); +#ifdef PADDLE_WITH_HIP + using search = SearchAlgorithm; + workspace_size = std::max(workspace_size, search::GetWorkspaceSize(args)); + algo = search::Find(args, false, deterministic, workspace_size, ctx); +#else using search = SearchAlgorithm; algo = search::Find(args, false, deterministic, ctx); workspace_size = std::max(workspace_size, search::GetWorkspaceSize(args, algo)); +#endif // ------------------- cudnn conv transpose forward --------------------- int input_offset = @@ -250,6 +263,17 @@ class CUDNNConvTransposeOpKernel : public framework::OpKernel { ScalingParamType beta = 0.0f; auto workspace_handle = dev_ctx.cudnn_workspace_handle(); for (int g = 0; g < groups; g++) { +#ifdef PADDLE_WITH_HIP + auto cudnn_func = [&](void* cudnn_workspace) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardData( + handle, &alpha, args.odesc.desc(), + input_data + input_offset * g, args.wdesc.desc(), + filter_data + filter_offset * g, args.cdesc.desc(), algo, &beta, + args.idesc.desc(), transformed_output_data + output_offset * g, + cudnn_workspace, workspace_size)); + }; +#else // PADDLE_WITH_HIP auto cudnn_func = [&](void* cudnn_workspace) { PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnConvolutionBackwardData( @@ -259,6 +283,7 @@ class CUDNNConvTransposeOpKernel : public framework::OpKernel { cudnn_workspace, workspace_size, &beta, args.idesc.desc(), transformed_output_data + output_offset * g)); }; +#endif // PADDLE_WITH_HIP workspace_handle.RunFunc(cudnn_func, workspace_size); } if (!is_sys_pad && strides.size() == 2U) { @@ -427,7 +452,7 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { int iwo_groups = groups; int c_groups = 1; -#if CUDNN_VERSION_MIN(7, 0, 1) +#if defined(PADDLE_WITH_HIP) || CUDNN_VERSION_MIN(7, 0, 1) iwo_groups = 1; c_groups = groups; groups = 1; @@ -449,8 +474,14 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { padding_common, dilations, dtype}; + +#ifdef PADDLE_WITH_HIP + miopenConvFwdAlgorithm_t data_algo{}; + miopenConvBwdWeightsAlgorithm_t filter_algo{}; +#else cudnnConvolutionFwdAlgo_t data_algo{}; cudnnConvolutionBwdFilterAlgo_t filter_algo{}; +#endif auto layout_tensor = GetCudnnTensorFormat(layout); size_t workspace_size = 0; @@ -472,10 +503,18 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { args1.odesc.set(input_transpose, iwo_groups); args1.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_groups); +#ifdef PADDLE_WITH_HIP + using search1 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search1::GetWorkspaceSize(args1)); + data_algo = + search1::Find(args1, false, deterministic, workspace_size, ctx); +#else using search1 = SearchAlgorithm; data_algo = search1::Find(args1, false, deterministic, ctx); workspace_size = std::max(workspace_size, search1::GetWorkspaceSize(args1, data_algo)); +#endif } if (filter_grad) { @@ -486,10 +525,18 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { args2.odesc.set(input_transpose, iwo_groups); args2.cdesc.set(dtype, padding_common, strides, dilations, platform::AllowTF32Cudnn(), c_groups); +#ifdef PADDLE_WITH_HIP + using search2 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search2::GetWorkspaceSize(args2)); + filter_algo = + search2::Find(args2, false, deterministic, workspace_size, ctx); +#else using search2 = SearchAlgorithm; filter_algo = search2::Find(args2, false, deterministic, ctx); workspace_size = std::max(workspace_size, search2::GetWorkspaceSize(args2, filter_algo)); +#endif } // ------------------- cudnn conv backward data --------------------- @@ -504,6 +551,18 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { if (input_grad) { // Because beta is zero, it is unnecessary to reset input_grad. for (int g = 0; g < groups; g++) { +#ifdef PADDLE_WITH_HIP + auto cudnn_func = [&](void* cudnn_workspace) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForward( + handle, &alpha, args1.idesc.desc(), + output_grad_data + output_grad_offset * g, args1.wdesc.desc(), + filter_data + filter_offset * g, args1.cdesc.desc(), + data_algo, &beta, args1.odesc.desc(), + input_grad_data + input_offset * g, cudnn_workspace, + workspace_size)); + }; +#else // PADDLE_WITH_HIP auto cudnn_func = [&](void* cudnn_workspace) { PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnConvolutionForward( @@ -513,6 +572,7 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { data_algo, cudnn_workspace, workspace_size, &beta, args1.odesc.desc(), input_grad_data + input_offset * g)); }; +#endif // PADDLE_WITH_HIP workspace_handle.RunFunc(cudnn_func, workspace_size); } @@ -540,6 +600,18 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { // Because beta is zero, it is unnecessary to reset filter_grad. // Gradient with respect to the filter for (int g = 0; g < groups; g++) { +#ifdef PADDLE_WITH_HIP + auto cudnn_func = [&](void* cudnn_workspace) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardWeights( + handle, &alpha, args2.odesc.desc(), + input_data + input_offset * g, args2.idesc.desc(), + output_grad_data + output_grad_offset * g, args2.cdesc.desc(), + filter_algo, &beta, args2.wdesc.desc(), + filter_grad_data + filter_offset * g, cudnn_workspace, + workspace_size)); + }; +#else // PADDLE_WITH_HIP auto cudnn_func = [&](void* cudnn_workspace) { PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnConvolutionBackwardFilter( @@ -549,6 +621,7 @@ class CUDNNConvTransposeGradOpKernel : public framework::OpKernel { filter_algo, cudnn_workspace, workspace_size, &beta, args2.wdesc.desc(), filter_grad_data + filter_offset * g)); }; +#endif // PADDLE_WITH_HIP workspace_handle.RunFunc(cudnn_func, workspace_size); } } @@ -840,7 +913,16 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { ConvArgs args4{ &transformed_dO, ddW, &transformed_dX_channel, strides, padding_common, dilations, dtype}; - +#ifdef PADDLE_WITH_HIP + miopenConvBwdDataAlgorithm_t bwd_algo1 = + static_cast(0); + miopenConvBwdDataAlgorithm_t bwd_algo2 = + static_cast(0); + miopenConvFwdAlgorithm_t data_algo = + static_cast(0); + miopenConvBwdWeightsAlgorithm_t filter_algo = + static_cast(0); +#else cudnnConvolutionBwdDataAlgo_t bwd_algo1 = static_cast(0); cudnnConvolutionBwdDataAlgo_t bwd_algo2 = @@ -849,6 +931,7 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { static_cast(0); cudnnConvolutionBwdFilterAlgo_t filter_algo = static_cast(0); +#endif auto layout = GetCudnnTensorFormat(platform::DataLayout::kNCHW); @@ -866,9 +949,16 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { args1.wdesc.set(*W, layout, iwo_group); args1.odesc.set(transformed_ddX, iwo_group); args1.cdesc.set(dtype, padding_common, strides, dilations, c_group); +#ifdef PADDLE_WITH_HIP + using search1 = SearchAlgorithm; + workspace_size = search1::GetWorkspaceSize(args1); + bwd_algo1 = + search1::Find(args1, false, deterministic, workspace_size, ctx); +#else using search1 = SearchAlgorithm; bwd_algo1 = search1::Find(args1, false, deterministic, ctx); workspace_size = search1::GetWorkspaceSize(args1, bwd_algo1); +#endif } if (ddW) { @@ -878,10 +968,18 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { args2.wdesc.set(*ddW, layout, iwo_group); args2.odesc.set(transformed_X, iwo_group); args2.cdesc.set(dtype, padding_common, strides, dilations, c_group); +#ifdef PADDLE_WITH_HIP + using search2 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search2::GetWorkspaceSize(args2)); + bwd_algo2 = + search2::Find(args2, false, deterministic, workspace_size, ctx); +#else using search2 = SearchAlgorithm; bwd_algo2 = search2::Find(args2, false, deterministic, ctx); workspace_size = std::max(workspace_size, search2::GetWorkspaceSize(args2, bwd_algo2)); +#endif } } @@ -894,11 +992,18 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { args3.odesc.set(transformed_ddX_channel, iwo_group); args3.cdesc.set(dtype, padding_common, strides, dilations, c_group); - +#ifdef PADDLE_WITH_HIP + using search3 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search3::GetWorkspaceSize(args3)); + filter_algo = + search3::Find(args3, false, deterministic, workspace_size, ctx); +#else using search3 = SearchAlgorithm; filter_algo = search3::Find(args3, false, deterministic, ctx); workspace_size = std::max(workspace_size, search3::GetWorkspaceSize(args3, filter_algo)); +#endif } if (ddW && dX) { @@ -909,11 +1014,18 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { args4.wdesc.set(*ddW, layout, iwo_group); args4.odesc.set(transformed_dX_channel, iwo_group); args4.cdesc.set(dtype, padding_common, strides, dilations, c_group); - +#ifdef PADDLE_WITH_HIP + using search4 = SearchAlgorithm; + workspace_size = + std::max(workspace_size, search4::GetWorkspaceSize(args4)); + data_algo = + search4::Find(args4, false, deterministic, workspace_size, ctx); +#else using search4 = SearchAlgorithm; data_algo = search4::Find(args4, false, deterministic, ctx); workspace_size = std::max(workspace_size, search4::GetWorkspaceSize(args4, data_algo)); +#endif } int i_n, i_c, i_d, i_h, i_w; @@ -939,6 +1051,20 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { if (ddX) { ddx = transformed_ddX.data(); for (int i = 0; i < groups; i++) { +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardData( + handle, &alpha, args1.odesc.desc(), + ddx + i * group_offset_in, args1.wdesc.desc(), + w + i * group_offset_filter, args1.cdesc.desc(), + bwd_algo1, &beta, args1.idesc.desc(), + transformed_ddy_channel + i * group_offset_out, + workspace_ptr, workspace_size)); + }, + workspace_size); +#else // PADDLE_WITH_HIP wkspace_handle.RunFunc( [&](void* workspace_ptr) { PADDLE_ENFORCE_CUDA_SUCCESS( @@ -951,10 +1077,25 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { transformed_ddy_channel + i * group_offset_out)); }, workspace_size); +#endif // PADDLE_WITH_HIP } } if (ddW) { for (int i = 0; i < groups; i++) { +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardData( + handle, &alpha, args2.odesc.desc(), + x + i * group_offset_in, args2.wdesc.desc(), + ddw + i * group_offset_filter, args2.cdesc.desc(), + bwd_algo2, &alpha, args2.idesc.desc(), + transformed_ddy_channel + i * group_offset_out, + workspace_ptr, workspace_size)); + }, + workspace_size); +#else // PADDLE_WITH_HIP wkspace_handle.RunFunc( [&](void* workspace_ptr) { PADDLE_ENFORCE_CUDA_SUCCESS( @@ -967,6 +1108,7 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { transformed_ddy_channel + i * group_offset_out)); }, workspace_size); +#endif // PADDLE_WITH_HIP } } if ((!is_sys_pad) && (!channel_last)) { @@ -997,6 +1139,20 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { if (dW && ddX) { ddx = transformed_ddX_channel.data(); for (int i = 0; i < groups; i++) { +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionBackwardWeights( + handle, &alpha, args3.odesc.desc(), + ddx + i * group_offset_in, args3.idesc.desc(), + transformed_dy_channel + i * group_offset_out, + args3.cdesc.desc(), filter_algo, &beta, + args3.wdesc.desc(), dw + i * group_offset_filter, + workspace_ptr, workspace_size)); + }, + workspace_size); +#else // PADDLE_WITH_HIP wkspace_handle.RunFunc( [&](void* workspace_ptr) { PADDLE_ENFORCE_CUDA_SUCCESS( @@ -1009,12 +1165,27 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { dw + i * group_offset_filter)); }, workspace_size); +#endif // PADDLE_WITH_HIP } } if (dX && ddW) { ddw = ddW->data(); for (int i = 0; i < groups; i++) { +#ifdef PADDLE_WITH_HIP + wkspace_handle.RunFunc( + [&](void* workspace_ptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenConvolutionForward( + handle, &alpha, args4.idesc.desc(), + transformed_dy_channel + i * group_offset_out, + args4.wdesc.desc(), ddw + i * group_offset_filter, + args4.cdesc.desc(), data_algo, &beta, args4.odesc.desc(), + transformed_dx + i * group_offset_in, workspace_ptr, + workspace_size)); + }, + workspace_size); +#else // PADDLE_WITH_HIP wkspace_handle.RunFunc( [&](void* workspace_ptr) { PADDLE_ENFORCE_CUDA_SUCCESS( @@ -1027,6 +1198,7 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { transformed_dx + i * group_offset_in)); }, workspace_size); +#endif // PADDLE_WITH_HIP } if (channel_last) { TransToChannelLast( @@ -1042,6 +1214,26 @@ class CUDNNConvTransposeDoubleGradOpKernel : public framework::OpKernel { namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_KERNEL(conv2d_transpose, CUDNN, ::paddle::platform::CUDAPlace, + ops::CUDNNConvTransposeOpKernel, + ops::CUDNNConvTransposeOpKernel); +REGISTER_OP_KERNEL(conv2d_transpose_grad, CUDNN, ::paddle::platform::CUDAPlace, + ops::CUDNNConvTransposeGradOpKernel, + ops::CUDNNConvTransposeGradOpKernel); +REGISTER_OP_KERNEL( + conv2d_transpose_grad_grad, CUDNN, plat::CUDAPlace, + paddle::operators::CUDNNConvTransposeDoubleGradOpKernel, + paddle::operators::CUDNNConvTransposeDoubleGradOpKernel); + +REGISTER_OP_KERNEL(conv3d_transpose, CUDNN, ::paddle::platform::CUDAPlace, + ops::CUDNNConvTransposeOpKernel, + ops::CUDNNConvTransposeOpKernel); +REGISTER_OP_KERNEL(conv3d_transpose_grad, CUDNN, ::paddle::platform::CUDAPlace, + ops::CUDNNConvTransposeGradOpKernel, + ops::CUDNNConvTransposeGradOpKernel); +#else REGISTER_OP_KERNEL(conv2d_transpose, CUDNN, ::paddle::platform::CUDAPlace, ops::CUDNNConvTransposeOpKernel, ops::CUDNNConvTransposeOpKernel, @@ -1064,3 +1256,4 @@ REGISTER_OP_KERNEL(conv3d_transpose_grad, CUDNN, ::paddle::platform::CUDAPlace, ops::CUDNNConvTransposeGradOpKernel, ops::CUDNNConvTransposeGradOpKernel, ops::CUDNNConvTransposeGradOpKernel); +#endif diff --git a/paddle/fluid/operators/conv_transpose_op.cc b/paddle/fluid/operators/conv_transpose_op.cc index dc4b416a609ae..4ea936d5104b8 100644 --- a/paddle/fluid/operators/conv_transpose_op.cc +++ b/paddle/fluid/operators/conv_transpose_op.cc @@ -183,7 +183,7 @@ framework::OpKernelType ConvTransposeOp::GetExpectedKernelType( bool use_cudnn = ctx.Attr("use_cudnn"); use_cudnn &= platform::is_gpu_place(ctx.GetPlace()); auto data_type = OperatorWithKernel::IndicateVarDataType(ctx, "Input"); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { auto& dev_ctx = ctx.template device_context(); use_cudnn &= dev_ctx.cudnn_handle() != nullptr; @@ -481,7 +481,7 @@ framework::OpKernelType ConvTransposeOpGrad::GetExpectedKernelType( const framework::ExecutionContext& ctx) const { bool use_cudnn = ctx.Attr("use_cudnn"); use_cudnn &= platform::is_gpu_place(ctx.GetPlace()); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { auto& dev_ctx = ctx.template device_context(); use_cudnn &= dev_ctx.cudnn_handle() != nullptr; @@ -581,7 +581,7 @@ framework::OpKernelType ConvTransposeOpDoubleGrad::GetExpectedKernelType( const framework::ExecutionContext& ctx) const { bool use_cudnn = ctx.Attr("use_cudnn"); use_cudnn &= platform::is_gpu_place(ctx.GetPlace()); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { auto& dev_ctx = ctx.template device_context(); use_cudnn &= dev_ctx.cudnn_handle() != nullptr; diff --git a/paddle/fluid/operators/conv_transpose_op.h b/paddle/fluid/operators/conv_transpose_op.h index 651719f105280..ecf5b6d774a26 100644 --- a/paddle/fluid/operators/conv_transpose_op.h +++ b/paddle/fluid/operators/conv_transpose_op.h @@ -682,9 +682,9 @@ class DepthwiseConvTransposeGradKernel : public framework::OpKernel { if (input_grad) { math::DepthwiseConvFunctor depthwiseConv; depthwiseConv( - dev_ctx, *output_grad, filter, strides, paddings, + dev_ctx, *output_grad, filter, strides, std::vector{paddings[0], paddings[2], paddings[1], paddings[3]}, - input_grad, data_layout); + dilations, input_grad, data_layout); } if (filter_grad) { diff --git a/paddle/fluid/operators/correlation_op.cu b/paddle/fluid/operators/correlation_op.cu index 6cf1ff5e72840..a51fce8132418 100644 --- a/paddle/fluid/operators/correlation_op.cu +++ b/paddle/fluid/operators/correlation_op.cu @@ -12,6 +12,9 @@ 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 PADDLE_WITH_HIP +// HIP not supported yet + #include #include #include "paddle/fluid/framework/op_registry.h" @@ -480,3 +483,5 @@ REGISTER_OP_CUDA_KERNEL(correlation, ops::CorrelationCUDAKernel, ops::CorrelationCUDAKernel); REGISTER_OP_CUDA_KERNEL(correlation_grad, ops::CorrelationCUDAGradKernel, ops::CorrelationCUDAGradKernel); + +#endif // not PADDLE_WITH_HIP diff --git a/paddle/fluid/operators/cudnn_lstm_op.cu.cc b/paddle/fluid/operators/cudnn_lstm_op.cu.cc index e935a3c0aac13..27f64b41948be 100644 --- a/paddle/fluid/operators/cudnn_lstm_op.cu.cc +++ b/paddle/fluid/operators/cudnn_lstm_op.cu.cc @@ -14,9 +14,14 @@ limitations under the License. */ #include "paddle/fluid/framework/generator.h" #include "paddle/fluid/framework/op_registry.h" -#include "paddle/fluid/operators/cudnn_lstm_cache.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/utils.h" +#ifdef PADDLE_WITH_CUDA +#include "paddle/fluid/operators/cudnn_lstm_cache.h" +#endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/operators/miopen_lstm_cache.h" +#endif namespace paddle { namespace platform { @@ -54,7 +59,7 @@ int size_sum(const std::vector &weight_list) { } template -void weight_to_tensor(const platform::Place &place, cudaStream_t stream, +void weight_to_tensor(const platform::Place &place, gpuStream_t stream, const std::vector &weight_list, Tensor *weight) { auto weight_data = weight->data(); @@ -72,7 +77,7 @@ void weight_to_tensor(const platform::Place &place, cudaStream_t stream, } template -void weight_to_tensor_list(const platform::Place &place, cudaStream_t stream, +void weight_to_tensor_list(const platform::Place &place, gpuStream_t stream, std::vector *weight_grad, const std::vector &weight_input, const Tensor *weight) { @@ -92,23 +97,36 @@ void weight_to_tensor_list(const platform::Place &place, cudaStream_t stream, } template +#ifdef PADDLE_WITH_HIP +void LSTMInferece(const bool &has_seq_length, const miopenHandle_t &handle, +#else void LSTMInferece(const bool &has_seq_length, const cudnnHandle_t &handle, +#endif const int &seq_length, ScopedRNNBase *rnn, const T *x_data, const T *init_h_data, const T *init_c_data, const T *w_data, T *out_data, T *last_h_data, T *last_c_data, framework::Tensor *workspace_data, const size_t &workspace_size) { if (!has_seq_length) { - // for inference - // This interface is used when the input/output is unpadded. +// for inference +// This interface is used when the input/output is unpadded. +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNForwardInference( + handle, rnn->rnn_desc(), seq_length, rnn->x_descs(), x_data, + rnn->init_h_desc(), init_h_data, rnn->init_c_desc(), init_c_data, + rnn->weight_desc(), w_data, rnn->y_descs(), out_data, + rnn->last_h_desc(), last_h_data, rnn->last_c_desc(), last_c_data, + workspace_data->data(), workspace_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardInference( handle, rnn->rnn_desc(), seq_length, rnn->x_descs(), x_data, rnn->init_h_desc(), init_h_data, rnn->init_c_desc(), init_c_data, rnn->weight_desc(), w_data, rnn->y_descs(), out_data, rnn->last_h_desc(), last_h_data, rnn->last_c_desc(), last_c_data, workspace_data->data(), workspace_size)); +#endif } else { -#if CUDNN_VERSION >= 7201 +#if !defined(PADDLE_WITH_HIP) && CUDNN_VERSION >= 7201 // for inference // This interface is used when the input/output is padded. PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardInferenceEx( @@ -256,8 +274,17 @@ class CudnnLSTMGPUKernel : public framework::OpKernel { last_c_data, &workspace_data_, workspace_size); } else { if (!has_seq_length) { - // for train - // This interface is used when the input/output is unpadded. +// for train +// This interface is used when the input/output is unpadded. +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNForwardTraining( + handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), x_data, + rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, + rnn.weight_desc(), w_data, rnn.y_descs(), out_data, + rnn.last_h_desc(), last_h_data, rnn.last_c_desc(), last_c_data, + workspace_data_.data(), workspace_size, reserve_data, + reserve_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardTraining( handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), x_data, rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, @@ -265,8 +292,9 @@ class CudnnLSTMGPUKernel : public framework::OpKernel { rnn.last_h_desc(), last_h_data, rnn.last_c_desc(), last_c_data, workspace_data_.data(), workspace_size, reserve_data, reserve_size)); +#endif } else { -#if CUDNN_VERSION >= 7201 +#if !defined(PADDLE_WITH_HIP) && CUDNN_VERSION >= 7201 // for train // This interface is used when the input/output is padded. PADDLE_ENFORCE_CUDA_SUCCESS( @@ -403,7 +431,23 @@ class CudnnLSTMGPUGradKernel : public framework::OpKernel { const uint8_t *reserve_data = reserve->data(); if (!has_seq_length) { - // This interface is used when the input/output is unpadded. +// This interface is used when the input/output is unpadded. +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNBackwardData( + handle, rnn.rnn_desc(), seq_length, rnn.y_descs(), out_data, + rnn.y_descs(), out_grad_data, rnn.last_h_desc(), last_h_grad_data, + rnn.last_c_desc(), last_c_grad_data, rnn.weight_desc(), weight_data, + rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, + rnn.x_descs(), in_grad_data, rnn.init_h_desc(), init_h_grad_data, + rnn.init_c_desc(), init_c_grad_data, workspace_data_.data(), + workspace_size, const_cast(reserve_data), reserve_size)); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNBackwardWeights( + handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), input->data(), + rnn.init_h_desc(), init_h->data(), rnn.y_descs(), out->data(), + rnn.weight_desc(), weight_grad_data, workspace_data_.data(), + workspace_size, const_cast(reserve_data), reserve_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNBackwardData( handle, rnn.rnn_desc(), seq_length, rnn.y_descs(), out_data, rnn.y_descs(), out_grad_data, rnn.last_h_desc(), last_h_grad_data, @@ -418,8 +462,9 @@ class CudnnLSTMGPUGradKernel : public framework::OpKernel { rnn.init_h_desc(), init_h->data(), rnn.y_descs(), out->data(), workspace_data_.data(), workspace_size, rnn.weight_desc(), weight_grad_data, const_cast(reserve_data), reserve_size)); +#endif } else { -#if CUDNN_VERSION >= 7201 +#if !defined(PADDLE_WITH_HIP) && CUDNN_VERSION >= 7201 // for train // This interface is used when the input/output is padded. PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNBackwardDataEx( @@ -452,7 +497,13 @@ class CudnnLSTMGPUGradKernel : public framework::OpKernel { } // namespace paddle namespace ops = paddle::operators; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL(cudnn_lstm, ops::CudnnLSTMGPUKernel); +REGISTER_OP_CUDA_KERNEL(cudnn_lstm_grad, ops::CudnnLSTMGPUGradKernel); +#else REGISTER_OP_CUDA_KERNEL(cudnn_lstm, ops::CudnnLSTMGPUKernel, ops::CudnnLSTMGPUKernel); REGISTER_OP_CUDA_KERNEL(cudnn_lstm_grad, ops::CudnnLSTMGPUGradKernel, ops::CudnnLSTMGPUGradKernel); +#endif diff --git a/paddle/fluid/operators/cumsum_op.cu b/paddle/fluid/operators/cumsum_op.cu index f75eb7fd9670f..854be76f24e98 100644 --- a/paddle/fluid/operators/cumsum_op.cu +++ b/paddle/fluid/operators/cumsum_op.cu @@ -16,7 +16,13 @@ limitations under the License. */ #include #include #include -#include "cub/cub.cuh" +#ifdef __NVCC__ +#include +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/cum_op.h" #include "paddle/fluid/platform/gpu_launch_config.h" diff --git a/paddle/fluid/operators/data_norm_op.cu b/paddle/fluid/operators/data_norm_op.cu index 9e284b1dcdaae..1043faa56f01b 100644 --- a/paddle/fluid/operators/data_norm_op.cu +++ b/paddle/fluid/operators/data_norm_op.cu @@ -17,7 +17,7 @@ limitations under the License. */ #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/operators/data_norm_op.h" #include "paddle/fluid/platform/cuda_primitives.h" -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/collective_helper.h" #include "paddle/fluid/platform/nccl_helper.h" #endif @@ -174,7 +174,7 @@ class DataNormGradKernel d_batch_sum, d_batch_square_sum); if (need_sync_stats) { -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) auto comm = platform::NCCLCommContext::Instance().Get(0, ctx.GetPlace()); PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::ncclAllReduce( reinterpret_cast(d_batch_size), @@ -188,7 +188,11 @@ class DataNormGradKernel reinterpret_cast(d_batch_square_sum), reinterpret_cast(d_batch_square_sum), C, platform::ToNCCLDataType(x->type()), ncclSum, comm->comm(), stream)); +#ifdef PADDLE_WITH_RCCL + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif #else PADDLE_THROW(platform::errors::PreconditionNotMet( "PaddlePaddle should compile with GPU, and need_sync_stats connot be " diff --git a/paddle/fluid/operators/detection/bbox_util.cu.h b/paddle/fluid/operators/detection/bbox_util.cu.h index 0d52fd4161382..725983f8153e4 100644 --- a/paddle/fluid/operators/detection/bbox_util.cu.h +++ b/paddle/fluid/operators/detection/bbox_util.cu.h @@ -23,6 +23,7 @@ limitations under the License. */ #ifdef __HIPCC__ #include #include "paddle/fluid/platform/miopen_helper.h" +namespace cub = hipcub; #endif #include "paddle/fluid/operators/gather.cu.h" #include "paddle/fluid/operators/math/math_function.h" @@ -64,27 +65,17 @@ static void SortDescending(const platform::CUDADeviceContext &ctx, // Determine temporary device storage requirements size_t temp_storage_bytes = 0; -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairsDescending( - nullptr, temp_storage_bytes, keys_in, keys_out, idx_in, idx_out, num); -#else cub::DeviceRadixSort::SortPairsDescending( - nullptr, temp_storage_bytes, keys_in, keys_out, idx_in, idx_out, num); -#endif + nullptr, temp_storage_bytes, keys_in, keys_out, idx_in, idx_out, num, 0, + sizeof(T) * 8, ctx.stream()); // Allocate temporary storage auto place = BOOST_GET_CONST(platform::CUDAPlace, ctx.GetPlace()); auto d_temp_storage = memory::Alloc(place, temp_storage_bytes); -// Run sorting operation -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairsDescending( - d_temp_storage->ptr(), temp_storage_bytes, keys_in, keys_out, idx_in, - idx_out, num); -#else + // Run sorting operation cub::DeviceRadixSort::SortPairsDescending( d_temp_storage->ptr(), temp_storage_bytes, keys_in, keys_out, idx_in, - idx_out, num); -#endif + idx_out, num, 0, sizeof(T) * 8, ctx.stream()); } template @@ -285,15 +276,19 @@ static void NMS(const platform::CUDADeviceContext &ctx, const Tensor &proposals, const T *boxes = proposals.data(); auto place = BOOST_GET_CONST(platform::CUDAPlace, ctx.GetPlace()); - framework::Vector mask(boxes_num * col_blocks); - NMSKernel<<>>(boxes_num, nms_threshold, boxes, - mask.CUDAMutableData(BOOST_GET_CONST( - platform::CUDAPlace, ctx.GetPlace())), - pixel_offset); + auto mask_ptr = memory::Alloc(ctx, boxes_num * col_blocks * sizeof(uint64_t)); + uint64_t *mask_dev = reinterpret_cast(mask_ptr->ptr()); + + NMSKernel<<>>( + boxes_num, nms_threshold, boxes, mask_dev, pixel_offset); std::vector remv(col_blocks); memset(&remv[0], 0, sizeof(uint64_t) * col_blocks); + std::vector mask_host(boxes_num * col_blocks); + memory::Copy(platform::CPUPlace(), mask_host.data(), place, mask_dev, + boxes_num * col_blocks * sizeof(uint64_t), ctx.stream()); + std::vector keep_vec; int num_to_keep = 0; for (int i = 0; i < boxes_num; i++) { @@ -303,7 +298,7 @@ static void NMS(const platform::CUDADeviceContext &ctx, const Tensor &proposals, if (!(remv[nblock] & (1ULL << inblock))) { ++num_to_keep; keep_vec.push_back(i); - uint64_t *p = &mask[0] + i * col_blocks; + uint64_t *p = mask_host.data() + i * col_blocks; for (int j = nblock; j < col_blocks; j++) { remv[j] |= p[j]; } diff --git a/paddle/fluid/operators/detection/collect_fpn_proposals_op.cu b/paddle/fluid/operators/detection/collect_fpn_proposals_op.cu index 4bb0f9ca67fb2..ffd9ac6b2af80 100644 --- a/paddle/fluid/operators/detection/collect_fpn_proposals_op.cu +++ b/paddle/fluid/operators/detection/collect_fpn_proposals_op.cu @@ -14,6 +14,7 @@ limitations under the License. */ #endif #ifdef __HIPCC__ #include +namespace cub = hipcub; #endif #include @@ -141,29 +142,18 @@ class GPUCollectFpnProposalsOpKernel : public framework::OpKernel { // Determine temporary device storage requirements size_t temp_storage_bytes = 0; -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairsDescending( - nullptr, temp_storage_bytes, concat_scores.data(), keys_out, idx_in, - idx_out, total_roi_num); -#else cub::DeviceRadixSort::SortPairsDescending( nullptr, temp_storage_bytes, concat_scores.data(), keys_out, idx_in, - idx_out, total_roi_num); -#endif + idx_out, total_roi_num, 0, sizeof(T) * 8, dev_ctx.stream()); // Allocate temporary storage auto d_temp_storage = memory::Alloc(place, temp_storage_bytes); -// Run sorting operation -// sort score to get corresponding index -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairsDescending( - d_temp_storage->ptr(), temp_storage_bytes, concat_scores.data(), - keys_out, idx_in, idx_out, total_roi_num); -#else + // Run sorting operation + // sort score to get corresponding index cub::DeviceRadixSort::SortPairsDescending( d_temp_storage->ptr(), temp_storage_bytes, concat_scores.data(), - keys_out, idx_in, idx_out, total_roi_num); -#endif + keys_out, idx_in, idx_out, total_roi_num, 0, sizeof(T) * 8, + dev_ctx.stream()); index_out_t.Resize({real_post_num}); Tensor sorted_rois; sorted_rois.mutable_data({real_post_num, kBBoxSize}, dev_ctx.GetPlace()); @@ -185,29 +175,19 @@ class GPUCollectFpnProposalsOpKernel : public framework::OpKernel { out_id_t.mutable_data({real_post_num}, dev_ctx.GetPlace()); // Determine temporary device storage requirements temp_storage_bytes = 0; -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairs( - nullptr, temp_storage_bytes, sorted_batch_id.data(), out_id_data, - batch_idx_in, index_out_t.data(), real_post_num); -#else cub::DeviceRadixSort::SortPairs( nullptr, temp_storage_bytes, sorted_batch_id.data(), out_id_data, - batch_idx_in, index_out_t.data(), real_post_num); -#endif + batch_idx_in, index_out_t.data(), real_post_num, 0, + sizeof(int) * 8, dev_ctx.stream()); // Allocate temporary storage d_temp_storage = memory::Alloc(place, temp_storage_bytes); -// Run sorting operation -// sort batch_id to get corresponding index -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairs( - d_temp_storage->ptr(), temp_storage_bytes, sorted_batch_id.data(), - out_id_data, batch_idx_in, index_out_t.data(), real_post_num); -#else + // Run sorting operation + // sort batch_id to get corresponding index cub::DeviceRadixSort::SortPairs( d_temp_storage->ptr(), temp_storage_bytes, sorted_batch_id.data(), - out_id_data, batch_idx_in, index_out_t.data(), real_post_num); -#endif + out_id_data, batch_idx_in, index_out_t.data(), real_post_num, 0, + sizeof(int) * 8, dev_ctx.stream()); GPUGather(dev_ctx, sorted_rois, index_out_t, fpn_rois); @@ -221,8 +201,8 @@ class GPUCollectFpnProposalsOpKernel : public framework::OpKernel { int threads = kNumCUDAThreads; // get length-based lod by batch ids - GetLengthLoD<<>>(real_post_num, out_id_data, - length_lod_data); + GetLengthLoD<<>>( + real_post_num, out_id_data, length_lod_data); std::vector length_lod_cpu(lod_size); memory::Copy(platform::CPUPlace(), length_lod_cpu.data(), place, length_lod_data, sizeof(int) * lod_size, dev_ctx.stream()); diff --git a/paddle/fluid/operators/detection/distribute_fpn_proposals_op.cu b/paddle/fluid/operators/detection/distribute_fpn_proposals_op.cu index 63f205947d9b5..7ccb354e1773a 100644 --- a/paddle/fluid/operators/detection/distribute_fpn_proposals_op.cu +++ b/paddle/fluid/operators/detection/distribute_fpn_proposals_op.cu @@ -17,6 +17,7 @@ limitations under the License. */ #endif #ifdef __HIPCC__ #include +namespace cub = hipcub; #endif #include @@ -130,11 +131,10 @@ class GPUDistributeFpnProposalsOpKernel : public framework::OpKernel { int dist_blocks = NumBlocks(roi_num); int threads = kNumCUDAThreads; // get target levels and sub_lod list - GPUDistFpnProposalsHelper<<>>( + GPUDistFpnProposalsHelper<<>>( roi_num, fpn_rois->data(), lod_size, refer_level, refer_scale, max_level, min_level, roi_batch_id_list_gpu.data(), sub_lod_list_data, target_lvls_data, pixel_offset); - dev_ctx.Wait(); auto place = BOOST_GET_CONST(platform::CUDAPlace, dev_ctx.GetPlace()); Tensor index_in_t; @@ -149,57 +149,40 @@ class GPUDistributeFpnProposalsOpKernel : public framework::OpKernel { // Determine temporary device storage requirements size_t temp_storage_bytes = 0; -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairs(nullptr, temp_storage_bytes, - target_lvls_data, keys_out, - idx_in, idx_out, roi_num); -#else - cub::DeviceRadixSort::SortPairs(nullptr, temp_storage_bytes, - target_lvls_data, keys_out, - idx_in, idx_out, roi_num); -#endif + cub::DeviceRadixSort::SortPairs( + nullptr, temp_storage_bytes, target_lvls_data, keys_out, idx_in, + idx_out, roi_num, 0, sizeof(int) * 8, dev_ctx.stream()); // Allocate temporary storage auto d_temp_storage = memory::Alloc(place, temp_storage_bytes); -// Run sorting operation -// sort target level to get corresponding index -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairs( - d_temp_storage->ptr(), temp_storage_bytes, target_lvls_data, keys_out, - idx_in, idx_out, roi_num); -#else + // Run sorting operation + // sort target level to get corresponding index cub::DeviceRadixSort::SortPairs( d_temp_storage->ptr(), temp_storage_bytes, target_lvls_data, keys_out, - idx_in, idx_out, roi_num); -#endif + idx_in, idx_out, roi_num, 0, sizeof(int) * 8, dev_ctx.stream()); int* restore_idx_data = restore_index->mutable_data({roi_num, 1}, dev_ctx.GetPlace()); -// sort current index to get restore index -#ifdef PADDLE_WITH_HIP - hipcub::DeviceRadixSort::SortPairs( - d_temp_storage->ptr(), temp_storage_bytes, idx_out, keys_out, idx_in, - restore_idx_data, roi_num); -#else + // sort current index to get restore index cub::DeviceRadixSort::SortPairs( d_temp_storage->ptr(), temp_storage_bytes, idx_out, keys_out, idx_in, - restore_idx_data, roi_num); -#endif + restore_idx_data, roi_num, 0, sizeof(int) * 8, dev_ctx.stream()); int start = 0; auto multi_rois_num = ctx.MultiOutput("MultiLevelRoIsNum"); + std::vector sub_lod_list_cpu(lod_size * num_level); + memory::Copy(platform::CPUPlace(), sub_lod_list_cpu.data(), place, + sub_lod_list_data, sizeof(int) * lod_size * num_level, + dev_ctx.stream()); + dev_ctx.Wait(); + for (int i = 0; i < num_level; ++i) { Tensor sub_lod = sub_lod_list.Slice(i, i + 1); - int* sub_lod_data = sub_lod.data(); // transfer length-based lod to offset-based lod std::vector offset(1, 0); - std::vector sub_lod_cpu(lod_size); - memory::Copy(platform::CPUPlace(), sub_lod_cpu.data(), place, - sub_lod_data, sizeof(int) * lod_size, dev_ctx.stream()); - dev_ctx.Wait(); for (int j = 0; j < lod_size; ++j) { - offset.emplace_back(offset.back() + sub_lod_cpu[j]); + offset.emplace_back(offset.back() + sub_lod_list_cpu[i * lod_size + j]); } int sub_rois_num = offset.back(); diff --git a/paddle/fluid/operators/detection/generate_proposals_op.cu b/paddle/fluid/operators/detection/generate_proposals_op.cu index 8359fbab519b3..e8ab628db16bd 100644 --- a/paddle/fluid/operators/detection/generate_proposals_op.cu +++ b/paddle/fluid/operators/detection/generate_proposals_op.cu @@ -198,7 +198,6 @@ class CUDAGenerateProposalsKernel : public framework::OpKernel { memory::Copy(place, rpn_roi_probs_data + num_proposals, place, scores.data(), sizeof(T) * scores.numel(), dev_ctx.stream()); - dev_ctx.Wait(); num_proposals += proposals.dims()[0]; offset.emplace_back(num_proposals); tmp_num.push_back(proposals.dims()[0]); diff --git a/paddle/fluid/operators/diag_embed_op.h b/paddle/fluid/operators/diag_embed_op.h index 8c4c68fb1ffa5..aff7d7e48a8d4 100644 --- a/paddle/fluid/operators/diag_embed_op.h +++ b/paddle/fluid/operators/diag_embed_op.h @@ -100,7 +100,7 @@ class DiagEmbedKernel : public framework::OpKernel { strides.push_back(stride[dim1_] + stride[dim2_]); const auto dims = vectorize(input->dims()); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector dims_vec(dims); const int64_t* dims_arr = thrust::raw_pointer_cast(dims_vec.data()); thrust::device_vector strides_vec(strides); diff --git a/paddle/fluid/operators/dist_op.cu b/paddle/fluid/operators/dist_op.cu index 499f5572910dd..90674969e283f 100644 --- a/paddle/fluid/operators/dist_op.cu +++ b/paddle/fluid/operators/dist_op.cu @@ -15,9 +15,18 @@ #include "paddle/fluid/operators/dist_op.h" namespace ops = paddle::operators; +#ifdef PADDLE_WITH_HIP +// Eigen3/unsupported/Eigen/CXX11/src/Tensor/TensorReductionGpu.h:922 +// do not support double in HIPCC platform (Eigen3 to be fixed) +REGISTER_OP_CUDA_KERNEL( + dist, ops::DistKernel); +REGISTER_OP_CUDA_KERNEL( + dist_grad, ops::DistGradKernel); +#else REGISTER_OP_CUDA_KERNEL( dist, ops::DistKernel, ops::DistKernel); REGISTER_OP_CUDA_KERNEL( dist_grad, ops::DistGradKernel, ops::DistGradKernel); +#endif diff --git a/paddle/fluid/operators/distributed_ops/CMakeLists.txt b/paddle/fluid/operators/distributed_ops/CMakeLists.txt index ec48a51baa212..e651f19fedbcf 100644 --- a/paddle/fluid/operators/distributed_ops/CMakeLists.txt +++ b/paddle/fluid/operators/distributed_ops/CMakeLists.txt @@ -30,7 +30,7 @@ endforeach() register_operators(EXCLUDES gen_nccl_id_op DEPS ${DISTRIBUTE_DEPS}) -if(WITH_NCCL) +if(WITH_NCCL OR WITH_RCCL) set(DISTRIBUTE_DEPS ${DISTRIBUTE_DEPS} nccl_common) endif() diff --git a/paddle/fluid/operators/distributed_ops/allreduce_op.h b/paddle/fluid/operators/distributed_ops/allreduce_op.h index e486faa575847..157924f08546b 100644 --- a/paddle/fluid/operators/distributed_ops/allreduce_op.h +++ b/paddle/fluid/operators/distributed_ops/allreduce_op.h @@ -21,7 +21,7 @@ limitations under the License. */ #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/op_registry.h" -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/nccl_helper.h" #endif @@ -36,7 +36,7 @@ class AllReduceOpKernel : public framework::OpKernel { PADDLE_ENFORCE_EQ(is_gpu_place(place), true, platform::errors::PreconditionNotMet( "AllReduce op can run on gpu place only for now.")); -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) auto& dev_ctx = ctx.template device_context(); auto in = ctx.Input("X"); auto out = ctx.Output("Out"); @@ -73,7 +73,11 @@ class AllReduceOpKernel : public framework::OpKernel { sendbuff, recvbuff, numel, static_cast(dtype), red_type, comm, stream)); if (ctx.Attr("sync_mode")) { +#ifdef PADDLE_WITH_RCCL + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } #else PADDLE_THROW(platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/operators/distributed_ops/broadcast_op.cu.cc b/paddle/fluid/operators/distributed_ops/broadcast_op.cu.cc index 337422f0bd643..1bfcc8af03e1e 100644 --- a/paddle/fluid/operators/distributed_ops/broadcast_op.cu.cc +++ b/paddle/fluid/operators/distributed_ops/broadcast_op.cu.cc @@ -20,7 +20,7 @@ limitations under the License. */ #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/op_registry.h" -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/platform/nccl_helper.h" #endif @@ -39,7 +39,7 @@ class NCCLBroadcastOpKernel : public framework::OpKernel { platform::errors::PreconditionNotMet( "The place of ExecutionContext should be CUDAPlace.")); -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) int dev_id = BOOST_GET_CONST(platform::CUDAPlace, ctx.GetPlace()).device; int root_dev_id = ctx.Attr("root"); @@ -68,7 +68,11 @@ class NCCLBroadcastOpKernel : public framework::OpKernel { << " From " << root_dev_id << " to " << dev_id; if (ctx.Attr("sync_mode")) { +#ifdef PADDLE_WITH_RCCL + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } #else PADDLE_THROW(platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/operators/distributed_ops/ref_by_trainer_id_op.h b/paddle/fluid/operators/distributed_ops/ref_by_trainer_id_op.h index d8639627c3ef6..c8c437c4965e7 100644 --- a/paddle/fluid/operators/distributed_ops/ref_by_trainer_id_op.h +++ b/paddle/fluid/operators/distributed_ops/ref_by_trainer_id_op.h @@ -30,7 +30,7 @@ class RefByTrainerIdKernel : public framework::OpKernel { int64_t trainer_id = 0; auto* trainer_id_data = trainer_id_t->data(); if (platform::is_gpu_place(context.GetPlace())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto stream = context.cuda_device_context().stream(); memory::Copy<>(platform::CPUPlace(), &trainer_id, BOOST_GET_CONST(platform::CUDAPlace, context.GetPlace()), diff --git a/paddle/fluid/operators/dot_op.h b/paddle/fluid/operators/dot_op.h index a197e2149ee02..1b607922eda1d 100644 --- a/paddle/fluid/operators/dot_op.h +++ b/paddle/fluid/operators/dot_op.h @@ -45,7 +45,7 @@ struct DotGradFunction> { const Tensor* tensor_dout, Tensor* tensor_dx, Tensor* tensor_dy, const paddle::framework::ExecutionContext& ctx) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) if (1 == tensor_dout->dims().size()) { auto dout = framework::EigenVector::Flatten(*tensor_dout); @@ -160,7 +160,7 @@ struct DotGradFunction> { const Tensor* tensor_dout, Tensor* tensor_dx, Tensor* tensor_dy, const paddle::framework::ExecutionContext& ctx) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) if (1 == tensor_dout->dims().size()) { auto dout = framework::EigenVector::Flatten(*tensor_dout); @@ -249,7 +249,7 @@ class DotKernel : public framework::OpKernel { auto* tensor_out = ctx.Output("Out"); tensor_out->mutable_data(ctx.GetPlace()); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) if (1 == tensor_out->dims().size()) { auto out = framework::EigenScalar::From(*tensor_out); auto x = framework::EigenVector::Flatten(*tensor_x); diff --git a/paddle/fluid/operators/dropout_op.cu b/paddle/fluid/operators/dropout_op.cu index cf90b9eb52b19..fbc145d3123d5 100644 --- a/paddle/fluid/operators/dropout_op.cu +++ b/paddle/fluid/operators/dropout_op.cu @@ -11,8 +11,17 @@ 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. */ + +#ifdef PADDLE_WITH_CUDA #include #include +#include "paddle/fluid/platform/dynload/curand.h" +#endif +#ifdef PADDLE_WITH_HIP +#include +#include +#include "paddle/fluid/platform/dynload/hiprand.h" +#endif #include #include #include @@ -21,7 +30,6 @@ limitations under the License. */ #include #include "paddle/fluid/memory/memcpy.h" #include "paddle/fluid/operators/dropout_op.h" -#include "paddle/fluid/platform/dynload/curand.h" #include "paddle/fluid/platform/float16.h" namespace paddle { @@ -32,15 +40,24 @@ __global__ void RandomGenerator(const size_t n, uint64_t seed, const float dropout_prob, const T* src, MaskType* mask_data, T* dst, bool is_upscale_in_train, uint64_t increment) { - curandStatePhilox4_32_10_t state; int idx = blockDim.x * blockIdx.x + threadIdx.x; +#ifdef PADDLE_WITH_HIP + hiprandStatePhilox4_32_10_t state; + hiprand_init(seed, idx, increment, &state); +#else + curandStatePhilox4_32_10_t state; curand_init(seed, idx, increment, &state); +#endif MaskType mask; T dest; for (; idx < n; idx += blockDim.x * gridDim.x) { T s = src[idx]; +#ifdef PADDLE_WITH_HIP + if (hiprand_uniform(&state) < dropout_prob) { +#else if (curand_uniform(&state) < dropout_prob) { +#endif mask = 0; dest = 0; } else { @@ -62,9 +79,15 @@ __global__ void VectorizedRandomGenerator(const size_t n, uint64_t seed, const T* src, MaskType* mask_data, T* dst, bool is_upscale_in_train, uint64_t increment) { +#ifdef PADDLE_WITH_HIP + int64_t idx = hipBlockDim_x * hipBlockIdx_x + hipThreadIdx_x; + hiprandStatePhilox4_32_10_t state; + hiprand_init(seed, idx, increment, &state); +#else int64_t idx = blockDim.x * blockIdx.x + threadIdx.x; curandStatePhilox4_32_10_t state; curand_init(seed, idx, increment, &state); +#endif MaskType mask; T dest; @@ -75,7 +98,11 @@ __global__ void VectorizedRandomGenerator(const size_t n, uint64_t seed, T src_vec[VecSize]; LoadT* value = reinterpret_cast(&src_vec); *value = *reinterpret_cast(&src[i]); +#ifdef PADDLE_WITH_HIP + float4 rand = hiprand_uniform4(&state); +#else float4 rand = curand_uniform4(&state); +#endif T dest_vec[VecSize]; MaskType mask_vec[VecSize]; @@ -131,10 +158,17 @@ class GPUDropoutKernel : public framework::OpKernel { auto* x_data = x->data(); auto* y_data = y->mutable_data(context.GetPlace()); if (dropout_prob == 1.0f) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + hipMemsetAsync(y_data, 0, x_numel * sizeof(T), stream)); + PADDLE_ENFORCE_CUDA_SUCCESS( + hipMemsetAsync(mask_data, 0, x_numel * sizeof(*mask_data), stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( cudaMemsetAsync(y_data, 0, x_numel * sizeof(T), stream)); PADDLE_ENFORCE_CUDA_SUCCESS(cudaMemsetAsync( mask_data, 0, x_numel * sizeof(*mask_data), stream)); +#endif return; } @@ -180,6 +214,20 @@ class GPUDropoutKernel : public framework::OpKernel { increment = offset; } +#ifdef __HIPCC__ + if (vec_size == 4 && size % 4 == 0) { + hipLaunchKernelGGL( + HIP_KERNEL_NAME(VectorizedRandomGenerator), + config.block_per_grid, config.thread_per_block, 0, stream, size, + seed_data, dropout_prob, x_data, mask_data, y_data, + upscale_in_train, increment); + } else { + hipLaunchKernelGGL(HIP_KERNEL_NAME(RandomGenerator), + config.block_per_grid, config.thread_per_block, 0, + stream, size, seed_data, dropout_prob, x_data, + mask_data, y_data, upscale_in_train, increment); + } +#else if (vec_size == 4 && size % 4 == 0) { VectorizedRandomGenerator< T, uint8_t, @@ -192,7 +240,7 @@ class GPUDropoutKernel : public framework::OpKernel { size, seed_data, dropout_prob, x_data, mask_data, y_data, upscale_in_train, increment); } - +#endif } else { auto X = EigenMatrix::Reshape(*x, 1); auto Y = EigenMatrix::Reshape(*y, 1); diff --git a/paddle/fluid/operators/dropout_op.h b/paddle/fluid/operators/dropout_op.h index d77193e485134..69c420e2c93ed 100644 --- a/paddle/fluid/operators/dropout_op.h +++ b/paddle/fluid/operators/dropout_op.h @@ -42,7 +42,7 @@ inline int VectorizedSize(const T* pointer) { return 1; } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template __global__ void DropoutGradCUDAKernel(const T* dout, const MaskType* mask, const T factor, const int64_t size, @@ -186,7 +186,7 @@ class DropoutGradKernel : public framework::OpKernel { int vec_size = VectorizedSize(grad_y->data()); if (platform::is_gpu_place(context.GetPlace()) && vec_size == 4 && size % 4 == 0) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) auto factor = static_cast(1.0f / (1.0f - dropout_prob)); auto stream = context.cuda_device_context().stream(); platform::GpuLaunchConfig config = platform::GetGpuLaunchConfig1D( diff --git a/paddle/fluid/operators/elementwise/elementwise_add_op.h b/paddle/fluid/operators/elementwise/elementwise_add_op.h index c46184f5badbc..abea9da942355 100644 --- a/paddle/fluid/operators/elementwise/elementwise_add_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_add_op.h @@ -20,12 +20,18 @@ limitations under the License. */ #include "paddle/fluid/operators/elementwise/elementwise_op_function.h" #include "paddle/fluid/operators/math/blas.h" #include "paddle/fluid/operators/math/math_function.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #ifdef __NVCC__ #include #include #include "cub/cub.cuh" #endif +#ifdef __HIPCC__ +#include +#include +#include +namespace cub = hipcub; +#endif #endif namespace paddle { @@ -179,7 +185,7 @@ __global__ void MatrixColReduce(const T *__restrict__ in, T *__restrict__ out, } } -#if CUDA_VERSION >= 10000 +#if defined(PADDLE_WITH_CUDA) && CUDA_VERSION >= 10000 template __global__ void VecFP16MatrixColReduce(const __half2 *__restrict__ in, __half2 *__restrict__ out, size_t width, @@ -287,7 +293,7 @@ bool static RunSpecialDims(const framework::DDim &dx_dims, return true; } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // cuda definition template typename std::enable_if< diff --git a/paddle/fluid/operators/elementwise/elementwise_div_op.h b/paddle/fluid/operators/elementwise/elementwise_div_op.h index 5f4321f7273c9..0be8d934b17af 100644 --- a/paddle/fluid/operators/elementwise/elementwise_div_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_div_op.h @@ -144,7 +144,7 @@ elementwise_div_grad(const framework::ExecutionContext& ctx, ctx, *x, *y, *out, *dout, axis, dx, dy, DivGradDX(), DivGradDY()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // cuda definition template typename std::enable_if< diff --git a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h index 721c23e38307f..06eb0b1cc8510 100644 --- a/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_floordiv_op.h @@ -25,10 +25,14 @@ namespace operators { template struct FloorDivFunctor { inline HOSTDEVICE T operator()(T a, T b) const { -#ifdef __CUDA_ARCH__ +#if defined(__HIPCC__) || defined(__CUDA_ARCH__) if (b == 0) { printf("Error: Divide by zero encounter in floor_divide\n"); +#ifdef __HIPCC__ + abort(); +#else asm("trap;"); +#endif } #else if (b == 0) @@ -42,10 +46,14 @@ struct FloorDivFunctor { template struct InverseFloorDivFunctor { inline HOSTDEVICE T operator()(T a, T b) const { -#ifdef __CUDA_ARCH__ +#if defined(__HIPCC__) || defined(__CUDA_ARCH__) if (a == 0) { printf("Error: Divide by zero encounter in floor_divide\n"); +#ifdef __HIPCC__ + abort(); +#else asm("trap;"); +#endif } #else if (a == 0) diff --git a/paddle/fluid/operators/elementwise/elementwise_mul_op.h b/paddle/fluid/operators/elementwise/elementwise_mul_op.h index 3bc12fe16d979..46a00268e4134 100644 --- a/paddle/fluid/operators/elementwise/elementwise_mul_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_mul_op.h @@ -192,7 +192,7 @@ elementwise_mul_grad(const framework::ExecutionContext& ctx, ctx, *x, *y, *out, *dout, axis, dx, dy, MulGradDX(), MulGradDY()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // cuda definition template typename std::enable_if< diff --git a/paddle/fluid/operators/elementwise/elementwise_op.h b/paddle/fluid/operators/elementwise/elementwise_op.h index a09fe4b676041..e09f94a6c0fee 100644 --- a/paddle/fluid/operators/elementwise/elementwise_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_op.h @@ -276,13 +276,15 @@ class ElementwiseOpGrad : public framework::OperatorWithKernel { #ifdef PADDLE_WITH_MKLDNN // If broadcasting is needed, use native implementation - auto CanMKLDNNElementwiseAddGradBeUsed = [&]() { - return (ctx.Input("X")->dims() == ctx.Input("Y")->dims()); + auto CanMKLDNNElementwiseGradBeUsed = [&]() { + auto dx_dims = ctx.Input("X")->dims(); + auto dy_dims = ctx.Input("Y")->dims(); + // No broadcast or broadcasting of data on inner dims is supported + return (dx_dims[dx_dims.size() - 1] == dy_dims[dy_dims.size() - 1]); }; if (this->CanMKLDNNBeUsed(ctx, input_data_type) && - (ctx.Type() != "elementwise_add_grad" || - CanMKLDNNElementwiseAddGradBeUsed())) { + CanMKLDNNElementwiseGradBeUsed()) { return framework::OpKernelType(input_data_type, ctx.GetPlace(), framework::DataLayout::kMKLDNN, framework::LibraryType::kMKLDNN); diff --git a/paddle/fluid/operators/elementwise/elementwise_op_function.cu.h b/paddle/fluid/operators/elementwise/elementwise_op_function.cu.h index afa87a0ad8a1f..1121d0ef68ce2 100644 --- a/paddle/fluid/operators/elementwise/elementwise_op_function.cu.h +++ b/paddle/fluid/operators/elementwise/elementwise_op_function.cu.h @@ -22,13 +22,19 @@ limitations under the License. */ #ifdef PADDLE_WITH_CUDA #include +#ifdef PADDLE_CUDA_FP16 +#include +#endif #endif // PADDLE_WITH_CUDA +#ifdef PADDLE_WITH_HIP +#include #ifdef PADDLE_CUDA_FP16 -#include +#include #endif +#endif // PADDLE_WITH_HIP -#if CUDA_VERSION < 9000 +#if defined(PADDLE_WITH_CUDA) && CUDA_VERSION < 9000 #define __h2div h2div #endif @@ -101,7 +107,7 @@ struct DivRangeFunctor< #ifdef PADDLE_CUDA_FP16 inline DEVICE half2 half2_add(const half2& a, const half2& b) { -#if defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530 +#if defined(__HIPCC__) || (defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530) return __hadd2(a, b); #else float a1 = __low2float(a); @@ -115,7 +121,7 @@ inline DEVICE half2 half2_add(const half2& a, const half2& b) { } inline DEVICE half2 half2_sub(const half2& a, const half2& b) { -#if defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530 +#if defined(__HIPCC__) || (defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530) return __hsub2(a, b); #else float a1 = __low2float(a); @@ -129,7 +135,7 @@ inline DEVICE half2 half2_sub(const half2& a, const half2& b) { } inline DEVICE half2 half2_mul(const half2& a, const half2& b) { -#if defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530 +#if defined(__HIPCC__) || (defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530) return __hmul2(a, b); #else float a1 = __low2float(a); @@ -143,7 +149,7 @@ inline DEVICE half2 half2_mul(const half2& a, const half2& b) { } inline DEVICE half2 half2_div(const half2& a, const half2& b) { -#if defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530 +#if defined(__HIPCC__) || (defined(__CUDA_ARCH__) && __CUDA_ARCH__ >= 530) return __h2div(a, b); #else float a1 = __low2float(a); diff --git a/paddle/fluid/operators/elementwise/elementwise_op_function.h b/paddle/fluid/operators/elementwise/elementwise_op_function.h index 46b477afeb535..c69baadb3c22e 100644 --- a/paddle/fluid/operators/elementwise/elementwise_op_function.h +++ b/paddle/fluid/operators/elementwise/elementwise_op_function.h @@ -29,8 +29,12 @@ limitations under the License. */ #include "paddle/fluid/platform/gpu_info.h" #include "paddle/fluid/platform/transform.h" +#if defined(__NVCC__) || defined(__HIPCC__) #ifdef __NVCC__ #include +#elif defined(__HIPCC__) +#include +#endif #include #include "paddle/fluid/platform/cuda_device_function.h" @@ -95,6 +99,7 @@ inline void get_mid_dims(const framework::DDim &x_dims, (*post) *= x_dims[i]; } } + inline int GetElementwiseIndex(const int *x_dims_array, const int max_dim, const int *index_array) { int index_ = 0; @@ -196,14 +201,18 @@ void CommonForwardBroadcastCPU(const framework::Tensor *x, } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template -__global__ void ElementwiseKernel(const T *x, const T *y, OutType *out, int pre, - int n, int post, int total, Functor func) { +__global__ void ElementwiseKernel(const T *__restrict__ x_data, + const T *__restrict__ y_data, + OutType *__restrict__ out_data, int n, + int post, const size_t total, Functor func) { int tid = threadIdx.x + blockDim.x * blockIdx.x; - int idx = tid / post % n; - if (tid < total) { - out[tid] = func(x[tid], y[idx]); + int stride = blockDim.x * gridDim.x; + + for (int i = tid; i < total; i += stride) { + int idx = i / post % n; + out_data[i] = func(x_data[i], y_data[idx]); } } @@ -220,14 +229,16 @@ void ComputeElementwiseCUDA(const framework::Tensor *x, int numel = pre * n * post; int threads = 256; int blocks = (numel + threads - 1) / threads; + if (is_xsize_larger) { ElementwiseKernel<<>>( - x_data, y_data, out_data, pre, n, post, numel, func); + x_data, y_data, out_data, n, post, numel, func); + } else { ElementwiseKernel<<>>( - y_data, x_data, out_data, pre, n, post, numel, func); + y_data, x_data, out_data, n, post, numel, func); } } @@ -310,7 +321,7 @@ void CommonForwardBroadcastCUDA( y_data, out_data, out_size, max_dim, func, is_xsize_larger); } -#endif // __NVCC__ +#endif // __NVCC__ or __HIPCC__ template void CommonGradBroadcastCPU( @@ -382,7 +393,7 @@ inline void ComputeBroadcastTranspositionArray(const int *x_one_indexs, } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template static __global__ void ElemwiseGradBroadcast1CUDAKernel( const T *x, const T *y, const T *out, const T *dout, int h, int w, @@ -1212,7 +1223,7 @@ void CommonGradBroadcastCUDA( } } -#endif // __NVCC__ +#endif // __NVCC__ or __HIPCC__ inline framework::DDim trim_trailing_singular_dims( const framework::DDim &dims) { @@ -1339,7 +1350,7 @@ class MidWiseTransformIterator int64_t post_; }; -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template class RowwiseTransformIterator : public thrust::iterator_adaptor< @@ -1504,10 +1515,10 @@ static void ElemwiseGradBroadcast1CPU(const T *x, const T *y, const T *out, } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template -static void ElemwiseGradBroadcast1CUDA(cudaStream_t stream, const T *x, +static void ElemwiseGradBroadcast1CUDA(gpuStream_t stream, const T *x, const T *y, const T *out, const T *dout, int h, int w, bool is_xsize_larger, DX_OP dx_op, DY_OP dy_op, T *dx, T *dy) { @@ -1577,7 +1588,7 @@ static void ElemwiseGradBroadcast2CPU(const T *x, const T *y, const T *out, } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template static __global__ void ElemwiseGradBroadcast2CUDAKernel( const T *x, const T *y, const T *out, const T *dout, int pre, int n, @@ -1646,7 +1657,7 @@ static __global__ void ElemwiseGradBroadcast2CUDAKernel( } template -static void ElemwiseGradBroadcast2CUDA(cudaStream_t stream, const T *x, +static void ElemwiseGradBroadcast2CUDA(gpuStream_t stream, const T *x, const T *y, const T *out, const T *dout, int pre, int n, int post, bool is_xsize_larger, DX_OP dx_op, @@ -1686,7 +1697,7 @@ void CommonElementwiseBroadcastBackward( << " ydim:" << framework::make_ddim(y_dims_array); if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) CommonGradBroadcastCUDA( x, y, out, dout, dx, dy, x_dims_array.data(), y_dims_array.data(), out_dims_array.data(), max_dim, @@ -1769,7 +1780,7 @@ void ElemwiseGradComputeWithBroadcast( } if (post == 1) { if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) ElemwiseGradBroadcast1CUDA( ctx.template device_context().stream(), x.data(), y.data(), out.data(), dout.data(), pre, n, is_xsize_larger, @@ -1786,7 +1797,7 @@ void ElemwiseGradComputeWithBroadcast( } } else { if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) ElemwiseGradBroadcast2CUDA( ctx.template device_context().stream(), x.data(), y.data(), out.data(), dout.data(), pre, n, post, @@ -1830,7 +1841,7 @@ void CommonElementwiseBroadcastForward( axis); if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) CommonForwardBroadcastCUDA( x, y, z, x_dims_array.data(), y_dims_array.data(), out_dims_array.data(), max_dim, @@ -1942,7 +1953,7 @@ void ElementwiseComputeEx(const framework::ExecutionContext &ctx, } if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) ComputeElementwiseCUDA( x, y, z, pre, n, post, ctx.template device_context(), func, @@ -2066,7 +2077,7 @@ static void FusedElemwiseAndActBroadcast2CPU(const T *x, const T *y, int pre, } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template static __global__ void FusedElemwiseAndActBroadcast1CUDAKernel( @@ -2107,7 +2118,7 @@ static __global__ void FusedElemwiseAndActBroadcast1CUDAKernel( template -static void FusedElemwiseAndActBroadcast1CUDA(cudaStream_t stream, const T *x, +static void FusedElemwiseAndActBroadcast1CUDA(gpuStream_t stream, const T *x, const T *y, CompoundFunctor compound_functor, int h, int w, T *out, @@ -2164,7 +2175,7 @@ static __global__ void FusedElemwiseAndActBroadcast2CUDAKernel( template -static void FusedElemwiseAndActBroadcast2CUDA(cudaStream_t stream, const T *x, +static void FusedElemwiseAndActBroadcast2CUDA(gpuStream_t stream, const T *x, const T *y, int pre, int n, int post, CompoundFunctor compound_functor, @@ -2219,7 +2230,7 @@ void FusedElemwiseAndActComputeWithBroadcast( int h = pre; int w = n; if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) FusedElemwiseAndActBroadcast1CUDA( @@ -2242,7 +2253,7 @@ void FusedElemwiseAndActComputeWithBroadcast( } } else { if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) FusedElemwiseAndActBroadcast2CUDA( @@ -2493,7 +2504,7 @@ static void FusedElemwiseAndActGradBroadcast2CPU( } } -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) template @@ -2593,7 +2604,7 @@ template static void FusedElemwiseAndActGradBroadcast1CUDA( - cudaStream_t stream, const T *x, const T *y, const T *intermediate_out, + gpuStream_t stream, const T *x, const T *y, const T *intermediate_out, const T *out, const T *dout, int h, int w, DX_OP dx_op, DY_OP dy_op, DIntermediate_OP dintermediate_op, T *dx, T *dy, T *d_intermediate) { int block_size = std::min(ELEMWISE_MAX_BLOCK_DIM, h); @@ -2708,7 +2719,7 @@ template static void FusedElemwiseAndActGradBroadcast2CUDA( - cudaStream_t stream, const T *x, const T *y, const T *intermediate_out, + gpuStream_t stream, const T *x, const T *y, const T *intermediate_out, const T *out, const T *dout, int pre, int n, int post, DX_OP dx_op, DY_OP dy_op, DIntermediate_OP dintermediate_op, T *dx, T *dy, T *dintermediate) { @@ -2748,7 +2759,7 @@ void FusedElemwiseAndActGradComputeWithBroadcast( int w = n; if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) FusedElemwiseAndActGradBroadcast1CUDA( @@ -2774,7 +2785,7 @@ void FusedElemwiseAndActGradComputeWithBroadcast( } } else { if (platform::is_gpu_place(ctx.GetPlace())) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) FusedElemwiseAndActGradBroadcast2CUDA( diff --git a/paddle/fluid/operators/elementwise/elementwise_pow_op.h b/paddle/fluid/operators/elementwise/elementwise_pow_op.h index 8cc4b166fc491..ee718a3ecd1ec 100755 --- a/paddle/fluid/operators/elementwise/elementwise_pow_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_pow_op.h @@ -25,7 +25,7 @@ struct PowFunctor { inline HOSTDEVICE T operator()(T a, T b) const { // TODO(wujionghao): A potential speed improvement is supporting different // types in C++. -#ifdef __CUDA_ARCH__ +#if defined(__CUDA_ARCH__) || defined(__HIPCC__) // On CUDAPlace, std::pow(3, 1) calls pow(float, float), and // it will return a float number like 2.99... , which floor to 2 // when cast to int by default and it is wrong. diff --git a/paddle/fluid/operators/elementwise/elementwise_sub_op.h b/paddle/fluid/operators/elementwise/elementwise_sub_op.h index 3e97366b6157d..4171d2eb9e5e5 100644 --- a/paddle/fluid/operators/elementwise/elementwise_sub_op.h +++ b/paddle/fluid/operators/elementwise/elementwise_sub_op.h @@ -86,7 +86,7 @@ elementwise_sub_grad(const framework::ExecutionContext& ctx, ctx, *x, *y, *out, *dout, axis, dx, dy, SubGradDX(), SubGradDY()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // cuda definition template typename std::enable_if< diff --git a/paddle/fluid/operators/elementwise/mkldnn/elementwise_add_mkldnn_op.cc b/paddle/fluid/operators/elementwise/mkldnn/elementwise_add_mkldnn_op.cc index 13acd3fa63680..b43dddfcf19db 100644 --- a/paddle/fluid/operators/elementwise/mkldnn/elementwise_add_mkldnn_op.cc +++ b/paddle/fluid/operators/elementwise/mkldnn/elementwise_add_mkldnn_op.cc @@ -61,17 +61,43 @@ class EltwiseAddMKLDNNGradKernel : public ElemwiseGradKernel { platform::EventRole::kUniqueOp); reorder_p->execute(astream, *reorder_src_memory_p, *reorder_dst_memory_p); astream.wait(); + + dx->set_layout(DataLayout::kMKLDNN); + dx->set_format(platform::GetMKLDNNFormat(*reorder_dst_memory_p)); } if (dy) { - auto reorder_dst_memory_p = - handler.AcquireDstMemory(dy, dout->format(), ctx.GetPlace()); - auto reorder_p = - handler.AcquireReorder(reorder_dst_memory_p, reorder_src_memory_p); - platform::RecordEvent record_reorder("int_reorder", - platform::EventRole::kUniqueOp); - reorder_p->execute(astream, *reorder_src_memory_p, *reorder_dst_memory_p); - astream.wait(); + // Direct copy + if (dout->dims() == dy->dims()) { + auto reorder_dst_memory_p = + handler.AcquireDstMemory(dy, dout->format(), ctx.GetPlace()); + auto reorder_p = + handler.AcquireReorder(reorder_dst_memory_p, reorder_src_memory_p); + platform::RecordEvent record_reorder("int_reorder", + platform::EventRole::kUniqueOp); + reorder_p->execute(astream, *reorder_src_memory_p, + *reorder_dst_memory_p); + astream.wait(); + + dy->set_layout(DataLayout::kMKLDNN); + dy->set_format(platform::GetMKLDNNFormat(*reorder_dst_memory_p)); + } else { + // Broadcasting + platform::ReductionMKLDNNHandler handler_sum( + dnnl::algorithm::reduction_sum, 0.0f, 0.0f, dev_ctx, onednn_engine, + ctx.GetPlace(), dout, dy, + ctx.InputName(framework::GradVarName("Out"))); + auto dy_memory_p = handler_sum.AcquireDstMemory(dy); + auto reduction_p = handler_sum.AcquireForwardPrimitive(); + reduction_p->execute(astream, {{DNNL_ARG_SRC, *reorder_src_memory_p}, + {DNNL_ARG_DST, *dy_memory_p}}); + astream.wait(); + + dy->set_layout(DataLayout::kMKLDNN); + dy->set_format( + platform::GetMKLDNNFormat(dy_memory_p->get_desc().reshape( + paddle::framework::vectorize(dy->dims())))); + } } } }; diff --git a/paddle/fluid/operators/elementwise/mkldnn/elementwise_mkldnn_op.h b/paddle/fluid/operators/elementwise/mkldnn/elementwise_mkldnn_op.h index 8a646e5865d92..df827117a0d30 100644 --- a/paddle/fluid/operators/elementwise/mkldnn/elementwise_mkldnn_op.h +++ b/paddle/fluid/operators/elementwise/mkldnn/elementwise_mkldnn_op.h @@ -15,7 +15,6 @@ #pragma once #include #include -#include "paddle/fluid/memory/memcpy.h" #include "paddle/fluid/operators/elementwise/elementwise_add_op.h" #include "paddle/fluid/operators/elementwise/elementwise_op_function.h" diff --git a/paddle/fluid/operators/elementwise/mkldnn/elementwise_mul_mkldnn_op.cc b/paddle/fluid/operators/elementwise/mkldnn/elementwise_mul_mkldnn_op.cc index 293b5a1a2d31b..c9209cc39d5e3 100644 --- a/paddle/fluid/operators/elementwise/mkldnn/elementwise_mul_mkldnn_op.cc +++ b/paddle/fluid/operators/elementwise/mkldnn/elementwise_mul_mkldnn_op.cc @@ -14,6 +14,118 @@ limitations under the License. */ #include "paddle/fluid/operators/elementwise/mkldnn/elementwise_mkldnn_op.h" +namespace paddle { +namespace framework { +class ExecutionContext; +} // namespace framework +namespace platform { +class CPUDeviceContext; +struct CPUPlace; +} // namespace platform +} // namespace paddle + +namespace paddle { +namespace operators { +template +class EltwiseMulMKLDNNGradKernel : public ElemwiseGradKernel { + public: + void Compute(const framework::ExecutionContext& ctx) const override { + ElemwiseGradKernel::Compute(ctx); + + auto& dev_ctx = + ctx.template device_context(); + const auto& mkldnn_engine = dev_ctx.GetEngine(); + + auto* x = ctx.Input("X"); + auto* y = ctx.Input("Y"); + auto* dout = ctx.Input(framework::GradVarName("Out")); + auto* dx = ctx.Output(framework::GradVarName("X")); + auto* dy = ctx.Output(framework::GradVarName("Y")); + int axis = ctx.Attr("axis"); + + auto& astream = platform::MKLDNNDeviceContext::tls().get_stream(); + + if (dx) { + // dx = dout*y + platform::BinaryMKLDNNHandler handler( + dnnl::algorithm::binary_mul, axis, dev_ctx, mkldnn_engine, + ctx.GetPlace(), dout, y, dx, 1.0f, 1.0f, 1.0f, + ctx.InputName(framework::GradVarName("Out"))); + + const auto src_dout_memory = handler.AcquireSrcMemory(dout); + const auto src_y_memory = handler.AcquireSecondSrcMemory(y); + const auto dst_dx_memory = handler.AcquireDstMemory(dx); + + const auto binary_prim = handler.AcquireForwardPrimitive(); + + const std::unordered_map args = { + {DNNL_ARG_SRC_0, *src_dout_memory}, + {DNNL_ARG_SRC_1, *src_y_memory}, + {DNNL_ARG_DST, *dst_dx_memory}}; + + binary_prim->execute(astream, args); + astream.wait(); + + dx->set_layout(framework::DataLayout::kMKLDNN); + dx->set_format(platform::GetMKLDNNFormat(*dst_dx_memory)); + } + + if (dy) { + // dy = dout*x + // Handler is having nullptr passed instead of output tensor as + // we want Dst buffer to be allocated by oneDNN not to use Tensor + platform::BinaryMKLDNNHandler handler( + dnnl::algorithm::binary_mul, axis, dev_ctx, mkldnn_engine, + ctx.GetPlace(), dout, x, nullptr, 1.0f, 1.0f, 1.0f, + ctx.InputName(framework::GradVarName("Out"))); + + const auto src_dout_memory = handler.AcquireSrcMemory(dout); + const auto src_x_memory = handler.AcquireSecondSrcMemory(x); + + // If broadcasting is in use then let's write to temporary + // buffer allocated by oneDNN + const auto dst_dy_memory = (dout->dims() == dy->dims()) + ? handler.AcquireDstMemory(dy) + : handler.AcquireDstMemory(); + + const auto binary_prim = handler.AcquireForwardPrimitive(); + + const std::unordered_map args = { + {DNNL_ARG_SRC_0, *src_dout_memory}, + {DNNL_ARG_SRC_1, *src_x_memory}, + {DNNL_ARG_DST, *dst_dy_memory}}; + + binary_prim->execute(astream, args); + astream.wait(); + + dy->set_layout(framework::DataLayout::kMKLDNN); + + // Reduction is needed for broadcasting scenario + if (dout->dims() != dy->dims()) { + platform::ReductionMKLDNNHandler handler_sum( + dnnl::algorithm::reduction_sum, 0.0f, 0.0f, dev_ctx, mkldnn_engine, + ctx.GetPlace(), dout, dy, + ctx.InputName(framework::GradVarName("Out"))); + auto dy_memory_p = handler_sum.AcquireDstMemory(dy); + auto reduction_p = handler_sum.AcquireForwardPrimitive(); + // As source we use mem object with results from binary operation + reduction_p->execute(astream, {{DNNL_ARG_SRC, *dst_dy_memory}, + {DNNL_ARG_DST, *dy_memory_p}}); + astream.wait(); + dy->set_format( + platform::GetMKLDNNFormat(dy_memory_p->get_desc().reshape( + paddle::framework::vectorize(dy->dims())))); + + } else { + dy->set_format(platform::GetMKLDNNFormat(*dst_dy_memory)); + } + } + } +}; + +} // namespace operators +} // namespace paddle + namespace ops = paddle::operators; REGISTER_OP_KERNEL( @@ -23,3 +135,7 @@ REGISTER_OP_KERNEL( dnnl::algorithm::binary_mul>, ops::EltwiseMKLDNNKernel, ops::EltwiseMKLDNNKernel) + +REGISTER_OP_KERNEL(elementwise_mul_grad, MKLDNN, ::paddle::platform::CPUPlace, + ops::EltwiseMulMKLDNNGradKernel, + ops::EltwiseMulMKLDNNGradKernel) diff --git a/paddle/fluid/operators/elementwise/test_elementwise_add_grad_grad.cc b/paddle/fluid/operators/elementwise/test_elementwise_add_grad_grad.cc index 15c31a4cece5c..12d82654362ac 100644 --- a/paddle/fluid/operators/elementwise/test_elementwise_add_grad_grad.cc +++ b/paddle/fluid/operators/elementwise/test_elementwise_add_grad_grad.cc @@ -60,7 +60,7 @@ TEST(test_elementwise_add_grad_grad_without_ddx, cpu_place) { TestElementwiseAddGradGradWithoutDDX test(p, dims); ASSERT_TRUE(test.Check()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(test_elementwise_add_grad_grad_without_ddx, gpu_place) { framework::DDim dims({32, 64}); platform::CUDAPlace p(0); diff --git a/paddle/fluid/operators/elementwise/test_elementwise_add_op_inplace.cc b/paddle/fluid/operators/elementwise/test_elementwise_add_op_inplace.cc index cf9e9dbb04b03..ab45b6f4de276 100644 --- a/paddle/fluid/operators/elementwise/test_elementwise_add_op_inplace.cc +++ b/paddle/fluid/operators/elementwise/test_elementwise_add_op_inplace.cc @@ -32,6 +32,8 @@ static void Memcpy(void *dst, const void *src, size_t n, bool copy_to_gpu) { #ifdef PADDLE_WITH_CUDA PADDLE_ENFORCE_CUDA_SUCCESS( cudaMemcpy(dst, src, n, cudaMemcpyHostToDevice)); +#elif defined(PADDLE_WITH_HIP) + PADDLE_ENFORCE_CUDA_SUCCESS(hipMemcpy(dst, src, n, hipMemcpyHostToDevice)); #else PADDLE_THROW( platform::errors::InvalidArgument("Check your paddle version, current " @@ -129,7 +131,7 @@ TEST(test_elementwise_add_not_inplace, cpu_place) { ASSERT_TRUE(TestMain(p, dims, false)); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(test_elementwise_add_inplace, gpu_place) { framework::DDim dims({32, 64}); platform::CUDAPlace p(0); diff --git a/paddle/fluid/operators/elementwise/test_elementwise_div_grad_grad.cc b/paddle/fluid/operators/elementwise/test_elementwise_div_grad_grad.cc index e1f893dd2b8ae..82448c681cd47 100644 --- a/paddle/fluid/operators/elementwise/test_elementwise_div_grad_grad.cc +++ b/paddle/fluid/operators/elementwise/test_elementwise_div_grad_grad.cc @@ -84,7 +84,7 @@ TEST(test_elementwise_div_grad_grad_without_dout, cpu_place) { ASSERT_TRUE(test.Check()); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(test_elementwise_div_grad_grad_without_dout, gpu_place) { framework::DDim dims({32, 64}); platform::CUDAPlace p(0); diff --git a/paddle/fluid/operators/elementwise/test_elementwise_op_grad_grad.h b/paddle/fluid/operators/elementwise/test_elementwise_op_grad_grad.h index 54e7c7d1b6aa9..8bfb566d496d0 100644 --- a/paddle/fluid/operators/elementwise/test_elementwise_op_grad_grad.h +++ b/paddle/fluid/operators/elementwise/test_elementwise_op_grad_grad.h @@ -88,7 +88,7 @@ class TestElementwiseOpGradGrad { auto dst_place = BOOST_GET_CONST(platform::CPUPlace, place_); memory::Copy(dst_place, dst, src_place, src, bytes); } else if (platform::is_gpu_place(place_)) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto dst_place = BOOST_GET_CONST(platform::CUDAPlace, place_); memory::Copy(dst_place, dst, src_place, src, bytes, nullptr); #else @@ -126,8 +126,14 @@ class TestElementwiseOpGradGrad { } auto *out_ptr = cpu_out.data(); size_t numel = static_cast(framework::product(dims_)); +#ifdef PADDLE_WITH_HIP + auto is_equal = std::equal( + out_ptr, out_ptr + numel, expected_outs_[out_name].data(), + [](const float &l, const float &r) { return fabs(l - r) < 1e-8; }); +#else auto is_equal = std::equal(out_ptr, out_ptr + numel, expected_outs_[out_name].data()); +#endif if (!is_equal) { all_equal = false; break; diff --git a/paddle/fluid/operators/fake_quantize_op.cu b/paddle/fluid/operators/fake_quantize_op.cu index 26dcf8bf39cf2..92127f9aebd0d 100644 --- a/paddle/fluid/operators/fake_quantize_op.cu +++ b/paddle/fluid/operators/fake_quantize_op.cu @@ -162,7 +162,11 @@ struct FindChannelAbsMaxFunctor { int grid = cout; int max_threads = 1024; +#ifdef PADDLE_WITH_HIP + hipMemset(out_abs_max, 0, sizeof(T) * cout); +#else cudaMemset(out_abs_max, 0, sizeof(T) * cout); +#endif for (int i = 0; i < cin / max_threads; i++) { int block = max_threads; diff --git a/paddle/fluid/operators/fill_constant_batch_size_like_op.h b/paddle/fluid/operators/fill_constant_batch_size_like_op.h index e8a35d2227707..432a9968ab0d9 100644 --- a/paddle/fluid/operators/fill_constant_batch_size_like_op.h +++ b/paddle/fluid/operators/fill_constant_batch_size_like_op.h @@ -65,7 +65,7 @@ class FillConstantBatchSizeLikeOpKernel : public framework::OpKernel { functor(reinterpret_cast(dev_ctx), out, static_cast(value)); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (!cpu_place) { math::SetConstant functor; out->mutable_data(ctx.GetPlace(), data_type); diff --git a/paddle/fluid/operators/fill_constant_op.h b/paddle/fluid/operators/fill_constant_op.h index 5d1f1fa781df2..4608f167548a3 100644 --- a/paddle/fluid/operators/fill_constant_op.h +++ b/paddle/fluid/operators/fill_constant_op.h @@ -121,7 +121,7 @@ class FillConstantKernel : public framework::OpKernel { functor(reinterpret_cast(dev_ctx), tensor, static_cast(value)); } else if (actual_place == 1) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) tensor->mutable_data(ctx.GetPlace(), data_type); math::SetConstant functor; functor(reinterpret_cast(dev_ctx), @@ -131,7 +131,7 @@ class FillConstantKernel : public framework::OpKernel { "PaddlePaddle should compile with GPU.")); #endif } else if (actual_place == 2) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) tensor->mutable_data(platform::CUDAPinnedPlace(), data_type); math::SetConstant functor; functor(reinterpret_cast(dev_ctx), diff --git a/paddle/fluid/operators/filter_by_instag_op.h b/paddle/fluid/operators/filter_by_instag_op.h index 9234f9be474a1..77bc9e466e808 100644 --- a/paddle/fluid/operators/filter_by_instag_op.h +++ b/paddle/fluid/operators/filter_by_instag_op.h @@ -31,7 +31,7 @@ namespace operators { using Tensor = framework::Tensor; using SelectedRows = framework::SelectedRows; using LoDTensor = framework::LoDTensor; -#if defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template using Vector = framework::Vector; #else diff --git a/paddle/fluid/operators/fused/CMakeLists.txt b/paddle/fluid/operators/fused/CMakeLists.txt index 95ae807c6ae04..287827ced5115 100644 --- a/paddle/fluid/operators/fused/CMakeLists.txt +++ b/paddle/fluid/operators/fused/CMakeLists.txt @@ -24,22 +24,28 @@ file(APPEND ${pybind_file} "USE_CPU_ONLY_OP(fusion_gru);\n") file(APPEND ${pybind_file} "USE_CPU_ONLY_OP(fusion_lstm);\n") -if (WITH_GPU) +if (WITH_GPU OR WITH_ROCM) # fused_bn_activation_op needs cudnn 7.4.1 above - if (NOT ${CUDNN_VERSION} VERSION_LESS 7401) + # HIP not support bn act fuse in MIOPEN + if ((NOT WITH_ROCM) AND (NOT ${CUDNN_VERSION} VERSION_LESS 7401)) op_library(fused_bn_activation_op) file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(fused_batch_norm_act);\n") endif() # conv_fusion_op needs cudnn 7 above - if (NOT ${CUDNN_VERSION} VERSION_LESS 7100) + # HIP not support cudnnConvolutionBiasActivationForward + if ((NOT WITH_ROCM) AND (NOT ${CUDNN_VERSION} VERSION_LESS 7100)) op_library(conv_fusion_op) file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(conv2d_fusion);\n") endif() # fusion_transpose_flatten_concat_op - op_library(fusion_transpose_flatten_concat_op) - file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(fusion_transpose_flatten_concat);\n") + # HIP not support cudnnTransformTensor + if(NOT WITH_ROCM) + op_library(fusion_transpose_flatten_concat_op) + file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(fusion_transpose_flatten_concat);\n") + endif() # fusion_conv_inception_op needs cudnn 7 above - if (NOT ${CUDNN_VERSION} VERSION_LESS 7100) + # HIP not support cudnnConvolutionBiasActivationForward + if ((NOT WITH_ROCM) AND (NOT ${CUDNN_VERSION} VERSION_LESS 7100)) op_library(fusion_conv_inception_op) file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(conv2d_inception_fusion);\n") endif() @@ -60,8 +66,9 @@ if (WITH_GPU) cc_test(test_fusion_group_op SRCS fusion_group_op_test.cc DEPS fusion_group_op) endif() # fused_bn_add_activation - if (NOT ${CUDNN_VERSION} VERSION_LESS 7401) - op_library(fused_bn_add_activation_op) - file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(fused_bn_add_activation);\n") + # HIP not support bn act fuse in MIOPEN + if ((NOT WITH_ROCM) AND (NOT ${CUDNN_VERSION} VERSION_LESS 7401)) + op_library(fused_bn_add_activation_op) + file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(fused_bn_add_activation);\n") endif() endif() diff --git a/paddle/fluid/operators/fused/fused_embedding_eltwise_layernorm_op.cu b/paddle/fluid/operators/fused/fused_embedding_eltwise_layernorm_op.cu index d8bd5d03a7d17..9711cc8d811d5 100644 --- a/paddle/fluid/operators/fused/fused_embedding_eltwise_layernorm_op.cu +++ b/paddle/fluid/operators/fused/fused_embedding_eltwise_layernorm_op.cu @@ -12,10 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include -#include // NOLINT #include "paddle/fluid/framework/framework.pb.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/memory/malloc.h" @@ -39,7 +37,11 @@ class EmbeddingEltWiseLayerNormKernel : public framework::OpKernel { in_embs_(framework::proto::VarType::INT64); framework::DDim in_dim{input_num}; int device_id; +#ifdef PADDLE_WITH_HIP + hipGetDevice(&device_id); +#else cudaGetDevice(&device_id); +#endif in_ids_.Resize(in_dim); in_embs_.Resize(in_dim); int64_t *in_ids_d = @@ -52,11 +54,17 @@ class EmbeddingEltWiseLayerNormKernel : public framework::OpKernel { in1s.push_back(reinterpret_cast(ids[i]->data())); in2s.push_back(reinterpret_cast(embs[i]->data())); } - +#ifdef PADDLE_WITH_HIP + hipMemcpyAsync(in_ids_d, in1s.data(), sizeof(int64_t) * input_num, + hipMemcpyHostToDevice, device_ctx.stream()); + hipMemcpyAsync(in_embs_d, in2s.data(), sizeof(int64_t) * input_num, + hipMemcpyHostToDevice, device_ctx.stream()); +#else cudaMemcpyAsync(in_ids_d, in1s.data(), sizeof(int64_t) * input_num, cudaMemcpyHostToDevice, device_ctx.stream()); cudaMemcpyAsync(in_embs_d, in2s.data(), sizeof(int64_t) * input_num, cudaMemcpyHostToDevice, device_ctx.stream()); +#endif auto *bias = context.Input("Bias"); auto *scale = context.Input("Scale"); diff --git a/paddle/fluid/operators/fused/fused_fc_elementwise_layernorm_op.cu b/paddle/fluid/operators/fused/fused_fc_elementwise_layernorm_op.cu index 845966187f9b8..dc068e02be4ec 100644 --- a/paddle/fluid/operators/fused/fused_fc_elementwise_layernorm_op.cu +++ b/paddle/fluid/operators/fused/fused_fc_elementwise_layernorm_op.cu @@ -12,7 +12,14 @@ 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. */ +#ifdef __NVCC__ #include +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/math/blas.h" #include "paddle/fluid/platform/cuda_device_function.h" diff --git a/paddle/fluid/operators/fused/fusion_lstm_op.cc b/paddle/fluid/operators/fused/fusion_lstm_op.cc index f14a05142512a..3c82be2c4e48d 100644 --- a/paddle/fluid/operators/fused/fusion_lstm_op.cc +++ b/paddle/fluid/operators/fused/fusion_lstm_op.cc @@ -249,6 +249,10 @@ void FusionLSTMOpMaker::Make() { AddAttr("use_mkldnn", "(bool, default false) Only used in mkldnn kernel") .SetDefault(false); + AddAttr("force_fp32_output", + "(bool, default false) Force INT8 kernel output FP32, only " + "used in MKL-DNN INT8") + .SetDefault(false); AddComment(R"DOC( Fusion Long-Short Term Memory (LSTM) Operator. This operator fuse the X into LSTM, more details can refer to LSTM op. diff --git a/paddle/fluid/operators/fused/mkldnn/fusion_gru_mkldnn_op.cc b/paddle/fluid/operators/fused/mkldnn/fusion_gru_mkldnn_op.cc index a3b59419b7f4c..8e0627fc15c22 100644 --- a/paddle/fluid/operators/fused/mkldnn/fusion_gru_mkldnn_op.cc +++ b/paddle/fluid/operators/fused/mkldnn/fusion_gru_mkldnn_op.cc @@ -89,6 +89,7 @@ class GRUMKLDNNHandler : public RNNMKLDNNHandler { } } + template std::shared_ptr AcquireWeightXMemory(const Tensor* weight_x, const bool origin_mode) { const std::string wx_key = this->memory_key_ + "@weight_x"; @@ -98,18 +99,18 @@ class GRUMKLDNNHandler : public RNNMKLDNNHandler { if (!memory_p) { auto user_md = MKLDNNMemDesc({1, 1, this->IC, this->G, this->OC}, - MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); + MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); auto user_memory = dnnl::memory(user_md, this->engine_); - auto* weight_x_data = - reinterpret_cast(user_memory.get_data_handle()); - memcpy(weight_x_data, weight_x->data(), - sizeof(float) * this->IC * this->G * this->OC); + auto* weight_x_data = reinterpret_cast(user_memory.get_data_handle()); + memcpy(weight_x_data, weight_x->data(), + sizeof(U) * this->IC * this->G * this->OC); if (origin_mode == false) { for (int64_t i = 0; i < this->IC; ++i) { for (int64_t j = 0; j < this->OC; ++j) { - weight_x_data[j] *= -1; + U minus_one(-1.0f); + weight_x_data[j] = minus_one * weight_x_data[j]; } weight_x_data += 3 * this->OC; } @@ -127,6 +128,7 @@ class GRUMKLDNNHandler : public RNNMKLDNNHandler { return memory_p; } + template std::shared_ptr AcquireWeightHMemory(const Tensor* weight_h, const bool origin_mode) { const std::string wh_key = this->memory_key_ + "@weight_h"; @@ -136,34 +138,33 @@ class GRUMKLDNNHandler : public RNNMKLDNNHandler { if (!memory_p) { auto user_md = MKLDNNMemDesc({1, 1, this->OC, this->G, this->OC}, - MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); + MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); auto user_memory = dnnl::memory(user_md, this->engine_); // Reorder weights_h from PP format [OC, 2OC] + [OC, OC] to // oneDNN format [OC, 3OC] - auto* weight_h_data = - reinterpret_cast(user_memory.get_data_handle()); - auto* user_weight_h_data = weight_h->data(); + auto* weight_h_data = reinterpret_cast(user_memory.get_data_handle()); + auto* user_weight_h_data = weight_h->data(); auto src1_iter = user_weight_h_data; auto src2_iter = user_weight_h_data + 2 * this->OC * this->OC; for (int64_t c = 0; c < this->OC; ++c) { - memcpy(weight_h_data, src1_iter, 2 * this->OC * sizeof(float)); - memcpy(weight_h_data + 2 * this->OC, src2_iter, - this->OC * sizeof(float)); + memcpy(weight_h_data, src1_iter, 2 * this->OC * sizeof(U)); + memcpy(weight_h_data + 2 * this->OC, src2_iter, this->OC * sizeof(U)); src1_iter += 2 * this->OC; src2_iter += this->OC; weight_h_data += 3 * this->OC; } - weight_h_data = reinterpret_cast(user_memory.get_data_handle()); + weight_h_data = reinterpret_cast(user_memory.get_data_handle()); if (origin_mode == false) { for (int64_t i = 0; i < this->OC; ++i) { for (int64_t j = 0; j < this->OC; ++j) { - weight_h_data[j] *= -1; + U minus_one(-1.0f); + weight_h_data[j] = minus_one * weight_h_data[j]; } weight_h_data += 3 * this->OC; } @@ -273,11 +274,34 @@ class FusionGRUMKLDNNKernel : public framework::OpKernel { auto input_memory_p = handler.AcquireInputMemoryWithReorder(input, is_reverse); - auto h0_memory_p = handler.AcquireH0Memory(h0); - auto weight_x_memory_p = - handler.AcquireWeightXMemory(weight_x, origin_mode); - auto weight_h_memory_p = - handler.AcquireWeightHMemory(weight_h, origin_mode); + + std::shared_ptr h0_memory_p, weight_h_memory_p, + weight_x_memory_p; + + if (weight_h->type() == paddle::framework::proto::VarType_Type_FP32) { + h0_memory_p = handler.template AcquireH0Memory(h0); + weight_x_memory_p = + handler.template AcquireWeightXMemory(weight_x, origin_mode); + weight_h_memory_p = + handler.template AcquireWeightHMemory(weight_h, origin_mode); + } else if (weight_h->type() == + paddle::framework::proto::VarType_Type_BF16) { + h0_memory_p = + handler.template AcquireH0Memory(h0); + weight_x_memory_p = + handler.template AcquireWeightXMemory( + weight_x, origin_mode); + weight_h_memory_p = + handler.template AcquireWeightHMemory( + weight_h, origin_mode); + } else { + h0_memory_p = handler.template AcquireH0Memory(h0); + weight_x_memory_p = + handler.template AcquireWeightXMemory(weight_x, origin_mode); + weight_h_memory_p = + handler.template AcquireWeightHMemory(weight_h, origin_mode); + } + auto bias_memory_p = handler.AcquireBiasMemory(bias, origin_mode); auto hidden_onednn_memory_p = handler.AcquireOutputMemory(); diff --git a/paddle/fluid/operators/fused/mkldnn/fusion_lstm_mkldnn_op.cc b/paddle/fluid/operators/fused/mkldnn/fusion_lstm_mkldnn_op.cc index f5ad0644c6aed..cf39968a9004f 100644 --- a/paddle/fluid/operators/fused/mkldnn/fusion_lstm_mkldnn_op.cc +++ b/paddle/fluid/operators/fused/mkldnn/fusion_lstm_mkldnn_op.cc @@ -81,8 +81,11 @@ class LSTMMKLDNNHandler MKLDNNMemoryFormat::tnc); auto h0_md = MKLDNNMemDesc({L, D, N, OC}, MKLDNNGetDataType(), MKLDNNMemoryFormat::ldnc); - auto c0_md = MKLDNNMemDesc({L, D, N, OC}, MKLDNNGetDataType(), - MKLDNNMemoryFormat::ldnc); + auto c0_md = MKLDNNMemDesc( + {L, D, N, OC}, MKLDNNGetDataType(), // Vanilla LSTM and LSTM + // with peepoles has c0 as + // fp32 + MKLDNNMemoryFormat::ldnc); // Create LSTM oneDNN primitive const auto direction = @@ -110,13 +113,14 @@ class LSTMMKLDNNHandler // needed // PaddlePaddle: {c, i, f, o} // oneDNN: {i, f, c, o} - void ReorderGates(float* weights, int64_t I) { + template + void ReorderGates(U* weights, int64_t I) { size_t inner_block_size = this->OC; size_t block_size = inner_block_size * this->G; for (size_t i = 0; i < (size_t)I; ++i) { size_t offset = i * block_size; - float* base_pos = weights + offset; + U* base_pos = weights + offset; std::swap_ranges(base_pos, base_pos + inner_block_size, base_pos + inner_block_size); // c <-> i std::swap_ranges(base_pos + inner_block_size, @@ -125,6 +129,7 @@ class LSTMMKLDNNHandler } } + template std::shared_ptr AcquireWeightXMemory(const Tensor* weight_x) { const std::string wx_key = this->memory_key_ + "@weight_x"; auto memory_p = @@ -133,13 +138,12 @@ class LSTMMKLDNNHandler if (!memory_p) { auto user_md = MKLDNNMemDesc({1, 1, this->IC, this->G, this->OC}, - MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); + MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); auto user_memory = dnnl::memory(user_md, this->engine_); - auto* weight_x_data = - reinterpret_cast(user_memory.get_data_handle()); - memcpy(weight_x_data, weight_x->data(), - sizeof(float) * this->IC * this->G * this->OC); + auto* weight_x_data = reinterpret_cast(user_memory.get_data_handle()); + memcpy(weight_x_data, weight_x->data(), + sizeof(U) * this->IC * this->G * this->OC); ReorderGates(weight_x_data, this->IC); @@ -155,6 +159,7 @@ class LSTMMKLDNNHandler return memory_p; } + template std::shared_ptr AcquireWeightHMemory(const Tensor* weight_h) { const std::string wh_key = this->memory_key_ + "@weight_h"; auto memory_p = @@ -163,13 +168,12 @@ class LSTMMKLDNNHandler if (!memory_p) { auto user_md = MKLDNNMemDesc({1, 1, this->OC, this->G, this->OC}, - MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); + MKLDNNGetDataType(), MKLDNNMemoryFormat::ldigo); auto user_memory = dnnl::memory(user_md, this->engine_); - auto* weight_h_data = - reinterpret_cast(user_memory.get_data_handle()); - memcpy(weight_h_data, weight_h->data(), - sizeof(float) * this->OC * this->G * this->OC); + auto* weight_h_data = reinterpret_cast(user_memory.get_data_handle()); + memcpy(weight_h_data, weight_h->data(), + sizeof(U) * this->OC * this->G * this->OC); ReorderGates(weight_h_data, this->OC); @@ -258,8 +262,8 @@ class LSTMMKLDNNHandler memset(user_c0_memory.get_data_handle(), 0, sizeof(float) * this->N * this->OC); } - memory_p = std::make_shared(this->fwd_pd_->src_iter_desc(), - this->engine_); + memory_p = std::make_shared( + this->fwd_pd_->src_iter_c_desc(), this->engine_); auto& astream = paddle::platform::MKLDNNDeviceContext::tls().get_stream(); dnnl::reorder(user_c0_memory, *memory_p, this->attr_) @@ -275,7 +279,15 @@ template class FusionLSTMMKLDNNKernel : public framework::OpKernel { public: void Compute(const framework::ExecutionContext& ctx) const override { - RunKernel(ctx); + const bool is_bf16 = std::is_same::value; + const bool force_fp32_output = ctx.Attr("force_fp32_output"); + + // BF16 does not support force output + if (!is_bf16 && force_fp32_output) { + RunKernel(ctx); + } else { + RunKernel(ctx); + } } template @@ -327,10 +339,29 @@ class FusionLSTMMKLDNNKernel : public framework::OpKernel { auto input_memory_p = handler.AcquireInputMemoryWithReorder(input, is_reverse); - auto h0_memory_p = handler.AcquireH0Memory(h0); auto c0_memory_p = handler.AcquireC0Memory(c0); - auto weight_x_memory_p = handler.AcquireWeightXMemory(weight_x); - auto weight_h_memory_p = handler.AcquireWeightHMemory(weight_h); + + std::shared_ptr h0_memory_p, weight_h_memory_p, + weight_x_memory_p; + + if (weight_h->type() == paddle::framework::proto::VarType_Type_FP32) { + h0_memory_p = handler.template AcquireH0Memory(h0); + weight_x_memory_p = + handler.template AcquireWeightXMemory(weight_x); + weight_h_memory_p = + handler.template AcquireWeightHMemory(weight_h); + } else if (weight_h->type() == + paddle::framework::proto::VarType_Type_BF16) { + h0_memory_p = + handler.template AcquireH0Memory(h0); + weight_x_memory_p = + handler.template AcquireWeightXMemory( + weight_x); + weight_h_memory_p = + handler.template AcquireWeightHMemory( + weight_h); + } + auto bias_memory_p = handler.AcquireBiasMemory(bias); auto hidden_onednn_memory_p = handler.AcquireOutputMemory(); @@ -374,4 +405,5 @@ class FusionLSTMMKLDNNKernel : public framework::OpKernel { namespace ops = paddle::operators; REGISTER_OP_KERNEL(fusion_lstm, MKLDNN, paddle::platform::CPUPlace, - ops::FusionLSTMMKLDNNKernel); + ops::FusionLSTMMKLDNNKernel, + ops::FusionLSTMMKLDNNKernel); diff --git a/paddle/fluid/operators/fused/mkldnn/fusion_rnn_mkldnn.h b/paddle/fluid/operators/fused/mkldnn/fusion_rnn_mkldnn.h index f102c535fdf56..5ef84eac4e672 100644 --- a/paddle/fluid/operators/fused/mkldnn/fusion_rnn_mkldnn.h +++ b/paddle/fluid/operators/fused/mkldnn/fusion_rnn_mkldnn.h @@ -179,6 +179,7 @@ class RNNMKLDNNHandler : public platform::MKLDNNHandlerT { // TODO(grygielski) H0 is for now persistable // TODO(jczaja) H0 should be updated each iter and of T type (Fusion pass does // not support in yet) + template std::shared_ptr AcquireH0Memory(const Tensor* h0) { const std::string h0_key = memory_key_ + "@h0"; auto memory_p = @@ -187,17 +188,14 @@ class RNNMKLDNNHandler : public platform::MKLDNNHandlerT { if (!memory_p) { auto user_h0_memory = dnnl::memory(); if (h0) { - user_h0_memory = - dnnl::memory({{1, 1, N, OC}, - MKLDNNGetDataType(), - MKLDNNMemoryFormat::ldnc}, - this->engine_, to_void_cast(h0->data())); + user_h0_memory = dnnl::memory( + {{1, 1, N, OC}, MKLDNNGetDataType(), MKLDNNMemoryFormat::ldnc}, + this->engine_, to_void_cast(h0->data())); } else { - user_h0_memory = dnnl::memory({{1, 1, N, OC}, - MKLDNNGetDataType(), - MKLDNNMemoryFormat::ldnc}, - this->engine_); - memset(user_h0_memory.get_data_handle(), 0, sizeof(float) * N * OC); + user_h0_memory = dnnl::memory( + {{1, 1, N, OC}, MKLDNNGetDataType(), MKLDNNMemoryFormat::ldnc}, + this->engine_); + memset(user_h0_memory.get_data_handle(), 0, sizeof(U) * N * OC); } memory_p = std::make_shared(this->fwd_pd_->src_iter_desc(), this->engine_); diff --git a/paddle/fluid/operators/fused/multihead_matmul_op.cu b/paddle/fluid/operators/fused/multihead_matmul_op.cu index fb5ce3468538a..c19e621b18fa7 100644 --- a/paddle/fluid/operators/fused/multihead_matmul_op.cu +++ b/paddle/fluid/operators/fused/multihead_matmul_op.cu @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include "paddle/fluid/framework/op_registry.h" @@ -89,7 +88,7 @@ __global__ void TransposeQkvKernel(const int H, const T *input, const T *bias, void TransQKVWithBias(const int batch, const int seq_len, const int head_size, const int head_num, const float *input, const float *bias, - float *output, cudaStream_t stream) { + float *output, gpuStream_t stream) { // BxSx3xNxH + 3xNxH -> 3xBxNxSxH int scratch_size = batch * head_num * seq_len * seq_len; const dim3 grid(seq_len, batch, 3); diff --git a/paddle/fluid/operators/fused/skip_layernorm_op.cu b/paddle/fluid/operators/fused/skip_layernorm_op.cu index 856d5e694bdf1..74cd9127711b1 100644 --- a/paddle/fluid/operators/fused/skip_layernorm_op.cu +++ b/paddle/fluid/operators/fused/skip_layernorm_op.cu @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include "paddle/fluid/framework/op_registry.h" diff --git a/paddle/fluid/operators/gelu_op.h b/paddle/fluid/operators/gelu_op.h index 936da8dee85fc..0446d7d284b22 100644 --- a/paddle/fluid/operators/gelu_op.h +++ b/paddle/fluid/operators/gelu_op.h @@ -54,7 +54,8 @@ struct GeluFunctor { } } else { #if defined(PADDLE_WITH_MKLML) && !defined(_WIN32) && !defined(__APPLE__) && \ - !defined(__OSX__) && !defined(PADDLE_WITH_CUDA) + !defined(__OSX__) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) auto x_data = x.data(); auto out_data = out.data(); int n = std::min(x.size(), out.size()); @@ -121,7 +122,8 @@ struct GeluGradFunctor { } } else { #if defined(PADDLE_WITH_MKLML) && !defined(_WIN32) && !defined(__APPLE__) && \ - !defined(__OSX__) && !defined(PADDLE_WITH_CUDA) + !defined(__OSX__) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) auto x_data = x.data(); auto dx_data = dx.data(); auto dout_data = dout.data(); diff --git a/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc b/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc index 89a5d81a227af..8ce7df7eec15e 100644 --- a/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc +++ b/paddle/fluid/operators/get_tensor_from_selected_rows_op.cc @@ -107,7 +107,7 @@ REGISTER_OP_CPU_KERNEL_FUNCTOR(get_tensor_from_selected_rows, float, ops::GetTensorFromSelectedRowsKernel, int64_t, ops::GetTensorFromSelectedRowsKernel); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) REGISTER_OP_CUDA_KERNEL_FUNCTOR(get_tensor_from_selected_rows, float, ops::GetTensorFromSelectedRowsKernel, double, ops::GetTensorFromSelectedRowsKernel, int, diff --git a/paddle/fluid/operators/grid_sampler_cudnn_op.cu.cc b/paddle/fluid/operators/grid_sampler_cudnn_op.cu.cc index f0903bdfce920..d2002b487ca33 100644 --- a/paddle/fluid/operators/grid_sampler_cudnn_op.cu.cc +++ b/paddle/fluid/operators/grid_sampler_cudnn_op.cu.cc @@ -12,6 +12,9 @@ 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 PADDLE_WITH_HIP +// HIP not support cudnnSpatialTfGridGeneratorForward + #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/cudnn_helper.h" @@ -140,3 +143,5 @@ REGISTER_OP_KERNEL(grid_sampler, CUDNN, plat::CUDAPlace, REGISTER_OP_KERNEL(grid_sampler_grad, CUDNN, plat::CUDAPlace, paddle::operators::CUDNNGridSampleGradOpKernel, paddle::operators::CUDNNGridSampleGradOpKernel); + +#endif // PADDLE_WITH_HIP diff --git a/paddle/fluid/operators/grid_sampler_op.cc b/paddle/fluid/operators/grid_sampler_op.cc index e357133be440d..a75ea538f2556 100644 --- a/paddle/fluid/operators/grid_sampler_op.cc +++ b/paddle/fluid/operators/grid_sampler_op.cc @@ -20,6 +20,9 @@ limitations under the License. */ #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" #endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -71,7 +74,7 @@ class GridSampleOp : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext& ctx) const override { framework::LibraryType library_{framework::LibraryType::kPlain}; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } @@ -191,7 +194,7 @@ class GridSampleOpGrad : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext& ctx) const override { framework::LibraryType library_{framework::LibraryType::kPlain}; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } diff --git a/paddle/fluid/operators/group_norm_op.cu b/paddle/fluid/operators/group_norm_op.cu index b7f79be45be84..45d97723a3e21 100644 --- a/paddle/fluid/operators/group_norm_op.cu +++ b/paddle/fluid/operators/group_norm_op.cu @@ -12,9 +12,17 @@ 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. */ +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/operators/group_norm_op.h" #include "paddle/fluid/platform/cuda_device_function.h" +#include "paddle/fluid/platform/cuda_primitives.h" namespace paddle { namespace operators { @@ -217,10 +225,10 @@ __global__ void GroupNormBackwardGetMeanAndVar( d_bias_data += dval; d_scale_data += val * dval; } - CudaAtomicAddWithWarp(&d_mean[bid * groups + gid], d_mean_data); - CudaAtomicAddWithWarp(&d_var[bid * groups + gid], d_var_data); - if (flags & kHasScale) CudaAtomicAddWithWarp(&d_scale[ccid], d_scale_data); - if (flags & kHasBias) CudaAtomicAddWithWarp(&d_bias[ccid], d_bias_data); + CudaAtomicAddWithWarp(&(d_mean[bid * groups + gid]), d_mean_data); + CudaAtomicAddWithWarp(&(d_var[bid * groups + gid]), d_var_data); + if (flags & kHasScale) CudaAtomicAddWithWarp(&(d_scale[ccid]), d_scale_data); + if (flags & kHasBias) CudaAtomicAddWithWarp(&(d_bias[ccid]), d_bias_data); } template diff --git a/paddle/fluid/operators/gru_unit_op.h b/paddle/fluid/operators/gru_unit_op.h index 2d1a89f9ae471..b727da4ae0cd3 100644 --- a/paddle/fluid/operators/gru_unit_op.h +++ b/paddle/fluid/operators/gru_unit_op.h @@ -24,13 +24,6 @@ namespace paddle { namespace operators { using Tensor = framework::Tensor; -template -using EigenMatrix = framework::EigenMatrix; - -template -using EigenVector = framework::EigenVector; enum GRUActivationType { identity = 0, sigmoid = 1, tanh = 2, relu = 3 }; @@ -73,17 +66,17 @@ class GRUUnitKernel : public framework::OpKernel { int batch_size = input->dims()[0]; int frame_size = hidden_prev->dims()[1]; - auto x = EigenMatrix::From(*input); - auto h_p = EigenMatrix::From(*hidden_prev); - auto g = EigenMatrix::From(*gate); - auto r_h_p = EigenMatrix::From(*reset_hidden_prev); - auto h = EigenMatrix::From(*hidden); + auto x = framework::EigenMatrix::From(*input); + auto h_p = framework::EigenMatrix::From(*hidden_prev); + auto g = framework::EigenMatrix::From(*gate); + auto r_h_p = framework::EigenMatrix::From(*reset_hidden_prev); + auto h = framework::EigenMatrix::From(*hidden); auto& place = *context.template device_context().eigen_device(); // calculate unactivated gate outputs if (bias) { - auto b = EigenMatrix::From(*bias); + auto b = framework::EigenMatrix::From(*bias); g.device(place) = x + b.reshape(Eigen::array({{1, frame_size * 3}})) .broadcast(Eigen::array({{batch_size, 1}})); @@ -177,11 +170,11 @@ class GRUUnitGradKernel : public framework::OpKernel { T* reset_hidden_prev_grad_data = reset_hidden_prev_grad.mutable_data( reset_hidden_prev->dims(), context.GetPlace()); - auto h_p = EigenMatrix::From(*hidden_prev); - auto g = EigenMatrix::From(*gate); - auto d_h = EigenMatrix::From(*hidden_grad); - auto d_g = EigenMatrix::From(gate_grad); - auto d_r_h_p = EigenMatrix::From(reset_hidden_prev_grad); + auto h_p = framework::EigenMatrix::From(*hidden_prev); + auto g = framework::EigenMatrix::From(*gate); + auto d_h = framework::EigenMatrix::From(*hidden_grad); + auto d_g = framework::EigenMatrix::From(gate_grad); + auto d_r_h_p = framework::EigenMatrix::From(reset_hidden_prev_grad); auto& place = *context.template device_context().eigen_device(); @@ -237,7 +230,7 @@ class GRUUnitGradKernel : public framework::OpKernel { if (hidden_prev_grad) { T* hidden_prev_grad_data = hidden_prev_grad->mutable_data(context.GetPlace()); - auto d_h_p = EigenMatrix::From(*hidden_prev_grad); + auto d_h_p = framework::EigenMatrix::From(*hidden_prev_grad); if (context.Attr("origin_mode")) { d_h_p.device(place) = d_r_h_p * r + d_h * u; } else { @@ -250,13 +243,13 @@ class GRUUnitGradKernel : public framework::OpKernel { // backward for input if (input_grad) { input_grad->mutable_data(context.GetPlace()); - auto d_x = EigenMatrix::From(*input_grad); + auto d_x = framework::EigenMatrix::From(*input_grad); d_x.device(place) = d_g; } // backward for bias if (bias_grad) { bias_grad->mutable_data(context.GetPlace()); - auto d_b = EigenVector::Flatten(*bias_grad); + auto d_b = framework::EigenVector::Flatten(*bias_grad); d_b.device(place) = d_g.sum(Eigen::array({{0}})); } } diff --git a/paddle/fluid/operators/index_select_op.cu b/paddle/fluid/operators/index_select_op.cu index 752e8b277da75..43761d97962a4 100644 --- a/paddle/fluid/operators/index_select_op.cu +++ b/paddle/fluid/operators/index_select_op.cu @@ -106,14 +106,22 @@ class IndexSelectCUDAKernel : public framework::OpKernel { (numel + PADDLE_CUDA_NUM_THREADS - 1) / PADDLE_CUDA_NUM_THREADS, PADDLE_CUDA_NUM_THREADS, 0, stream>>>(in_data, out_data, index_data, numel, stride, size, delta); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } else { const int* index_data = index->data(); index_select_cuda_kernel<<<(numel + PADDLE_CUDA_NUM_THREADS - 1) / PADDLE_CUDA_NUM_THREADS, PADDLE_CUDA_NUM_THREADS, 0, stream>>>( in_data, out_data, index_data, numel, stride, size, delta); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } } }; @@ -164,7 +172,11 @@ class IndexSelectGradCUDAKernel : public framework::OpKernel { PADDLE_CUDA_NUM_THREADS, 0, stream>>>(output_grad_data, in_grad_data, index_data, index_nums, numel, stride, size, delta); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } else { const int* index_data = index->data(); index_select_grad_cuda_kernel<<< @@ -172,7 +184,11 @@ class IndexSelectGradCUDAKernel : public framework::OpKernel { PADDLE_CUDA_NUM_THREADS, 0, stream>>>(output_grad_data, in_grad_data, index_data, index_nums, numel, stride, size, delta); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream)); +#endif } } }; diff --git a/paddle/fluid/operators/inplace_abn_op.cu b/paddle/fluid/operators/inplace_abn_op.cu index 9e12a8291c0f2..be7a7bd71711e 100644 --- a/paddle/fluid/operators/inplace_abn_op.cu +++ b/paddle/fluid/operators/inplace_abn_op.cu @@ -84,9 +84,18 @@ class InplaceABNGradKernel namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL(inplace_abn, + ops::InplaceABNKernel); +REGISTER_OP_CUDA_KERNEL( + inplace_abn_grad, + ops::InplaceABNGradKernel); +#else REGISTER_OP_CUDA_KERNEL(inplace_abn, ops::InplaceABNKernel, ops::InplaceABNKernel); REGISTER_OP_CUDA_KERNEL( inplace_abn_grad, ops::InplaceABNGradKernel, ops::InplaceABNGradKernel); +#endif diff --git a/paddle/fluid/operators/instance_norm_op.cu b/paddle/fluid/operators/instance_norm_op.cu index 51313835ebad4..affd0b7e1edd7 100644 --- a/paddle/fluid/operators/instance_norm_op.cu +++ b/paddle/fluid/operators/instance_norm_op.cu @@ -16,11 +16,22 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/operators/instance_norm_op.h" #include "paddle/fluid/operators/math/math_function.h" +#ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" +#endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -99,6 +110,15 @@ class InstanceNormKernel auto *y = ctx.Output("Y"); y->mutable_data(ctx.GetPlace()); +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t data_desc_; + miopenTensorDescriptor_t in_param_desc_; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&in_param_desc_)); +#else cudnnTensorDescriptor_t data_desc_; cudnnTensorDescriptor_t in_param_desc_; @@ -106,7 +126,7 @@ class InstanceNormKernel platform::dynload::cudnnCreateTensorDescriptor(&data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnCreateTensorDescriptor(&in_param_desc_)); - +#endif if (epsilon <= CUDNN_BN_MIN_EPSILON - FLT_EPSILON) { LOG(ERROR) << "Provided epsilon is smaller than " << "CUDNN_BN_MIN_EPSILON. Setting it to " @@ -122,12 +142,22 @@ class InstanceNormKernel auto &dev_ctx = ctx.template device_context(); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + data_desc_, CudnnDataType::type, + x_dims.size() > 3 ? x_dims.size() : 4, const_cast(dims.data()), + const_cast(strides.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDeriveBNTensorDescriptor( + in_param_desc_, data_desc_, miopenBNSpatial)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetTensorNdDescriptor( data_desc_, CudnnDataType::type, x_dims.size() > 3 ? x_dims.size() : 4, dims.data(), strides.data())); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDeriveBNTensorDescriptor( in_param_desc_, data_desc_, CUDNN_BATCHNORM_SPATIAL)); +#endif const auto *scale = ctx.Input("Scale"); const auto *bias = ctx.Input("Bias"); @@ -171,6 +201,35 @@ class InstanceNormKernel functor(dev_ctx, saved_mean, static_cast>(0)); functor(dev_ctx, saved_variance, static_cast>(0)); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenBatchNormalizationForwardTraining( + handle, miopenBNSpatial, + const_cast( + static_cast(CudnnDataType::kOne())), + const_cast( + static_cast(CudnnDataType::kZero())), + data_desc_, static_cast(x_tmp.template data()), + data_desc_, + static_cast(y->template mutable_data(ctx.GetPlace())), + in_param_desc_, + const_cast(static_cast( + scale_tmp.template data>())), + const_cast(static_cast( + bias_tmp.template data>())), + 0, nullptr, nullptr, epsilon, + static_cast( + saved_mean->template mutable_data>( + ctx.GetPlace())), + static_cast( + saved_variance->template mutable_data>( + ctx.GetPlace())))); + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(in_param_desc_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnBatchNormalizationForwardTraining( handle, CUDNN_BATCHNORM_SPATIAL, CudnnDataType::kOne(), @@ -188,6 +247,7 @@ class InstanceNormKernel platform::dynload::cudnnDestroyTensorDescriptor(data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(in_param_desc_)); +#endif } }; @@ -332,6 +392,15 @@ class InstanceNormGradKernel return; } +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t data_desc_; + miopenTensorDescriptor_t in_param_desc_; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&in_param_desc_)); +#else cudnnTensorDescriptor_t data_desc_; cudnnTensorDescriptor_t in_param_desc_; @@ -339,6 +408,8 @@ class InstanceNormGradKernel platform::dynload::cudnnCreateTensorDescriptor(&data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnCreateTensorDescriptor(&in_param_desc_)); +#endif + if (epsilon <= CUDNN_BN_MIN_EPSILON - FLT_EPSILON) { LOG(ERROR) << "Provided epsilon is smaller than " << "CUDNN_BN_MIN_EPSILON. Setting it to " @@ -346,12 +417,22 @@ class InstanceNormGradKernel } epsilon = std::max(epsilon, CUDNN_BN_MIN_EPSILON); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + data_desc_, CudnnDataType::type, + x_dims.size() > 3 ? x_dims.size() : 4, const_cast(dims.data()), + const_cast(strides.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDeriveBNTensorDescriptor( + in_param_desc_, data_desc_, miopenBNSpatial)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetTensorNdDescriptor( data_desc_, CudnnDataType::type, x_dims.size() > 3 ? x_dims.size() : 4, dims.data(), strides.data())); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDeriveBNTensorDescriptor( in_param_desc_, data_desc_, CUDNN_BATCHNORM_SPATIAL)); +#endif const auto *saved_mean = ctx.Input("SavedMean"); const auto *saved_var = ctx.Input("SavedVariance"); @@ -360,6 +441,21 @@ class InstanceNormGradKernel const auto *saved_var_data = saved_var->template data>(); if (d_scale && d_bias) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenBatchNormalizationBackward( + dev_ctx.cudnn_handle(), miopenBNSpatial, CudnnDataType::kOne(), + CudnnDataType::kZero(), CudnnDataType::kOne(), + CudnnDataType::kZero(), data_desc_, x_tmp.template data(), + data_desc_, d_y_tmp.template data(), data_desc_, + d_x->template mutable_data(ctx.GetPlace()), in_param_desc_, + scale_tmp.template data>(), + d_scale_tmp.template mutable_data>( + ctx.GetPlace()), + d_bias_tmp.template mutable_data>( + ctx.GetPlace()), + epsilon, saved_mean_data, saved_var_data)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnBatchNormalizationBackward( dev_ctx.cudnn_handle(), CUDNN_BATCHNORM_SPATIAL, @@ -373,6 +469,7 @@ class InstanceNormGradKernel d_bias_tmp.template mutable_data>( ctx.GetPlace()), epsilon, saved_mean_data, saved_var_data)); +#endif } else { if (d_x) { GradComputeDX<<>>( @@ -389,10 +486,17 @@ class InstanceNormGradKernel d_bias_tmp.data(), d_bias->data(), N, C); } +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(data_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(in_param_desc_)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(data_desc_)); PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDestroyTensorDescriptor(in_param_desc_)); +#endif } }; @@ -693,6 +797,17 @@ class InstanceNormDoubleGradKernel namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL( + instance_norm, ops::InstanceNormKernel); +REGISTER_OP_CUDA_KERNEL( + instance_norm_grad, + ops::InstanceNormGradKernel); +REGISTER_OP_CUDA_KERNEL(instance_norm_grad_grad, + ops::InstanceNormDoubleGradKernel< + paddle::platform::CUDADeviceContext, float>); +#else REGISTER_OP_CUDA_KERNEL( instance_norm, ops::InstanceNormKernel, ops::InstanceNormKernel); @@ -706,3 +821,4 @@ REGISTER_OP_CUDA_KERNEL( float>, ops::InstanceNormDoubleGradKernel); +#endif diff --git a/paddle/fluid/operators/kldiv_loss_op.h b/paddle/fluid/operators/kldiv_loss_op.h index 857ecda303c26..0bc53d7dd7b3b 100644 --- a/paddle/fluid/operators/kldiv_loss_op.h +++ b/paddle/fluid/operators/kldiv_loss_op.h @@ -19,10 +19,6 @@ namespace paddle { namespace operators { using Tensor = framework::Tensor; -template -using EigenVector = framework::EigenVector; - using Array1 = Eigen::DSizes; template @@ -64,9 +60,9 @@ class KLDivLossKernel : public framework::OpKernel { const int n = input->dims()[0]; loss->mutable_data(ctx.GetPlace()); - auto input_t = EigenVector::Flatten(*input); - auto target_t = EigenVector::Flatten(*target); - auto loss_t = EigenVector::Flatten(*loss); + auto input_t = framework::EigenVector::Flatten(*input); + auto target_t = framework::EigenVector::Flatten(*target); + auto loss_t = framework::EigenVector::Flatten(*loss); auto output = target_t.binaryExpr(input_t, KLDivLossForward()); if ("none" == reduction) { loss_t.device(place) = output; @@ -101,10 +97,10 @@ class KLDivLossGradKernel : public framework::OpKernel { input_grad->mutable_data(ctx.GetPlace()); - auto target_t = EigenVector::Flatten(*target); + auto target_t = framework::EigenVector::Flatten(*target); - auto input_grad_t = EigenVector::Flatten(*input_grad); - auto loss_grad_t = EigenVector::Flatten(*loss_grad); + auto input_grad_t = framework::EigenVector::Flatten(*input_grad); + auto loss_grad_t = framework::EigenVector::Flatten(*loss_grad); auto loss_grad_expand = loss_grad_t.broadcast(Array1(expand)); auto grad_t = target_t * loss_grad_expand; diff --git a/paddle/fluid/operators/kron_op.h b/paddle/fluid/operators/kron_op.h index 2af3716ae4361..6815fd460fa1f 100644 --- a/paddle/fluid/operators/kron_op.h +++ b/paddle/fluid/operators/kron_op.h @@ -18,7 +18,7 @@ limitations under the License. */ #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/for_range.h" -#if __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) #include "paddle/fluid/operators/reduce_ops/cub_reduce.h" #include "thrust/device_vector.h" #endif @@ -87,7 +87,7 @@ struct KronOpFunctor { const int64_t *p_stride_x = nullptr, *p_stride_y = nullptr, *p_stride_out = nullptr, *p_shape_y = nullptr; -#if __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector d_stride_x(ndims); thrust::device_vector d_stride_y(ndims); thrust::device_vector d_stride_out(ndims); @@ -326,7 +326,7 @@ struct KronGradOpFunctor { const int64_t* p_stride_y = nullptr; const int64_t* p_stride_dout = nullptr; const int64_t* p_shape_y = nullptr; -#if __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector d_stride_x(ndims); thrust::device_vector d_stride_y(ndims); thrust::device_vector d_stride_dout(ndims); @@ -369,7 +369,7 @@ struct KronGradOpFunctor { for_range(func); // reduce_sum along aixs 1 -#if __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) auto stream = dev_ctx.stream(); // it is a cuda device_context if (dx) { TensorReduce>( diff --git a/paddle/fluid/operators/layer_norm_op.cu b/paddle/fluid/operators/layer_norm_op.cu index 6883ba009c53d..3656de3525d32 100644 --- a/paddle/fluid/operators/layer_norm_op.cu +++ b/paddle/fluid/operators/layer_norm_op.cu @@ -12,14 +12,25 @@ 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 +#ifdef __NVCC__ +#include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include #include #include "paddle/fluid/framework/ddim.h" #include "paddle/fluid/operators/layer_norm_op.h" -#include "paddle/fluid/platform/cudnn_helper.h" #include "paddle/fluid/platform/float16.h" +#ifdef PADDLE_WITH_CUDA +#include "paddle/fluid/platform/cudnn_helper.h" +#endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -32,7 +43,11 @@ template using LayerNormParamType = typename CudnnDataType::BatchNormParamType; inline static int GetDesiredBlockDim(int block_dim) { +#ifdef __HIPCC__ + const int kMaxBlockDim = 256; +#else const int kMaxBlockDim = 512; +#endif return block_dim >= kMaxBlockDim ? kMaxBlockDim : (1 << (static_cast(std::log2f(block_dim)))); @@ -348,7 +363,11 @@ __global__ void LayerNormBackwardComputeGradInput( // epsilon, const T* gamma, const U *__restrict__ mean, const U *__restrict__ var, const float epsilon, const U *gamma, T *grad_input) { +#ifdef __HIPCC__ + for (auto i1 = hipBlockIdx_y; i1 < n1; i1 += hipGridDim_y) { +#else for (auto i1 = blockIdx.y; i1 < n1; i1 += gridDim.y) { +#endif U sum_loss1 = U(0); U sum_loss2 = U(0); const U c_mean = mean[i1]; @@ -392,12 +411,19 @@ __global__ void LayerNormBackwardComputeGradInput( } // intra-warp reductions for (int mask = BDIMX / 2; mask > 0; mask /= 2) { +#ifdef PADDLE_WITH_HIP + sum_loss1 += __shfl_xor(sum_loss1, mask, + warpSize); // WARP_SHFL_XOR(sum_loss1, mask); + sum_loss2 += __shfl_xor(sum_loss2, mask, + warpSize); // WARP_SHFL_XOR(sum_loss2, mask); +#else sum_loss1 += __shfl_xor_sync(0xffffffff, sum_loss1, mask, warpSize); // WARP_SHFL_XOR(sum_loss1, mask); sum_loss2 += __shfl_xor_sync(0xffffffff, sum_loss2, mask, warpSize); // WARP_SHFL_XOR(sum_loss2, mask); +#endif } // inter-warp reductions if (BDIMY > 1) { @@ -676,8 +702,11 @@ static void LayerNormBackward(const T *x, const T *d_y, const U *scale, const framework::ExecutionContext &ctx) { auto &dev_ctx = ctx.cuda_device_context(); auto stream = dev_ctx.stream(); - +#ifdef __HIPCC__ + const int kMaxBlockDim = 256; +#else const int kMaxBlockDim = 512; +#endif const int kMaxBlockNum = 128; int gradient_flag = ((d_x != nullptr ? 1 : 0) << 2) | ((d_scale != nullptr ? 1 : 0) << 1) | @@ -821,7 +850,7 @@ static void LayerNormBackward(const T *x, const T *d_y, const U *scale, } template -void LayerNormDirectCUDAFunctor::operator()(cudaStream_t stream, +void LayerNormDirectCUDAFunctor::operator()(gpuStream_t stream, const T *input, std::vector input_shape, const T *bias, const T *scale, @@ -942,6 +971,18 @@ template class LayerNormDirectCUDAFunctor; namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL( + layer_norm, + ops::LayerNormKernel, + ops::LayerNormKernel); +REGISTER_OP_CUDA_KERNEL( + layer_norm_grad, + ops::LayerNormGradKernel, + ops::LayerNormGradKernel); +#else REGISTER_OP_CUDA_KERNEL( layer_norm, ops::LayerNormKernel, @@ -953,3 +994,4 @@ REGISTER_OP_CUDA_KERNEL( ops::LayerNormGradKernel, ops::LayerNormGradKernel); +#endif diff --git a/paddle/fluid/operators/layer_norm_op.h b/paddle/fluid/operators/layer_norm_op.h index 931cd6d179491..c9ba37d0008ba 100644 --- a/paddle/fluid/operators/layer_norm_op.h +++ b/paddle/fluid/operators/layer_norm_op.h @@ -51,7 +51,7 @@ struct RowwiseMean2D { const framework::Tensor& input, framework::Tensor* vec); }; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class RowwiseMean2D { public: @@ -97,7 +97,7 @@ struct ColwiseSum2D { const framework::Tensor& input, framework::Tensor* vec); }; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class ColwiseSum2D { public: @@ -163,11 +163,11 @@ using Tensor = framework::Tensor; using LoDTensor = framework::LoDTensor; using DataLayout = framework::DataLayout; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class LayerNormDirectCUDAFunctor { public: - void operator()(cudaStream_t stream, const T* input, + void operator()(gpuStream_t stream, const T* input, std::vector input_shape, const T* bias, const T* scale, T* output, T* mean, T* variance, int begin_norm_axis, float eps); diff --git a/paddle/fluid/operators/linear_chain_crf_op.h b/paddle/fluid/operators/linear_chain_crf_op.h index d4f3fc5d7a622..eacc5f467d229 100644 --- a/paddle/fluid/operators/linear_chain_crf_op.h +++ b/paddle/fluid/operators/linear_chain_crf_op.h @@ -47,9 +47,6 @@ struct ScalarMul { using framework::LoDTensor; using framework::LoD; using framework::Tensor; -template -using EigenMatrix = framework::EigenMatrix; template class LinearChainCRFOpKernel : public framework::OpKernel { @@ -127,16 +124,16 @@ class LinearChainCRFOpKernel : public framework::OpKernel { platform::CPUPlace()); auto& place = *ctx.template device_context() .eigen_device(); - auto x = EigenMatrix::From(emission_weights_tmp); - auto x_row_max = EigenMatrix::From(emission_row_max); + auto x = framework::EigenMatrix::From(emission_weights_tmp); + auto x_row_max = framework::EigenMatrix::From(emission_row_max); x_row_max.device(place) = x.maximum(Eigen::DSizes(1)) .reshape(Eigen::DSizes(static_cast(batch_size), 1)); - auto x_exps = EigenMatrix::From(emission_exps_tmp); + auto x_exps = framework::EigenMatrix::From(emission_exps_tmp); x_exps.device(place) = (x - x_row_max.broadcast(Eigen::DSizes(1, tag_num))).exp(); - auto w = EigenMatrix::From(*transition_weights); - auto w_exps = EigenMatrix::From(*transition_exps); + auto w = framework::EigenMatrix::From(*transition_weights); + auto w_exps = framework::EigenMatrix::From(*transition_exps); w_exps.device(place) = w.exp(); T* log_likelihood = ll->data(); for (int64_t i = 0; i < seq_num; ++i) { @@ -355,9 +352,9 @@ class LinearChainCRFGradOpKernel : public framework::OpKernel { NormalizeL1(beta_value + k * tag_num, tag_num); } - auto x_grad_mat = EigenMatrix::From(*emission_grad); - auto alpha_mat = EigenMatrix::From(alpha); - auto beta_mat = EigenMatrix::From(*beta); + auto x_grad_mat = framework::EigenMatrix::From(*emission_grad); + auto alpha_mat = framework::EigenMatrix::From(alpha); + auto beta_mat = framework::EigenMatrix::From(*beta); auto* place = ctx.eigen_device(); auto prob = alpha_mat * beta_mat; @@ -381,13 +378,13 @@ class LinearChainCRFGradOpKernel : public framework::OpKernel { x_grad_mat(/*to end state*/ seq_length - 1, k); } - auto x_exps_mat = EigenMatrix::From(emission_exps); + auto x_exps_mat = framework::EigenMatrix::From(emission_exps); // TODO(caoying): Fix this to avoid using this local variable if we can // profile the training process. Tensor tmp; tmp.mutable_data(beta->dims(), platform::CPUPlace()); - auto tmp_mat = EigenMatrix::From(tmp); + auto tmp_mat = framework::EigenMatrix::From(tmp); auto prob = beta_mat * x_exps_mat; auto row_sum = prob.sum(Eigen::DSizes(1)) .reshape(Eigen::DSizes(seq_length, 1)) diff --git a/paddle/fluid/operators/lite/CMakeLists.txt b/paddle/fluid/operators/lite/CMakeLists.txt index 96ccdd1f1795c..5bb7892590848 100644 --- a/paddle/fluid/operators/lite/CMakeLists.txt +++ b/paddle/fluid/operators/lite/CMakeLists.txt @@ -1,3 +1,2 @@ op_library(lite_engine_op DEPS lite_engine lite_tensor_utils) -# TODO(wilber): fix the ut. -#cc_test(test_lite_engine_op SRCS lite_engine_op_test.cc DEPS lite_engine_op analysis) +cc_test(test_lite_engine_op SRCS lite_engine_op_test.cc DEPS lite_engine_op analysis) diff --git a/paddle/fluid/operators/lite/lite_engine_op.h b/paddle/fluid/operators/lite/lite_engine_op.h index f6d65704388e6..ec9f5dd95d4d0 100644 --- a/paddle/fluid/operators/lite/lite_engine_op.h +++ b/paddle/fluid/operators/lite/lite_engine_op.h @@ -83,7 +83,7 @@ class LiteEngineOp : public framework::OperatorBase { << engine_->GetInputNames()[i] << ")"; inference::lite::utils::TensorCopy(&dst_t, &src_t, *ctx, zero_copy_); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(dev_place)) { platform::GpuStreamSync( static_cast(ctx)->stream()); @@ -101,7 +101,7 @@ class LiteEngineOp : public framework::OperatorBase { << engine_->GetOutputNames()[i] << ")"; inference::lite::utils::TensorCopy(dst_t, &src_t, *ctx, zero_copy_); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(dev_place)) { platform::GpuStreamSync( static_cast(ctx)->stream()); diff --git a/paddle/fluid/operators/lite/lite_engine_op_test.cc b/paddle/fluid/operators/lite/lite_engine_op_test.cc index 14088351cc895..44ba1e4e497bf 100644 --- a/paddle/fluid/operators/lite/lite_engine_op_test.cc +++ b/paddle/fluid/operators/lite/lite_engine_op_test.cc @@ -67,7 +67,7 @@ TEST(LiteEngineOp, engine_op) { *block_->add_ops() = *elt_add->Proto(); *block_->add_ops() = *fetch->Proto(); framework::Scope scope; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) platform::CUDAPlace place; platform::CUDADeviceContext ctx(place); #else @@ -84,11 +84,11 @@ TEST(LiteEngineOp, engine_op) { std::vector repetitive_params{"x", "y"}; inference::lite::EngineConfig config; config.valid_places = { -#ifdef PADDLE_WITH_CUDA - paddle::lite_api::Place({TARGET(kCUDA), PRECISION(kFloat)}), +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + paddle::lite_api::Place({TARGET(kCUDA), PRECISION(kFloat)}), #endif - paddle::lite_api::Place({TARGET(kX86), PRECISION(kFloat)}), - paddle::lite_api::Place({TARGET(kHost), PRECISION(kAny)}), + paddle::lite_api::Place({TARGET(kX86), PRECISION(kFloat)}), + paddle::lite_api::Place({TARGET(kHost), PRECISION(kAny)}), }; serialize_params(&(config.param), &scope, repetitive_params); config.model = program.Proto()->SerializeAsString(); diff --git a/paddle/fluid/operators/lite/ut_helper.h b/paddle/fluid/operators/lite/ut_helper.h index bc049dae77df6..08dd41e7b341b 100644 --- a/paddle/fluid/operators/lite/ut_helper.h +++ b/paddle/fluid/operators/lite/ut_helper.h @@ -55,7 +55,7 @@ void AddFetchListToBlockDesc(framework::proto::BlockDesc* block, void serialize_params(std::string* str, framework::Scope* scope, const std::vector& params) { std::ostringstream os; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) platform::CUDAPlace place; platform::CUDADeviceContext ctx(place); #else @@ -106,7 +106,7 @@ void CreateTensor(framework::Scope* scope, const std::string& name, tensor->Resize(dims); platform::Place place; if (in_cuda) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) place = platform::CUDAPlace(0); #else PADDLE_THROW(platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/operators/lod_tensor_to_array_op.cc b/paddle/fluid/operators/lod_tensor_to_array_op.cc index cb857e5d90699..e02972bd75353 100644 --- a/paddle/fluid/operators/lod_tensor_to_array_op.cc +++ b/paddle/fluid/operators/lod_tensor_to_array_op.cc @@ -63,7 +63,7 @@ struct LoDTensorToArrayFunctor : public boost::static_visitor { if (std::is_same::value) { Apply(static_cast(dev_ctx)); } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) Apply(static_cast(dev_ctx)); #else PADDLE_THROW( diff --git a/paddle/fluid/operators/lookup_table_op.cc b/paddle/fluid/operators/lookup_table_op.cc index 1b482235da54b..2e8b551ea4e43 100644 --- a/paddle/fluid/operators/lookup_table_op.cc +++ b/paddle/fluid/operators/lookup_table_op.cc @@ -19,6 +19,7 @@ limitations under the License. */ #include "paddle/fluid/framework/no_need_buffer_vars_inference.h" #include "paddle/fluid/framework/op_version_registry.h" #include "paddle/fluid/framework/var_type_inference.h" +#include "paddle/fluid/platform/bfloat16.h" namespace paddle { namespace operators { @@ -222,9 +223,11 @@ REGISTER_OPERATOR(lookup_table_grad, ops::LookupTableOpGrad, REGISTER_OP_CPU_KERNEL(lookup_table, ops::LookupTableKernel, ops::LookupTableKernel, - ops::LookupTableKernel); + ops::LookupTableKernel, + ops::LookupTableKernel); REGISTER_OP_CPU_KERNEL(lookup_table_grad, ops::LookupTableGradKernel, - ops::LookupTableGradKernel); + ops::LookupTableGradKernel, + ops::LookupTableGradKernel); /* ========================== register checkpoint ===========================*/ diff --git a/paddle/fluid/operators/lookup_table_op.cu b/paddle/fluid/operators/lookup_table_op.cu index 3e06e5caed317..6985b91675717 100644 --- a/paddle/fluid/operators/lookup_table_op.cu +++ b/paddle/fluid/operators/lookup_table_op.cu @@ -105,9 +105,24 @@ class LookupTableCUDAKernel : public framework::OpKernel { auto *table = table_t->data(); auto *output = output_t->mutable_data(context.GetPlace()); +#ifdef PADDLE_WITH_HIP + dim3 threads(64, 4); +#else dim3 threads(128, 8); +#endif // PADDLE_WITH_HIP dim3 grids(8, 1); - +#ifdef PADDLE_WITH_HIP + if (padding_idx == -1) + LookupTable< + T, 64, 4, 8, + false><<>>( + output, table, ids, N, K, D, padding_idx); + else + LookupTable< + T, 64, 4, 8, + true><<>>( + output, table, ids, N, K, D, padding_idx); +#else if (padding_idx == -1) LookupTable< T, 128, 8, 8, @@ -118,6 +133,7 @@ class LookupTableCUDAKernel : public framework::OpKernel { T, 128, 8, 8, true><<>>( output, table, ids, N, K, D, padding_idx); +#endif // PADDLE_WITH_HIP } }; @@ -185,10 +201,20 @@ class LookupTableGradCUDAKernel : public framework::OpKernel { auto t = framework::EigenVector::Flatten(*d_table_t); t.device(*dev_ctx.eigen_device()) = t.constant(static_cast(0)); +#ifdef PADDLE_WITH_HIP + dim3 threads(64, 4); +#else dim3 threads(128, 8); +#endif // PADDLE_WITH_HIP dim3 grids(8, 1); + +#ifdef PADDLE_WITH_HIP + LookupTableGrad<<>>( + d_table, d_output, ids, N, K, D); +#else LookupTableGrad<<>>( d_table, d_output, ids, N, K, D); +#endif // PADDLE_WITH_HIP } } }; diff --git a/paddle/fluid/operators/lookup_table_op.h b/paddle/fluid/operators/lookup_table_op.h index 8baa3bccceb1a..e385d72d1f43f 100644 --- a/paddle/fluid/operators/lookup_table_op.h +++ b/paddle/fluid/operators/lookup_table_op.h @@ -102,7 +102,8 @@ class LookupTableKernel : public framework::OpKernel { auto id_index = table_t.GetIndexFromId(ids[i]); if (id_index != -1) { - if (input_data_type == framework::proto::VarType::INT8) { + if (input_data_type == framework::proto::VarType::INT8 || + input_data_type == framework::proto::VarType::BF16) { memcpy(output + i * row_width, table + id_index * row_width, row_width * sizeof(T)); } else { @@ -128,7 +129,8 @@ class LookupTableKernel : public framework::OpKernel { "the input key should be exists. But received %d.", id_index)); - if (input_data_type == framework::proto::VarType::INT8) { + if (input_data_type == framework::proto::VarType::INT8 || + input_data_type == framework::proto::VarType::BF16) { memcpy(output + i * row_width, table + id_index * row_width, row_width * sizeof(T)); } else { diff --git a/paddle/fluid/operators/math/CMakeLists.txt b/paddle/fluid/operators/math/CMakeLists.txt index 2430e68225cbd..fdbc0c68525ba 100644 --- a/paddle/fluid/operators/math/CMakeLists.txt +++ b/paddle/fluid/operators/math/CMakeLists.txt @@ -28,15 +28,12 @@ function(math_library TARGET) if (EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/${TARGET}.cu.cc) list(APPEND cu_srcs ${TARGET}.cu.cc) endif() - if (EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/${TARGET}.hip.cu) - list(APPEND hip_srcs ${TARGET}.hip.cu) - endif() list(LENGTH cc_srcs cc_srcs_len) if (WITH_GPU) nv_library(${TARGET} SRCS ${cc_srcs} ${cu_srcs} DEPS ${math_library_DEPS} ${math_common_deps}) - elseif (WITH_ROCM_PLATFORM AND (${hip_srcs} MATCHES ".*\\.hip.cu$")) - hip_library_ops(${TARGET} SRCS ${cc_srcs} ${hip_srcs} DEPS ${math_library_DEPS} ${math_common_deps}) + elseif (WITH_ROCM) + hip_library(${TARGET} SRCS ${cc_srcs} ${cu_srcs} DEPS ${math_library_DEPS} ${math_common_deps}) elseif(${cc_srcs_len} GREATER 0) cc_library(${TARGET} SRCS ${cc_srcs} DEPS ${math_library_DEPS} ${math_common_deps}) endif() @@ -89,6 +86,10 @@ if(WITH_GPU) nv_test(math_function_gpu_test SRCS math_function_test.cu DEPS math_function) nv_test(selected_rows_functor_gpu_test SRCS selected_rows_functor_test.cu.cc DEPS selected_rows_functor math_function) endif() +if(WITH_ROCM) + hip_test(math_function_gpu_test SRCS math_function_test.cu DEPS math_function tensor) + hip_test(selected_rows_functor_gpu_test SRCS selected_rows_functor_test.cu.cc DEPS selected_rows_functor math_function) +endif() cc_test(concat_test SRCS concat_test.cc DEPS concat_and_split) cc_test(cpu_vec_test SRCS cpu_vec_test.cc DEPS blas cpu_info) if(WITH_TESTING AND TEST im2col_test) diff --git a/paddle/fluid/operators/math/algorithm.h b/paddle/fluid/operators/math/algorithm.h index 2e75b6abce5e1..864cb94cec1e7 100644 --- a/paddle/fluid/operators/math/algorithm.h +++ b/paddle/fluid/operators/math/algorithm.h @@ -41,7 +41,7 @@ HOSTDEVICE inline int64_t BinarySearch(const T *x, int64_t num, const T &val) { template HOSTDEVICE inline size_t LowerBound(const T *x, size_t num, const T &val) { -#ifdef __CUDA_ARCH__ +#if defined(__CUDA_ARCH__) || defined(__HIPCC__) // @{ Group LowerBound // The following code is from // https://en.cppreference.com/w/cpp/algorithm/lower_bound auto *first = x; @@ -59,12 +59,12 @@ HOSTDEVICE inline size_t LowerBound(const T *x, size_t num, const T &val) { return static_cast(first - x); #else return static_cast(std::lower_bound(x, x + num, val) - x); -#endif +#endif // @} End Group LowerBound } template HOSTDEVICE inline size_t UpperBound(const T *x, size_t num, const T &val) { -#ifdef __CUDA_ARCH__ +#if defined(__CUDA_ARCH__) || defined(__HIPCC__) // @{ Group UpperBound // The following code is from // https://en.cppreference.com/w/cpp/algorithm/upper_bound auto *first = x; @@ -82,7 +82,7 @@ HOSTDEVICE inline size_t UpperBound(const T *x, size_t num, const T &val) { return static_cast(first - x); #else return static_cast(std::upper_bound(x, x + num, val) - x); -#endif +#endif // @} End Group UpperBound } } // namespace math diff --git a/paddle/fluid/operators/math/beam_search_test.cc b/paddle/fluid/operators/math/beam_search_test.cc index 97ce3d3f87832..0df06621d9bab 100644 --- a/paddle/fluid/operators/math/beam_search_test.cc +++ b/paddle/fluid/operators/math/beam_search_test.cc @@ -134,7 +134,7 @@ TEST(BeamSearch, CPU) { paddle::platform::CPUPlace>(); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(BeamSearch, GPU) { TestBeamSearch(); diff --git a/paddle/fluid/operators/math/bert_encoder_functor.cu b/paddle/fluid/operators/math/bert_encoder_functor.cu index 2373042815cd0..512f9c62415e5 100644 --- a/paddle/fluid/operators/math/bert_encoder_functor.cu +++ b/paddle/fluid/operators/math/bert_encoder_functor.cu @@ -12,7 +12,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include #include #include "paddle/fluid/framework/tensor.h" #include "paddle/fluid/framework/tensor_util.h" @@ -145,6 +144,8 @@ __global__ void EmbEltwiseLayernormKernel(int hidden, const int64_t *ids, LayerNorm(thread_data, hidden, out_offset, bias, scale, output, eps); } +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#ifndef __HIPCC__ // @{ Half kernel: EmbEltwiseLayernormKernel template <> __global__ void EmbEltwiseLayernormKernel( int hidden, const int64_t *ids, const float *scale, const float *bias, @@ -188,12 +189,13 @@ __global__ void EmbEltwiseLayernormKernel( eps); #endif } +#endif // @} End Half kernel: EmbEltwiseLayernormKernel template void EmbEltwiseLayerNormFunctor::operator()( int batch, int seq_len, int hidden, const int64_t *ids, const float *scale, const float *bias, const int64_t *embs, T *output, float eps, int input_num, - cudaStream_t stream) { + gpuStream_t stream) { const unsigned tpb = 256; const dim3 grid(seq_len, batch, 1); const dim3 block(tpb, 1, 1); @@ -205,7 +207,8 @@ void EmbEltwiseLayerNormFunctor::operator()( template class EmbEltwiseLayerNormFunctor; // device function 'operator()' is not supportted until cuda 10.0 -#if CUDA_VERSION >= 10000 +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#if defined(PADDLE_WITH_CUDA) && CUDA_VERSION >= 10000 template class EmbEltwiseLayerNormFunctor; #endif @@ -230,6 +233,8 @@ __global__ void SoftmaxKernelWithEltadd(T *qk_buf_, const T *bias_qk_, qk_buf_[threadIdx.x + qk_offset] = (T)(qk_tmp / sum_val); } +// HIP defined __HIP_NO_HALF_CONVERSIONS__ +#ifndef __HIPCC__ // @{ Half kernel: SoftmaxKernelWithEltadd template <> __global__ void SoftmaxKernelWithEltadd( half *qk_buf_, const half *bias_qk_, const int batch_size, @@ -251,6 +256,7 @@ __global__ void SoftmaxKernelWithEltadd( qk_buf_[threadIdx.x + qk_offset] = (half)(qk_tmp / sum_val); #endif } +#endif // @} End Half kernel: SoftmaxKernelWithEltadd template __global__ void SoftmaxKernelWithEltadd2(T *qk_buf_, const T *bias_qk_, @@ -282,7 +288,9 @@ __global__ void SoftmaxKernelWithEltadd2( half2 *qk_buf_, const half2 *bias_qk_, const int batch_size, const int head_num, const int seq_len, const unsigned mask) { // operator "+" of half only suppotted after cuda version 10.0 -#if CUDA_ARCH_FP16_SUPPORTED(__CUDA_ARCH__) && CUDA_VERSION >= 10000 +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#if defined(PADDLE_WITH_CUDA) && \ + (CUDA_ARCH_FP16_SUPPORTED(__CUDA_ARCH__) && CUDA_VERSION >= 10000) int qk_offset = blockIdx.x * seq_len; int idx = threadIdx.x; assert(blockDim.x % 32 == 0); @@ -398,7 +406,8 @@ void MultiHeadGPUComputeFunctor::operator()( template class MultiHeadGPUComputeFunctor; // device function 'operator()' is not supportted until cuda 10.0 -#if CUDA_VERSION >= 10000 +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#if defined(PADDLE_WITH_CUDA) && CUDA_VERSION >= 10000 template class MultiHeadGPUComputeFunctor; #endif @@ -422,6 +431,8 @@ __global__ void SkipLayerNormSmallKernel(int num, int hidden, const T *input1, eps); } +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#ifndef __HIPCC__ // @{ Half kernel: SkipLayerNormSmallKernel template <> __global__ void SkipLayerNormSmallKernel( int num, int hidden, const half *input1, const half *input2, half *output, @@ -484,6 +495,7 @@ __global__ void SkipLayerNormSmallKernel( eps); #endif } +#endif // @} End Half kernel: SkipLayerNormSmallKernel template __global__ void SkipLayerNormKernel(int num, int hidden, const T *input1, @@ -505,6 +517,8 @@ __global__ void SkipLayerNormKernel(int num, int hidden, const T *input1, LayerNorm(thread_data, hidden, offset, bias, scale, output, eps); } +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#ifndef __HIPCC__ // @{ Half kernel: SkipLayerNormKernel template <> __global__ void SkipLayerNormKernel(int num, int hidden, const half *input1, @@ -527,6 +541,7 @@ __global__ void SkipLayerNormKernel(int num, int hidden, LayerNorm(thread_data, hidden, offset, bias, scale, output, eps); #endif } +#endif // @} End Half kernel: SkipLayerNormKernel template __global__ void SkipLayerNormKernel2(int num, int hidden, const T2 *input1, @@ -549,6 +564,8 @@ __global__ void SkipLayerNormKernel2(int num, int hidden, const T2 *input1, LayerNorm2(thread_data, hidden, offset, bias, scale, output, eps); } +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#ifndef __HIPCC__ // @{ Half kernel: SkipLayerNormKernel2 template <> __global__ void SkipLayerNormKernel2( int num, int hidden, const half2 *input1, const half2 *input2, @@ -572,13 +589,13 @@ __global__ void SkipLayerNormKernel2( eps); #endif } +#endif // @} End Half kernel: SkipLayerNormKernel2 template void SkipLayerNormFunctor::operator()(const int num, const int hidden, const T *input1, const T *input2, const float *scale, const float *bias, - T *output, T eps, - cudaStream_t stream) { + T *output, T eps, gpuStream_t stream) { int block = num / hidden; if (hidden <= 32) { const int threads = 32; @@ -603,6 +620,8 @@ void SkipLayerNormFunctor::operator()(const int num, const int hidden, reinterpret_cast(output), reinterpret_cast(scale), reinterpret_cast(bias), eps); +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#ifndef __HIPCC__ } else if (std::is_same::value) { SkipLayerNormKernel2<__half, __half2, threads><<>>( @@ -611,6 +630,7 @@ void SkipLayerNormFunctor::operator()(const int num, const int hidden, reinterpret_cast<__half2 *>(output), reinterpret_cast(scale), reinterpret_cast(bias), eps); +#endif } else { assert(false); // should not be here @@ -625,7 +645,8 @@ void SkipLayerNormFunctor::operator()(const int num, const int hidden, template class SkipLayerNormFunctor; // device function 'operator()' is not supportted until cuda 10.0 -#if CUDA_VERSION >= 10000 +// HIP defined __HIP_NO_HALF_CONVERSIONS__ in hip.cmake +#if defined(PADDLE_WITH_CUDA) && CUDA_VERSION >= 10000 template class SkipLayerNormFunctor; #endif diff --git a/paddle/fluid/operators/math/bert_encoder_functor.h b/paddle/fluid/operators/math/bert_encoder_functor.h index fdbddd96a57d2..683606ec73383 100644 --- a/paddle/fluid/operators/math/bert_encoder_functor.h +++ b/paddle/fluid/operators/math/bert_encoder_functor.h @@ -13,9 +13,18 @@ See the License for the specific language governing permissions and limitations under the License. */ #pragma once + +#ifdef PADDLE_WITH_CUDA #include #include #include // NOLINT +#endif +#ifdef PADDLE_WITH_HIP +#include +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/float16.h" @@ -36,7 +45,7 @@ struct CUDATypeTraits { typedef float TYPE; }; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // This functor involves a fusion calculation in Ernie or Bert. // The fusion mode is as follows: // @@ -55,7 +64,7 @@ class EmbEltwiseLayerNormFunctor { public: void operator()(int batch, int seq_len, int hidden, const int64_t *ids, const float *scale, const float *bias, const int64_t *embs, - T *output, float eps, int input_num, cudaStream_t stream); + T *output, float eps, int input_num, gpuStream_t stream); }; // This functor involves a fusion calculation in Ernie or Bert. @@ -97,7 +106,7 @@ class SkipLayerNormFunctor { public: void operator()(const int num, const int hidden, const T *input1, const T *input2, const float *scale, const float *bias, - T *output, T eps, cudaStream_t stream); + T *output, T eps, gpuStream_t stream); }; #endif diff --git a/paddle/fluid/operators/math/blas.h b/paddle/fluid/operators/math/blas.h index 6e61031ec1cdb..bbf7516c538fc 100644 --- a/paddle/fluid/operators/math/blas.h +++ b/paddle/fluid/operators/math/blas.h @@ -102,7 +102,7 @@ class Blas { T alpha, const T* A, int lda, const T* B, int ldb, T beta, T* C, int ldc) const; -#ifdef PADDLE_WITH_MKLML +#ifdef PADDLE_WITH_MKLML // @{ Group MKLML: class Blas template T* GEMM_ALLOC(const CBLAS_IDENTIFIER id, const int M, const int N, const int K) const; @@ -126,7 +126,7 @@ class Blas { const int* indx, const int* pntrb, const int* pntre, const T* b, const int* ldb, const T* beta, T* c, const int* ldc) const; -#if !defined(PADDLE_WITH_CUDA) +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) template void MatMulWithHead(const framework::Tensor& mat_a, const MatDescriptor& dim_a, @@ -135,7 +135,7 @@ class Blas { framework::Tensor* mat_out, T beta, bool mat_y_split_vertical) const; #endif -#endif +#endif // @} End Group MKLML: class Blas template void MatMul(const int M, const int N, const int K, const T* A, const T* B, @@ -210,7 +210,8 @@ class Blas { int K, T alpha, const T** A, const T** B, T beta, T** C, int batchCount) const; -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) template void BatchedGEMMWithHead(CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int W1, int H1, int W2, int H2, T alpha, const T* A, @@ -235,7 +236,7 @@ class Blas { CBLAS_DIAG diag, int M, int N, T alpha, const T* A, int lda, T* B, int ldb) const; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template void BatchedGETRF(int n, T** a, int* ipiv, int* info, int batch_size) const; @@ -262,7 +263,7 @@ class BlasT : private Blas { Base()->template GEMM(args...); } -#ifdef PADDLE_WITH_MKLML +#ifdef PADDLE_WITH_MKLML // @{ Group MKLML: class BlasT template T* GEMM_ALLOC(ARGS... args) const { return Base()->template GEMM_ALLOC(args...); @@ -288,13 +289,13 @@ class BlasT : private Blas { Base()->template CSRMM(args...); } -#if !defined(PADDLE_WITH_CUDA) +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) template void MatMulWithHead(ARGS... args) const { Base()->template MatMulWithHead(args...); } #endif -#endif +#endif // @} End Group MKLML: class BlasT template void MatMul(ARGS... args) const { @@ -386,7 +387,7 @@ class BlasT : private Blas { Base()->template TRSM(args...); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template void BatchedGETRF(ARGS... args) const { Base()->template BatchedGETRF(args...); @@ -429,3 +430,6 @@ inline BlasT GetBlas(const DeviceContext& dev_ctx) { #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/operators/math/blas_impl.cu.h" #endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/operators/math/blas_impl.hip.h" +#endif diff --git a/paddle/fluid/operators/math/blas_impl.h b/paddle/fluid/operators/math/blas_impl.h index 5ccdeabf96bf3..64b533de098ca 100644 --- a/paddle/fluid/operators/math/blas_impl.h +++ b/paddle/fluid/operators/math/blas_impl.h @@ -21,6 +21,7 @@ #include #include "paddle/fluid/operators/math/math_function.h" +#include "paddle/fluid/platform/bfloat16.h" #include "paddle/fluid/platform/complex128.h" #include "paddle/fluid/platform/complex64.h" @@ -40,6 +41,16 @@ struct CBlas { } }; +template <> +struct CBlas { + template + static void VCOPY(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "Blas VCOPY do not supported on CPU with bfloat16," + " please check your code")); + } +}; + #ifdef PADDLE_WITH_MKLML template <> struct CBlas { @@ -1046,7 +1057,8 @@ void Blas::BatchedGEMM( #endif } -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) // @{ Group Blas MKLML: BatchedGEMMWithHead template <> template void Blas::BatchedGEMMWithHead( @@ -1116,7 +1128,7 @@ void Blas::BatchedGEMMWithHead( } } } -#endif +#endif // @} End Group Blas MKLML: BatchedGEMMWithHead template template @@ -1192,7 +1204,9 @@ void Blas::MatMul(const framework::Tensor &mat_a, } } -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) +// @{ Group Blas MKLML: MatMulWithHead /* * Multiple two matrixes with multiple heads * @@ -1319,7 +1333,7 @@ void Blas::MatMulWithHead(const framework::Tensor &mat_a, dim_a.stride_, dim_b.stride_, head_number, mat_b_split_vertical); } } -#endif +#endif // @} End Group Blas MKLML: MatMulWithHead template template diff --git a/paddle/fluid/operators/math/blas_impl.hip.h b/paddle/fluid/operators/math/blas_impl.hip.h new file mode 100644 index 0000000000000..81110b591a1cb --- /dev/null +++ b/paddle/fluid/operators/math/blas_impl.hip.h @@ -0,0 +1,712 @@ +// Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include "paddle/fluid/operators/math/math_function.h" +#include "paddle/fluid/platform/dynload/rocblas.h" +#include "paddle/fluid/platform/gpu_info.h" + +DECLARE_bool(enable_cublas_tensor_op_math); + +namespace paddle { +namespace operators { +namespace math { + +template +struct CUBlas; + +template <> +struct CUBlas { + template + static void GEMM(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_sgemm(args...)); + } + + template + static void AXPY(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_saxpy(args...)); + } + + template + static void SCAL(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_sscal(args...)); + } + + template + static void VCOPY(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_scopy(args...)); + } + + template + static void GEMV(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_sgemv(args...)); + } + + template + static void GEMM_STRIDED_BATCH(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::rocblas_sgemm_strided_batched(args...)); + } + + // HIP not supportted, refer to the doc here: + // https://github.com/ROCm-Developer-Tools/HIP/blob/roc-3.5.x/docs/markdown/CUBLAS_API_supported_by_HIP.md + template + static void GEMM_EX(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasSgemmEx is not supported on HIP platform.")); + } + + template + static void TRSM(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_strsm(args...)); + } + + template + static void GETRF_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasSgetrfBatched is not supported on HIP platform.")); + } + + template + static void GETRI_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasSgetriBatched is not supported on HIP platform.")); + } + + template + static void MATINV_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasSmatinvBatched is not supported on HIP platform.")); + } +}; + +template <> +struct CUBlas { + template + static void GEMM(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_dgemm(args...)); + } + + template + static void AXPY(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_daxpy(args...)); + } + + template + static void SCAL(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_dscal(args...)); + } + + template + static void VCOPY(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_dcopy(args...)); + } + + template + static void GEMV(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_dgemv(args...)); + } + + template + static void GEMM_STRIDED_BATCH(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::rocblas_dgemm_strided_batched(args...)); + } + + template + static void GEMM_EX(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "Currently there are not cublasDgemmEx.")); + } + + template + static void TRSM(ARGS... args) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_dtrsm(args...)); + } + + template + static void GETRF_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasDgetrfBatched is not supported on HIP platform.")); + } + + template + static void GETRI_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasDgetriBatched is not supported on HIP platform.")); + } + + template + static void MATINV_BATCH(ARGS... args) { + PADDLE_THROW(platform::errors::Unimplemented( + "cublasDmatinvBatched is not supported on HIP platform.")); + } +}; + +template <> +struct CUBlas { + using float16 = platform::float16; + + static void GEMM(rocblas_handle handle, rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const float16 *alpha, const float16 *A, int lda, + const float16 *B, int ldb, const float16 *beta, float16 *C, + int ldc) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_hgemm( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, + reinterpret_cast(B), ldb, + reinterpret_cast(beta), + reinterpret_cast(C), ldc)); + } + + static void GEMM_STRIDED_BATCH(rocblas_handle handle, + rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const float16 *alpha, const float16 *A, + int lda, long long int strideA, // NOLINT + const float16 *B, // NOLINT + int ldb, long long int strideB, // NOLINT + const float16 *beta, float16 *C, int ldc, + long long int strideC, // NOLINT + int batchCount) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::rocblas_hgemm_strided_batched( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, strideA, + reinterpret_cast(B), ldb, strideB, + reinterpret_cast(beta), + reinterpret_cast(C), ldc, strideC, batchCount)); + } + + // NOTES: GEMM_EX can use Tensor Core to accelerate matrix multiply. + // https://docs.nvidia.com/cuda/cublas/index.html#cublassetmathmode + template + static void GEMM_EX(platform::CUDADeviceContext *dev_ctx, + rocblas_operation transa, rocblas_operation transb, int m, + int n, int k, const void *alpha, const void *A, + rocblas_datatype Atype, int lda, const void *B, + rocblas_datatype Btype, int ldb, const void *beta, + void *C, rocblas_datatype Ctype, int ldc, + rocblas_datatype computeType) { + rocblas_gemm_algo algo = rocblas_gemm_algo_standard; + dev_ctx->TensorCoreCublasCallIfAvailable([&](rocblas_handle handle) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_gemm_ex( + handle, transa, transb, m, n, k, alpha, A, Atype, lda, B, Btype, ldb, + beta, C, Ctype, ldc, C, Ctype, ldc, computeType, algo, 0, 0)); + }); + } +}; + +template <> +struct CUBlas { + using complex64 = platform::complex64; + + static void GEMV(rocblas_handle handle, rocblas_operation transa, int m, + int n, const complex64 *alpha, const complex64 *A, int lda, + const complex64 *B, int ldb, const complex64 *beta, + complex64 *C, int ldc) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_cgemv( + handle, transa, m, n, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, + reinterpret_cast(B), ldb, + reinterpret_cast(beta), + reinterpret_cast(C), ldc)); + } + + static void AXPY(rocblas_handle handle, int n, const complex64 *alpha, + const complex64 *X, const int incX, complex64 *Y, + const int incY) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_caxpy( + handle, n, reinterpret_cast(alpha), + reinterpret_cast(X), incX, + reinterpret_cast(Y), incY)); + } + + static void GEMM_STRIDED_BATCH(rocblas_handle handle, + rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const complex64 *alpha, const complex64 *A, + int lda, long long int strideA, // NOLINT + const complex64 *B, // NOLINT + int ldb, long long int strideB, // NOLINT + const complex64 *beta, complex64 *C, int ldc, + long long int strideC, // NOLINT + int batchCount) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::rocblas_cgemm_strided_batched( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, strideA, + reinterpret_cast(B), ldb, strideB, + reinterpret_cast(beta), + reinterpret_cast(C), ldc, strideC, + batchCount)); + } + + static void GEMM(rocblas_handle handle, rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const complex64 *alpha, const complex64 *A, int lda, + const complex64 *B, int ldb, const complex64 *beta, + complex64 *C, int ldc) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_cgemm( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, + reinterpret_cast(B), ldb, + reinterpret_cast(beta), + reinterpret_cast(C), ldc)); + } + + // NOTES: GEMM_EX can use Tensor Core to accelerate matrix multiply. + // https://docs.nvidia.com/cuda/cublas/index.html#cublassetmathmode + template + static void GEMM_EX(platform::CUDADeviceContext *dev_ctx, + rocblas_operation transa, rocblas_operation transb, int m, + int n, int k, const void *alpha, const void *A, + rocblas_datatype Atype, int lda, const void *B, + rocblas_datatype Btype, int ldb, const void *beta, + void *C, rocblas_datatype Ctype, int ldc, + rocblas_datatype computeType) { + rocblas_gemm_algo algo = rocblas_gemm_algo_standard; + dev_ctx->TensorCoreCublasCallIfAvailable([&](rocblas_handle handle) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_gemm_ex( + handle, transa, transb, m, n, k, alpha, A, Atype, lda, B, Btype, ldb, + beta, C, Ctype, ldc, C, Ctype, ldc, computeType, algo, 0, 0)); + }); + } +}; + +template <> +struct CUBlas { + using complex128 = platform::complex128; + + static void GEMV(rocblas_handle handle, rocblas_operation transa, int m, + int n, const complex128 *alpha, const complex128 *A, int lda, + const complex128 *B, int ldb, const complex128 *beta, + complex128 *C, int ldc) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_zgemv( + handle, transa, m, n, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, + reinterpret_cast(B), ldb, + reinterpret_cast(beta), + reinterpret_cast(C), ldc)); + } + + static void AXPY(rocblas_handle handle, int n, const complex128 *alpha, + const complex128 *X, const int incX, complex128 *Y, + const int incY) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_zaxpy( + handle, n, reinterpret_cast(alpha), + reinterpret_cast(X), incX, + reinterpret_cast(Y), incY)); + } + + static void GEMM_STRIDED_BATCH(rocblas_handle handle, + rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const complex128 *alpha, const complex128 *A, + int lda, long long int strideA, // NOLINT + const complex128 *B, // NOLINT + int ldb, long long int strideB, // NOLINT + const complex128 *beta, complex128 *C, int ldc, + long long int strideC, // NOLINT + int batchCount) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::rocblas_zgemm_strided_batched( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, strideA, + reinterpret_cast(B), ldb, strideB, + reinterpret_cast(beta), + reinterpret_cast(C), ldc, strideC, + batchCount)); + } + + static void GEMM(rocblas_handle handle, rocblas_operation transa, + rocblas_operation transb, int m, int n, int k, + const complex128 *alpha, const complex128 *A, int lda, + const complex128 *B, int ldb, const complex128 *beta, + complex128 *C, int ldc) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_zgemm( + handle, transa, transb, m, n, k, + reinterpret_cast(alpha), + reinterpret_cast(A), lda, + reinterpret_cast(B), ldb, + reinterpret_cast(beta), + reinterpret_cast(C), ldc)); + } + + // NOTES: GEMM_EX can use Tensor Core to accelerate matrix multiply. + // https://docs.nvidia.com/cuda/cublas/index.html#cublassetmathmode + template + static void GEMM_EX(platform::CUDADeviceContext *dev_ctx, + rocblas_operation transa, rocblas_operation transb, int m, + int n, int k, const void *alpha, const void *A, + rocblas_datatype Atype, int lda, const void *B, + rocblas_datatype Btype, int ldb, const void *beta, + void *C, rocblas_datatype Ctype, int ldc, + rocblas_datatype computeType) { + rocblas_gemm_algo algo = rocblas_gemm_algo_standard; + dev_ctx->TensorCoreCublasCallIfAvailable([&](rocblas_handle handle) { + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::rocblas_gemm_ex( + handle, transa, transb, m, n, k, alpha, A, Atype, lda, B, Btype, ldb, + beta, C, Ctype, ldc, C, Ctype, ldc, computeType, algo, 0, 0)); + }); + } +}; + +template <> +template +void Blas::GEMM(CBLAS_TRANSPOSE transA, + CBLAS_TRANSPOSE transB, int M, + int N, int K, T alpha, const T *A, + const T *B, T beta, T *C) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + int lda = (transA == CblasNoTrans) ? K : M; + int ldb = (transB == CblasNoTrans) ? N : K; + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_operation cuTransB = (transB == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GEMM(handle, cuTransB, cuTransA, N, M, K, &alpha, B, ldb, A, lda, + &beta, C, N); + }); +} + +template <> +template <> +inline void Blas::GEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + platform::float16 alpha, const platform::float16 *A, + const platform::float16 *B, platform::float16 beta, + platform::float16 *C) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + int lda = (transA == CblasNoTrans) ? K : M; + int ldb = (transB == CblasNoTrans) ? N : K; + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_operation cuTransB = (transB == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + + // TODO(kexinzhao): add processing code for compute capability < 53 case + PADDLE_ENFORCE_GE( + context_.GetComputeCapability(), 53, + platform::errors::InvalidArgument( + "cublas fp16 gemm requires GPU compute capability >= 53," + "but received %d", + context_.GetComputeCapability())); + + float h_alpha = static_cast(alpha); + float h_beta = static_cast(beta); + + auto &cuda_ctx = const_cast(context_); + CUBlas::GEMM_EX( + &cuda_ctx, cuTransB, cuTransA, N, M, K, &h_alpha, B, + rocblas_datatype_f16_r, ldb, A, rocblas_datatype_f16_r, lda, &h_beta, C, + rocblas_datatype_f16_r, N, rocblas_datatype_f32_r); +} + +template <> +template <> +inline void Blas::GEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + platform::complex64 alpha, const platform::complex64 *A, + const platform::complex64 *B, platform::complex64 beta, + platform::complex64 *C) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + int lda = (transA == CblasNoTrans) ? K : M; + int ldb = (transB == CblasNoTrans) ? N : K; + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_operation cuTransB = (transB == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + + // TODO(kexinzhao): add processing code for compute capability < 53 case + PADDLE_ENFORCE_GE( + context_.GetComputeCapability(), 53, + platform::errors::InvalidArgument( + "cublas complex64 gemm requires GPU compute capability >= 53," + "but received %d", + context_.GetComputeCapability())); + + thrust::complex c_alpha = + thrust::complex(alpha.real, alpha.imag); + thrust::complex c_beta = thrust::complex(beta.real, beta.imag); + + auto &cuda_ctx = const_cast(context_); + CUBlas::GEMM_EX( + &cuda_ctx, cuTransB, cuTransA, N, M, K, &c_alpha, B, + rocblas_datatype_f32_c, ldb, A, rocblas_datatype_f32_c, lda, &c_beta, C, + rocblas_datatype_f32_c, N, rocblas_datatype_f32_c); +} + +template <> +template <> +inline void Blas::GEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + platform::complex128 alpha, const platform::complex128 *A, + const platform::complex128 *B, platform::complex128 beta, + platform::complex128 *C) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + int lda = (transA == CblasNoTrans) ? K : M; + int ldb = (transB == CblasNoTrans) ? N : K; + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_operation cuTransB = (transB == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + + // TODO(kexinzhao): add processing code for compute capability < 53 case + PADDLE_ENFORCE_GE( + context_.GetComputeCapability(), 53, + platform::errors::InvalidArgument( + "cublas complex128 gemm requires GPU compute capability >= 53," + "but received %d", + context_.GetComputeCapability())); + + thrust::complex c_alpha = + thrust::complex(alpha.real, alpha.imag); + thrust::complex c_beta = + thrust::complex(beta.real, beta.imag); + + auto &cuda_ctx = const_cast(context_); + CUBlas::GEMM_EX( + &cuda_ctx, cuTransB, cuTransA, N, M, K, &c_alpha, B, + rocblas_datatype_f64_c, ldb, A, rocblas_datatype_f64_c, lda, &c_beta, C, + rocblas_datatype_f64_c, N, rocblas_datatype_f64_c); +} + +template <> +template +void Blas::GEMM(bool transA, bool transB, int M, + int N, int K, T alpha, const T *A, + int lda, const T *B, int ldb, + T beta, T *C, int ldc) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + rocblas_operation cuTransA = + transA ? rocblas_operation_transpose : rocblas_operation_none; + rocblas_operation cuTransB = + transB ? rocblas_operation_transpose : rocblas_operation_none; + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GEMM(handle, cuTransB, cuTransA, N, M, K, &alpha, B, ldb, A, lda, + &beta, C, ldc); + }); +} + +template <> +template <> +inline void Blas::GEMM( + bool transA, bool transB, int M, int N, int K, platform::float16 alpha, + const platform::float16 *A, int lda, const platform::float16 *B, int ldb, + platform::float16 beta, platform::float16 *C, int ldc) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + rocblas_operation cuTransA = + transA ? rocblas_operation_transpose : rocblas_operation_none; + rocblas_operation cuTransB = + transB ? rocblas_operation_transpose : rocblas_operation_none; + + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GEMM(handle, cuTransB, cuTransA, N, M, K, &alpha, + B, ldb, A, lda, &beta, C, ldc); + }); +} + +template <> +template +void Blas::AXPY(int n, T alpha, const T *x, + T *y) const { + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::AXPY(handle, n, &alpha, x, 1, y, 1); + }); +} + +template <> +template +void Blas::SCAL(int n, const T alpha, T *x) const { + context_.CublasCall( + [&](rocblas_handle handle) { CUBlas::SCAL(handle, n, &alpha, x, 1); }); +} + +template <> +template +void Blas::VCOPY(int n, const T *x, T *y) const { + context_.CublasCall( + [&](rocblas_handle handle) { CUBlas::VCOPY(handle, n, x, 1, y, 1); }); +} + +template <> +template +void Blas::GEMV(bool trans_a, int M, int N, + T alpha, const T *A, const T *B, + T beta, T *C) const { + rocblas_operation cuTransA = + !trans_a ? rocblas_operation_transpose : rocblas_operation_none; + + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GEMV(handle, cuTransA, N, M, &alpha, A, N, B, 1, &beta, C, 1); + }); +} + +template <> +template <> +inline void Blas::GEMV( + bool trans_a, int M, int N, platform::float16 alpha, + const platform::float16 *A, const platform::float16 *B, + platform::float16 beta, platform::float16 *C) const { + // Because cublas doesn't support half gemv, we use cublasHgemm to achieve it. + if (trans_a) { + this->template GEMM(CblasNoTrans, CblasNoTrans, 1, N, M, + alpha, B, A, beta, C); + } else { + this->template GEMM(CblasNoTrans, CblasNoTrans, M, 1, N, + alpha, A, B, beta, C); + } +} + +template <> +template +void Blas::BatchedGEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + T alpha, const T *A, const T *B, T beta, T *C, int batchCount, + int64_t strideA, int64_t strideB) const { + // Note that cublas follows fortran order, so the order is different from + // the cblas convention. + int lda = (transA == CblasNoTrans) ? K : M; + int ldb = (transB == CblasNoTrans) ? N : K; + int ldc = N; + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_operation cuTransB = (transB == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + const int64_t strideC = M * N; + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GEMM_STRIDED_BATCH(handle, cuTransB, cuTransA, N, M, K, &alpha, + B, ldb, strideB, A, lda, strideA, &beta, C, + ldc, strideC, batchCount); + }); +} + +template <> +template +void Blas::BatchedGEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + T alpha, const T **A, const T **B, T beta, T **C, int batchCount) const { + for (int k = 0; k < batchCount; ++k) { + this->template GEMM(transA, transB, M, N, K, alpha, A[k], B[k], beta, + C[k]); + } +} + +template <> +template <> +inline void Blas::BatchedGEMM( + CBLAS_TRANSPOSE transA, CBLAS_TRANSPOSE transB, int M, int N, int K, + platform::float16 alpha, const platform::float16 **A, + const platform::float16 **B, platform::float16 beta, platform::float16 **C, + int batchCount) const { + for (int k = 0; k < batchCount; ++k) { + this->template GEMM(transA, transB, M, N, K, alpha, A[k], + B[k], beta, C[k]); + } +} + +template <> +template +void Blas::TRSM(CBLAS_SIDE side, CBLAS_UPLO uplo, + CBLAS_TRANSPOSE transA, + CBLAS_DIAG diag, int M, int N, + T alpha, const T *A, int lda, T *B, + int ldb) const { + // solve row major `op ( A ) X = α B` by taking it as `X' op ( A' ) = α B'` + // where ' stands for transpose + rocblas_side cuSide = + (side == CblasLeft) ? rocblas_side_right : rocblas_side_left; + rocblas_fill cuUplo = + (uplo == CblasLower) ? rocblas_fill_upper : rocblas_fill_lower; + // use CUBLAS_OP_C (conjugate transpose) for complex + rocblas_operation cuTransA = (transA == CblasNoTrans) + ? rocblas_operation_none + : rocblas_operation_transpose; + rocblas_diagonal cuDiag = + (diag == CblasUnit) ? rocblas_diagonal_unit : rocblas_diagonal_non_unit; + + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::TRSM(handle, cuSide, cuUplo, cuTransA, cuDiag, N, M, &alpha, A, + lda, B, ldb); + }); +} + +template <> +template +void Blas::BatchedGETRF(int n, T **a, int *ipiv, + int *info, + int batch_size) const { + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GETRF_BATCH(handle, n, a, n, ipiv, info, batch_size); + }); +} + +template <> +template +void Blas::BatchedGETRI(int n, const T **a, + const int *ipiv, T **a_inv, + int *info, + int batch_size) const { + PADDLE_ENFORCE_NE( + a_inv, a, + platform::errors::InvalidArgument( + "cuBLAS fuction 'cublasgetrfBatched' cannot be executed " + "in-place. The memory space of output matrix (address: %p) cannot " + "overlap memory space of input matrix (address: %p).", + a_inv, a)); + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::GETRI_BATCH(handle, n, a, n, ipiv, a_inv, n, info, batch_size); + }); +} + +template <> +template +void Blas::BatchedMatInv(int n, const T **a, + T **a_inv, int *info, + int batch_size) const { + context_.CublasCall([&](rocblas_handle handle) { + CUBlas::MATINV_BATCH(handle, n, a, n, a_inv, n, info, batch_size); + }); +} + +} // namespace math +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/math/concat_test.cc b/paddle/fluid/operators/math/concat_test.cc index 094e2059c4d4c..011c85caf04bb 100644 --- a/paddle/fluid/operators/math/concat_test.cc +++ b/paddle/fluid/operators/math/concat_test.cc @@ -442,7 +442,7 @@ void TestConcatMain() { TEST(math, concat) { TestConcatMain(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TestConcatMain(); #endif diff --git a/paddle/fluid/operators/math/depthwise_conv.cu b/paddle/fluid/operators/math/depthwise_conv.cu index 882b914f94fe4..7439a959d3828 100644 --- a/paddle/fluid/operators/math/depthwise_conv.cu +++ b/paddle/fluid/operators/math/depthwise_conv.cu @@ -14,7 +14,13 @@ limitations under the License. */ #include #include -#include "cub/cub.cuh" +#ifdef __NVCC__ +#include +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/math/depthwise_conv.h" #include "paddle/fluid/platform/cuda_device_function.h" #include "paddle/fluid/platform/cuda_primitives.h" @@ -27,7 +33,14 @@ template __device__ __inline__ void CudaAtomicAddWithWarp(T* sum, T value) { typedef cub::WarpReduce WarpReduce; typename WarpReduce::TempStorage temp_storage; + +#ifdef __HIPCC__ + int block_size = min(blockDim.x * blockDim.y * blockDim.z, warpSize); + value = WarpReduce(temp_storage).Sum(value, block_size); +#else value = WarpReduce(temp_storage).Sum(value); +#endif + if (cub::LaneId() == 0) platform::CudaAtomicAdd(sum, value); } diff --git a/paddle/fluid/operators/math/detail/activation_functions.h b/paddle/fluid/operators/math/detail/activation_functions.h index 883ddec8fa1c3..38bd1a3dadb63 100644 --- a/paddle/fluid/operators/math/detail/activation_functions.h +++ b/paddle/fluid/operators/math/detail/activation_functions.h @@ -130,6 +130,8 @@ struct Active { typedef T (*ActGrad)(T, T); }; +#ifdef PADDLE_WITH_CUDA + static DEVICE Active::Act kActFloat[] = { &forward::Sigmoid, &forward::SigmoidV2, &forward::Relu, &forward::Tanh, @@ -171,6 +173,99 @@ inline DEVICE double activation(double a, double b, int index) { } } // namespace backward +#else // PADDLE_WITH_CUDA + +// Note(qili93): The above implementing not work in HIP +// It will throw compile error when calling detail::forward::lstm() +// Which used ActivationType in lstm_kernel.h, compile error is: +// lstm_gpu_kernel.h:33:17: error: unsupported indirect call to function +// + +// To-do(qili93): fix this after HIP issue fixed: +// https://github.com/ROCm-Developer-Tools/HIP/issues/2186 + +namespace forward { +inline DEVICE float activation(float a, int index) { + switch (index) { + case 0: + return Sigmoid(a); + case 1: + return SigmoidV2(a); + case 2: + return Relu(a); + case 3: + return Tanh(a); + case 4: + return TanhV2(a); + case 5: + return Identity(a); + default: + return 0.0f; + } +} + +inline DEVICE double activation(double a, int index) { + switch (index) { + case 0: + return Sigmoid(a); + case 1: + return SigmoidV2(a); + case 2: + return Relu(a); + case 3: + return Tanh(a); + case 4: + return TanhV2(a); + case 5: + return Identity(a); + default: + return 0.0f; + } +} +} // namespace forward + +namespace backward { +inline DEVICE float activation(float a, float b, int index) { + switch (index) { + case 0: + return Sigmoid(a, b); + case 1: + return Sigmoid(a, b); + case 2: + return Relu(a, b); + case 3: + return Tanh(a, b); + case 4: + return Tanh(a, b); + case 5: + return Identity(a, b); + default: + return 0.0f; + } +} + +inline DEVICE double activation(double a, double b, int index) { + switch (index) { + case 0: + return Sigmoid(a, b); + case 1: + return Sigmoid(a, b); + case 2: + return Relu(a, b); + case 3: + return Tanh(a, b); + case 4: + return Tanh(a, b); + case 5: + return Identity(a, b); + default: + return 0.0f; + } +} +} // namespace backward + +#endif // PADDLE_WITH_CUDA + #ifdef __AVX__ namespace forward { namespace avx { diff --git a/paddle/fluid/operators/math/detail/gru_cpu_kernel.h b/paddle/fluid/operators/math/detail/gru_cpu_kernel.h index 7818e94e37ea0..cbbfbc321b566 100644 --- a/paddle/fluid/operators/math/detail/gru_cpu_kernel.h +++ b/paddle/fluid/operators/math/detail/gru_cpu_kernel.h @@ -28,8 +28,7 @@ template using EigenVector = framework::EigenVector; -#ifndef __NVCC__ - +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group for GRU CPU template void hl_naive_gru_forward_reset_output( OpResetOutput op_reset_output, T *gate_value, T *reset_output_value, @@ -799,7 +798,7 @@ inline void cpu_gru_backward(const platform::CPUDeviceContext &context, } } -#endif +#endif // @} End Group for GRU CPU } // namespace detail } // namespace math diff --git a/paddle/fluid/operators/math/detail/gru_kernel.h b/paddle/fluid/operators/math/detail/gru_kernel.h index faa4a6a06ec98..d9be8e80658fa 100644 --- a/paddle/fluid/operators/math/detail/gru_kernel.h +++ b/paddle/fluid/operators/math/detail/gru_kernel.h @@ -42,7 +42,7 @@ class gru_resetOutput { (*value_reset_output + *value_reset_bias) * (*value_reset_gate); } } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group GRU reset output #ifndef __AVX__ static const bool avx = false; #else @@ -65,7 +65,7 @@ class gru_resetOutput { } } #endif -#endif +#endif // @} End Group GRU reset output }; template @@ -84,7 +84,7 @@ class gru_finalOutput { ((*value_update_gate) * (*value_frame_state)); } } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group GRU final output #ifndef __AVX__ static const bool avx = false; #else @@ -107,7 +107,7 @@ class gru_finalOutput { } } #endif -#endif +#endif // @} End Group GRU final output }; } // namespace forward @@ -137,7 +137,7 @@ class gru_stateGrad { *value_frame_state, act_input); } } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group GRU state grad #ifndef __AVX__ static const bool avx = false; #else @@ -170,7 +170,7 @@ class gru_stateGrad { } } #endif -#endif +#endif // @} End Group GRU state grad }; template @@ -187,7 +187,7 @@ class gru_resetGrad { *grad_reset_gate = activation(*grad_reset_gate, *value_reset_gate, act_gate); } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group GRU reset grad #ifndef __AVX__ static const bool avx = false; #else @@ -206,7 +206,7 @@ class gru_resetGrad { activation(*grad_reset_gate, *value_reset_gate, act_gate); } #endif -#endif +#endif // @} End Group GRU reset grad }; template class gru { @@ -230,7 +230,7 @@ class gru { *value_reset_gate, act_gate); *grad_reset_output = (*value_reset_gate) * (*grad_frame_state); } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group GRU CPU #ifndef __AVX__ static const bool avx = false; #else @@ -261,7 +261,7 @@ class gru { *grad_reset_output = _mm256_mul_ps(*value_reset_gate, *grad_frame_state); } #endif -#endif +#endif // @} End Group GRU CPU }; } // namespace backward diff --git a/paddle/fluid/operators/math/detail/lstm_cpu_kernel.h b/paddle/fluid/operators/math/detail/lstm_cpu_kernel.h index 1e7b4b35f749e..169c5488bb57a 100644 --- a/paddle/fluid/operators/math/detail/lstm_cpu_kernel.h +++ b/paddle/fluid/operators/math/detail/lstm_cpu_kernel.h @@ -35,7 +35,7 @@ template using EigenVector = framework::EigenVector; -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group LSTM CPU template void naive_lstm_forward_one_sequence(Op op, LstmMetaValue value, @@ -467,7 +467,7 @@ void cpu_lstm_backward(const platform::CPUDeviceContext &context, Op op, } } -#endif +#endif // @{ End Group LSTM CPU } // namespace detail } // namespace math diff --git a/paddle/fluid/operators/math/detail/lstm_kernel.h b/paddle/fluid/operators/math/detail/lstm_kernel.h index 8149686c97a03..003ec194366c9 100644 --- a/paddle/fluid/operators/math/detail/lstm_kernel.h +++ b/paddle/fluid/operators/math/detail/lstm_kernel.h @@ -50,7 +50,7 @@ class lstm { *state_atv = activation(*state, active_state); *output = (*value_og) * (*state_atv); } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group LSTM FWD #ifndef __AVX__ // If not compiled with AVX instructs. Disable AVX by default static const bool avx = false; #else @@ -87,7 +87,7 @@ class lstm { *output = _mm256_mul_ps(*value_og, *state_atv); } #endif -#endif +#endif // @} End Group LSTM FWD }; } // namespace forward @@ -132,7 +132,7 @@ class lstm { *checkFGrad = (*grad_fg) * (*prev_state); *checkOGrad = (*grad_og) * (*state); } -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // @{ Group LSTM BWD #ifndef __AVX__ // If not compiled with AVX instructs. Disable AVX by default static const bool avx = false; #else @@ -177,7 +177,7 @@ class lstm { *checkOGrad = _mm256_mul_ps(*grad_og, *state); } #endif -#endif +#endif // @} End Group LSTM BWD }; } // namespace backward diff --git a/paddle/fluid/operators/math/fc.cu b/paddle/fluid/operators/math/fc.cu index 1de3fa44faf1d..69f62d1d53d72 100644 --- a/paddle/fluid/operators/math/fc.cu +++ b/paddle/fluid/operators/math/fc.cu @@ -61,7 +61,7 @@ __global__ void InplaceAddReluKernel(const int N, const T* bias, T* data) { for (int i = threadIdx.x; i < N; i += BlockDim) { T temp; -#if __CUDA_ARCH__ >= 350 +#if defined(__HIPCC__) || __CUDA_ARCH__ >= 350 temp = __ldg(data + offset + i) + __ldg(bias + i); #else temp = data[offset + i] + bias[i]; diff --git a/paddle/fluid/operators/math/gru_compute.cc b/paddle/fluid/operators/math/gru_compute.cc index 6468296546c22..b7a3974ae33e7 100644 --- a/paddle/fluid/operators/math/gru_compute.cc +++ b/paddle/fluid/operators/math/gru_compute.cc @@ -32,7 +32,7 @@ struct GRUUnitFunctor { const detail::ActivationType active_node, const detail::ActivationType active_gate, bool origin_mode) { -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) auto blas = math::GetBlas(context); if (value.prev_out_value) { blas.GEMM(false, false, batch_size, frame_size * 2, frame_size, 1, @@ -66,7 +66,7 @@ struct GRUUnitGradFunctor { const detail::ActivationType active_node, const detail::ActivationType active_gate, bool origin_mode) { -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) detail::backward_state_grad(detail::backward::gru_stateGrad(), value, grad, frame_size, batch_size, active_node, origin_mode); @@ -108,7 +108,7 @@ struct GRUUnitFunctorV2 { GRUMetaValue value, int frame_size, int batch_size, const detail::ActivationType active_node, const detail::ActivationType active_gate) { -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) auto blas = math::GetBlas(context); if (value.prev_out_value) { blas.GEMM(CblasNoTrans, CblasTrans, batch_size, frame_size, frame_size, 1, @@ -142,7 +142,7 @@ struct GRUUnitGradFunctorV2 { int frame_size, int batch_size, const detail::ActivationType active_node, const detail::ActivationType active_gate) { -#ifndef __NVCC__ +#if !defined(__NVCC__) && !defined(__HIPCC___) // calculate grad_update_gate, grad_frame_state, // grad_reset_output, grad_reset_gate detail::cpu_gru_backward(context, detail::backward::gru(), value, grad, diff --git a/paddle/fluid/operators/math/im2col_test.cc b/paddle/fluid/operators/math/im2col_test.cc index e65bda44b3b9e..0122e6cdeb474 100644 --- a/paddle/fluid/operators/math/im2col_test.cc +++ b/paddle/fluid/operators/math/im2col_test.cc @@ -162,7 +162,7 @@ void testIm2col() { TEST(math, im2col) { testIm2col(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) testIm2col(); #endif diff --git a/paddle/fluid/operators/math/math_cuda_utils.h b/paddle/fluid/operators/math/math_cuda_utils.h index 65961f33aa4f9..e97dbd20ca142 100644 --- a/paddle/fluid/operators/math/math_cuda_utils.h +++ b/paddle/fluid/operators/math/math_cuda_utils.h @@ -13,7 +13,14 @@ See the License for the specific language governing permissions and limitations under the License. */ #pragma once + +#ifdef PADDLE_WITH_CUDA #include +#endif +#ifdef PADDLE_WITH_HIP +#include +#endif + #include namespace paddle { @@ -96,7 +103,7 @@ __device__ __forceinline__ float exp_func(float a) { template <> __device__ __forceinline__ half exp_func(half a) { -#if CUDA_ARCH_FP16_SUPPORTED(__CUDA_ARCH__) +#if defined(__HIPCC__) || CUDA_ARCH_FP16_SUPPORTED(__CUDA_ARCH__) return hexp(a); #else return FromFloat(expf(ToFloat(a))); @@ -137,6 +144,7 @@ struct KeyValuePair { operator+(const KeyValuePair &a) const { const half2 a2 = __halves2half2(key, value); const half2 b2 = __halves2half2(a.key, a.value); +#ifdef PADDLE_WITH_CUDA #if CUDA_ARCH_FP16_SUPPORTED(__CUDA_ARCH__) const half2 res = __hadd2(a2, b2); #else @@ -149,6 +157,10 @@ struct KeyValuePair { const half2 res = __floats2half2_rn(r1, r2); #endif return KeyValuePair(res.x, res.y); +#else // PADDLE_WITH_HIP + const half2 res = __hadd2(a2, b2); + return KeyValuePair(__low2half(res), __high2half(res)); +#endif } }; @@ -159,7 +171,7 @@ struct KeyValuePair { template __inline__ __device__ T warpReduceSum(T val, unsigned lane_mask) { for (int mask = HALF_WARP; mask > 0; mask >>= 1) -#if __CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000 +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) val += __shfl_xor_sync(lane_mask, val, mask, warpSize); #else val += __shfl_xor(val, mask, warpSize); @@ -191,7 +203,7 @@ __inline__ __device__ T blockReduceSum(T val, unsigned mask) { template __inline__ __device__ T warpReduceMax(T val, unsigned lane_mask) { for (int mask = HALF_WARP; mask > 0; mask >>= 1) -#if __CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000 +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) val = max(val, __shfl_xor_sync(lane_mask, val, mask, warpSize)); #else val = max(val, __shfl_xor(val, mask, warpSize)); @@ -199,6 +211,39 @@ __inline__ __device__ T warpReduceMax(T val, unsigned lane_mask) { return val; } +template +__inline__ __device__ T warpReduceMin(T val, unsigned lane_mask) { + for (int mask = HALF_WARP; mask > 0; mask >>= 1) +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) + val = min(val, __shfl_xor_sync(lane_mask, val, mask, warpSize)); +#else + val = min(val, __shfl_xor(val, mask, warpSize)); +#endif + return val; +} + +/* Calculate the minimum of all elements in a warp when actual quantity of + * threads are less than warpSize.*/ +template +__inline__ __device__ T PartialWarpReduceMin(T val, unsigned lane_mask) { +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) + T warp_val = __shfl_sync(lane_mask, val, 0, warpSize); +#else + T warp_val = __shfl( + val, 0, warpSize); // To fullfill the data in each thread of this warp. +#endif + warp_val = val; + + for (int offset = HALF_WARP; offset > 0; offset >>= 1) +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) + warp_val = + min(warp_val, __shfl_down_sync(lane_mask, warp_val, offset, warpSize)); +#else + warp_val = min(warp_val, __shfl_down(warp_val, offset, warpSize)); +#endif + return warp_val; +} + /* Calculate the maximum of all elements in a block */ template __inline__ __device__ T blockReduceMax(T val, unsigned mask) { @@ -220,6 +265,55 @@ __inline__ __device__ T blockReduceMax(T val, unsigned mask) { return val; } +/* Calculate the minimum of all elements in a block */ +template +__inline__ __device__ T blockReduceMin(T val, unsigned mask) { + static __shared__ T shared[WARP_SIZE]; + int lane = threadIdx.x & 0x1f; + int wid = threadIdx.x >> 5; + + val = warpReduceMin(val, mask); + if (lane == 0) shared[wid] = val; + __syncthreads(); + + // align block_span to warpSize + int block_span = (blockDim.x + warpSize - 1) >> 5; + val = (lane < block_span) ? shared[lane] : 1e10f; + val = warpReduceMin(val, mask); + + return val; +} + +/* Calculate the minimum of all elements in a warp when actual quantity of + * threads are less than warpSize.*/ +template +__inline__ __device__ T PartialBlockReduceMin(T val, unsigned mask) { + static __shared__ T shared[WARP_SIZE]; + static __shared__ T min_value; + int lane = threadIdx.x & 0x1f; + int wid = threadIdx.x >> 5; + + val = PartialWarpReduceMin(val, mask); + if (lane == 0) shared[wid] = val; + __syncthreads(); + + shared[lane] = PartialWarpReduceMin(shared[lane], mask); +#if defined(PADDLE_WITH_HIP) + // HIP do not support __syncwarp, using __syncthreads() instead is ok, + // although bringing a few performance decrease. + __syncthreads(); +#else + __syncwarp(); +#endif + +#if defined(PADDLE_WITH_CUDA) && (__CUDA_ARCH__ >= 350 && CUDA_VERSION >= 9000) + val = __shfl_sync(mask, shared[lane], 0, warpSize); +#else + val = __shfl(shared[lane], 0, warpSize); +#endif + return val; +} + } // namespace math } // namespace operators } // namespace paddle diff --git a/paddle/fluid/operators/math/math_function.cc b/paddle/fluid/operators/math/math_function.cc index 5afda787339db..a61b50faa757c 100644 --- a/paddle/fluid/operators/math/math_function.cc +++ b/paddle/fluid/operators/math/math_function.cc @@ -180,7 +180,7 @@ struct TensorSetConstantWithPlace : public boost::static_visitor { void set_constant(const platform::DeviceContext& context, framework::Tensor* tensor, float value) { TensorSetConstantWithPlace func(context, tensor, value); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) tensor->place().apply_visitor(func); #else func(platform::CPUPlace()); diff --git a/paddle/fluid/operators/math/pooling.cc b/paddle/fluid/operators/math/pooling.cc index 4df49a1b69886..f2e5e955ec487 100644 --- a/paddle/fluid/operators/math/pooling.cc +++ b/paddle/fluid/operators/math/pooling.cc @@ -30,8 +30,9 @@ class Pool2dFunctor { void operator()(const platform::CPUDeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_height = input.dims()[2]; const int input_width = input.dims()[3]; @@ -104,8 +105,8 @@ class Pool2dFunctor { const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_process) { bool channel_last = (data_format == "NHWC"); const int batch_size = input.dims()[0]; @@ -249,8 +250,8 @@ class Pool2dGradFunctor { const platform::CPUDeviceContext& context, const framework::Tensor& input, const framework::Tensor& output, const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_grad_process, - bool exclusive, bool adaptive, framework::Tensor* input_grad) { + const std::vector& paddings, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_grad_process) { const int batch_size = input.dims()[0]; const int input_height = input.dims()[2]; const int input_width = input.dims()[3]; @@ -328,8 +329,8 @@ class Pool2dGradFunctor { const framework::Tensor& output, const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, const std::string data_format, - PoolProcess pool_grad_process, bool exclusive, bool adaptive, - framework::Tensor* input_grad) { + bool exclusive, bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_grad_process) { bool channel_last = (data_format == "NHWC"); const int batch_size = input.dims()[0]; @@ -678,8 +679,9 @@ class Pool3dFunctor { void operator()(const platform::CPUDeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_depth = input.dims()[2]; const int input_height = input.dims()[3]; @@ -773,8 +775,8 @@ class Pool3dFunctor { const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_process) { bool channel_last = (data_format == "NDHWC"); const int batch_size = input.dims()[0]; @@ -970,8 +972,8 @@ class Pool3dGradFunctor { const platform::CPUDeviceContext& context, const framework::Tensor& input, const framework::Tensor& output, const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_grad_process, - bool exclusive, bool adaptive, framework::Tensor* input_grad) { + const std::vector& paddings, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_grad_process) { const int batch_size = input.dims()[0]; const int input_depth = input.dims()[2]; const int input_height = input.dims()[3]; @@ -1071,8 +1073,8 @@ class Pool3dGradFunctor { const framework::Tensor& output, const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, const std::string data_format, - PoolProcess pool_grad_process, bool exclusive, bool adaptive, - framework::Tensor* input_grad) { + bool exclusive, bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_grad_process) { bool channel_last = (data_format == "NDHWC"); const int batch_size = input.dims()[0]; diff --git a/paddle/fluid/operators/math/pooling.cu b/paddle/fluid/operators/math/pooling.cu index b64dbb771a339..e51fb4204b8cb 100644 --- a/paddle/fluid/operators/math/pooling.cu +++ b/paddle/fluid/operators/math/pooling.cu @@ -237,8 +237,8 @@ void Pool2dDirectCUDAFunctor::operator()( const T* input, const std::vector& input_shape, const std::vector& output_shape, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - PoolProcess pool_compute, bool exclusive, bool adaptive, T* output, - cudaStream_t stream) { + bool exclusive, bool adaptive, T* output, gpuStream_t stream, + PoolProcess pool_compute) { const int batch_size = input_shape[0]; const int input_channels = input_shape[1]; const int input_height = input_shape[2]; @@ -277,8 +277,9 @@ class Pool2dFunctor { void operator()(const platform::CUDADeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_channels = input.dims()[1]; const int input_height = input.dims()[2]; @@ -311,8 +312,8 @@ class Pool2dFunctor { const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_process) { bool channel_last = (data_format == "NHWC"); const int batch_size = input.dims()[0]; @@ -367,9 +368,9 @@ class Pool2dGradFunctor { const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, - framework::Tensor* input_grad) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_channels = input.dims()[1]; const int input_height = input.dims()[2]; @@ -399,13 +400,15 @@ class Pool2dGradFunctor { ksize_width, stride_height, stride_width, padding_height, padding_width, pool_process, exclusive, adaptive, input_grad_data); } - void operator()( - const platform::CUDADeviceContext& context, - const framework::Tensor& input, const framework::Tensor& output, - const framework::Tensor& output_grad, const std::vector& ksize, - const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, bool exclusive, - bool adaptive, framework::Tensor* input_grad) { + void operator()(const platform::CUDADeviceContext& context, + const framework::Tensor& input, + const framework::Tensor& output, + const framework::Tensor& output_grad, + const std::vector& ksize, + const std::vector& strides, + const std::vector& paddings, + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_process) { bool channel_last = (data_format == "NHWC"); const int batch_size = input.dims()[0]; @@ -881,8 +884,9 @@ class Pool3dFunctor { void operator()(const platform::CUDADeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_channels = input.dims()[1]; const int input_depth = input.dims()[2]; @@ -922,8 +926,8 @@ class Pool3dFunctor { const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, - bool exclusive, bool adaptive, framework::Tensor* output) { + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_process) { bool channel_last = (data_format == "NDHWC"); const int batch_size = input.dims()[0]; @@ -988,9 +992,9 @@ class Pool3dGradFunctor { const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_process, - bool exclusive, bool adaptive, - framework::Tensor* input_grad) { + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_process) { const int batch_size = input.dims()[0]; const int input_channels = input.dims()[1]; const int input_depth = input.dims()[2]; @@ -1028,13 +1032,15 @@ class Pool3dGradFunctor { stride_height, stride_width, padding_depth, padding_height, padding_width, pool_process, exclusive, adaptive, input_grad_data); } - void operator()( - const platform::CUDADeviceContext& context, - const framework::Tensor& input, const framework::Tensor& output, - const framework::Tensor& output_grad, const std::vector& ksize, - const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_process, bool exclusive, - bool adaptive, framework::Tensor* input_grad) { + void operator()(const platform::CUDADeviceContext& context, + const framework::Tensor& input, + const framework::Tensor& output, + const framework::Tensor& output_grad, + const std::vector& ksize, + const std::vector& strides, + const std::vector& paddings, + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_process) { bool channel_last = (data_format == "NDHWC"); const int batch_size = input.dims()[0]; diff --git a/paddle/fluid/operators/math/pooling.h b/paddle/fluid/operators/math/pooling.h index 5a6ae224789a2..21d588cc01f32 100644 --- a/paddle/fluid/operators/math/pooling.h +++ b/paddle/fluid/operators/math/pooling.h @@ -97,7 +97,7 @@ HOSTDEVICE inline int AdaptEndIndex(int ph, int input_size, int output_size) { * This is different from average pooling. So we rewrite the max_pool_grad: * MaxPool2dGradFunctor, MaxPool3dGradFunctor. */ -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class Pool2dDirectCUDAFunctor { public: @@ -105,9 +105,9 @@ class Pool2dDirectCUDAFunctor { const std::vector& output_shape, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_compute, - bool exclusive, bool adaptive, T* output, - cudaStream_t stream); + const std::vector& paddings, bool exclusive, + bool adaptive, T* output, gpuStream_t stream, + PoolProcess pool_compute); }; #endif @@ -117,16 +117,17 @@ class Pool2dFunctor { void operator()(const DeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* output); + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_compute); // overload operator() to support argument data_format void operator()(const DeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* output); + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_compute); }; template @@ -137,8 +138,9 @@ class Pool2dGradFunctor { const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* input_grad); + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_compute); // overload operator() to support argument data_format void operator()(const DeviceContext& context, const framework::Tensor& input, const framework::Tensor& output, @@ -146,8 +148,8 @@ class Pool2dGradFunctor { const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* input_grad); + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_compute); }; template @@ -176,15 +178,16 @@ class Pool3dFunctor { void operator()(const DeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* output); + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* output, + PoolProcess pool_compute); // overload operator() to support argument data_format void operator()(const DeviceContext& context, const framework::Tensor& input, const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* output); + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* output, PoolProcess pool_compute); }; template @@ -195,8 +198,9 @@ class Pool3dGradFunctor { const framework::Tensor& output_grad, const std::vector& ksize, const std::vector& strides, - const std::vector& paddings, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* input_grad); + const std::vector& paddings, bool exclusive, + bool adaptive, framework::Tensor* input_grad, + PoolProcess pool_compute); // overload operator() to support argument data_format void operator()(const DeviceContext& context, const framework::Tensor& input, const framework::Tensor& output, @@ -204,8 +208,8 @@ class Pool3dGradFunctor { const std::vector& ksize, const std::vector& strides, const std::vector& paddings, - const std::string data_format, PoolProcess pool_compute, - bool exclusive, bool adaptive, framework::Tensor* input_grad); + const std::string data_format, bool exclusive, bool adaptive, + framework::Tensor* input_grad, PoolProcess pool_compute); }; template diff --git a/paddle/fluid/operators/math/prelu.cu b/paddle/fluid/operators/math/prelu.cu index 323c3ad30649e..42c4c799c574f 100644 --- a/paddle/fluid/operators/math/prelu.cu +++ b/paddle/fluid/operators/math/prelu.cu @@ -61,7 +61,7 @@ __global__ void PReluScalarKernel(const T *input, const T *alpha, T *output, template void PreluChannelWiseDirectCUDAFunctor::operator()( - cudaStream_t stream, const T *input, const T *alpha, T *output, + gpuStream_t stream, const T *input, const T *alpha, T *output, size_t batch_size, size_t channel, size_t numel) { PReluChannelWiseKernel<<>>(input, alpha, output, channel, @@ -69,7 +69,7 @@ void PreluChannelWiseDirectCUDAFunctor::operator()( } template -void PreluElementWiseDirectCUDAFunctor::operator()(cudaStream_t stream, +void PreluElementWiseDirectCUDAFunctor::operator()(gpuStream_t stream, const T *input, const T *alpha, T *output, size_t batch_size, @@ -80,7 +80,7 @@ void PreluElementWiseDirectCUDAFunctor::operator()(cudaStream_t stream, } template -void PreluScalarDirectCUDAFunctor::operator()(cudaStream_t stream, +void PreluScalarDirectCUDAFunctor::operator()(gpuStream_t stream, const T *input, const T *alpha, T *output, size_t numel) { PReluScalarKernel<<>>( diff --git a/paddle/fluid/operators/math/prelu.h b/paddle/fluid/operators/math/prelu.h index 93c7035d4496a..efa493a06c47f 100644 --- a/paddle/fluid/operators/math/prelu.h +++ b/paddle/fluid/operators/math/prelu.h @@ -16,32 +16,36 @@ limitations under the License. */ #include #include "paddle/fluid/operators/math/math_function.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif namespace paddle { namespace operators { namespace math { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class PreluChannelWiseDirectCUDAFunctor { public: - void operator()(cudaStream_t stream, const T *input, const T *alpha, - T *output, size_t batch_size, size_t channel, size_t numel); + void operator()(gpuStream_t stream, const T *input, const T *alpha, T *output, + size_t batch_size, size_t channel, size_t numel); }; template class PreluElementWiseDirectCUDAFunctor { public: - void operator()(cudaStream_t stream, const T *input, const T *alpha, - T *output, size_t batch_size, size_t numel); + void operator()(gpuStream_t stream, const T *input, const T *alpha, T *output, + size_t batch_size, size_t numel); }; template class PreluScalarDirectCUDAFunctor { public: - void operator()(cudaStream_t stream, const T *input, const T *alpha, - T *output, size_t numel); + void operator()(gpuStream_t stream, const T *input, const T *alpha, T *output, + size_t numel); }; #endif diff --git a/paddle/fluid/operators/math/sample_prob.cu b/paddle/fluid/operators/math/sample_prob.cu index 6aabfb069454e..446acc033eb7f 100644 --- a/paddle/fluid/operators/math/sample_prob.cu +++ b/paddle/fluid/operators/math/sample_prob.cu @@ -142,16 +142,30 @@ void GPUSampleWithProb::operator()( int num_tries = UniqSampler(sampler, num_samples, s_data); VLOG(1) << "num_tries: " << num_tries; + +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipMemcpy(samples_data + num_true, s_data, + sizeof(int64_t) * num_samples, + hipMemcpyHostToDevice)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaMemcpy(samples_data + num_true, s_data, sizeof(int64_t) * num_samples, cudaMemcpyHostToDevice)); +#endif int threads = 512; const size_t size = batch_size * num_sampled_classes; int grid = (batch_size * num_sampled_classes + threads - 1) / threads; +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL(HIP_KERNEL_NAME(SamplingCondidate), dim3(grid), + dim3(threads), 0, context.stream(), size, num_tries, range, + log_range, num_true, num_samples, label_data, samples_data, + probabilities_data); +#else SamplingCondidate<<>>( size, num_tries, range, log_range, num_true, num_samples, label_data, samples_data, probabilities_data); +#endif } template class GPUSampleWithProb; diff --git a/paddle/fluid/operators/math/sample_prob.h b/paddle/fluid/operators/math/sample_prob.h index 3653ccb693cf2..8968ba546ad75 100644 --- a/paddle/fluid/operators/math/sample_prob.h +++ b/paddle/fluid/operators/math/sample_prob.h @@ -110,7 +110,7 @@ class SampleWithProb { } }; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template class GPUSampleWithProb { public: diff --git a/paddle/fluid/operators/math/selected_rows_functor_test.cu.cc b/paddle/fluid/operators/math/selected_rows_functor_test.cu.cc index 5cb1cc5dc0371..ebcd97b32c4a3 100644 --- a/paddle/fluid/operators/math/selected_rows_functor_test.cu.cc +++ b/paddle/fluid/operators/math/selected_rows_functor_test.cu.cc @@ -37,9 +37,15 @@ TEST(selected_rows_functor, gpu_add) { {static_cast(rows1.size()), row_numel}), gpu_place); functor(ctx, in1_value, 1.0); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_EQ(hipDeviceSynchronize(), 0, + paddle::platform::errors::PreconditionNotMet( + "The all synchronization on the cuda is error!")); +#else PADDLE_ENFORCE_EQ(cudaDeviceSynchronize(), 0, paddle::platform::errors::PreconditionNotMet( "The all synchronization on the cuda is error!")); +#endif std::vector rows2{0, 5, 7, 9}; std::unique_ptr selected_rows2{ diff --git a/paddle/fluid/operators/math/sequence_padding_test.cc b/paddle/fluid/operators/math/sequence_padding_test.cc index 1f7e9f9ae053f..ea31b10c5558f 100644 --- a/paddle/fluid/operators/math/sequence_padding_test.cc +++ b/paddle/fluid/operators/math/sequence_padding_test.cc @@ -105,7 +105,7 @@ TEST(Seq2BatchPadding, CPU) { 128); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(SequencePadding, CUDA) { auto place = paddle::platform::CUDAPlace(0); auto *context = static_cast( diff --git a/paddle/fluid/operators/math/sequence_pooling_test.cc b/paddle/fluid/operators/math/sequence_pooling_test.cc index 4ece42ab8066b..775d8029bfd3a 100644 --- a/paddle/fluid/operators/math/sequence_pooling_test.cc +++ b/paddle/fluid/operators/math/sequence_pooling_test.cc @@ -123,7 +123,7 @@ TEST(SequencePoolingGrad, CPU_SUM) { lod2, 128); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(SequencePoolingGrad, CUDA_SUM) { auto place = paddle::platform::CUDAPlace(0); auto *context = static_cast( diff --git a/paddle/fluid/operators/math/sequence_scale.cu b/paddle/fluid/operators/math/sequence_scale.cu index 4a952afe15f75..5578f1f0138c4 100644 --- a/paddle/fluid/operators/math/sequence_scale.cu +++ b/paddle/fluid/operators/math/sequence_scale.cu @@ -44,10 +44,18 @@ class ScaleLoDTensorFunctor { framework::LoD abs_offset_lod = framework::ToAbsOffset(lod); T* seq_data = seq->mutable_data(context.GetPlace()); +#ifdef PADDLE_WITH_HIP + hipLaunchKernelGGL( + HIP_KERNEL_NAME(SequenceScaleKernel), + dim3(num_seq), dim3(PADDLE_CUDA_NUM_THREADS), 0, context.stream(), + seq_data, abs_offset_lod[level].CUDAMutableData(context.GetPlace()), + scales, seq_width); +#else SequenceScaleKernel<<< num_seq, PADDLE_CUDA_NUM_THREADS, 0, context.stream()>>>( seq_data, abs_offset_lod[level].CUDAMutableData(context.GetPlace()), scales, seq_width); +#endif } }; diff --git a/paddle/fluid/operators/math/softmax.cu b/paddle/fluid/operators/math/softmax.cu index 742dc7f4449e2..879e367281c0a 100644 --- a/paddle/fluid/operators/math/softmax.cu +++ b/paddle/fluid/operators/math/softmax.cu @@ -16,7 +16,11 @@ limitations under the License. */ #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/math/softmax.h" #include "paddle/fluid/operators/math/softmax_impl.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif namespace paddle { namespace operators { @@ -45,6 +49,16 @@ void SoftmaxCUDNNFunctor::operator()( if (cudnn_tensor_dims.size() <= 2) { cudnn_tensor_dims.resize(4, 1); } +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t cudnn_x_desc = + xDesc.descriptor(layout, cudnn_tensor_dims); + miopenTensorDescriptor_t cudnn_y_desc = + xDesc.descriptor(layout, cudnn_tensor_dims); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSoftmaxForward( + context.cudnn_handle(), CudnnDataType::kOne(), cudnn_x_desc, + X->data(), CudnnDataType::kZero(), cudnn_y_desc, + Y->mutable_data(context.GetPlace()))); +#else cudnnTensorDescriptor_t cudnn_x_desc = xDesc.descriptor(layout, cudnn_tensor_dims); cudnnTensorDescriptor_t cudnn_y_desc = @@ -54,6 +68,7 @@ void SoftmaxCUDNNFunctor::operator()( CUDNN_SOFTMAX_MODE_INSTANCE, CudnnDataType::kOne(), cudnn_x_desc, X->data(), CudnnDataType::kZero(), cudnn_y_desc, Y->mutable_data(context.GetPlace()))); +#endif } template @@ -74,6 +89,19 @@ void SoftmaxGradCUDNNFunctor::operator()( if (cudnn_tensor_dims.size() <= 2) { cudnn_tensor_dims.resize(4, 1); } +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t cudnn_y_desc = + yDesc.descriptor(layout, cudnn_tensor_dims); + miopenTensorDescriptor_t cudnn_xgrad_desc = + dxDesc.descriptor(layout, cudnn_tensor_dims); + miopenTensorDescriptor_t cudnn_ygrad_desc = + dyDesc.descriptor(layout, cudnn_tensor_dims); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSoftmaxBackward( + context.cudnn_handle(), CudnnDataType::kOne(), cudnn_y_desc, + Y->data(), cudnn_ygrad_desc, YGrad->data(), + CudnnDataType::kZero(), cudnn_xgrad_desc, + XGrad->mutable_data(context.GetPlace()))); +#else cudnnTensorDescriptor_t cudnn_y_desc = yDesc.descriptor(layout, cudnn_tensor_dims); cudnnTensorDescriptor_t cudnn_xgrad_desc = @@ -86,15 +114,20 @@ void SoftmaxGradCUDNNFunctor::operator()( Y->data(), cudnn_ygrad_desc, YGrad->data(), CudnnDataType::kZero(), cudnn_xgrad_desc, XGrad->mutable_data(context.GetPlace()))); +#endif } -template class SoftmaxCUDNNFunctor; template class SoftmaxCUDNNFunctor; -template class SoftmaxCUDNNFunctor; +template class SoftmaxCUDNNFunctor; template class SoftmaxGradCUDNNFunctor; -template class SoftmaxGradCUDNNFunctor; template class SoftmaxGradCUDNNFunctor; +// MIOPEN do not support double +#ifndef PADDLE_WITH_HIP +template class SoftmaxCUDNNFunctor; +template class SoftmaxGradCUDNNFunctor; +#endif + template class SoftmaxFunctor; template class SoftmaxFunctor class SoftmaxCUDNNFunctor { public: diff --git a/paddle/fluid/operators/math/vol2col_test.cc b/paddle/fluid/operators/math/vol2col_test.cc index 6ed5a0943ebb3..cc3b838cbcf1d 100644 --- a/paddle/fluid/operators/math/vol2col_test.cc +++ b/paddle/fluid/operators/math/vol2col_test.cc @@ -120,7 +120,7 @@ void testVol2col() { TEST(math, vol2col) { testVol2col(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) testVol2col(); #endif // PADDLE_WITH_CUDA diff --git a/paddle/fluid/operators/matmul_op.cc b/paddle/fluid/operators/matmul_op.cc index e97565a662318..9b64e99c94472 100644 --- a/paddle/fluid/operators/matmul_op.cc +++ b/paddle/fluid/operators/matmul_op.cc @@ -76,7 +76,8 @@ class MatMulKernel : public framework::OpKernel { auto scale = static_cast(context.Attr("alpha")); int head_number = 1; -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) head_number = context.Attr("head_number"); #endif @@ -89,7 +90,8 @@ class MatMulKernel : public framework::OpKernel { mat_dim_a.batch_size_ = 0; } } -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) bool split_vertical_y = (mat_dim_a.width_ != mat_dim_b.height_); if (head_number > 1) { @@ -228,7 +230,8 @@ class MatMulGradKernel : public framework::OpKernel { auto mat_dim_b = math::CreateMatrixDescriptor(b.dims(), 0, trans_b); int head_number = 1; -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) head_number = context.Attr("head_number"); #endif @@ -362,7 +365,8 @@ class MatMulDoubleGradKernel : public framework::OpKernel { auto mat_dim_b = math::CreateMatrixDescriptor(b.dims(), 0, trans_b); int head_number = 1; -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) head_number = context.Attr("head_number"); #endif @@ -562,7 +566,8 @@ class MatMulOp : public framework::OperatorWithKernel { DumpMatrixShape(mat_dim_y).c_str())); } int64_t dim_out_y = mat_dim_y.width_; -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) int head_number = context->Attrs().Get("head_number"); bool split_vertical_y = (mat_dim_x.width_ != mat_dim_y.height_); if (context->IsRuntime()) { @@ -750,7 +755,8 @@ class MatMulOpMaker : public framework::OpProtoAndCheckerMaker { "used in MKL-DNN INT8") .SetDefault(false); -#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_MKLML) && !defined(PADDLE_WITH_CUDA) && \ + !defined(PADDLE_WITH_HIP) AddAttr("head_number", "The number of heads of the matrix") .SetDefault(1); #endif @@ -916,7 +922,7 @@ REGISTER_OP_CPU_KERNEL( ops::MatMulDoubleGradKernel, ops::MatMulDoubleGradKernel); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) REGISTER_OP_CUDA_KERNEL( matmul, ops::MatMulKernel, ops::MatMulKernel, diff --git a/paddle/fluid/operators/matmul_op_xpu.cc b/paddle/fluid/operators/matmul_op_xpu.cc index 8834e95758bf2..f92cff2f6cd21 100644 --- a/paddle/fluid/operators/matmul_op_xpu.cc +++ b/paddle/fluid/operators/matmul_op_xpu.cc @@ -159,23 +159,14 @@ static void MatMulXPUFunction(const Tensor *x, const Tensor *y, Tensor *out, "XPU fc_fusion kernel return wrong value[%d %s]", r, XPUAPIErrorMsg[r])); } else { - // batch matmul - int x_stride = mat_dim_a.stride_; - int y_stride = mat_dim_b.stride_; - int out_stride = m * n; - for (int i = 0; i < batch_size; ++i) { - const float *x_data = x->data() + x_stride * i; - const float *y_data = y->data() + y_stride * i; - float *out_data = data_c + out_stride * i; - int r = xpu::fc_fusion( - dev_ctx.x_context(), x_data, y_data, out_data, m, n, k, - mat_dim_a.trans_, mat_dim_b.trans_, nullptr, nullptr, nullptr, ldx, - ldy, ldout, alpha, 0, nullptr, xpu::Activation_t::LINEAR); - PADDLE_ENFORCE_EQ(r, XPU_SUCCESS, - platform::errors::External( - "XPU fc_fusion kernel return wrong value[%d %s]", r, - XPUAPIErrorMsg[r])); - } + int r = xpu::fc_batched( + dev_ctx.x_context(), batch_size, mat_dim_a.trans_, mat_dim_b.trans_, m, + n, k, alpha, x->data(), mat_dim_a.stride_, y->data(), + mat_dim_b.stride_, 0.0, data_c, m * n, nullptr, nullptr); + PADDLE_ENFORCE_EQ(r, XPU_SUCCESS, + platform::errors::External( + "XPU fc_batched kernel return wrong value[%d %s]", r, + XPUAPIErrorMsg[r])); } } diff --git a/paddle/fluid/operators/matmul_v2_op.h b/paddle/fluid/operators/matmul_v2_op.h index b6eac7bf0cc4b..ca20efaad074d 100644 --- a/paddle/fluid/operators/matmul_v2_op.h +++ b/paddle/fluid/operators/matmul_v2_op.h @@ -25,7 +25,7 @@ limitations under the License. */ #include "paddle/fluid/operators/math/complex_functors.h" #include "paddle/fluid/operators/reduce_ops/reduce_sum_op.h" -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) #include "paddle/fluid/operators/reduce_ops/cub_reduce.h" #endif @@ -45,7 +45,7 @@ template void ReduceSumForMatmulGrad(const Tensor* input, Tensor* output, const std::vector& reduce_dims, const paddle::framework::ExecutionContext& ctx) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) auto stream = ctx.cuda_device_context().stream(); TensorReduce>( *input, output, reduce_dims, static_cast(0), cub::Sum(), diff --git a/paddle/fluid/operators/matmul_v2_op_xpu.cc b/paddle/fluid/operators/matmul_v2_op_xpu.cc index 765a380c6b84f..dbb1d7bfb0a3d 100644 --- a/paddle/fluid/operators/matmul_v2_op_xpu.cc +++ b/paddle/fluid/operators/matmul_v2_op_xpu.cc @@ -79,22 +79,14 @@ static void MatMulXPUFunction(const Tensor* x, const Tensor* y, Tensor* out, "XPU fc_fusion kernel return wrong value[%d %s]", r, XPUAPIErrorMsg[r])); } else { - // batch matmul - int x_stride = mat_dim_a.stride_; - int y_stride = mat_dim_b.stride_; - int out_stride = m * n; - for (int i = 0; i < batch_size; ++i) { - const float* x_data = x->data() + x_stride * i; - const float* y_data = y->data() + y_stride * i; - float* out_data = data_c + out_stride * i; - int r = xpu::fc( - dev_ctx.x_context(), x_data, y_data, out_data, m, n, k, - mat_dim_a.trans_, mat_dim_b.trans_, nullptr, nullptr, nullptr); - PADDLE_ENFORCE_EQ(r, XPU_SUCCESS, - platform::errors::External( - "XPU fc_fusion kernel return wrong value[%d %s]", r, - XPUAPIErrorMsg[r])); - } + int r = xpu::fc_batched( + dev_ctx.x_context(), batch_size, mat_dim_a.trans_, mat_dim_b.trans_, m, + n, k, 1.0, x->data(), mat_dim_a.stride_, y->data(), + mat_dim_b.stride_, 0.0, data_c, m * n, nullptr, nullptr); + PADDLE_ENFORCE_EQ(r, XPU_SUCCESS, + platform::errors::External( + "XPU fc_batched kernel return wrong value[%d %s]", r, + XPUAPIErrorMsg[r])); } } diff --git a/paddle/fluid/operators/mean_op.cu b/paddle/fluid/operators/mean_op.cu index 081c077ab73c2..430036bc67de7 100644 --- a/paddle/fluid/operators/mean_op.cu +++ b/paddle/fluid/operators/mean_op.cu @@ -11,7 +11,13 @@ 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. */ +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/mean_op.h" #include "paddle/fluid/platform/cuda_primitives.h" #include "paddle/fluid/platform/float16.h" diff --git a/paddle/fluid/operators/merge_lod_tensor_op.cc b/paddle/fluid/operators/merge_lod_tensor_op.cc index 584de34c5d329..5024148fe5888 100644 --- a/paddle/fluid/operators/merge_lod_tensor_op.cc +++ b/paddle/fluid/operators/merge_lod_tensor_op.cc @@ -65,7 +65,7 @@ class MergeLoDTensorOp : public framework::OperatorBase { if (platform::is_cpu_place(mask.place())) { cpu_mask->ShareDataWith(mask); } else if (platform::is_gpu_place(mask.place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) framework::TensorCopy(mask, platform::CPUPlace(), dev_ctx, cpu_mask.get()); #else diff --git a/paddle/fluid/operators/miopen_lstm_cache.h b/paddle/fluid/operators/miopen_lstm_cache.h new file mode 100644 index 0000000000000..7c0faa86be0be --- /dev/null +++ b/paddle/fluid/operators/miopen_lstm_cache.h @@ -0,0 +1,141 @@ +/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. + +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. */ + +#pragma once + +#include +#include "paddle/fluid/framework/tensor.h" +#include "paddle/fluid/platform/miopen_helper.h" + +namespace paddle { +namespace operators { + +class ScopedRNNBase { + public: + ScopedRNNBase(int seq_length, int batch_size, int input_size, int hidden_size, + int num_layers, float dropout_prob, int seed, int weight_numel, + bool initialized, bool is_bidirec) + : seq_length_(seq_length), + batch_size_(batch_size), + input_size_(input_size), + hidden_size_(hidden_size), + num_layers_(num_layers), + dropout_prob_(dropout_prob), + seed_(seed), + weight_numel_(weight_numel), + initialized_(initialized), + is_bidirec_(is_bidirec) {} + + template + void Create(const miopenHandle_t& handle, const platform::Place& place, + const std::vector& sequence_length, size_t* workspace_size, + size_t* reserve_size, framework::Tensor* dropout_state) { + int numDirections = is_bidirec_ ? 2 : 1; + miopenDataType_t miopen_type = platform::CudnnDataType::type; + + // ------------------- miopen x, y descriptors --------------------- + std::vector dims_x = {batch_size_, input_size_, 1}; + std::vector strides_x = {input_size_, 1, 1}; + std::vector dims_y = {batch_size_, hidden_size_ * numDirections, 1}; + std::vector strides_y = {hidden_size_ * numDirections, 1, 1}; + for (int i = 0; i < seq_length_; ++i) { + x_descs_.emplace_back(x_desc_.descriptor(dims_x, strides_x)); + y_descs_.emplace_back(y_desc_.descriptor(dims_y, strides_y)); + } + + // ------------------- miopen hx, hy, cx, cy descriptors---------- + std::vector dims_hx = {num_layers_ * numDirections, batch_size_, + hidden_size_}; + std::vector strides_hx = {hidden_size_ * batch_size_, hidden_size_, 1}; + init_h_desc_.descriptor(dims_hx, strides_hx); + init_c_desc_.descriptor(dims_hx, strides_hx); + last_h_desc_.descriptor(dims_hx, strides_hx); + last_c_desc_.descriptor(dims_hx, strides_hx); + + // ------------------- miopen dropout descriptors --------------------- + size_t state_size; + if (!initialized_) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDropoutGetStatesSize(handle, &state_size)); + dropout_state->mutable_data({static_cast(state_size)}, + place); + } + dropout_desc_.descriptor(handle, place, initialized_, dropout_prob_, + dropout_state, seed_, state_size); + + // ------------------- miopen rnn descriptors --------------------- + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetRNNDescriptor( + rnn_desc_.desc(), hidden_size_, num_layers_, miopenRNNlinear, + is_bidirec_ ? miopenRNNbidirection : miopenRNNunidirection, miopenLSTM, + miopenRNNNoBias, miopenRNNdefault, miopen_type)); + + // ------------------- miopen weights_size --------------------- + size_t weights_size_; + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNParamsSize( + handle, rnn_desc_.desc(), x_descs_[0], &weights_size_, miopen_type)); + PADDLE_ENFORCE_EQ( + weights_size_, sizeof(T) * weight_numel_, + platform::errors::InvalidArgument( + "The miopen lstm and setting weight size should be same.")); + // ------------------- miopen weight descriptors --------------------- + platform::DataLayout layout = platform::DataLayout::kNCHW; + int dim_tmp = weights_size_ / sizeof(T); + std::vector dim_w = {dim_tmp, 1, 1}; + weight_desc_.descriptor(layout, dim_w); + // ------------------- miopen workspace, reserve size --------------------- + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNWorkspaceSize( + handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), + workspace_size)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenGetRNNTrainingReserveSize( + handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), + reserve_size)); + } + miopenTensorDescriptor_t* x_descs() { return x_descs_.data(); } + miopenTensorDescriptor_t* y_descs() { return y_descs_.data(); } + miopenTensorDescriptor_t init_h_desc() { return init_h_desc_.desc(); } + miopenTensorDescriptor_t init_c_desc() { return init_c_desc_.desc(); } + miopenTensorDescriptor_t last_h_desc() { return last_h_desc_.desc(); } + miopenTensorDescriptor_t last_c_desc() { return last_c_desc_.desc(); } + miopenRNNDescriptor_t rnn_desc() { return rnn_desc_.desc(); } + miopenDropoutDescriptor_t dropout_desc() { return dropout_desc_.desc(); } + miopenTensorDescriptor_t weight_desc() { return weight_desc_.desc(); } + + private: + int seq_length_; + int batch_size_; + int input_size_; + int hidden_size_; + int num_layers_; + float dropout_prob_; + int seed_; + int weight_numel_; + bool initialized_; + bool is_bidirec_; + std::vector x_descs_; + std::vector y_descs_; + + platform::ScopedTensorDescriptor x_desc_; + platform::ScopedTensorDescriptor y_desc_; + platform::ScopedTensorDescriptor init_h_desc_; + platform::ScopedTensorDescriptor init_c_desc_; + platform::ScopedTensorDescriptor last_h_desc_; + platform::ScopedTensorDescriptor last_c_desc_; + platform::ScopedDropoutDescriptor dropout_desc_; + platform::ScopedFilterDescriptor weight_desc_; + platform::ScopedRNNDescriptor rnn_desc_; +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/miopen_rnn_cache.h b/paddle/fluid/operators/miopen_rnn_cache.h new file mode 100644 index 0000000000000..97d608331ccb5 --- /dev/null +++ b/paddle/fluid/operators/miopen_rnn_cache.h @@ -0,0 +1,267 @@ +/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. + +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. */ + +#pragma once + +#include +#include "paddle/fluid/framework/tensor.h" +#include "paddle/fluid/platform/miopen_helper.h" + +namespace paddle { +namespace operators { + +struct CudnnRNNCache { + CudnnRNNCache() { + x_desc_ = NULL; + y_desc_ = NULL; + } + ~CudnnRNNCache() { release(); } + + miopenRNNDescriptor_t rnn_desc_; + miopenTensorDescriptor_t *x_desc_; + miopenTensorDescriptor_t *y_desc_; + + miopenTensorDescriptor_t hx_desc_; + miopenTensorDescriptor_t cx_desc_; + miopenTensorDescriptor_t hy_desc_; + miopenTensorDescriptor_t cy_desc_; + + miopenTensorDescriptor_t dhx_desc_; + miopenTensorDescriptor_t dcx_desc_; + miopenTensorDescriptor_t dhy_desc_; + miopenTensorDescriptor_t dcy_desc_; + + miopenTensorDescriptor_t output_x_desc_; + miopenTensorDescriptor_t output_y_desc_; + + miopenDropoutDescriptor_t dropout_desc_; + + size_t weights_size_; + miopenTensorDescriptor_t w_desc_; + miopenTensorDescriptor_t dw_desc_; + + size_t workspace_size_; + framework::Tensor workspace_data_; + + size_t seq_length_; + + float dropout_prob_; + bool is_bidirec_; + + int batch_size_; + int input_size_; + int hidden_size_; + int num_layers_; + int seed_; + + void init(miopenHandle_t handle, const platform::Place &place, size_t seq_len, + int batch_size, int input_size, int hidden_size, int num_layers, + float dropout_prob, bool is_bidirec, int seed, int weight_numel, + size_t *reserve_size_, framework::Tensor *dropout_state_, + bool initialized, miopenDataType_t miopen_type) { + seq_length_ = seq_len; + batch_size_ = batch_size; + input_size_ = input_size; + hidden_size_ = hidden_size; + num_layers_ = num_layers; + dropout_prob_ = dropout_prob; + is_bidirec_ = is_bidirec; + seed_ = seed; + + const auto numDirections = is_bidirec_ ? 2 : 1; + + PADDLE_ENFORCE_EQ(miopen_type, miopenFloat, + platform::errors::InvalidArgument( + "MIOPEN do not support double datatype.")); + auto miopen_size = sizeof(float); + + x_desc_ = new miopenTensorDescriptor_t[seq_length_]; + y_desc_ = new miopenTensorDescriptor_t[seq_length_]; + std::vector dims = {batch_size_, input_size_, 1}; + std::vector strides = {input_size_, 1, 1}; + + std::vector dims_y = {batch_size_, hidden_size_ * numDirections, 1}; + std::vector strides_y = {hidden_size_ * numDirections, 1, 1}; + + for (size_t i = 0; i < seq_length_; ++i) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&x_desc_[i])); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&y_desc_[i])); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + x_desc_[i], miopen_type, 3, const_cast(dims.data()), + const_cast(strides.data()))); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + y_desc_[i], miopen_type, 3, const_cast(dims_y.data()), + const_cast(strides_y.data()))); + } + + std::vector dims_hx = {num_layers_ * numDirections, batch_size_, + hidden_size_}; + std::vector strides_hx = {hidden_size_ * batch_size_, hidden_size_, 1}; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&hx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&cx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&hy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&cy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&dhx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&dcx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&dhy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&dcy_desc_)); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + hx_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + cx_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + hy_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + cy_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + dhx_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + dcx_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + dhy_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + dcy_desc_, miopen_type, 3, const_cast(dims_hx.data()), + const_cast(strides_hx.data()))); + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateDropoutDescriptor(&dropout_desc_)); + + size_t state_size; + if (!initialized) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDropoutGetStatesSize(handle, &state_size)); + dropout_state_->Resize({static_cast(state_size)}); + uint8_t *dropout_state_data = + dropout_state_->mutable_data(place); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetDropoutDescriptor( + dropout_desc_, handle, dropout_prob_, dropout_state_data, state_size, + seed_, false, false, MIOPEN_RNG_PSEUDO_XORWOW)); + } else { + uint8_t *dropout_state_data = dropout_state_->data(); + auto dropout_state_dims = dropout_state_->dims(); + state_size = dropout_state_dims[0]; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenRestoreDropoutDescriptor( + dropout_desc_, handle, dropout_prob_, dropout_state_data, + state_size, 0, false, false, MIOPEN_RNG_PSEUDO_XORWOW)); + } + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateRNNDescriptor(&rnn_desc_)); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetRNNDescriptor( + rnn_desc_, hidden_size_, num_layers_, miopenRNNlinear, + is_bidirec_ ? miopenRNNbidirection : miopenRNNunidirection, miopenLSTM, + miopenRNNNoBias, miopenRNNdefault, miopen_type)); + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&w_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenCreateTensorDescriptor(&dw_desc_)); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNParamsSize( + handle, rnn_desc_, x_desc_[0], &weights_size_, miopen_type)); + + PADDLE_ENFORCE_EQ( + weights_size_, miopen_size * weight_numel, + platform::errors::InvalidArgument( + "The miopen lstm and setting weight size should be same.")); + + int dim_w[3]; + dim_w[0] = weights_size_ / miopen_size; + dim_w[1] = 1; + dim_w[2] = 1; + + int dim_s[2]; + dim_s[1] = 1; + dim_s[0] = dim_w[1]; + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + w_desc_, miopen_type, 3, dim_w, dim_s)); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetTensorDescriptor( + dw_desc_, miopen_type, 3, dim_w, dim_s)); + + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNWorkspaceSize( + handle, rnn_desc_, seq_length_, x_desc_, &workspace_size_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenGetRNNTrainingReserveSize( + handle, rnn_desc_, seq_length_, x_desc_, reserve_size_)); + + workspace_data_.Resize({static_cast(workspace_size_)}); + workspace_data_.mutable_data(place); + } + + void release() { + for (size_t i = 0; i < seq_length_; ++i) { + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(x_desc_[i])); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(y_desc_[i])); + } + + delete[] x_desc_; + delete[] y_desc_; + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(hx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(cx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(hy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(cy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(dhx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(dcx_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(dhy_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(dcy_desc_)); + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyDropoutDescriptor(dropout_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyRNNDescriptor(rnn_desc_)); + + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(w_desc_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDestroyTensorDescriptor(dw_desc_)); + } +}; + +} // namespace operators +} // namespace paddle diff --git a/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc index 4beb7ad017851..df1b5af121da9 100644 --- a/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/concat_mkldnn_op.cc @@ -71,6 +71,15 @@ static const std::vector ReduceMultiInput( return reduced; } +static const std::vector GetDimsForKey( + const std::vector& inputs) { + auto dims_key = paddle::framework::vectorize(inputs[0]->dims()); + for (auto it = std::next(inputs.begin()); it != inputs.end(); ++it) { + dims_key.push_back((*it)->dims()[0]); + } + return dims_key; +} + template class ConcatPrimitiveFactory { public: @@ -134,6 +143,8 @@ template class ConcatMKLDNNOpKernel : public paddle::framework::OpKernel { public: void Compute(const paddle::framework::ExecutionContext& ctx) const override { + // If any of the multiple inputs of concat has an input size of 0, the + // actual size of the multi_input will change auto multi_input = ReduceMultiInput(ctx.MultiInput("X")); EnforceLayouts(multi_input); Tensor* output = ctx.Output("Out"); @@ -156,12 +167,9 @@ class ConcatMKLDNNOpKernel : public paddle::framework::OpKernel { paddle::framework::ToMKLDNNDataType(multi_input[0]->type()); ConcatPrimitiveFactory prim_creator; - // If one of the multiple inputs of concat has an input size of 0, the - // actual size of the multi_input will change - std::string key = platform::CreateKey( - dev_ctx, paddle::framework::vectorize(multi_input[0]->dims()), - multi_input.size(), ctx.OutputName("Out"), dt, - platform::ThreadIDasStr()); + std::string key = + platform::CreateKey(dev_ctx, GetDimsForKey(multi_input), + multi_input.size(), ctx.OutputName("Out"), dt); key = platform::ExtendKeyWithThreadInfoIfNeeded(dev_ctx, key); const std::string key_prim = key + "@concat_p"; diff --git a/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc b/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc index 05b71e14c52c2..73530eac09e99 100644 --- a/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc +++ b/paddle/fluid/operators/mkldnn/conv_mkldnn_op.cc @@ -981,12 +981,7 @@ class ConvMKLDNNGradOpKernel : public paddle::framework::OpKernel { * ('any') which lets a primitive (conv backward in this case) choose * the memory format preferred for best performance */ - // TODO: NHWC is preferred starting from oneDNN 2.1 . Any may crash - auto chosen_memory_format = - platform::MayIUse(platform::cpu_isa_t::avx512_core) && - is_conv3d == false - ? MKLDNNMemoryFormat::nhwc - : MKLDNNMemoryFormat::any; + auto chosen_memory_format = MKLDNNMemoryFormat::any; weights_format = MKLDNNMemoryFormat::any; auto src_md = platform::MKLDNNMemDesc( diff --git a/paddle/fluid/operators/modified_huber_loss_op.h b/paddle/fluid/operators/modified_huber_loss_op.h index 17621095c4925..398676ba74151 100644 --- a/paddle/fluid/operators/modified_huber_loss_op.h +++ b/paddle/fluid/operators/modified_huber_loss_op.h @@ -29,8 +29,8 @@ using EigenVector = framework::EigenVector; template struct CheckLabelValue { HOSTDEVICE T operator()(const T& val) const { - PADDLE_ENFORCE( - val == static_cast(0) || val == static_cast(1), + PADDLE_ENFORCE_EQ( + val == static_cast(0) || val == static_cast(1), true, platform::errors::InvalidArgument( "Input(label) value of modified_huber_loss_op expected to be 0 " "or 1, but got %ld. Please check label value.", diff --git a/paddle/fluid/operators/multinomial_op.cu b/paddle/fluid/operators/multinomial_op.cu index 92f7c992ed976..2d97111709a0f 100644 --- a/paddle/fluid/operators/multinomial_op.cu +++ b/paddle/fluid/operators/multinomial_op.cu @@ -12,6 +12,10 @@ 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 PADDLE_WITH_HIP +// To-do(qili93): fix this after issue resolved +// https://github.com/ROCmSoftwarePlatform/rocPRIM/issues/202 + #include #include #include @@ -155,13 +159,24 @@ class MultinomialOpKernel T* cpu_in_data = new T[in_data_numel]; int64_t* cpu_out_data = new int64_t[out_data_numel]; +#ifdef PADDLE_WITH_HIP + hipMemcpy(cpu_in_data, in_data, in_data_numel * sizeof(T), + hipMemcpyDeviceToHost); +#else cudaMemcpy(cpu_in_data, in_data, in_data_numel * sizeof(T), cudaMemcpyDeviceToHost); +#endif MultinomialFunctor(cpu_out_data, cpu_in_data, num_samples, replacement, num_categories, num_distributions); + +#ifdef PADDLE_WITH_HIP + hipMemcpy(out_data, cpu_out_data, out_data_numel * sizeof(int64_t), + hipMemcpyHostToDevice); +#else cudaMemcpy(out_data, cpu_out_data, out_data_numel * sizeof(int64_t), cudaMemcpyHostToDevice); +#endif delete[] cpu_in_data; delete[] cpu_out_data; @@ -250,5 +265,7 @@ namespace ops = paddle::operators; namespace plat = paddle::platform; REGISTER_OP_CUDA_KERNEL( - multinomial, ops::MultinomialOpKernel, - ops::MultinomialOpKernel); + multinomial, ops::MultinomialOpKernel, + ops::MultinomialOpKernel); + +#endif diff --git a/paddle/fluid/operators/nccl/CMakeLists.txt b/paddle/fluid/operators/nccl/CMakeLists.txt index 4f1fe372f5849..9a412228255d0 100644 --- a/paddle/fluid/operators/nccl/CMakeLists.txt +++ b/paddle/fluid/operators/nccl/CMakeLists.txt @@ -1,4 +1,4 @@ -if (NOT WITH_NCCL) +if (NOT (WITH_NCCL OR WITH_RCCL)) return() endif() @@ -6,12 +6,20 @@ if(WITH_GPU AND NOT WIN32) nv_library(nccl_common SRCS nccl_gpu_common.cc DEPS device_context operator ) endif() -if(WITH_GPU) +if(WITH_ROCM AND NOT WIN32) + hip_library(nccl_common SRCS nccl_gpu_common.cc DEPS device_context operator ) +endif() + +if(WITH_GPU OR WITH_ROCM) op_library(nccl_op DEPS nccl_common) file(APPEND ${pybind_file} "USE_CUDA_ONLY_OP(ncclAllReduce);\n") set(OPERATOR_DEPS ${OPERATOR_DEPS} nccl_common PARENT_SCOPE) endif() -if(NOT WIN32) +if(WITH_GPU AND NOT WIN32) nv_test(nccl_op_test SRCS nccl_op_test.cu.cc DEPS nccl_op gpu_info device_context) endif() + +if(WITH_ROCM AND NOT WIN32) + hip_test(nccl_op_test SRCS nccl_op_test.cu.cc DEPS nccl_op gpu_info device_context) +endif() diff --git a/paddle/fluid/operators/nccl/nccl_gpu_common.h b/paddle/fluid/operators/nccl/nccl_gpu_common.h index 558ff4cc09603..01905d8ca84b3 100644 --- a/paddle/fluid/operators/nccl/nccl_gpu_common.h +++ b/paddle/fluid/operators/nccl/nccl_gpu_common.h @@ -23,7 +23,11 @@ limitations under the License. */ #include #include "paddle/fluid/platform/device_context.h" +#ifdef PADDLE_WITH_RCCL +#include "paddle/fluid/platform/dynload/rccl.h" +#else #include "paddle/fluid/platform/dynload/nccl.h" +#endif #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/macros.h" diff --git a/paddle/fluid/operators/nll_loss_op.cu b/paddle/fluid/operators/nll_loss_op.cu index 531c175e03e5e..b6e7cd256e18d 100644 --- a/paddle/fluid/operators/nll_loss_op.cu +++ b/paddle/fluid/operators/nll_loss_op.cu @@ -11,7 +11,6 @@ limitations under the License. */ #include #include #include -#include "cub/cub.cuh" #include "paddle/fluid/operators/math.h" #include "paddle/fluid/operators/nll_loss_op.h" #include "paddle/fluid/platform/cuda_primitives.h" @@ -361,7 +360,11 @@ class NLLLossCUDAKernel : public framework::OpKernel { auto total_weight_data = total_weight->mutable_data(ctx.GetPlace()); auto label_data = labels->data(); auto weight_data = weight ? weight->data() : nullptr; +#ifdef PADDLE_WITH_HIP + hipMemset(total_weight_data, 0, sizeof(T)); +#else cudaMemset(total_weight_data, 0, sizeof(T)); +#endif auto x_dims = x->dims(); auto batch_size = x_dims[0]; auto n_classes = x_dims[1]; @@ -429,7 +432,11 @@ class NLLLossGradCUDAKernel : public framework::OpKernel { auto total_weight_data = total_weight->data(); auto ignore_index = ctx.Attr("ignore_index"); auto reduction = ctx.Attr("reduction"); +#ifdef PADDLE_WITH_HIP + hipMemset(dx_data, 0, dx->numel() * sizeof(T)); +#else cudaMemset(dx_data, 0, dx->numel() * sizeof(T)); +#endif int64_t size_average = (int64_t)(reduction == "mean"); auto x_dims = x->dims(); diff --git a/paddle/fluid/operators/norm_op.cu b/paddle/fluid/operators/norm_op.cu index 67449aa4c67be..4c1674ded1a44 100644 --- a/paddle/fluid/operators/norm_op.cu +++ b/paddle/fluid/operators/norm_op.cu @@ -13,7 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/norm_op.h" namespace paddle { @@ -73,8 +79,11 @@ class NormCUDAKernel : public framework::OpKernel { GetDims(xdim, axis, &pre, &n, &post); auto& dev_ctx = ctx.cuda_device_context(); - +#ifdef __HIPCC__ + const int block = 256; +#else const int block = 512; +#endif int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); const int max_blocks = std::max(max_threads / block, 1); int grid = std::min(max_blocks, pre * post); @@ -140,7 +149,11 @@ class NormGradCUDAKernel : public framework::OpKernel { auto& dev_ctx = ctx.cuda_device_context(); +#ifdef __HIPCC__ + const int block = 256; +#else const int block = 512; +#endif int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); const int max_blocks = std::max(max_threads / block, 1); int grid = std::min(max_blocks, pre * post); diff --git a/paddle/fluid/operators/norm_utils.cu.h b/paddle/fluid/operators/norm_utils.cu.h index 02dcb4045f4cd..9fcc629233891 100644 --- a/paddle/fluid/operators/norm_utils.cu.h +++ b/paddle/fluid/operators/norm_utils.cu.h @@ -17,10 +17,20 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/operators/math/math_function.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif namespace paddle { namespace operators { diff --git a/paddle/fluid/operators/optimizers/lamb_op_xpu.cc b/paddle/fluid/operators/optimizers/lamb_op_xpu.cc new file mode 100644 index 0000000000000..e7cbe4aa8dd4b --- /dev/null +++ b/paddle/fluid/operators/optimizers/lamb_op_xpu.cc @@ -0,0 +1,125 @@ +/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved. + +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 "paddle/fluid/operators/optimizers/lamb_op.h" +#include "gflags/gflags.h" + +namespace paddle { +namespace operators { + +using Tensor = framework::Tensor; + +#ifdef PADDLE_WITH_XPU +template +class LambOpXPUKernel : public framework::OpKernel { + public: + void Compute(const framework::ExecutionContext& ctx) const override { + using paddle::framework::LoDTensor; + const auto* param_var = ctx.InputVar("Param"); + PADDLE_ENFORCE_EQ(param_var->IsType(), true, + platform::errors::InvalidArgument( + "The Var(%s)'s type should be LoDTensor, " + "but the received is %s", + ctx.InputNames("Param").front(), + framework::ToTypeName(param_var->Type()))); + + using paddle::framework::LoDTensor; + + // inputs + T epsilon = static_cast(ctx.Attr("epsilon")); + T weight_decay = static_cast(ctx.Attr("weight_decay")); + T beta1 = static_cast(ctx.Attr("beta1")); + T beta2 = static_cast(ctx.Attr("beta2")); + auto& param = GET_DATA_SAFELY(ctx.Input("Param"), "Input", + "Param", "Lamb"); + auto* grad_var = ctx.InputVar("Grad"); + auto& mom1 = GET_DATA_SAFELY(ctx.Input("Moment1"), "Input", + "Moment1", "Lamb"); + auto& mom2 = GET_DATA_SAFELY(ctx.Input("Moment2"), "Input", + "Moment2", "Lamb"); + auto& lr = GET_DATA_SAFELY(ctx.Input("LearningRate"), "Input", + "LearningRate", "Lamb"); + + auto& beta1_pow = GET_DATA_SAFELY(ctx.Input("Beta1Pow"), "Input", + "Beta1Pow", "Lamb"); + auto& beta2_pow = GET_DATA_SAFELY(ctx.Input("Beta2Pow"), "Input", + "Beta2Pow", "Lamb"); + + auto& param_out = GET_DATA_SAFELY(ctx.Output("ParamOut"), + "Output", "ParamOut", "Lamb"); + auto& mom1_out = GET_DATA_SAFELY(ctx.Output("Moment1Out"), + "Output", "Moment1Out", "Lamb"); + auto& mom2_out = GET_DATA_SAFELY(ctx.Output("Moment2Out"), + "Output", "Moment2Out", "Lamb"); + auto& beta1_pow_out = GET_DATA_SAFELY(ctx.Output("Beta1PowOut"), + "Output", "Beta1PowOut", "Lamb"); + auto& beta2_pow_out = GET_DATA_SAFELY(ctx.Output("Beta2PowOut"), + "Output", "Beta2PowOut", "Lamb"); + auto& dev_ctx = ctx.template device_context(); + + if (grad_var->IsType()) { + auto& grad = *ctx.Input("Grad"); + int r = xpu::lamb(dev_ctx.x_context(), grad.template data(), + mom1.template data(), mom2.template data(), + param.template data(), beta1_pow.template data(), + beta2_pow.template data(), beta1, beta2, epsilon, + weight_decay, lr.template data(), + mom1_out.template mutable_data(ctx.GetPlace()), + mom2_out.template mutable_data(ctx.GetPlace()), + param_out.template mutable_data(ctx.GetPlace()), + beta1_pow_out.template mutable_data(ctx.GetPlace()), + beta2_pow_out.template mutable_data(ctx.GetPlace()), + param.numel()); + + if (r == xpu::Error_t::INVALID_PARAM) { + PADDLE_ENFORCE_EQ( + r, xpu::Error_t::SUCCESS, + platform::errors::InvalidArgument( + "XPU kernel error of LambOp, error message: INVALID_PARAM, " + "please check your input & output.")); + } else if (r == xpu::Error_t::RUNTIME_ERROR) { + PADDLE_ENFORCE_EQ(r, xpu::Error_t::SUCCESS, + platform::errors::Unavailable( + "XPU kernel error of LambOp, error message: " + "RUNTIME_ERROR, please check whether Baidu " + "Kunlun Card is properly installed.")); + } else if (r == xpu::Error_t::NO_ENOUGH_WORKSPACE) { + PADDLE_ENFORCE_EQ(r, xpu::Error_t::SUCCESS, + platform::errors::ResourceExhausted( + "XPU kernel error of LambOp, error " + "message: NO_ENOUGH_WORKSPACE, XPU " + "has no enough memory.")); + } else { + PADDLE_ENFORCE_EQ(r, xpu::Error_t::SUCCESS, + platform::errors::ResourceExhausted( + "XPU kernel error of LambOp, error " + "message: OTHER " + "XPU API returns error code: %d.", + r)); + } + } else { + PADDLE_THROW(platform::errors::InvalidArgument( + "Variable type not supported by lamb_op. Expect LoDTensor, " + "but got %s", + framework::ToTypeName(param_var->Type()))); + } + } +}; +} // namespace operators +} // namespace paddle + +namespace ops = paddle::operators; +REGISTER_OP_XPU_KERNEL( + lamb, ops::LambOpXPUKernel); +#endif diff --git a/paddle/fluid/operators/p_norm_op.cu b/paddle/fluid/operators/p_norm_op.cu index ba0d46f4c73ec..bd6694abdbf76 100644 --- a/paddle/fluid/operators/p_norm_op.cu +++ b/paddle/fluid/operators/p_norm_op.cu @@ -13,7 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. */ #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/p_norm_op.h" namespace paddle { @@ -136,7 +142,12 @@ class PnormCUDAKernel : public framework::OpKernel { auto& dev_ctx = ctx.cuda_device_context(); +#ifdef __HIPCC__ + const int block = 256; +#else const int block = 512; +#endif + int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); const int max_blocks = std::max(max_threads / block, 1); int grid = std::min(max_blocks, pre * post); @@ -238,7 +249,12 @@ class PnormGradCUDAKernel : public framework::OpKernel { auto& dev_ctx = ctx.cuda_device_context(); +#ifdef __HIPCC__ + const int block = 256; +#else const int block = 512; +#endif + int max_threads = dev_ctx.GetMaxPhysicalThreadCount(); const int max_blocks = std::max(max_threads / block, 1); int grid = std::min(max_blocks, pre * post); diff --git a/paddle/fluid/operators/pool_cudnn_op.cu.cc b/paddle/fluid/operators/pool_cudnn_op.cu.cc index 3dc184facc78b..8ceb22d8cc4c3 100644 --- a/paddle/fluid/operators/pool_cudnn_op.cu.cc +++ b/paddle/fluid/operators/pool_cudnn_op.cu.cc @@ -16,7 +16,12 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/pool_op.h" +#ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" +#endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -122,7 +127,32 @@ class PoolCUDNNOpKernel : public framework::OpKernel { out_dims_vec[3] = output->dims()[2]; out_dims_vec[4] = output->dims()[3]; transformed_output.Resize(framework::make_ddim(out_dims_vec)); +#ifdef PADDLE_WITH_HIP + // MIOPEN not support NHWC data layout + } else if (data_format == str_NHWC) { + layout = DataLayout::kNCHW; + auto &dev_ctx = + ctx.template device_context(); + std::vector axis{0, 3, 1, 2}; + + transformed_input.Resize(input->dims()); + auto in_dims_vec = framework::vectorize(input->dims()); + in_dims_vec[1] = input->dims()[3]; + in_dims_vec[2] = input->dims()[1]; + in_dims_vec[3] = input->dims()[2]; + transformed_input.Resize(framework::make_ddim(in_dims_vec)); + transformed_input.mutable_data(ctx.GetPlace(), input->type()); + math::Transpose trans; + trans(dev_ctx, *input, &transformed_input, axis); + + transformed_output.Resize(output->dims()); + auto out_dims_vec = framework::vectorize(output->dims()); + out_dims_vec[1] = output->dims()[3]; + out_dims_vec[2] = output->dims()[1]; + out_dims_vec[3] = output->dims()[2]; + transformed_output.Resize(framework::make_ddim(out_dims_vec)); +#endif } else { layout = getLayoutFromStr(data_format); transformed_input = *input; @@ -138,11 +168,17 @@ class PoolCUDNNOpKernel : public framework::OpKernel { ScopedTensorDescriptor output_desc; ScopedPoolingDescriptor pool_desc; +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t cudnn_input_desc = input_desc.descriptor( + layout, framework::vectorize(transformed_input.dims())); + miopenTensorDescriptor_t cudnn_output_desc = output_desc.descriptor( + layout, framework::vectorize(transformed_output.dims())); +#else cudnnTensorDescriptor_t cudnn_input_desc = input_desc.descriptor( layout, framework::vectorize(transformed_input.dims())); cudnnTensorDescriptor_t cudnn_output_desc = output_desc.descriptor( layout, framework::vectorize(transformed_output.dims())); - +#endif PoolingMode pooling_mode; if (pooling_type == "max") { pooling_mode = PoolingMode::kMaximum; @@ -151,17 +187,36 @@ class PoolCUDNNOpKernel : public framework::OpKernel { : PoolingMode::kAverageInclusive; } +#ifdef PADDLE_WITH_HIP + miopenPoolingDescriptor_t cudnn_pool_desc = + pool_desc.descriptor(pooling_mode, ksize, paddings, strides); +#else cudnnPoolingDescriptor_t cudnn_pool_desc = pool_desc.descriptor(pooling_mode, ksize, paddings, strides); +#endif // ------------------- cudnn pool algorithm --------------------- auto handle = ctx.cuda_device_context().cudnn_handle(); ScalingParamType alpha = 1.0f, beta = 0.0f; +#ifdef PADDLE_WITH_HIP + char *pool_workspace; + size_t pool_worksize = 0; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenPoolingGetWorkSpaceSizeV2( + cudnn_pool_desc, cudnn_output_desc, &pool_worksize)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipMalloc(&pool_workspace, pool_worksize)); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenPoolingForward( + handle, cudnn_pool_desc, &alpha, cudnn_input_desc, + tranformed_input_data, &beta, cudnn_output_desc, tranformed_output_data, + false, pool_workspace, pool_worksize)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipFree(pool_workspace)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnPoolingForward( handle, cudnn_pool_desc, &alpha, cudnn_input_desc, tranformed_input_data, &beta, cudnn_output_desc, tranformed_output_data)); +#endif // add if (data_format == str_NDHWC) { auto &dev_ctx = @@ -170,6 +225,16 @@ class PoolCUDNNOpKernel : public framework::OpKernel { math::Transpose trans5_v2; trans5_v2(dev_ctx, transformed_output, output, axis); } +#ifdef PADDLE_WITH_HIP + // MIOPEN not support NHWC data layout + if (data_format == str_NHWC) { + auto &dev_ctx = + ctx.template device_context(); + std::vector axis{0, 2, 3, 1}; + math::Transpose trans; + trans(dev_ctx, transformed_output, output, axis); + } +#endif } }; @@ -272,6 +337,49 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { // input grad transformed_input_grad.Resize(framework::make_ddim(in_dims_vec)); +#ifdef PADDLE_WITH_HIP + // MIOPEN not support NHWC data layout + } else if (data_format == str_NHWC) { + layout = DataLayout::kNCHW; + auto &dev_ctx = + ctx.template device_context(); + std::vector axis{0, 3, 1, 2}; + + // input + transformed_input.Resize(input->dims()); + auto in_dims_vec = framework::vectorize(input->dims()); + in_dims_vec[1] = input->dims()[3]; + in_dims_vec[2] = input->dims()[1]; + in_dims_vec[3] = input->dims()[2]; + transformed_input.Resize(framework::make_ddim(in_dims_vec)); + transformed_input.mutable_data(ctx.GetPlace(), input->type()); + + math::Transpose trans4; + trans4(dev_ctx, *input, &transformed_input, axis); + + // output + transformed_output.Resize(output->dims()); + auto out_dims_vec = framework::vectorize(output->dims()); + out_dims_vec[1] = output->dims()[3]; + out_dims_vec[2] = output->dims()[1]; + out_dims_vec[3] = output->dims()[2]; + transformed_output.Resize(framework::make_ddim(out_dims_vec)); + + transformed_output.mutable_data(ctx.GetPlace(), output->type()); + + math::Transpose trans4_v2; + trans4_v2(dev_ctx, *output, &transformed_output, axis); + + // output grad + transformed_output_grad.Resize(framework::make_ddim(out_dims_vec)); + transformed_output_grad.mutable_data(ctx.GetPlace(), output_grad->type()); + + math::Transpose trans4_v3; + trans4_v3(dev_ctx, *output_grad, &transformed_output_grad, axis); + + // input grad + transformed_input_grad.Resize(framework::make_ddim(in_dims_vec)); +#endif } else { layout = getLayoutFromStr(data_format); transformed_input = *input; @@ -289,11 +397,17 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { ScopedTensorDescriptor output_desc; ScopedPoolingDescriptor pool_desc; +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t cudnn_input_desc = input_desc.descriptor( + layout, framework::vectorize(transformed_input.dims())); + miopenTensorDescriptor_t cudnn_output_desc = output_desc.descriptor( + layout, framework::vectorize(transformed_output.dims())); +#else cudnnTensorDescriptor_t cudnn_input_desc = input_desc.descriptor( layout, framework::vectorize(transformed_input.dims())); cudnnTensorDescriptor_t cudnn_output_desc = output_desc.descriptor( layout, framework::vectorize(transformed_output.dims())); - +#endif PoolingMode pooling_mode; if (pooling_type == "max") { if (FLAGS_cudnn_deterministic) { @@ -306,8 +420,13 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { : PoolingMode::kAverageInclusive; } +#ifdef PADDLE_WITH_HIP + miopenPoolingDescriptor_t cudnn_pool_desc = + pool_desc.descriptor(pooling_mode, ksize, paddings, strides); +#else cudnnPoolingDescriptor_t cudnn_pool_desc = pool_desc.descriptor(pooling_mode, ksize, paddings, strides); +#endif // ------------------- cudnn pool algorithm --------------------- auto handle = ctx.cuda_device_context().cudnn_handle(); @@ -315,11 +434,25 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { if (input_grad) { T *input_grad_data = transformed_input_grad.mutable_data( transformed_input_grad.dims(), ctx.GetPlace()); - // Because beta is zero, it is unnecessary to reset input_grad. +// Because beta is zero, it is unnecessary to reset input_grad. +#ifdef PADDLE_WITH_HIP + char *pool_workspace; + size_t pool_worksize = 0; + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenPoolingGetWorkSpaceSizeV2( + cudnn_pool_desc, cudnn_output_desc, &pool_worksize)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipMalloc(&pool_workspace, pool_worksize)); + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenPoolingBackward( + handle, cudnn_pool_desc, &alpha, cudnn_output_desc, output_data, + cudnn_output_desc, output_grad_data, cudnn_input_desc, input_data, + &beta, cudnn_input_desc, input_grad_data, pool_workspace)); + PADDLE_ENFORCE_CUDA_SUCCESS(hipFree(pool_workspace)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnPoolingBackward( handle, cudnn_pool_desc, &alpha, cudnn_output_desc, output_data, cudnn_output_desc, output_grad_data, cudnn_input_desc, input_data, &beta, cudnn_input_desc, input_grad_data)); +#endif if (data_format == str_NDHWC) { auto &dev_ctx = @@ -328,6 +461,16 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { math::Transpose trans5_v4; trans5_v4(dev_ctx, transformed_input_grad, input_grad, axis); } +#ifdef PADDLE_WITH_HIP + // MIOPEN not support NHWC data layout + if (data_format == str_NHWC) { + auto &dev_ctx = + ctx.template device_context(); + std::vector axis{0, 2, 3, 1}; + math::Transpose trans4_v4; + trans4_v4(dev_ctx, transformed_input_grad, input_grad, axis); + } +#endif } } }; @@ -338,6 +481,21 @@ class PoolCUDNNGradOpKernel : public framework::OpKernel { namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_KERNEL(pool2d, CUDNN, plat::CUDAPlace, + ops::PoolCUDNNOpKernel, + ops::PoolCUDNNOpKernel); +REGISTER_OP_KERNEL(pool2d_grad, CUDNN, plat::CUDAPlace, + ops::PoolCUDNNGradOpKernel, + ops::PoolCUDNNGradOpKernel); + +REGISTER_OP_KERNEL(pool3d, CUDNN, plat::CUDAPlace, + ops::PoolCUDNNOpKernel, + ops::PoolCUDNNOpKernel); +REGISTER_OP_KERNEL(pool3d_grad, CUDNN, plat::CUDAPlace, + ops::PoolCUDNNGradOpKernel); +#else REGISTER_OP_KERNEL(pool2d, CUDNN, plat::CUDAPlace, ops::PoolCUDNNOpKernel, ops::PoolCUDNNOpKernel, @@ -354,3 +512,4 @@ REGISTER_OP_KERNEL(pool3d, CUDNN, plat::CUDAPlace, REGISTER_OP_KERNEL(pool3d_grad, CUDNN, plat::CUDAPlace, ops::PoolCUDNNGradOpKernel, ops::PoolCUDNNGradOpKernel); +#endif diff --git a/paddle/fluid/operators/pool_op.cc b/paddle/fluid/operators/pool_op.cc index 2d4ef64cc896a..feb47a73ee405 100644 --- a/paddle/fluid/operators/pool_op.cc +++ b/paddle/fluid/operators/pool_op.cc @@ -18,6 +18,9 @@ limitations under the License. */ #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" #endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif #ifdef PADDLE_WITH_MKLDNN #include "paddle/fluid/platform/mkldnn_helper.h" #endif @@ -180,7 +183,7 @@ framework::OpKernelType PoolOp::GetExpectedKernelType( framework::DataLayout layout_ = framework::StringToDataLayout(data_format); auto data_type = OperatorWithKernel::IndicateVarDataType(ctx, "X"); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } @@ -235,7 +238,7 @@ framework::OpKernelType PoolOpGrad::GetExpectedKernelType( framework::DataLayout layout_ = framework::StringToDataLayout(data_format); auto input_data_type = OperatorWithKernel::IndicateVarDataType(ctx, "X"); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } diff --git a/paddle/fluid/operators/pool_op.cu.cc b/paddle/fluid/operators/pool_op.cu similarity index 100% rename from paddle/fluid/operators/pool_op.cu.cc rename to paddle/fluid/operators/pool_op.cu diff --git a/paddle/fluid/operators/pool_op.h b/paddle/fluid/operators/pool_op.h index 71bef11b67225..9117b1b95ed26 100644 --- a/paddle/fluid/operators/pool_op.h +++ b/paddle/fluid/operators/pool_op.h @@ -22,8 +22,20 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/math/pooling.h" +#if defined(__HIPCC__) || defined(__NVCC__) +#include "paddle/fluid/operators/reduce_ops/cub_reduce.h" +#endif + namespace paddle { namespace operators { +template +struct DivideFunctor { + HOSTDEVICE explicit inline DivideFunctor(int n) : n_inv((T)(1.0 / n)) {} + HOSTDEVICE inline T operator()(const T& x) const { return x * n_inv; } + + private: + T n_inv; +}; using Tensor = framework::Tensor; @@ -124,6 +136,26 @@ inline void UpdateKsize(std::vector* ksize, } } +inline int getReduceNum(const framework::Tensor& input, + const framework::Tensor* output, + const std::string data_format, + std::vector* reduce_dim) { + // data_format only can be NCHW + bool channel_last = (data_format == "NHWC"); + if (channel_last) { + return 0; + } + int reduce_num = 0; + const int output_height = output->dims()[2]; + const int output_width = output->dims()[3]; + if ((output_height == 1) && (output_width == 1)) { + reduce_dim->push_back(2); + reduce_dim->push_back(3); + reduce_num = input.dims()[2] * input.dims()[3]; + } + return reduce_num; +} + template class PoolKernel : public framework::OpKernel { public: @@ -164,7 +196,6 @@ class PoolKernel : public framework::OpKernel { if (global_pooling) { UpdateKsize(&ksize, data_dims); } - auto& dev_ctx = context.template device_context(); switch (ksize.size()) { case 2: { @@ -174,15 +205,35 @@ class PoolKernel : public framework::OpKernel { pool2d_forward; paddle::operators::math::MaxPool pool_process; pool2d_forward(dev_ctx, *in_x, ksize, strides, paddings, data_format, - pool_process, true, false, out); + true, false, out, pool_process); } else if (pooling_type == "avg") { - paddle::operators::math::Pool2dFunctor< - DeviceContext, paddle::operators::math::AvgPool, T> - pool2d_forward; - paddle::operators::math::AvgPool pool_process; - pool2d_forward(dev_ctx, *in_x, ksize, strides, paddings, data_format, - pool_process, exclusive, adaptive, out); + std::vector reduce_dim; + int reduce_num = getReduceNum(*in_x, out, data_format, &reduce_dim); + + if (reduce_num > 0 && + adaptive) { // for adaptive_avg_pool2d && output_size == 1 +#if defined(__HIPCC__) || defined(__NVCC__) + auto stream = dev_ctx.stream(); + TensorReduce>( + *in_x, out, reduce_dim, static_cast(0), cub::Sum(), + DivideFunctor(reduce_num), stream); +#else // for cpu + paddle::operators::math::Pool2dFunctor< + DeviceContext, paddle::operators::math::AvgPool, T> + pool2d_forward; + paddle::operators::math::AvgPool pool_process; + pool2d_forward(dev_ctx, *in_x, ksize, strides, paddings, + data_format, exclusive, adaptive, out, pool_process); +#endif + } else { // avgpool_2d or adaptive_avg_pool2d && output_size != 1 + paddle::operators::math::Pool2dFunctor< + DeviceContext, paddle::operators::math::AvgPool, T> + pool2d_forward; + paddle::operators::math::AvgPool pool_process; + pool2d_forward(dev_ctx, *in_x, ksize, strides, paddings, + data_format, exclusive, adaptive, out, pool_process); + } } } break; case 3: { @@ -192,7 +243,7 @@ class PoolKernel : public framework::OpKernel { pool3d_forward; paddle::operators::math::MaxPool pool_process; pool3d_forward(dev_ctx, *in_x, ksize, strides, paddings, data_format, - pool_process, true, false, out); + true, false, out, pool_process); } else if (pooling_type == "avg") { paddle::operators::math::Pool3dFunctor< @@ -200,7 +251,7 @@ class PoolKernel : public framework::OpKernel { pool3d_forward; paddle::operators::math::AvgPool pool_process; pool3d_forward(dev_ctx, *in_x, ksize, strides, paddings, data_format, - pool_process, exclusive, adaptive, out); + exclusive, adaptive, out, pool_process); } } break; default: { @@ -273,8 +324,8 @@ class PoolGradKernel : public framework::OpKernel { pool2d_backward; paddle::operators::math::AvgPoolGrad pool_process; pool2d_backward(dev_ctx, *in_x, *out, *out_grad, ksize, strides, - paddings, data_format, pool_process, exclusive, - adaptive, in_x_grad); + paddings, data_format, exclusive, adaptive, + in_x_grad, pool_process); } } break; case 3: { @@ -289,8 +340,8 @@ class PoolGradKernel : public framework::OpKernel { pool3d_backward; paddle::operators::math::AvgPoolGrad pool_process; pool3d_backward(dev_ctx, *in_x, *out, *out_grad, ksize, strides, - paddings, data_format, pool_process, exclusive, - adaptive, in_x_grad); + paddings, data_format, exclusive, adaptive, + in_x_grad, pool_process); } } break; default: { diff --git a/paddle/fluid/operators/prelu_op.cu b/paddle/fluid/operators/prelu_op.cu index 2f61c53f877d5..ca01487549fe6 100644 --- a/paddle/fluid/operators/prelu_op.cu +++ b/paddle/fluid/operators/prelu_op.cu @@ -95,7 +95,7 @@ __global__ void PReluOpGradKernel(const T* x_ptr, const T* alpha_ptr, template class PreluOpGradFunctor { public: - void operator()(cudaStream_t stream, const T* x, const T* alpha, const T* dy, + void operator()(gpuStream_t stream, const T* x, const T* alpha, const T* dy, T* dx, T* dalpha, const framework::DDim& input_dims, PRELU_MODE mode) { size_t numel = 1; diff --git a/paddle/fluid/operators/prroi_pool_op.cu b/paddle/fluid/operators/prroi_pool_op.cu index b85352ae6508c..a21f565dae71d 100644 --- a/paddle/fluid/operators/prroi_pool_op.cu +++ b/paddle/fluid/operators/prroi_pool_op.cu @@ -13,7 +13,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/operators/prroi_pool_op.h" -#include "paddle/fluid/platform/cuda_primitives.h" namespace paddle { namespace operators { @@ -29,22 +28,6 @@ static inline int NumBlocks(const int N) { kNumMaximumNumBlocks); } -template -DEVICE void PrRoIPoolingDistributeDiffCUDA(T* diff, const T top_diff, - const int h, const int w, - const int height, const int width, - const T coeff) { - bool overflow = (h < 0) || (w < 0) || (h >= height) || (w >= width); - if (!overflow) { - paddle::platform::CudaAtomicAdd(diff + h * width + w, top_diff * coeff); - } -} - -template -DEVICE void GPUAccumulateRois(T* offset, T data) { - paddle::platform::CudaAtomicAdd(offset, data); -} - template __global__ void GPUPRROIPoolForward( const int nthreads, const T* input_data, const T* input_rois, @@ -170,25 +153,23 @@ __global__ void GPUPRROIPoolBackward( for (int w_iter = s_w; w_iter < e_w; ++w_iter) { for (int h_iter = s_h; h_iter < e_h; ++h_iter) { - PrRoIPoolingMatDistributeDiff( + PrRoIPoolingMatDistributeDiff( offset_input_grad_data, sum_out, h_iter, w_iter, h_iter + 1, w_iter + 1, max(win_start_h, static_cast(h_iter)), max(win_start_w, static_cast(w_iter)), min(win_end_h, static_cast(h_iter) + static_cast(1.0)), min(win_end_w, static_cast(w_iter) + static_cast(1.0)), - height, width, PrRoIPoolingDistributeDiffCUDA); + height, width); } } const T* offset_out_data = out_data + i; const T* offset_in_data = in_data + input_offset; - PrRoIPoolingCoorBackward( + PrRoIPoolingCoorBackward( s_w, e_w, s_h, e_h, width, height, win_start_w, win_start_h, win_end_w, win_end_h, pw, ph, pooled_width, pooled_height, win_size, spatial_scale, offset_in_data, offset_out_data, offset_input_roi_grad_data, - offset_output_grad_data, GPUAccumulateRois, - [](const T x, const T y) { return max(x, y); }, - [](const T x, const T y) { return min(x, y); }); + offset_output_grad_data); } } diff --git a/paddle/fluid/operators/prroi_pool_op.h b/paddle/fluid/operators/prroi_pool_op.h index 11ecff8845216..f9e2b78d5d31a 100644 --- a/paddle/fluid/operators/prroi_pool_op.h +++ b/paddle/fluid/operators/prroi_pool_op.h @@ -16,6 +16,9 @@ limitations under the License. */ #include #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/math/math_function.h" +#if defined(__NVCC__) || defined(__HIPCC__) +#include "paddle/fluid/platform/cuda_primitives.h" +#endif namespace paddle { namespace operators { @@ -73,6 +76,17 @@ inline HOSTDEVICE T PrRoIPoolingMatCalculation(const T* this_data, return sum_out; } +#if defined(__NVCC__) || defined(__HIPCC__) +template +DEVICE void PrRoIPoolingDistributeDiff(T* diff, const T top_diff, const int h, + const int w, const int height, + const int width, const T coeff) { + bool overflow = (h < 0) || (w < 0) || (h >= height) || (w >= width); + if (!overflow) { + paddle::platform::CudaAtomicAdd(diff + h * width + w, top_diff * coeff); + } +} +#else template inline HOSTDEVICE void PrRoIPoolingDistributeDiff(T* diff, const T top_diff, const int h, const int w, @@ -84,12 +98,15 @@ inline HOSTDEVICE void PrRoIPoolingDistributeDiff(T* diff, const T top_diff, *(diff + h * width + w) += top_diff * coeff; } } +#endif -template -HOSTDEVICE void PrRoIPoolingMatDistributeDiff( - T* diff, const T top_diff, const int s_h, const int s_w, const int e_h, - const int e_w, const T y0, const T x0, const T y1, const T x1, const int h0, - const int w0, Functor functor) { +template +HOSTDEVICE void PrRoIPoolingMatDistributeDiff(T* diff, const T top_diff, + const int s_h, const int s_w, + const int e_h, const int e_w, + const T y0, const T x0, + const T y1, const T x1, + const int h0, const int w0) { T alpha, beta, lim_alpha, lim_beta, tmp; alpha = x0 - static_cast(s_w); @@ -99,14 +116,14 @@ HOSTDEVICE void PrRoIPoolingMatDistributeDiff( tmp = (lim_alpha - 0.5f * lim_alpha * lim_alpha - alpha + 0.5f * alpha * alpha) * (lim_beta - 0.5f * lim_beta * lim_beta - beta + 0.5f * beta * beta); - functor(diff, top_diff, s_h, s_w, h0, w0, tmp); + PrRoIPoolingDistributeDiff(diff, top_diff, s_h, s_w, h0, w0, tmp); alpha = static_cast(e_w) - x1; lim_alpha = static_cast(e_w) - x0; tmp = (lim_alpha - 0.5f * lim_alpha * lim_alpha - alpha + 0.5f * alpha * alpha) * (lim_beta - 0.5f * lim_beta * lim_beta - beta + 0.5f * beta * beta); - functor(diff, top_diff, s_h, e_w, h0, w0, tmp); + PrRoIPoolingDistributeDiff(diff, top_diff, s_h, e_w, h0, w0, tmp); alpha = x0 - static_cast(s_w); beta = static_cast(e_h) - y1; @@ -115,20 +132,47 @@ HOSTDEVICE void PrRoIPoolingMatDistributeDiff( tmp = (lim_alpha - 0.5f * lim_alpha * lim_alpha - alpha + 0.5f * alpha * alpha) * (lim_beta - 0.5f * lim_beta * lim_beta - beta + 0.5f * beta * beta); - functor(diff, top_diff, e_h, s_w, h0, w0, tmp); + PrRoIPoolingDistributeDiff(diff, top_diff, e_h, s_w, h0, w0, tmp); alpha = static_cast(e_w) - x1; lim_alpha = static_cast(e_w) - x0; tmp = (lim_alpha - 0.5f * lim_alpha * lim_alpha - alpha + 0.5f * alpha * alpha) * (lim_beta - 0.5f * lim_beta * lim_beta - beta + 0.5f * beta * beta); - functor(diff, top_diff, e_h, e_w, h0, w0, tmp); + PrRoIPoolingDistributeDiff(diff, top_diff, e_h, e_w, h0, w0, tmp); } +#if defined(__NVCC__) || defined(__HIPCC__) +template +DEVICE void AccumulateRois(T* offset, T data) { + paddle::platform::CudaAtomicAdd(offset, data); +} +#else template -inline HOSTDEVICE void CPUAccumulateRois(T* offset, T data) { +inline HOSTDEVICE void AccumulateRois(T* offset, T data) { *offset += data; } +#endif + +#if defined(__NVCC__) || defined(__HIPCC__) +template +DEVICE T MaxFunctor(const T x, const T y) { + return max(x, y); +} +template +DEVICE T MinFunctor(const T x, const T y) { + return min(x, y); +} +#else +template +inline HOSTDEVICE T MaxFunctor(const T x, const T y) { + return std::max(x, y); +} +template +inline HOSTDEVICE T MinFunctor(const T x, const T y) { + return std::max(x, y); +} +#endif template inline HOSTDEVICE static T PrRoIPoolingGetCoeff(T dh, T dw) { @@ -172,15 +216,13 @@ inline HOSTDEVICE T PrRoIPoolingSingleCoorIntegral(T s, T t, T c1, T c2) { (t - 0.5f * t * t - s + 0.5f * s * s) * c1; } -template +template inline HOSTDEVICE void PrRoIPoolingCoorBackward( int s_w, int e_w, int s_h, int e_h, int width, int height, T win_start_w, T win_start_h, T win_end_w, T win_end_h, int pw, int ph, const int pooled_width, const int pooled_height, T win_size, const float spatial_scale, const T* this_bottom_data, - const T* this_top_data, T* this_data_grad, const T* this_out_grad, - Functor functor, MaxFunctor maxFunctor, MinFunctor minFunctor) { + const T* this_top_data, T* this_data_grad, const T* this_out_grad) { T g_x1_y = 0.f; T g_x2_y = 0.f; T g_x_y1 = 0.f; @@ -188,16 +230,16 @@ inline HOSTDEVICE void PrRoIPoolingCoorBackward( for (int h_iter = s_h; h_iter < e_h; ++h_iter) { g_x1_y += PrRoIPoolingSingleCoorIntegral( - maxFunctor(win_start_h, static_cast(h_iter)) - h_iter, - minFunctor(win_end_h, static_cast(h_iter + 1)) - h_iter, + MaxFunctor(win_start_h, static_cast(h_iter)) - h_iter, + MinFunctor(win_end_h, static_cast(h_iter + 1)) - h_iter, PrRoIPoolingInterpolation(this_bottom_data, h_iter, win_start_w, height, width), PrRoIPoolingInterpolation(this_bottom_data, h_iter + 1, win_start_w, height, width)); g_x2_y += PrRoIPoolingSingleCoorIntegral( - maxFunctor(win_start_h, static_cast(h_iter)) - h_iter, - minFunctor(win_end_h, static_cast(h_iter + 1)) - h_iter, + MaxFunctor(win_start_h, static_cast(h_iter)) - h_iter, + MinFunctor(win_end_h, static_cast(h_iter + 1)) - h_iter, PrRoIPoolingInterpolation(this_bottom_data, h_iter, win_end_w, height, width), PrRoIPoolingInterpolation(this_bottom_data, h_iter + 1, win_end_w, @@ -206,16 +248,16 @@ inline HOSTDEVICE void PrRoIPoolingCoorBackward( for (int w_iter = s_w; w_iter < e_w; ++w_iter) { g_x_y1 += PrRoIPoolingSingleCoorIntegral( - maxFunctor(win_start_w, static_cast(w_iter)) - w_iter, - minFunctor(win_end_w, static_cast(w_iter + 1)) - w_iter, + MaxFunctor(win_start_w, static_cast(w_iter)) - w_iter, + MinFunctor(win_end_w, static_cast(w_iter + 1)) - w_iter, PrRoIPoolingInterpolation(this_bottom_data, win_start_h, w_iter, height, width), PrRoIPoolingInterpolation(this_bottom_data, win_start_h, w_iter + 1, height, width)); g_x_y2 += PrRoIPoolingSingleCoorIntegral( - maxFunctor(win_start_w, static_cast(w_iter)) - w_iter, - minFunctor(win_end_w, static_cast(w_iter + 1)) - w_iter, + MaxFunctor(win_start_w, static_cast(w_iter)) - w_iter, + MinFunctor(win_end_w, static_cast(w_iter + 1)) - w_iter, PrRoIPoolingInterpolation(this_bottom_data, win_end_h, w_iter, height, width), PrRoIPoolingInterpolation(this_bottom_data, win_end_h, w_iter + 1, @@ -232,22 +274,24 @@ inline HOSTDEVICE void PrRoIPoolingCoorBackward( partial_y1 = partial_y1 / win_size * spatial_scale; partial_y2 = partial_y2 / win_size * spatial_scale; - functor(this_data_grad + 0, - (partial_x1 * (1.0 - static_cast(pw) / pooled_width) + - partial_x2 * (1.0 - static_cast(pw + 1) / pooled_width)) * - (*this_out_grad)); - functor(this_data_grad + 1, - (partial_y1 * (1.0 - static_cast(ph) / pooled_height) + - partial_y2 * (1.0 - static_cast(ph + 1) / pooled_height)) * - (*this_out_grad)); - functor(this_data_grad + 2, - (partial_x2 * static_cast(pw + 1) / pooled_width + - partial_x1 * static_cast(pw) / pooled_width) * - (*this_out_grad)); - functor(this_data_grad + 3, - (partial_y2 * static_cast(ph + 1) / pooled_height + - partial_y1 * static_cast(ph) / pooled_height) * - (*this_out_grad)); + AccumulateRois( + this_data_grad + 0, + (partial_x1 * (1.0 - static_cast(pw) / pooled_width) + + partial_x2 * (1.0 - static_cast(pw + 1) / pooled_width)) * + (*this_out_grad)); + AccumulateRois( + this_data_grad + 1, + (partial_y1 * (1.0 - static_cast(ph) / pooled_height) + + partial_y2 * (1.0 - static_cast(ph + 1) / pooled_height)) * + (*this_out_grad)); + AccumulateRois(this_data_grad + 2, + (partial_x2 * static_cast(pw + 1) / pooled_width + + partial_x1 * static_cast(pw) / pooled_width) * + (*this_out_grad)); + AccumulateRois(this_data_grad + 3, + (partial_y2 * static_cast(ph + 1) / pooled_height + + partial_y1 * static_cast(ph) / pooled_height) * + (*this_out_grad)); } template @@ -516,7 +560,7 @@ class CPUPRROIPoolGradOpKernel : public framework::OpKernel { for (int w_iter = s_w; w_iter < e_w; ++w_iter) { for (int h_iter = s_h; h_iter < e_h; ++h_iter) { - PrRoIPoolingMatDistributeDiff( + PrRoIPoolingMatDistributeDiff( offset_input_grad_data, sum_out, h_iter, w_iter, h_iter + 1, w_iter + 1, std::max(win_start_h, static_cast(h_iter)), std::max(win_start_w, static_cast(w_iter)), @@ -524,19 +568,16 @@ class CPUPRROIPoolGradOpKernel : public framework::OpKernel { static_cast(h_iter) + static_cast(1.0)), std::min(win_end_w, static_cast(w_iter) + static_cast(1.0)), - height, width, PrRoIPoolingDistributeDiff); + height, width); } } const T* offset_in_data = in_data + input_offset; - PrRoIPoolingCoorBackward( + PrRoIPoolingCoorBackward( s_w, e_w, s_h, e_h, width, height, win_start_w, win_start_h, win_end_w, win_end_h, pw, ph, pooled_width, pooled_height, win_size, spatial_scale, offset_in_data, offset_out_data, - offset_input_roi_grad_data, offset_output_grad_data, - CPUAccumulateRois, - [](const T x, const T y) { return std::max(x, y); }, - [](const T x, const T y) { return std::min(x, y); }); + offset_input_roi_grad_data, offset_output_grad_data); } } } diff --git a/paddle/fluid/operators/pscore/CMakeLists.txt b/paddle/fluid/operators/pscore/CMakeLists.txt index 3e388b8d5ea10..12168e61ba5a9 100644 --- a/paddle/fluid/operators/pscore/CMakeLists.txt +++ b/paddle/fluid/operators/pscore/CMakeLists.txt @@ -22,7 +22,7 @@ foreach (src ${OPS}) set_source_files_properties(${src} PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) endforeach () -register_operators() +register_operators(DEPS ${DISTRIBUTE_DEPS}) set(OPERATOR_DEPS ${OPERATOR_DEPS} ${DISTRIBUTE_DEPS} PARENT_SCOPE) diff --git a/paddle/fluid/operators/pull_box_sparse_op.h b/paddle/fluid/operators/pull_box_sparse_op.h index 48e42c3232479..48903012b595e 100644 --- a/paddle/fluid/operators/pull_box_sparse_op.h +++ b/paddle/fluid/operators/pull_box_sparse_op.h @@ -47,7 +47,8 @@ static void PullBoxSparseFunctor(const framework::ExecutionContext &ctx) { box_ptr->PullSparse(ctx.GetPlace(), all_keys, all_values, slot_lengths, hidden_size, 0); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) auto hidden_size = ctx.Attr("size"); auto gpu_ps_ptr = paddle::framework::PSGPUWrapper::GetInstance(); gpu_ps_ptr->PullSparse(ctx.GetPlace(), 0, all_keys, all_values, slot_lengths, @@ -90,7 +91,8 @@ static void PushBoxSparseFunctor(const framework::ExecutionContext &ctx) { box_ptr->PushSparseGrad(ctx.GetPlace(), all_keys, all_grad_values, slot_lengths, hidden_size, 0, batch_size); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) auto hidden_size = ctx.Attr("size"); auto gpu_ps_ptr = paddle::framework::PSGPUWrapper::GetInstance(); gpu_ps_ptr->PushSparseGrad(ctx.GetPlace(), 0, all_keys, all_grad_values, diff --git a/paddle/fluid/operators/random_crop_op.h b/paddle/fluid/operators/random_crop_op.h index 62edb298d1a41..ee111a0ec7c09 100644 --- a/paddle/fluid/operators/random_crop_op.h +++ b/paddle/fluid/operators/random_crop_op.h @@ -18,7 +18,7 @@ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/for_range.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include #endif @@ -36,7 +36,7 @@ struct Random { using UniformIntDist = std::uniform_int_distribution; }; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template <> struct Random { using Engine = thrust::minstd_rand; diff --git a/paddle/fluid/operators/range_op.cu b/paddle/fluid/operators/range_op.cu index c527bc74eee93..f2c78e0f70b32 100644 --- a/paddle/fluid/operators/range_op.cu +++ b/paddle/fluid/operators/range_op.cu @@ -12,6 +12,7 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ +#include #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/range_op.h" #include "paddle/fluid/platform/cuda_primitives.h" @@ -33,13 +34,26 @@ class CUDARangeKernel : public framework::OpKernel { auto* step_t = context.Input("Step"); auto* out = context.Output("Out"); + T start, end, step; framework::Tensor n; - framework::TensorCopy(*start_t, platform::CPUPlace(), &n); - T start = n.data()[0]; - framework::TensorCopy(*end_t, platform::CPUPlace(), &n); - T end = n.data()[0]; - framework::TensorCopy(*step_t, platform::CPUPlace(), &n); - T step = n.data()[0]; + if (::paddle::platform::is_cpu_place(start_t->place())) { + start = start_t->data()[0]; + } else { + framework::TensorCopy(*start_t, platform::CPUPlace(), &n); + start = n.data()[0]; + } + if (::paddle::platform::is_cpu_place(end_t->place())) { + end = end_t->data()[0]; + } else { + framework::TensorCopy(*end_t, platform::CPUPlace(), &n); + end = n.data()[0]; + } + if (::paddle::platform::is_cpu_place(step_t->place())) { + step = step_t->data()[0]; + } else { + framework::TensorCopy(*step_t, platform::CPUPlace(), &n); + step = n.data()[0]; + } int64_t size = 0; GetSize(start, end, step, &size); @@ -47,7 +61,7 @@ class CUDARangeKernel : public framework::OpKernel { T* out_data = out->mutable_data(context.GetPlace()); auto stream = context.cuda_device_context().stream(); - int block = 512; + int block = std::min(size, static_cast(256)); int grid = (size + block - 1) / block; RangeKernel<<>>(start, step, size, out_data); } diff --git a/paddle/fluid/operators/rank_attention.cu.h b/paddle/fluid/operators/rank_attention.cu.h index 27fe67e73cde0..8ec138c8824fa 100644 --- a/paddle/fluid/operators/rank_attention.cu.h +++ b/paddle/fluid/operators/rank_attention.cu.h @@ -50,7 +50,7 @@ __global__ void expand_input_by_rank_kernel( } template -void expand_rank_attention_input(cudaStream_t stream, const T* input, +void expand_rank_attention_input(gpuStream_t stream, const T* input, int input_row, int input_col, T* output, int output_row, int output_col, const int* rank_offset, int rank_offset_row, @@ -93,7 +93,7 @@ __global__ void expand_rank_attention_param_kernel( } template -void expand_rank_attention_param(cudaStream_t stream, const T* input, +void expand_rank_attention_param(gpuStream_t stream, const T* input, int input_row, int input_col, const int* rank_offset, int rank_offset_row, int rank_offset_col, const T* param, @@ -133,7 +133,7 @@ __global__ void merge_param_gradient_kernel( } template -void merge_rank_attention_param_grad(cudaStream_t stream, T* expanded_grad, +void merge_rank_attention_param_grad(gpuStream_t stream, T* expanded_grad, int expanded_grad_row, int expanded_grad_col, T* param_grad, int param_grad_row, int param_grad_col, diff --git a/paddle/fluid/operators/rank_attention_op.cu b/paddle/fluid/operators/rank_attention_op.cu index 6c242e156a5b4..aaa4eec7c1bf3 100644 --- a/paddle/fluid/operators/rank_attention_op.cu +++ b/paddle/fluid/operators/rank_attention_op.cu @@ -12,7 +12,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include #include #include "paddle/fluid/framework/eigen.h" #include "paddle/fluid/operators/math/blas.h" diff --git a/paddle/fluid/operators/reader/blocking_queue.h b/paddle/fluid/operators/reader/blocking_queue.h index 8929da20b53c2..f126070a7eb96 100644 --- a/paddle/fluid/operators/reader/blocking_queue.h +++ b/paddle/fluid/operators/reader/blocking_queue.h @@ -45,7 +45,11 @@ class BlockingQueue { std::unique_lock lock(mutex_); send_cv_.wait( lock, [&] { return queue_.size() < capacity_ || closed_ || killed_; }); - EnforceNotKilled(); + if (killed_) { + VLOG(3) + << "WARNING:: Sending an element to a killed reader::BlokcingQueue"; + return false; + } if (closed_) { VLOG(5) << "WARNING: Sending an element to a closed reader::BlokcingQueue."; @@ -66,7 +70,11 @@ class BlockingQueue { std::unique_lock lock(mutex_); send_cv_.wait( lock, [&] { return queue_.size() < capacity_ || closed_ || killed_; }); - EnforceNotKilled(); + if (killed_) { + VLOG(3) + << "WARNING:: Sending an element to a killed reader::BlokcingQueue"; + return false; + } if (closed_) { VLOG(5) << "WARNING: Sending an element to a closed reader::BlokcingQueue."; diff --git a/paddle/fluid/operators/reader/buffered_reader.cc b/paddle/fluid/operators/reader/buffered_reader.cc index 2bd53a35b6d9d..b29493404f453 100644 --- a/paddle/fluid/operators/reader/buffered_reader.cc +++ b/paddle/fluid/operators/reader/buffered_reader.cc @@ -39,7 +39,7 @@ BufferedReader::BufferedReader( buffer_size_(buffer_size), pin_memory_(pin_memory) { VLOG(1) << "BufferedReader"; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(place_) && !pin_memory) { int dev_idx = BOOST_GET_CONST(platform::CUDAPlace, place_).device; compute_stream_ = @@ -74,7 +74,7 @@ void BufferedReader::ReadAsync(size_t i) { return -1UL; } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) // @{ Group GPU Place if (platform::is_gpu_place(place_)) { TensorVec &cuda = cuda_buffer_[i]; if (cuda.empty()) { @@ -142,10 +142,17 @@ void BufferedReader::ReadAsync(size_t i) { // cuda memory immediately without waiting cuda kernel ends platform::SetDeviceId( BOOST_GET_CONST(platform::CUDAPlace, place_).device); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + hipEventRecord(events_[i].get(), compute_stream_)); + PADDLE_ENFORCE_CUDA_SUCCESS( + hipStreamWaitEvent(stream_.get(), events_[i].get(), 0)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( cudaEventRecord(events_[i].get(), compute_stream_)); PADDLE_ENFORCE_CUDA_SUCCESS( cudaStreamWaitEvent(stream_.get(), events_[i].get(), 0)); +#endif platform::RecordEvent record_event("BufferedReader:MemoryCopy"); for (size_t i = 0; i < cpu.size(); ++i) { @@ -174,14 +181,22 @@ void BufferedReader::ReadAsync(size_t i) { memory::Copy(BOOST_GET_CONST(platform::CUDAPlace, place_), gpu_ptr, cuda_pinned_place, cuda_pinned_ptr, size, stream_.get()); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream_.get())); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream_.get())); +#endif } cuda[i].set_lod(cpu[i].lod()); } +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(hipStreamSynchronize(stream_.get())); +#else PADDLE_ENFORCE_CUDA_SUCCESS(cudaStreamSynchronize(stream_.get())); +#endif } } -#endif +#endif // @} End Group GPU Place return i; })); } diff --git a/paddle/fluid/operators/reader/buffered_reader.h b/paddle/fluid/operators/reader/buffered_reader.h index 041d36a93432e..fbc46aceb8130 100644 --- a/paddle/fluid/operators/reader/buffered_reader.h +++ b/paddle/fluid/operators/reader/buffered_reader.h @@ -21,7 +21,7 @@ #include "ThreadPool.h" #include "paddle/fluid/framework/reader.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include "paddle/fluid/platform/cuda_resource_pool.h" #include "paddle/fluid/platform/gpu_info.h" #endif @@ -68,8 +68,8 @@ class BufferedReader : public framework::DecoratedReader { std::vector cpu_buffer_; std::vector cuda_buffer_; size_t prev_pos_{-1UL}; -#ifdef PADDLE_WITH_CUDA - cudaStream_t compute_stream_; +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + gpuStream_t compute_stream_; std::shared_ptr stream_; std::vector> events_; #endif diff --git a/paddle/fluid/operators/recurrent_op.cc b/paddle/fluid/operators/recurrent_op.cc index 9766008963be0..92e5e4a0cd120 100644 --- a/paddle/fluid/operators/recurrent_op.cc +++ b/paddle/fluid/operators/recurrent_op.cc @@ -210,9 +210,10 @@ void RecurrentOp::RunImpl(const framework::Scope &scope, auto *block = Attr(kStepBlock); auto *program = block->Program(); - auto ctx = executor.Prepare( - *program, block->ID(), Attr>( - kSkipEagerDeletionVars) /*skip_ref_cnt_vars*/); + auto ctx = executor.Prepare(*program, block->ID(), + Attr>( + kSkipEagerDeletionVars), /*skip_ref_cnt_vars*/ + true); static std::mutex mutex; std::lock_guard lock(mutex); @@ -255,16 +256,6 @@ void RecurrentOp::RunImpl(const framework::Scope &scope, // Link inside::output -> outside::output // outside::output[seq_offset: seq_offset + 1] = inside::output executor.CreateVariables(ctx->prog_, &cur_scope, ctx->block_id_); - if (i > 0) { - LinkTensorWithCallback(scope, Outputs(kOutputs), cur_scope, - Outputs(kOutputs), - [&](const framework::LoDTensor &src_tensor, - framework::LoDTensor *dst_tensor) { - framework::Tensor src_slice = - src_tensor.Slice(seq_offset, seq_offset + 1); - dst_tensor->ShareDataWith(src_slice); - }); - } // Linked now, execute! executor.RunPreparedContext(ctx.get(), &cur_scope, @@ -284,6 +275,14 @@ void RecurrentOp::RunImpl(const framework::Scope &scope, // early. framework::TensorCopy(src_tensor, place, dev_ctx, &dst_out); }); + } else { + LinkTensorWithCallback( + cur_scope, Outputs(kOutputs), scope, Outputs(kOutputs), + [&](const framework::LoDTensor &src_tensor, + framework::LoDTensor *dst_tensor) { + auto dst_out = dst_tensor->Slice(seq_offset, seq_offset + 1); + framework::TensorCopy(src_tensor, place, dev_ctx, &dst_out); + }); } scopes.ForwardNext(); diff --git a/paddle/fluid/operators/reduce_ops/CMakeLists.txt b/paddle/fluid/operators/reduce_ops/CMakeLists.txt index c32301e5e08c5..92107c9dc442e 100644 --- a/paddle/fluid/operators/reduce_ops/CMakeLists.txt +++ b/paddle/fluid/operators/reduce_ops/CMakeLists.txt @@ -13,7 +13,7 @@ else() register_operators() endif() -if(WITH_GPU) +if(WITH_GPU OR WITH_ROCM) file(GLOB OPS RELATIVE "${CMAKE_CURRENT_SOURCE_DIR}" "*.part.cu") string(REPLACE ".part.cu" "" OPS "${OPS}") @@ -38,3 +38,7 @@ if(WITH_GPU) nv_test(check_reduce_rank_test SRCS check_reduce_rank_test.cu DEPS tensor) endif() endif() + +if(WITH_ROCM) + hip_test(check_reduce_rank_test SRCS check_reduce_rank_test.cu DEPS tensor) +endif() diff --git a/paddle/fluid/operators/reduce_ops/cub_reduce.h b/paddle/fluid/operators/reduce_ops/cub_reduce.h index 49bcbf3abb1b3..29e46e091d068 100644 --- a/paddle/fluid/operators/reduce_ops/cub_reduce.h +++ b/paddle/fluid/operators/reduce_ops/cub_reduce.h @@ -20,7 +20,15 @@ #include #include -#include // NOLINT +#ifdef __NVCC__ +#include "cub/cub.cuh" // NOLINT +#endif + +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/framework/tensor.h" #include "paddle/fluid/framework/tensor_util.h" @@ -153,7 +161,11 @@ static inline std::vector GetStrides(const std::vector& dims, return strides; } +#ifdef __HIPCC__ +constexpr int kMaxBlockDim = 256; +#else constexpr int kMaxBlockDim = 512; +#endif static inline int GetDesiredBlockDim(int block_dim) { return block_dim >= kMaxBlockDim @@ -188,7 +200,7 @@ static void TensorReduceImpl( int left_num, int reduce_num, const std::vector& x_strides, const std::vector& reduce_dim, const std::vector& reduce_strides, const std::vector& left_dim, const std::vector& left_strides, - cudaStream_t stream) { + gpuStream_t stream) { #define CUB_RANK_CASE(i, ...) \ case i: { \ constexpr auto kRank = i; \ @@ -280,7 +292,7 @@ template void TensorReduce(const framework::Tensor& x, framework::Tensor* y, std::vector origin_reduce_dims, const Ty& init, const ReduceOp& reducer, const TransformOp& transformer, - cudaStream_t stream) { + gpuStream_t stream) { auto x_dim = framework::vectorize(x.dims()); std::vector new_x_dim, new_reduce_dims; int is_reduced = 0; @@ -362,11 +374,11 @@ struct TensorReduceFunctor { const double& init; const ReduceOp& reducer; const TransformOp& transformer; - cudaStream_t stream; + gpuStream_t stream; TensorReduceFunctor(const framework::Tensor& x, framework::Tensor* y, std::vector origin_reduce_dims, const double& init, const ReduceOp& reducer, const TransformOp& transformer, - cudaStream_t stream) + gpuStream_t stream) : x(x), y(y), origin_reduce_dims(origin_reduce_dims), diff --git a/paddle/fluid/operators/reduce_ops/reduce_prod_op.cu b/paddle/fluid/operators/reduce_ops/reduce_prod_op.cu index 4434937f75397..44e76c78b1f3e 100644 --- a/paddle/fluid/operators/reduce_ops/reduce_prod_op.cu +++ b/paddle/fluid/operators/reduce_ops/reduce_prod_op.cu @@ -14,6 +14,17 @@ #include "paddle/fluid/operators/reduce_ops/reduce_prod_op.h" +#ifdef __HIPCC__ +// Eigen3/unsupported/Eigen/CXX11/src/Tensor/TensorReductionGpu.h:922 +// do not support double in HIPCC platform (Eigen3 to be fixed) +REGISTER_OP_CUDA_KERNEL(reduce_prod, + ops::ReduceKernel, + ops::ReduceKernel, + ops::ReduceKernel); +#else REGISTER_OP_CUDA_KERNEL(reduce_prod, ops::ReduceKernel, @@ -23,3 +34,4 @@ REGISTER_OP_CUDA_KERNEL(reduce_prod, int, ops::ProdFunctor>, ops::ReduceKernel); +#endif diff --git a/paddle/fluid/operators/reshape_op.cc b/paddle/fluid/operators/reshape_op.cc index 0e11771d87c99..94efa70e467bc 100644 --- a/paddle/fluid/operators/reshape_op.cc +++ b/paddle/fluid/operators/reshape_op.cc @@ -654,7 +654,7 @@ REGISTER_OP_CPU_KERNEL_FUNCTOR( ops::ReshapeDoubleGradKernel, paddle::platform::complex128, ops::ReshapeDoubleGradKernel); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) REGISTER_OP_CUDA_KERNEL_FUNCTOR(reshape, float, ops::ReshapeKernel, double, ops::ReshapeKernel, int, ops::ReshapeKernel, uint8_t, ops::ReshapeKernel, int64_t, diff --git a/paddle/fluid/operators/rnn_op.cu.cc b/paddle/fluid/operators/rnn_op.cu.cc index 91d7d0f6783c7..2be59c620441d 100644 --- a/paddle/fluid/operators/rnn_op.cu.cc +++ b/paddle/fluid/operators/rnn_op.cu.cc @@ -16,7 +16,12 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/utils.h" +#ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" +#endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif namespace paddle { namespace operators { @@ -28,7 +33,11 @@ class RNNDescriptors { public: RNNDescriptors(int seq_length, int batch_size, int input_size, int hidden_size, int num_layers, float dropout_prob, int seed, +#ifdef PADDLE_WITH_HIP + int weight_numel, miopenRNNMode_t mode, bool is_bidirec, +#else int weight_numel, cudnnRNNMode_t mode, bool is_bidirec, +#endif bool is_test) : seq_length_(seq_length), batch_size_(batch_size), @@ -40,15 +49,23 @@ class RNNDescriptors { weight_numel_(weight_numel), mode_(mode), is_bidirec_(is_bidirec), - is_test_(is_test) {} + is_test_(is_test) { + } template +#ifdef PADDLE_WITH_HIP + void Create(const miopenHandle_t &handle, const platform::Place &place, +#else void Create(const cudnnHandle_t &handle, const platform::Place &place, +#endif const std::vector &sequence_length, size_t *workspace_size, size_t *reserve_size, framework::Tensor *dropout_state) { int numDirections = is_bidirec_ ? 2 : 1; +#ifdef PADDLE_WITH_HIP + miopenDataType_t cudnn_type = platform::CudnnDataType::type; +#else cudnnDataType_t cudnn_type = platform::CudnnDataType::type; - +#endif // ------------------- cudnn x, y descriptors --------------------- std::vector dims_x = {batch_size_, input_size_, 1}; std::vector strides_x = {input_size_, 1, 1}; @@ -59,7 +76,7 @@ class RNNDescriptors { y_descs_.emplace_back(y_desc_.descriptor(dims_y, strides_y)); } -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 if (!sequence_length.empty()) { x_seq_desc_.descriptor(seq_length_, batch_size_, input_size_, true, sequence_length); @@ -82,17 +99,30 @@ class RNNDescriptors { size_t state_size; bool is_initialized = dropout_state->IsInitialized(); if (!is_test_ && !is_initialized) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenDropoutGetStatesSize(handle, &state_size)); + dropout_state->mutable_data({static_cast(state_size)}, + place); +#else PADDLE_ENFORCE_CUDA_SUCCESS( platform::dynload::cudnnDropoutGetStatesSize(handle, &state_size)); dropout_state->mutable_data({static_cast(state_size)}, place); +#endif } dropout_desc_.descriptor(handle, place, is_initialized, dropout_prob_, is_test_ ? nullptr : dropout_state, seed_, state_size); // ------------------- cudnn rnn descriptors --------------------- -#if CUDNN_VERSION >= 6000 +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSetRNNDescriptor_V2( + rnn_desc_.desc(), hidden_size_, num_layers_, dropout_desc_.desc(), + miopenRNNlinear, + is_bidirec_ ? miopenRNNbidirection : miopenRNNunidirection, mode_, + miopenRNNwithBias, miopenRNNdefault, cudnn_type)); +#elif CUDNN_VERSION >= 6000 PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetRNNDescriptor_v6( handle, rnn_desc_.desc(), hidden_size_, num_layers_, dropout_desc_.desc(), CUDNN_LINEAR_INPUT, @@ -106,7 +136,7 @@ class RNNDescriptors { cudnn_type)); #endif -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 if (!sequence_length.empty()) { PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSetRNNPaddingMode( rnn_desc_.desc(), CUDNN_RNN_PADDED_IO_ENABLED)); @@ -115,8 +145,13 @@ class RNNDescriptors { // ------------------- cudnn weights_size --------------------- size_t weights_size_; +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNParamsSize( + handle, rnn_desc_.desc(), x_descs_[0], &weights_size_, cudnn_type)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnGetRNNParamsSize( handle, rnn_desc_.desc(), x_descs_[0], &weights_size_, cudnn_type)); +#endif PADDLE_ENFORCE_EQ( weights_size_, sizeof(T) * weight_numel_, platform::errors::InvalidArgument( @@ -126,7 +161,16 @@ class RNNDescriptors { int dim_tmp = weights_size_ / sizeof(T); std::vector dim_w = {dim_tmp, 1, 1}; weight_desc_.descriptor(layout, dim_w); - // ------------------- cudnn workspace, reserve size --------------------- +// ------------------- cudnn workspace, reserve size --------------------- +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenGetRNNWorkspaceSize( + handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), + workspace_size)); + PADDLE_ENFORCE_CUDA_SUCCESS( + platform::dynload::miopenGetRNNTrainingReserveSize( + handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), + reserve_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnGetRNNWorkspaceSize( handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), workspace_size)); @@ -134,7 +178,19 @@ class RNNDescriptors { platform::dynload::cudnnGetRNNTrainingReserveSize( handle, rnn_desc_.desc(), seq_length_, x_descs_.data(), reserve_size)); +#endif } +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t *x_descs() { return x_descs_.data(); } + miopenTensorDescriptor_t *y_descs() { return y_descs_.data(); } + miopenTensorDescriptor_t init_h_desc() { return init_h_desc_.desc(); } + miopenTensorDescriptor_t init_c_desc() { return init_c_desc_.desc(); } + miopenTensorDescriptor_t last_h_desc() { return last_h_desc_.desc(); } + miopenTensorDescriptor_t last_c_desc() { return last_c_desc_.desc(); } + miopenRNNDescriptor_t rnn_desc() { return rnn_desc_.desc(); } + miopenDropoutDescriptor_t dropout_desc() { return dropout_desc_.desc(); } + miopenTensorDescriptor_t weight_desc() { return weight_desc_.desc(); } +#else cudnnTensorDescriptor_t *x_descs() { return x_descs_.data(); } cudnnTensorDescriptor_t *y_descs() { return y_descs_.data(); } #if CUDNN_VERSION >= 7201 @@ -148,6 +204,7 @@ class RNNDescriptors { cudnnRNNDescriptor_t rnn_desc() { return rnn_desc_.desc(); } cudnnDropoutDescriptor_t dropout_desc() { return dropout_desc_.desc(); } cudnnFilterDescriptor_t weight_desc() { return weight_desc_.desc(); } +#endif private: int seq_length_; @@ -158,15 +215,24 @@ class RNNDescriptors { float dropout_prob_; int seed_; int weight_numel_; +#ifdef PADDLE_WITH_HIP + miopenRNNMode_t mode_; +#else cudnnRNNMode_t mode_; +#endif bool is_bidirec_; bool is_test_; +#ifdef PADDLE_WITH_HIP + std::vector x_descs_; + std::vector y_descs_; +#else std::vector x_descs_; std::vector y_descs_; +#endif platform::ScopedTensorDescriptor x_desc_; platform::ScopedTensorDescriptor y_desc_; -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 platform::ScopedRNNTensorDescriptor x_seq_desc_; platform::ScopedRNNTensorDescriptor y_seq_desc_; #endif @@ -193,7 +259,7 @@ bool is_continuous(const Type &weight_list) { } template -void weight_to_tensor(const platform::Place &place, cudaStream_t stream, +void weight_to_tensor(const platform::Place &place, gpuStream_t stream, const std::vector &weight_list, Tensor *weight) { auto weight_data = weight->data(); @@ -211,7 +277,7 @@ void weight_to_tensor(const platform::Place &place, cudaStream_t stream, } template -void weight_to_tensor_list(const platform::Place &place, cudaStream_t stream, +void weight_to_tensor_list(const platform::Place &place, gpuStream_t stream, std::vector *weight_grad, const std::vector &weight_input, const Tensor *weight) { @@ -247,6 +313,17 @@ class RNNCudnnKernel : public framework::OpKernel { int hidden_size = ctx.Attr("hidden_size"); int num_layers = ctx.Attr("num_layers"); auto mode = ctx.Attr("mode"); +#ifdef PADDLE_WITH_HIP + miopenRNNMode_t rnn_mode = miopenLSTM; + if (mode == "LSTM") + rnn_mode = miopenLSTM; + else if (mode == "GRU") + rnn_mode = miopenGRU; + else if (mode == "RNN_RELU") + rnn_mode = miopenRNNRELU; + else if (mode == "RNN_TANH") + rnn_mode = miopenRNNTANH; +#else cudnnRNNMode_t rnn_mode = CUDNN_LSTM; if (mode == "LSTM") rnn_mode = CUDNN_LSTM; @@ -256,6 +333,7 @@ class RNNCudnnKernel : public framework::OpKernel { rnn_mode = CUDNN_RNN_RELU; else if (mode == "RNN_TANH") rnn_mode = CUDNN_RNN_TANH; +#endif else PADDLE_THROW(platform::errors::InvalidArgument( "rnn_mode should be LSTM, GRU, RNN_RELU or RNN_TANH, but received: " @@ -285,7 +363,11 @@ class RNNCudnnKernel : public framework::OpKernel { T *out_data = out->mutable_data(ctx.GetPlace()); T *last_h_data = state[0]->mutable_data(ctx.GetPlace()); T *last_c_data = nullptr; +#ifdef PADDLE_WITH_HIP + if (rnn_mode == miopenLSTM) { +#else if (rnn_mode == CUDNN_LSTM) { +#endif init_c_data = pre_state[1]->data(); last_c_data = state[1]->mutable_data(ctx.GetPlace()); } @@ -362,8 +444,17 @@ class RNNCudnnKernel : public framework::OpKernel { &workspace_data_, workspace_size); } else { if (!has_seq_length) { - // for train - // This interface is used when the input/output is unpadded. +// for train +// This interface is used when the input/output is unpadded. +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNForwardTraining( + handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), x_data, + rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, + rnn.weight_desc(), w_data, rnn.y_descs(), out_data, + rnn.last_h_desc(), last_h_data, rnn.last_c_desc(), last_c_data, + workspace_data_.data(), workspace_size, reserve_data, + reserve_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardTraining( handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), x_data, rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, @@ -371,8 +462,9 @@ class RNNCudnnKernel : public framework::OpKernel { rnn.last_h_desc(), last_h_data, rnn.last_c_desc(), last_c_data, workspace_data_.data(), workspace_size, reserve_data, reserve_size)); +#endif } else { -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 // for train // This interface is used when the input/output is padded. PADDLE_ENFORCE_CUDA_SUCCESS( @@ -394,23 +486,36 @@ class RNNCudnnKernel : public framework::OpKernel { } } +#ifdef PADDLE_WITH_HIP + void RNNInferece(const bool &has_seq_length, const miopenHandle_t &handle, +#else void RNNInferece(const bool &has_seq_length, const cudnnHandle_t &handle, +#endif const int &seq_length, RNNDescriptors *rnn, const T *x_data, const T *init_h_data, const T *init_c_data, const T *w_data, T *out_data, T *last_h_data, T *last_c_data, framework::Tensor *workspace_data, const size_t &workspace_size) const { if (!has_seq_length) { - // for inference - // This interface is used when the input/output is unpadded. +// for inference +// This interface is used when the input/output is unpadded. +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNForwardInference( + handle, rnn->rnn_desc(), seq_length, rnn->x_descs(), x_data, + rnn->init_h_desc(), init_h_data, rnn->init_c_desc(), init_c_data, + rnn->weight_desc(), w_data, rnn->y_descs(), out_data, + rnn->last_h_desc(), last_h_data, rnn->last_c_desc(), last_c_data, + workspace_data->data(), workspace_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardInference( handle, rnn->rnn_desc(), seq_length, rnn->x_descs(), x_data, rnn->init_h_desc(), init_h_data, rnn->init_c_desc(), init_c_data, rnn->weight_desc(), w_data, rnn->y_descs(), out_data, rnn->last_h_desc(), last_h_data, rnn->last_c_desc(), last_c_data, workspace_data->data(), workspace_size)); +#endif } else { -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 // for inference // This interface is used when the input/output is padded. PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNForwardInferenceEx( @@ -457,6 +562,17 @@ class RNNGradCudnnKernel : public framework::OpKernel { int hidden_size = ctx.Attr("hidden_size"); int num_layers = ctx.Attr("num_layers"); auto mode = ctx.Attr("mode"); +#ifdef PADDLE_WITH_HIP + miopenRNNMode_t rnn_mode = miopenLSTM; + if (mode == "LSTM") + rnn_mode = miopenLSTM; + else if (mode == "GRU") + rnn_mode = miopenGRU; + else if (mode == "RNN_RELU") + rnn_mode = miopenRNNRELU; + else if (mode == "RNN_TANH") + rnn_mode = miopenRNNTANH; +#else cudnnRNNMode_t rnn_mode = CUDNN_LSTM; if (mode == "LSTM") rnn_mode = CUDNN_LSTM; @@ -466,6 +582,7 @@ class RNNGradCudnnKernel : public framework::OpKernel { rnn_mode = CUDNN_RNN_RELU; else if (mode == "RNN_TANH") rnn_mode = CUDNN_RNN_TANH; +#endif else PADDLE_THROW(platform::errors::InvalidArgument( "rnn_mode should be LSTM, GRU, RNN_RELU or RNN_TANH, but received: " @@ -532,7 +649,11 @@ class RNNGradCudnnKernel : public framework::OpKernel { ? pre_state_grad[0]->mutable_data(ctx.GetPlace()) : nullptr; T *init_c_grad_data = nullptr; +#ifdef PADDLE_WITH_HIP + if (rnn_mode == miopenLSTM) { +#else if (rnn_mode == CUDNN_LSTM) { +#endif init_c_data = pre_state[1]->data(); // last_c_data = state[1]->data(); last_c_grad_data = state_grad[1]->data(); @@ -579,6 +700,17 @@ class RNNGradCudnnKernel : public framework::OpKernel { if (!has_seq_length) { if (in_grad) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNBackwardData( + handle, rnn.rnn_desc(), seq_length, rnn.y_descs(), out_data, + rnn.y_descs(), out_grad_data, rnn.last_h_desc(), last_h_grad_data, + rnn.last_c_desc(), last_c_grad_data, rnn.weight_desc(), weight_data, + rnn.init_h_desc(), init_h_data, rnn.init_c_desc(), init_c_data, + rnn.x_descs(), in_grad_data, rnn.init_h_desc(), init_h_grad_data, + rnn.init_c_desc(), init_c_grad_data, + workspace_data_.data(), workspace_size, + const_cast(reserve_data), reserve_size)); +#else // This interface is used when the input/output is unpadded. PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNBackwardData( handle, rnn.rnn_desc(), seq_length, rnn.y_descs(), out_data, @@ -589,17 +721,27 @@ class RNNGradCudnnKernel : public framework::OpKernel { rnn.init_c_desc(), init_c_grad_data, workspace_data_.data(), workspace_size, const_cast(reserve_data), reserve_size)); +#endif } if (!weight_grad_list.empty()) { +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenRNNBackwardWeights( + handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), input->data(), + rnn.init_h_desc(), init_h_data, rnn.y_descs(), out->data(), + rnn.weight_desc(), weight_grad_data, + workspace_data_.data(), workspace_size, + const_cast(reserve_data), reserve_size)); +#else PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnRNNBackwardWeights( handle, rnn.rnn_desc(), seq_length, rnn.x_descs(), input->data(), rnn.init_h_desc(), init_h_data, rnn.y_descs(), out->data(), workspace_data_.data(), workspace_size, rnn.weight_desc(), weight_grad_data, const_cast(reserve_data), reserve_size)); +#endif } } else { -#if CUDNN_VERSION >= 7201 +#if defined(PADDLE_WITH_CUDA) && CUDNN_VERSION >= 7201 // for train // This interface is used when the input/output is padded. if (in_grad) { @@ -638,7 +780,13 @@ class RNNGradCudnnKernel : public framework::OpKernel { } // namespace paddle namespace ops = paddle::operators; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL(rnn, ops::RNNCudnnKernel); +REGISTER_OP_CUDA_KERNEL(rnn_grad, ops::RNNGradCudnnKernel); +#else REGISTER_OP_CUDA_KERNEL(rnn, ops::RNNCudnnKernel, ops::RNNCudnnKernel); REGISTER_OP_CUDA_KERNEL(rnn_grad, ops::RNNGradCudnnKernel, ops::RNNGradCudnnKernel); +#endif diff --git a/paddle/fluid/operators/roi_align_op.cu b/paddle/fluid/operators/roi_align_op.cu index 074a00fb1c33c..d6ba399439d02 100644 --- a/paddle/fluid/operators/roi_align_op.cu +++ b/paddle/fluid/operators/roi_align_op.cu @@ -124,11 +124,9 @@ __global__ void GPUROIAlignForward( T roi_width = roi_xmax - roi_xmin; T roi_height = roi_ymax - roi_ymin; + roi_width = max(roi_width, static_cast(1.)); + roi_height = max(roi_height, static_cast(1.)); - if (!continuous_coordinate) { - roi_width = max(roi_width, static_cast(1.)); - roi_height = max(roi_height, static_cast(1.)); - } T bin_size_h = static_cast(roi_height) / static_cast(pooled_height); T bin_size_w = static_cast(roi_width) / static_cast(pooled_width); @@ -182,10 +180,9 @@ __global__ void GPUROIAlignBackward( T roi_width = roi_xmax - roi_xmin; T roi_height = roi_ymax - roi_ymin; - if (!continuous_coordinate) { - roi_width = max(roi_width, static_cast(1.)); - roi_height = max(roi_height, static_cast(1.)); - } + roi_width = max(roi_width, static_cast(1.)); + roi_height = max(roi_height, static_cast(1.)); + T bin_size_h = static_cast(roi_height) / static_cast(pooled_height); T bin_size_w = static_cast(roi_width) / static_cast(pooled_width); diff --git a/paddle/fluid/operators/roi_align_op.h b/paddle/fluid/operators/roi_align_op.h index d03cd617e6df6..46564ed4f629d 100644 --- a/paddle/fluid/operators/roi_align_op.h +++ b/paddle/fluid/operators/roi_align_op.h @@ -226,10 +226,8 @@ class CPUROIAlignOpKernel : public framework::OpKernel { T roi_width = roi_xmax - roi_xmin; T roi_height = roi_ymax - roi_ymin; - if (!aligned) { - roi_width = std::max(roi_width, static_cast(1.)); - roi_height = std::max(roi_height, static_cast(1.)); - } + roi_width = std::max(roi_width, static_cast(1.)); + roi_height = std::max(roi_height, static_cast(1.)); T bin_size_h = static_cast(roi_height) / static_cast(pooled_height); T bin_size_w = static_cast(roi_width) / static_cast(pooled_width); @@ -362,11 +360,9 @@ class CPUROIAlignGradOpKernel : public framework::OpKernel { T roi_width = roi_xmax - roi_xmin; T roi_height = roi_ymax - roi_ymin; + roi_width = std::max(roi_width, static_cast(1.)); + roi_height = std::max(roi_height, static_cast(1.)); - if (!aligned) { - roi_width = std::max(roi_width, static_cast(1.)); - roi_height = std::max(roi_height, static_cast(1.)); - } T bin_size_h = static_cast(roi_height) / static_cast(pooled_height); T bin_size_w = static_cast(roi_width) / static_cast(pooled_width); for (int c = 0; c < channels; ++c) { diff --git a/paddle/fluid/operators/scale_op.cc b/paddle/fluid/operators/scale_op.cc index 281689d3bdaff..a9b1f299dab82 100644 --- a/paddle/fluid/operators/scale_op.cc +++ b/paddle/fluid/operators/scale_op.cc @@ -128,6 +128,8 @@ REGISTER_OPERATOR(scale, ops::ScaleOp, ops::ScaleOpMaker, REGISTER_OP_CPU_KERNEL( scale, ops::ScaleKernel, ops::ScaleKernel, + ops::ScaleKernel, ops::ScaleKernel, ops::ScaleKernel, ops::ScaleKernel, diff --git a/paddle/fluid/operators/seed_op.cu b/paddle/fluid/operators/seed_op.cu index 8070f01e9b5a1..c84407ba52dfd 100644 --- a/paddle/fluid/operators/seed_op.cu +++ b/paddle/fluid/operators/seed_op.cu @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include "paddle/fluid/operators/seed_op.h" namespace paddle { diff --git a/paddle/fluid/operators/segment_pool_op.h b/paddle/fluid/operators/segment_pool_op.h index 23b0c31608d26..5f9635c8ae111 100644 --- a/paddle/fluid/operators/segment_pool_op.h +++ b/paddle/fluid/operators/segment_pool_op.h @@ -63,7 +63,7 @@ void SegmentKernelLaunchHelper(const framework::ExecutionContext& context) { auto& dev_ctx = context.template device_context(); set_zero(dev_ctx, output, static_cast(0)); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (!cpu_place) { Tensor length; length.mutable_data(framework::make_ddim({1}), @@ -71,9 +71,15 @@ void SegmentKernelLaunchHelper(const framework::ExecutionContext& context) { IndexT* length_data = length.data(); const IndexT* segment_ids = segment->data(); +#ifdef PADDLE_WITH_HIP + PADDLE_ENFORCE_CUDA_SUCCESS( + hipMemcpy(length_data, segment_ids + num_indices - 1, sizeof(IndexT), + hipMemcpyDeviceToHost)); +#else PADDLE_ENFORCE_CUDA_SUCCESS( cudaMemcpy(length_data, segment_ids + num_indices - 1, sizeof(IndexT), cudaMemcpyDeviceToHost)); +#endif IndexT length_host = length_data[0]; length_host++; diff --git a/paddle/fluid/operators/select_op_helper.h b/paddle/fluid/operators/select_op_helper.h index 5df4f8c4a543b..322843021766f 100644 --- a/paddle/fluid/operators/select_op_helper.h +++ b/paddle/fluid/operators/select_op_helper.h @@ -37,7 +37,7 @@ inline int GetBranchNumber(const framework::LoDTensor &mask) { } // when platform::is_gpu_place(mask.place()) is ture std::unique_ptr cpu_mask{new framework::LoDTensor()}; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) framework::TensorCopySync(mask, platform::CPUPlace(), cpu_mask.get()); #else PADDLE_THROW(platform::errors::PreconditionNotMet( diff --git a/paddle/fluid/operators/sequence_ops/sequence_mask_op.h b/paddle/fluid/operators/sequence_ops/sequence_mask_op.h index 3abaeccb28375..2ce0b02d437b7 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_mask_op.h +++ b/paddle/fluid/operators/sequence_ops/sequence_mask_op.h @@ -14,7 +14,7 @@ #pragma once -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) #include #include #include @@ -107,7 +107,7 @@ class SequenceMaskKernel : public framework::OpKernel { auto *x_data = x->data(); auto x_numel = x->numel(); if (maxlen < 0) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) VLOG(10) << "SequenceMaskOp on GPU may be slow when maxlen is not provided."; maxlen = static_cast( diff --git a/paddle/fluid/operators/sequence_ops/sequence_reverse_op.h b/paddle/fluid/operators/sequence_ops/sequence_reverse_op.h index c84028bd63a8e..2094572a78a52 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_reverse_op.h +++ b/paddle/fluid/operators/sequence_ops/sequence_reverse_op.h @@ -130,13 +130,13 @@ class SequenceReverseOpKernel : public framework::OpKernel { const size_t *lod; size_t lod_count = x.lod()[0].size(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { lod = x.lod()[0].CUDAData(ctx.GetPlace()); } else { #endif lod = x.lod()[0].data(); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) } #endif diff --git a/paddle/fluid/operators/sequence_ops/sequence_softmax_cudnn_op.cu.cc b/paddle/fluid/operators/sequence_ops/sequence_softmax_cudnn_op.cu.cc index b33d87e644fd2..46e4196585bc8 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_softmax_cudnn_op.cu.cc +++ b/paddle/fluid/operators/sequence_ops/sequence_softmax_cudnn_op.cu.cc @@ -104,9 +104,18 @@ class SequenceSoftmaxGradCUDNNKernel : public framework::OpKernel { } // namespace paddle namespace ops = paddle::operators; + +#ifdef PADDLE_WITH_HIP +// MIOPEN not support float64 +REGISTER_OP_KERNEL(sequence_softmax, CUDNN, ::paddle::platform::CUDAPlace, + ops::SequenceSoftmaxCUDNNKernel); +REGISTER_OP_KERNEL(sequence_softmax_grad, CUDNN, ::paddle::platform::CUDAPlace, + ops::SequenceSoftmaxGradCUDNNKernel); +#else REGISTER_OP_KERNEL(sequence_softmax, CUDNN, ::paddle::platform::CUDAPlace, ops::SequenceSoftmaxCUDNNKernel, ops::SequenceSoftmaxCUDNNKernel); REGISTER_OP_KERNEL(sequence_softmax_grad, CUDNN, ::paddle::platform::CUDAPlace, ops::SequenceSoftmaxGradCUDNNKernel, ops::SequenceSoftmaxGradCUDNNKernel); +#endif diff --git a/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cc b/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cc index 992a0b458b1af..9a7bb67bdfc87 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cc +++ b/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cc @@ -36,7 +36,7 @@ class SequenceSoftmaxOp : public framework::OperatorWithKernel { // choose cudnn kernel if the runtime supported. bool use_cudnn = ctx.Attr("use_cudnn"); bool runtime_cudnn_support = false; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { auto& dev_ctx = ctx.template device_context(); @@ -132,7 +132,7 @@ class SequenceSoftmaxGradOp : public framework::OperatorWithKernel { // choose cudnn kernel if the runtime supported. bool use_cudnn = ctx.Attr("use_cudnn"); bool runtime_cudnn_support = false; -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_gpu_place(ctx.GetPlace())) { auto& dev_ctx = ctx.template device_context(); diff --git a/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cu b/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cu index 58022c076cfde..220165ac1bd4f 100644 --- a/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cu +++ b/paddle/fluid/operators/sequence_ops/sequence_softmax_op.cu @@ -13,7 +13,16 @@ See the License for the specific language governing permissions and limitations under the License. */ #include -#include // NOLINT + +#ifdef __NVCC__ +#include +#endif + +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif + #include "paddle/fluid/operators/math.h" #include "paddle/fluid/operators/sequence_ops/sequence_softmax_op.h" diff --git a/paddle/fluid/operators/set_value_op.cc b/paddle/fluid/operators/set_value_op.cc index a18238adcae19..94d34c648d174 100644 --- a/paddle/fluid/operators/set_value_op.cc +++ b/paddle/fluid/operators/set_value_op.cc @@ -57,8 +57,7 @@ class SetValue : public framework::OperatorWithKernel { framework::OpKernelType GetExpectedKernelType( const framework::ExecutionContext &ctx) const override { return framework::OpKernelType( - framework::proto::VarType::Type(ctx.Attr("dtype")), - ctx.GetPlace()); + OperatorWithKernel::IndicateVarDataType(ctx, "Input"), ctx.GetPlace()); } framework::OpKernelType GetKernelTypeForVar( diff --git a/paddle/fluid/operators/set_value_op.h b/paddle/fluid/operators/set_value_op.h index 6347bcd24791a..325a2b0b865e9 100644 --- a/paddle/fluid/operators/set_value_op.h +++ b/paddle/fluid/operators/set_value_op.h @@ -174,14 +174,13 @@ class SetValueKernel : public framework::OpKernel { auto steps_tensor_list = ctx.MultiInput("StepsTensorList"); - auto dtype = - static_cast(ctx.Attr("dtype")); auto axes = ctx.Attr>("axes"); auto starts = ctx.Attr>("starts"); auto ends = ctx.Attr>("ends"); auto steps = ctx.Attr>("steps"); auto shape = ctx.Attr>("shape"); + auto dtype = in->type(); if (!starts_tensor_list.empty()) { starts = GetDataFromTensorList(starts_tensor_list); } diff --git a/paddle/fluid/operators/shuffle_batch_op.h b/paddle/fluid/operators/shuffle_batch_op.h index ac8e3f0538f1d..f05af3f249ce0 100644 --- a/paddle/fluid/operators/shuffle_batch_op.h +++ b/paddle/fluid/operators/shuffle_batch_op.h @@ -33,7 +33,7 @@ namespace paddle { namespace operators { using Tensor = framework::Tensor; using LoDTensor = framework::LoDTensor; -#if defined(PADDLE_WITH_CUDA) +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) template using Vector = framework::Vector; #else diff --git a/paddle/fluid/operators/sigmoid_cross_entropy_with_logits_op.cu b/paddle/fluid/operators/sigmoid_cross_entropy_with_logits_op.cu index cdcd51904e884..8611249a29f63 100644 --- a/paddle/fluid/operators/sigmoid_cross_entropy_with_logits_op.cu +++ b/paddle/fluid/operators/sigmoid_cross_entropy_with_logits_op.cu @@ -11,7 +11,13 @@ 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. */ +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/memory/malloc.h" #include "paddle/fluid/operators/math.h" #include "paddle/fluid/operators/sigmoid_cross_entropy_with_logits_op.h" @@ -23,7 +29,11 @@ namespace operators { using Tensor = framework::Tensor; +#ifdef __HIPCC__ +static constexpr int kNumCUDAThreads = 256; +#else static constexpr int kNumCUDAThreads = 512; +#endif static constexpr int kNumMaxinumNumBlocks = 4096; static inline int NumBlocks(const int N) { diff --git a/paddle/fluid/operators/softmax_cudnn_op.cu b/paddle/fluid/operators/softmax_cudnn_op.cu index ac7963dd8ad43..b62d71bdbc4db 100644 --- a/paddle/fluid/operators/softmax_cudnn_op.cu +++ b/paddle/fluid/operators/softmax_cudnn_op.cu @@ -16,7 +16,11 @@ limitations under the License. */ #include "paddle/fluid/operators/math/math_cuda_utils.h" #include "paddle/fluid/operators/softmax_op.h" #include "paddle/fluid/platform/cuda_device_function.h" +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#else #include "paddle/fluid/platform/cudnn_helper.h" +#endif #include "paddle/fluid/platform/gpu_launch_config.h" namespace paddle { @@ -388,18 +392,30 @@ class SoftmaxCUDNNKernel : public framework::OpKernel { ScopedTensorDescriptor desc; std::vector tensor_dims = {N, dim, D, 1}; DataLayout layout = DataLayout::kNCHW; +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t desc_ = desc.descriptor(layout, tensor_dims); +#else cudnnTensorDescriptor_t desc_ = desc.descriptor(layout, tensor_dims); +#endif auto& dev_ctx = ctx.template device_context(); auto handle = dev_ctx.cudnn_handle(); + +#ifdef PADDLE_WITH_HIP + auto mode = axis == rank - 1 ? MIOPEN_SOFTMAX_MODE_INSTANCE + : MIOPEN_SOFTMAX_MODE_CHANNEL; + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSoftmaxForward( + handle, platform::CudnnDataType::kOne(), desc_, x->data(), + platform::CudnnDataType::kZero(), desc_, out_data)); +#else auto mode = axis == rank - 1 ? CUDNN_SOFTMAX_MODE_INSTANCE : CUDNN_SOFTMAX_MODE_CHANNEL; - PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSoftmaxForward( handle, CUDNN_SOFTMAX_ACCURATE, mode, platform::CudnnDataType::kOne(), desc_, x->data(), platform::CudnnDataType::kZero(), desc_, out_data)); +#endif } } }; @@ -496,19 +512,32 @@ class SoftmaxGradCUDNNKernel : public framework::OpKernel { ScopedTensorDescriptor desc; std::vector tensor_dims = {N, dim, D, 1}; DataLayout layout = DataLayout::kNCHW; +#ifdef PADDLE_WITH_HIP + miopenTensorDescriptor_t desc_ = desc.descriptor(layout, tensor_dims); +#else cudnnTensorDescriptor_t desc_ = desc.descriptor(layout, tensor_dims); +#endif auto& dev_ctx = ctx.template device_context(); auto handle = dev_ctx.cudnn_handle(); + +#ifdef PADDLE_WITH_HIP + auto mode = axis == rank - 1 ? MIOPEN_SOFTMAX_MODE_INSTANCE + : MIOPEN_SOFTMAX_MODE_CHANNEL; + PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::miopenSoftmaxBackward( + handle, platform::CudnnDataType::kOne(), desc_, out->data(), + desc_, dout->data(), platform::CudnnDataType::kZero(), desc_, + dx_data)); +#else auto mode = axis == rank - 1 ? CUDNN_SOFTMAX_MODE_INSTANCE : CUDNN_SOFTMAX_MODE_CHANNEL; - PADDLE_ENFORCE_CUDA_SUCCESS(platform::dynload::cudnnSoftmaxBackward( handle, CUDNN_SOFTMAX_ACCURATE, mode, platform::CudnnDataType::kOne(), desc_, out->data(), desc_, dout->data(), platform::CudnnDataType::kZero(), desc_, dx_data)); +#endif } } }; @@ -518,6 +547,15 @@ class SoftmaxGradCUDNNKernel : public framework::OpKernel { namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_KERNEL(softmax, CUDNN, plat::CUDAPlace, + ops::SoftmaxCUDNNKernel, + ops::SoftmaxCUDNNKernel); +REGISTER_OP_KERNEL(softmax_grad, CUDNN, plat::CUDAPlace, + ops::SoftmaxGradCUDNNKernel, + ops::SoftmaxGradCUDNNKernel); +#else REGISTER_OP_KERNEL(softmax, CUDNN, plat::CUDAPlace, ops::SoftmaxCUDNNKernel, ops::SoftmaxCUDNNKernel, @@ -526,3 +564,4 @@ REGISTER_OP_KERNEL(softmax_grad, CUDNN, plat::CUDAPlace, ops::SoftmaxGradCUDNNKernel, ops::SoftmaxGradCUDNNKernel, ops::SoftmaxGradCUDNNKernel); +#endif diff --git a/paddle/fluid/operators/softmax_op.cc b/paddle/fluid/operators/softmax_op.cc index 64030486eb4a5..a21ef252c03f7 100644 --- a/paddle/fluid/operators/softmax_op.cc +++ b/paddle/fluid/operators/softmax_op.cc @@ -22,6 +22,10 @@ limitations under the License. */ #include "paddle/fluid/platform/cudnn_helper.h" #endif +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif + #ifdef PADDLE_WITH_MKLDNN #include "paddle/fluid/platform/mkldnn_helper.h" #endif @@ -66,7 +70,7 @@ class SoftmaxOp : public framework::OperatorWithKernel { framework::DataLayout layout_ = framework::StringToDataLayout(data_format); auto input_data_type = OperatorWithKernel::IndicateVarDataType(ctx, "X"); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } @@ -190,7 +194,7 @@ class SoftmaxOpGrad : public framework::OperatorWithKernel { framework::DataLayout layout_ = framework::StringToDataLayout(data_format); auto input_data_type = OperatorWithKernel::IndicateVarDataType( ctx, framework::GradVarName("Out")); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::CanCUDNNBeUsed(ctx)) { library_ = framework::LibraryType::kCUDNN; } diff --git a/paddle/fluid/operators/softmax_with_cross_entropy_op.cc b/paddle/fluid/operators/softmax_with_cross_entropy_op.cc index 946ede475ce68..e58b39252ce5f 100644 --- a/paddle/fluid/operators/softmax_with_cross_entropy_op.cc +++ b/paddle/fluid/operators/softmax_with_cross_entropy_op.cc @@ -17,6 +17,7 @@ limitations under the License. */ #include #include #include +#include "paddle/fluid/framework/op_version_registry.h" namespace paddle { namespace operators { @@ -53,6 +54,10 @@ class SoftmaxWithCrossEntropyOpMaker "(bool, default: false), A flag to indicate whether to interpretant " "the given labels as soft labels.") .SetDefault(false); + AddAttr( + "softmax_switch", + "(bool, default: true), A flag to indicate whether to do softmax ") + .SetDefault(true); AddAttr( "numeric_stable_mode", "(bool, default: true), A flag to indicate whether to use more " @@ -312,3 +317,10 @@ REGISTER_OP_CPU_KERNEL(softmax_with_cross_entropy, REGISTER_OP_CPU_KERNEL(softmax_with_cross_entropy_grad, ops::SoftmaxWithCrossEntropyGradKernel, ops::SoftmaxWithCrossEntropyGradKernel); +REGISTER_OP_VERSION(softmax_with_cross_entropy) + .AddCheckpoint( + R"ROC( + Add a new attribute [softmax_switch] )ROC", + paddle::framework::compatible::OpVersionDesc().NewAttr( + "softmax_switch", "A flag to indicate whether to do softmax", + true)); diff --git a/paddle/fluid/operators/softmax_with_cross_entropy_op.cu b/paddle/fluid/operators/softmax_with_cross_entropy_op.cu index f3e7a33d9b1ab..2257d816d8921 100644 --- a/paddle/fluid/operators/softmax_with_cross_entropy_op.cu +++ b/paddle/fluid/operators/softmax_with_cross_entropy_op.cu @@ -8,7 +8,13 @@ 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 +#ifdef __NVCC__ +#include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#endif #include "paddle/fluid/operators/math/cross_entropy.h" #include "paddle/fluid/operators/math/math_function.h" #include "paddle/fluid/operators/softmax_with_cross_entropy_op.h" @@ -66,6 +72,57 @@ __global__ void SoftCrossEntropyGradientKernel(T* logit_grad, } } +template +__global__ void SoftLabelCrossEntropyGradientKernel(T* logit_grad, + const T* loss_grad, + const T* labels, + const int n, const int d, + const int remain) { + int ids = blockIdx.x * blockDim.x + threadIdx.x; + if (ids < n * d) { + int idx_n = ids / d; + int idx_remain = ids % remain; + int idx_loss = idx_n * remain + idx_remain; + logit_grad[ids] = loss_grad[idx_loss] * (-labels[ids] / logit_grad[ids]); + } +} + +template +__global__ void HardLabelCrossEntropyGradientKernel(T* logit_grad, + const int64_t* labels, + const int n, const int d, + const int remain, + const int ignore_index) { + CUDA_KERNEL_LOOP(index, n * remain) { + int idx_n = index / remain; + int idx_remain = index % remain; + int tmp = labels[index]; + int idx = idx_n * d + tmp * remain + idx_remain; + if (ignore_index != tmp) { + logit_grad[idx] = -static_cast(1.) / logit_grad[idx]; + } + } +} + +template +__global__ void ScaleCrossEntropyGradient(T* logit_grad, const T* loss_grad, + const int num, const int d, + const int remain, + const int64_t* labels, + const int ignore_index) { + CUDA_KERNEL_LOOP(index, num) { + int idx_n = index / d; + int idx_remain = index % remain; + int idx_lbl = idx_n * remain + idx_remain; + int k = (index % d) / remain; + if (labels[idx_lbl] == ignore_index || labels[idx_lbl] != k) { + logit_grad[index] = static_cast(0.); + } else { + logit_grad[index] *= loss_grad[idx_lbl]; + } + } +} + } // namespace static __device__ __forceinline__ platform::float16 exp_on_device( @@ -214,6 +271,60 @@ static __global__ void RowReductionForDiffMaxSum(const T* logits_data, if (threadIdx.x == 0) max_data[blockIdx.x] = 0; } +#ifdef __HIPCC__ // @{ HIP Seperate Kernel for RowReductionForDiffMaxSum +// Note(qili93): HIP do not support return in kernel, need to seperate +// RowReductionForDiffMaxSum into two kernels below +template +static __global__ void RowReductionForSum(const T* logits_data, T* max_data, + T* softmax, int64_t d, int axis_dim) { + __shared__ BlockReduceTempStorage temp_storage; + + int64_t remain = d / axis_dim; + int64_t idx_n = blockIdx.x / remain; + int64_t idx_remain = blockIdx.x % remain; + int64_t beg_idx = idx_n * d + threadIdx.x * remain + idx_remain; + int64_t end_idx = (idx_n + 1) * d; + + auto block_max = max_data[blockIdx.x]; + int64_t step = BlockDim * remain; + + softmax[beg_idx] = logits_data[beg_idx] - block_max; + T diff_max_sum = exp_on_device(softmax[beg_idx]); + auto idx = beg_idx + step; + while (idx < end_idx) { + softmax[idx] = logits_data[idx] - block_max; + diff_max_sum += exp_on_device(softmax[idx]); + idx += step; + } + + diff_max_sum = + BlockReduce(temp_storage).Reduce(diff_max_sum, cub::Sum()); + if (threadIdx.x == 0) max_data[blockIdx.x] = log_on_device(diff_max_sum); +} + +template +static __global__ void RowReductionForDiff(const T* logits_data, T* max_data, + T* softmax, int d, int axis_dim) { + int remain = d / axis_dim; + int idx_n = blockIdx.x / remain; + int idx_remain = blockIdx.x % remain; + int beg_idx = idx_n * d + threadIdx.x * remain + idx_remain; + int end_idx = (idx_n + 1) * d; + int step = BlockDim * remain; + + T diff_max_sum = max_data[blockIdx.x]; + softmax[beg_idx] -= diff_max_sum; + beg_idx += step; + while (beg_idx < end_idx) { + softmax[beg_idx] -= diff_max_sum; + beg_idx += step; + } + + __syncthreads(); + if (threadIdx.x == 0) max_data[blockIdx.x] = 0; +} +#endif // @} End HIP Seperate Kernel for RowReductionForDiffMaxSum + // Make sure that BlockDim <= axis_dim template static __global__ void RowReductionForSoftmaxAndCrossEntropy( @@ -248,6 +359,155 @@ static __global__ void RowReductionForSoftmaxAndCrossEntropy( if (threadIdx.x == 0) loss_data[blockIdx.x] = loss; } +// Make sure that BlockDim <= axis_dim +template +static __global__ void RowReductionForCrossEntropy(const T* logits_data, + const T* labels_data, + T* loss_data, int d, + int axis_dim) { + __shared__ BlockReduceTempStorage temp_storage; + + // logits, softmax, labels data view as [n, axis_dim, remain] + // loss_data view as [n, 1, remain] + // blockDim = n * remain, split blockIdx to idx_n and idx_remain + int remain = d / axis_dim; + int idx_n = blockIdx.x / remain; + int idx_remain = blockIdx.x % remain; + int beg_idx = idx_n * d + threadIdx.x * remain + idx_remain; + int end_idx = (idx_n + 1) * d; + + // log_diff_max_sum shares memory with loss + auto block_log_diff_max_sum = loss_data[blockIdx.x]; + auto tmp = log_on_device(logits_data[beg_idx]); // when not with softmax, + // softmax is stored in + // logits_data + auto loss = -labels_data[beg_idx] * tmp; + int step = BlockDim * remain; + beg_idx += step; + while (beg_idx < end_idx) { + tmp = log_on_device(logits_data[beg_idx]); // when not with softmax, + // softmax is stored in + // logits_data + loss -= (labels_data[beg_idx] * tmp); + beg_idx += step; + } + + loss = BlockReduce(temp_storage).Reduce(loss, cub::Sum()); + if (threadIdx.x == 0) loss_data[blockIdx.x] = loss; +} + +template +struct HardLabelCrossEntropyFunctor { + public: + HardLabelCrossEntropyFunctor(const int64_t* labels, T* loss, + const T* logits_data, int d, int axis_dim) + : labels_(labels), + loss_(loss), + logits_data_(logits_data), + d_(d), + axis_dim_(axis_dim) {} + + __device__ void operator()(int idx) const { + // logits view as [n, axis_dim, remain], where d = axis_dim * remain + int remain = d_ / axis_dim_; + int idx_n = idx / d_; + int idx_axis = (idx % d_) / remain; + int idx_remain = idx % remain; + // labels, loss view as [n, remain] + int idx_lbl = idx_n * remain + idx_remain; + // It also would ignore labels not in range(class_num). + if (idx_axis != labels_[idx_lbl]) { + } else { + loss_[idx_lbl] = -log_on_device(logits_data_[idx]); + } + } + + private: + const int64_t* labels_; + T* loss_; + const T* logits_data_; + int d_; + int axis_dim_; +}; + +template +struct HardLabelCrossEntropyFunctorWithIgnoreIdx { + public: + HardLabelCrossEntropyFunctorWithIgnoreIdx(const int64_t* labels, T* loss, + const T* logits_data, int d, + int axis_dim, int ignore_idx) + : labels_(labels), + loss_(loss), + logits_data_(logits_data), + d_(d), + axis_dim_(axis_dim), + ignore_idx_(ignore_idx) {} + + __device__ void operator()(int idx) const { + // logits view as [n, axis_dim, remain], where d = axis_dim * remain + int remain = d_ / axis_dim_; + int idx_n = idx / d_; + int idx_axis = (idx % d_) / remain; + int idx_remain = idx % remain; + // labels, loss view as [n, remain] + int idx_lbl = idx_n * remain + idx_remain; + + if (idx_axis == labels_[idx_lbl] && idx_axis != ignore_idx_) { + loss_[idx_lbl] = -log_on_device(logits_data_[idx]); + } + } + + private: + const int64_t* labels_; + T* loss_; + const T* logits_data_; + int d_; + int axis_dim_; + int ignore_idx_; +}; + +template +static void HardLabelCrossEntropy(const platform::CUDADeviceContext& ctx, + const T* logits_data, + const int64_t* labels_data, T* loss_data, + int n, int d, int axis_dim, int ignore_idx) { + constexpr int kMaxBlockDim = 512; + int block_dim = axis_dim >= kMaxBlockDim + ? kMaxBlockDim + : (1 << static_cast(std::log2(axis_dim))); + int grid_dim = n * d / axis_dim; + auto stream = ctx.stream(); + +#define CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ + case BlockDim: { \ + platform::ForRange for_range(ctx, n* d); \ + if (ignore_idx >= 0 && ignore_idx < axis_dim) { \ + for_range(HardLabelCrossEntropyFunctorWithIgnoreIdx( \ + labels_data, loss_data, logits_data, d, axis_dim, ignore_idx)); \ + } else { \ + for_range(HardLabelCrossEntropyFunctor(labels_data, loss_data, \ + logits_data, d, axis_dim)); \ + } \ + } break + + switch (block_dim) { + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(512); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(256); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(128); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(64); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(32); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(16); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(8); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(4); + CALL_HARD_LABEL_CROSS_ENTROPY_FUSED_KERNEL(2); + default: + PADDLE_THROW(platform::errors::Unavailable( + "Block Dimension must be 2^n in softmax_with_cross_entropy_op.")); + break; + } +#undef CALL_HARD_LABEL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL +} + template struct HardLabelSoftmaxWithCrossEntropyFunctor { public: @@ -338,13 +598,40 @@ static void HardLabelSoftmaxWithCrossEntropy( const platform::CUDADeviceContext& ctx, const T* logits_data, const int64_t* labels_data, T* loss_data, T* softmax_data, int64_t n, int64_t d, int axis_dim, int ignore_idx) { +#ifdef __HIPCC__ + // HIP platform will have loss nan if dim size > 256 + constexpr int kMaxBlockDim = 256; +#else constexpr int kMaxBlockDim = 512; +#endif int64_t block_dim = axis_dim >= kMaxBlockDim ? kMaxBlockDim : (1 << static_cast(std::log2(axis_dim))); int64_t grid_dim = n * d / axis_dim; auto stream = ctx.stream(); +#ifdef __HIPCC__ +#define CALL_HARD_LABEL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ + case BlockDim: { \ + hipLaunchKernelGGL(HIP_KERNEL_NAME(RowReductionForMax), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, \ + loss_data, d, axis_dim); \ + hipLaunchKernelGGL(HIP_KERNEL_NAME(RowReductionForSum), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, \ + loss_data, softmax_data, d, axis_dim); \ + hipLaunchKernelGGL(HIP_KERNEL_NAME(RowReductionForDiff), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, \ + loss_data, softmax_data, d, axis_dim); \ + platform::ForRange for_range(ctx, n* d); \ + if (ignore_idx >= 0 && ignore_idx < axis_dim) { \ + for_range(HardLabelSoftmaxWithCrossEntropyFunctorWithIgnoreIdx( \ + labels_data, loss_data, softmax_data, d, axis_dim, ignore_idx)); \ + } else { \ + for_range(HardLabelSoftmaxWithCrossEntropyFunctor( \ + labels_data, loss_data, softmax_data, d, axis_dim, ignore_idx)); \ + } \ + } break +#else #define CALL_HARD_LABEL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ case BlockDim: { \ RowReductionForMax<<>>( \ @@ -361,6 +648,7 @@ static void HardLabelSoftmaxWithCrossEntropy( labels_data, loss_data, softmax_data, d, axis_dim, ignore_idx)); \ } \ } break +#endif switch (block_dim) { CALL_HARD_LABEL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(512); @@ -383,13 +671,27 @@ static void HardLabelSoftmaxWithCrossEntropy( template static void SoftmaxWithCrossEntropyFusedKernel( const T* logits_data, const T* labels_data, T* softmax_data, T* loss_data, - int64_t n, int64_t d, int axis_dim, cudaStream_t stream) { + int64_t n, int64_t d, int axis_dim, gpuStream_t stream) { constexpr int kMaxBlockDim = 512; int64_t block_dim = axis_dim >= kMaxBlockDim ? kMaxBlockDim : (1 << static_cast(std::log2(axis_dim))); int64_t grid_dim = n * d / axis_dim; - +#ifdef __HIPCC__ +#define CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ + case BlockDim: \ + hipLaunchKernelGGL(HIP_KERNEL_NAME(RowReductionForMax), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, \ + loss_data, d, axis_dim); \ + hipLaunchKernelGGL(HIP_KERNEL_NAME(RowReductionForSum), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, \ + loss_data, softmax_data, d, axis_dim); \ + hipLaunchKernelGGL( \ + HIP_KERNEL_NAME(RowReductionForSoftmaxAndCrossEntropy), \ + dim3(grid_dim), dim3(BlockDim), 0, stream, logits_data, labels_data, \ + loss_data, softmax_data, d, axis_dim); \ + break +#else #define CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ case BlockDim: \ RowReductionForMax<<>>( \ @@ -400,6 +702,44 @@ static void SoftmaxWithCrossEntropyFusedKernel( T, BlockDim><<>>( \ logits_data, labels_data, loss_data, softmax_data, d, axis_dim); \ break +#endif + + switch (block_dim) { + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(512); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(256); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(128); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(64); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(32); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(16); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(8); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(4); + CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(2); + default: + PADDLE_THROW(platform::errors::Unavailable( + "Block Dimension must be 2^n in softmax_with_cross_entropy_op.")); + break; + } + +#undef CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL +} + +// not with softmax +template +static void CrossEntropyFusedKernel(const T* logits_data, const T* labels_data, + T* loss_data, int n, int d, int axis_dim, + gpuStream_t stream) { + constexpr int kMaxBlockDim = 512; + int block_dim = axis_dim >= kMaxBlockDim + ? kMaxBlockDim + : (1 << static_cast(std::log2(axis_dim))); + int grid_dim = n * d / axis_dim; + +#define CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(BlockDim) \ + case BlockDim: \ + RowReductionForCrossEntropy<<>>( \ + logits_data, labels_data, loss_data, d, axis_dim); \ + break switch (block_dim) { CALL_SOFTMAX_WITH_CROSS_ENTROPY_FUSED_KERNEL(512); @@ -428,6 +768,73 @@ class SoftmaxWithCrossEntropyCUDAKernel : public framework::OpKernel { platform::is_gpu_place(context.GetPlace()), true, platform::errors::Unavailable("softmax_with_cross_entropy operator's " "CUDA kernel only runs on GPU device.")); + const bool softmax_switch = context.Attr("softmax_switch"); + + // do not with softmax op, and input is softmax + if (!softmax_switch) { + const Tensor* softmax = context.Input("Logits"); + const Tensor* labels = context.Input("Label"); + Tensor* softmax_out = context.Output("Softmax"); + Tensor* loss = context.Output("Loss"); + + const int rank = softmax->dims().size(); + const int axis = CanonicalAxis(context.Attr("axis"), rank); + int axis_dim = softmax->dims()[axis]; + + const int n = SizeToAxis(axis, softmax->dims()); + const int d = SizeFromAxis(axis, softmax->dims()); + + auto* softmax_out_data = softmax_out->mutable_data(context.GetPlace()); + auto* loss_data = loss->mutable_data(context.GetPlace()); + + math::SetConstant set_constant; + set_constant(context.cuda_device_context(), loss, static_cast(0)); + if (axis_dim == 1) { + set_constant(context.cuda_device_context(), softmax_out, + static_cast(1)); + return; + } + + auto soft_label = context.Attr("soft_label"); + auto ignore_index = context.Attr("ignore_index"); + + Tensor softmax_2d, labels_2d, loss_2d, softmax_out_2d; + softmax_2d.ShareDataWith(*softmax).Resize({n, d}); + labels_2d.ShareDataWith(*labels).Resize({n, labels->numel() / n}); + loss_2d.ShareDataWith(*loss).Resize({n, 1}); + softmax_out_2d.ShareDataWith(*softmax_out).Resize({n, d}); + + // math::CrossEntropyFunctor support axis is the last + if (axis == -1) { + math::CrossEntropyFunctor()( + context.cuda_device_context(), &loss_2d, &softmax_2d, &labels_2d, + soft_label, ignore_index, axis_dim); + return; + } + + // if axis is not the last, we need a new impliment + if (soft_label) { + auto* logits_data = softmax->data(); + auto* labels_data = labels->data(); + CrossEntropyFusedKernel(logits_data, labels_data, loss_data, n, d, + axis_dim, + context.cuda_device_context().stream()); + } else { // HardLabel + auto* logits_data = softmax->data(); + auto* labels_data = labels->data(); + HardLabelCrossEntropy(context.cuda_device_context(), logits_data, + labels_data, loss_data, n, d, axis_dim, + ignore_index); + } + + // cause of input is softmax + // copy to output softmax, directly + framework::TensorCopy(*softmax, context.GetPlace(), + context.device_context(), softmax_out); + + return; + } + const Tensor* logits = context.Input("Logits"); const Tensor* labels = context.Input("Label"); Tensor* softmax = context.Output("Softmax"); @@ -514,6 +921,34 @@ class SoftmaxWithCrossEntropyGradCUDAKernel : public framework::OpKernel { int block = 512; auto stream = context.cuda_device_context().stream(); auto ignore_index = context.Attr("ignore_index"); + auto softmax_switch = context.Attr("softmax_switch"); + + // do not with softmax op, and input is softmax + if (!softmax_switch) { + if (context.Attr("soft_label")) { + int grid = (n * d + block - 1) / block; + const T* label_data = labels->data(); + SoftLabelCrossEntropyGradientKernel<<>>( + logit_grad_data, loss_grad_data, label_data, n, d, remain); + } else { + Tensor logits_grad_2d; + logits_grad_2d.ShareDataWith(*logit_grad).Resize({n, d}); + int grid = (n * remain + block - 1) / block; + const int64_t* label_data = labels->data(); + HardLabelCrossEntropyGradientKernel<<>>( + logit_grad_data, label_data, n, d, remain, ignore_index); + int num = n * d; + grid = (num + block - 1) / block; + ScaleCrossEntropyGradient<<>>( + logit_grad_data, loss_grad_data, num, d, remain, label_data, + ignore_index); + } + + return; + } + + // with softmax, continue + if (context.Attr("soft_label")) { int64_t grid = (n * d + block - 1) / block; const T* label_data = labels->data(); @@ -536,6 +971,16 @@ class SoftmaxWithCrossEntropyGradCUDAKernel : public framework::OpKernel { } // namespace paddle namespace ops = paddle::operators; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL( + softmax_with_cross_entropy, ops::SoftmaxWithCrossEntropyCUDAKernel, + ops::SoftmaxWithCrossEntropyCUDAKernel); +REGISTER_OP_CUDA_KERNEL( + softmax_with_cross_entropy_grad, + ops::SoftmaxWithCrossEntropyGradCUDAKernel, + ops::SoftmaxWithCrossEntropyGradCUDAKernel); +#else REGISTER_OP_CUDA_KERNEL( softmax_with_cross_entropy, ops::SoftmaxWithCrossEntropyCUDAKernel, ops::SoftmaxWithCrossEntropyCUDAKernel, @@ -545,3 +990,4 @@ REGISTER_OP_CUDA_KERNEL( ops::SoftmaxWithCrossEntropyGradCUDAKernel, ops::SoftmaxWithCrossEntropyGradCUDAKernel, ops::SoftmaxWithCrossEntropyGradCUDAKernel); +#endif diff --git a/paddle/fluid/operators/softmax_with_cross_entropy_op.h b/paddle/fluid/operators/softmax_with_cross_entropy_op.h index 35663bd9b77c2..55b811cbe31e4 100644 --- a/paddle/fluid/operators/softmax_with_cross_entropy_op.h +++ b/paddle/fluid/operators/softmax_with_cross_entropy_op.h @@ -31,6 +31,46 @@ class SoftmaxWithCrossEntropyKernel : public framework::OpKernel { PADDLE_ENFORCE_EQ( platform::is_cpu_place(context.GetPlace()), true, platform::errors::Unimplemented("This kernel only runs on CPU.")); + const bool softmax_switch = context.Attr("softmax_switch"); + + // do not with softmax op, and input is softmax + if (!softmax_switch) { + const Tensor* softmax = context.Input("Logits"); + const Tensor* labels = context.Input("Label"); + Tensor* softmax_out = context.Output("Softmax"); + Tensor* loss = context.Output("Loss"); + const bool soft_label = context.Attr("soft_label"); + const int rank = softmax->dims().size(); + const int axis = CanonicalAxis(context.Attr("axis"), rank); + int axis_dim = softmax->dims()[axis]; + + softmax_out->mutable_data(context.GetPlace()); + loss->mutable_data(context.GetPlace()); + + const int n = SizeToAxis(axis, softmax->dims()); + const int d = SizeFromAxis(axis, softmax->dims()); + + Tensor softmax_2d, labels_2d, loss_2d, softmax_out_2d; + softmax_2d.ShareDataWith(*softmax).Resize({n, d}); + labels_2d.ShareDataWith(*labels).Resize({n, labels->numel() / n}); + loss_2d.ShareDataWith(*loss).Resize({n, d / axis_dim}); + softmax_out_2d.ShareDataWith(*softmax_out).Resize({n, d}); + + auto& dev_ctx = + context.template device_context(); + + math::CrossEntropyFunctor()( + dev_ctx, &loss_2d, &softmax_2d, &labels_2d, soft_label, + context.Attr("ignore_index"), axis_dim); + + // cause of input is softmax + // copy to output softmax, directly + framework::TensorCopy(*softmax, context.GetPlace(), + context.device_context(), softmax_out); + + return; + } + const Tensor* logits = context.Input("Logits"); const Tensor* labels = context.Input("Label"); Tensor* softmax = context.Output("Softmax"); @@ -73,7 +113,9 @@ class SoftmaxWithCrossEntropyGradKernel : public framework::OpKernel { context.Output(framework::GradVarName("Logits")); const Tensor* softmax = context.Input("Softmax"); - if (logit_grad != softmax) { + const bool softmax_switch = context.Attr("softmax_switch"); + + if (logit_grad != softmax || !softmax_switch) { framework::TensorCopy(*softmax, context.GetPlace(), context.device_context(), logit_grad); } @@ -96,28 +138,94 @@ class SoftmaxWithCrossEntropyGradKernel : public framework::OpKernel { auto logit_grad_mat = framework::EigenMatrix::From(logit_grad_2d); auto& place = *context.template device_context() .eigen_device(); + if (!softmax_switch) { + // softmax_switch step1 + if (soft_label) { + auto lbl_mat = framework::EigenMatrix::From(labels_2d); + logit_grad_mat.device(place) = + (-lbl_mat / logit_grad_mat); // for each sample ,i is sample id + logit_grad_mat.device(place) = + out_grad_mat.broadcast(Eigen::DSizes(1, axis_dim)) * + logit_grad_mat; + } + // softmax_switch step2 + else { + const int64_t* label_data = labels->data(); + T* logit_grad_data = logit_grad->data(); + const T* out_grad_data = out_grad->data(); + const int remain = d / axis_dim; + for (int i = 0; i < n; ++i) { // for each sample_1_dim + for (int j = 0; j < remain; j++) { // for each sample_other_dims + int idx = i * remain + j; // this sample's label_idx. for 1d case, + // remain=1 and j=0, so, idx = i + if (label_data[idx] == ignore_index) { + for (int k = 0; k < axis_dim; ++k) { // for each class id's label + logit_grad_data[i * d + k * remain + j] = 0; + } + } else { + // only for this sample's label_idx, the label is 1, others is 0, + // so, only compute this label_idx's class + logit_grad_data[i * d + label_data[idx] * remain + j] = + (-1 / logit_grad_data[i * d + label_data[idx] * remain + j]) * + out_grad_data[idx]; + for (int k = 0; k < axis_dim; ++k) { // for each class id's label + if (k != + label_data[idx]) { // label_data[idx]: this sample's label + logit_grad_data[i * d + k * remain + j] = 0; + } + } + } + } + } + } + return; + } + + // for softmax_switch=False, continue + if (soft_label) { + // when soft_label = True, ignore_index is not supported auto lbl_mat = framework::EigenMatrix::From(labels_2d); logit_grad_mat.device(place) = out_grad_mat.broadcast(Eigen::DSizes(1, axis_dim)) * - (logit_grad_mat - lbl_mat); + (logit_grad_mat - lbl_mat); // for each sample ,i is sample id + // 1) compute dy/dx by p_j - y_j or P-Y, where j is class id, + // P=logit_grad_mat[i] is all class's probs, Y=lbl_mat[i] is + // all class's labels + // 2) compute dy * dy/dx by Chain rule, dy=out_grad_mat[i] + // for high dims, e.g. (n,c) or (n,d1,...,dm, c), compute grad by matrix + // operation + } else { logit_grad_mat.device(place) = - logit_grad_mat * + logit_grad_mat * // element_wise multiply out_grad_mat.broadcast(Eigen::DSizes(1, axis_dim)); const int64_t* label_data = labels->data(); T* logit_grad_data = logit_grad->data(); const T* out_grad_data = out_grad->data(); const int remain = d / axis_dim; - for (int i = 0; i < n; ++i) { - for (int j = 0; j < remain; j++) { - int idx = i * remain + j; + for (int i = 0; i < n; ++i) { // for each sample_1_dim + for (int j = 0; j < remain; j++) { // for each sample_other_dims + int idx = i * remain + j; // this sample's label_idx. for 1d case, + // remain=1 and j=0, so, idx = i if (label_data[idx] == ignore_index) { - for (int k = 0; k < axis_dim; ++k) { + for (int k = 0; k < axis_dim; ++k) { // for each class id's label logit_grad_data[i * d + k * remain + j] = 0; } } else { + // only for this sample's label_idx, the label is 1, others is 0, + // so, only compute this label_idx's class + // for 1d case, remain=1 and j=0, so, [i * d + label_data[idx] * + // remain + j] = [i * d + label_data[idx]] + // let idx_x = i * d + label_data[idx] * remain + j, + // logit_grad_data[idx_x] = logit_grad_data[idx_x] - + // out_grad_data[idx] + // note: logit_grad_mat = logit_grad_mat * out_grad_mat + // so: logit_grad_data[idx_x] = (logit_grad_data[idx_x] - 1) * + // out_grad_data[idx] + // means: dy/dp * dy= ( p - y ) * dy + logit_grad_data[i * d + label_data[idx] * remain + j] -= out_grad_data[idx]; } diff --git a/paddle/fluid/operators/split_lod_tensor_op.cc b/paddle/fluid/operators/split_lod_tensor_op.cc index 4adbbacc844c6..fe646b2830b66 100644 --- a/paddle/fluid/operators/split_lod_tensor_op.cc +++ b/paddle/fluid/operators/split_lod_tensor_op.cc @@ -65,7 +65,7 @@ class SplitLoDTensorOp : public framework::OperatorBase { if (platform::is_cpu_place(mask.place())) { cpu_mask->ShareDataWith(mask); } else if (platform::is_gpu_place(mask.place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) framework::TensorCopy(mask, platform::CPUPlace(), dev_ctx, cpu_mask.get()); #else diff --git a/paddle/fluid/operators/split_selected_rows_op.h b/paddle/fluid/operators/split_selected_rows_op.h index 8d88da24c632f..281f9fb7e596f 100644 --- a/paddle/fluid/operators/split_selected_rows_op.h +++ b/paddle/fluid/operators/split_selected_rows_op.h @@ -82,7 +82,7 @@ class SplitSelectedRowsOpKernel : public framework::OpKernel { platform::CPUPlace(), dst + j * row_numel, platform::CPUPlace(), src + outs_dense_idx[i][j] * row_numel, sizeof(T) * row_numel); } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto stream = ctx.cuda_device_context().stream(); memory::Copy(platform::CUDAPlace(), dst + j * row_numel, platform::CUDAPlace(), diff --git a/paddle/fluid/operators/spp_op.h b/paddle/fluid/operators/spp_op.h index 3c2d51ec9111e..6f78b88573404 100644 --- a/paddle/fluid/operators/spp_op.h +++ b/paddle/fluid/operators/spp_op.h @@ -56,14 +56,14 @@ class SppKernel : public framework::OpKernel { math::Pool2dFunctor, T> pool_forward; math::MaxPool max_process; pool_forward(context.template device_context(), *in_x, - kernel_size, strides, paddings, max_process, true, false, - &out_level); + kernel_size, strides, paddings, true, false, &out_level, + max_process); } else if (pooling_type == "avg") { math::Pool2dFunctor, T> pool_forward; math::AvgPool avg_process; pool_forward(context.template device_context(), *in_x, - kernel_size, strides, paddings, avg_process, true, false, - &out_level); + kernel_size, strides, paddings, true, false, &out_level, + avg_process); } // flatten pooling output shape int output_flatten_w = in_x->dims()[1] * bins * bins; @@ -156,7 +156,7 @@ class SppGradKernel : public framework::OpKernel { math::AvgPoolGrad avg_process; pool_backward(context.template device_context(), *in_x, *&out_level, *&outgrad_level, kernel_size, strides, - paddings, avg_process, true, false, in_x_grad); + paddings, true, false, in_x_grad, avg_process); } } } diff --git a/paddle/fluid/operators/strided_memcpy.h b/paddle/fluid/operators/strided_memcpy.h index 48d6cf8b3619a..eb15fe016d911 100644 --- a/paddle/fluid/operators/strided_memcpy.h +++ b/paddle/fluid/operators/strided_memcpy.h @@ -98,7 +98,7 @@ inline void StridedNumelCopyWithAxis(const platform::DeviceContext& ctx, memory::Copy(cpu_place, dst + i * dst_after, cpu_place, src + i * src_after, sizeof(T) * size); } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto& gpu_place = BOOST_GET_CONST(platform::CUDAPlace, place); auto& cuda_ctx = reinterpret_cast(ctx); diff --git a/paddle/fluid/operators/strided_memcpy_test.cc b/paddle/fluid/operators/strided_memcpy_test.cc index 83480b44d5be0..1ab036e869294 100644 --- a/paddle/fluid/operators/strided_memcpy_test.cc +++ b/paddle/fluid/operators/strided_memcpy_test.cc @@ -72,7 +72,7 @@ TEST(StridedMemcpy, CPUConcat) { } } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) TEST(StridedMemcpy, GPUCrop) { // clang-format off int src[] = { diff --git a/paddle/fluid/operators/sync_batch_norm_op.cu b/paddle/fluid/operators/sync_batch_norm_op.cu index 26fbe39a3c369..1c9e732b194ad 100644 --- a/paddle/fluid/operators/sync_batch_norm_op.cu +++ b/paddle/fluid/operators/sync_batch_norm_op.cu @@ -91,6 +91,16 @@ class SyncBatchNormGradKernel namespace ops = paddle::operators; namespace plat = paddle::platform; +#ifdef PADDLE_WITH_HIP +// MIOPEN do not support double +REGISTER_OP_CUDA_KERNEL( + sync_batch_norm, ops::SyncBatchNormKernel, + ops::SyncBatchNormKernel); +REGISTER_OP_CUDA_KERNEL( + sync_batch_norm_grad, + ops::SyncBatchNormGradKernel, + ops::SyncBatchNormGradKernel); +#else REGISTER_OP_CUDA_KERNEL( sync_batch_norm, ops::SyncBatchNormKernel, ops::SyncBatchNormKernel, @@ -100,5 +110,6 @@ REGISTER_OP_CUDA_KERNEL( ops::SyncBatchNormGradKernel, ops::SyncBatchNormGradKernel, ops::SyncBatchNormGradKernel); +#endif // clang-format on diff --git a/paddle/fluid/operators/sync_batch_norm_op.cu.h b/paddle/fluid/operators/sync_batch_norm_op.cu.h index d52eaecb94c12..d08a34ade77f2 100644 --- a/paddle/fluid/operators/sync_batch_norm_op.cu.h +++ b/paddle/fluid/operators/sync_batch_norm_op.cu.h @@ -19,12 +19,19 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#include "paddle/fluid/platform/cudnn_helper.h" +#endif +#ifdef __HIPCC__ +#include +namespace cub = hipcub; +#include "paddle/fluid/platform/miopen_helper.h" +#endif #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/memory/malloc.h" #include "paddle/fluid/operators/batch_norm_op.h" #include "paddle/fluid/operators/norm_utils.h" -#include "paddle/fluid/platform/cudnn_helper.h" #include "paddle/fluid/platform/float16.h" #include "paddle/fluid/platform/nccl_helper.h" @@ -186,7 +193,7 @@ void SyncBatchNormFunctor(const framework::ExecutionContext &ctx, auto gplace = BOOST_GET_CONST(platform::CUDAPlace, ctx.GetPlace()); memory::Copy(platform::CPUPlace(), c_g_st_d, gplace, stats, bytes, 0); -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) auto *comm = dev_ctx.nccl_comm(); if (comm) { int dtype = platform::ToNCCLDataType(mean_out->type()); @@ -460,7 +467,7 @@ void SyncBatchNormGradFunctor( dy_d, x_d, saved_mean, N, fsize, C, stats); } -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) auto *comm = dev_ctx.nccl_comm(); if (comm) { int dtype = platform::ToNCCLDataType(scale->type()); diff --git a/paddle/fluid/operators/temporal_shift_op.cc b/paddle/fluid/operators/temporal_shift_op.cc index 2e87447ed166e..acf99d09ffb90 100644 --- a/paddle/fluid/operators/temporal_shift_op.cc +++ b/paddle/fluid/operators/temporal_shift_op.cc @@ -80,7 +80,8 @@ class TemporalShiftOpMaker : public framework::OpProtoAndCheckerMaker { void Make() override { AddInput("X", "The input tensor of temporal shift operator. " - "This is a 4-D tensor with shape of [N*T, C, H, W]. " + "This is a 4-D tensor with shape of [N*T, C, H, W] " + "or [N*T, H, W, C]. " "While N is the batch size, T is the temporal segment " "number, C is the channel number, H is the height of " "features and W is the width of features. " @@ -100,15 +101,23 @@ class TemporalShiftOpMaker : public framework::OpProtoAndCheckerMaker { "by 1 along the temporal dimension. :attr:`shift_ratio` should be in " "range [0, 0.5]. Default 0.25.") .SetDefault(0.25); + AddAttr( + "data_format", + "(string, default NCHW) Only used in " + "an optional string from: \"NHWC\", \"NCHW\". " + "Specify that the data format of the input and output data is " + "channel_first or channel_last.") + .SetDefault("NCHW"); AddComment(R"DOC( This operator calculates the temporal shifting features for Input(X). - Input(X) should be in shape of [N*T, C, H, W], while N is the batch - size, T is the temporal segment number specified by :attr:`seg_num`, - C is the channel number, H and W is the height and width of features. + Input(X) should be in shape of [N*T, C, H, W] or [N*T, H, W, C], while + N is the batch size, T is the temporal segment number specified by + :attr:`seg_num`, C is the channel number, H and W is the height and + width of features. - Temporal Shifting is calculated as follows: + Temporal Shifting is calculated as follows when data format is NCHW: Step 1: Reshape Input(X) to [N, T, C, H, W]. diff --git a/paddle/fluid/operators/temporal_shift_op.cu b/paddle/fluid/operators/temporal_shift_op.cu index b61d9aeff7d4c..cb1ff5335cdf0 100644 --- a/paddle/fluid/operators/temporal_shift_op.cu +++ b/paddle/fluid/operators/temporal_shift_op.cu @@ -19,22 +19,46 @@ namespace operators { using framework::Tensor; template -__global__ void KeTemporalShiftFw(const T* input, T* output, const int ntchw, - const int tchw, const int chw, const int hw, - const int w, const int t, const int c, - const float shift_ratio) { +__global__ void KeTemporalShiftFwNCHW(const T* input, T* output, + const int ntchw, const int tchw, + const int chw, const int hw, const int t, + const int c1, const int c2) { int tid = blockIdx.x * blockDim.x + threadIdx.x; int stride = blockDim.x * gridDim.x; int src_it = 0; + for (; tid < ntchw; tid += stride) { - int in = tid / tchw; int it = (tid % tchw) / chw; int ic = (tid % chw) / hw; - int ih = (tid % hw) / w; - int iw = tid % w; - const int c1 = static_cast(c * shift_ratio); - const int c2 = static_cast(c * 2 * shift_ratio); + if (ic < c1) { + src_it = it - 1; + } else if (ic < c2) { + src_it = it + 1; + } else { + src_it = it; + } + + if (src_it < 0 || src_it >= t) { + output[tid] = 0; + } else { + output[tid] = input[tid + (src_it - it) * chw]; + } + } +} + +template +__global__ void KeTemporalShiftFwNHWC(const T* input, T* output, + const int nthwc, const int thwc, + const int hwc, const int t, const int c, + const int c1, const int c2) { + int tid = blockIdx.x * blockDim.x + threadIdx.x; + int stride = blockDim.x * gridDim.x; + int src_it = 0; + + for (; tid < nthwc; tid += stride) { + int it = (tid % thwc) / hwc; + int ic = tid % c; if (ic < c1) { src_it = it - 1; @@ -47,42 +71,65 @@ __global__ void KeTemporalShiftFw(const T* input, T* output, const int ntchw, if (src_it < 0 || src_it >= t) { output[tid] = 0; } else { - int src_idx = GetEntryIndex(in, src_it, ic, ih, iw, tchw, chw, hw, w); - output[tid] = input[src_idx]; + output[tid] = input[tid + (src_it - it) * hwc]; } } } template -__global__ void KeTemporalShiftBw(const T* output_grad, T* input_grad, - const int ntchw, const int tchw, - const int chw, const int hw, const int w, - const int t, const int c, - const float shift_ratio) { +__global__ void KeTemporalShiftBwNCHW(const T* output_grad, T* input_grad, + const int ntchw, const int tchw, + const int chw, const int hw, const int t, + const int c1, const int c2) { int tid = blockIdx.x * blockDim.x + threadIdx.x; int stride = blockDim.x * gridDim.x; int src_it = 0; + for (; tid < ntchw; tid += stride) { - int in = tid / tchw; int it = (tid % tchw) / chw; int ic = (tid % chw) / hw; - int ih = (tid % hw) / w; - int iw = tid % w; - - const int c1 = static_cast(c * shift_ratio); - const int c2 = static_cast(c * 2 * shift_ratio); if (ic < c1) { - src_it = it - 1; + src_it = it + 1; } else if (ic < c2) { + src_it = it - 1; + } else { + src_it = it; + } + + if (src_it >= 0 && src_it < t) { + input_grad[tid] = output_grad[tid + (src_it - it) * chw]; + } else { + input_grad[tid] = 0; + } + } +} + +template +__global__ void KeTemporalShiftBwNHWC(const T* output_grad, T* input_grad, + const int nthwc, const int thwc, + const int hwc, const int t, const int c, + const int c1, const int c2) { + int tid = blockIdx.x * blockDim.x + threadIdx.x; + int stride = blockDim.x * gridDim.x; + int src_it = 0; + + for (; tid < nthwc; tid += stride) { + int it = (tid % thwc) / hwc; + int ic = tid % c; + + if (ic < c1) { src_it = it + 1; + } else if (ic < c2) { + src_it = it - 1; } else { src_it = it; } if (src_it >= 0 && src_it < t) { - int src_idx = GetEntryIndex(in, src_it, ic, ih, iw, tchw, chw, hw, w); - input_grad[src_idx] = output_grad[tid]; + input_grad[tid] = output_grad[tid + (src_it - it) * hwc]; + } else { + input_grad[tid] = 0; } } } @@ -98,27 +145,48 @@ class TemporalShiftOpCUDAKernel : public framework::OpKernel { auto* output = ctx.Output("Out"); int t = ctx.Attr("seg_num"); float shift_ratio = ctx.Attr("shift_ratio"); + const std::string data_format_str = ctx.Attr("data_format"); + const DataLayout data_layout = + framework::StringToDataLayout(data_format_str); const int nt = input->dims()[0]; - const int c = input->dims()[1]; - const int h = input->dims()[2]; - const int w = input->dims()[3]; + const int c = (data_layout == DataLayout::kNCHW ? input->dims()[1] + : input->dims()[3]); + const int h = (data_layout == DataLayout::kNCHW ? input->dims()[2] + : input->dims()[1]); + const int w = (data_layout == DataLayout::kNCHW ? input->dims()[3] + : input->dims()[2]); const int hw = h * w; const int chw = c * hw; const int tchw = t * chw; const int ntchw = nt * chw; + const int c1 = static_cast(c * shift_ratio); + const int c2 = static_cast(c * 2 * shift_ratio); + + framework::DDim out_dims = (data_layout == DataLayout::kNCHW + ? framework::make_ddim({nt, c, h, w}) + : framework::make_ddim({nt, h, w, c})); const T* input_data = input->data(); - T* output_data = output->mutable_data({nt, c, h, w}, ctx.GetPlace()); + T* output_data = output->mutable_data(out_dims, ctx.GetPlace()); int pixelNum = nt * chw; - platform::GpuLaunchConfig config = - platform::GetGpuLaunchConfig1D(ctx.cuda_device_context(), pixelNum); + int threads = 1024; + int grid = (pixelNum + threads - 1) / threads; + const auto& dev_ctx = ctx.cuda_device_context(); + int blocks_per_sm = dev_ctx.GetMaxPhysicalThreadCount() / threads; + grid = std::min(dev_ctx.GetSMCount() * blocks_per_sm, grid); - KeTemporalShiftFw<<>>( - input_data, output_data, ntchw, tchw, chw, hw, w, t, c, shift_ratio); + if (data_layout == DataLayout::kNCHW) { + KeTemporalShiftFwNCHW< + T><<>>( + input_data, output_data, ntchw, tchw, chw, hw, t, c1, c2); + } else { + KeTemporalShiftFwNHWC< + T><<>>( + input_data, output_data, ntchw, tchw, chw, t, c, c1, c2); + } } }; @@ -130,32 +198,49 @@ class TemporalShiftGradOpCUDAKernel : public framework::OpKernel { auto* output_grad = ctx.Input(framework::GradVarName("Out")); int t = ctx.Attr("seg_num"); float shift_ratio = ctx.Attr("shift_ratio"); + const std::string data_format_str = ctx.Attr("data_format"); + const DataLayout data_layout = + framework::StringToDataLayout(data_format_str); const int nt = output_grad->dims()[0]; - const int c = output_grad->dims()[1]; - const int h = output_grad->dims()[2]; - const int w = output_grad->dims()[3]; + const int c = (data_layout == DataLayout::kNCHW ? output_grad->dims()[1] + : output_grad->dims()[3]); + const int h = (data_layout == DataLayout::kNCHW ? output_grad->dims()[2] + : output_grad->dims()[1]); + const int w = (data_layout == DataLayout::kNCHW ? output_grad->dims()[3] + : output_grad->dims()[2]); const int hw = h * w; const int chw = c * hw; const int tchw = t * chw; const int ntchw = nt * chw; + const int c1 = static_cast(c * shift_ratio); + const int c2 = static_cast(c * 2 * shift_ratio); + + framework::DDim in_grad_dims = (data_layout == DataLayout::kNCHW + ? framework::make_ddim({nt, c, h, w}) + : framework::make_ddim({nt, h, w, c})); const T* output_grad_data = output_grad->data(); T* input_grad_data = - input_grad->mutable_data({nt, c, h, w}, ctx.GetPlace()); - math::SetConstant()( - ctx.template device_context(), input_grad, - static_cast(0)); + input_grad->mutable_data(in_grad_dims, ctx.GetPlace()); int pixelNum = nt * chw; - platform::GpuLaunchConfig config = - platform::GetGpuLaunchConfig1D(ctx.cuda_device_context(), pixelNum); + int threads = 1024; + int grid = (pixelNum + threads - 1) / threads; + const auto& dev_ctx = ctx.cuda_device_context(); + int blocks_per_sm = dev_ctx.GetMaxPhysicalThreadCount() / threads; + grid = std::min(dev_ctx.GetSMCount() * blocks_per_sm, grid); - KeTemporalShiftBw<<>>( - output_grad_data, input_grad_data, ntchw, tchw, chw, hw, w, t, c, - shift_ratio); + if (data_layout == DataLayout::kNCHW) { + KeTemporalShiftBwNCHW< + T><<>>( + output_grad_data, input_grad_data, ntchw, tchw, chw, hw, t, c1, c2); + } else { + KeTemporalShiftBwNHWC< + T><<>>( + output_grad_data, input_grad_data, ntchw, tchw, chw, t, c, c1, c2); + } } }; @@ -163,8 +248,11 @@ class TemporalShiftGradOpCUDAKernel : public framework::OpKernel { } // namespace paddle namespace ops = paddle::operators; -REGISTER_OP_CUDA_KERNEL(temporal_shift, ops::TemporalShiftOpCUDAKernel, - ops::TemporalShiftOpCUDAKernel); -REGISTER_OP_CUDA_KERNEL(temporal_shift_grad, - ops::TemporalShiftGradOpCUDAKernel, - ops::TemporalShiftGradOpCUDAKernel); +REGISTER_OP_CUDA_KERNEL( + temporal_shift, ops::TemporalShiftOpCUDAKernel, + ops::TemporalShiftOpCUDAKernel, + ops::TemporalShiftOpCUDAKernel); +REGISTER_OP_CUDA_KERNEL( + temporal_shift_grad, ops::TemporalShiftGradOpCUDAKernel, + ops::TemporalShiftGradOpCUDAKernel, + ops::TemporalShiftGradOpCUDAKernel); diff --git a/paddle/fluid/operators/temporal_shift_op.h b/paddle/fluid/operators/temporal_shift_op.h index 4c7eed5af471a..05364b94c92c6 100644 --- a/paddle/fluid/operators/temporal_shift_op.h +++ b/paddle/fluid/operators/temporal_shift_op.h @@ -17,12 +17,106 @@ namespace paddle { namespace operators { using Tensor = framework::Tensor; +using DataLayout = framework::DataLayout; -static HOSTDEVICE inline int GetEntryIndex(int in, int it, int ic, int ih, - int iw, const int tchw, - const int chw, const int hw, - const int w) { - return in * tchw + it * chw + ic * hw + ih * w + iw; +template +void TemporalShiftFwNCHW(const T* input, T* output, const int ntchw, + const int tchw, const int chw, const int hw, + const int t, const int c1, const int c2) { + int src_it = 0; + for (int i = 0; i < ntchw; i++) { + int it = (i % tchw) / chw; + int ic = (i % chw) / hw; + + if (ic < c1) { + src_it = it - 1; + } else if (ic < c2) { + src_it = it + 1; + } else { + src_it = it; + } + + if (src_it < 0 || src_it >= t) { + output[i] = 0; + } else { + output[i] = input[i + (src_it - it) * chw]; + } + } +} + +template +void TemporalShiftFwNHWC(const T* input, T* output, const int nthwc, + const int thwc, const int hwc, const int t, + const int c, const int c1, const int c2) { + int src_it = 0; + for (int i = 0; i < nthwc; i++) { + int it = (i % thwc) / hwc; + int ic = i % c; + + if (ic < c1) { + src_it = it - 1; + } else if (ic < c2) { + src_it = it + 1; + } else { + src_it = it; + } + + if (src_it < 0 || src_it >= t) { + output[i] = 0; + } else { + output[i] = input[i + (src_it - it) * hwc]; + } + } +} + +template +void TemporalShiftBwNCHW(const T* output_grad, T* input_grad, const int ntchw, + const int tchw, const int chw, const int hw, + const int t, const int c1, const int c2) { + int src_it = 0; + for (int i = 0; i < ntchw; i++) { + int it = (i % tchw) / chw; + int ic = (i % chw) / hw; + + if (ic < c1) { + src_it = it + 1; + } else if (ic < c2) { + src_it = it - 1; + } else { + src_it = it; + } + + if (src_it >= 0 && src_it < t) { + input_grad[i] = output_grad[i + (src_it - it) * chw]; + } else { + input_grad[i] = 0; + } + } +} + +template +void TemporalShiftBwNHWC(const T* output_grad, T* input_grad, const int nthwc, + const int thwc, const int hwc, const int t, + const int c, const int c1, const int c2) { + int src_it = 0; + for (int i = 0; i < nthwc; i++) { + int it = (i % thwc) / hwc; + int ic = i % c; + + if (ic < c1) { + src_it = it + 1; + } else if (ic < c2) { + src_it = it - 1; + } else { + src_it = it; + } + + if (src_it >= 0 && src_it < t) { + input_grad[i] = output_grad[i + (src_it - it) * hwc]; + } else { + input_grad[i] = 0; + } + } } template @@ -33,44 +127,38 @@ class TemporalShiftKernel : public framework::OpKernel { auto* output = ctx.Output("Out"); int t = ctx.Attr("seg_num"); float shift_ratio = ctx.Attr("shift_ratio"); + const std::string data_format_str = ctx.Attr("data_format"); + const DataLayout data_layout = + framework::StringToDataLayout(data_format_str); const int nt = input->dims()[0]; - const int c = input->dims()[1]; - const int h = input->dims()[2]; - const int w = input->dims()[3]; - - const int c1 = static_cast(c * shift_ratio); - const int c2 = static_cast(c * 2 * shift_ratio); + const int c = (data_layout == DataLayout::kNCHW ? input->dims()[1] + : input->dims()[3]); + const int h = (data_layout == DataLayout::kNCHW ? input->dims()[2] + : input->dims()[1]); + const int w = (data_layout == DataLayout::kNCHW ? input->dims()[3] + : input->dims()[2]); const int hw = h * w; const int chw = c * hw; const int tchw = t * chw; + const int ntchw = nt * chw; + const int c1 = static_cast(c * shift_ratio); + const int c2 = static_cast(c * 2 * shift_ratio); + + framework::DDim out_dims = (data_layout == DataLayout::kNCHW + ? framework::make_ddim({nt, c, h, w}) + : framework::make_ddim({nt, h, w, c})); const T* input_data = input->data(); - T* output_data = output->mutable_data({nt, c, h, w}, ctx.GetPlace()); - - int src_it = 0; - for (int i = 0; i < output->numel(); i++) { - int in = i / tchw; - int it = (i % tchw) / chw; - int ic = (i % chw) / hw; - int ih = (i % hw) / w; - int iw = i % w; - - if (ic < c1) { - src_it = it - 1; - } else if (ic < c2) { - src_it = it + 1; - } else { - src_it = it; - } - - if (src_it < 0 || src_it >= t) { - output_data[i] = 0; - } else { - int src_idx = GetEntryIndex(in, src_it, ic, ih, iw, tchw, chw, hw, w); - output_data[i] = input_data[src_idx]; - } + T* output_data = output->mutable_data(out_dims, ctx.GetPlace()); + + if (data_layout == DataLayout::kNCHW) { + TemporalShiftFwNCHW(input_data, output_data, ntchw, tchw, chw, hw, t, + c1, c2); + } else { + TemporalShiftFwNHWC(input_data, output_data, ntchw, tchw, chw, t, c, + c1, c2); } } }; @@ -83,44 +171,39 @@ class TemporalShiftGradKernel : public framework::OpKernel { auto* output_grad = ctx.Input(framework::GradVarName("Out")); int t = ctx.Attr("seg_num"); float shift_ratio = ctx.Attr("shift_ratio"); + const std::string data_format_str = ctx.Attr("data_format"); + const DataLayout data_layout = + framework::StringToDataLayout(data_format_str); const int nt = output_grad->dims()[0]; - const int c = output_grad->dims()[1]; - const int h = output_grad->dims()[2]; - const int w = output_grad->dims()[3]; - - const int c1 = static_cast(c * shift_ratio); - const int c2 = static_cast(c * 2 * shift_ratio); + const int c = (data_layout == DataLayout::kNCHW ? output_grad->dims()[1] + : output_grad->dims()[3]); + const int h = (data_layout == DataLayout::kNCHW ? output_grad->dims()[2] + : output_grad->dims()[1]); + const int w = (data_layout == DataLayout::kNCHW ? output_grad->dims()[3] + : output_grad->dims()[2]); const int hw = h * w; const int chw = c * hw; const int tchw = t * chw; + const int ntchw = nt * chw; + + const int c1 = static_cast(c * shift_ratio); + const int c2 = static_cast(c * 2 * shift_ratio); + framework::DDim in_grad_dims = (data_layout == DataLayout::kNCHW + ? framework::make_ddim({nt, c, h, w}) + : framework::make_ddim({nt, h, w, c})); const T* output_grad_data = output_grad->data(); T* input_grad_data = - input_grad->mutable_data({nt, c, h, w}, ctx.GetPlace()); - memset(input_grad_data, 0, input_grad->numel() * sizeof(T)); - - int src_it = 0; - for (int i = 0; i < output_grad->numel(); i++) { - int in = i / tchw; - int it = (i % tchw) / chw; - int ic = (i % chw) / hw; - int ih = (i % hw) / w; - int iw = i % w; - - if (ic < c1) { - src_it = it - 1; - } else if (ic < c2) { - src_it = it + 1; - } else { - src_it = it; - } - - if (src_it >= 0 && src_it < t) { - int src_idx = GetEntryIndex(in, src_it, ic, ih, iw, tchw, chw, hw, w); - input_grad_data[src_idx] = output_grad_data[i]; - } + input_grad->mutable_data(in_grad_dims, ctx.GetPlace()); + + if (data_layout == DataLayout::kNCHW) { + TemporalShiftBwNCHW(output_grad_data, input_grad_data, ntchw, tchw, + chw, hw, t, c1, c2); + } else { + TemporalShiftBwNHWC(output_grad_data, input_grad_data, ntchw, tchw, + chw, t, c, c1, c2); } } }; diff --git a/paddle/fluid/operators/tensorrt/tensorrt_engine_op.h b/paddle/fluid/operators/tensorrt/tensorrt_engine_op.h index b8805c025a768..1f0ae40798e4d 100644 --- a/paddle/fluid/operators/tensorrt/tensorrt_engine_op.h +++ b/paddle/fluid/operators/tensorrt/tensorrt_engine_op.h @@ -89,6 +89,7 @@ class TensorRTEngineOp : public framework::OperatorBase { bool use_calib_mode_; std::string calibration_data_; std::string engine_key_; + std::string calibration_engine_key_; bool calibration_mode_; int predictor_id_; int device_id_; @@ -109,6 +110,7 @@ class TensorRTEngineOp : public framework::OperatorBase { use_calib_mode_ = Attr("use_calib_mode"); calibration_data_ = Attr("calibration_data"); engine_key_ = Attr("engine_key"); + calibration_engine_key_ = Attr("calibration_engine_key"); predictor_id_ = Attr("predictor_id"); auto params = Attr>("parameters"); @@ -172,9 +174,11 @@ class TensorRTEngineOp : public framework::OperatorBase { "Paddle TRT int8..."; int runtime_batch = 1; - if (!Singleton::Global().Has(engine_key_)) { + if (!Singleton::Global().Has( + calibration_engine_key_)) { TRTCalibratorEngine *calib_res = - Singleton::Global().Create(engine_key_); + Singleton::Global().Create( + calibration_engine_key_); std::unordered_map calib_buffers; for (auto &x : input_names_) { if (param_names_.count(x)) continue; @@ -185,7 +189,7 @@ class TensorRTEngineOp : public framework::OperatorBase { runtime_batch = t_shape[0]; } calib_res->calib_.reset(new TRTInt8Calibrator( - calib_buffers, runtime_batch, engine_key_, dev_place)); + calib_buffers, runtime_batch, calibration_engine_key_, dev_place)); calib_res->thr_.reset(new std::thread([&]() { calib_res->engine_.reset(new TensorRTEngine( max_batch_size_, workspace_size_, precision_mode_, @@ -198,7 +202,7 @@ class TensorRTEngineOp : public framework::OperatorBase { TRTInt8Calibrator *temp_calibrator = Singleton::Global() - .Get(engine_key_) + .Get(calibration_engine_key_) ->calib_.get(); std::unordered_map calib_data; diff --git a/paddle/fluid/operators/tensorrt/tensorrt_engine_op_test.cc b/paddle/fluid/operators/tensorrt/tensorrt_engine_op_test.cc index 1dcaccd6e9264..4e88d79dfe4d2 100644 --- a/paddle/fluid/operators/tensorrt/tensorrt_engine_op_test.cc +++ b/paddle/fluid/operators/tensorrt/tensorrt_engine_op_test.cc @@ -102,6 +102,8 @@ TEST(TensorRTEngineOp, manual) { engine_op_desc.SetAttr("workspace_size", static_cast(1 << 20)); engine_op_desc.SetAttr("parameters", std::vector({})); engine_op_desc.SetAttr("engine_key", std::string("a_engine")); + engine_op_desc.SetAttr("calibration_engine_key", + std::string("a_calib_engine")); engine_op_desc.SetAttr("predictor_id", 1); engine_op_desc.SetAttr("calibration_data", std::string("")); engine_op_desc.SetAttr("enable_int8", static_cast(false)); @@ -204,6 +206,8 @@ void Execute(int batch_size, int input_dim, int output_dim, int nlayers = 1) { engine_op_desc.SetAttr("parameters", std::vector({"y0", "y1", "y2", "y3"})); engine_op_desc.SetAttr("engine_key", std::string("b_engine")); + engine_op_desc.SetAttr("calibration_engine_key", + std::string("b_calib_engine")); engine_op_desc.SetAttr("predictor_id", 1); engine_op_desc.SetAttr("calibration_data", std::string("")); engine_op_desc.SetAttr("enable_int8", static_cast(false)); diff --git a/paddle/fluid/operators/test_leaky_relu_grad_grad_functor.h b/paddle/fluid/operators/test_leaky_relu_grad_grad_functor.h index ce94ba1ce9e8c..2d7fed2987f4b 100644 --- a/paddle/fluid/operators/test_leaky_relu_grad_grad_functor.h +++ b/paddle/fluid/operators/test_leaky_relu_grad_grad_functor.h @@ -91,7 +91,7 @@ static bool TestLeakyReluGradGradMain(const framework::DDim &dim, int64_t limit = x.numel(); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) if (platform::is_gpu_place(place)) { auto &cuda_dev_ctx = dynamic_cast(dev_ctx); functor(cuda_dev_ctx, &x, out, &ddx, &ddout, dout, dx); @@ -105,7 +105,7 @@ static bool TestLeakyReluGradGradMain(const framework::DDim &dim, platform::ForRange for_range(cpu_dev_ctx, limit); for_range(actual_functor); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) } #endif diff --git a/paddle/fluid/operators/top_k_function_cuda.h b/paddle/fluid/operators/top_k_function_cuda.h index 0fd5f2ac01df3..a7d7ea260ecdf 100644 --- a/paddle/fluid/operators/top_k_function_cuda.h +++ b/paddle/fluid/operators/top_k_function_cuda.h @@ -16,11 +16,26 @@ limitations under the License. */ #include #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +#endif #include "paddle/fluid/operators/top_k_op.h" #include "paddle/fluid/platform/cuda_device_function.h" #include "paddle/fluid/platform/float16.h" +#ifdef __HIPCC__ +namespace rocprim { +namespace detail { +template <> +struct radix_key_codec_base + : radix_key_codec_integral {}; +} // namespace detail +} // namespace rocprim +namespace cub = hipcub; +#else // set cub base traits in order to handle float16 namespace cub { template <> @@ -28,6 +43,7 @@ struct NumericTraits : BaseTraits {}; } // namespace cub +#endif namespace paddle { namespace operators { @@ -439,6 +455,16 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, input_indices.data(), sorted_indices_ptr, num_cols * num_rows, num_rows, segment_offsets_t, segment_offsets_t + 1, 0, sizeof(T) * 8, cu_stream); +#ifdef __HIPCC__ + if (err != hipSuccess) { + LOG(ERROR) << "TopKOP failed as could not launch " + "hipcub::DeviceSegmentedRadixSort::SortPairsDescending to " + "calculate " + "temp_storage_bytes, status: " + << hipGetErrorString(err); + return false; + } +#else if (err != cudaSuccess) { LOG(ERROR) << "TopKOP failed as could not launch " @@ -447,12 +473,22 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, << cudaGetErrorString(err); return false; } +#endif } else { auto err = cub::DeviceSegmentedRadixSort::SortPairs( nullptr, temp_storage_bytes, input, sorted_values_ptr, input_indices.data(), sorted_indices_ptr, num_cols * num_rows, num_rows, segment_offsets_t, segment_offsets_t + 1, 0, sizeof(T) * 8, cu_stream); +#ifdef __HIPCC__ + if (err != hipSuccess) { + LOG(ERROR) << "TopKOP failed as could not launch " + "hipcub::DeviceSegmentedRadixSort::SortPairs to calculate " + "temp_storage_bytes, status: " + << hipGetErrorString(err); + return false; + } +#else if (err != cudaSuccess) { LOG(ERROR) << "TopKOP failed as could not launch " "cub::DeviceSegmentedRadixSort::SortPairs to calculate " @@ -460,6 +496,7 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, << cudaGetErrorString(err); return false; } +#endif } Tensor temp_storage; temp_storage.mutable_data(ctx.GetPlace(), temp_storage_bytes); @@ -470,6 +507,17 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, sorted_values_ptr, input_indices.data(), sorted_indices_ptr, num_cols * num_rows, num_rows, segment_offsets_t, segment_offsets_t + 1, 0, sizeof(T) * 8, cu_stream); +#ifdef __HIPCC__ + if (err != hipSuccess) { + LOG(ERROR) << "TopKOP failed as could not launch " + "hipcub::DeviceSegmentedRadixSort::SortPairsDescending to " + "sort input, " + "temp_storage_bytes: " + << temp_storage_bytes + << ", status: " << hipGetErrorString(err); + return false; + } +#else if (err != cudaSuccess) { LOG(ERROR) << "TopKOP failed as could not launch " "cub::DeviceSegmentedRadixSort::SortPairsDescending to " @@ -479,12 +527,24 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, << ", status: " << cudaGetErrorString(err); return false; } +#endif } else { auto err = cub::DeviceSegmentedRadixSort::SortPairs( temp_storage.data(), temp_storage_bytes, input, sorted_values_ptr, input_indices.data(), sorted_indices_ptr, num_cols * num_rows, num_rows, segment_offsets_t, segment_offsets_t + 1, 0, sizeof(T) * 8, cu_stream); +#ifdef __HIPCC__ + if (err != hipSuccess) { + LOG(ERROR) << "TopKOP failed as could not launch " + "hipcub::DeviceSegmentedRadixSort::SortPairs to " + "sort input, " + "temp_storage_bytes: " + << temp_storage_bytes + << ", status: " << hipGetErrorString(err); + return false; + } +#else if (err != cudaSuccess) { LOG(ERROR) << "TopKOP failed as could not launch " "cub::DeviceSegmentedRadixSort::SortPairs to " @@ -494,19 +554,21 @@ bool SortTopk(const platform::CUDADeviceContext& ctx, << ", status: " << cudaGetErrorString(err); return false; } +#endif } auto& dev = *ctx.eigen_device(); if (k < num_cols) { // copy sliced data to output. const Eigen::DSizes slice_indices{0, 0}; const Eigen::DSizes slice_sizes{num_rows, k}; - auto e_indices = EigenMatrix::From(*indices_tensor, dim); - auto e_tmp_indices = EigenMatrix::From(temp_indices); + auto e_indices = + framework::EigenMatrix::From(*indices_tensor, dim); + auto e_tmp_indices = framework::EigenMatrix::From(temp_indices); std::vector odims = {static_cast(num_rows), static_cast(k)}; auto dim = framework::make_ddim(odims); - auto e_values = EigenMatrix::From(*out_tensor, dim); - auto e_tmp_values = EigenMatrix::From(temp_values); + auto e_values = framework::EigenMatrix::From(*out_tensor, dim); + auto e_tmp_values = framework::EigenMatrix::From(temp_values); e_indices.device(dev) = e_tmp_indices.slice(slice_indices, slice_sizes); e_values.device(dev) = e_tmp_values.slice(slice_indices, slice_sizes); diff --git a/paddle/fluid/operators/top_k_op.cu b/paddle/fluid/operators/top_k_op.cu index 39a56f874d950..498f51d53adc7 100644 --- a/paddle/fluid/operators/top_k_op.cu +++ b/paddle/fluid/operators/top_k_op.cu @@ -15,7 +15,12 @@ limitations under the License. */ #pragma once #include #include +#ifdef __NVCC__ #include "cub/cub.cuh" +#endif +#ifdef __HIPCC__ +#include +#endif #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/operators/top_k_function_cuda.h" #include "paddle/fluid/operators/top_k_op.h" diff --git a/paddle/fluid/operators/top_k_op.h b/paddle/fluid/operators/top_k_op.h index 1ba01d93acc3e..f279b9529cc09 100644 --- a/paddle/fluid/operators/top_k_op.h +++ b/paddle/fluid/operators/top_k_op.h @@ -25,14 +25,6 @@ namespace operators { using Tensor = framework::Tensor; -template -using EigenMatrix = framework::EigenMatrix; - -template -using EigenVector = framework::EigenVector; - template class TopkKernel : public framework::OpKernel { public: @@ -70,12 +62,13 @@ class TopkKernel : public framework::OpKernel { vec.reserve(col); // 1D vector if (inputdims.size() == 1) { - auto eg_input = EigenVector::Flatten(*input); + auto eg_input = framework::EigenVector::Flatten(*input); for (size_t j = 0; j < col; j++) { vec.push_back(std::pair(eg_input(j), j)); } } else { - auto eg_input = EigenMatrix::Reshape(*input, inputdims.size() - 1); + auto eg_input = + framework::EigenMatrix::Reshape(*input, inputdims.size() - 1); for (size_t j = 0; j < col; j++) { vec.push_back(std::pair(eg_input(i, j), j)); } diff --git a/paddle/fluid/operators/top_k_v2_op.cu b/paddle/fluid/operators/top_k_v2_op.cu index a2c97aee92a1a..0f2da4b8f6fbb 100644 --- a/paddle/fluid/operators/top_k_v2_op.cu +++ b/paddle/fluid/operators/top_k_v2_op.cu @@ -150,7 +150,8 @@ class TopkV2OpCUDAKernel : public framework::OpKernel { if (k > input_width) k = input_width; - if ((input_width <= 1024 || k >= 128 || k == input_width)) { + if (((input_width <= 1024 && input_height <= 2048) || k >= 128 || + k == input_width)) { if (SortTopk(dev_ctx, &trans_input, input_width, input_height, k, &trans_out, &trans_ind, largest)) { // last step, tranpose back the indices and output diff --git a/paddle/fluid/operators/top_k_v2_op.h b/paddle/fluid/operators/top_k_v2_op.h index 89b5d36b1b3f9..c836c993b2910 100644 --- a/paddle/fluid/operators/top_k_v2_op.h +++ b/paddle/fluid/operators/top_k_v2_op.h @@ -61,12 +61,12 @@ static void FullTopK(Type input_height, Type input_width, int input_dim, std::vector> col_vec; col_vec.reserve(input_width); if (input_dim == 1) { - auto e_input = EigenVector::Flatten(*input); + auto e_input = framework::EigenVector::Flatten(*input); for (Type j = 0; j < input_width; ++j) { col_vec.emplace_back(std::pair(e_input(j), j)); } } else { - auto e_input = EigenMatrix::Reshape(*input, input_dim - 1); + auto e_input = framework::EigenMatrix::Reshape(*input, input_dim - 1); for (Type j = 0; j < input_width; ++j) { col_vec.emplace_back(std::pair(e_input(i, j), j)); } @@ -142,14 +142,15 @@ static void FullTopKAssign(const Type& input_height, const Type& input_width, #endif for (Type i = 0; i < input_height; ++i) { if (input_dim == 1) { - auto e_input = EigenVector::Flatten(*input); - auto e_indices = EigenVector::Flatten(*indices); + auto e_input = framework::EigenVector::Flatten(*input); + auto e_indices = framework::EigenVector::Flatten(*indices); for (Type j = 0; j < k; ++j) { output_data[i * input_width + e_indices(j)] = e_input(j); } } else { - auto e_input = EigenMatrix::Reshape(*input, input_dim - 1); - auto e_indices = EigenMatrix::Reshape(*indices, input_dim - 1); + auto e_input = framework::EigenMatrix::Reshape(*input, input_dim - 1); + auto e_indices = + framework::EigenMatrix::Reshape(*indices, input_dim - 1); for (Type j = 0; j < k; ++j) { output_data[i * input_width + e_indices(i, j)] = e_input(i, j); } diff --git a/paddle/fluid/operators/trace_op.h b/paddle/fluid/operators/trace_op.h index 54c4251a38cf1..b7a6e559ed4ef 100644 --- a/paddle/fluid/operators/trace_op.h +++ b/paddle/fluid/operators/trace_op.h @@ -145,7 +145,7 @@ framework::Tensor Diagonal(const framework::ExecutionContext& context, int64_t pos = std::abs(offset) * offset_stride; int64_t dim_size = ret_strides.size(); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector diag_vec(vectorize(dig_stride)); const int64_t* diag_arr = thrust::raw_pointer_cast(diag_vec.data()); thrust::device_vector ret_vec(ret_strides); @@ -238,7 +238,7 @@ class TraceGradKernel : public framework::OpKernel { int64_t diag_size = len2 < len1 ? len2 : len1; int64_t pos = std::abs(offset) * offset_stride; if (diag_size > 0) { -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector output_vec(vectorize(output_stride)); const int64_t* output_arr = thrust::raw_pointer_cast(output_vec.data()); thrust::device_vector input_vec(vectorize(input_stride)); diff --git a/paddle/fluid/operators/unique_op.cu b/paddle/fluid/operators/unique_op.cu index d22406f27c470..87a46e11d9f91 100644 --- a/paddle/fluid/operators/unique_op.cu +++ b/paddle/fluid/operators/unique_op.cu @@ -16,6 +16,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include diff --git a/paddle/fluid/operators/unity_build_rule.cmake b/paddle/fluid/operators/unity_build_rule.cmake index d21f6b2d69d84..cd8b31d72e72a 100644 --- a/paddle/fluid/operators/unity_build_rule.cmake +++ b/paddle/fluid/operators/unity_build_rule.cmake @@ -5,6 +5,7 @@ # If there are some redefined error in compiling with the source file which # in combination rule, you can remove the source file from the following rules. register_unity_group(cc + abs_op.cc add_position_encoding_op.cc addmm_op.cc affine_channel_op.cc @@ -33,7 +34,11 @@ register_unity_group(cc chunk_eval_op.cc clip_by_norm_op.cc clip_op.cc - coalesce_tensor_op.cc) + coalesce_tensor_op.cc + mkldnn/activation_mkldnn_op.cc + mkldnn/interpolate_mkldnn_op.cc + mkldnn/pool_mkldnn_op.cc + mkldnn/softmax_mkldnn_op.cc) register_unity_group(cc center_loss_op.cc mkldnn/concat_mkldnn_op.cc @@ -42,7 +47,12 @@ register_unity_group(cc correlation_op.cc cos_sim_op.cc crf_decoding_op.cc - crop_op.cc) + crop_op.cc + ascend_trigger_op.cc + conj_op.cc + imag_op.cc + kldiv_loss_op.cc + memcpy_op.cc) register_unity_group(cc cross_entropy_op.cc cross_op.cc @@ -69,7 +79,14 @@ register_unity_group(cc edit_distance_op.cc empty_op.cc enqueue_op.cc - erf_op.cc) + erf_op.cc + py_func_op.cc + real_op.cc + sync_batch_norm_op.cc + top_k_op.cc + conv_op.cc + conv_transpose_op.cc + gru_unit_op.cc) register_unity_group(cc expand_v2_op.cc fake_dequantize_op.cc @@ -309,6 +326,29 @@ register_unity_group(cc unbind_op.cu.cc unpool_op.cu.cc unsqueeze_op.cu.cc) +register_unity_group(cc + arg_max_op.cc + arg_min_op.cc + squared_l2_distance_op.cc) +register_unity_group(cc + linear_chain_crf_op.cc + lstm_op.cc + partial_concat_op.cc + pyramid_hash_op.cc + recurrent_op.cc + run_program_op.cc + softmax_with_cross_entropy_op.cc + warpctc_op.cc) +register_unity_group(cc + conv_op.cu.cc + lstm_op.cu.cc + rnn_op.cu.cc + split_op.cu.cc + activation_cudnn_op.cu.cc + assign_value_op.cu.cc + merge_selected_rows_op.cu.cc + run_program_op.cu.cc + warpctc_op.cu.cc) register_unity_group(cu addmm_op.cu affine_channel_op.cu diff --git a/paddle/fluid/operators/unstack_op.h b/paddle/fluid/operators/unstack_op.h index 6344ea16f81cd..82118b692707f 100644 --- a/paddle/fluid/operators/unstack_op.h +++ b/paddle/fluid/operators/unstack_op.h @@ -18,7 +18,7 @@ limitations under the License. */ #include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/platform/for_range.h" -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) #include #include "paddle/fluid/framework/array.h" #endif @@ -103,7 +103,7 @@ class UnStackGradKernel : public framework::OpKernel { for (auto i = 0; i < axis; ++i) pre *= dim[i]; for (auto i = axis; i < dim.size(); ++i) post *= dim[i]; -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) int total_num = pre * n * post; auto &dev_ctx = ctx.template device_context(); @@ -156,14 +156,14 @@ class UnStackKernel : public framework::OpKernel { int post = total_num / (n * pre); auto &dev_ctx = ctx.template device_context(); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) thrust::device_vector device_dx_vec(dx_datas); auto dx_data_arr = device_dx_vec.data().get(); #else auto dx_data_arr = dx_datas.data(); #endif StackGradFunctorForRange(dev_ctx, dx_data_arr, dy_data, total_num, n, post); -#ifdef __NVCC__ +#if defined(__NVCC__) || defined(__HIPCC__) // Wait() must be called because device_dx_vec may be destructed before // kernel ends dev_ctx.Wait(); diff --git a/paddle/fluid/operators/warpctc_op.cc b/paddle/fluid/operators/warpctc_op.cc index f043b01794919..f38f5d9f72357 100644 --- a/paddle/fluid/operators/warpctc_op.cc +++ b/paddle/fluid/operators/warpctc_op.cc @@ -16,6 +16,9 @@ limitations under the License. */ #include +#ifdef PADDLE_WITH_HIP +#include "paddle/fluid/platform/miopen_helper.h" +#endif #ifdef PADDLE_WITH_CUDA #include "paddle/fluid/platform/cudnn_helper.h" #endif diff --git a/paddle/fluid/operators/warpctc_op.h b/paddle/fluid/operators/warpctc_op.h index 8b9276d4fa03f..7451cac63d0ce 100644 --- a/paddle/fluid/operators/warpctc_op.h +++ b/paddle/fluid/operators/warpctc_op.h @@ -159,6 +159,7 @@ class WarpCTCFunctor { warpctc_version_ = platform::dynload::get_warpctc_version(); if (platform::is_gpu_place(ctx.GetPlace())) { +// HIP not support ctcOptions in third-party warpctc #ifdef PADDLE_WITH_CUDA options_.loc = CTC_GPU; options_.stream = reinterpret_cast( diff --git a/paddle/fluid/platform/bfloat16.h b/paddle/fluid/platform/bfloat16.h index f373e5ddb6d8c..6cb4901f1dde3 100644 --- a/paddle/fluid/platform/bfloat16.h +++ b/paddle/fluid/platform/bfloat16.h @@ -15,22 +15,27 @@ #pragma once #include + +#include +#include +#include #include + #if !defined(_WIN32) #define PADDLE_ALIGN(x) __attribute__((aligned(x))) #else #define PADDLE_ALIGN(x) __declspec(align(x)) #endif -#include - -#include "paddle/fluid/platform/hostdevice.h" -#include "unsupported/Eigen/CXX11/Tensor" - -namespace Eigen { -template -struct NumTraits; -} // namespace Eigen +#if (defined(__CUDACC__) || defined(__HIPCC__)) +#define HOSTDEVICE __host__ __device__ +#define DEVICE __device__ +#define HOST __host__ +#else +#define HOSTDEVICE +#define DEVICE +#define HOST +#endif namespace paddle { namespace platform { @@ -351,105 +356,3 @@ struct numeric_limits { }; } // namespace std - -namespace Eigen { - -using bfloat16 = paddle::platform::bfloat16; - -template <> -struct NumTraits : GenericNumTraits { - enum { - IsSigned = true, - IsInteger = false, - IsComplex = false, - RequireInitialization = false - }; - - HOSTDEVICE static inline bfloat16 epsilon() { - return paddle::platform::raw_uint16_to_bfloat16(0x3400); - } - HOSTDEVICE static inline bfloat16 dummy_precision() { - return bfloat16(1e-5f); - } - HOSTDEVICE static inline bfloat16 highest() { - return paddle::platform::raw_uint16_to_bfloat16(0x7f7f); - } - HOSTDEVICE static inline bfloat16 lowest() { - return paddle::platform::raw_uint16_to_bfloat16(0xff7f); - } - HOSTDEVICE static inline bfloat16 infinity() { - return paddle::platform::raw_uint16_to_bfloat16(0x7f80); - } - HOSTDEVICE static inline bfloat16 quiet_NaN() { - return paddle::platform::raw_uint16_to_bfloat16(0xffc1); - } -}; -namespace numext { - -template <> -HOSTDEVICE inline bool(isnan)(const bfloat16& a) { - return (paddle::platform::isnan)(a); -} - -template <> -HOSTDEVICE inline bool(isinf)(const bfloat16& a) { - return (paddle::platform::isinf)(a); -} - -template <> -HOSTDEVICE inline bool(isfinite)(const bfloat16& a) { - return (paddle::platform::isfinite)(a); -} - -template <> -HOSTDEVICE inline bfloat16 exp(const bfloat16& a) { - return bfloat16(::expf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 erf(const bfloat16& a) { - return bfloat16(::erff(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 log(const bfloat16& a) { - return bfloat16(::logf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 tanh(const bfloat16& a) { - return bfloat16(::tanhf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 sqrt(const bfloat16& a) { - return bfloat16(::sqrtf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 ceil(const bfloat16& a) { - return bfloat16(::ceilf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 floor(const bfloat16& a) { - return bfloat16(::floorf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 round(const bfloat16& a) { - return bfloat16(::roundf(static_cast(a))); -} - -template <> -HOSTDEVICE inline bfloat16 pow(const bfloat16& a, const bfloat16& b) { - return bfloat16(::powf(static_cast(a), static_cast(b))); -} - -template <> -HOSTDEVICE inline bfloat16 abs(const bfloat16& a) { - return bfloat16(::fabs(static_cast(a))); -} - -} // namespace numext -} // namespace Eigen diff --git a/paddle/fluid/platform/bfloat16_test.cc b/paddle/fluid/platform/bfloat16_test.cc index 3adfcd89be917..dc2d3aa73ba60 100644 --- a/paddle/fluid/platform/bfloat16_test.cc +++ b/paddle/fluid/platform/bfloat16_test.cc @@ -10,6 +10,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/platform/bfloat16.h" +#include "paddle/fluid/platform/eigen_ext.h" #define GLOG_NO_ABBREVIATED_SEVERITIES // msvc conflict logging with windows.h #include "gtest/gtest.h" diff --git a/paddle/fluid/platform/collective_helper.cc b/paddle/fluid/platform/collective_helper.cc index 4b16a67b235fd..f2b478f7d20e9 100644 --- a/paddle/fluid/platform/collective_helper.cc +++ b/paddle/fluid/platform/collective_helper.cc @@ -15,6 +15,8 @@ #include "paddle/fluid/platform/collective_helper.h" #include +#include "paddle/fluid/platform/cuda_resource_pool.h" + namespace paddle { namespace platform { #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) @@ -43,12 +45,31 @@ class NCCLCommImpl : public NCCLComm { } CUDADeviceContext* dev_context() const override { return dev_ctx_.get(); } + gpuEvent_t compute_event() const override { return compute_event_.get(); } + + gpuEvent_t comm_event() const override { return comm_event_.get(); } + + void set_compute_event( + std::shared_ptr&& compute_event) { + compute_event_ = std::move(compute_event); + } + + void set_comm_event(std::shared_ptr&& comm_event) { + comm_event_ = std::move(comm_event); + } + private: int ring_id_; int nranks_; int rank_; ncclComm_t comm_; std::unique_ptr dev_ctx_; + + // used for comm wait compute, compute_stream-->event-->comm_stream + std::shared_ptr compute_event_; + + // used for compute wait comm, comm_stream-->event-->compute_stream + std::shared_ptr comm_event_; }; NCCLComm* NCCLCommContext::CreateNCCLComm(ncclUniqueId* nccl_id, int nranks, @@ -124,12 +145,19 @@ NCCLComm* NCCLCommContext::AssignNCCLComm(ncclComm_t comm, int nranks, int rank, std::unique_ptr dev_ctx( new CUDADeviceContext(CUDAPlace(dev_id))); + std::shared_ptr compute_event( + platform::CudaEventResourcePool::Instance().New(dev_id)); + std::shared_ptr comm_event( + platform::CudaEventResourcePool::Instance().New(dev_id)); + NCCLCommImpl* c = new NCCLCommImpl; c->set_ring_id(ring_id); c->set_nranks(nranks); c->set_rank(rank); c->set_comm(comm); c->set_dev_ctx(std::move(dev_ctx)); + c->set_compute_event(std::move(compute_event)); + c->set_comm_event(std::move(comm_event)); comm_map_mutex_.lock(); if (comm_map_.count(ring_id) == 0) { diff --git a/paddle/fluid/platform/collective_helper.h b/paddle/fluid/platform/collective_helper.h index 8a6719ab685b8..197f905ba68a2 100644 --- a/paddle/fluid/platform/collective_helper.h +++ b/paddle/fluid/platform/collective_helper.h @@ -57,6 +57,8 @@ class NCCLComm { virtual int device_id() const = 0; virtual ncclComm_t comm() const = 0; virtual gpuStream_t stream() const = 0; + virtual gpuEvent_t compute_event() const = 0; + virtual gpuEvent_t comm_event() const = 0; virtual CUDADeviceContext* dev_context() const = 0; virtual ~NCCLComm() = default; }; diff --git a/paddle/fluid/platform/complex128.h b/paddle/fluid/platform/complex128.h index c50ff2f810393..d6fddd672a0f8 100644 --- a/paddle/fluid/platform/complex128.h +++ b/paddle/fluid/platform/complex128.h @@ -16,12 +16,10 @@ #include +#include +#include +#include #include -#if !defined(_WIN32) -#define PADDLE_ALIGN(x) __attribute__((aligned(x))) -#else -#define PADDLE_ALIGN(x) __declspec(align(x)) -#endif #ifdef PADDLE_WITH_CUDA #include @@ -33,15 +31,21 @@ #include // NOLINT #endif -#include - -#include "paddle/fluid/platform/hostdevice.h" -#include "unsupported/Eigen/CXX11/Tensor" +#if !defined(_WIN32) +#define PADDLE_ALIGN(x) __attribute__((aligned(x))) +#else +#define PADDLE_ALIGN(x) __declspec(align(x)) +#endif -namespace Eigen { -template -struct NumTraits; -} // namespace Eigen +#if (defined(__CUDACC__) || defined(__HIPCC__)) +#define HOSTDEVICE __host__ __device__ +#define DEVICE __device__ +#define HOST __host__ +#else +#define HOSTDEVICE +#define DEVICE +#define HOST +#endif namespace paddle { namespace platform { @@ -509,97 +513,5 @@ struct numeric_limits { }; } // namespace std -namespace Eigen { - -using complex128 = paddle::platform::complex128; - -template <> -struct NumTraits : GenericNumTraits> { - typedef double Real; - typedef typename NumTraits::Literal Literal; - enum { - IsComplex = 1, - RequireInitialization = NumTraits::RequireInitialization, - ReadCost = 2 * NumTraits::ReadCost, - AddCost = 2 * NumTraits::AddCost, - MulCost = 4 * NumTraits::MulCost + 2 * NumTraits::AddCost - }; - - EIGEN_DEVICE_FUNC - static inline Real epsilon() { return NumTraits::epsilon(); } - EIGEN_DEVICE_FUNC - static inline Real dummy_precision() { - return NumTraits::dummy_precision(); - } - EIGEN_DEVICE_FUNC - static inline int digits10() { return NumTraits::digits10(); } -}; -namespace numext { - -template <> -HOSTDEVICE inline bool(isnan)(const complex128& a) { - return (paddle::platform::isnan)(a); -} - -template <> -HOSTDEVICE inline bool(isinf)(const complex128& a) { - return (paddle::platform::isinf)(a); -} - -template <> -HOSTDEVICE inline bool(isfinite)(const complex128& a) { - return (paddle::platform::isfinite)(a); -} - -template <> -HOSTDEVICE inline complex128 exp(const complex128& a) { - double com = ::expf(a.real); - double res_real = com * ::cosf(a.imag); - double res_imag = com * ::sinf(a.imag); - return complex128(res_real, res_imag); -} - -template <> -HOSTDEVICE inline complex128 log(const complex128& a) { - return paddle::platform::log(a); -} - -template <> -HOSTDEVICE inline complex128 tanh(const complex128& a) { - return paddle::platform::tanh(a); -} - -template <> -HOSTDEVICE inline complex128 sqrt(const complex128& a) { - return paddle::platform::sqrt(a); -} - -template <> -HOSTDEVICE inline complex128 ceil(const complex128& a) { - return complex128(::ceilf(a.real), ::ceilf(a.imag)); -} - -template <> -HOSTDEVICE inline complex128 floor(const complex128& a) { - return complex128(::floorf(a.real), ::floor(a.imag)); -} - -template <> -HOSTDEVICE inline complex128 round(const complex128& a) { - return complex128(::roundf(a.real), ::roundf(a.imag)); -} - -template <> -HOSTDEVICE inline complex128 pow(const complex128& a, const complex128& b) { - return paddle::platform::pow(a, b); -} - -template <> -HOSTDEVICE inline double abs(const complex128& a) { - return paddle::platform::abs(a); -} - -} // namespace numext -} // namespace Eigen #define MKL_Complex16 paddle::platform::complex128 diff --git a/paddle/fluid/platform/complex64.h b/paddle/fluid/platform/complex64.h index b91fdbab28b0b..9d55ba19105a6 100644 --- a/paddle/fluid/platform/complex64.h +++ b/paddle/fluid/platform/complex64.h @@ -15,12 +15,11 @@ #pragma once #include + +#include +#include +#include #include -#if !defined(_WIN32) -#define PADDLE_ALIGN(x) __attribute__((aligned(x))) -#else -#define PADDLE_ALIGN(x) __declspec(align(x)) -#endif #ifdef PADDLE_WITH_CUDA #include @@ -32,16 +31,23 @@ #include // NOLINT #endif -#include +#if !defined(_WIN32) +#define PADDLE_ALIGN(x) __attribute__((aligned(x))) +#else +#define PADDLE_ALIGN(x) __declspec(align(x)) +#endif -#include "paddle/fluid/platform/complex128.h" -#include "paddle/fluid/platform/hostdevice.h" -#include "unsupported/Eigen/CXX11/Tensor" +#if (defined(__CUDACC__) || defined(__HIPCC__)) +#define HOSTDEVICE __host__ __device__ +#define DEVICE __device__ +#define HOST __host__ +#else +#define HOSTDEVICE +#define DEVICE +#define HOST +#endif -namespace Eigen { -template -struct NumTraits; -} // namespace Eigen +#include "complex128.h" // NOLINT namespace paddle { namespace platform { @@ -510,98 +516,5 @@ struct numeric_limits { }; } // namespace std -namespace Eigen { - -using complex64 = paddle::platform::complex64; - -template <> -struct NumTraits : GenericNumTraits> { - typedef float Real; - typedef typename NumTraits::Literal Literal; - enum { - IsComplex = 1, - RequireInitialization = NumTraits::RequireInitialization, - ReadCost = 2 * NumTraits::ReadCost, - AddCost = 2 * NumTraits::AddCost, - MulCost = 4 * NumTraits::MulCost + 2 * NumTraits::AddCost - }; - - EIGEN_DEVICE_FUNC - static inline Real epsilon() { return NumTraits::epsilon(); } - EIGEN_DEVICE_FUNC - static inline Real dummy_precision() { - return NumTraits::dummy_precision(); - } - EIGEN_DEVICE_FUNC - static inline int digits10() { return NumTraits::digits10(); } -}; - -namespace numext { - -template <> -HOSTDEVICE inline bool(isnan)(const complex64& a) { - return (paddle::platform::isnan)(a); -} - -template <> -HOSTDEVICE inline bool(isinf)(const complex64& a) { - return (paddle::platform::isinf)(a); -} - -template <> -HOSTDEVICE inline bool(isfinite)(const complex64& a) { - return (paddle::platform::isfinite)(a); -} - -template <> -HOSTDEVICE inline complex64 exp(const complex64& a) { - float com = ::expf(a.real); - float res_real = com * ::cosf(a.imag); - float res_imag = com * ::sinf(a.imag); - return complex64(res_real, res_imag); -} - -template <> -HOSTDEVICE inline complex64 log(const complex64& a) { - return paddle::platform::log(a); -} - -template <> -HOSTDEVICE inline complex64 tanh(const complex64& a) { - return paddle::platform::tanh(a); -} - -template <> -HOSTDEVICE inline complex64 sqrt(const complex64& a) { - return paddle::platform::sqrt(a); -} - -template <> -HOSTDEVICE inline complex64 ceil(const complex64& a) { - return complex64(::ceilf(a.real), ::ceilf(a.imag)); -} - -template <> -HOSTDEVICE inline complex64 floor(const complex64& a) { - return complex64(::floorf(a.real), ::floor(a.imag)); -} - -template <> -HOSTDEVICE inline complex64 round(const complex64& a) { - return complex64(::roundf(a.real), ::roundf(a.imag)); -} - -template <> -HOSTDEVICE inline complex64 pow(const complex64& a, const complex64& b) { - return paddle::platform::pow(a, b); -} - -template <> -HOSTDEVICE inline float abs(const complex64& a) { - return paddle::platform::abs(a); -} - -} // namespace numext -} // namespace Eigen #define MKL_Complex8 paddle::platform::complex64 diff --git a/paddle/fluid/platform/cuda_helper.h b/paddle/fluid/platform/cuda_helper.h index bfefeb2f4a3da..fa4ef3f8c124e 100644 --- a/paddle/fluid/platform/cuda_helper.h +++ b/paddle/fluid/platform/cuda_helper.h @@ -76,10 +76,17 @@ namespace platform { * */ +#ifdef __HIPCC__ +#define CUDA_KERNEL_LOOP_TYPE(i, num, index_type) \ + int64_t __index__ = hipBlockIdx_x * hipBlockDim_x + hipThreadIdx_x; \ + for (index_type i = __index__; __index__ < (num); \ + __index__ += hipBlockDim_x * hipGridDim_x, i = __index__) +#else #define CUDA_KERNEL_LOOP_TYPE(i, num, index_type) \ int64_t __index__ = blockIdx.x * blockDim.x + threadIdx.x; \ for (index_type i = __index__; __index__ < (num); \ __index__ += blockDim.x * gridDim.x, i = __index__) +#endif #define CUDA_KERNEL_LOOP(i, num) CUDA_KERNEL_LOOP_TYPE(i, num, int) @@ -108,7 +115,11 @@ class CublasHandleHolder { } #endif +#ifdef PADDLE_WITH_HIP + const rocblas_handle& GetCublasHandle() const { return handle_; } +#else const cublasHandle_t& GetCublasHandle() const { return handle_; } +#endif ~CublasHandleHolder() PADDLE_MAY_THROW { #ifdef PADDLE_WITH_HIP diff --git a/paddle/fluid/platform/device_context.cc b/paddle/fluid/platform/device_context.cc index 98dcf72aa4fb4..22daaf101cf20 100644 --- a/paddle/fluid/platform/device_context.cc +++ b/paddle/fluid/platform/device_context.cc @@ -459,9 +459,15 @@ cudnnHandle_t CUDADeviceContext::cudnn_handle() const { return context()->CudnnHandle(); } +#ifdef PADDLE_WITH_HIP +rocblas_handle CUDADeviceContext::cublas_handle() const { + return context()->CublasHandle()->GetCublasHandle(); +} +#else cublasHandle_t CUDADeviceContext::cublas_handle() const { return context()->CublasHandle()->GetCublasHandle(); } +#endif CudnnWorkspaceHandle CUDADeviceContext::cudnn_workspace_handle() const { return CudnnWorkspaceHandle(*this, &cudnn_handle_mtx_); diff --git a/paddle/fluid/platform/device_context.h b/paddle/fluid/platform/device_context.h index 11123c4e658ed..411fe09c864aa 100644 --- a/paddle/fluid/platform/device_context.h +++ b/paddle/fluid/platform/device_context.h @@ -409,8 +409,12 @@ class CUDADeviceContext : public DeviceContext { cudnnHandle_t cudnn_handle() const; #endif - /*! \brief Return cublas handle in the device context. */ +/*! \brief Return cublas handle in the device context. */ +#ifdef PADDLE_WITH_HIP + rocblas_handle cublas_handle() const; +#else cublasHandle_t cublas_handle() const; +#endif /*! \brief Return a cudnn workspace handle to call multiple cudnn * functions without interrupting by other threads. diff --git a/paddle/fluid/platform/device_context_test.cu b/paddle/fluid/platform/device_context_test.cu index 3e9fe461d746c..2f9413c4f3ea7 100644 --- a/paddle/fluid/platform/device_context_test.cu +++ b/paddle/fluid/platform/device_context_test.cu @@ -47,7 +47,11 @@ TEST(Device, CUDADeviceContext) { cudnnHandle_t cudnn_handle = device_context->cudnn_handle(); #endif ASSERT_NE(nullptr, cudnn_handle); +#ifdef PADDLE_WITH_HIP + rocblas_handle cublas_handle = device_context->cublas_handle(); +#else cublasHandle_t cublas_handle = device_context->cublas_handle(); +#endif ASSERT_NE(nullptr, cublas_handle); delete device_context; } diff --git a/paddle/fluid/platform/device_memory_aligment.cc b/paddle/fluid/platform/device_memory_aligment.cc index b287d11a9fe62..f8e031104415e 100644 --- a/paddle/fluid/platform/device_memory_aligment.cc +++ b/paddle/fluid/platform/device_memory_aligment.cc @@ -23,6 +23,9 @@ size_t Alignment(size_t size, const platform::Place &place) { } else { #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) alignment = GpuMinChunkSize(); +#elif defined(PADDLE_WITH_XPU) + // TODO(wangxi): add XpuMinChunkSize + alignment = alignment; #else PADDLE_THROW(platform::errors::PreconditionNotMet( "Fluid is not compiled with CUDA.")); diff --git a/paddle/fluid/platform/dynload/miopen.h b/paddle/fluid/platform/dynload/miopen.h index 43a3e1a1079d9..15de4c64e3e64 100644 --- a/paddle/fluid/platform/dynload/miopen.h +++ b/paddle/fluid/platform/dynload/miopen.h @@ -125,6 +125,7 @@ extern void EnforceCUDNNLoaded(const char* fn_name); __macro(miopenCreateRNNDescriptor); \ __macro(miopenDestroyRNNDescriptor); \ __macro(miopenSetRNNDescriptor); \ + __macro(miopenSetRNNDescriptor_V2); \ __macro(miopenGetRNNParamsSize); \ __macro(miopenGetRNNWorkspaceSize); \ __macro(miopenGetRNNTrainingReserveSize); \ diff --git a/paddle/fluid/platform/eigen_ext.h b/paddle/fluid/platform/eigen_ext.h new file mode 100644 index 0000000000000..a8ad729a31a4d --- /dev/null +++ b/paddle/fluid/platform/eigen_ext.h @@ -0,0 +1,402 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include "paddle/fluid/platform/bfloat16.h" +#include "paddle/fluid/platform/complex128.h" +#include "paddle/fluid/platform/complex64.h" +#include "paddle/fluid/platform/float16.h" +#include "paddle/fluid/platform/hostdevice.h" + +#include "unsupported/Eigen/CXX11/Tensor" + +namespace Eigen { + +using bfloat16 = paddle::platform::bfloat16; +using complex64 = paddle::platform::complex64; +using complex128 = paddle::platform::complex128; +using float16 = paddle::platform::float16; + +template +struct NumTraits; + +template <> +struct NumTraits : GenericNumTraits { + enum { + IsSigned = true, + IsInteger = false, + IsComplex = false, + RequireInitialization = false + }; + + HOSTDEVICE static inline bfloat16 epsilon() { + return paddle::platform::raw_uint16_to_bfloat16(0x3400); + } + HOSTDEVICE static inline bfloat16 dummy_precision() { + return bfloat16(1e-5f); + } + HOSTDEVICE static inline bfloat16 highest() { + return paddle::platform::raw_uint16_to_bfloat16(0x7f7f); + } + HOSTDEVICE static inline bfloat16 lowest() { + return paddle::platform::raw_uint16_to_bfloat16(0xff7f); + } + HOSTDEVICE static inline bfloat16 infinity() { + return paddle::platform::raw_uint16_to_bfloat16(0x7f80); + } + HOSTDEVICE static inline bfloat16 quiet_NaN() { + return paddle::platform::raw_uint16_to_bfloat16(0xffc1); + } +}; + +template <> +struct NumTraits : GenericNumTraits> { + typedef float Real; + typedef typename NumTraits::Literal Literal; + enum { + IsComplex = 1, + RequireInitialization = NumTraits::RequireInitialization, + ReadCost = 2 * NumTraits::ReadCost, + AddCost = 2 * NumTraits::AddCost, + MulCost = 4 * NumTraits::MulCost + 2 * NumTraits::AddCost + }; + + EIGEN_DEVICE_FUNC + static inline Real epsilon() { return NumTraits::epsilon(); } + EIGEN_DEVICE_FUNC + static inline Real dummy_precision() { + return NumTraits::dummy_precision(); + } + EIGEN_DEVICE_FUNC + static inline int digits10() { return NumTraits::digits10(); } +}; + +template <> +struct NumTraits : GenericNumTraits> { + typedef double Real; + typedef typename NumTraits::Literal Literal; + enum { + IsComplex = 1, + RequireInitialization = NumTraits::RequireInitialization, + ReadCost = 2 * NumTraits::ReadCost, + AddCost = 2 * NumTraits::AddCost, + MulCost = 4 * NumTraits::MulCost + 2 * NumTraits::AddCost + }; + + EIGEN_DEVICE_FUNC + static inline Real epsilon() { return NumTraits::epsilon(); } + EIGEN_DEVICE_FUNC + static inline Real dummy_precision() { + return NumTraits::dummy_precision(); + } + EIGEN_DEVICE_FUNC + static inline int digits10() { return NumTraits::digits10(); } +}; + +template <> +struct NumTraits : GenericNumTraits { + enum { + IsSigned = true, + IsInteger = false, + IsComplex = false, + RequireInitialization = false + }; + + HOSTDEVICE static inline float16 epsilon() { + return paddle::platform::raw_uint16_to_float16(0x0800); + } + HOSTDEVICE static inline float16 dummy_precision() { return float16(1e-2f); } + HOSTDEVICE static inline float16 highest() { + return paddle::platform::raw_uint16_to_float16(0x7bff); + } + HOSTDEVICE static inline float16 lowest() { + return paddle::platform::raw_uint16_to_float16(0xfbff); + } + HOSTDEVICE static inline float16 infinity() { + return paddle::platform::raw_uint16_to_float16(0x7c00); + } + HOSTDEVICE static inline float16 quiet_NaN() { + return paddle::platform::raw_uint16_to_float16(0x7c01); + } +}; + +namespace numext { + +//////////// bfloat methods ///////////// + +template <> +HOSTDEVICE inline bool(isnan)(const bfloat16& a) { + return (paddle::platform::isnan)(a); +} + +template <> +HOSTDEVICE inline bool(isinf)(const bfloat16& a) { + return (paddle::platform::isinf)(a); +} + +template <> +HOSTDEVICE inline bool(isfinite)(const bfloat16& a) { + return (paddle::platform::isfinite)(a); +} + +template <> +HOSTDEVICE inline bfloat16 exp(const bfloat16& a) { + return bfloat16(::expf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 erf(const bfloat16& a) { + return bfloat16(::erff(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 log(const bfloat16& a) { + return bfloat16(::logf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 tanh(const bfloat16& a) { + return bfloat16(::tanhf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 sqrt(const bfloat16& a) { + return bfloat16(::sqrtf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 ceil(const bfloat16& a) { + return bfloat16(::ceilf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 floor(const bfloat16& a) { + return bfloat16(::floorf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 round(const bfloat16& a) { + return bfloat16(::roundf(static_cast(a))); +} + +template <> +HOSTDEVICE inline bfloat16 pow(const bfloat16& a, const bfloat16& b) { + return bfloat16(::powf(static_cast(a), static_cast(b))); +} + +template <> +HOSTDEVICE inline bfloat16 abs(const bfloat16& a) { + return bfloat16(::fabs(static_cast(a))); +} + +//////////// complex64 methods ///////////// + +template <> +HOSTDEVICE inline bool(isnan)(const complex64& a) { + return (paddle::platform::isnan)(a); +} + +template <> +HOSTDEVICE inline bool(isinf)(const complex64& a) { + return (paddle::platform::isinf)(a); +} + +template <> +HOSTDEVICE inline bool(isfinite)(const complex64& a) { + return (paddle::platform::isfinite)(a); +} + +template <> +HOSTDEVICE inline complex64 exp(const complex64& a) { + float com = ::expf(a.real); + float res_real = com * ::cosf(a.imag); + float res_imag = com * ::sinf(a.imag); + return complex64(res_real, res_imag); +} + +template <> +HOSTDEVICE inline complex64 log(const complex64& a) { + return paddle::platform::log(a); +} + +template <> +HOSTDEVICE inline complex64 tanh(const complex64& a) { + return paddle::platform::tanh(a); +} + +template <> +HOSTDEVICE inline complex64 sqrt(const complex64& a) { + return paddle::platform::sqrt(a); +} + +template <> +HOSTDEVICE inline complex64 ceil(const complex64& a) { + return complex64(::ceilf(a.real), ::ceilf(a.imag)); +} + +template <> +HOSTDEVICE inline complex64 floor(const complex64& a) { + return complex64(::floorf(a.real), ::floor(a.imag)); +} + +template <> +HOSTDEVICE inline complex64 round(const complex64& a) { + return complex64(::roundf(a.real), ::roundf(a.imag)); +} + +template <> +HOSTDEVICE inline complex64 pow(const complex64& a, const complex64& b) { + return paddle::platform::pow(a, b); +} + +template <> +HOSTDEVICE inline float abs(const complex64& a) { + return paddle::platform::abs(a); +} + +//////////// complex128 methods ///////////// + +template <> +HOSTDEVICE inline bool(isnan)(const complex128& a) { + return (paddle::platform::isnan)(a); +} + +template <> +HOSTDEVICE inline bool(isinf)(const complex128& a) { + return (paddle::platform::isinf)(a); +} + +template <> +HOSTDEVICE inline bool(isfinite)(const complex128& a) { + return (paddle::platform::isfinite)(a); +} + +template <> +HOSTDEVICE inline complex128 exp(const complex128& a) { + double com = ::expf(a.real); + double res_real = com * ::cosf(a.imag); + double res_imag = com * ::sinf(a.imag); + return complex128(res_real, res_imag); +} + +template <> +HOSTDEVICE inline complex128 log(const complex128& a) { + return paddle::platform::log(a); +} + +template <> +HOSTDEVICE inline complex128 tanh(const complex128& a) { + return paddle::platform::tanh(a); +} + +template <> +HOSTDEVICE inline complex128 sqrt(const complex128& a) { + return paddle::platform::sqrt(a); +} + +template <> +HOSTDEVICE inline complex128 ceil(const complex128& a) { + return complex128(::ceilf(a.real), ::ceilf(a.imag)); +} + +template <> +HOSTDEVICE inline complex128 floor(const complex128& a) { + return complex128(::floorf(a.real), ::floor(a.imag)); +} + +template <> +HOSTDEVICE inline complex128 round(const complex128& a) { + return complex128(::roundf(a.real), ::roundf(a.imag)); +} + +template <> +HOSTDEVICE inline complex128 pow(const complex128& a, const complex128& b) { + return paddle::platform::pow(a, b); +} + +template <> +HOSTDEVICE inline double abs(const complex128& a) { + return paddle::platform::abs(a); +} + +//////////// float16 methods ///////////// + +template <> +HOSTDEVICE inline bool(isnan)(const float16& a) { + return (paddle::platform::isnan)(a); +} + +template <> +HOSTDEVICE inline bool(isinf)(const float16& a) { + return (paddle::platform::isinf)(a); +} + +template <> +HOSTDEVICE inline bool(isfinite)(const float16& a) { + return (paddle::platform::isfinite)(a); +} + +template <> +HOSTDEVICE inline float16 exp(const float16& a) { + return float16(::expf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 erf(const float16& a) { + return float16(::erff(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 log(const float16& a) { + return float16(::logf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 tanh(const float16& a) { + return float16(::tanhf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 sqrt(const float16& a) { + return float16(::sqrtf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 ceil(const float16& a) { + return float16(::ceilf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 floor(const float16& a) { + return float16(::floorf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 round(const float16& a) { + return float16(::roundf(static_cast(a))); +} + +template <> +HOSTDEVICE inline float16 pow(const float16& a, const float16& b) { + return float16(::powf(static_cast(a), static_cast(b))); +} + +template <> +HOSTDEVICE inline float16 abs(const float16& a) { + return float16(::fabs(static_cast(a))); +} + +} // namespace numext +} // namespace Eigen diff --git a/paddle/fluid/platform/float16.h b/paddle/fluid/platform/float16.h index df2a24400b438..bdd4d54b3d1a1 100644 --- a/paddle/fluid/platform/float16.h +++ b/paddle/fluid/platform/float16.h @@ -15,6 +15,9 @@ limitations under the License. */ #pragma once #include + +#include +#include #include #ifdef PADDLE_WITH_CUDA @@ -25,18 +28,6 @@ limitations under the License. */ #include #endif -#ifdef __GNUC__ -#define PADDLE_GNUC_VER (__GNUC__ * 10 + __GNUC_MINOR__) -#else -#define PADDLE_GNUC_VER 0 -#endif // __GNUC__ - -#ifdef __clang__ -#define PADDLE_CLANG_VER (__clang_major__ * 10 + __clang_minor__) -#else -#define PADDLE_CLANG_VER 0 -#endif // __clang__ - #if defined(__CUDACC__) && CUDA_VERSION >= 7050 #define PADDLE_CUDA_FP16 #include @@ -55,17 +46,15 @@ limitations under the License. */ #define CUDA_ARCH_FP16_SUPPORTED(CUDA_ARCH) (CUDA_ARCH >= 600) -namespace paddle { -namespace platform { - -// Forward declare float16 for eigen.h -struct float16; - -} // namespace platform -} // namespace paddle - -#include "paddle/fluid/platform/hostdevice.h" -#include "unsupported/Eigen/CXX11/Tensor" +#if (defined(__CUDACC__) || defined(__HIPCC__)) +#define HOSTDEVICE __host__ __device__ +#define DEVICE __device__ +#define HOST __host__ +#else +#define HOSTDEVICE +#define DEVICE +#define HOST +#endif namespace paddle { namespace platform { @@ -73,7 +62,7 @@ namespace platform { // Use PADDLE_ALIGNED(2) to ensure that each float16 will be allocated // and aligned at least on a 2-byte boundary, which leads to efficient // memory access of float16 struct and also makes float16 compatible -// with CUDA half, ARM float16_t, and Eigen::half data types. +// with CUDA half, ARM float16_t data types. struct PADDLE_ALIGN(2) float16 { public: uint16_t x; @@ -100,8 +89,6 @@ struct PADDLE_ALIGN(2) float16 { } #endif // PADDLE_CUDA_FP16 - HOSTDEVICE inline explicit float16(const Eigen::half& h) : x(h.x) {} - #ifdef PADDLE_WITH_NATIVE_FP16 // __fp16 is a native half precision data type for arm cpu, // float16_t is an alias for __fp16 @@ -163,11 +150,6 @@ struct PADDLE_ALIGN(2) float16 { } #endif - HOSTDEVICE inline float16& operator=(const Eigen::half& rhs) { - x = rhs.x; - return *this; - } - #ifdef PADDLE_WITH_NATIVE_FP16 HOSTDEVICE inline float16& operator=(const float16_t& rhs) { x = *reinterpret_cast(&rhs); @@ -245,12 +227,6 @@ struct PADDLE_ALIGN(2) float16 { } #endif // PADDLE_CUDA_FP16 - HOSTDEVICE inline explicit operator Eigen::half() const { - Eigen::half h; - h.x = x; - return h; - } - #ifdef PADDLE_WITH_NATIVE_FP16 HOSTDEVICE inline explicit operator float16_t() const { return *reinterpret_cast(this); @@ -1108,105 +1084,3 @@ HOSTDEVICE inline paddle::platform::float16 abs( } } // namespace std - -namespace Eigen { - -using float16 = paddle::platform::float16; - -template <> -struct NumTraits : GenericNumTraits { - enum { - IsSigned = true, - IsInteger = false, - IsComplex = false, - RequireInitialization = false - }; - - HOSTDEVICE static inline float16 epsilon() { - return paddle::platform::raw_uint16_to_float16(0x0800); - } - HOSTDEVICE static inline float16 dummy_precision() { return float16(1e-2f); } - HOSTDEVICE static inline float16 highest() { - return paddle::platform::raw_uint16_to_float16(0x7bff); - } - HOSTDEVICE static inline float16 lowest() { - return paddle::platform::raw_uint16_to_float16(0xfbff); - } - HOSTDEVICE static inline float16 infinity() { - return paddle::platform::raw_uint16_to_float16(0x7c00); - } - HOSTDEVICE static inline float16 quiet_NaN() { - return paddle::platform::raw_uint16_to_float16(0x7c01); - } -}; - -namespace numext { - -template <> -HOSTDEVICE inline bool(isnan)(const float16& a) { - return (paddle::platform::isnan)(a); -} - -template <> -HOSTDEVICE inline bool(isinf)(const float16& a) { - return (paddle::platform::isinf)(a); -} - -template <> -HOSTDEVICE inline bool(isfinite)(const float16& a) { - return (paddle::platform::isfinite)(a); -} - -template <> -HOSTDEVICE inline float16 exp(const float16& a) { - return float16(::expf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 erf(const float16& a) { - return float16(::erff(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 log(const float16& a) { - return float16(::logf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 tanh(const float16& a) { - return float16(::tanhf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 sqrt(const float16& a) { - return float16(::sqrtf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 ceil(const float16& a) { - return float16(::ceilf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 floor(const float16& a) { - return float16(::floorf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 round(const float16& a) { - return float16(::roundf(static_cast(a))); -} - -template <> -HOSTDEVICE inline float16 pow(const float16& a, const float16& b) { - return float16(::powf(static_cast(a), static_cast(b))); -} - -template <> -HOSTDEVICE inline float16 abs(const float16& a) { - return float16(::fabs(static_cast(a))); -} - -} // namespace numext - -} // namespace Eigen diff --git a/paddle/fluid/platform/float16_test.cc b/paddle/fluid/platform/float16_test.cc index f607988d92024..56633a3511671 100644 --- a/paddle/fluid/platform/float16_test.cc +++ b/paddle/fluid/platform/float16_test.cc @@ -8,26 +8,19 @@ 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 "paddle/fluid/platform/float16.h" #define GLOG_NO_ABBREVIATED_SEVERITIES // msvc conflict logging with windows.h #include "gtest/gtest.h" #include "paddle/fluid/framework/lod_tensor.h" +#include "paddle/fluid/platform/eigen_ext.h" #include "paddle/fluid/platform/enforce.h" namespace paddle { namespace platform { TEST(float16, conversion_cpu) { - // Explicit conversion from Eigen::half - EXPECT_EQ(float16(Eigen::half(1.0f)).x, 0x3c00); - EXPECT_EQ(float16(Eigen::half(0.5f)).x, 0x3800); - EXPECT_EQ(float16(Eigen::half(0.33333f)).x, 0x3555); - EXPECT_EQ(float16(Eigen::half(0.0f)).x, 0x0000); - EXPECT_EQ(float16(Eigen::half(-0.0f)).x, 0x8000); - EXPECT_EQ(float16(Eigen::half(65504.0f)).x, 0x7bff); - EXPECT_EQ(float16(Eigen::half(65536.0f)).x, 0x7c00); - // Conversion from float EXPECT_EQ(float16(1.0f).x, 0x3c00); EXPECT_EQ(float16(0.5f).x, 0x3800); @@ -61,8 +54,6 @@ TEST(float16, conversion_cpu) { float16 v_assign; v_assign = float16(0); EXPECT_EQ(v_assign.x, 0x0000); - v_assign = Eigen::half(1.0f); - EXPECT_EQ(v_assign.x, 0x3c00); v_assign = 0.5f; EXPECT_EQ(v_assign.x, 0x3800); v_assign = 0.33333; @@ -73,7 +64,6 @@ TEST(float16, conversion_cpu) { EXPECT_EQ(v_assign.x, 0x3c00); // Conversion operator - EXPECT_EQ(Eigen::half(float16(1.0f)).x, 0x3c00); EXPECT_EQ(static_cast(float16(0.5f)), 0.5f); EXPECT_NEAR(static_cast(float16(0.33333)), 0.33333, 0.0001); EXPECT_EQ(static_cast(float16(-1)), -1); diff --git a/paddle/fluid/platform/float16_test.cu b/paddle/fluid/platform/float16_test.cu index 527da790414b1..d181660e31196 100644 --- a/paddle/fluid/platform/float16_test.cu +++ b/paddle/fluid/platform/float16_test.cu @@ -19,6 +19,7 @@ limitations under the License. */ #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/tensor_util.h" +#include "paddle/fluid/platform/eigen_ext.h" #include "paddle/fluid/platform/enforce.h" #define ARITHMETIC_KERNEL(op_type, sign) \ diff --git a/paddle/fluid/platform/for_range.h b/paddle/fluid/platform/for_range.h index 22d187b25902f..2cd6e44dd7a1a 100644 --- a/paddle/fluid/platform/for_range.h +++ b/paddle/fluid/platform/for_range.h @@ -62,7 +62,12 @@ struct ForRange { template inline void operator()(Function func) const { +#ifdef __HIPCC__ + // HIP will throw core dump when threads > 256 + constexpr int num_threads = 256; +#else constexpr int num_threads = 1024; +#endif size_t block_size = limit_ <= num_threads ? limit_ : num_threads; size_t grid_size = (limit_ + num_threads - 1) / num_threads; diff --git a/paddle/fluid/platform/gen_comm_id_helper.cc b/paddle/fluid/platform/gen_comm_id_helper.cc index ffe82371b18e6..f38603e80fb11 100644 --- a/paddle/fluid/platform/gen_comm_id_helper.cc +++ b/paddle/fluid/platform/gen_comm_id_helper.cc @@ -36,6 +36,8 @@ limitations under the License. */ namespace paddle { namespace platform { +std::once_flag SocketServer::init_flag_; + constexpr char COMM_HEAD[] = "_pd_gen_comm_id_"; // Check system calls, such as socket, bind. @@ -330,6 +332,22 @@ void RecvBroadCastCommID(int server_fd, std::string endpoint, CloseSocket(client); } +SocketServer& SocketServer::GetInstance(const std::string& end_point) { + static SocketServer instance; + std::call_once(init_flag_, [&]() { + instance.server_fd_ = CreateListenSocket(end_point); + instance.end_point_ = end_point; + }); + PADDLE_ENFORCE_NE(instance.server_fd_, -1, + platform::errors::Unavailable( + "listen socket failed with end_point=%s", end_point)); + PADDLE_ENFORCE_EQ(instance.end_point_, end_point, + platform::errors::InvalidArgument( + "old end_point=%s must equal with new end_point=%s", + instance.end_point_, end_point)); + return instance; +} + /// template instantiation #define INSTANT_TEMPLATE(Type) \ template void SendBroadCastCommID(std::vector servers, \ diff --git a/paddle/fluid/platform/gen_comm_id_helper.h b/paddle/fluid/platform/gen_comm_id_helper.h index 6014a2b4ff98d..c51c5ac6c8ac7 100644 --- a/paddle/fluid/platform/gen_comm_id_helper.h +++ b/paddle/fluid/platform/gen_comm_id_helper.h @@ -17,6 +17,8 @@ limitations under the License. */ #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) || \ defined(PADDLE_WITH_XPU_BKCL) #include +#include +#include #include #include @@ -39,6 +41,24 @@ void RecvBroadCastCommID(std::string endpoint, template void RecvBroadCastCommID(int server_fd, std::string endpoint, std::vector* nccl_ids); + +class SocketServer { + public: + SocketServer() = default; + + ~SocketServer() { CloseSocket(server_fd_); } + + int socket() const { return server_fd_; } + + static SocketServer& GetInstance(const std::string& end_point); + + private: + int server_fd_{-1}; + std::string end_point_; + + static std::once_flag init_flag_; +}; + } // namespace platform } // namespace paddle diff --git a/paddle/fluid/platform/gpu_launch_config.h b/paddle/fluid/platform/gpu_launch_config.h index 422e5a987b6ad..6c265677d63e9 100644 --- a/paddle/fluid/platform/gpu_launch_config.h +++ b/paddle/fluid/platform/gpu_launch_config.h @@ -41,7 +41,12 @@ struct GpuLaunchConfig { inline GpuLaunchConfig GetGpuLaunchConfig1D( const platform::CUDADeviceContext& context, int element_count, +#ifdef PADDLE_WITH_HIP + // HIP will throw GPU memory access fault if threads > 256 + int max_threads = 256) { +#else int max_threads = 1024) { +#endif PADDLE_ENFORCE_GT(element_count, 0, platform::errors::InvalidArgument( "element count should be greater than 0," diff --git a/paddle/fluid/platform/miopen_desc.h b/paddle/fluid/platform/miopen_desc.h index 68db32bac103b..c82e61ceb122c 100644 --- a/paddle/fluid/platform/miopen_desc.h +++ b/paddle/fluid/platform/miopen_desc.h @@ -37,9 +37,9 @@ namespace platform { using framework::Tensor; template -inline miopenDataType_t ToMIOpenDataType(const T& t) { +inline miopenDataType_t ToCudnnDataType(const T& t) { auto type = framework::ToDataType(t); - return ToMIOpenDataType(type); + return ToCudnnDataType(type); } inline std::vector TransformDimOrder(const std::vector& dims) { @@ -66,7 +66,7 @@ inline std::vector TransformDimOrder(const std::vector& dims) { } template <> -inline miopenDataType_t ToMIOpenDataType( +inline miopenDataType_t ToCudnnDataType( const framework::proto::VarType::Type& t) { miopenDataType_t type = miopenFloat; switch (t) { @@ -84,37 +84,54 @@ inline miopenDataType_t ToMIOpenDataType( class ActivationDescriptor { public: + using T = miopenActivationDescriptor; + struct Deleter { + void operator()(T* t) { + if (t != nullptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + dynload::miopenDestroyActivationDescriptor(t)); + t = nullptr; + } + } + }; ActivationDescriptor() { + T* raw_ptr; PADDLE_ENFORCE_CUDA_SUCCESS( - dynload::miopenCreateActivationDescriptor(&desc_)); - } - ~ActivationDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS( - dynload::miopenDestroyActivationDescriptor(desc_)); + dynload::miopenCreateActivationDescriptor(&raw_ptr)); + desc_.reset(raw_ptr); } template void set(miopenActivationMode_t mode, const T& coef) { PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenSetActivationDescriptor( - desc_, mode, static_cast(coef), 0.0, 0.0)); + desc_.get(), mode, static_cast(coef), 0.0, 0.0)); } - miopenActivationDescriptor_t desc() { return desc_; } - miopenActivationDescriptor_t desc() const { return desc_; } + T* desc() { return desc_.get(); } + T* desc() const { return desc_.get(); } private: - miopenActivationDescriptor_t desc_; + std::unique_ptr desc_; }; class TensorDescriptor { public: + using T = miopenTensorDescriptor; + struct Deleter { + void operator()(T* t) { + if (t != nullptr) { + PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenDestroyTensorDescriptor(t)); + t = nullptr; + } + } + }; TensorDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenCreateTensorDescriptor(&desc_)); - } - ~TensorDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenDestroyTensorDescriptor(desc_)); + T* raw_ptr; + PADDLE_ENFORCE_CUDA_SUCCESS( + dynload::miopenCreateTensorDescriptor(&raw_ptr)); + desc_.reset(raw_ptr); } - miopenTensorDescriptor_t desc() { return desc_; } - miopenTensorDescriptor_t desc() const { return desc_; } + T* desc() { return desc_.get(); } + T* desc() const { return desc_.get(); } void set(const Tensor& tensor, const int groups = 1) { auto dims = framework::vectorize(tensor.dims()); @@ -128,7 +145,7 @@ class TensorDescriptor { dims_with_group[1] = dims_with_group[1] / groups; } PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenSetTensorDescriptor( - desc_, ToMIOpenDataType(tensor.type()), + (miopenTensorDescriptor_t)(desc_.get()), ToCudnnDataType(tensor.type()), static_cast(dims_with_group.size()), const_cast(dims_with_group.data()), const_cast(strides.data()))); @@ -136,6 +153,9 @@ class TensorDescriptor { void set(const Tensor& tensor, const miopenTensorFormat_t format) { const int groups = 1; + PADDLE_ENFORCE_EQ(format, MIOPEN_TENSOR_NCHW, + platform::errors::InvalidArgument( + "format should ONLY be NCHW in MIOPEN.")); auto dims = framework::vectorize(tensor.dims()); std::vector strides(dims.size()); strides[dims.size() - 1] = 1; @@ -147,74 +167,98 @@ class TensorDescriptor { dims_with_group[1] = dims_with_group[1] / groups; } PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenSetTensorDescriptor( - desc_, ToMIOpenDataType(tensor.type()), + (miopenTensorDescriptor_t)(desc_.get()), ToCudnnDataType(tensor.type()), static_cast(dims_with_group.size()), const_cast(dims_with_group.data()), const_cast(strides.data()))); } private: - miopenTensorDescriptor_t desc_; + std::unique_ptr desc_; }; class FilterDescriptor { public: + using T = miopenTensorDescriptor; + struct Deleter { + void operator()(T* t) { + if (t != nullptr) { + PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenDestroyTensorDescriptor(t)); + t = nullptr; + } + } + }; FilterDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenCreateTensorDescriptor(&desc_)); - } - ~FilterDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenDestroyTensorDescriptor(desc_)); + T* raw_ptr; + PADDLE_ENFORCE_CUDA_SUCCESS( + dynload::miopenCreateTensorDescriptor(&raw_ptr)); + desc_.reset(raw_ptr); } - miopenTensorDescriptor_t desc() { return desc_; } - miopenTensorDescriptor_t desc() const { return desc_; } + T* desc() { return desc_.get(); } + T* desc() const { return desc_.get(); } void set(const Tensor& tensor, const miopenTensorFormat_t format, const int groups = 1) { - auto dims = framework::vectorize(tensor.dims()); - std::vector transformed_dims; PADDLE_ENFORCE_EQ(format, MIOPEN_TENSOR_NCHW, platform::errors::InvalidArgument( "format should ONLY be NCHW in MIOPEN.")); - transformed_dims = dims; + auto dims = framework::vectorize(tensor.dims()); + std::vector strides(dims.size()); + strides[dims.size() - 1] = 1; + for (int i = dims.size() - 2; i >= 0; i--) { + strides[i] = dims[i + 1] * strides[i + 1]; + } + std::vector dims_with_group(dims.begin(), dims.end()); if (groups > 1) { - transformed_dims[1] = transformed_dims[1] / groups; + dims_with_group[1] = dims_with_group[1] / groups; } PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenSetTensorDescriptor( - desc_, ToMIOpenDataType(tensor.type()), - static_cast(transformed_dims.size()), - const_cast(transformed_dims.data()), nullptr)); + (miopenTensorDescriptor_t)(desc_.get()), ToCudnnDataType(tensor.type()), + static_cast(dims_with_group.size()), + const_cast(dims_with_group.data()), + const_cast(strides.data()))); } private: - miopenTensorDescriptor_t desc_; + std::unique_ptr desc_; }; class ConvolutionDescriptor { public: + using T = miopenConvolutionDescriptor; + struct Deleter { + void operator()(T* t) { + if (t != nullptr) { + PADDLE_ENFORCE_CUDA_SUCCESS( + dynload::miopenDestroyConvolutionDescriptor(t)); + t = nullptr; + } + } + }; ConvolutionDescriptor() { + T* raw_ptr; PADDLE_ENFORCE_CUDA_SUCCESS( - dynload::miopenCreateConvolutionDescriptor(&desc_)); - } - ~ConvolutionDescriptor() { - PADDLE_ENFORCE_CUDA_SUCCESS( - dynload::miopenDestroyConvolutionDescriptor(desc_)); + dynload::miopenCreateConvolutionDescriptor(&raw_ptr)); + desc_.reset(raw_ptr); } - miopenConvolutionDescriptor_t desc() { return desc_; } - miopenConvolutionDescriptor_t desc() const { return desc_; } + T* desc() { return desc_.get(); } + T* desc() const { return desc_.get(); } void set(miopenDataType_t dtype, const std::vector& pads, const std::vector& strides, const std::vector& dilations, bool allow_tf32, const int groups = 1) { PADDLE_ENFORCE_CUDA_SUCCESS(dynload::miopenInitConvolutionNdDescriptor( - desc_, static_cast(pads.size()), const_cast(pads.data()), + (miopenConvolutionDescriptor_t)desc_.get(), + static_cast(pads.size()), const_cast(pads.data()), const_cast(strides.data()), const_cast(dilations.data()), miopenConvolution)); PADDLE_ENFORCE_CUDA_SUCCESS( - platform::dynload::miopenSetConvolutionGroupCount(desc_, groups)); + platform::dynload::miopenSetConvolutionGroupCount( + (miopenConvolutionDescriptor_t)desc_.get(), groups)); } private: - miopenConvolutionDescriptor_t desc_; + std::unique_ptr desc_; }; } // namespace platform diff --git a/paddle/fluid/platform/miopen_helper.h b/paddle/fluid/platform/miopen_helper.h index f6045130851ee..435d28d518df1 100644 --- a/paddle/fluid/platform/miopen_helper.h +++ b/paddle/fluid/platform/miopen_helper.h @@ -43,23 +43,6 @@ typedef enum { MIOPEN_TENSOR_NHWC = 1, } miopenTensorFormat_t; -// MIOPEN do not support indirect function call defined in cudnnWorkspaceHandle -struct miopenWorkspace { - explicit miopenWorkspace(size_t size) : size(size), data(NULL) { - PADDLE_ENFORCE_CUDA_SUCCESS(hipMalloc(&data, size)); - } - miopenWorkspace(const miopenWorkspace&) = delete; - miopenWorkspace(miopenWorkspace&&) = default; - miopenWorkspace& operator=(miopenWorkspace&&) = default; - ~miopenWorkspace() { - if (data) { - hipFree(data); - } - } - size_t size; - void* data; -}; - inline const char* miopenGetErrorString(miopenStatus_t status) { switch (status) { case miopenStatusSuccess: diff --git a/paddle/fluid/platform/mkldnn_reuse.h b/paddle/fluid/platform/mkldnn_reuse.h index 3e02a8672c360..c79b642c51b1f 100644 --- a/paddle/fluid/platform/mkldnn_reuse.h +++ b/paddle/fluid/platform/mkldnn_reuse.h @@ -13,6 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #pragma once +#include #include #include #include @@ -86,6 +87,11 @@ class MKLDNNHandlerT { "@dst_mem_p"); } + template + std::shared_ptr AcquireDstMemory(void) { + return this->AcquireMemoryFromPrimitive(fwd_pd_->dst_desc(), "@dstt_mem_p"); + } + template std::shared_ptr AcquireDstMemory( const framework::Tensor* output) { @@ -560,7 +566,10 @@ class BinaryMKLDNNHandler : public platform::MKLDNNHandlerT { const auto src_x_tz = framework::vectorize(x->dims()); const auto src_y_tz = framework::vectorize(y->dims()); - const auto dst_tz = framework::vectorize(z->dims()); + // if output tensor(z) is nullptr then we are computing into oneDNN + // managed buffer + const auto dst_tz = + (z == nullptr) ? src_x_tz : framework::vectorize(z->dims()); const auto src0_md = dnnl::memory::desc( src_x_tz, platform::MKLDNNGetDataType(), x->format()); @@ -621,6 +630,49 @@ class BinaryMKLDNNHandler : public platform::MKLDNNHandlerT { } }; +template +class ReductionMKLDNNHandler + : public platform::MKLDNNHandlerT { + public: + ReductionMKLDNNHandler(const dnnl::algorithm algo, const float p, + const float eps, const MKLDNNDeviceContext& dev_ctx, + const mkldnn::engine engine, platform::Place cpu_place, + const Tensor* x, const Tensor* y, + const std::string& uniq_name) + : platform::MKLDNNHandlerT( + dev_ctx, engine, cpu_place, + platform::CreateKey(dev_ctx, framework::vectorize(x->dims()), + uniq_name, + (std::to_string(static_cast(algo))))) { + if (!this->isCached()) { + PADDLE_ENFORCE_EQ( + x->layout(), DataLayout::kMKLDNN, + platform::errors::InvalidArgument("Wrong layout set for X tensor.")); + PADDLE_ENFORCE_NE( + x->format(), MKLDNNMemoryFormat::undef, + platform::errors::InvalidArgument("Wrong format set for X tensor.")); + + const auto src_tz = framework::vectorize(x->dims()); + const auto dst_tz = framework::vectorize(y->dims()); + + // For oneDNN dimensionality should match so we need to + // extend Y tensor dims with values of 1 (before and after pattern) + int j = 0; + std::vector dst_tz_ex(src_tz.size(), 1); + for (size_t i = 0; i < src_tz.size(); ++i) { + dst_tz_ex[i] = (src_tz[i] != dst_tz[j]) ? 1 : dst_tz[j++]; + } + + const auto src_md = dnnl::memory::desc( + src_tz, platform::MKLDNNGetDataType(), x->format()); + const auto dst_md = memory::desc( + dst_tz_ex, platform::MKLDNNGetDataType(), x->format()); + + this->AcquireForwardPrimitiveDescriptor(algo, src_md, dst_md, p, eps); + } + } +}; + template class ActivationMKLDNNHandler : public MKLDNNHandlerT 1) { -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #if NCCL_VERSION_CODE >= 2212 imperative::AllReduce(self.Var(), self.MutableVar(), strategy); #else @@ -984,7 +984,7 @@ void BindImperative(py::module *m_ptr) { PADDLE_THROW(platform::errors::Unimplemented( "Imperative allreduce is not supported when paddle is " "not compiled with NCCL.")); -#endif // PADDLE_WITH_NCCL +#endif // PADDLE_WITH_NCCL or PADDLE_WITH_RCCL } }, py::call_guard()) @@ -1016,7 +1016,7 @@ void BindImperative(py::module *m_ptr) { )DOC") .def("pin_memory", [](const std::shared_ptr &self) { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) PADDLE_THROW(platform::errors::PermissionDenied( "Cannot copy this Tensor to pinned memory in CPU version " "Paddle, " @@ -1050,7 +1050,7 @@ void BindImperative(py::module *m_ptr) { .def("cuda", [](const std::shared_ptr &self, int device_id, bool blocking) { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) PADDLE_THROW(platform::errors::PermissionDenied( "Cannot copy this Tensor to GPU in CPU version Paddle, " "Please recompile or reinstall Paddle with CUDA support.")); @@ -1412,7 +1412,8 @@ void BindImperative(py::module *m_ptr) { }, py::call_guard()); -#if (defined PADDLE_WITH_NCCL) || (defined PADDLE_WITH_XPU_BKCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) || \ + defined(PADDLE_WITH_XPU_BKCL) py::class_>(m, "ParallelContext"); @@ -1434,7 +1435,7 @@ void BindImperative(py::module *m_ptr) { py::call_guard()); #endif -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) py::class_>( m, "NCCLParallelContext") diff --git a/paddle/fluid/pybind/ps_gpu_wrapper_py.cc b/paddle/fluid/pybind/ps_gpu_wrapper_py.cc index 96acfd7bc0404..5bff9178fdfa5 100644 --- a/paddle/fluid/pybind/ps_gpu_wrapper_py.cc +++ b/paddle/fluid/pybind/ps_gpu_wrapper_py.cc @@ -32,7 +32,8 @@ namespace py = pybind11; namespace paddle { namespace pybind { -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) void BindPSGPUWrapper(py::module* m) { py::class_>( *m, "PSGPU") diff --git a/paddle/fluid/pybind/ps_gpu_wrapper_py.h b/paddle/fluid/pybind/ps_gpu_wrapper_py.h index 4048e88a55abc..8bd6ee13cf50b 100644 --- a/paddle/fluid/pybind/ps_gpu_wrapper_py.h +++ b/paddle/fluid/pybind/ps_gpu_wrapper_py.h @@ -22,7 +22,8 @@ namespace py = pybind11; namespace paddle { namespace pybind { -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) void BindPSGPUWrapper(py::module* m); #endif } // namespace pybind diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 39d2a9f1535b6..f8cb94a6cd9e7 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -84,7 +84,7 @@ limitations under the License. */ #include "paddle/fluid/pybind/ps_gpu_wrapper_py.h" #include "paddle/fluid/pybind/pybind_boost_headers.h" -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/pybind/nccl_wrapper_py.h" #endif #include "paddle/fluid/framework/data_type.h" @@ -93,11 +93,13 @@ limitations under the License. */ #include "paddle/fluid/pybind/reader_py.h" #include "paddle/fluid/pybind/tensor_py.h" #include "paddle/fluid/string/to_string.h" -#ifdef PADDLE_WITH_CUDA -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #include "paddle/fluid/operators/nccl/nccl_gpu_common.h" #endif +#ifndef PADDLE_WITH_HIP #include "paddle/fluid/platform/cuda_profiler.h" +#endif #include "paddle/fluid/platform/gpu_info.h" #endif @@ -126,7 +128,15 @@ PYBIND11_MAKE_OPAQUE(paddle::framework::FetchType); namespace paddle { namespace pybind { bool IsCompiledWithCUDA() { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) + return false; +#else + return true; +#endif +} + +bool IsCompiledWithROCM() { +#ifndef PADDLE_WITH_HIP return false; #else return true; @@ -376,7 +386,7 @@ PYBIND11_MODULE(core_noavx, m) { m.def("set_num_threads", &platform::SetNumThreads); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) m.def("cudnn_version", &platform::CudnnVersion); #endif @@ -390,7 +400,7 @@ PYBIND11_MODULE(core_noavx, m) { if (dl.ctx.device_type == kDLCPU) { paddle::framework::TensorFromDLPack(dl, &tensor); } -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (dl.ctx.device_type == kDLGPU) { paddle::framework::TensorFromDLPack(dl, &tensor); } @@ -1039,7 +1049,7 @@ PYBIND11_MODULE(core_noavx, m) { .def("height", &SelectedRows::height) .def("set_rows", [](SelectedRows &self, std::vector rows) { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) self.set_rows(rows); #else Vector new_rows(rows); @@ -1094,7 +1104,7 @@ All parameter, weight, gradient are variables in Paddle. .def("get_fetch_list", [](Variable &self) { return self.GetMutable(); }, py::return_value_policy::reference) -#if (defined(PADDLE_WITH_NCCL)) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) .def("get_communicator", [](Variable &self) -> platform::Communicator * { return self.GetMutable(); @@ -1333,7 +1343,7 @@ All parameter, weight, gradient are variables in Paddle. .def_static("create", [](paddle::platform::CUDAPlace& place) -> paddle::platform::DeviceContext* { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) PADDLE_THROW( platform::errors::PermissionDenied( "Cannot use CUDAPlace in CPU only version, " @@ -1345,7 +1355,7 @@ All parameter, weight, gradient are variables in Paddle. .def_static("create", [](paddle::platform::CUDAPinnedPlace& place) -> paddle::platform::DeviceContext* { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) PADDLE_THROW( platform::errors::PermissionDenied( "Cannot use CUDAPinnedPlace in CPU only version, " @@ -1355,7 +1365,7 @@ All parameter, weight, gradient are variables in Paddle. #endif });; // clang-format on -#if defined(PADDLE_WITH_NCCL) +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) py::class_(m, "Communicator").def(py::init<>()); #endif py::class_(m, "CUDAPlace", R"DOC( @@ -1384,7 +1394,7 @@ All parameter, weight, gradient are variables in Paddle. )DOC") .def("__init__", [](platform::CUDAPlace &self, int dev_id) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (UNLIKELY(dev_id < 0)) { LOG(ERROR) << string::Sprintf( "Invalid CUDAPlace(%d), device id must be 0 or " @@ -1422,7 +1432,7 @@ All parameter, weight, gradient are variables in Paddle. std::exit(-1); #endif }) -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) .def("get_device_id", [](const platform::CUDAPlace &self) { return self.GetDeviceId(); }) .def("_type", &PlaceIndex) @@ -1538,7 +1548,7 @@ All parameter, weight, gradient are variables in Paddle. )DOC") .def("__init__", [](platform::CUDAPinnedPlace &self) { -#ifndef PADDLE_WITH_CUDA +#if !defined(PADDLE_WITH_CUDA) && !defined(PADDLE_WITH_HIP) PADDLE_THROW(platform::errors::PermissionDenied( "Cannot use CUDAPinnedPlace in CPU only version, " "Please recompile or reinstall Paddle with CUDA support.")); @@ -1726,6 +1736,7 @@ All parameter, weight, gradient are variables in Paddle. m.def("init_devices", []() { framework::InitDevices(); }); m.def("is_compiled_with_cuda", IsCompiledWithCUDA); + m.def("is_compiled_with_rocm", IsCompiledWithROCM); m.def("is_compiled_with_xpu", IsCompiledWithXPU); m.def("is_compiled_with_mkldnn", IsCompiledWithMKLDNN); m.def("supports_bfloat16", SupportsBfloat16); @@ -1769,7 +1780,7 @@ All parameter, weight, gradient are variables in Paddle. py::arg("cmd"), py::arg("time_out") = 0, py::arg("sleep_inter") = 0, py::arg("redirect_stderr") = false); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) m.def("is_float16_supported", [](const platform::CUDAPlace &place) -> bool { // Only GPUs with Compute Capability >= 53 support float16 return platform::GetCUDAComputeCapability(place.device) >= 53; @@ -1943,10 +1954,10 @@ All parameter, weight, gradient are variables in Paddle. py::return_value_policy::take_ownership); m.def("op_support_gpu", OpSupportGPU); -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) m.def("get_cuda_device_count", platform::GetCUDADeviceCount); -#ifndef _WIN32 +#if !defined(PADDLE_WITH_HIP) && !defined(_WIN32) m.def("nvprof_init", platform::CudaProfilerInit); m.def("nvprof_start", platform::CudaProfilerStart); m.def("nvprof_stop", platform::CudaProfilerStop); @@ -1987,6 +1998,13 @@ All parameter, weight, gradient are variables in Paddle. m.def("size_of_dtype", framework::SizeOfType); +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) + m.def("set_cublas_switch", platform::SetAllowTF32Cublas); + m.def("get_cublas_switch", platform::AllowTF32Cublas); + m.def("set_cudnn_switch", platform::SetAllowTF32Cudnn); + m.def("get_cudnn_switch", platform::AllowTF32Cudnn); +#endif // PADDLE_WITH_CUDA + using VarQuantScale = std::unordered_map>; @@ -2451,6 +2469,12 @@ All parameter, weight, gradient are variables in Paddle. [](BuildStrategy &self, int nccl_comm_num) { self.nccl_comm_num_ = nccl_comm_num; }) + .def_property( + "bkcl_comm_num", + [](const BuildStrategy &self) { return self.bkcl_comm_num_; }, + [](BuildStrategy &self, int bkcl_comm_num) { + self.bkcl_comm_num_ = bkcl_comm_num; + }) .def_property("use_hierarchical_allreduce", [](const BuildStrategy &self) { return self.use_hierarchical_allreduce_; @@ -2812,7 +2836,8 @@ All parameter, weight, gradient are variables in Paddle. #ifdef PADDLE_WITH_PSLIB BindHeterWrapper(&m); #endif -#if (defined PADDLE_WITH_NCCL) && (defined PADDLE_WITH_PSLIB) +#if (defined PADDLE_WITH_NCCL || defined PADDLE_WITH_RCCL) && \ + (defined PADDLE_WITH_PSLIB) BindPSGPUWrapper(&m); #endif BindGlooWrapper(&m); @@ -2820,7 +2845,7 @@ All parameter, weight, gradient are variables in Paddle. #ifdef PADDLE_WITH_BOX_PS BindBoxWrapper(&m); #endif -#ifdef PADDLE_WITH_NCCL +#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) BindNCCLWrapper(&m); #endif #ifdef PADDLE_WITH_GLOO diff --git a/paddle/fluid/pybind/reader_py.cc b/paddle/fluid/pybind/reader_py.cc index 856c5aac5eb38..abe1977eb6978 100644 --- a/paddle/fluid/pybind/reader_py.cc +++ b/paddle/fluid/pybind/reader_py.cc @@ -223,6 +223,10 @@ class MultiDeviceFeedReader { ReadAsync(); } + void Shutdown() { + for (auto &r : readers_) r->Shutdown(); + } + ~MultiDeviceFeedReader() { queue_->Close(); pool_.reset(); @@ -266,10 +270,6 @@ class MultiDeviceFeedReader { } } - void Shutdown() { - for (auto &r : readers_) r->Shutdown(); - } - void Start() { for (auto &r : readers_) r->Start(); } @@ -362,6 +362,8 @@ void BindMultiDeviceReader(py::module *module, const char *reader_name) { }, py::call_guard()) .def("reset", &ReaderType::Reset, + py::call_guard()) + .def("shutdown", &ReaderType::Shutdown, py::call_guard()); } diff --git a/paddle/fluid/pybind/tensor_py.h b/paddle/fluid/pybind/tensor_py.h index 5ddb498980d77..5f25217007017 100644 --- a/paddle/fluid/pybind/tensor_py.h +++ b/paddle/fluid/pybind/tensor_py.h @@ -27,7 +27,7 @@ limitations under the License. */ #include "paddle/fluid/operators/math/concat_and_split.h" #include "paddle/fluid/operators/strided_memcpy.h" #include "paddle/fluid/platform/bfloat16.h" -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #include "paddle/fluid/platform/cuda_device_guard.h" #endif #include "paddle/fluid/platform/device_context.h" @@ -226,7 +226,7 @@ T TensorGetElement(const framework::Tensor &self, size_t offset) { paddle::memory::Copy(platform::CPUPlace(), &b, p, a + offset, sizeof(T)); #endif } else if (platform::is_gpu_place(self.place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) const T *a = self.data(); auto p = BOOST_GET_CONST(platform::CUDAPlace, self.place()); paddle::memory::Copy(platform::CPUPlace(), &b, p, a + offset, sizeof(T), @@ -250,7 +250,7 @@ void TensorSetElement(framework::Tensor *self, size_t offset, T elem) { paddle::memory::Copy(p, a + offset, platform::CPUPlace(), &elem, sizeof(T)); #endif } else if (platform::is_gpu_place(self->place())) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) auto p = BOOST_GET_CONST(platform::CUDAPlace, self->place()); T *a = self->mutable_data(p); paddle::memory::Copy(p, a + offset, platform::CPUPlace(), &elem, sizeof(T), @@ -296,7 +296,7 @@ void SetTensorFromPyArrayT( "Please recompile or reinstall Paddle with XPU support.")); #endif } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (paddle::platform::is_gpu_place(place)) { // NOTE(wangxi): When copying data to the accelerator card, // we need set_device(dev_id) first. @@ -304,8 +304,13 @@ void SetTensorFromPyArrayT( platform::CUDADeviceGuard guard( BOOST_GET_CONST(platform::CUDAPlace, tmp_place).device); auto dst = self->mutable_data(place); +#ifdef PADDLE_WITH_HIP + paddle::platform::GpuMemcpySync(dst, array.data(), array.nbytes(), + hipMemcpyHostToDevice); +#else paddle::platform::GpuMemcpySync(dst, array.data(), array.nbytes(), cudaMemcpyHostToDevice); +#endif } else if (paddle::platform::is_cuda_pinned_place(place)) { auto dst = self->mutable_data(place); @@ -474,7 +479,7 @@ inline framework::Tensor *_getTensor(const framework::Tensor &self, self.type()); #endif } else { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) if (platform::is_cuda_pinned_place(place)) { output->mutable_data(BOOST_GET_CONST(platform::CUDAPinnedPlace, place), self.type()); @@ -707,7 +712,7 @@ inline py::array TensorToPyArray(const framework::Tensor &tensor, "Please recompile or reinstall Paddle with XPU support.")); #endif } else if (is_gpu_tensor) { -#ifdef PADDLE_WITH_CUDA +#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) py::array py_arr(py::dtype(py_dtype_str.c_str()), py_dims, py_strides); PADDLE_ENFORCE_EQ(py_arr.writeable(), true, platform::errors::InvalidArgument( diff --git a/paddle/fluid/string/tinyformat/tinyformat.h b/paddle/fluid/string/tinyformat/tinyformat.h index a5c1798e10027..7498c6a46e38a 100644 --- a/paddle/fluid/string/tinyformat/tinyformat.h +++ b/paddle/fluid/string/tinyformat/tinyformat.h @@ -777,7 +777,7 @@ inline void formatImpl(std::ostream &out, const char *fmt, // Print remaining part of format string. fmt = printFormatStringLiteral(out, fmt); - if (*fmt != '\0') + if (fmt != nullptr && *fmt != '\0' && *fmt != 0) TINYFORMAT_ERROR( "tinyformat: Too many conversion specifiers in format string"); diff --git a/paddle/scripts/paddle_build.bat b/paddle/scripts/paddle_build.bat index dc2e3ab593c22..2edb062ac806f 100644 --- a/paddle/scripts/paddle_build.bat +++ b/paddle/scripts/paddle_build.bat @@ -30,8 +30,9 @@ taskkill /f /im op_function_generator.exe wmic process where name="op_function_generator.exe" call terminate taskkill /f /im python.exe 2>NUL + rem ------initialize common variable------ -if not defined GENERATOR set GENERATOR="Visual Studio 14 2015 Win64" +if not defined GENERATOR set GENERATOR="Visual Studio 15 2017 Win64" if not defined BRANCH set BRANCH=develop if not defined WITH_TENSORRT set WITH_TENSORRT=ON if not defined TENSORRT_ROOT set TENSORRT_ROOT=D:/TensorRT @@ -80,7 +81,7 @@ git show-ref --verify --quiet refs/heads/last_pr if %ERRORLEVEL% EQU 0 ( git diff HEAD last_pr --stat --name-only git diff HEAD last_pr --stat --name-only | findstr "setup.py.in" - if %ERRORLEVEL% EQU 0 ( + if !ERRORLEVEL! EQU 0 ( rmdir build /s/q ) git branch -D last_pr @@ -146,11 +147,22 @@ rem set CLCACHE_OBJECT_CACHE_TIMEOUT_MS=1000000 :: set maximum cache size to 20G rem clcache.exe -M 21474836480 +:: install ninja if GENERATOR is Ninja +if %GENERATOR% == "Ninja" ( + pip install ninja + if %errorlevel% NEQ 0 ( + echo pip install ninja failed! + exit /b 7 + ) +) + rem ------show summary of current environment---------- cmake --version -nvcc --version -where nvidia-smi -nvidia-smi +if "%WITH_GPU%"=="ON" ( + nvcc --version + where nvidia-smi + nvidia-smi +) python %work_dir%\tools\summary_env.py %cache_dir%\tools\busybox64.exe bash %work_dir%\tools\get_cpu_info.sh @@ -232,7 +244,9 @@ echo ======================================== echo Step 1. Cmake ... echo ======================================== -call "C:\Program Files (x86)\Microsoft Visual Studio 14.0\VC\vcvarsall.bat" amd64 +rem Configure the environment for 64-bit builds. 'DISTUTILS_USE_SDK' indicates that the user has selected the compiler. +call "C:\Program Files (x86)\Microsoft Visual Studio\2017\Community\VC\Auxiliary\Build\vcvars64.bat" +set DISTUTILS_USE_SDK=1 for /F %%# in ('wmic os get localdatetime^|findstr 20') do set start=%%# set start=%start:~4,10% @@ -252,16 +266,16 @@ if %day_now% NEQ %day_before% ( echo %day_now% > %cache_dir%\day.txt type %cache_dir%\day.txt if %day_now% EQU 21 ( - rmdir %cache_dir%\third_party_GPU/ /s/q - rmdir %cache_dir%\third_party/ /s/q + rmdir %cache_dir%\third_party_GPU /s/q + rmdir %cache_dir%\third_party /s/q ) if %day_now% EQU 11 ( - rmdir %cache_dir%\third_party_GPU/ /s/q - rmdir %cache_dir%\third_party/ /s/q + rmdir %cache_dir%\third_party_GPU /s/q + rmdir %cache_dir%\third_party /s/q ) if %day_now% EQU 01 ( - rmdir %cache_dir%\third_party_GPU/ /s/q - rmdir %cache_dir%\third_party/ /s/q + rmdir %cache_dir%\third_party_GPU /s/q + rmdir %cache_dir%\third_party /s/q ) ) @@ -285,14 +299,14 @@ if "%WITH_GPU%"=="ON" ( ) :cmake_impl -echo cmake .. -G %GENERATOR% -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ +echo cmake .. -G %GENERATOR% -T host=x64 -DCMAKE_BUILD_TYPE=Release -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ -DWITH_TESTING=%WITH_TESTING% -DWITH_PYTHON=%WITH_PYTHON% -DPYTHON_EXECUTABLE=%PYTHON_EXECUTABLE% -DON_INFER=%ON_INFER% ^ -DWITH_INFERENCE_API_TEST=%WITH_INFERENCE_API_TEST% -DTHIRD_PARTY_PATH=%THIRD_PARTY_PATH% ^ -DINFERENCE_DEMO_INSTALL_DIR=%INFERENCE_DEMO_INSTALL_DIR% -DWITH_STATIC_LIB=%WITH_STATIC_LIB% ^ -DWITH_TENSORRT=%WITH_TENSORRT% -DTENSORRT_ROOT="%TENSORRT_ROOT%" -DMSVC_STATIC_CRT=%MSVC_STATIC_CRT% ^ -DWITH_UNITY_BUILD=%WITH_UNITY_BUILD% -DCUDA_ARCH_NAME=%CUDA_ARCH_NAME% -cmake .. -G %GENERATOR% -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ +cmake .. -G %GENERATOR% -DCMAKE_BUILD_TYPE=Release -T host=x64 -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ -DWITH_TESTING=%WITH_TESTING% -DWITH_PYTHON=%WITH_PYTHON% -DPYTHON_EXECUTABLE=%PYTHON_EXECUTABLE% -DON_INFER=%ON_INFER% ^ -DWITH_INFERENCE_API_TEST=%WITH_INFERENCE_API_TEST% -DTHIRD_PARTY_PATH=%THIRD_PARTY_PATH% ^ -DINFERENCE_DEMO_INSTALL_DIR=%INFERENCE_DEMO_INSTALL_DIR% -DWITH_STATIC_LIB=%WITH_STATIC_LIB% ^ @@ -313,11 +327,17 @@ echo ======================================== echo Step 2. Buile Paddle ... echo ======================================== -for /F %%# in ('wmic cpu get NumberOfLogicalProcessors^|findstr [0-9]') do set /a PARALLEL_PROJECT_COUNT=%%#*2/3 +for /F %%# in ('wmic cpu get NumberOfLogicalProcessors^|findstr [0-9]') do set /a PARALLEL_PROJECT_COUNT=%%#*4/5 +echo "PARALLEL PROJECT COUNT is %PARALLEL_PROJECT_COUNT%" set build_times=1 :build_tp echo Build third_party the %build_times% time: -msbuild /m /p:Configuration=Release /verbosity:quiet third_party.vcxproj + +if %GENERATOR% == "Ninja" ( + ninja third_party +) else ( + MSBuild /m /p:PreferredToolArchitecture=x64 /p:Configuration=Release /verbosity:quiet third_party.vcxproj +) if %ERRORLEVEL% NEQ 0 ( set /a build_times=%build_times%+1 if %build_times% GTR 2 ( @@ -335,10 +355,14 @@ set build_times=1 rem clcache.exe -z echo Build Paddle the %build_times% time: -if "%WITH_CLCACHE%"=="OFF" ( - msbuild /m:%PARALLEL_PROJECT_COUNT% /p:Configuration=Release /verbosity:%LOG_LEVEL% paddle.sln +if %GENERATOR% == "Ninja" ( + ninja -j %PARALLEL_PROJECT_COUNT% ) else ( - msbuild /m:%PARALLEL_PROJECT_COUNT% /p:TrackFileAccess=false /p:CLToolExe=clcache.exe /p:CLToolPath=%PYTHON_ROOT%\Scripts /p:Configuration=Release /verbosity:%LOG_LEVEL% paddle.sln + if "%WITH_CLCACHE%"=="OFF" ( + MSBuild /m:%PARALLEL_PROJECT_COUNT% /p:PreferredToolArchitecture=x64 /p:Configuration=Release /verbosity:%LOG_LEVEL% ALL_BUILD.vcxproj + ) else ( + MSBuild /m:%PARALLEL_PROJECT_COUNT% /p:PreferredToolArchitecture=x64 /p:TrackFileAccess=false /p:CLToolExe=clcache.exe /p:CLToolPath=%PYTHON_ROOT%\Scripts /p:Configuration=Release /verbosity:%LOG_LEVEL% paddle.sln + ) ) if %ERRORLEVEL% NEQ 0 ( @@ -476,7 +500,7 @@ setlocal enabledelayedexpansion :: for /F %%# in ('cmd /C nvidia-smi -L ^|find "GPU" /C') do set CUDA_DEVICE_COUNT=%%# set CUDA_DEVICE_COUNT=1 -%cache_dir%\tools\busybox64.exe bash %work_dir%\tools\windows\run_unittests.sh %NIGHTLY_MODE% +%cache_dir%\tools\busybox64.exe bash %work_dir%\tools\windows\run_unittests.sh %NIGHTLY_MODE% %PRECISION_TEST% goto:eof @@ -562,7 +586,7 @@ echo git fetch upstream $BRANCH # develop is not fetched>> check_change_of_ echo fi>> check_change_of_unittest.sh echo git checkout -b origin_pr >> check_change_of_unittest.sh echo git checkout -f $BRANCH >> check_change_of_unittest.sh -echo cmake .. -G %GENERATOR% -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ +echo cmake .. -G %GENERATOR% -T host=x64 -DWITH_AVX=%WITH_AVX% -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% ^ -DWITH_TESTING=%WITH_TESTING% -DWITH_PYTHON=%WITH_PYTHON% -DPYTHON_EXECUTABLE=%PYTHON_EXECUTABLE% -DON_INFER=%ON_INFER% ^ -DWITH_INFERENCE_API_TEST=%WITH_INFERENCE_API_TEST% -DTHIRD_PARTY_PATH=%THIRD_PARTY_PATH% ^ -DINFERENCE_DEMO_INSTALL_DIR=%INFERENCE_DEMO_INSTALL_DIR% -DWITH_STATIC_LIB=%WITH_STATIC_LIB% ^ @@ -679,7 +703,7 @@ echo ======================================== echo Clean up environment at the end ... echo ======================================== taskkill /f /im cmake.exe 2>NUL -taskkill /f /im msbuild.exe 2>NUL +taskkill /f /im MSBuild.exe 2>NUL taskkill /f /im git.exe 2>NUL taskkill /f /im cl.exe 2>NUL taskkill /f /im lib.exe 2>NUL diff --git a/paddle/scripts/paddle_build.sh b/paddle/scripts/paddle_build.sh index 9ca426ae029aa..7a360ac22960e 100755 --- a/paddle/scripts/paddle_build.sh +++ b/paddle/scripts/paddle_build.sh @@ -205,6 +205,13 @@ function cmake_base() { -DPYTHON_INCLUDE_DIR:PATH=/opt/_internal/cpython-3.8.0/include/python3.8 -DPYTHON_LIBRARIES:FILEPATH=/opt/_internal/cpython-3.8.0/lib/libpython3.so" pip3.8 install -r ${PADDLE_ROOT}/python/requirements.txt + elif [ "$1" == "conda-python3.7" ]; then + export LD_LIBRARY_PATH=/opt/conda/lib/:${LD_LIBRARY_PATH} + export PATH=/opt/conda/bin/:${PATH} + export PYTHON_FLAGS="-DPYTHON_EXECUTABLE:FILEPATH=/opt/conda/bin/python + -DPYTHON_INCLUDE_DIR:PATH=/opt/conda/include/python3.7m + -DPYTHON_LIBRARIES:FILEPATH=/opt/conda/lib/libpython3.so" + /opt/conda/bin/pip install -r ${PADDLE_ROOT}/python/requirements.txt fi else pip install -r ${PADDLE_ROOT}/python/requirements.txt @@ -230,7 +237,8 @@ function cmake_base() { ${PYTHON_FLAGS} -DWITH_GPU=${WITH_GPU:-OFF} -DWITH_TENSORRT=${WITH_TENSORRT:-ON} - -DWITH_AMD_GPU=${WITH_AMD_GPU:-OFF} + -DWITH_ROCM=${WITH_ROCM:-OFF} + -DWITH_RCCL=${WITH_RCCL:-OFF} -DWITH_DISTRIBUTE=${distibuted_flag} -DWITH_MKL=${WITH_MKL:-ON} -DWITH_AVX=${WITH_AVX:-OFF} @@ -267,7 +275,8 @@ EOF ${PYTHON_FLAGS} \ -DWITH_GPU=${WITH_GPU:-OFF} \ -DWITH_TENSORRT=${WITH_TENSORRT:-ON} \ - -DWITH_AMD_GPU=${WITH_AMD_GPU:-OFF} \ + -DWITH_ROCM=${WITH_ROCM:-OFF} \ + -DWITH_RCCL=${WITH_RCCL:-OFF} \ -DWITH_DISTRIBUTE=${distibuted_flag} \ -DWITH_MKL=${WITH_MKL:-ON} \ -DWITH_AVX=${WITH_AVX:-OFF} \ @@ -607,7 +616,16 @@ EOF echo "Unittests with nightly labels are only run at night" echo "=========================================" fi - ctest -E "($disable_ut_quickly)" -LE ${nightly_label} --output-on-failure -j $2 | tee $tmpfile + bash $PADDLE_ROOT/tools/check_added_ut.sh + get_precision_ut_mac + if [[ "$on_precision" == "0" ]];then + ctest -E "($disable_ut_quickly)" -LE ${nightly_label} --output-on-failure -j $2 | tee $tmpfile + else + ctest -R "($UT_list_prec)" -E "($disable_ut_quickly)" -LE ${nightly_label} --output-on-failure -j $2 | tee $tmpfile + tmpfile_rand=`date +%s%N` + tmpfile=$tmp_dir/$tmpfile_rand + ctest -R "($UT_list_prec_1)" -E "($disable_ut_quickly)" -LE ${nightly_label} --output-on-failure -j $2 | tee $tmpfile + fi failed_test_lists='' collect_failed_tests mactest_error=0 @@ -615,18 +633,26 @@ EOF retry_time=3 exec_times=0 exec_time_array=('first' 'second' 'third') - exec_retry_threshold=20 + exec_retry_threshold=10 + is_retry_execuate=0 if [ -n "$failed_test_lists" ];then mactest_error=1 read need_retry_ut_str <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) need_retry_ut_arr=(${need_retry_ut_str}) need_retry_ut_count=${#need_retry_ut_arr[@]} + read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) if [ $need_retry_ut_count -lt $exec_retry_threshold ];then - while ( [ $exec_times -lt $retry_time ] && [ -n "${failed_test_lists}" ] ) + while ( [ $exec_times -lt $retry_time ] ) do retry_unittests_record="$retry_unittests_record$failed_test_lists" failed_test_lists_ult=`echo "${failed_test_lists}"` - read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) + if [[ "${exec_times}" == "1" ]];then + if [[ "${failed_test_lists}" == "" ]];then + break + else + read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(.+\)" | sed 's/(.\+)//' | sed 's/- //' ) + fi + fi echo "=========================================" echo "This is the ${exec_time_array[$exec_times]} time to re-run" echo "=========================================" @@ -650,9 +676,8 @@ EOF exec_times=$[$exec_times+1] done else - echo "=========================================" - echo "There are more than 20 failed unit tests, so no unit test retry!!!" - echo "=========================================" + # There are more than 10 failed unit tests, so no unit test retry + is_retry_execuate=1 fi fi @@ -665,24 +690,46 @@ EOF set +x export http_proxy=$my_proxy export https_proxy=$my_proxy - set -x if [ "$mactest_error" != 0 ];then - if [[ "$failed_test_lists" == "" ]]; then - echo "========================================" - echo "There are failed tests, which have been successful after re-run:" - echo "========================================" - echo "The following tests have been re-ran:" - echo "${retry_unittests_record}" + show_ut_retry_result + fi + set -x + fi +} + +function get_precision_ut_mac() { + on_precision=0 + set -x + UT_list=$(ctest -N | awk -F ': ' '{print $2}' | sed '/^$/d' | sed '$d') + precison_cases="" + if [ ${PRECISION_TEST:-OFF} == "ON" ]; then + python3.7 $PADDLE_ROOT/tools/get_pr_ut.py + if [[ -f "ut_list" ]]; then + set +x + echo "PREC length: "`wc -l ut_list` + precision_cases=`cat ut_list` + set -x + fi + fi + if [ ${PRECISION_TEST:-OFF} == "ON" ] && [[ "$precision_cases" != "" ]];then + UT_list_re='' + on_precision=1 + re=$(cat ut_list|awk -F ' ' '{print }' | awk 'BEGIN{ all_str=""}{if (all_str==""){all_str=$1}else{all_str=all_str"$|^"$1}} END{print "^"all_str"$"}') + UT_list_prec_1='ut_list_prec2' + for case in $UT_list; do + flag=$(echo $case|grep -oE $re) + if [ -n "$flag" ];then + if [ -z "$UT_list_prec" ];then + UT_list_prec="^$case$" + elif [[ "${#UT_list_prec}" -gt 10000 ]];then + UT_list_prec_1="$UT_list_prec_1|^$case$" + else + UT_list_prec="$UT_list_prec|^$case$" + fi else - failed_test_lists_ult=`echo "${failed_test_lists}"` - echo "========================================" - echo "Summary Failed Tests... " - echo "========================================" - echo "The following tests FAILED: " - echo "${failed_test_lists_ult}" - exit 8; + echo ${case} "won't run in PRECISION_TEST mode." fi - fi + done fi } @@ -990,6 +1037,8 @@ function card_test() { # get the CUDA device count, XPU device count is one if [ "${WITH_XPU}" == "ON" ];then CUDA_DEVICE_COUNT=1 + elif [ "${WITH_ROCM}" == "ON" ];then + CUDA_DEVICE_COUNT=4 else CUDA_DEVICE_COUNT=$(nvidia-smi -L | wc -l) fi @@ -1204,23 +1253,33 @@ set +x retry_unittests_record='' retry_time=3 exec_time_array=('first' 'second' 'third') - exec_retry_threshold=20 + exec_retry_threshold=10 + is_retry_execuate=0 if [ -n "$failed_test_lists" ];then + if [ ${TIMEOUT_DEBUG_HELP:-OFF} == "ON" ];then + bash $PADDLE_ROOT/tools/timeout_debug_help.sh "$failed_test_lists" # cat logs for tiemout uts which killed by ctest + fi read need_retry_ut_str <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(.+\)" | sed 's/(.\+)//' | sed 's/- //' ) need_retry_ut_arr=(${need_retry_ut_str}) need_retry_ut_count=${#need_retry_ut_arr[@]} + read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(.+\)" | sed 's/(.\+)//' | sed 's/- //' ) if [ $need_retry_ut_count -lt $exec_retry_threshold ];then - while ( [ $exec_times -lt $retry_time ] && [ -n "${failed_test_lists}" ] ) + while ( [ $exec_times -lt $retry_time ] ) do - retry_unittests_record="$retry_unittests_record$failed_test_lists" failed_test_lists_ult=`echo "${failed_test_lists}" |grep -Po '[^ ].*$'` - read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(.+\)" | sed 's/(.\+)//' | sed 's/- //' ) + if [[ "${exec_times}" == "1" ]];then + if [[ "${failed_test_lists}" == "" ]];then + break + else + read retry_unittests <<< $(echo "$failed_test_lists" | grep -oEi "\-.+\(.+\)" | sed 's/(.\+)//' | sed 's/- //' ) + fi + fi echo "=========================================" echo "This is the ${exec_time_array[$exec_times]} time to re-run" echo "=========================================" echo "The following unittest will be re-run:" - echo "${failed_test_lists_ult}" + echo "${retry_unittests}" for line in ${retry_unittests[@]} ; do @@ -1270,36 +1329,51 @@ set +x one_card_retry='' multiple_card_retry='' exclusive_retry='' - retry_unittests='' done else - echo "=========================================" - echo "There are more than 20 failed unit tests, so no unit test retry!!!" - echo "=========================================" + # There are more than 10 failed unit tests, so no unit test retry + is_retry_execuate=1 fi fi if [[ "$EXIT_CODE" != "0" ]]; then - if [[ "$failed_test_lists" == "" ]]; then - echo "========================================" - echo "There are failed tests, which have been successful after re-run:" - echo "========================================" - echo "The following tests have been re-ran:" - echo "${retry_unittests_record}" - else - failed_test_lists_ult=`echo "${failed_test_lists}" |grep -Po '[^ ].*$'` - echo "========================================" - echo "Summary Failed Tests... " - echo "========================================" - echo "The following tests FAILED: " - echo "${failed_test_lists_ult}" - exit 8; - fi + show_ut_retry_result fi set -ex fi } +function show_ut_retry_result() { + if [[ "$is_retry_execuate" != "0" ]];then + failed_test_lists_ult=`echo "${failed_test_lists}" | grep -Po '[^ ].*$'` + echo "=========================================" + echo "There are more than 10 failed unit tests, so no unit test retry!!!" + echo "=========================================" + echo "The following tests FAILED: " + echo "${failed_test_lists_ult}" + exit 8; + else + read retry_unittests_ut_name <<< $(echo "$retry_unittests_record" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) + retry_unittests_record_judge=$(echo ${retry_unittests_ut_name}| tr ' ' '\n' | sort | uniq -c | awk '{if ($1 >=3) {print $2}}') + if [ -z "${retry_unittests_record_judge}" ];then + echo "========================================" + echo "There are failed tests, which have been successful after re-run:" + echo "========================================" + echo "The following tests have been re-ran:" + echo "${retry_unittests_record}" + else + failed_ut_re=$(echo "${retry_unittests_record_judge}" | awk BEGIN{RS=EOF}'{gsub(/\n/,"|");print}') + echo "========================================" + echo "There are failed tests, which have been executed re-run,but success rate is less than 50%:" + echo "Summary Failed Tests... " + echo "========================================" + echo "The following tests FAILED: " + echo "${retry_unittests_record}" | sort -u | grep -E "$failed_ut_re" + exit 8; + fi + fi +} + function parallel_test_base_cpu() { mkdir -p ${PADDLE_ROOT}/build cd ${PADDLE_ROOT}/build @@ -1360,7 +1434,7 @@ function parallel_test() { mkdir -p ${PADDLE_ROOT}/build cd ${PADDLE_ROOT}/build pip install ${PADDLE_ROOT}/build/python/dist/*whl - if [ "$WITH_GPU" == "ON" ];then + if [ "$WITH_GPU" == "ON" ] || [ "$WITH_ROCM" == "ON" ];then parallel_test_base_gpu else if [ "$WITH_XPU" == "ON" ];then @@ -1877,6 +1951,8 @@ function main() { assert_api_spec_approvals ;; test_inference) + PADDLE_ROOT="$( cd "$( dirname "${BASH_SOURCE[0]}")/../../" && pwd )" + python ${PADDLE_ROOT}/tools/remove_grad_op_and_kernel.py gen_fluid_lib ${parallel_number} test_fluid_lib #test_fluid_lib_train @@ -1917,6 +1993,11 @@ function main() { parallel_test check_coverage ;; + check_rocm_coverage) + cmake_gen_and_build ${PYTHON_ABI:-""} ${parallel_number} + parallel_test + check_coverage + ;; cmake_gen) cmake_gen ${PYTHON_ABI:-""} ;; diff --git a/paddle/scripts/windows_build/build.bat b/paddle/scripts/windows_build/build.bat index 6f99c23ccd262..9a2ed349e5b92 100644 --- a/paddle/scripts/windows_build/build.bat +++ b/paddle/scripts/windows_build/build.bat @@ -61,8 +61,8 @@ echo Current directory : %cd% call:rest_env -echo cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=OFF -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=OFF -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All +echo cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=OFF -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All +cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=OFF -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All set MSBUILDDISABLENODEREUSE=1 @@ -82,8 +82,8 @@ echo Current directory : %cd% call:rest_env -echo cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=ON -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -DNOAVX_CORE_FILE=%dst_path%_noavx\python\paddle\fluid\core_noavx.pyd -cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=ON -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -DNOAVX_CORE_FILE=%dst_path%_noavx\python\paddle\fluid\core_noavx.pyd +echo cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=ON -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -DNOAVX_CORE_FILE=%dst_path%_noavx\python\paddle\fluid\core_noavx.pyd +cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DWITH_AVX=ON -DPYTHON_INCLUDE_DIR=%PYTHON_DIR%\include\ -DPYTHON_LIBRARY=%PYTHON_DIR%\libs\ -DPYTHON_EXECUTABLE=%PYTHON_DIR%\python.exe -DCMAKE_BUILD_TYPE=Release -DWITH_TESTING=OFF -DWITH_PYTHON=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -DNOAVX_CORE_FILE=%dst_path%_noavx\python\paddle\fluid\core_noavx.pyd set MSBUILDDISABLENODEREUSE=1 @@ -107,8 +107,8 @@ echo Current directory : %cd% call:rest_env -echo cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DCMAKE_BUILD_TYPE=Release -DWITH_PYTHON=OFF -DON_INFER=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All -cmake %dst_path%\..\Paddle -G "Visual Studio 14 2015 Win64" -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DCMAKE_BUILD_TYPE=Release -DWITH_PYTHON=OFF -DON_INFER=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All +echo cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DCMAKE_BUILD_TYPE=Release -DWITH_PYTHON=OFF -DON_INFER=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All +cmake %dst_path%\..\Paddle -G "Visual Studio 15 2017 Win64" -T host=x64 -DWITH_GPU=%WITH_GPU% -DWITH_MKL=%WITH_MKL% -DCMAKE_BUILD_TYPE=Release -DWITH_PYTHON=OFF -DON_INFER=ON -DCUDA_TOOLKIT_ROOT_DIR=%CUDA_DIR% -DCUDA_ARCH_NAME=All set MSBUILDDISABLENODEREUSE=1 diff --git a/paddle/scripts/windows_build/config.ini b/paddle/scripts/windows_build/config.ini index 32638d2873ca1..750d7af8c2926 100644 --- a/paddle/scripts/windows_build/config.ini +++ b/paddle/scripts/windows_build/config.ini @@ -11,7 +11,7 @@ http_proxy=#please edit your proxy# https_proxy=#please edit your proxy# # Just for example, please set by your windows environment -vcvarsall_dir="C:\Program Files (x86)\Microsoft Visual Studio 14.0\VC\vcvarsall.bat" +vcvarsall_dir="C:\Program Files (x86)\Microsoft Visual Studio\2017\Community\VC\Auxiliary\Build\vcvarsall.bat" PYTHON3_PATH=C:\Python37 CUDA_PATH="C:/Program Files/NVIDIA GPU Computing Toolkit/CUDA/v10.0" "C:/Program Files/NVIDIA GPU Computing Toolkit/CUDA/v9.0" diff --git a/patches/cryptopp/CMakeLists.txt b/patches/cryptopp/CMakeLists.txt new file mode 100644 index 0000000000000..c533b707350d6 --- /dev/null +++ b/patches/cryptopp/CMakeLists.txt @@ -0,0 +1,1239 @@ +# Please ensure your changes or patch meets minimum requirements. +# The minimum requirements are 2.8.6. It roughly equates to +# Ubuntu 14.05 LTS or Solaris 11.3. Please do not check in something +# for 3.5.0 or higher because it will break LTS operating systems +# and a number of developer boards used for testing. To test your +# changes, please set up a Ubuntu 14.05 LTS system. + +# Should we be setting things like this? We are not a C project +# so nothing should be done with the C compiler. But there is +# no reliable way to tell CMake we are C++. +# Cannot set this... Breaks Linux PowerPC with Clang: +# SET(CMAKE_C_COMPILER ${CMAKE_CXX_COMPILER}) +# # error "The CMAKE_C_COMPILER is set to a C++ compiler" + +if(NOT DEFINED cryptocpp_DISPLAY_CMAKE_SUPPORT_WARNING) + set(cryptocpp_DISPLAY_CMAKE_SUPPORT_WARNING 1) +endif() +if(cryptocpp_DISPLAY_CMAKE_SUPPORT_WARNING) + message( STATUS +"*************************************************************************\n" +"The Crypto++ library does not officially support CMake. CMake support is a\n" +"community effort, and the library works with the folks using CMake to help\n" +"improve it. If you find an issue then please fix it or report it at\n" +"https://github.com/noloader/cryptopp-cmake.\n" +"-- *************************************************************************" +) +endif() + +# Print useful information +message( STATUS "CMake version ${CMAKE_VERSION}" ) + +cmake_minimum_required(VERSION 2.8.6) +if (${CMAKE_VERSION} VERSION_LESS "3.0.0") + project(cryptopp) + set(cryptopp_VERSION_MAJOR 8) + set(cryptopp_VERSION_MINOR 2) + set(cryptopp_VERSION_PATCH 0) +else () + cmake_policy(SET CMP0048 NEW) + project(cryptopp VERSION 8.2.0) + if (NOT ${CMAKE_VERSION} VERSION_LESS "3.1.0") + cmake_policy(SET CMP0054 NEW) + endif () +endif () + +# Need to set SRC_DIR manually after removing the Python library code. +set(SRC_DIR ${CMAKE_CURRENT_SOURCE_DIR}) + +# Make RelWithDebInfo the default (it does e.g. add '-O2 -g -DNDEBUG' for GNU) +# If not in multi-configuration environments, no explicit build type or CXX +# flags are set by the user and if we are the root CMakeLists.txt file. +if (NOT CMAKE_CONFIGURATION_TYPES AND + NOT CMAKE_NO_BUILD_TYPE AND + NOT CMAKE_BUILD_TYPE AND + NOT CMAKE_CXX_FLAGS AND + CMAKE_SOURCE_DIR STREQUAL CMAKE_CURRENT_SOURCE_DIR) + set(CMAKE_BUILD_TYPE RelWithDebInfo) +endif () + +include(GNUInstallDirs) +include(CheckCXXCompilerFlag) + +# We now carry around test programs. test_cxx.cxx is the default C++ one. +# Also see https://github.com/weidai11/cryptopp/issues/741. +set(TEST_PROG_DIR ${SRC_DIR}/TestPrograms) +set(TEST_CXX_FILE ${TEST_PROG_DIR}/test_cxx.cxx) + +#============================================================================ +# Settable options +#============================================================================ + +option(BUILD_STATIC "Build static library" ON) +option(BUILD_SHARED "Build shared library" ON) +option(BUILD_TESTING "Build library tests" ON) +option(BUILD_DOCUMENTATION "Use Doxygen to create the HTML based API documentation" OFF) +option(USE_INTERMEDIATE_OBJECTS_TARGET "Use a common intermediate objects target for the static and shared library targets" ON) + +# These are IA-32 options. TODO: Add ARM A-32, Aarch64 and Power8 options. +option(DISABLE_ASM "Disable ASM" OFF) +option(DISABLE_SSSE3 "Disable SSSE3" OFF) +option(DISABLE_SSE4 "Disable SSE4" OFF) +option(DISABLE_AESNI "Disable AES-NI" OFF) +option(DISABLE_SHA "Disable SHA" OFF) +option(DISABLE_AVX "Disable AVX" OFF) +option(DISABLE_AVX2 "Disable AVX2" OFF) +option(CRYPTOPP_NATIVE_ARCH "Enable native architecture" OFF) +set(CRYPTOPP_DATA_DIR "" CACHE PATH "Crypto++ test data directory") + +#============================================================================ +# Compiler options +#============================================================================ + +set(CRYPTOPP_COMPILE_DEFINITIONS) +set(CRYPTOPP_COMPILE_OPTIONS) + +# Stop hiding the damn output... +# set(CMAKE_VERBOSE_MAKEFILE on) + +# Always 1 ahead in Master. Also see http://groups.google.com/forum/#!topic/cryptopp-users/SFhqLDTQPG4 +set(LIB_VER ${cryptopp_VERSION_MAJOR}${cryptopp_VERSION_MINOR}${cryptopp_VERSION_PATCH}) + +# Don't use RPATH's. The resulting binary could fail a security audit. +set(CMAKE_MACOSX_RPATH 0) + +if (CMAKE_CXX_COMPILER_ID STREQUAL "Intel") + list(APPEND CRYPTOPP_COMPILE_OPTIONS -wd68 -wd186 -wd279 -wd327 -wd161 -wd3180) +endif () + +# Also see http://github.com/weidai11/cryptopp/issues/395 +if (DISABLE_ASM) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_ASM) +endif () +if (DISABLE_SSSE3) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SSSE3) +endif () +if (DISABLE_SSE4) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SSSE4) +endif () +if (DISABLE_AESNI) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_AESNI) +endif () +if (DISABLE_SHA) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SHA) +endif () +if (DISABLE_ALTIVEC) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_ALTIVEC) +endif () +if (DISABLE_POWER7) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_POWER7) +endif () +if (DISABLE_POWER8) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_POWER8) +endif () +if (DISABLE_POWER9) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_POWER9) +endif () +if (NOT CRYPTOPP_DATA_DIR STREQUAL "") + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS "CRYPTOPP_DATA_DIR=${CRYPTOPP_DATA_DIR}") +endif () + +############################################################################### + +# Try to find a Posix compatible grep and sed. Solaris, Digital Unix, +# Tru64, HP-UX and a few others need tweaking + +if (EXISTS /usr/xpg4/bin/grep) + set(GREP_CMD /usr/xpg4/bin/grep) +elseif (EXISTS /usr/gnu/bin/grep) + set(GREP_CMD /usr/gnu/bin/grep) +elseif (EXISTS /usr/linux/bin/grep) + set(GREP_CMD /usr/linux/bin/grep) +else () + set(GREP_CMD grep) +endif () + +if (EXISTS /usr/xpg4/bin/sed) + set(SED_CMD /usr/xpg4/bin/sed) +elseif (EXISTS /usr/gnu/bin/sed) + set(SED_CMD /usr/gnu/bin/sed) +elseif (EXISTS /usr/linux/bin/sed) + set(SED_CMD /usr/linux/bin/sed) +else () + set(SED_CMD sed) +endif () + +############################################################################### + +function(CheckCompileOption opt var) + + if (MSVC) + + # TODO: improve this... + CHECK_CXX_COMPILER_FLAG(${opt} ${var}) + + elseif (CMAKE_CXX_COMPILER_ID MATCHES "SunPro") + + message(STATUS "Performing Test ${var}") + execute_process( + COMMAND sh -c "${CMAKE_CXX_COMPILER} ${CMAKE_CXX_FLAGS} ${opt} -c ${TEST_CXX_FILE} 2>&1" + COMMAND ${GREP_CMD} -i -c -E "illegal value ignored" + RESULT_VARIABLE COMMAND_RESULT + OUTPUT_VARIABLE COMMAND_OUTPUT + OUTPUT_STRIP_TRAILING_WHITESPACE) + + # No dereference below. Thanks for the warning, CMake (not!). + if (COMMAND_RESULT AND NOT COMMAND_OUTPUT) + set(${var} 1 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Success") + else () + set(${var} 0 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Failed") + endif () + + # Must use CMAKE_CXX_COMPILER here due to XLC 13.1 and LLVM front-end. + elseif (CMAKE_CXX_COMPILER MATCHES "xlC") + + message(STATUS "Performing Test ${var}") + execute_process( + COMMAND sh -c "${CMAKE_CXX_COMPILER} ${CMAKE_CXX_FLAGS} ${opt} -c ${TEST_CXX_FILE} 2>&1" + COMMAND ${GREP_CMD} -i -c -E "Unrecognized value" + RESULT_VARIABLE COMMAND_RESULT + OUTPUT_VARIABLE COMMAND_OUTPUT + OUTPUT_STRIP_TRAILING_WHITESPACE) + + # No dereference below. Thanks for the warning, CMake (not!). + if (COMMAND_RESULT AND NOT COMMAND_OUTPUT) + set(${var} 1 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Success") + else () + set(${var} 0 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Failed") + endif () + + else () + + CHECK_CXX_COMPILER_FLAG(${opt} ${var}) + + endif () + +endfunction(CheckCompileOption) + +function(CheckCompileLinkOption opt var prog) + + if (MSVC) + + # TODO: improve this... + CHECK_CXX_COMPILER_FLAG(${opt} ${var}) + + else () + + message(STATUS "Performing Test ${var}") + execute_process( + COMMAND sh -c "${CMAKE_CXX_COMPILER} ${CMAKE_CXX_FLAGS} ${opt} ${prog} 2>&1" + RESULT_VARIABLE COMMAND_RESULT + OUTPUT_VARIABLE COMMAND_OUTPUT + OUTPUT_STRIP_TRAILING_WHITESPACE) + + # message(STATUS "RESULT_VARIABLE ${RESULT_VARIABLE}") + # message(STATUS "COMMAND_RESULT ${COMMAND_RESULT}") + # message(STATUS "OUTPUT_VARIABLE ${OUTPUT_VARIABLE}") + # message(STATUS "COMMAND_OUTPUT ${COMMAND_OUTPUT}") + + # This test is strict. We require two things. First, the invocation + # of the compile command must return 0. Second, there must be no + # messages on the console. We are interested in diagnostics like + # warnings to decide when to reject an option. But we will probably + # capture chatty compiler that want to say, "Hooray, success". For + # chatty compilers we will need to find a quiet option and use it + # for the test. Microsoft compilers come to mind. + if ("${COMMAND_RESULT}" EQUAL 0 AND "${COMMAND_OUTPUT}" STREQUAL "") + set(${var} 1 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Success") + else () + set(${var} 0 PARENT_SCOPE) + message(STATUS "Performing Test ${var} - Failed") + endif () + + endif () + +endfunction(CheckCompileLinkOption) + +function(AddCompileOption opt) + + if ("${COMMAND_OUTPUT}" NOT STREQUAL "") + list(APPEND CRYPTOPP_COMPILE_OPTIONS "${opt}") + endif () + +endfunction(AddCompileOption) + +############################################################################### + +function(DumpMachine output pattern) + + if (MSVC) + + # CMake does not provide a generic shell/terminal mechanism + # and Microsoft environments don't know what 'sh' is. + set(${output} 0 PARENT_SCOPE) + + else () + + execute_process( + COMMAND sh -c "${CMAKE_CXX_COMPILER} -dumpmachine 2>&1" + COMMAND ${GREP_CMD} -i -c -E "${pattern}" + OUTPUT_VARIABLE ${output} + OUTPUT_STRIP_TRAILING_WHITESPACE) + set(${output} "${${output}}" PARENT_SCOPE) + + endif() + +endfunction(DumpMachine) + +# Thansk to Anonimal for MinGW; see http://github.com/weidai11/cryptopp/issues/466 +DumpMachine(CRYPTOPP_AMD64 "amd64|x86_64") +DumpMachine(CRYPTOPP_I386 "i.86") +DumpMachine(CRYPTOPP_MINGW32 "\\") +DumpMachine(CRYPTOPP_MINGW64 "w64-mingw32|mingw64") +DumpMachine(CRYPTOPP_X32 "x32") +DumpMachine(CRYPTOPP_AARCH32 "Aarch32") +DumpMachine(CRYPTOPP_AARCH64 "Aarch64") +DumpMachine(CRYPTOPP_ARMHF "armhf|arm7l|eabihf") +DumpMachine(CRYPTOPP_ARM "\\") + +# Detecting PowerPC is only good with GCC. IBM XLC compiler is +# a little different and I don't know how to ask to the triplet +# XLC is targeting. Below we punt by setting CRYPTOPP_POWERPC64 +# if we detect the compiler is XLC. +DumpMachine(CRYPTOPP_POWERPC "ppc|powerpc") +DumpMachine(CRYPTOPP_POWERPC64 "ppc64") + +############################################################################### + +# Test SunCC for a string like 'CC: Sun C++ 5.13 SunOS_i386' +if (NOT CRYPTOPP_SOLARIS) + execute_process(COMMAND sh -c "${CMAKE_CXX_COMPILER} -V 2>&1" + COMMAND ${GREP_CMD} -i -c "SunOS" + OUTPUT_VARIABLE CRYPTOPP_SOLARIS + OUTPUT_STRIP_TRAILING_WHITESPACE) +endif () + +# Test GCC for a string like 'i386-pc-solaris2.11' +if (NOT CRYPTOPP_SOLARIS) + execute_process(COMMAND sh -c "${CMAKE_CXX_COMPILER} -dumpmachine 2>&1" + COMMAND ${GREP_CMD} -i -c "Solaris" + OUTPUT_VARIABLE CRYPTOPP_SOLARIS + OUTPUT_STRIP_TRAILING_WHITESPACE) +endif () + +# Fixup PowerPC. If both 32-bit and 64-bit use 64-bit. +if (CRYPTOPP_POWERPC AND CRYPTOPP_POWERPC64) + unset(CRYPTOPP_POWERPC) +endif () + +# Fixup for xlC compiler. -dumpmachine fails so we miss PowerPC +# TODO: something better than proxying the platform via compiler +# Must use CMAKE_CXX_COMPILER here due to XLC 13.1 and LLVM front-end. +if (CMAKE_CXX_COMPILER MATCHES "xlC") + message ("-- Fixing platform due to IBM xlC") + set(CRYPTOPP_POWERPC64 1) +endif () + +# DumpMachine SunCC style +if (CMAKE_CXX_COMPILER_ID STREQUAL "SunPro") + + # SunCC is 32-bit, but it builds both 32 and 64 bit. Use + execute_process(COMMAND sh -c "${CMAKE_CXX_COMPILER} -V 2>&1" + COMMAND ${GREP_CMD} -i -c "Sparc" + OUTPUT_VARIABLE CRYPTOPP_SPARC + OUTPUT_STRIP_TRAILING_WHITESPACE) + + execute_process(COMMAND sh -c "${CMAKE_CXX_COMPILER} -V 2>&1" + COMMAND ${GREP_CMD} -i -c -E "i386|i86" + OUTPUT_VARIABLE CRYPTOPP_I386 + OUTPUT_STRIP_TRAILING_WHITESPACE) + + execute_process(COMMAND isainfo -k + COMMAND ${GREP_CMD} -i -c "i386" + OUTPUT_VARIABLE KERNEL_I386 + OUTPUT_STRIP_TRAILING_WHITESPACE) + + execute_process(COMMAND isainfo -k + COMMAND ${GREP_CMD} -i -c "amd64" + OUTPUT_VARIABLE KERNEL_AMD64 + OUTPUT_STRIP_TRAILING_WHITESPACE) + + execute_process(COMMAND isainfo -k + COMMAND ${GREP_CMD} -i -c "Sparc" + OUTPUT_VARIABLE KERNEL_SPARC + OUTPUT_STRIP_TRAILING_WHITESPACE) + + execute_process(COMMAND isainfo -k + COMMAND ${GREP_CMD} -i -c -E "UltraSarc|Sparc64|SparcV9" + OUTPUT_VARIABLE KERNEL_SPARC64 + OUTPUT_STRIP_TRAILING_WHITESPACE) + +endif () + +############################################################################### + +if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + + execute_process(COMMAND sh -c "${CMAKE_CXX_COMPILER} --version 2>&1" + COMMAND ${GREP_CMD} -i -c "macports" + OUTPUT_VARIABLE MACPORTS + OUTPUT_STRIP_TRAILING_WHITESPACE) + + if (MACPORTS EQUAL 0) + # Get GAS version, add defs + set as appropriate + set(GAS_CMD sh -c "${CMAKE_CXX_COMPILER} -xc -c /dev/null -Wa,-v -o/dev/null 2>&1") + + execute_process(COMMAND ${GAS_CMD} + OUTPUT_VARIABLE GAS_STRING + OUTPUT_STRIP_TRAILING_WHITESPACE) + string(FIND "${GAS_STRING}" "GNU assembler" GAS_OUTPUT) + + if (NOT GAS_OUTPUT EQUAL -1) + #.intel_syntax wasn't supported until GNU assembler 2.10 + + # TODO(unassigned): string() REGEX was not cooperating at time of writing. Re-implement as needed. + execute_process(COMMAND echo ${GAS_STRING} + COMMAND ${GREP_CMD} -i -c -E "GNU.[Aa]ssembler.*(2\\.[1-9][0-9]|[3-9])" + OUTPUT_VARIABLE GAS210_OR_LATER) + if (GAS210_OR_LATER EQUAL 0) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_ASM) + set(DISABLE_ASM 1) + endif () + + execute_process(COMMAND echo ${GAS_STRING} + COMMAND ${GREP_CMD} -i -c -E "GNU.[Aa]ssembler.*(2\\.1[7-9]|2\\.[2-9]|[3-9])" + OUTPUT_VARIABLE GAS217_OR_LATER) + if (GAS217_OR_LATER EQUAL 0) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SSSE3) + set(DISABLE_SSSE3 1) + endif () + + # OpenBSD and CentOS 5 needed this one due to ARIA and BLAKE2 + execute_process(COMMAND echo ${GAS_STRING} + COMMAND ${GREP_CMD} -i -c -E "GNU.[Aa]ssembler.*(2\\.1[8-9]|2\\.[2-9]|[3-9])" + OUTPUT_VARIABLE GAS218_OR_LATER) + if (GAS218_OR_LATER EQUAL 0) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SSSE4) + set(DISABLE_SSE4 1) + endif () + + execute_process(COMMAND echo ${GAS_STRING} + COMMAND ${GREP_CMD} -i -c -E "GNU.[Aa]ssembler.*(2\\.19|2\\.[2-9]|[3-9])" + OUTPUT_VARIABLE GAS219_OR_LATER) + if (GAS219_OR_LATER EQUAL 0) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_AESNI) + set(DISABLE_AESNI 1) + endif () + + # Ubuntu 10 and Ubuntu 12 needed this one + execute_process(COMMAND echo ${GAS_STRING} + COMMAND ${GREP_CMD} -i -c -E "GNU.[Aa]ssembler.*(2\\.2[3-9]|2\\.[3-9]|[3-9])" + OUTPUT_VARIABLE GAS223_OR_LATER) + if (GAS223_OR_LATER EQUAL 0) + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS CRYPTOPP_DISABLE_SHA) + set(DISABLE_SHA 1) + endif () + endif () + endif () +endif () + +# TODO: what about ICC and LLVM on Windows? +if (MSVC) + if (CMAKE_SYSTEM_VERSION MATCHES "10\\.0.*") + list(APPEND CRYPTOPP_COMPILE_DEFINITIONS "_WIN32_WINNT=0x0A00") + endif () + list(APPEND CRYPTOPP_COMPILE_OPTIONS "/FIwinapifamily.h") +endif () + +# Enable PIC for all target machines except 32-bit i386 due to register pressures. +if (NOT CRYPTOPP_I386) + SET(CMAKE_POSITION_INDEPENDENT_CODE 1) +endif () + +# IBM XLC compiler options for AIX and Linux. +# Must use CMAKE_CXX_COMPILER here due to XLC 13.1 and LLVM front-end. +if (CMAKE_CXX_COMPILER MATCHES "xlC") + + #CheckCompileLinkOption("-qxlcompatmacros" CRYPTOPP_XLC_COMPAT "${TEST_CXX_FILE}") + #if (CRYPTOPP_XLC_COMPAT) + # list(APPEND CRYPTOPP_COMPILE_OPTIONS "-qxlcompatmacros") + #endif () + + CheckCompileLinkOption("-qrtti" CRYPTOPP_PPC_RTTI "${TEST_CXX_FILE}") + if (CRYPTOPP_PPC_RTTI) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-qrtti") + endif () + + CheckCompileLinkOption("-qmaxmem=-1" CRYPTOPP_PPC_MAXMEM "${TEST_CXX_FILE}") + if (CRYPTOPP_PPC_MAXMEM) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-qmaxmem=-1") + endif () + + CheckCompileLinkOption("-qthreaded" CRYPTOPP_PPC_THREADED "${TEST_CXX_FILE}") + if (CRYPTOPP_PPC_THREADED) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-qthreaded") + endif () +endif () + +# Solaris specific +if (CRYPTOPP_SOLARIS) + + # SunCC needs -template=no%extdef + if (CMAKE_CXX_COMPILER_ID STREQUAL "SunPro") + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-template=no%extdef") + endif () + + # SunCC needs -xregs=no%appl on Sparc (not x86) for libraries (not test program) + # TODO: wire this up properly + if (CMAKE_CXX_COMPILER_ID STREQUAL "SunPro" AND (CRYPTOPP_SPARC OR CRYPTOPP_SPARC64)) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-xregs=no%appl") + endif () + + # GCC needs to enable use of '/' for division in the assembler + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-Wa,--divide") + endif () + +endif () + +#============================================================================ +# Sources & headers +#============================================================================ + +# Library headers +file(GLOB cryptopp_HEADERS ${SRC_DIR}/*.h) + +# Remove headers used to build test suite +list(REMOVE_ITEM cryptopp_HEADERS + ${SRC_DIR}/bench.h + ${SRC_DIR}/validate.h + ) + +# Test sources. You can use the GNUmakefile to generate the list: `make sources`. +set(cryptopp_SOURCES_TEST + ${SRC_DIR}/test.cpp + ${SRC_DIR}/bench1.cpp + ${SRC_DIR}/bench2.cpp + ${SRC_DIR}/bench3.cpp + ${SRC_DIR}/validat0.cpp + ${SRC_DIR}/validat1.cpp + ${SRC_DIR}/validat2.cpp + ${SRC_DIR}/validat3.cpp + ${SRC_DIR}/validat4.cpp + ${SRC_DIR}/validat5.cpp + ${SRC_DIR}/validat6.cpp + ${SRC_DIR}/validat7.cpp + ${SRC_DIR}/validat8.cpp + ${SRC_DIR}/validat9.cpp + ${SRC_DIR}/validat10.cpp + ${SRC_DIR}/regtest1.cpp + ${SRC_DIR}/regtest2.cpp + ${SRC_DIR}/regtest3.cpp + ${SRC_DIR}/regtest4.cpp + ${SRC_DIR}/datatest.cpp + ${SRC_DIR}/fipsalgt.cpp + ${SRC_DIR}/fipstest.cpp + ${SRC_DIR}/dlltest.cpp + #${SRC_DIR}/adhoc.cpp + ) + +# Library sources. You can use the GNUmakefile to generate the list: `make sources`. +# Makefile sorted them at http://github.com/weidai11/cryptopp/pull/426. +file(GLOB cryptopp_SOURCES ${SRC_DIR}/*.cpp) +list(SORT cryptopp_SOURCES) +list(REMOVE_ITEM cryptopp_SOURCES + ${SRC_DIR}/cryptlib.cpp + ${SRC_DIR}/cpu.cpp + ${SRC_DIR}/integer.cpp + ${SRC_DIR}/pch.cpp + ${SRC_DIR}/simple.cpp + ${SRC_DIR}/adhoc.cpp + ${cryptopp_SOURCES_TEST} + ) +set(cryptopp_SOURCES + ${SRC_DIR}/cryptlib.cpp + ${SRC_DIR}/cpu.cpp + ${SRC_DIR}/integer.cpp + ${cryptopp_SOURCES} + ) + +set(cryptopp_SOURCES_ASM) + +if (MSVC AND NOT DISABLE_ASM) + if (${CMAKE_GENERATOR} MATCHES ".*ARM") + message(STATUS "Disabling ASM because ARM is specified as target platform.") + else () + enable_language(ASM_MASM) + list(APPEND cryptopp_SOURCES_ASM + ${SRC_DIR}/rdrand.asm + ) + if (CMAKE_SIZEOF_VOID_P EQUAL 8) + list(APPEND cryptopp_SOURCES_ASM + ${SRC_DIR}/x64dll.asm + ${SRC_DIR}/x64masm.asm + ) + set_source_files_properties(${cryptopp_SOURCES_ASM} PROPERTIES COMPILE_DEFINITIONS "_M_X64") + else () + set_source_files_properties(${cryptopp_SOURCES_ASM} PROPERTIES COMPILE_DEFINITIONS "_M_X86" COMPILE_FLAGS "/safeseh") + endif () + set_source_files_properties(${cryptopp_SOURCES_ASM} PROPERTIES LANGUAGE ASM_MASM) + endif () +endif () + +#============================================================================ +# Architecture flags +#============================================================================ + +# TODO: Android, AIX, IBM xlC, iOS and a few other profiles are missing. + +# New as of Pull Request 461, http://github.com/weidai11/cryptopp/pull/461. +# Must use CMAKE_CXX_COMPILER here due to XLC 13.1 and LLVM front-end. +if (CMAKE_CXX_COMPILER_ID MATCHES "Clang" OR CMAKE_CXX_COMPILER_ID STREQUAL "GNU" OR CMAKE_CXX_COMPILER_ID STREQUAL "Intel" OR CMAKE_CXX_COMPILER MATCHES "xlC") + + if (CRYPTOPP_AMD64 OR CRYPTOPP_I386 OR CRYPTOPP_X32) + + CheckCompileLinkOption("-msse2" CRYPTOPP_IA32_SSE2 + "${TEST_PROG_DIR}/test_x86_sse2.cxx") + CheckCompileLinkOption("-mssse3" CRYPTOPP_IA32_SSSE3 + "${TEST_PROG_DIR}/test_x86_ssse3.cxx") + CheckCompileLinkOption("-msse4.1" CRYPTOPP_IA32_SSE41 + "${TEST_PROG_DIR}/test_x86_sse41.cxx") + CheckCompileLinkOption("-msse4.2" CRYPTOPP_IA32_SSE42 + "${TEST_PROG_DIR}/test_x86_sse42.cxx") + CheckCompileLinkOption("-mssse3 -mpclmul" CRYPTOPP_IA32_CLMUL + "${TEST_PROG_DIR}/test_x86_clmul.cxx") + CheckCompileLinkOption("-msse4.1 -maes" CRYPTOPP_IA32_AES + "${TEST_PROG_DIR}/test_x86_aes.cxx") + CheckCompileLinkOption("-mavx" CRYPTOPP_IA32_AVX + "${TEST_PROG_DIR}/test_x86_avx.cxx") + CheckCompileLinkOption("-mavx2" CRYPTOPP_IA32_AVX2 + "${TEST_PROG_DIR}/test_x86_avx2.cxx") + CheckCompileLinkOption("-msse4.2 -msha" CRYPTOPP_IA32_SHA + "${TEST_PROG_DIR}/test_x86_sha.cxx") + CheckCompileLinkOption("" CRYPTOPP_MIXED_ASM + "${TEST_PROG_DIR}/test_mixed_asm.cxx") + + # https://github.com/weidai11/cryptopp/issues/756 + if (NOT CRYPTOPP_MIXED_ASM) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_MIXED_ASM") + endif () + + if (NOT CRYPTOPP_IA32_SSE2 AND NOT DISABLE_ASM) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_ASM") + elseif (CRYPTOPP_IA32_SSE2 AND NOT DISABLE_ASM) + set_source_files_properties(${SRC_DIR}/sse_simd.cpp PROPERTIES COMPILE_FLAGS "-msse2") + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS "-msse2") + set_source_files_properties(${SRC_DIR}/donna_sse.cpp PROPERTIES COMPILE_FLAGS "-msse2") + endif () + if (NOT CRYPTOPP_IA32_SSSE3 AND NOT DISABLE_SSSE3) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_SSSE3") + elseif (CRYPTOPP_IA32_SSSE3 AND NOT DISABLE_SSSE3) + set_source_files_properties(${SRC_DIR}/aria_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/cham_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/keccak_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/lea_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/simeck_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/simon128_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + set_source_files_properties(${SRC_DIR}/speck128_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3") + if (NOT CRYPTOPP_IA32_SSE41 AND NOT DISABLE_SSE4) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_SSE4") + elseif (CRYPTOPP_IA32_SSE41 AND NOT DISABLE_SSE4) + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.1") + set_source_files_properties(${SRC_DIR}/blake2b_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.1") + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.1") + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.1") + endif () + if (NOT CRYPTOPP_IA32_SSE42 AND NOT DISABLE_SSE4) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_SSE4") + elseif (CRYPTOPP_IA32_SSE42 AND NOT DISABLE_SSE4) + set_source_files_properties(${SRC_DIR}/crc_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.2") + if (NOT CRYPTOPP_IA32_CLMUL AND NOT DISABLE_AES) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_CLMUL") + elseif (CRYPTOPP_IA32_CLMUL AND NOT DISABLE_AES) + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3 -mpclmul") + set_source_files_properties(${SRC_DIR}/gf2n_simd.cpp PROPERTIES COMPILE_FLAGS "-mpclmul") + endif () + if (NOT CRYPTOPP_IA32_AES AND NOT DISABLE_AES) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_AESNI") + elseif (CRYPTOPP_IA32_AES AND NOT DISABLE_AES) + set_source_files_properties(${SRC_DIR}/rijndael_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.1 -maes") + set_source_files_properties(${SRC_DIR}/sm4_simd.cpp PROPERTIES COMPILE_FLAGS "-mssse3 -maes") + endif () + #if (NOT CRYPTOPP_IA32_AVX AND NOT DISABLE_AVX) + # list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_AVX") + #elseif (CRYPTOPP_IA32_AVX AND NOT DISABLE_AVX) + # set_source_files_properties(${SRC_DIR}/XXX_avx.cpp PROPERTIES COMPILE_FLAGS "-mavx") + #endif () + if (NOT CRYPTOPP_IA32_AVX2 AND NOT DISABLE_AVX2) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_AVX2") + elseif (CRYPTOPP_IA32_AVX2 AND NOT DISABLE_AVX2) + set_source_files_properties(${SRC_DIR}/chacha_avx.cpp PROPERTIES COMPILE_FLAGS "-mavx2") + endif () + if (NOT CRYPTOPP_IA32_SHA AND NOT DISABLE_SHA) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_SHANI") + elseif (CRYPTOPP_IA32_SHA AND NOT DISABLE_SHA) + set_source_files_properties(${SRC_DIR}/sha_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.2 -msha") + set_source_files_properties(${SRC_DIR}/shacal2_simd.cpp PROPERTIES COMPILE_FLAGS "-msse4.2 -msha") + endif () + endif () + endif () + + elseif (CRYPTOPP_AARCH32 OR CRYPTOPP_AARCH64) + + CheckCompileOption("-march=armv8-a" CRYPTOPP_ARMV8A_ASIMD) + CheckCompileOption("-march=armv8-a+crc" CRYPTOPP_ARMV8A_CRC) + CheckCompileOption("-march=armv8-a+crypto" CRYPTOPP_ARMV8A_CRYPTO) + CheckCompileOption("-march=armv8-a" CRYPTOPP_ARMV8A_NATIVE) + + if (CRYPTOPP_ARMV8A_ASIMD) + set_source_files_properties(${SRC_DIR}/aria_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/blake2b_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/cham_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/lea_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/neon_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/simeck_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/simon128_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + set_source_files_properties(${SRC_DIR}/speck128_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a") + endif () + if (CRYPTOPP_ARMV8A_CRC) + set_source_files_properties(${SRC_DIR}/crc_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crc") + endif () + if (CRYPTOPP_ARMV8A_CRYPTO) + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crypto") + set_source_files_properties(${SRC_DIR}/gf2n_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crypto") + set_source_files_properties(${SRC_DIR}/rijndael_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crypto") + set_source_files_properties(${SRC_DIR}/sha_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crypto") + set_source_files_properties(${SRC_DIR}/shacal2_simd.cpp PROPERTIES COMPILE_FLAGS "-march=armv8-a+crypto") + endif () + + elseif (CRYPTOPP_ARM OR CRYPTOPP_ARMHF) + + # Need to set floating point ABI to something, like "hard" of "softfp". + # Most Linux use hard floats. + CheckCompileLinkOption("-march=armv7-a -mfpu=neon" CRYPTOPP_ARMV7A_NEON + "${TEST_PROG_DIR}/test_arm_neon.cxx") + CheckCompileLinkOption("-march=armv7-a -mfloat-abi=hard -mfpu=neon" CRYPTOPP_ARMV7A_HARD + "${TEST_PROG_DIR}/test_arm_neon.cxx") + CheckCompileLinkOption("-march=armv7-a -mfloat-abi=softfp -mfpu=neon" CRYPTOPP_ARMV7A_SOFTFP + "${TEST_PROG_DIR}/test_arm_neon.cxx") + + if (CRYPTOPP_ARMV7A_HARD) + set(CRYPTOPP_ARMV7A_FLAGS "-march=armv7-a -mfloat-abi=hard -mfpu=neon") + elseif (CRYPTOPP_ARMV7A_SOFTFP) + set(CRYPTOPP_ARMV7A_FLAGS "-march=armv7-a -mfloat-abi=softfp -mfpu=neon") + else () + AddCompileOption("-DCRYPTOPP_DISABLE_NEON") + endif() + + if (CRYPTOPP_ARMV7A_HARD OR CRYPTOPP_ARMV7A_SOFTFP) + # Add ASM files for ARM + if (NOT MSVC) + list(APPEND cryptopp_SOURCES ${SRC_DIR}/aes_armv4.S) + set_source_files_properties(${SRC_DIR}/aes_armv4.S PROPERTIES LANGUAGE C) + endif () + + set_source_files_properties(${SRC_DIR}/aes_armv4.S PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/aria_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/blake2b_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/cham_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/crc_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/lea_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/rijndael_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/neon_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/sha_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/simeck_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/simon128_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/speck128_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + set_source_files_properties(${SRC_DIR}/sm4_simd.cpp PROPERTIES COMPILE_FLAGS "${CRYPTOPP_ARMV7A_FLAGS}") + endif () + + elseif (CRYPTOPP_POWERPC OR CRYPTOPP_POWERPC64) + + if (CMAKE_CXX_COMPILER MATCHES "xlC") + set(CRYPTOPP_ALTIVEC_FLAGS "-qaltivec") + set(CRYPTOPP_POWER4_FLAGS "-qarch=pwr4 -qaltivec") + set(CRYPTOPP_POWER5_FLAGS "-qarch=pwr5 -qaltivec") + set(CRYPTOPP_POWER6_FLAGS "-qarch=pwr6 -qaltivec") + set(CRYPTOPP_POWER7_FLAGS "-qarch=pwr7 -qaltivec") + set(CRYPTOPP_POWER8_FLAGS "-qarch=pwr8 -qaltivec") + set(CRYPTOPP_POWER9_FLAGS "-qarch=pwr9 -qaltivec") + else () + set(CRYPTOPP_ALTIVEC_FLAGS "-maltivec") + set(CRYPTOPP_POWER7_FLAGS "-mcpu=power7 -maltivec") + set(CRYPTOPP_POWER8_FLAGS "-mcpu=power8 -maltivec") + set(CRYPTOPP_POWER9_FLAGS "-mcpu=power9 -maltivec") + endif () + + CheckCompileLinkOption("${CRYPTOPP_ALTIVEC_FLAGS}" PPC_ALTIVEC_FLAG + "${TEST_PROG_DIR}/test_ppc_altivec.cxx") + + # Hack for XLC + if (CMAKE_CXX_COMPILER MATCHES "xlC") + if (NOT PPC_ALTIVEC_FLAG) + CheckCompileLinkOption("${CRYPTOPP_POWER4_FLAGS}" PPC_POWER4_FLAG + "${TEST_PROG_DIR}/test_ppc_altivec.cxx") + if (PPC_POWER4_FLAG) + set(PPC_ALTIVEC_FLAG 1) + set(CRYPTOPP_ALTIVEC_FLAGS "${CRYPTOPP_POWER4_FLAGS}") + endif () + endif () + if (NOT PPC_ALTIVEC_FLAG) + CheckCompileLinkOption("${CRYPTOPP_POWER5_FLAGS}" PPC_POWER5_FLAG + "${TEST_PROG_DIR}/test_ppc_altivec.cxx") + if (PPC_POWER5_FLAG) + set(PPC_ALTIVEC_FLAG 1) + set(CRYPTOPP_ALTIVEC_FLAGS "${CRYPTOPP_POWER5_FLAGS}") + endif () + endif () + if (NOT PPC_ALTIVEC_FLAG) + CheckCompileLinkOption("${CRYPTOPP_POWER6_FLAGS}" PPC_POWER6_FLAG + "${TEST_PROG_DIR}/test_ppc_altivec.cxx") + if (PPC_POWER6_FLAG) + set(PPC_ALTIVEC_FLAG 1) + set(CRYPTOPP_ALTIVEC_FLAGS "${CRYPTOPP_POWER6_FLAGS}") + endif () + endif () + endif () + + CheckCompileLinkOption("${CRYPTOPP_POWER7_FLAGS}" PPC_POWER7_FLAG + "${TEST_PROG_DIR}/test_ppc_power7.cxx") + + CheckCompileLinkOption("${CRYPTOPP_POWER8_FLAGS}" PPC_POWER8_FLAG + "${TEST_PROG_DIR}/test_ppc_power8.cxx") + + CheckCompileLinkOption("${CRYPTOPP_POWER9_FLAGS}" PPC_POWER9_FLAG + "${TEST_PROG_DIR}/test_ppc_power9.cxx") + + if (PPC_POWER9_FLAG AND NOT DISABLE_POWER9) + set_source_files_properties(${SRC_DIR}/ppc_power9.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER9_FLAGS}) + endif () + + if (PPC_POWER8_FLAG AND NOT DISABLE_POWER8) + set_source_files_properties(${SRC_DIR}/ppc_power8.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/blake2b_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + #set_source_files_properties(${SRC_DIR}/crc_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/gf2n_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/rijndael_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/sha_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/shacal2_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/simon128_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + set_source_files_properties(${SRC_DIR}/speck128_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER8_FLAGS}) + endif () + + if (PPC_POWER7_FLAG AND NOT DISABLE_POWER7) + set_source_files_properties(${SRC_DIR}/ppc_power7.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/aria_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/cham_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/lea_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/simeck_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + endif () + + if (PPC_ALTIVEC_FLAG AND NOT DISABLE_ALTIVEC) + set_source_files_properties(${SRC_DIR}/ppc_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_ALTIVEC_FLAGS}) + endif () + + # Drop to Power7 if Power8 unavailable + if (NOT PPC_POWER8_FLAG) + if (PPC_POWER7_FLAG) + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_POWER7_FLAGS}) + endif () + endif () + + # Drop to Altivec if Power7 unavailable + if (NOT PPC_POWER7_FLAG) + if (PPC_ALTIVEC_FLAG) + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_ALTIVEC_FLAGS}) + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_ALTIVEC_FLAGS}) + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_ALTIVEC_FLAGS}) + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS ${CRYPTOPP_ALTIVEC_FLAGS}) + endif () + endif () + + if (NOT PPC_ALTIVEC_FLAG) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_ALTIVEC") + elseif (NOT PPC_POWER7_FLAG) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_POWER7") + elseif (NOT PPC_POWER8_FLAG) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_POWER8") + elseif (NOT PPC_POWER9_FLAG) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-DCRYPTOPP_DISABLE_POWER9") + endif () + + endif () +endif () + +# New as of Pull Request 461, http://github.com/weidai11/cryptopp/pull/461. +if (CMAKE_CXX_COMPILER_ID STREQUAL "SunPro") + + if (CRYPTOPP_AMD64 OR CRYPTOPP_I386 OR CRYPTOPP_X32) + + CheckCompileLinkOption("-xarch=sse2" CRYPTOPP_IA32_SSE2 + "${TEST_PROG_DIR}/test_x86_sse2.cxx") + CheckCompileLinkOption("-xarch=ssse3" CRYPTOPP_IA32_SSSE3 + "${TEST_PROG_DIR}/test_x86_ssse3.cxx") + CheckCompileLinkOption("-xarch=sse4_1" CRYPTOPP_IA32_SSE41 + "${TEST_PROG_DIR}/test_x86_sse41.cxx") + CheckCompileLinkOption("-xarch=sse4_2" CRYPTOPP_IA32_SSE42 + "${TEST_PROG_DIR}/test_x86_sse42.cxx") + CheckCompileLinkOption("-xarch=aes" CRYPTOPP_IA32_CLMUL + "${TEST_PROG_DIR}/test_x86_clmul.cxx") + CheckCompileLinkOption("-xarch=aes" CRYPTOPP_IA32_AES + "${TEST_PROG_DIR}/test_x86_aes.cxx") + CheckCompileLinkOption("-xarch=avx" CRYPTOPP_IA32_AVX + "${TEST_PROG_DIR}/test_x86_avx.cxx") + CheckCompileLinkOption("-xarch=avx2" CRYPTOPP_IA32_AVX2 + "${TEST_PROG_DIR}/test_x86_avx2.cxx") + CheckCompileLinkOption("-xarch=sha" CRYPTOPP_IA32_SHA + "${TEST_PROG_DIR}/test_x86_sha.cxx") + + # Each -xarch=XXX options must be added to LDFLAGS if the option is used during a compile. + set(XARCH_LDFLAGS "") + + if (CRYPTOPP_IA32_SSE2 AND NOT DISABLE_ASM) + set_source_files_properties(${SRC_DIR}/sse_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse2") + set_source_files_properties(${SRC_DIR}/chacha_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse2") + set(XARCH_LDFLAGS "-xarch=sse2") + endif () + if (CRYPTOPP_IA32_SSSE3 AND NOT DISABLE_SSSE3) + set_source_files_properties(${SRC_DIR}/aria_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set_source_files_properties(${SRC_DIR}/cham_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set_source_files_properties(${SRC_DIR}/lea_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set_source_files_properties(${SRC_DIR}/simeck_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set_source_files_properties(${SRC_DIR}/simon128_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set_source_files_properties(${SRC_DIR}/speck128_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=ssse3") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=ssse3") + if (CRYPTOPP_IA32_SSE41 AND NOT DISABLE_SSE4) + set_source_files_properties(${SRC_DIR}/blake2s_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse4_1") + set_source_files_properties(${SRC_DIR}/blake2b_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse4_1") + set_source_files_properties(${SRC_DIR}/simon64_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse4_1") + set_source_files_properties(${SRC_DIR}/speck64_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse4_1") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=sse4_1") + endif () + if (CRYPTOPP_IA32_SSE42 AND NOT DISABLE_SSE4) + set_source_files_properties(${SRC_DIR}/crc_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sse4_2") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=sse4_2") + if (CRYPTOPP_IA32_CLMUL AND NOT DISABLE_CLMUL) + set_source_files_properties(${SRC_DIR}/gcm_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=aes") + set_source_files_properties(${SRC_DIR}/gf2n_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=aes") + endif () + if (CRYPTOPP_IA32_AES AND NOT DISABLE_AES) + set_source_files_properties(${SRC_DIR}/rijndael_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=aes") + set_source_files_properties(${SRC_DIR}/sm4_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=aes") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=aes") + endif () + #if (CRYPTOPP_IA32_AVX AND NOT DISABLE_AVX) + # set_source_files_properties(${SRC_DIR}/XXX_avx.cpp PROPERTIES COMPILE_FLAGS "-xarch=avx2") + # set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=avx") + #endif () + if (CRYPTOPP_IA32_AVX2 AND NOT DISABLE_AVX2) + set_source_files_properties(${SRC_DIR}/chacha_avx.cpp PROPERTIES COMPILE_FLAGS "-xarch=avx2") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=avx2") + endif () + if (CRYPTOPP_IA32_SHA AND NOT DISABLE_SHA) + set_source_files_properties(${SRC_DIR}/sha_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sha") + set_source_files_properties(${SRC_DIR}/shacal2_simd.cpp PROPERTIES COMPILE_FLAGS "-xarch=sha") + set(XARCH_LDFLAGS "${XARCH_LDFLAGS} -xarch=sha") + endif () + endif () + endif () + + # https://stackoverflow.com/a/6088646/608639 + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${XARCH_LDFLAGS} -M${SRC_DIR}/cryptopp.mapfile") + set(CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} ${XARCH_LDFLAGS} -M${SRC_DIR}/cryptopp.mapfile") + set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${XARCH_LDFLAGS} -M${SRC_DIR}/cryptopp.mapfile") + + # elseif (CRYPTOPP_SPARC OR CRYPTOPP_SPARC64) + + endif () +endif () + +# Attempt to determine a suitable native option +if (CRYPTOPP_NATIVE_ARCH) + + CheckCompileOption("-march=native" NATIVE_ARCH) + if (NATIVE_ARCH) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-march=native") + else () + CheckCompileOption("-native" NATIVE_ARCH) + if (NATIVE_ARCH) + list(APPEND CRYPTOPP_COMPILE_OPTIONS "-native") + endif () + endif () + + if (NOT NATIVE_ARCH) + message(WARNING "CRYPTOPP_NATIVE_ARCH enabled, but failed to detect native architecture option") + endif () + +endif() + +#============================================================================ +# Compile targets +#============================================================================ + +# Work around the archaic versions of cmake that do not support +# target_compile_xxxx commands +# !!! DO NOT try to use the old way for newer version - it does not work !!! +function(cryptopp_target_compile_properties target) + if (NOT ${CMAKE_VERSION} VERSION_LESS "2.8.11") + target_compile_definitions(${target} PUBLIC ${CRYPTOPP_COMPILE_DEFINITIONS}) + else() + string (REPLACE ";" " " PROP_STR "${CRYPTOPP_COMPILE_DEFINITIONS}") + set_target_properties(${target} PROPERTIES COMPILE_DEFINITIONS "${CRYPTOPP_COMPILE_DEFINITIONS}") + endif() + if (NOT ${CMAKE_VERSION} VERSION_LESS "2.8.12") + target_compile_options(${target} PUBLIC ${CRYPTOPP_COMPILE_OPTIONS}) + else() + string (REPLACE ";" " " PROP_STR "${CRYPTOPP_COMPILE_OPTIONS}") + set_target_properties(${target} PROPERTIES COMPILE_FLAGS "${PROP_STR}") + endif() +endfunction() + +set(cryptopp_LIBRARY_SOURCES ${cryptopp_SOURCES_ASM}) +if (USE_INTERMEDIATE_OBJECTS_TARGET AND NOT ${CMAKE_VERSION} VERSION_LESS "2.8.8") + add_library(cryptopp-object OBJECT ${cryptopp_SOURCES}) + cryptopp_target_compile_properties(cryptopp-object) + + list(APPEND cryptopp_LIBRARY_SOURCES + $ + ) +else () + list(APPEND cryptopp_LIBRARY_SOURCES + ${cryptopp_SOURCES} + ) +endif () + +if (BUILD_STATIC) + add_library(cryptopp-static STATIC ${cryptopp_LIBRARY_SOURCES}) + cryptopp_target_compile_properties(cryptopp-static) + if (NOT ${CMAKE_VERSION} VERSION_LESS "2.8.11") + target_include_directories(cryptopp-static PUBLIC $ $) + else () + set_target_properties(cryptopp-static PROPERTIES INCLUDE_DIRECTORIES "$ $") + endif () +endif () + +if (BUILD_SHARED) + add_library(cryptopp-shared SHARED ${cryptopp_LIBRARY_SOURCES}) + cryptopp_target_compile_properties(cryptopp-shared) + if (NOT ${CMAKE_VERSION} VERSION_LESS "2.8.11") + target_include_directories(cryptopp-shared PUBLIC $ $) + else () + set_target_properties(cryptopp-shared PROPERTIES INCLUDE_DIRECTORIES "$ $") + endif () +endif () + +# Set filenames for targets to be "cryptopp" +if (NOT MSVC) + set(COMPAT_VERSION ${cryptopp_VERSION_MAJOR}.${cryptopp_VERSION_MINOR}) + + if (BUILD_STATIC) + set_target_properties(cryptopp-static + PROPERTIES + OUTPUT_NAME cryptopp) + endif () + if (BUILD_SHARED) + set_target_properties(cryptopp-shared + PROPERTIES + SOVERSION ${COMPAT_VERSION} + OUTPUT_NAME cryptopp) + endif () +endif () + +# Add alternate ways to invoke the build for the shared library that are +# similar to how the crypto++ 'make' tool works. +# see https://github.com/noloader/cryptopp-cmake/issues/32 +if (BUILD_STATIC) + add_custom_target(static DEPENDS cryptopp-static) +endif () +if (BUILD_SHARED) + add_custom_target(shared DEPENDS cryptopp-shared) + add_custom_target(dynamic DEPENDS cryptopp-shared) +endif () + +#============================================================================ +# Third-party libraries +#============================================================================ + +if (WIN32) + if (BUILD_STATIC) + target_link_libraries(cryptopp-static ws2_32) + endif () + if (BUILD_SHARED) + target_link_libraries(cryptopp-shared ws2_32) + endif () +endif () + +# This may need to be expanded to "Solaris" +if (CRYPTOPP_SOLARIS) + if (BUILD_STATIC) + target_link_libraries(cryptopp-static nsl socket) + endif () + if (BUILD_SHARED) + target_link_libraries(cryptopp-shared nsl socket) + endif () +endif () + +find_package(Threads) +if (BUILD_STATIC) + target_link_libraries(cryptopp-static ${CMAKE_THREAD_LIBS_INIT}) +endif () +if (BUILD_SHARED) + target_link_libraries(cryptopp-shared ${CMAKE_THREAD_LIBS_INIT}) +endif () + +#============================================================================ +# Tests +#============================================================================ + +enable_testing() +if (BUILD_TESTING) + add_executable(cryptest ${cryptopp_SOURCES_TEST}) + target_link_libraries(cryptest cryptopp-static) + + # Setting "cryptest" binary name to "cryptest.exe" + if (NOT (WIN32 OR CYGWIN)) + set_target_properties(cryptest PROPERTIES OUTPUT_NAME cryptest.exe) + endif () + if (NOT TARGET cryptest.exe) + add_custom_target(cryptest.exe) + add_dependencies(cryptest.exe cryptest) + endif () + + file(COPY ${SRC_DIR}/TestData DESTINATION ${PROJECT_BINARY_DIR}) + file(COPY ${SRC_DIR}/TestVectors DESTINATION ${PROJECT_BINARY_DIR}) + + add_test(NAME build_cryptest COMMAND "${CMAKE_COMMAND}" --build ${CMAKE_BINARY_DIR} --target cryptest) + add_test(NAME cryptest COMMAND $ v) + set_tests_properties(cryptest PROPERTIES DEPENDS build_cryptest) +endif () + +#============================================================================ +# Doxygen documentation +#============================================================================ + +if (BUILD_DOCUMENTATION) + find_package(Doxygen REQUIRED) + + set(in_source_DOCS_DIR "${SRC_DIR}/html-docs") + set(out_source_DOCS_DIR "${PROJECT_BINARY_DIR}/html-docs") + + add_custom_target(docs ALL + COMMAND ${DOXYGEN_EXECUTABLE} Doxyfile -d CRYPTOPP_DOXYGEN_PROCESSING + WORKING_DIRECTORY ${SRC_DIR} + SOURCES ${SRC_DIR}/Doxyfile + ) + + if (NOT ${in_source_DOCS_DIR} STREQUAL ${out_source_DOCS_DIR}) + add_custom_command( + TARGET docs POST_BUILD + COMMAND ${CMAKE_COMMAND} -E copy_directory "${in_source_DOCS_DIR}" "${out_source_DOCS_DIR}" + COMMAND ${CMAKE_COMMAND} -E remove_directory "${in_source_DOCS_DIR}" + ) + endif () +endif () + +#============================================================================ +# Install +#============================================================================ + +set(export_name "cryptopp-targets") + +# Runtime package +if (BUILD_SHARED) + export(TARGETS cryptopp-shared FILE ${export_name}.cmake ) + install( + TARGETS cryptopp-shared + EXPORT ${export_name} + DESTINATION ${CMAKE_INSTALL_LIBDIR} + RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} + LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} + ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} + ) +endif () + +# Development package +if (BUILD_STATIC) + export(TARGETS cryptopp-static FILE ${export_name}.cmake ) + install(TARGETS cryptopp-static EXPORT ${export_name} DESTINATION ${CMAKE_INSTALL_LIBDIR}) +endif () +install(FILES ${cryptopp_HEADERS} DESTINATION include/cryptopp) + +# CMake Package +if (NOT CMAKE_VERSION VERSION_LESS 2.8.8) + include(CMakePackageConfigHelpers) + write_basic_package_version_file("${PROJECT_BINARY_DIR}/cryptopp-config-version.cmake" VERSION ${cryptopp_VERSION_MAJOR}.${cryptopp_VERSION_MINOR}.${cryptopp_VERSION_PATCH} COMPATIBILITY SameMajorVersion) + install(FILES cryptopp-config.cmake ${PROJECT_BINARY_DIR}/cryptopp-config-version.cmake DESTINATION "lib/cmake/cryptopp") + install(EXPORT ${export_name} DESTINATION "lib/cmake/cryptopp") +endif () + +# Tests +if (BUILD_TESTING) + install(TARGETS cryptest DESTINATION ${CMAKE_INSTALL_BINDIR}) + install(DIRECTORY ${SRC_DIR}/TestData DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/cryptopp) + install(DIRECTORY ${SRC_DIR}/TestVectors DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/cryptopp) +endif () + +# Documentation +if (BUILD_DOCUMENTATION) + install(DIRECTORY "${out_source_DOCS_DIR}" DESTINATION ${CMAKE_INSTALL_DOCDIR}) +endif () + +# Print a configuration summary. We want CXX and CXXFLAGS, but they are not includd in ALL. +if (CRYPTOPP_I386) + message(STATUS "Platform: i386/i686") +elseif (CRYPTOPP_AMD64) + message(STATUS "Platform: x86_64") +elseif (CRYPTOPP_X32) + message(STATUS "Platform: x86_64-x32") +elseif (CRYPTOPP_ARMHF) + message(STATUS "Platform: armhf") +elseif (CRYPTOPP_ARM) + message(STATUS "Platform: arm") +elseif (CRYPTOPP_AARCH32) + message(STATUS "Platform: Aarch32") +elseif (CRYPTOPP_AARCH64) + message(STATUS "Platform: Aarch64") +elseif (CRYPTOPP_SPARC) + message(STATUS "Platform: Sparc") +elseif (CRYPTOPP_SPARC64) + message(STATUS "Platform: Sparc64") +elseif (CRYPTOPP_POWERPC) + message(STATUS "Platform: PowerPC") +elseif (CRYPTOPP_POWERPC64) + message(STATUS "Platform: PowerPC-64") +elseif (CRYPTOPP_MINGW32) + message(STATUS "Platform: MinGW-32") +elseif (CRYPTOPP_MINGW32) + message(STATUS "Platform: MinGW-64") +endif () +if (CRYPTOPP_ARMV7A_NEON) + message(STATUS "NEON: TRUE") +endif () +if (CRYPTOPP_NATIVE_ARCH) + message(STATUS "Native arch: TRUE") +else () + message(STATUS "Native arch: FALSE") +endif () +message(STATUS "Compiler: ${CMAKE_CXX_COMPILER}") +message(STATUS "Compiler options: ${CMAKE_CXX_FLAGS} ${CRYPTOPP_COMPILE_OPTIONS}") +message(STATUS "Compiler definitions: ${CRYPTOPP_COMPILE_DEFINITIONS}") +message(STATUS "Build type: ${CMAKE_BUILD_TYPE}") diff --git a/python/CMakeLists.txt b/python/CMakeLists.txt index e0e845601cf35..938547f363cfb 100644 --- a/python/CMakeLists.txt +++ b/python/CMakeLists.txt @@ -77,6 +77,7 @@ IF(WIN32) add_custom_command(OUTPUT ${PADDLE_PYTHON_BUILD_DIR}/.timestamp COMMAND ${CMAKE_COMMAND} -E copy_directory ${PADDLE_SOURCE_DIR}/python/paddle ${PADDLE_BINARY_DIR}/python/paddle/ COMMAND ${CMAKE_COMMAND} -E env ${py_env} ${PYTHON_EXECUTABLE} setup.py bdist_wheel + COMMENT "Packing whl packages------>>>" DEPENDS copy_paddle_pybind ${FLUID_CORE} framework_py_proto profiler_py_proto ${PY_FILES}) ELSE(WIN32) add_custom_command(OUTPUT ${PADDLE_PYTHON_BUILD_DIR}/.timestamp diff --git a/python/paddle/distributed/cloud_utils.py b/python/paddle/distributed/cloud_utils.py index ae603a0e60b90..962ba62b15f4a 100644 --- a/python/paddle/distributed/cloud_utils.py +++ b/python/paddle/distributed/cloud_utils.py @@ -17,9 +17,9 @@ from paddle.distributed.utils import get_cluster, logger, get_gpus, get_cluster_from_args -def get_cloud_cluster(args_node_ips, args_node_ip, args_port, selected_gpus): +def get_cloud_cluster(args_node_ips, args_node_ip, args_port, selected_devices): """ - args_node_ips:string, args_node_ip:string, args_port: int, selected_gpus:list + args_node_ips:string, args_node_ip:string, args_port: int, selected_devices:list """ #you can automatically get ip info while using paddlecloud multi nodes mode. node_ips = os.getenv("PADDLE_TRAINERS") @@ -60,7 +60,7 @@ def get_cloud_cluster(args_node_ips, args_node_ip, args_port, selected_gpus): paddle_port = int(os.getenv("PADDLE_PORT", "")) if paddle_ports_num >= len( - selected_gpus) and paddle_port != args_port: + selected_devices) and paddle_port != args_port: logger.warning("Use Cloud specified port:{}.".format( paddle_port)) started_port = paddle_port @@ -72,7 +72,7 @@ def get_cloud_cluster(args_node_ips, args_node_ip, args_port, selected_gpus): if started_port is None: started_port = 6170 ports = [ - x for x in range(started_port, started_port + len(selected_gpus)) + x for x in range(started_port, started_port + len(selected_devices)) ] trainer_endpoints = [] for ip in node_ips: @@ -90,7 +90,7 @@ def get_cloud_cluster(args_node_ips, args_node_ip, args_port, selected_gpus): .format(node_ips, node_ip, node_rank, trainer_endpoints)) cluster, pod = get_cluster(node_ips, node_ip, trainer_endpoints, - selected_gpus) + selected_devices) return cluster, cluster.pods[node_rank] @@ -100,20 +100,20 @@ def _get_trainers_num(): def get_cluster_and_pod(args): # parse arguments, used for cloud-single-machine and local - selected_gpus = get_gpus(args.selected_gpus) + selected_devices = get_gpus(args.selected_devices) trainers_num = _get_trainers_num() - logger.debug("parsed from args trainerss_num:{} selected_gpus:{}".format( - trainers_num, selected_gpus)) + logger.debug("parsed from args trainerss_num:{} selected_devices:{}".format( + trainers_num, selected_devices)) cluster = None pod = None if args.use_paddlecloud and trainers_num != 1: cluster, pod = get_cloud_cluster(args.cluster_node_ips, args.node_ip, - args.started_port, selected_gpus) + args.started_port, selected_devices) logger.info("get cluster from cloud:{}".format(cluster)) else: - cluster, pod = get_cluster_from_args(args, selected_gpus) + cluster, pod = get_cluster_from_args(args, selected_devices) logger.info("get cluster from args:{}".format(cluster)) return cluster, pod diff --git a/python/paddle/distributed/fleet/base/fleet_base.py b/python/paddle/distributed/fleet/base/fleet_base.py index f4075e92c4c44..19ba637cc9680 100644 --- a/python/paddle/distributed/fleet/base/fleet_base.py +++ b/python/paddle/distributed/fleet/base/fleet_base.py @@ -637,6 +637,11 @@ def distributed_optimizer(self, optimizer, strategy=None): self._user_defined_strategy = copy.deepcopy(strategy) self._context = {} + + # TODO(shenliang03): This is a temporary solution to support amp. In the case of a dynamic graph, + # the optimizer is returned directly. This problem will be fixed in the future. + if paddle.fluid.framework.in_dygraph_mode(): + return optimizer return self @dygraph_only diff --git a/python/paddle/distributed/fleet/launch_utils.py b/python/paddle/distributed/fleet/launch_utils.py index b4f1f93149052..c5cb1ec94ac3d 100644 --- a/python/paddle/distributed/fleet/launch_utils.py +++ b/python/paddle/distributed/fleet/launch_utils.py @@ -280,7 +280,7 @@ def get_cluster(node_ips, node_ip, trainer_endpoints, device_mode, if isinstance(devices_per_proc[i], (list, tuple)): trainer.gpus.extend(devices_per_proc[i]) else: - trainer.gpus.extend(devices_per_proc[i]) + trainer.gpus.append(devices_per_proc[i]) trainer.endpoint = "%s" % (cur_node_endpoints[i]) trainer.rank = trainer_rank trainer_rank += 1 diff --git a/python/paddle/distributed/fleet/meta_optimizers/pipeline_optimizer.py b/python/paddle/distributed/fleet/meta_optimizers/pipeline_optimizer.py index 9e46bf3368235..9535c9ef53c2e 100644 --- a/python/paddle/distributed/fleet/meta_optimizers/pipeline_optimizer.py +++ b/python/paddle/distributed/fleet/meta_optimizers/pipeline_optimizer.py @@ -138,15 +138,22 @@ def __init__(self, optimizer): super(PipelineOptimizer, self).__init__(optimizer) self.inner_opt = optimizer # we do not allow meta optimizer to be inner optimizer currently - self.meta_optimizers_white_list = [] + self.meta_optimizers_white_list = [ + "RecomputeOptimizer", + "AMPOptimizer", + ] self.meta_optimizers_black_list = ["GraphExecutionOptimizer", ] def _set_basic_info(self, loss, role_maker, user_defined_optimizer, user_defined_strategy): super(PipelineOptimizer, self)._set_basic_info( loss, role_maker, user_defined_optimizer, user_defined_strategy) + self.micro_batch_size = user_defined_strategy.pipeline_configs[ + 'micro_batch_size'] self.num_microbatches = user_defined_strategy.pipeline_configs[ - 'micro_batch'] + 'accumulate_steps'] + self.schedule_mode = user_defined_strategy.pipeline_configs[ + 'schedule_mode'] def _can_apply(self): if not self.role_maker._is_collective: @@ -162,7 +169,11 @@ def _disable_strategy(self, dist_strategy): def _enable_strategy(self, dist_strategy, context): dist_strategy.pipeline = True - dist_strategy.pipeline_configs = {"micro_batch": 1, } + dist_strategy.pipeline_configs = { + "micro_batch_size": 1, + "accumulate_steps": 1, + "schedule_mode": "1F1B", + } def minimize_impl(self, loss, @@ -185,6 +196,9 @@ def minimize_impl(self, loss.block.program._pipeline_opt = dict() loss.block.program._pipeline_opt['local_rank'] = self.rank + loss.block.program._pipeline_opt[ + 'micro_batch_size'] = self.micro_batch_size + loss.block.program._pipeline_opt['schedule_mode'] = self.schedule_mode optimize_ops, params_grads, prog_list = self.wrapped_opt.minimize( loss, startup_program, parameter_list, no_grad_set) assert prog_list diff --git a/python/paddle/distributed/fleet/runtime/the_one_ps.py b/python/paddle/distributed/fleet/runtime/the_one_ps.py index abec4710f5dc9..a56868060055e 100644 --- a/python/paddle/distributed/fleet/runtime/the_one_ps.py +++ b/python/paddle/distributed/fleet/runtime/the_one_ps.py @@ -150,7 +150,8 @@ def parse_by_optimizer(self, grad_name, is_sparse, total_dims, oop = None for op in optimizer_ops: - if op.input("Param")[0] == param_name: + if ("Param" in op.input_names) and ( + op.input("Param")[0] == param_name): oop = op break diff --git a/python/paddle/distributed/fleet/utils/fs.py b/python/paddle/distributed/fleet/utils/fs.py index 221f09a796a6f..7e62e551fe8d5 100644 --- a/python/paddle/distributed/fleet/utils/fs.py +++ b/python/paddle/distributed/fleet/utils/fs.py @@ -447,9 +447,6 @@ def __init__( configs, time_out=5 * 60 * 1000, # ms sleep_inter=1000): # ms - # Raise exception if JAVA_HOME not exists. - java_home = os.environ["JAVA_HOME"] - self.pre_commands = [] hadoop_bin = '%s/bin/hadoop' % hadoop_home self.pre_commands.append(hadoop_bin) diff --git a/python/paddle/distributed/spawn.py b/python/paddle/distributed/spawn.py index 911fed416c050..56e59ac88efee 100644 --- a/python/paddle/distributed/spawn.py +++ b/python/paddle/distributed/spawn.py @@ -50,10 +50,10 @@ def __init__(self): self.print_config = True # It's for gpu training and the training process will run - # on the selected_gpus, each process is bound to a single GPU. + # on the selected_devices, each process is bound to a single GPU. # And if it's not set, this module will use all the gpu cards # for training. - self.selected_gpus = None + self.selected_devices = None def _py_supported_check(): @@ -67,9 +67,9 @@ def _py_supported_check(): def _options_valid_check(options): # `print_config` keeped as a debug options, not show to users - supported_options = ['start_method', 'ips', 'gpus', 'print_config'] + supported_options = ['start_method', 'ips', 'gpus', 'xpus', 'print_config'] deprecated_options = [ - 'selected_gpus', 'started_port', 'cluster_node_ips', 'node_ip', + 'selected_devices', 'started_port', 'cluster_node_ips', 'node_ip', 'use_paddlecloud' ] for key in options: @@ -109,47 +109,83 @@ def _get_subprocess_env_list(nprocs, options): if args.cluster_node_ips is None: args.cluster_node_ips = "127.0.0.1" - # deal with `gpus` - # set default selected gpus + # deal with `gpus` or `xpus` + # set default selected devices(gpus or xpus) # e.g. if the nprocs is 4, the selected gpus is "0,1,2,3" - # NOTE(chenweihang): [ why not use FLAGS_selected_gpus directly? ] - # because the FLAGS_selected_gpus may be used in other place, - # if we set FLAGS_selected_gpus to be `0,1,2,3`, it may cause error + # NOTE(chenweihang): [ why not use FLAGS_selected_gpus or FLAGS_selected_xpus directly? ] + # because the FLAGS_selected_gpus or FLAGS_selected_xpus may be used in other place, + # if we set FLAGS_selected_gpus or FLAGS_selected_xpus to be `0,1,2,3`, it may cause error # when using `ParallelEnv` - # NOTE(chenweihang): use absolute gpu card id - args.selected_gpus = options.get('gpus', None) - if args.selected_gpus is None: - args.selected_gpus = options.get('selected_gpus', None) - env_devices = os.getenv("CUDA_VISIBLE_DEVICES", None) - if env_devices is None or env_devices == "": - env_devices_list = [ - str(x) for x in six.moves.range(core.get_cuda_device_count()) - ] - else: - env_devices_list = env_devices.split(',') - if args.selected_gpus is None: - if len(env_devices_list) < nprocs: - raise RuntimeError( - "the number of visible devices(%d) is less than the number " - "of spawn processes(%d), please ensure that the correct " - "`nprocs` argument is passed or the environment variable " - "`CUDA_VISIBLE_DEVICES` is correctly configured." % - (len(env_devices_list), nprocs)) - args.selected_gpus = ",".join( - [str(env_devices_list[x]) for x in range(0, nprocs)]) - else: - selected_gpu_list = args.selected_gpus.split(',') - if len(selected_gpu_list) != nprocs: - raise ValueError( - "The number of selected gpus(%s) is not equal to " - "the number of spawn processes(%d), please ensure that the " - "correct `nprocs` and `gpus` arguments are passed." % - (len(selected_gpu_list), nprocs)) - for card_id in selected_gpu_list: - if card_id not in env_devices_list: - raise ValueError("The selected gpu card %s cannot found in " - "CUDA_VISIBLE_DEVICES (%s)." % - (card_id, ",".join(env_devices_list))) + # NOTE(chenweihang): use absolute gpu or xpu card id + if core.is_compiled_with_cuda(): + args.selected_devices = options.get('gpus', None) + if args.selected_devices is None: + args.selected_devices = options.get('selected_devices', None) + env_devices = os.getenv("CUDA_VISIBLE_DEVICES", None) + if env_devices is None or env_devices == "": + env_devices_list = [ + str(x) for x in six.moves.range(core.get_cuda_device_count()) + ] + else: + env_devices_list = env_devices.split(',') + if args.selected_devices is None: + if len(env_devices_list) < nprocs: + raise RuntimeError( + "the number of visible devices(%d) is less than the number " + "of spawn processes(%d), please ensure that the correct " + "`nprocs` argument is passed or the environment variable " + "`CUDA_VISIBLE_DEVICES` is correctly configured." % + (len(env_devices_list), nprocs)) + args.selected_devices = ",".join( + [str(env_devices_list[x]) for x in range(0, nprocs)]) + else: + selected_device_list = args.selected_devices.split(',') + if len(selected_device_list) != nprocs: + raise ValueError( + "The number of selected devices(%s) is not equal to " + "the number of spawn processes(%d), please ensure that the " + "correct `nprocs` and `gpus` arguments are passed." % + (len(selected_device_list), nprocs)) + for card_id in selected_device_list: + if card_id not in env_devices_list: + raise ValueError("The selected gpu card %s cannot found in " + "CUDA_VISIBLE_DEVICES (%s)." % + (card_id, ",".join(env_devices_list))) + + elif core.is_compiled_with_xpu(): + args.selected_devices = options.get('xpus', None) + if args.selected_devices is None: + args.selected_devices = options.get('selected_devices', None) + env_devices = os.getenv("XPU_VISIBLE_DEVICES", None) + if env_devices is None or env_devices == "": + env_devices_list = [ + str(x) for x in six.moves.range(core.get_xpu_device_count()) + ] + else: + env_devices_list = env_devices.split(',') + if args.selected_devices is None: + if len(env_devices_list) < nprocs: + raise RuntimeError( + "the number of visible devices(%d) is less than the number " + "of spawn processes(%d), please ensure that the correct " + "`nprocs` argument is passed or the environment variable " + "`XPU_VISIBLE_DEVICES` is correctly configured." % + (len(env_devices_list), nprocs)) + args.selected_devices = ",".join( + [str(env_devices_list[x]) for x in range(0, nprocs)]) + else: + selected_device_list = args.selected_devices.split(',') + if len(selected_device_list) != nprocs: + raise ValueError( + "The number of selected devices(%s) is not equal to " + "the number of spawn processes(%d), please ensure that the " + "correct `nprocs` and `xpus` arguments are passed." % + (len(selected_device_list), nprocs)) + for card_id in selected_device_list: + if card_id not in env_devices_list: + raise ValueError("The selected xpu card %s cannot found in " + "XPU_VISIBLE_DEVICES (%s)." % + (card_id, ",".join(env_devices_list))) # set other inner args args.node_ip = options.get('node_ip', None) @@ -185,12 +221,17 @@ def _remove_risky_env(): def _set_trainer_env(env_dict): - # NOTE(chenweihang): [ Why need set FLAGS_selected_gpus here? ] + # NOTE(chenweihang): [ Why need set FLAGS_selected_gpus or FLAGS_selected_xpus here? ] # When the child process starts, it will inherit the configuration of the # main process and set the FLAGS once, but the environment variable has - # not been set at this time, which leads to the FLAGS_selected_gpus + # not been set at this time, which leads to the FLAGS_selected_gpus or FLAGS_selected_xpus # is keep same with mainprocess(usually empty), so manually update the flags here - set_flags({'FLAGS_selected_gpus': env_dict['FLAGS_selected_gpus']}) + if core.is_compiled_with_cuda(): + set_flags({'FLAGS_selected_gpus': env_dict['FLAGS_selected_gpus']}) + elif core.is_compiled_with_xpu(): + set_flags({'FLAGS_selected_xpus': env_dict['FLAGS_selected_xpus']}) + else: + raise ValueError("PaddlePaddle should be compiled with XPU or CUDA.") for var_name in env_dict: os.environ[var_name] = env_dict[var_name] @@ -407,8 +448,14 @@ def train(print_result=False): if device == 'cpu': # TODO: not supports cpu parallel now nprocs = _cpu_num() - else: + elif device == 'gpu': nprocs = core.get_cuda_device_count() + elif device == 'xpu': + nprocs = core.get_xpu_device_count() + else: + raise ValueError( + "`device` should be a string of `cpu`, 'gpu' or 'xpu', but got {}". + format(device)) # NOTE(chenweihang): [ why need get cluster info before run? ] # when using `paddle.distributed.spawn` start parallel training, diff --git a/python/paddle/distributed/utils.py b/python/paddle/distributed/utils.py index 54efce052ea4d..f40a7b31b83e6 100644 --- a/python/paddle/distributed/utils.py +++ b/python/paddle/distributed/utils.py @@ -24,6 +24,7 @@ import subprocess from contextlib import closing import socket +from paddle.fluid import core logger = logging.getLogger("root") logger.propagate = False @@ -401,13 +402,24 @@ def __free_port(): def _prepare_trainer_env(cluster, trainer): - proc_env = { - "FLAGS_selected_gpus": "%s" % ",".join([str(g) for g in trainer.gpus]), - "PADDLE_TRAINER_ID": "%d" % trainer.rank, - "PADDLE_CURRENT_ENDPOINT": "%s" % trainer.endpoint, - "PADDLE_TRAINERS_NUM": "%d" % cluster.trainers_nranks(), - "PADDLE_TRAINER_ENDPOINTS": ",".join(cluster.trainers_endpoints()) - } + if core.is_compiled_with_xpu(): + proc_env = { + "FLAGS_selected_xpus": + "%s" % ",".join([str(g) for g in trainer.gpus]), + "PADDLE_TRAINER_ID": "%d" % trainer.rank, + "PADDLE_CURRENT_ENDPOINT": "%s" % trainer.endpoint, + "PADDLE_TRAINERS_NUM": "%d" % cluster.trainers_nranks(), + "PADDLE_TRAINER_ENDPOINTS": ",".join(cluster.trainers_endpoints()) + } + elif core.is_compiled_with_cuda(): + proc_env = { + "FLAGS_selected_gpus": + "%s" % ",".join([str(g) for g in trainer.gpus]), + "PADDLE_TRAINER_ID": "%d" % trainer.rank, + "PADDLE_CURRENT_ENDPOINT": "%s" % trainer.endpoint, + "PADDLE_TRAINERS_NUM": "%d" % cluster.trainers_nranks(), + "PADDLE_TRAINER_ENDPOINTS": ",".join(cluster.trainers_endpoints()) + } return proc_env diff --git a/python/paddle/fluid/__init__.py b/python/paddle/fluid/__init__.py index 1a88d3512eaaa..b24da29d0f5fd 100644 --- a/python/paddle/fluid/__init__.py +++ b/python/paddle/fluid/__init__.py @@ -175,7 +175,6 @@ def __bootstrap__(): sysstr = platform.system() read_env_flags = [ 'check_nan_inf', - 'fast_check_nan_inf', 'benchmark', 'eager_delete_scope', 'fraction_of_cpu_memory_to_use', diff --git a/python/paddle/fluid/compiler.py b/python/paddle/fluid/compiler.py index a04d58ff25edf..2698f1a00dc80 100644 --- a/python/paddle/fluid/compiler.py +++ b/python/paddle/fluid/compiler.py @@ -360,10 +360,6 @@ def _compile_data_parallel(self, places, use_device, scope=None): else: self._exec_strategy.num_threads = len(places) * 2 - if self._exec_strategy._use_device == DeviceType.XPU: - assert self._exec_strategy.num_threads == 1, \ - "Currently only single thread is supported in Kunlun XPU." - if self._build_strategy.num_trainers > 1: assert self._is_data_parallel, \ "If you use multi-trainer to train the model, you should use "\ diff --git a/python/paddle/fluid/contrib/mixed_precision/__init__.py b/python/paddle/fluid/contrib/mixed_precision/__init__.py index a580ae5574c35..571b755b50d2a 100644 --- a/python/paddle/fluid/contrib/mixed_precision/__init__.py +++ b/python/paddle/fluid/contrib/mixed_precision/__init__.py @@ -20,7 +20,10 @@ from .fp16_lists import * from . import fp16_utils from .fp16_utils import * +from . import bf16 +from .bf16 import * __all__ = decorator.__all__ __all__ += fp16_lists.__all__ __all__ += fp16_utils.__all__ +__all__ += bf16.__all__ diff --git a/python/paddle/fluid/contrib/mixed_precision/bf16/__init__.py b/python/paddle/fluid/contrib/mixed_precision/bf16/__init__.py new file mode 100644 index 0000000000000..8c05bc4899cf7 --- /dev/null +++ b/python/paddle/fluid/contrib/mixed_precision/bf16/__init__.py @@ -0,0 +1,24 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserve. +# +# 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. + +from __future__ import print_function + +from . import amp_lists +from .amp_lists import * +from . import amp_utils +from .amp_utils import * + +__all__ = [] +__all__ += amp_lists.__all__ +__all__ += amp_utils.__all__ diff --git a/python/paddle/fluid/contrib/mixed_precision/bf16/amp_lists.py b/python/paddle/fluid/contrib/mixed_precision/bf16/amp_lists.py new file mode 100644 index 0000000000000..81dc32d114b14 --- /dev/null +++ b/python/paddle/fluid/contrib/mixed_precision/bf16/amp_lists.py @@ -0,0 +1,97 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import copy +from ..fp16_lists import white_list as white_list_fp16, black_list as black_list_fp16,\ + gray_list as gray_list_fp16, unsupported_fp16_list + +__all__ = ["AutoMixedPrecisionListsBF16"] + + +class AutoMixedPrecisionListsBF16(object): + """ + AutoMixedPrecisionListsBF16 is a class for fp32/bf16 op types list. The lists are used for an + algorithm which determines op's execution mode (fp32 or bf16).It can update pre-defined + fp32 list and bf16 list according to users' custom fp32 bf16 lists. + + Args: + custom_bf16_list (set): Users' custom bf16 list. + custom_fp32_list (set): Users' custom fp32 list. + custom_fp32_varnames (set): Users' custom fp32 variables' names. + + Examples: + .. code-block:: python + import paddle + paddle.enable_static() + with paddle.static.amp.bf16_guard(): + paddle.static.amp.AutoMixedPrecisionListsBF16(custom_fp32_list={'lstm'}) + """ + + def __init__(self, + custom_bf16_list=None, + custom_fp32_list=None, + custom_fp32_varnames=None): + self._custom_bf16_list = custom_bf16_list + self._custom_fp32_list = custom_fp32_list + self.bf16_list = copy.copy(bf16_list) + self.fp32_list = copy.copy(fp32_list) + self.gray_list = copy.copy(gray_list) + self.unsupported_list = copy.copy(unsupported_list) + self.fp32_varnames = copy.copy(custom_fp32_varnames) + self._update_list() + + def _update_list(self): + """ + Update fp32 and bf16 list according to users' custom list. + """ + if self._custom_bf16_list and self._custom_fp32_list: + for op_name in self._custom_bf16_list: + if op_name in self._custom_fp32_list: + raise ValueError("Custom bf16 list overlap " + "custom fp32 list") + if self._custom_bf16_list: + for op_name in self._custom_bf16_list: + if op_name in self.fp32_list: + self.fp32_list.remove(op_name) + elif op_name in self.gray_list: + self.gray_list.remove(op_name) + self.bf16_list.add(op_name) + if self._custom_fp32_list: + for op_name in self._custom_fp32_list: + if op_name in self.bf16_list: + self.bf16_list.remove(op_name) + elif op_name in self.gray_list: + self.gray_list.remove(op_name) + self.fp32_list.add(op_name) + self.unsupported_list.add(op_name) + + +# always bf16 +bf16_list = {'elementwise_add', } + +# depends on the prev_op type +gray_list = { + 'reshape2', + 'lookup_table', +} + +unsupported_list = unsupported_fp16_list.copy().copy() +fp32_list = black_list_fp16.copy().copy() +fp32_list |= white_list_fp16 +fp32_list |= gray_list_fp16 + +fp32_list -= bf16_list +fp32_list -= gray_list +unsupported_list -= bf16_list +unsupported_list -= gray_list diff --git a/python/paddle/fluid/contrib/mixed_precision/bf16/amp_utils.py b/python/paddle/fluid/contrib/mixed_precision/bf16/amp_utils.py new file mode 100644 index 0000000000000..c2c01f88c7431 --- /dev/null +++ b/python/paddle/fluid/contrib/mixed_precision/bf16/amp_utils.py @@ -0,0 +1,296 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function +import struct + +from .... import core +from .... import framework +from ....log_helper import get_logger +from ....wrapped_decorator import signature_safe_contextmanager +from .amp_lists import AutoMixedPrecisionListsBF16 +from ..fp16_utils import find_true_prev_op, find_true_post_op, _rename_arg, find_op_index +import logging +import numpy as np + +__all__ = ["bf16_guard", "rewrite_program_bf16", "convert_float_to_uint16"] + +_logger = get_logger( + __name__, logging.INFO, fmt='%(asctime)s-%(levelname)s: %(message)s') + +_valid_types = [ + core.VarDesc.VarType.LOD_TENSOR, core.VarDesc.VarType.SELECTED_ROWS, + core.VarDesc.VarType.LOD_TENSOR_ARRAY +] + +_bf16_guard_pattern = "__use_bf16__" + + +def convert_float_to_uint16(in_list): + in_list = np.asarray(in_list) + out = np.vectorize( + lambda x: struct.unpack('> 16, + otypes=[np.uint16])(in_list.flat) + return np.reshape(out, in_list.shape) + + +def _dtype_to_str(dtype): + """ + Convert specific variable type to its corresponding string. + + Args: + dtype (VarType): Variable type. + """ + if dtype == core.VarDesc.VarType.BF16: + return 'bf16' + else: + return 'fp32' + + +def _insert_cast_op(block, op, idx, src_dtype, dest_dtype): + """ + Insert cast op and rename args of input and output. + + Args: + block (Program): The block in which the operator is. + op (Operator): The operator to insert cast op. + idx (int): The index of current operator. + src_dtype (VarType): The input variable dtype of cast op. + dest_dtype (VarType): The output variable dtype of cast op. + + Returns: + num_cast_op (int): The number of cast ops that have been inserted. + """ + num_cast_ops = 0 + + for in_name in op.input_names: + if src_dtype == core.VarDesc.VarType.FP32 and op.type in [ + 'batch_norm', 'fused_bn_add_activation', 'layer_norm' + ]: + if in_name not in {'X', 'Z'}: + continue + for in_var_name in op.input(in_name): + in_var = block.var(in_var_name) + if in_var.type not in _valid_types or in_var.dtype == dest_dtype: + continue + if in_var.dtype == src_dtype: + cast_name = in_var.name + '.cast_' + _dtype_to_str(dest_dtype) + out_var = block.vars.get(cast_name) + if out_var is None or out_var.dtype != dest_dtype: + out_var = block.create_var( + name=cast_name, + dtype=dest_dtype, + persistable=False, + stop_gradient=in_var.stop_gradient) + + block._insert_op( + idx, + type="cast", + inputs={"X": in_var}, + outputs={"Out": out_var}, + attrs={ + "in_dtype": in_var.dtype, + "out_dtype": out_var.dtype + }) + num_cast_ops += 1 + _rename_arg(op, in_var.name, out_var.name) + else: + if op.has_attr('in_dtype'): + op._set_attr('in_dtype', dest_dtype) + if src_dtype == core.VarDesc.VarType.FP32 and dest_dtype == core.VarDesc.VarType.BF16: + for out_name in op.output_names: + if op.type in [ + 'batch_norm', 'fused_bn_add_activation', 'layer_norm' + ] and out_name != 'Y': + continue + for out_var_name in op.output(out_name): + out_var = block.var(out_var_name) + if out_var.type not in _valid_types: + continue + if out_var.dtype == core.VarDesc.VarType.FP32: + out_var.desc.set_dtype(core.VarDesc.VarType.BF16) + if op.has_attr('out_dtype'): + op._set_attr('out_dtype', core.VarDesc.VarType.BF16) + return num_cast_ops + + +def _is_in_fp32_varnames(op, amp_lists): + for in_name in op.input_arg_names: + if in_name in amp_lists.fp32_varnames: + return True + + for out_name in op.output_arg_names: + if out_name in amp_lists.fp32_varnames: + return True + + return False + + +def _need_keep_fp32(op, unsupported_op_list, use_bf16_guard): + if op.type in unsupported_op_list: + # the highest priority condition: If ops don't have bf16 computing kernels, + # they must be executed in fp32 calculation pattern. + return True + + # process ops about learning rate + in_out_arg_names = [] + in_out_arg_names.extend(list(op.input_arg_names)) + in_out_arg_names.extend(list(op.output_arg_names)) + for name in in_out_arg_names: + if "learning_rate" in name: + return True + + if use_bf16_guard: + if op.has_attr("op_namescope") and \ + (_bf16_guard_pattern in op.attr("op_namescope")): + # op in bf16 guard + return False + else: + # op not in bf16 guard + return True + else: + return False + + +@signature_safe_contextmanager +def bf16_guard(): + """ + As for the pure bf16 training, if users set `use_bf16_guard` to True, + only those ops created in the context manager `bf16_guard` will be + transformed as float16 type. + + Examples: + .. code-block:: python + + import numpy as np + import paddle + import paddle.nn.functional as F + paddle.enable_static() + data = paddle.static.data(name='X', shape=[None, 1, 28, 28], dtype='float32') + conv2d = paddle.static.nn.conv2d(input=data, num_filters=6, filter_size=3) + + with paddle.static.amp.bf16_guard(): + bn = paddle.static.nn.batch_norm(input=conv2d, act="relu") + pool = F.max_pool2d(bn, kernel_size=2, stride=2) + hidden = paddle.static.nn.fc(pool, size=10) + loss = paddle.mean(hidden) + """ + with framework.name_scope(prefix=_bf16_guard_pattern): + yield + + +def rewrite_program_bf16(main_prog, amp_lists=None, use_bf16_guard=False): + """ + Traverse all ops in current block and insert cast op according to + which set current op belongs to. + + 1. When an op belongs to the fp32 list, add it to fp32 set + 2. When an op belongs to the bf16 list, add it to bf16 set + 3. When an op belongs to the gray list. If one + of its inputs is the output of fp32 set op or fp32 list op, + add it to fp32 set. If all of its previous ops are not fp32 + op and one of its inputs is the output of bf16 set op or + bf16 list op, add it to bf16 set. + 4. When an op isn't in the lists, add it to fp32 op set. + 5. Add necessary cast ops to make sure that fp32 set op will be + computed in fp32 mode, while bf16 set op will be computed in + bf16 mode. + + Args: + main_prog (Program): The main program for training. + """ + if amp_lists is None: + amp_lists = AutoMixedPrecisionListsBF16() + block = main_prog.global_block() + ops = block.ops + bf16_op_set = set() + fp32_op_set = set() + for op in ops: + + # NOTE(zhiqiu): 'create_py_reader' and 'read' is used in non-iterable DataLoder, + # we don't need to handle reader op and the input of 'create_py_reader' is not + # in block, which may result in errors. + # See GeneratorLoader._init_non_iterable() for details. + if op.type == 'create_py_reader' or op.type == 'read': + continue + + if amp_lists.fp32_varnames is not None and _is_in_fp32_varnames( + op, amp_lists): + fp32_op_set.add(op) + continue + + if op.type in amp_lists.fp32_list or _need_keep_fp32( + op, amp_lists.unsupported_list, use_bf16_guard): + fp32_op_set.add(op) + elif op.type in amp_lists.bf16_list: + bf16_op_set.add(op) + elif op.type in amp_lists.gray_list: + is_fp32_op = False + is_bf16_op = False + for in_name in op.input_names: + # if this op has inputs + if in_name: + for in_var_name in op.input(in_name): + in_var = block.var(in_var_name) + # this in_var isn't the output of other op + if in_var.op is None: + continue + elif in_var.op is op: + prev_op = find_true_prev_op(ops, op, in_var_name) + if prev_op is None: + continue + else: + prev_op = in_var.op + # if it's one of inputs + if prev_op in fp32_op_set or \ + prev_op.type in amp_lists.fp32_list: + is_fp32_op = True + elif prev_op in bf16_op_set or \ + prev_op.type in amp_lists.bf16_list: + is_bf16_op = True + if is_fp32_op: + fp32_op_set.add(op) + elif is_bf16_op: + bf16_op_set.add(op) + else: + pass + else: + # For numerical safe, we apply fp32 computation on ops that + # are not determined which list they should stay. + fp32_op_set.add(op) + + idx = 0 + while idx < len(ops): + op = ops[idx] + num_cast_ops = 0 + if op in fp32_op_set: + num_cast_ops = _insert_cast_op(block, op, idx, + core.VarDesc.VarType.BF16, + core.VarDesc.VarType.FP32) + elif op in bf16_op_set: + if op.has_attr('use_mkldnn'): + op._set_attr('use_mkldnn', True) + op._set_attr('mkldnn_data_type', 'bfloat16') + elif op.has_attr('dtype') and op.attr( + 'dtype') == core.VarDesc.VarType.FP32: + op._set_attr('dtype', core.VarDesc.VarType.BF16) + + num_cast_ops = _insert_cast_op(block, op, idx, + core.VarDesc.VarType.FP32, + core.VarDesc.VarType.BF16) + else: + pass + + idx += num_cast_ops + 1 diff --git a/python/paddle/fluid/contrib/mixed_precision/fp16_lists.py b/python/paddle/fluid/contrib/mixed_precision/fp16_lists.py index c88ae2d9cbf60..6a524af4ee240 100644 --- a/python/paddle/fluid/contrib/mixed_precision/fp16_lists.py +++ b/python/paddle/fluid/contrib/mixed_precision/fp16_lists.py @@ -69,7 +69,7 @@ def _update_list(self): self.unsupported_list.add(op_name) -# The three sets listed below are changed dynamiclly. They don't contain all +# The three sets listed below are changed dynamiclly. They don't contain all # paddle ops currently. # The set of ops that support fp16 calculation and are considered numerically- diff --git a/python/paddle/fluid/contrib/slim/quantization/imperative/qat.py b/python/paddle/fluid/contrib/slim/quantization/imperative/qat.py index c5ee9ea675100..ea2e8e073b508 100644 --- a/python/paddle/fluid/contrib/slim/quantization/imperative/qat.py +++ b/python/paddle/fluid/contrib/slim/quantization/imperative/qat.py @@ -17,105 +17,98 @@ import numpy as np import sys import os +import warnings + import paddle -from paddle.fluid import dygraph, core, framework +from paddle.fluid import dygraph, core, framework, unique_name from paddle.fluid.executor import Executor +from paddle.fluid.param_attr import ParamAttr +from paddle.fluid.initializer import Constant from paddle.fluid.dygraph.io import INFER_MODEL_SUFFIX, INFER_PARAMS_SUFFIX -from paddle.nn import Linear, Conv2D, Conv2DTranspose, MaxPool2D, MaxPool1D, BatchNorm1D, BatchNorm2D, BatchNorm3D -from paddle.fluid.dygraph.nn import BatchNorm, Pool2D from paddle.fluid.io import load_inference_model, save_inference_model -from paddle.nn.layer.activation import ReLU, LeakyReLU, Sigmoid, ReLU6, Tanh, Softmax, PReLU, Swish from paddle.fluid.log_helper import get_logger from . import quant_nn from .. import quantization_pass +from . import utils -__all__ = ['ImperativeQuantAware', 'ImperativeCalcOutScale'] +__all__ = ['ImperativeQuantAware'] _logger = get_logger( __name__, logging.INFO, fmt='%(asctime)s-%(levelname)s: %(message)s') -_op_real_in_out_name = { - "conv2d": [["Input", "Filter"], ["Output"]], - "depthwise_conv2d": [["Input", "Filter"], ["Output"]], - "pool2d": [["X"], ["Out"]], - "elementwise_add": [["X", "Y"], ["Out"]], - "softmax": [["X"], ["Out"]], - "relu": [["X"], ["Out"]], - "relu6": [["X"], ["Out"]], - "leaky_relu": [["X"], ["Out"]], - "prelu": [["X"], ["Out"]], - "tanh": [["X"], ["Out"]], - "batch_norm": [["X"], ["Y"]], - "sigmoid": [["X"], ["Out"]], - "swish": [["X"], ["Out"]], -} - class ImperativeQuantAware(object): """ - Add the fake quant logic for given quantizable layers, namely add the quant_dequant - computational logic both for activation inputs and weight inputs. + Applying quantization aware training (QAT) to dgraph model. """ def __init__(self, - weight_bits=8, - activation_bits=8, + quantizable_layer_type=['Conv2D', 'Linear'], weight_quantize_type='abs_max', activation_quantize_type='moving_average_abs_max', + weight_bits=8, + activation_bits=8, moving_rate=0.9, - quantizable_layer_type=['Conv2D', 'Linear'], weight_preprocess_layer=None, act_preprocess_layer=None, weight_quantize_layer=None, act_quantize_layer=None): - r""" + """ The constructor for ImperativeQuantAware. Args: - weight_bits(int): quantization bit number for weights, - whereas the bias is not quantized. - activation_bits(int): quantization bit number for activations. + quantizable_layer_type(list[str | layer]): List the type of + layers that will be quantized. Default is ['Conv2D', 'Linear']. weight_quantize_type(str): quantization type for weights, - which supports 'abs_max' now. The 'moving_average_abs_max' - usually is not used for weights, since weights are fixed once the - model is well trained. + which supports 'abs_max' and 'channel_wise_abs_max'. activation_quantize_type(str): quantization type for activations, which supports 'abs_max' and 'moving_average_abs_max' now. - If using 'abs_max' mode, the quantization scale will be calculated - dynamically each step in both training and testing period. If using - 'moving_average_abs_max', the static quantization scale will be calculated - during training and used in inference. - moving_rate(float): the parameter for 'moving_average_abs_max' quantization. - quantizable_layer_type(list[str]): List the type of layers that will be quantized. - Default is ['Conv2D', 'Linear']. The quantizable_op_type in - QuantizationFreezePass and ConvertToInt8Pass must be the same as this. - weight_preprocess_layer(paddle.nn.Layer, optional): A paddle Layer that defines how to preprocess - weight before quantization. Using this can quickly test if user's - preprocess method works or not. The input is non-quantized - weight and function returns processed weight to be quantized. - If None, the weight will be quantized directly. Default is None. - act_preprocess_layer(paddle.nn.Layer, optional): A paddle Layer that defines how to preprocess - activation before quantization. Using this can quickly test if user's - preprocess method works or not. The input is non-quantized - activation and function returns processed activation to be quantized. - If None, the activation will be quantized directly. Default is None. - weight_quantize_layer(paddle.nn.Layer, optional): A paddle Layer that defines how to quantize weight. + If using 'abs_max' mode, the quantization scale will be + calculated dynamically each step in both training and testing + period. If using 'moving_average_abs_max', the static + quantization scale will be calculated during training and + used in inference. + weight_bits(int): quantization bit number for weights, whereas + the bias is not quantized. + activation_bits(int): quantization bit number for activations. + moving_rate(float): the parameter for 'moving_average_abs_max' + quantization. + weight_preprocess_layer(paddle.nn.Layer, optional): A paddle + Layer that defines how to preprocess weight before quantization. + Using this can quickly test if user's preprocess method works + or not. The input is non-quantized weight and function returns + processed weight to be quantized. + If None, the weight will be quantized directly. + Default is None. + act_preprocess_layer(paddle.nn.Layer, optional): A paddle Layer + that defines how to preprocess activation before quantization. + Using this can quickly test if user's preprocess method works + or not. The input is non-quantized activation and function returns + processed activation to be quantized. + If None, the activation will be quantized directly. + Default is None. + weight_quantize_layer(paddle.nn.Layer, optional): A paddle Layer that + defines how to quantize weight. Using this can quickly test if user's quantization method works or not. In this layer, user should both define quantization method and dequantization method, that is, the function's input is non-quantized - weight and returns dequantized weight. If None, will use - quantization op defined by 'weight_quantize_type'. Default is None. - act_quantize_layer(paddle.nn.Layer, optional): A paddle Layer that defines how to quantize activation. + weight and returns dequantized weight. + If None, will use uantization op defined by 'weight_quantize_type'. + Default is None. + act_quantize_layer(paddle.nn.Layer, optional): A paddle Layer that defines + how to quantize activation. Using this can quickly test if user's quantization method works or not. In this layer, user should both define quantization method and dequantization method, that is, the function's input is non-quantized - activation and returns dequantized activation. If None, will use - quantization op defined by 'activation_quantize_type'. Default is None. + activation and returns dequantized activation. + If None, will use quantization op defined by 'activation_quantize_type'. + Default is None. Note: - If user sets attribute 'skip_quant' to a Layer that support dynamic quantization and sets - it to true, the layer would not be quantized during training. If this attribute is not sets - or the attribute is false, the Layer would be qunatized in training. + If user sets attribute 'skip_quant' to a Layer that support dynamic + quantization and sets it to true, the layer would not be quantized + during training. If this attribute is not sets or the attribute is + false, the Layer would be qunatized in training. Examples 1: .. code-block:: python @@ -192,168 +185,201 @@ def forward(self, inputs): model_path="./imperative_model_qat") """ super(ImperativeQuantAware, self).__init__() - self._weight_bits = weight_bits - self._activation_bits = activation_bits - self._moving_rate = moving_rate - self._activation_quantize_type = activation_quantize_type - self._weight_quantize_type = weight_quantize_type - - self._weight_pre_layer = weight_preprocess_layer - self._act_pre_layer = act_preprocess_layer - self._weight_quant_layer = weight_quantize_layer - self._act_quant_layer = act_quantize_layer - self._out_scale = ImperativeCalcOutScale() - - t_check = lambda method: method is None or issubclass(method, dygraph.layers.Layer) - assert t_check( - self._weight_pre_layer), "weight_preprocess should be nn.Layer" - assert t_check(self._act_pre_layer), "act_preprocess should be nn.Layer" - assert t_check( - self._weight_quant_layer), "weight_quantize should be nn.Layer" - assert t_check(self._act_quant_layer), "act_quantize should be nn.Layer" - - quant_type = { - 'abs_max', 'moving_average_abs_max', 'channel_wise_abs_max' - } - assert activation_quantize_type != 'channel_wise_abs_max', \ - "The activation quantization type does not support 'channel_wise_abs_max'." - if activation_quantize_type not in quant_type: - raise ValueError( - "Unknown activation_quantize_type : '%s'. It can only be " - "'abs_max' or 'moving_average_abs_max' now." % - (str(activation_quantize_type))) - if weight_quantize_type not in quant_type: - raise ValueError( - "Unknown weight_quantize_type: '%s'. It can only be " - "'abs_max' or 'moving_average_abs_max' or 'channel_wise_abs_max' now." - % (str(weight_quantize_type))) - - self._quant_layers_map = { - 'Conv2D': Conv2D, - 'Linear': Linear, - 'Pool2D': Pool2D, - 'ReLU': ReLU, - 'LeakyReLU': LeakyReLU, - 'ReLU6': ReLU6, - 'Softmax': Softmax, - 'Tanh': Tanh, - 'Swish': Swish + kwargs = { + "quantizable_layer_type": quantizable_layer_type, + "weight_quantize_type": weight_quantize_type, + "activation_quantize_type": activation_quantize_type, + "weight_bits": weight_bits, + "activation_bits": activation_bits, + "moving_rate": moving_rate, + "weight_preprocess_layer": weight_preprocess_layer, + "act_preprocess_layer": act_preprocess_layer, + "weight_quantize_layer": weight_quantize_layer, + "act_quantize_layer": act_quantize_layer } - self._quantizable_layer_type = tuple( - self._quant_layers_map[layer] - if layer in self._quant_layers_map else layer - for layer in quantizable_layer_type) - for layer in self._quantizable_layer_type: - assert not isinstance( - layer, str), "{} is unspported to be quantized.".format(layer) + + self._quantize_inputs = ImperativeQuantizeInputs(**kwargs) + + self._calc_output_scale = ImperativeCalcOutputScale() def quantize(self, model): """ - According to weights' and activations' quantization types, the model will be added some fake - quant ops, such as fake_quantize_dequantize_moving_average_abs_max, fake_quantize_dequantize_abs_max - and so on. At the same time, the out_scale value of outputs would be calculated. + According to weights' and activations' quantization types, + the model will be added some fake quant ops, such as + fake_quantize_dequantize_moving_average_abs_max, + fake_quantize_dequantize_abs_max and so on. At the same time, + the out_scale value of outputs would be calculated. Args: model(fluid.dygraph.Layer): the model to be quantized. Returns: None """ + assert isinstance(model, dygraph.Layer), \ + "The model must be the instance of dygraph.Layer." + self._quantize_inputs.apply(model) + self._calc_output_scale.apply(model) + + def save_quantized_model(self, layer, path, input_spec=None, **config): + self._calc_output_scale.save_quantized_model(layer, path, input_spec, + **config) + + +class ImperativeQuantizeInputs(object): + """ + Based on the input params, add the quant_dequant computational + logic both for activation inputs and weight inputs. + """ + + def __init__(self, + quantizable_layer_type=['Conv2D', 'Linear'], + weight_quantize_type='abs_max', + activation_quantize_type='moving_average_abs_max', + weight_bits=8, + activation_bits=8, + moving_rate=0.9, + weight_preprocess_layer=None, + act_preprocess_layer=None, + weight_quantize_layer=None, + act_quantize_layer=None): + """ + The constructor for ImperativeQuantizeInputs. + + Please refer to the args of ImperativeQuantAware. + """ + super(ImperativeQuantizeInputs, self).__init__() + + self._quantizable_layer_type = tuple( + utils.quant_input_layers_map[layer] + if layer in utils.quant_input_layers_map else layer + for layer in quantizable_layer_type) + for layer in self._quantizable_layer_type: + assert not isinstance(layer, str), \ + "%s is unspported to be quantized." % layer + + quantize_type = { + 'abs_max', 'moving_average_abs_max', 'channel_wise_abs_max' + } + assert weight_quantize_type in quantize_type, \ + "Unsupported weight_quantize_type: %s. It can only " \ + "be abs_max or moving_average_abs_max or " \ + "channel_wise_abs_max." % weight_quantize_type + assert activation_quantize_type != 'channel_wise_abs_max' \ + and activation_quantize_type in quantize_type, \ + "Unsupported activation_quantize_type: %s. It can " \ + "only be abs_max or moving_average_abs_max now." \ + % activation_quantize_type + + bits_check = lambda bits: isinstance(bits, int) \ + and bits >= 0 and bits <= 16 + assert bits_check(weight_bits), \ + "weight_bits should be 1, 2,... or 16." + assert bits_check(activation_bits), \ + "activation_bits should be 1, 2,... or 16." + + layer_check = lambda method: method is None or \ + issubclass(method, dygraph.layers.Layer) + assert layer_check(weight_preprocess_layer), \ + "weight_preprocess should be nn.Layer." + assert layer_check(act_preprocess_layer), \ + "act_preprocess should be nn.Layer." + assert layer_check(weight_quantize_layer), \ + "weight_quantize should be nn.Layer." + assert layer_check(act_quantize_layer), \ + "act_quantize should be nn.Layer." + + self._kwargs = { + "weight_quantize_type": weight_quantize_type, + "activation_quantize_type": activation_quantize_type, + "weight_bits": weight_bits, + "activation_bits": activation_bits, + "moving_rate": moving_rate, + "weight_pre_layer": weight_preprocess_layer, + "act_pre_layer": act_preprocess_layer, + "weight_quant_layer": weight_quantize_layer, + "act_quant_layer": act_quantize_layer + } + + def apply(self, model): + assert isinstance(model, dygraph.Layer), \ + "The model must be the instance of dygraph.Layer." + for name, layer in model.named_sublayers(): - if not isinstance(layer, self._quantizable_layer_type): - continue - if hasattr(layer, "skip_quant") and layer.skip_quant == True: + if not isinstance(layer, self._quantizable_layer_type) \ + or (hasattr(layer, "skip_quant") \ + and layer.skip_quant == True): continue + # TODO(jc): optimize this module last_idx = 0 idx = 0 obj = model - parent = model - while idx < len(name): if (name[idx] == '.'): - if hasattr(parent, name[last_idx:idx]): + if hasattr(obj, name[last_idx:idx]): obj = getattr(obj, name[last_idx:idx]) - parent = obj last_idx = idx + 1 idx += 1 target = name[last_idx:idx] - quant_layer = self._get_quantized_counterpart(layer) - setattr(quant_layer, "layer_name", layer.full_name()) + quant_layer = self._get_quantized_layer(layer) setattr(obj, target, quant_layer) - self._out_scale.calc_out_scale(model) - - def _get_quantized_counterpart(self, layer): - quant_layers = tuple(self._quant_layers_map.values()) - quantized_counterpart = tuple('Quantized' + k - for k in self._quant_layers_map.keys()) - - predicate = lambda value: isinstance(layer, value) - index_generator = (i for i, v in enumerate(quant_layers) - if predicate(v)) - - try: - index = next(index_generator) - except StopIteration: - _logger.fatal("The layer {} is unsupported to be quantized.".format( - layer.full_name())) - sys.exit(-1) + def _get_quantized_layer(self, layer): + quant_layer_name = None + for key, value in utils.quant_input_layers_map.items(): + if isinstance(layer, value): + quant_layer_name = 'Quantized' + key + break + assert quant_layer_name is not None, \ + "The layer %s is unsupported to be quantized." \ + % layer.full_name() layer_with_weight = ['QuantizedConv2D', 'QuantizedLinear'] - if quantized_counterpart[index] not in layer_with_weight: - quant_layer_class_name = 'QuantizedNoweightLayer' - else: - quant_layer_class_name = quantized_counterpart[index] - quantized_layer = quant_nn.__dict__[quant_layer_class_name]( - layer, self._weight_bits, self._activation_bits, self._moving_rate, - self._weight_quantize_type, self._activation_quantize_type, - self._weight_pre_layer, self._act_pre_layer, - self._weight_quant_layer, self._act_quant_layer) - return quantized_layer + if quant_layer_name not in layer_with_weight: + quant_layer_name = 'QuantizedNoweightLayer' - def save_quantized_model(self, layer, path, input_spec=None, **config): - self._out_scale.save_quantized_model(layer, path, input_spec, **config) + return quant_nn.__dict__[quant_layer_name](layer, **self._kwargs) -class ImperativeCalcOutScale(object): +class ImperativeCalcOutputScale(object): def __init__(self, moving_rate=0.9): """ - Add the logic of calculating and setting output quantization scales of some layers. - These output quantization scales may be used by tensorRT or some other inference engines. + Add the logic of calculating and setting output scales of some layers. Args: - moving_rate(float): The decay coefficient of moving average. The default value is 0.9. + moving_rate(float): The decay coefficient of moving average. + The default value is 0.9. """ - super(ImperativeCalcOutScale, self).__init__() + super(ImperativeCalcOutputScale, self).__init__() self._moving_rate = moving_rate - self._out_scale_layer_type_list = ( - BatchNorm, BatchNorm1D, BatchNorm2D, BatchNorm3D, Conv2D, LeakyReLU, - Linear, PReLU, Pool2D, MaxPool1D, MaxPool2D, ReLU, ReLU6, Sigmoid, - Softmax, Tanh, Swish) self._register_hook_handle_list = [] self._out_scale_dict = collections.OrderedDict() - def calc_out_scale(self, model): + def apply(self, model): """ - Insert the `moving_average_abs_max_scale` op to calculate output scale of Specific layers in model. + Insert the `moving_average_abs_max_scale` op to calculate output + scale of specific layers in model. Args: - model(fluid.dygraph.Layer): The target model which would be calculate the output quantization scale. + model(fluid.dygraph.Layer): The target model which would be + calculate the output quantization scale. Returns: None """ - assert isinstance( - model, dygraph.Layer), "model must be the instance of dygraph.Layer" + assert isinstance(model, dygraph.Layer), \ + "The model must be the instance of dygraph.Layer." + + # Calculate the target ops's output scale, and don't consider + # the skip_quant attr for _, layer in model.named_sublayers(): - if not isinstance(layer, self._out_scale_layer_type_list): - if 'quantized_' not in layer.full_name(): - continue - forward_post_hook_handle = layer.register_forward_post_hook( - self._forward_post_hook) - self._register_hook_handle_list.append(forward_post_hook_handle) + if self._is_target_layer(layer): + self._init_scale_params(layer) + hook_handle = layer.register_forward_post_hook( + self._calc_output_scale_hook) + self._register_hook_handle_list.append(hook_handle) def save_quantized_model(self, layer, path, input_spec=None, **config): """ @@ -361,162 +387,267 @@ def save_quantized_model(self, layer, path, input_spec=None, **config): Args: layer (Layer): The Layer to be saved. - path (str): The path prefix to save model. The format is ``dirname/file_prefix`` or ``file_prefix``. - input_spec (list[InputSpec|Tensor], optional): Describes the input of the saved model's forward - method, which can be described by InputSpec or example Tensor. If None, all input variables of - the original Layer's forward method would be the inputs of the saved model. Default None. - **configs (dict, optional): Other save configuration options for compatibility. We do not - recommend using these configurations, they may be removed in the future. If not necessary, - DO NOT use them. Default None. + path (str): The path prefix to save model. The format is + ``dirname/file_prefix`` or ``file_prefix``. + input_spec (list[InputSpec|Tensor], optional): Describes the input + of the saved model's forward method, which can be described by + InputSpec or example Tensor. If None, all input variables of + the original Layer's forward method would be the inputs of + the saved model. Default None. + **configs (dict, optional): Other save configuration options for + compatibility. We do not recommend using these configurations, + they may be removed in the future. If not necessary, DO NOT use + them. Default None. The following options are currently supported: - (1) output_spec (list[Tensor]): Selects the output targets of the saved model. - By default, all return variables of original Layer's forward method are kept as the - output of the saved model. If the provided ``output_spec`` list is not all output variables, - the saved model will be pruned according to the given ``output_spec`` list. + (1) output_spec (list[Tensor]): Selects the output targets of + the saved model. By default, all return variables of original + Layer's forward method are kept as the output of the saved model. + If the provided ``output_spec`` list is not all output variables, + the saved model will be pruned according to the given + ``output_spec`` list. Returns: None """ - assert isinstance( - layer, dygraph.Layer), "model must be the instance of dygraph.Layer" - is_dynamic_mode = False + assert isinstance(layer, dygraph.Layer), \ + "The model must be the instance of dygraph.Layer." + + self._gather_output_scale(layer) + with dygraph.guard(): layer.eval() for handle in self._register_hook_handle_list: handle.remove() - for key in self._out_scale_dict: - self._out_scale_dict[key] = float(self._out_scale_dict[key] - .numpy()) + paddle.jit.save(layer=layer, path=path, input_spec=input_spec, **config) + + if len(self._out_scale_dict) == 0: + warnings.warn("Warning: No Layer of the model while to be " \ + "saved contains the out_threshold attribute, so the " \ + "generated inference model would not contain the " \ + "out_threshold.") + return + # load static model + is_dynamic_mode = False if paddle.in_dynamic_mode(): is_dynamic_mode = True paddle.enable_static() - paddle.jit.save(layer=layer, path=path, input_spec=input_spec, **config) - - if core.is_compiled_with_cuda(): - place = core.CUDAPlace(0) - else: - place = core.CPUPlace() + place = core.CUDAPlace(0) if core.is_compiled_with_cuda() \ + else core.CPUPlace() exe = Executor(place) - file_prefix = os.path.basename(path) dirname = os.path.dirname(path) - model_filename = file_prefix + INFER_MODEL_SUFFIX - params_filename = file_prefix + INFER_PARAMS_SUFFIX + basename = os.path.basename(path) + model_filename = basename + INFER_MODEL_SUFFIX + params_filename = basename + INFER_PARAMS_SUFFIX - [inference_program, feed_target_names, fetch_targets] = ( + [infer_program, feed_target_names, fetch_targets] = ( load_inference_model( dirname=dirname, executor=exe, model_filename=model_filename, params_filename=params_filename)) - # Traverse all ops in the program and find out the op matching - # the Layer in the dynamic graph. - layer_var_dict = collections.OrderedDict() - ops_list = [key for key, _ in self._out_scale_dict.items()] - op_count = 0 - conv_count = 0 + # TODO(jc): analyse whether the dygraph model has + # several blocks before applying qat + assert infer_program.num_blocks == 1, \ + "Quantization aware training (QAT) requires the program " \ + "only has a block for now. When the model has if-else or " \ + "while, the program will have several blocks." - for block in inference_program.blocks: - for op in block.ops: - if op.type in _op_real_in_out_name: - if op.type in ["batch_norm", "pool2d"]: - if op.type == "pool2d" and op.attr( - "pooling_type") != "max": - continue - op_count = self.op_match(op, ops_list, op_count) - if op_count >= len(ops_list): - continue - op._set_attr('out_threshold', - self._out_scale_dict[ops_list[op_count]]) - op_count += 1 - else: - output_var_names = quantization_pass._get_op_output_var_names( - op) - for output_var_name in output_var_names: - output_var_tensor = block.var(output_var_name) - if output_var_tensor.dtype not in [ - core.VarDesc.VarType.FP64, - core.VarDesc.VarType.FP32 - ]: - continue - # Because the Layer in dygraph may correspond to multiple ops - # in static program after being saved. To ensure correctness, - # the outscale collected for output of dygraph Layer can only - # be set to the last op in the corresponding ops in static program. - # - # We can judge the execution order of the ops which corresponding - # to dygraph Layer by the name of output. And use dict to save - # the corresponding relationship between the dygraph Layer and the - # static graph op that needs to set the outscale attribute. - if '.' not in output_var_name: - continue - dynamic_layer_name, var_name_suffix = output_var_name.split( - ".") - if dynamic_layer_name in layer_var_dict: - if layer_var_dict[dynamic_layer_name][ - 0] < var_name_suffix: - layer_var_dict[dynamic_layer_name] = [ - var_name_suffix, op - ] - else: - layer_var_dict[dynamic_layer_name] = [ - var_name_suffix, op - ] - - # Because the naming styles of static and dynamic graph are different, - # in order to avoid mistakes, we unify the name here. - for (layer_name, var_name_op_list) in layer_var_dict.items(): - if 'prelu' in layer_name: - layer_name = layer_name.replace('prelu', 'p_re_lu') - if 'relu' in layer_name: - layer_name = layer_name.replace('relu', 're_lu') - if 'conv2d' in layer_name: - layer_name = 'conv2d_' + str(conv_count) - conv_count = conv_count + 1 - if layer_name not in self._out_scale_dict: - continue - var_name_op_list[1]._set_attr('out_threshold', - self._out_scale_dict[layer_name]) + # set output scales to the static model + self._save_output_scale(infer_program) - # Save the processed program. + # process skip quant + self._set_skip_quant_attr(infer_program) + + # save the final quantized model that has output scales save_inference_model( dirname=dirname, feeded_var_names=feed_target_names, target_vars=fetch_targets, executor=exe, - main_program=inference_program.clone(), + main_program=infer_program.clone(), model_filename=model_filename, params_filename=params_filename) if is_dynamic_mode: paddle.disable_static() - def op_match(self, op, ops_list, op_count): - while op_count < len(ops_list) and op.type not in ops_list[op_count]: - op_count += 1 - while op_count < len(ops_list) and op.type is "pool2d" and op.attr( - "pooling_type") != "max": - op_count += 1 - return op_count - - def _forward_post_hook(self, layer, input, output): - assert isinstance( - output, (core.VarBase, framework.Variable) - ), "Multiple outputs are not currently supported in ImperativeOutScale." - if output.dtype not in [ - core.VarDesc.VarType.FP32, core.VarDesc.VarType.FP64 - ]: + def _gather_output_scale(self, layer): + """ + Gather all output scales to self._out_scale_dict + """ + with dygraph.guard(): + layer.eval() + for _, sub_layer in layer.named_sublayers(): + if self._is_target_layer(sub_layer): + layer_name = sub_layer.full_name() + if hasattr(sub_layer, "_quant_out_scale"): + self._out_scale_dict[layer_name] = float( + sub_layer._quant_out_scale) + + def _save_output_scale(self, infer_program): + """ + Save all output scales to the corresponding ops in static + inference program. + + Because the Layer in dygraph may correspond to multiple ops + in static program after being saved. To ensure correctness, + the outscale collected for output of dygraph Layer can only + be set to the last op in the corresponding ops in static program. + """ + assert infer_program.num_blocks == 1, \ + "The inference program should only have a block." + + global_block = infer_program.global_block() + target_ops = global_block.ops + + scale_idx = 0 + op_idx = 0 + attr_name = "out_threshold" + + for scale_name, scale_value in self._out_scale_dict.items(): + while True: + if op_idx >= len(target_ops): + break + + op = target_ops[op_idx] + if not self._is_scale_op_matched(scale_name, op, global_block): + op_idx += 1 + else: + if op.type in utils.weight_op_types \ + and op_idx + 1 < len(target_ops) \ + and target_ops[op_idx+1].type == "elementwise_add": + target_ops[op_idx + 1]._set_attr(attr_name, scale_value) + op_idx += 2 + else: + op._set_attr(attr_name, scale_value) + op_idx += 1 + scale_idx += 1 + break + + if scale_idx != len(self._out_scale_dict): + _logger.warning("Warning: the model have %s output scales, "\ + "but it only saves %s output scales." \ + % (len(self._out_scale_dict), scale_idx)) + + def _is_target_layer(self, layer): + return isinstance(layer, tuple(utils.quant_output_layers_map.values())) \ + or ('quantized_' in layer.full_name() and \ + 'quantized_noweight' not in layer.full_name()) + + def _init_scale_params(self, layer, name=None): + """ + Init the scale params for calculating output scales and save them in the + target layer. + After the users define the dygraph model, the hooks for calculating output + scales will not execute immediately. If the users load parameters form + checkpoint and save the quantized inference model immediately, the inference + model would not be saved successfully. Beacuse the dygraph_to_static requires + that the parameters created in __init__, but the uniqueness of hook make it + impossible to create parameters in __init__. To avoid this mistake, we define + the scale parameters in the beginning instead of hook. + """ + + def _create_param(in_layer, first_name, last_name, dtype): + prefix = '{}.{}'.format(first_name, last_name) \ + if first_name else 'outscale.{}'.format(last_name) + attr = ParamAttr( + name=unique_name.generate(prefix), + initializer=Constant(1), + trainable=False) + param = in_layer.create_parameter(shape=[1], attr=attr, dtype=dtype) + return param + + dtype = layer._dtype if layer._dtype is not None else "float32" + if dtype not in ["float32", "float64"]: return - if not hasattr(layer, "_out_scale"): - layer._out_scale = quant_nn.MovingAverageAbsMaxScale( - output.name, self._moving_rate, output.dtype) - scale_out = layer._out_scale(output) - if hasattr(layer, 'layer_name'): - layer_name = layer.layer_name - else: - layer_name = layer.full_name() - self._out_scale_dict[layer_name] = scale_out + + layer._quant_out_scale = _create_param(layer, name, "scale", dtype) + layer._quant_out_scale.stop_gradient = True + + layer._quant_out_state = _create_param(layer, name, "state", dtype) + layer._quant_out_state.stop_gradient = True + + layer._quant_out_accum = _create_param(layer, name, "accum", dtype) + layer._quant_out_accum.stop_gradient = True + + def _is_scale_op_matched(self, scale_name, op, block): + """ + Based on the op name and attrs to judge whether the op in + program matches the scale_name. We must know the corresponding + name between dgraph and static model. + """ + fp_type = [core.VarDesc.VarType.FP64, core.VarDesc.VarType.FP32] + if op.type in quantization_pass._op_real_in_out_name.keys(): + output_var_names = quantization_pass._get_op_output_var_names(op) + for output_var_name in output_var_names: + output_var_tensor = block.var(output_var_name) + if output_var_tensor.dtype not in fp_type: + return False + + # corresponding_map: [name, op_types, function] + # Note that, the items have priority in corresponding_map + corresponding_map = [ + ['conv2d_tranpose', ['conv2d_transpose', \ + 'depthwise_conv2d_transpose'], None], + ['conv2d', ['conv2d', 'depthwise_conv2d'], None], + ['linear', ['matmul'], None], + ['re_lu6', ['relu6'], None], + ['p_re_lu', ['prelu'], None], + ['leaky_re_lu', ['leaky_relu'], None], + ['re_lu', ['relu'], None], + ] + + for item in corresponding_map: + if item[0] in scale_name: + return (op.type in item[1]) and \ + (len(item) == 2 or item[2] is None or item[2](op)) + + return op.type in scale_name + + def _set_skip_quant_attr(self, program): + block = program.global_block() + for op in block.ops: + if self._is_skip_quant_op(block, op): + op._set_attr("skip_quant", True) + + def _is_skip_quant_op(self, block, in_op): + """ + The input op should be skipped quantization. + 1. the type of input op should be conv2d, depthwise_conv2d or matmul + 2. the previous ops of the input op are not fake_quantize_dequantize ops + """ + + def _find_previous_op(block, var_name): + for op in block.ops: + if var_name in op.output_arg_names: + return op + + target_op_types = ["conv2d", "depthwise_conv2d", "matmul"] + if in_op.type not in target_op_types: + return False + + previous_ops = [_find_previous_op(block, arg_name) \ + for arg_name in in_op.input_arg_names] + return any(op is not None and op.type not in utils.fake_quantize_dequantize_types \ + for op in previous_ops ) + + def _calc_output_scale_hook(self, layer, input, output): + """ + Create the MovingAverageAbsMaxScale layer for the target layer if needed. + Execute MovingAverageAbsMaxScale layer to calculate the output scale. + """ + assert isinstance(output, (core.VarBase, framework.Variable)), \ + "Multiple outputs are not currently supported in ImperativeOutScale." + + fp_types = [core.VarDesc.VarType.FP32, core.VarDesc.VarType.FP64] + if output.dtype in fp_types: + if not hasattr(layer, "_out_scale"): + self._out_scale = quant_nn.MovingAverageAbsMaxScale( + layer, output.name, self._moving_rate, output.dtype) + # TODO (jc): consider the ops that have several outputs + self._out_scale(output) diff --git a/python/paddle/fluid/contrib/slim/quantization/imperative/quant_nn.py b/python/paddle/fluid/contrib/slim/quantization/imperative/quant_nn.py index 0469de7aef207..3c4fb323bc505 100644 --- a/python/paddle/fluid/contrib/slim/quantization/imperative/quant_nn.py +++ b/python/paddle/fluid/contrib/slim/quantization/imperative/quant_nn.py @@ -499,11 +499,15 @@ def __init__(self, def forward(self, input): quant_input = self._fake_quant_input(input) + # TODO (jc): support ops that have several inputs + if isinstance(input, list): + assert len(input) == 1, \ + "The QuantizedNoweightLayer should only have one input." return self._layer.forward(quant_input) class MovingAverageAbsMaxScale(layers.Layer): - def __init__(self, name=None, moving_rate=0.9, dtype='float32'): + def __init__(self, layer=None, name=None, moving_rate=0.9, dtype='float32'): r""" MovingAverageMaxScale layer is used to calculating the output quantization scale of Layer. Its computational formula is described as below: @@ -514,33 +518,48 @@ def __init__(self, name=None, moving_rate=0.9, dtype='float32'): super(MovingAverageAbsMaxScale, self).__init__() self._moving_rate = moving_rate self._dtype = dtype + self._layer = layer - scale_prefix = '{}.scale'.format(name) if name else 'outscale.scale' - name = unique_name.generate(scale_prefix) - scale_attr = ParamAttr( - name=name, initializer=Constant(1), trainable=False) - self._scale = self.create_parameter( - shape=[1], attr=scale_attr, dtype=self._dtype) - self._scale.stop_gradient = True + if self._layer is None or not hasattr(self._layer, "_quant_out_scale"): + scale_prefix = '{}.scale'.format(name) if name else 'outscale.scale' + scale_name = unique_name.generate(scale_prefix) + scale_attr = ParamAttr( + name=scale_name, initializer=Constant(1), trainable=False) + self._scale = self.create_parameter( + shape=[1], attr=scale_attr, dtype=self._dtype) + self._scale.stop_gradient = True + if self._layer is not None: + setattr(self._layer, "_quant_out_scale", self._scale) + else: + self._scale = self._layer._quant_out_scale - state_prefix = "{}.state".format(name) if name else 'outscale.state' - state_attr = ParamAttr( - name=unique_name.generate(state_prefix), - initializer=Constant(1), - trainable=False) - self._state = self.create_parameter( - shape=[1], attr=state_attr, dtype=self._dtype) - self._state.stop_gradient = True + if self._layer is None or not hasattr(self._layer, "_quant_out_state"): + state_prefix = "{}.state".format(name) if name else 'outscale.state' + state_attr = ParamAttr( + name=unique_name.generate(state_prefix), + initializer=Constant(1), + trainable=False) + self._state = self.create_parameter( + shape=[1], attr=state_attr, dtype=self._dtype) + self._state.stop_gradient = True + if self._layer is not None: + setattr(self._layer, "_quant_out_state", self._state) + else: + self._state = self._layer._quant_out_state - accum_prefix = "{}.accum".format(name) if name else 'outscale.accum' - accum_attr = ParamAttr( - name=unique_name.generate(accum_prefix), - initializer=Constant(1), - trainable=False) - self._accum = self.create_parameter( - shape=[1], attr=accum_attr, dtype=self._dtype) - self._accum.stop_gradient = True - MovingAverageAbsMaxScale._has_create = True + if self._layer is None or not hasattr(self._layer, "_quant_out_accum"): + accum_prefix = "{}.accum".format(name) if name else 'outscale.accum' + accum_attr = ParamAttr( + name=unique_name.generate(accum_prefix), + initializer=Constant(1), + trainable=False) + self._accum = self.create_parameter( + shape=[1], attr=accum_attr, dtype=self._dtype) + self._accum.stop_gradient = True + if self._layer is not None: + setattr(self._layer, "_quant_out_accum", self._accum) + else: + self._accum = self._layer._quant_out_accum def forward(self, input): if in_dygraph_mode(): @@ -549,18 +568,17 @@ def forward(self, input): state = self._state if self.training else None accum = self._accum if self.training else None - out_scale, _, _ = core.ops.moving_average_abs_max_scale( + self._scale, _, _ = core.ops.moving_average_abs_max_scale( input, accum, state, self._scale, state, accum, *attrs) - return out_scale + return self._scale check_variable_and_dtype(input, 'input', ['float32', 'float64'], 'MovingAverageAbsMaxScale') - scale_out = self._scale attrs = {'moving_rate': self._moving_rate, 'is_test': not self.training} inputs = {"X": [input]} - outputs = {"OutScale": [scale_out]} + outputs = {"OutScale": [self._scale]} if self.training: inputs['InState'] = [self._state] @@ -574,4 +592,4 @@ def forward(self, input): outputs=outputs, attrs=attrs) - return scale_out + return self._scale diff --git a/python/paddle/fluid/contrib/slim/quantization/imperative/utils.py b/python/paddle/fluid/contrib/slim/quantization/imperative/utils.py new file mode 100644 index 0000000000000..090f6cda389af --- /dev/null +++ b/python/paddle/fluid/contrib/slim/quantization/imperative/utils.py @@ -0,0 +1,87 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import paddle + +op_real_in_out_name = { + "conv2d": [["Input", "Filter"], ["Output"]], + "depthwise_conv2d": [["Input", "Filter"], ["Output"]], + "pool2d": [["X"], ["Out"]], + "elementwise_add": [["X", "Y"], ["Out"]], + "softmax": [["X"], ["Out"]], + "relu": [["X"], ["Out"]], + "relu6": [["X"], ["Out"]], + "leaky_relu": [["X"], ["Out"]], + "prelu": [["X"], ["Out"]], + "tanh": [["X"], ["Out"]], + "batch_norm": [["X"], ["Y"]], + "sigmoid": [["X"], ["Out"]], + "swish": [["X"], ["Out"]], +} + +quant_input_layers_map = { + 'Conv2D': paddle.nn.Conv2D, + 'Linear': paddle.nn.Linear, + 'AdaptiveAvgPool2D': paddle.nn.AdaptiveAvgPool2D, + 'AdaptiveMaxPool2D': paddle.nn.AdaptiveMaxPool2D, + 'AvgPool2D': paddle.nn.AvgPool2D, + 'MaxPool2D': paddle.nn.MaxPool2D, + 'Hardswish': paddle.nn.Hardswish, + 'LeakyReLU': paddle.nn.LeakyReLU, + 'PReLU': paddle.nn.PReLU, + 'ReLU': paddle.nn.ReLU, + 'ReLU6': paddle.nn.ReLU6, + 'Sigmoid': paddle.nn.Sigmoid, + 'Softmax': paddle.nn.Softmax, + 'Swish': paddle.nn.Swish, + 'Tanh': paddle.nn.Tanh, + 'Hardswish': paddle.nn.Hardswish, + 'BatchNorm': paddle.nn.BatchNorm, + 'GroupNorm': paddle.nn.GroupNorm, + 'LayerNorm': paddle.nn.LayerNorm, +} + +fake_quantize_dequantize_types = [ + "fake_quantize_dequantize_abs_max", + "fake_quantize_dequantize_channel_wise_abs_max", + "fake_quantize_dequantize_moving_average_abs_max" +] + +quant_output_layers_map = { + 'Conv2D': paddle.nn.Conv2D, + 'Conv2DTranspose': paddle.nn.Conv2DTranspose, + 'Linear': paddle.nn.Linear, + 'AdaptiveAvgPool2D': paddle.nn.AdaptiveAvgPool2D, + 'AdaptiveMaxPool2D': paddle.nn.AdaptiveMaxPool2D, + 'AvgPool2D': paddle.nn.AvgPool2D, + 'MaxPool2D': paddle.nn.MaxPool2D, + 'BatchNorm': paddle.nn.BatchNorm, + 'BatchNorm2D': paddle.nn.BatchNorm2D, + 'SyncBatchNorm': paddle.nn.SyncBatchNorm, + 'ELU': paddle.nn.ELU, + 'GELU': paddle.nn.GELU, + 'LeakyReLU': paddle.nn.LeakyReLU, + 'PReLU': paddle.nn.PReLU, + 'ReLU': paddle.nn.ReLU, + 'ReLU6': paddle.nn.ReLU6, + 'Sigmoid': paddle.nn.Sigmoid, + 'Softmax': paddle.nn.Softmax, + 'Tanh': paddle.nn.Tanh, + 'Swish': paddle.nn.Swish, +} + +weight_op_types = [ + "conv2d", "depthwise_conv2d", "matmul", "conv2d_transpose", + "depthwise_conv2d_transpose" +] diff --git a/python/paddle/fluid/contrib/slim/quantization/quant2_int8_mkldnn_pass.py b/python/paddle/fluid/contrib/slim/quantization/quant2_int8_mkldnn_pass.py index d93a2059bdcf0..68cc8106c9c07 100644 --- a/python/paddle/fluid/contrib/slim/quantization/quant2_int8_mkldnn_pass.py +++ b/python/paddle/fluid/contrib/slim/quantization/quant2_int8_mkldnn_pass.py @@ -62,9 +62,8 @@ def __init__(self, self._ops_to_quantize = _ops_to_quantize self._op_ids_to_skip = _op_ids_to_skip if _op_ids_to_skip is not None else set( [-1]) - self._scale_immutable_ops = [ - 'transpose2', 'reshape2', 'pool2d', 'scale' - ] + self._scale_immutable_ops = ['transpose2', 'reshape2', 'pool2d'] + self._scale_ops = ['scale'] self._conv_ops = ['conv2d', 'depthwise_conv2d'] self._pool_ops = ['pool2d'] self._mul_ops = ['mul'] @@ -87,8 +86,8 @@ def apply(self, graph): self._reset_pass_idx_and_group('int8') graph = self._label_skip_quantized_op(graph) graph = self._gather_weight_thresholds_from_fake(graph) - graph = self._gather_output_scales_from_attr(graph) graph = self._gather_input_scales_from_fake(graph) + graph = self._gather_output_scales_from_attr(graph) graph = self._remove_fake_ops(graph) graph = self._dequantize_weights(graph) graph = self._optimize_fp32_graph(graph) @@ -160,12 +159,16 @@ def _label_skip_quantized_op(self, graph): op_node.op()._set_attr("skip_quant", True) return graph - def _gather_input_scales_from_fake(self, graph): - def _add_scale_for_vars(var_names, use_unsigned_int, lod_tensor): - scales = self._var_quant_scales - for var_name in var_names: + def _add_scale_for_vars(self, var_names, use_unsigned_int, lod_tensor): + """ + Save quantization scales for variables. Do not overwrite. + """ + scales = self._var_quant_scales + for var_name in var_names: + if var_name not in scales: scales[var_name] = (use_unsigned_int, lod_tensor) + def _gather_input_scales_from_fake(self, graph): # fake_quantize_dequantize_abs_max doesn't have scale value fake_ops = ['fake_quantize_dequantize_moving_average_abs_max'] fake_ops.extend(self._fake_quantize_types) @@ -185,8 +188,8 @@ def _add_scale_for_vars(var_names, use_unsigned_int, lod_tensor): scale[scale == np.Inf] = 0.0 lod_tensor = self._convert_scale2tensor(scale) use_unsigned_int = False - _add_scale_for_vars([input_name, output_name], use_unsigned_int, - lod_tensor) + self._add_scale_for_vars([input_name, output_name], + use_unsigned_int, lod_tensor) return graph @@ -219,8 +222,8 @@ def _gather_output_scales_from_attr(self, graph): use_unsigned_int = False for output_name in op.op().outputs(): for out_var_name in op.op().output(output_name): - self._var_quant_scales[out_var_name] = ( - use_unsigned_int, scale_lod_tensor) + self._add_scale_for_vars( + [out_var_name], use_unsigned_int, scale_lod_tensor) return graph @@ -239,24 +242,21 @@ def _update_scales(graph): output_name = op.output("Out")[0] tensor_names = [input_name, output_name] - # Scale is not quantized, so if it doesn't have any scales - # to propagate, its tensors won't be added to the waiting list. - if all(name not in self._var_quant_scales for name in tensor_names) \ - and op.name() != 'scale': + if all(name not in self._var_quant_scales + for name in tensor_names): waiting_for_scale.update(tensor_names) continue - - if input_name in self._var_quant_scales: + elif input_name in self._var_quant_scales: self._var_quant_scales[ output_name] = self._var_quant_scales[input_name] elif output_name in self._var_quant_scales: - if op.name() == 'scale': - _update_scale_op_in_scale(op, input_name, - output_name) - else: - self._var_quant_scales[ - input_name] = self._var_quant_scales[ - output_name] + self._var_quant_scales[ + input_name] = self._var_quant_scales[output_name] + elif op.name() in self._scale_ops: + input_name = op.input("X")[0] + output_name = op.output("Out")[0] + if output_name in self._var_quant_scales: + _update_scale_op_in_scale(op, input_name, output_name) return waiting_for_scale waiting_for_scale = _update_scales(graph) diff --git a/python/paddle/fluid/contrib/slim/tests/test_imperative_out_scale.py b/python/paddle/fluid/contrib/slim/tests/test_imperative_out_scale.py index 47e21910b48df..600174e503feb 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_imperative_out_scale.py +++ b/python/paddle/fluid/contrib/slim/tests/test_imperative_out_scale.py @@ -19,6 +19,8 @@ import random import unittest import logging +import warnings + import paddle import paddle.fluid as fluid import paddle.fluid.layers as layers @@ -29,9 +31,8 @@ from paddle.fluid.contrib.slim.quantization import OutScaleForTrainingPass, OutScaleForInferencePass, QuantizationTransformPass from paddle.fluid.dygraph.container import Sequential from paddle.fluid.dygraph.io import INFER_MODEL_SUFFIX, INFER_PARAMS_SUFFIX -from paddle.nn.layer import ReLU, LeakyReLU, Sigmoid, Softmax, ReLU6 +from paddle.nn.layer import ReLU, LeakyReLU, Sigmoid, Softmax, PReLU from paddle.nn import Linear, Conv2D, Softmax, BatchNorm2D, MaxPool2D -from paddle.fluid.dygraph.nn import Pool2D from paddle.fluid.log_helper import get_logger from paddle.fluid.dygraph import nn @@ -45,6 +46,14 @@ __name__, logging.INFO, fmt='%(asctime)s-%(levelname)s: %(message)s') +def get_vaild_warning_num(warning, w): + num = 0 + for i in range(len(w)): + if warning in str(w[i].message): + num += 1 + return num + + def StaticLenet(data, num_classes=10, classifier_activation='softmax'): conv2d_w1_attr = fluid.ParamAttr(name="conv2d_w_1") conv2d_w2_attr = fluid.ParamAttr(name="conv2d_w_2") @@ -76,9 +85,9 @@ def StaticLenet(data, num_classes=10, classifier_activation='softmax'): param_attr=conv2d_w2_attr, bias_attr=conv2d_b2_attr) batch_norm2 = layers.batch_norm(conv2) - relu6_1 = layers.relu6(batch_norm2) + prelu1 = layers.prelu(batch_norm2, mode='all') pool2 = fluid.layers.pool2d( - relu6_1, pool_size=2, pool_type='max', pool_stride=2) + prelu1, pool_size=2, pool_type='max', pool_stride=2) fc1 = fluid.layers.fc(input=pool2, size=120, @@ -121,8 +130,8 @@ def __init__(self, num_classes=10): bias_attr=False), BatchNorm2D(6), ReLU(), - Pool2D( - pool_size=2, pool_type='max', pool_stride=2), + MaxPool2D( + kernel_size=2, stride=2), Conv2D( in_channels=6, out_channels=16, @@ -132,7 +141,7 @@ def __init__(self, num_classes=10): weight_attr=conv2d_w2_attr, bias_attr=conv2d_b2_attr), BatchNorm2D(16), - ReLU6(), + PReLU(), MaxPool2D( kernel_size=2, stride=2)) @@ -246,6 +255,10 @@ def _build_static_lenet(main, startup, is_test=False, seed=1000): lenet.eval() + param_save_path = "test_save_quantized_model/lenet.pdparams" + save_dict = lenet.state_dict() + paddle.save(save_dict, param_save_path) + path = "./dynamic_outscale_infer_model/lenet" dynamic_save_dir = "./dynamic_outscale_infer_model" @@ -285,6 +298,8 @@ def _build_static_lenet(main, startup, is_test=False, seed=1000): for param in main.all_parameters(): if "batch_norm" in param.name: param_name = param.name.replace("norm", "norm2d") + elif 'prelu' in param.name: + param_name = param.name.replace("prelu", 'p_re_lu') else: param_name = param.name param_tensor = scope.var(param.name).get_tensor() @@ -341,7 +356,6 @@ def _build_static_lenet(main, startup, is_test=False, seed=1000): "diff({}) at {}, dynamic loss = {}, static loss = {}". format(diff, i, loss_d, loss_s)) break - self.assertTrue( np.allclose( np.array(dynamic_loss_rec), @@ -377,12 +391,117 @@ def _build_static_lenet(main, startup, is_test=False, seed=1000): if 'fake' in op.type: static_ops.remove(op) + op_count = 0 for i in range(len(dynamic_ops)): if dynamic_ops[i].has_attr("out_threshold"): + op_count += 1 self.assertTrue(dynamic_ops[i].type == static_ops[i].type) + if dynamic_ops[i].attr("out_threshold") != static_ops[i].attr( + "out_threshold"): + _logger.info(dynamic_ops[i].attr("out_threshold")) + _logger.info(static_ops[i].attr("out_threshold")) self.assertTrue(dynamic_ops[i].attr("out_threshold") == static_ops[i].attr("out_threshold")) + _logger.info("op_cout: {}".format(op_count)) + self.assertTrue(op_count == 14) + + +class TestSaveQuanztizedModelFromCheckPoint(unittest.TestCase): + def test_save_quantized_model(self): + weight_quantize_type = 'abs_max' + activation_quantize_type = 'moving_average_abs_max' + load_param_path = "test_save_quantized_model/lenet.pdparams" + path = "./dynamic_outscale_infer_model_from_checkpoint/lenet" + dynamic_model_save_dir = "./dynamic_outscale_infer_model_from_checkpoint" + static_model_save_dir = "./static_outscale_infer_model" + + imperative_out_scale = ImperativeQuantAware( + weight_quantize_type=weight_quantize_type, + activation_quantize_type=activation_quantize_type) + + with fluid.dygraph.guard(): + lenet = ImperativeLenet() + load_dict = paddle.load(load_param_path) + imperative_out_scale.quantize(lenet) + lenet.set_dict(load_dict) + + imperative_out_scale.save_quantized_model( + layer=lenet, + path=path, + input_spec=[ + paddle.static.InputSpec( + shape=[None, 1, 28, 28], dtype='float32') + ]) + + if core.is_compiled_with_cuda(): + place = core.CUDAPlace(0) + else: + place = core.CPUPlace() + exe = fluid.Executor(place) + + # load dynamic model + [dynamic_inference_program, feed_target_names, fetch_targets] = ( + fluid.io.load_inference_model( + dirname=dynamic_model_save_dir, + executor=exe, + model_filename="lenet" + INFER_MODEL_SUFFIX, + params_filename="lenet" + INFER_PARAMS_SUFFIX)) + # load static model + [static_inference_program, feed_target_names, fetch_targets] = ( + fluid.io.load_inference_model( + dirname=static_model_save_dir, + executor=exe, + model_filename="lenet" + INFER_MODEL_SUFFIX, + params_filename="lenet" + INFER_PARAMS_SUFFIX)) + + dynamic_ops = dynamic_inference_program.global_block().ops + static_ops = static_inference_program.global_block().ops + + for op in dynamic_ops[:]: + if op.type == "flatten2" or 'fake' in op.type: + dynamic_ops.remove(op) + + for op in static_ops[:]: + if 'fake' in op.type: + static_ops.remove(op) + + op_count = 0 + for i in range(len(dynamic_ops)): + if dynamic_ops[i].has_attr("out_threshold"): + op_count += 1 + self.assertTrue(dynamic_ops[i].type == static_ops[i].type) + self.assertTrue(dynamic_ops[i].attr("out_threshold") == + static_ops[i].attr("out_threshold")) + + _logger.info("op_cout: {}".format(op_count)) + self.assertTrue(op_count == 14) + + +class TestSaveQuantizedModel_Warning(unittest.TestCase): + def test_warning(self): + path = "./dynamic_outscale_infer_model_with_warnings/lenet" + imperative_out_scale = ImperativeQuantAware() + with fluid.dygraph.guard(): + lenet = ImperativeLenet() + + with warnings.catch_warnings(record=True) as w: + warnings.simplefilter("always") + imperative_out_scale.save_quantized_model( + layer=lenet, + path=path, + input_spec=[ + paddle.static.InputSpec( + shape=[None, 1, 28, 28], dtype='float32') + ]) + + warning_message = "Warning: No Layer of the model while to be " \ + "saved contains the out_threshold attribute, so the " \ + "generated inference model would not contain the " \ + "out_threshold." + num = get_vaild_warning_num(warning_message, w) + assert num == 1 + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/contrib/slim/tests/test_imperative_qat_addquantdequant.py b/python/paddle/fluid/contrib/slim/tests/test_imperative_qat_addquantdequant.py index 9d2b2d726e35f..d76e4825e0d62 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_imperative_qat_addquantdequant.py +++ b/python/paddle/fluid/contrib/slim/tests/test_imperative_qat_addquantdequant.py @@ -191,8 +191,8 @@ def test_qat_save(self): weight_quantize_type='abs_max', activation_quantize_type='moving_average_abs_max', quantizable_layer_type=[ - 'Conv2D', 'Linear', 'ReLU', 'Pool2D', 'LeakyReLU', 'ReLU6', - 'Tanh', 'Swish' + 'Conv2D', 'Linear', 'ReLU', 'LeakyReLU', 'ReLU6', 'Tanh', + 'Swish' ]) with fluid.dygraph.guard(): diff --git a/python/paddle/fluid/contrib/slim/tests/test_imperative_skip_op.py b/python/paddle/fluid/contrib/slim/tests/test_imperative_skip_op.py index 0561055e6e057..bda02769cea86 100644 --- a/python/paddle/fluid/contrib/slim/tests/test_imperative_skip_op.py +++ b/python/paddle/fluid/contrib/slim/tests/test_imperative_skip_op.py @@ -200,9 +200,12 @@ def test_out_scale_acc(self): params_filename="lenet" + INFER_PARAMS_SUFFIX)) model_ops = inference_program.global_block().ops - conv2d_count, mul_count = 0, 0 + conv2d_count, matmul_count = 0, 0 + conv2d_skip_count, matmul_skip_count = 0, 0 for i, op in enumerate(model_ops): if op.type == 'conv2d': + if op.has_attr("skip_quant"): + conv2d_skip_count += 1 if conv2d_count > 0: self.assertTrue( 'fake_quantize_dequantize' in model_ops[i - 1].type) @@ -211,14 +214,19 @@ def test_out_scale_acc(self): 'fake_quantize_dequantize' not in model_ops[i - 1].type) conv2d_count += 1 - if op.type == 'mul': - if mul_count > 0: + if op.type == 'matmul': + if op.has_attr("skip_quant"): + matmul_skip_count += 1 + if matmul_count > 0: self.assertTrue( 'fake_quantize_dequantize' in model_ops[i - 1].type) else: self.assertTrue( 'fake_quantize_dequantize' not in model_ops[i - 1].type) - mul_count += 1 + matmul_count += 1 + + self.assertTrue(conv2d_skip_count == 1) + self.assertTrue(matmul_skip_count == 1) if __name__ == '__main__': diff --git a/python/paddle/fluid/contrib/tests/test_bf16_utils.py b/python/paddle/fluid/contrib/tests/test_bf16_utils.py new file mode 100644 index 0000000000000..faf2307f8147b --- /dev/null +++ b/python/paddle/fluid/contrib/tests/test_bf16_utils.py @@ -0,0 +1,144 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. +import copy +import unittest +import paddle.fluid as fluid +import paddle.fluid.contrib.mixed_precision as amp +from paddle.fluid import core +import paddle + +paddle.enable_static() + + +class AMPTest(unittest.TestCase): + def setUp(self): + self.bf16_list = copy.copy(amp.bf16.amp_lists.bf16_list) + self.fp32_list = copy.copy(amp.bf16.amp_lists.fp32_list) + self.gray_list = copy.copy(amp.bf16.amp_lists.gray_list) + self.amp_lists_ = None + + def tearDown(self): + self.assertEqual(self.amp_lists_.bf16_list, self.bf16_list) + self.assertEqual(self.amp_lists_.fp32_list, self.fp32_list) + self.assertEqual(self.amp_lists_.gray_list, self.gray_list) + + def test_amp_lists(self): + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16() + + def test_amp_lists_1(self): + # 1. w={'exp}, b=None + self.bf16_list.add('exp') + self.fp32_list.remove('exp') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16({'exp'}) + + def test_amp_lists_2(self): + # 2. w={'tanh'}, b=None + self.fp32_list.remove('tanh') + self.bf16_list.add('tanh') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16({'tanh'}) + + def test_amp_lists_3(self): + # 3. w={'lstm'}, b=None + self.bf16_list.add('lstm') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16({'lstm'}) + + def test_amp_lists_4(self): + # 4. w=None, b={'elementwise_add'} + self.bf16_list.remove('elementwise_add') + self.fp32_list.add('elementwise_add') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16( + custom_fp32_list={'elementwise_add'}) + + def test_amp_lists_5(self): + # 5. w=None, b={'elementwise_add'} + self.fp32_list.add('elementwise_add') + self.bf16_list.remove('elementwise_add') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16( + custom_fp32_list={'elementwise_add'}) + + def test_amp_lists_6(self): + # 6. w=None, b={'lstm'} + self.fp32_list.add('lstm') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16( + custom_fp32_list={'lstm'}) + + def test_amp_lists_7(self): + self.fp32_list.add('reshape2') + self.gray_list.remove('reshape2') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16( + custom_fp32_list={'reshape2'}) + + def test_amp_list_8(self): + self.bf16_list.add('reshape2') + self.gray_list.remove('reshape2') + + self.amp_lists_ = amp.AutoMixedPrecisionListsBF16( + custom_bf16_list={'reshape2'}) + + +class AMPTest2(unittest.TestCase): + def test_amp_lists_(self): + # 7. w={'lstm'} b={'lstm'} + # raise ValueError + self.assertRaises(ValueError, amp.AutoMixedPrecisionListsBF16, + {'lstm'}, {'lstm'}) + + def test_find_op_index(self): + block = fluid.default_main_program().global_block() + op_desc = core.OpDesc() + idx = amp.bf16.amp_utils.find_op_index(block.desc, op_desc) + assert (idx == -1) + + def test_is_in_fp32_varnames(self): + block = fluid.default_main_program().global_block() + + var1 = block.create_var(name="X", shape=[3], dtype='float32') + var2 = block.create_var(name="Y", shape=[3], dtype='float32') + var3 = block.create_var(name="Z", shape=[3], dtype='float32') + op1 = block.append_op( + type="abs", inputs={"X": [var1]}, outputs={"Out": [var2]}) + op2 = block.append_op( + type="abs", inputs={"X": [var2]}, outputs={"Out": [var3]}) + amp_lists_1 = amp.AutoMixedPrecisionListsBF16( + custom_fp32_varnames={'X'}) + assert amp.bf16.amp_utils._is_in_fp32_varnames(op1, amp_lists_1) + amp_lists_2 = amp.AutoMixedPrecisionListsBF16( + custom_fp32_varnames={'Y'}) + assert amp.bf16.amp_utils._is_in_fp32_varnames(op2, amp_lists_2) + assert amp.bf16.amp_utils._is_in_fp32_varnames(op1, amp_lists_2) + + def test_find_true_post_op(self): + + block = fluid.default_main_program().global_block() + + var1 = block.create_var(name="X", shape=[3], dtype='float32') + var2 = block.create_var(name="Y", shape=[3], dtype='float32') + var3 = block.create_var(name="Z", shape=[3], dtype='float32') + op1 = block.append_op( + type="abs", inputs={"X": [var1]}, outputs={"Out": [var2]}) + op2 = block.append_op( + type="abs", inputs={"X": [var2]}, outputs={"Out": [var3]}) + res = amp.bf16.amp_utils.find_true_post_op(block.ops, op1, "Y") + assert (res == [op2]) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/contrib/tests/test_model_cast_to_bf16.py b/python/paddle/fluid/contrib/tests/test_model_cast_to_bf16.py new file mode 100644 index 0000000000000..40ddcf2e66b75 --- /dev/null +++ b/python/paddle/fluid/contrib/tests/test_model_cast_to_bf16.py @@ -0,0 +1,138 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import paddle +import paddle.fluid as fluid +import contextlib +import unittest +import numpy as np +import paddle.fluid.layers as layers +import paddle.static.amp as amp +from paddle.fluid import core + +paddle.enable_static() + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestModelCastBF16(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.seed = 111 + + @classmethod + def tearDownClass(cls): + pass + + @contextlib.contextmanager + def static_graph(self): + with self.scope_prog_guard(): + paddle.seed(self.seed) + paddle.framework.random._manual_program_seed(self.seed) + yield + + @contextlib.contextmanager + def scope_prog_guard(self): + prog = fluid.Program() + startup_prog = fluid.Program() + scope = fluid.core.Scope() + with fluid.scope_guard(scope): + with fluid.program_guard(prog, startup_prog): + yield + + def get_static_graph_result(self, feed, fetch_list, amp_fun, + with_lod=False): + exe = fluid.Executor(core.CPUPlace()) + exe.run(fluid.default_startup_program()) + prog = fluid.default_main_program() + if amp_fun is not None: + amp_fun(prog) + return exe.run(prog, + feed=feed, + fetch_list=fetch_list, + return_numpy=(not with_lod)) + + def test_graph_rewrite(self): + size = 3 + n = np.ones([size, size], dtype='float32') * 3.2 + nn = np.ones([size, size], dtype='float32') * -2.7 + + n_bf16 = amp.convert_float_to_uint16(n) + nn_bf16 = amp.convert_float_to_uint16(nn) + + with self.static_graph(): + t_bf16 = layers.data( + name='t_bf16', shape=[size, size], dtype=np.uint16) + tt_bf16 = layers.data( + name='tt_bf16', shape=[size, size], dtype=np.uint16) + t = layers.data(name='t', shape=[size, size], dtype='float32') + tt = layers.data(name='tt', shape=[size, size], dtype='float32') + + ret = layers.elementwise_add(t, tt) + ret = layers.elementwise_mul(ret, t) + ret = layers.reshape(ret, [0, 0]) + + with amp.bf16_guard(): + ret_bf16 = layers.elementwise_add(t_bf16, tt_bf16) + ret_bf16 = layers.elementwise_mul(ret_bf16, t_bf16) + ret_bf16 = layers.reshape(ret_bf16, [0, 0]) + + with amp.bf16_guard(): + ret_fp32bf16 = layers.elementwise_add(t, tt) + ret_fp32bf16 = layers.elementwise_mul(ret_fp32bf16, t) + ret_fp32bf16 = layers.reshape(ret_fp32bf16, [0, 0]) + + static_ret_bf16, static_ret, ret_fp32bf16 = self.get_static_graph_result( + feed={ + 't': n, + 'tt': nn, + 't_bf16': n_bf16, + 'tt_bf16': nn_bf16, + }, + fetch_list=[ret_bf16, ret, ret_fp32bf16], + amp_fun=lambda prog: amp.rewrite_program_bf16(prog, use_bf16_guard=True)) + + self.assertTrue(np.allclose(static_ret_bf16, static_ret, 1e-2)) + self.assertTrue(np.allclose(static_ret_bf16, ret_fp32bf16, 1e-2)) + + with self.static_graph(): + t = layers.data(name='t', shape=[size, size], dtype='float32') + tt = layers.data(name='tt', shape=[size, size], dtype='float32') + + with amp.bf16_guard(): + ret = layers.elementwise_add(t, tt) + ret = layers.reshape(ret, [0, 0], act='elu') + ret = layers.elementwise_mul(ret, t) + ret = layers.elementwise_add(ret, tt) + + static_ret_bf16 = \ + self.get_static_graph_result( + feed={'t': n, 'tt': nn}, + fetch_list=[ret], + amp_fun=lambda prog: amp.rewrite_program_bf16( + prog, + amp.AutoMixedPrecisionListsBF16( + custom_fp32_varnames={'elementwise_add_0.tmp_0'}), + use_bf16_guard=True + ) + ) + self.assertTrue( + static_ret_bf16, np.ones( + [size, size], dtype='float32') * -1.1) + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/data_feeder.py b/python/paddle/fluid/data_feeder.py index b2db00296bf95..52be7493cf229 100644 --- a/python/paddle/fluid/data_feeder.py +++ b/python/paddle/fluid/data_feeder.py @@ -29,6 +29,7 @@ _PADDLE_DTYPE_2_NUMPY_DTYPE = { core.VarDesc.VarType.BOOL: 'bool', core.VarDesc.VarType.FP16: 'float16', + core.VarDesc.VarType.BF16: 'uint16', core.VarDesc.VarType.FP32: 'float32', core.VarDesc.VarType.FP64: 'float64', core.VarDesc.VarType.INT8: 'int8', @@ -47,16 +48,18 @@ def convert_dtype(dtype): return _PADDLE_DTYPE_2_NUMPY_DTYPE[dtype] elif isinstance(dtype, type): if dtype in [ - np.bool, np.float16, np.float32, np.float64, np.int8, np.int16, - np.int32, np.int64, np.uint8, np.complex64, np.complex128 + np.bool, np.float16, np.uint16, np.float32, np.float64, np.int8, + np.int16, np.int32, np.int64, np.uint8, np.complex64, + np.complex128 ]: return dtype.__name__ else: if dtype in [ - 'bool', 'float16', 'float32', 'float64', 'int8', 'int16', - 'int32', 'int64', 'uint8', 'complex64', 'complex128', u'bool', - u'float16', u'float32', u'float64', u'int8', u'int16', u'int32', - u'int64', u'uint8', u'complex64', u'complex128' + 'bool', 'float16', 'uint16', 'float32', 'float64', 'int8', + 'int16', 'int32', 'int64', 'uint8', 'complex64', 'complex128', + u'bool', u'float16', u'uint16', u'float32', u'float64', u'int8', + u'int16', u'int32', u'int64', u'uint8', u'complex64', + u'complex128' ]: # this code is a little bit dangerous, since error could happen # when casting no-ascii code to str in python2. @@ -66,7 +69,7 @@ def convert_dtype(dtype): return str(dtype) raise TypeError( - "dtype must be any of [bool, float16, float32, float64, int8, int16, " + "dtype must be any of [bool, float16, uint16, float32, float64, int8, int16, " "int32, int64, uint8, complex64, complex128], but received %s" % dtype) @@ -123,6 +126,12 @@ def check_dtype(input_dtype, warnings.warn( "The data type of '%s' in %s only support float16 in GPU now. %s" % (input_name, op_name, extra_message)) + if convert_dtype(input_dtype) in ['uint16'] and op_name not in [ + 'reshape', 'lookup_table', 'scale' + ]: + warnings.warn( + "The data type of '%s' in %s only support bfloat16 in OneDNN now. %s" + % (input_name, op_name, extra_message)) if convert_dtype(input_dtype) not in expected_dtype: raise TypeError( "The data type of '%s' in %s must be %s, but received %s. %s" % diff --git a/python/paddle/fluid/dataloader/collate.py b/python/paddle/fluid/dataloader/collate.py new file mode 100644 index 0000000000000..ddc010d04280c --- /dev/null +++ b/python/paddle/fluid/dataloader/collate.py @@ -0,0 +1,87 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import paddle +import numbers +import numpy as np +from ..framework import in_dygraph_mode +from .. import core, layers + +try: + from collections.abc import Sequence, Mapping +except: + from collections import Sequence, Mapping + + +def default_collate_fn(batch): + """ + Default batch collating function for :code:`paddle.io.DataLoader`, + batch should be a list of samples, and each sample should be a list + of fields as follows: + + [[filed1, filed2, ...], [filed1, filed2, ...], ...] + + This default collate function zipped each filed together and stack + each filed as the batch field as follows: + + [batch_filed1, batch_filed2, ...] + + Args: + batch(list of list of numpy array|paddle.Tensor): the batch data, each fields + should be a numpy array, each sample should be a list of + fileds, and batch should be a list of sample. + + Returns: + a list of numpy array|Paddle.Tensor: collated batch of input batch data, + fields data type as same as fields in each sample. + """ + sample = batch[0] + if isinstance(sample, np.ndarray): + batch = np.stack(batch, axis=0) + return batch + elif isinstance(sample, paddle.Tensor): + return layers.stack(batch, axis=0) + elif isinstance(sample, numbers.Number): + batch = np.array(batch) + return batch + elif isinstance(sample, (str, bytes)): + return batch + elif isinstance(sample, Mapping): + return { + key: default_collate_fn([d[key] for d in batch]) + for key in sample + } + elif isinstance(sample, Sequence): + sample_fields_num = len(sample) + if not all(len(sample) == sample_fields_num for sample in iter(batch)): + raise RuntimeError( + "fileds number not same among samples in a batch") + return [default_collate_fn(fields) for fields in zip(*batch)] + + raise TypeError("batch data con only contains: tensor, numpy.ndarray, " + "dict, list, number, but got {}".format(type(sample))) + return outputs + + +def default_convert_fn(batch): + if isinstance(batch, (paddle.Tensor, np.ndarray)): + return batch + elif isinstance(batch, (str, bytes)): + return batch + elif isinstance(batch, Mapping): + return {key: default_convert_fn(batch[key]) for key in batch} + elif isinstance(batch, Sequence): + return [default_convert_fn(d) for d in batch] + else: + return batch diff --git a/python/paddle/fluid/dataloader/dataloader_iter.py b/python/paddle/fluid/dataloader/dataloader_iter.py index 0dd2420691aea..0cd12e874d9e3 100644 --- a/python/paddle/fluid/dataloader/dataloader_iter.py +++ b/python/paddle/fluid/dataloader/dataloader_iter.py @@ -35,181 +35,16 @@ import paddle from .. import core, layers from ..framework import in_dygraph_mode -from ..multiprocess_utils import CleanupFuncRegistrar, _cleanup_mmap, _set_SIGCHLD_handler +from ..multiprocess_utils import _set_SIGCHLD_handler, MP_STATUS_CHECK_INTERVAL from .fetcher import _IterableDatasetFetcher, _MapDatasetFetcher from .batch_sampler import _InfiniteIterableSampler +from .collate import default_collate_fn, default_convert_fn +from .worker import ParentWatchDog, get_worker_info, _worker_loop, \ + _DatasetKind, _IterableDatasetStopIteration, _WorkerException +from .flat import _flatten_batch, _restore_batch __all__ = ['get_worker_info'] -# multi-process worker check indices queue interval, avoid -# hanging in subprocess data loading -MP_INDICES_CHECK_INTERVAL = 5 - -_IterableDatasetStopIteration = namedtuple('_IterableDatasetStopIteration', - ['worker_id']) - - -def default_collate_fn(batch): - """ - Default batch collating function for :code:`fluid.io.DataLoader`, - batch should be a list of samples, and each sample should be a list - of fields as follows: - - [[filed1, filed2, ...], [filed1, filed2, ...], ...] - - This default collate function zipped each filed together and stack - each filed as the batch field as follows: - - [batch_filed1, batch_filed2, ...] - - Args: - batch(list of list of numpy array): the batch data, each fields - should be a numpy array, each sample should be a list of - fileds, and batch should be a list of sample. - - Returns: - a list of numpy array: collated batch - """ - sample = batch[0] - # dataset has only 1 field - if isinstance(sample, np.ndarray): - return [np.stack(batch, axis=0)] - - # batch each field - slots = [] - for items in batch: - for i, item in enumerate(items): - if len(slots) < len(items): - slots.append([item]) - else: - slots[i].append(item) - - outputs = [] - for slot in slots: - if isinstance(slot[0], (np.ndarray, np.bool, numbers.Number)): - tmp = np.stack(slot, axis=0) - outputs.append(tmp) - elif isinstance(slot[0], paddle.Tensor): - tmp = layers.stack(slot, axis=0) - outputs.append(tmp) - else: - raise RuntimeError("Unknown data type {}".format(type(slot[0]))) - return outputs - - -class _DatasetKind(object): - MAP = 0 - ITER = 1 - - @staticmethod - def create_fetcher(kind, dataset, auto_collate_batch, collate_fn, - drop_last): - if kind == _DatasetKind.MAP: - return _MapDatasetFetcher(dataset, auto_collate_batch, collate_fn, - drop_last) - elif kind == _DatasetKind.ITER: - return _IterableDatasetFetcher(dataset, auto_collate_batch, - collate_fn, drop_last) - else: - raise NotImplementedError("unknown Dataset kind {}".format(kind)) - - -class ParentWatchDog(object): - def __init__(self): - self._parent_pid = os.getppid() - self._parent_alive = True - - def is_alive(self): - if self._parent_alive: - self._parent_alive = os.getppid() == self._parent_pid - return self._parent_alive - - -# worker information for each workers, used for splitting data copy -# for IteratorDataset in worker processes. -_worker_info = None - - -def get_worker_info(): - """ - Get DataLoader worker process information function, this function is - used to split data copy in worker process for IterableDataset - (see :code:`paddle.io.IterableDataset`), worker information contains - following fields: - - :attr:`num_workers`: total worker process number, see `paddle.io.DataLoader` - - :attr:`id`: the worker processs id, count from 0 to :attr:`num_workers - 1` - - :attr:`dataset`: the dataset object in this worker process - - Returns: - WorkerInfo: an instance of WorkerInfo which contains fields above. - - .. note:: - For mode usage and exampls, please see :code:`paddle.io.IterableDataset` - - Example: - - .. code-block:: python - - import math - import paddle - import numpy as np - from paddle.io import IterableDataset, DataLoader, get_worker_info - - class SplitedIterableDataset(IterableDataset): - def __init__(self, start, end): - self.start = start - self.end = end - - def __iter__(self): - worker_info = get_worker_info() - if worker_info is None: - iter_start = self.start - iter_end = self.end - else: - per_worker = int( - math.ceil((self.end - self.start) / float( - worker_info.num_workers))) - worker_id = worker_info.id - iter_start = self.start + worker_id * per_worker - iter_end = min(iter_start + per_worker, self.end) - - for i in range(iter_start, iter_end): - yield np.array([i]) - - place = paddle.CPUPlace() - dataset = SplitedIterableDataset(start=2, end=9) - dataloader = DataLoader( - dataset, - places=place, - num_workers=2, - batch_size=1, - drop_last=True) - - for data in dataloader: - print(data) - # outputs: [2, 5, 3, 6, 4, 7] - - """ - return _worker_info - - -class WorkerInfo(object): - __initialized = False - - def __init__(self, **kwargs): - for k, v in kwargs.items(): - setattr(self, k, v) - self.__initialized = True - - def __setattr__(self, key, val): - if self.__initialized: - raise RuntimeError("Cannot assign attributes to {} objects".format( - self.__class__.__name__)) - return super(WorkerInfo, self).__setattr__(key, val) - class _DataLoaderIterBase(object): """ @@ -230,7 +65,7 @@ def __init__(self, loader): self._num_workers = loader.num_workers self._use_buffer_reader = loader.use_buffer_reader self._use_shared_memory = loader.use_shared_memory - self._timeout = loader.timeout if loader.timeout > 0 else MP_INDICES_CHECK_INTERVAL + self._timeout = loader.timeout if loader.timeout > 0 else MP_STATUS_CHECK_INTERVAL self._worker_init_fn = loader.worker_init_fn self._dataset_kind = loader.dataset_kind self._pin_memory = loader.pin_memory @@ -244,7 +79,7 @@ def __init__(self, loader): else: self._sampler_iter = iter( _InfiniteIterableSampler(self._dataset, 1)) - self._collate_fn = loader.collate_fn + self._collate_fn = loader.collate_fn or default_convert_fn # LoDTensorBlockingQueue instance for create_py_reader and a thread # to put mini-batch data to self._blocking_queue, mini-batch data @@ -275,6 +110,14 @@ def __init__(self, loader): self._dataset_kind, self._dataset, self._auto_collate_batch, self._collate_fn, True) + # NOTE: _structrue_infos used to record the data structure of + # batch to restore batch structure after reading Tensor + # from blocking_queue in single-process mode. Note that + # only single process is used in single-process mode, we + # can record the data structure sequencely in a list without + # recording the send and recv index + self._structure_infos = [] + # NOTE: len(self._places) batch data compose as an output # iteration, set blocking_queue can cache 2 iteration datas # at most here @@ -316,16 +159,14 @@ def _thread_loop(self, legacy_expected_place): # read data from dataset in mini-batch batch = self._dataset_fetcher.fetch(indices) + # flat batch and record structure infos + batch, structure = _flatten_batch(batch) + self._structure_infos.append(structure) + # pack as LoDTensorArray array = core.LoDTensorArray() for slot in batch: if not isinstance(slot, core.LoDTensor): - # FIXME(dkp): blocking_queue only support - # core.LoDTensorArray as input now, read - # numpy data into a LoDTensorArray here, - # should support paddle.Tensor list later - if isinstance(slot, paddle.Tensor): - slot = slot.numpy() tmp = core.LoDTensor() tmp.set(slot, core.CPUPlace()) slot = tmp @@ -348,20 +189,29 @@ def _thread_loop(self, legacy_expected_place): def __next__(self): try: if in_dygraph_mode(): - return self._reader.read_next_var_list() + data = self._reader.read_next_var_list() + data = _restore_batch(data, self._structure_infos.pop(0)) else: if self._return_list: + data = self._reader.read_next_list() + data = [ + _restore_batch(d, s) + for d, s in zip(data, self._structure_infos[:len( + self._places)]) + ] + self._structure_infos = self._structure_infos[len( + self._places):] # static graph organized data on multi-device with list, if # place number is 1, there is only 1 device, extra the data # from list for devices to be compatible with dygraph mode if len(self._places) == 1: - return self._reader.read_next_list()[0] - else: - return self._reader.read_next_list() + data = data[0] else: - return self._reader.read_next() + data = self._reader.read_next() + + return data except StopIteration: - self._reader.reset() + self._reader.shutdown() six.reraise(*sys.exc_info()) # python2 compatibility @@ -375,97 +225,6 @@ def __del__(self): self._blocking_queue.close() -# NOTE(chenweihang): _worker_loop must be top level method to be pickled -def _worker_loop(dataset, dataset_kind, indices_queue, out_queue, done_event, - auto_collate_batch, collate_fn, init_fn, worker_id, - num_workers, use_shared_memory): - try: - # NOTE: [ mmap files clear ] When the child process exits unexpectedly, - # some shared memory objects may have been applied for but have not yet - # been put into the inter-process Queue. This part of the object needs - # to be cleaned up when the process ends. - CleanupFuncRegistrar.register(_cleanup_mmap) - - # set signal handler - core._set_process_signal_handler() - - global _worker_info - _worker_info = WorkerInfo( - id=worker_id, num_workers=num_workers, dataset=dataset) - - init_exception = None - try: - if init_fn is not None: - init_fn(worker_id) - fetcher = _DatasetKind.create_fetcher( - dataset_kind, dataset, auto_collate_batch, collate_fn, True) - except: - init_exception = Exception("init_fn failed in worker {}: " \ - "{}".format(worker_id, sys.exc_info())) - - iterator_drained = False - parent_watch_dog = ParentWatchDog() - - while parent_watch_dog.is_alive(): - try: - data = indices_queue.get(MP_INDICES_CHECK_INTERVAL) - except queue.Empty: - continue - - # None as poison piil, so worker event should be set - if data is None: - assert done_event.is_set() or iterator_drained, \ - "get None when worker done_event set" - break - # If worker done event is set but get still get data in - # indices_queue, remaining data should be get and skipped. - if done_event.is_set() or iterator_drained: - continue - - idx, indices = data - try: - if init_exception is not None: - batch = init_exception - init_exception = None - else: - batch = fetcher.fetch(indices) - except Exception as e: - if isinstance( - e, StopIteration) and dataset_kind == _DatasetKind.ITER: - out_queue.put(_IterableDatasetStopIteration(worker_id)) - iterator_drained = True - else: - out_queue.put((idx, e)) - else: - if use_shared_memory: - # FIXME(dkp): _convert_to_tensor_list only support np.array - # list now, should support paddle.Tensor list - new_batch = [] - for sample in batch: - new_sample = [] - for s in sample: - if isinstance(s, paddle.Tensor): - new_sample.append(s.numpy()) - else: - new_sample.append(s) - new_batch.append(new_sample) - batch = new_batch - - tensor_list = core._convert_to_tensor_list(batch) - out_queue.put((idx, tensor_list)) - core._remove_tensor_list_mmap_fds(tensor_list) - else: - out_queue.put((idx, batch)) - except KeyboardInterrupt: - # NOTE: Main process will raise KeyboardInterrupt anyways, ignore it in child process - pass - except: - six.reraise(*sys.exc_info()) - finally: - if use_shared_memory: - _cleanup_mmap() - - class _DataLoaderIterMultiProcess(_DataLoaderIterBase): def __init__(self, loader): super(_DataLoaderIterMultiProcess, self).__init__(loader) @@ -483,6 +242,7 @@ def __init__(self, loader): self._rcvd_idx = 0 self._batches_outstanding = 0 self._task_infos = {} + self._structure_infos = [] # indices outstand as _outstanding_capacity at first, and # blocking_queue capacity is also _outstanding_capacity. @@ -617,8 +377,6 @@ def _thread_loop(self, legacy_expected_place): if not self._thread_done_event.is_set(): if batch is None: self._exit_thread_expectedly() - elif isinstance(batch, Exception): - self._exit_thread_unexpectedly() else: try: # pack as LoDTensorArray @@ -654,8 +412,9 @@ def _get_data(self): # batch indices and increase _rcvd_idx if self._dataset_kind == _DatasetKind.ITER: while self._rcvd_idx < self._send_idx: + sys.stdout.flush() info = self._task_infos[self._rcvd_idx] - if len(info) == 2 or self._worker_status[info[0]]: + if len(info) == 3 or self._worker_status[info[0]]: break del self._task_infos[self._rcvd_idx] self._rcvd_idx += 1 @@ -669,13 +428,15 @@ def _get_data(self): continue if self._rcvd_idx in self._task_infos and \ - len(self._task_infos[self._rcvd_idx]) == 2: - return self._task_infos.pop(self._rcvd_idx)[1] + len(self._task_infos[self._rcvd_idx]) == 3: + info = self._task_infos.pop(self._rcvd_idx) + self._structure_infos.append(info[2]) + return info[1] try: # [ avoid hang ]: main process may blocking at _reader.read_next when # KeyboardInterrupt, we do following tradeoff: - # 1. get data with timeout, MP_INDICES_CHECK_INTERVAL(5s) as timeout + # 1. get data with timeout, MP_STATUS_CHECK_INTERVAL(5s) as timeout # default, if KeyboardInterrupt blocking, failed workers will be # checked and raise RuntimeError to quit DataLoader in timeout # exception handling. @@ -721,12 +482,17 @@ def _get_data(self): self._try_put_indices() continue - idx, batch = data + idx, batch, structure = data + if isinstance(batch, _WorkerException): + self._exit_thread_unexpectedly() + batch.reraise() + if idx == self._rcvd_idx: del self._task_infos[idx] + self._structure_infos.append(structure) return batch else: - self._task_infos[idx] += (batch, ) + self._task_infos[idx] += (batch, structure) continue def _try_put_indices(self): @@ -777,9 +543,17 @@ def __next__(self): if in_dygraph_mode(): data = self._reader.read_next_var_list() + data = _restore_batch(data, self._structure_infos.pop(0)) else: if self._return_list: data = self._reader.read_next_list() + data = [ + _restore_batch(d, s) + for d, s in zip(data, self._structure_infos[:len( + self._places)]) + ] + self._structure_infos = self._structure_infos[len( + self._places):] # static graph organized data on multi-device with list, if # place number is 1, there is only 1 device, extra the data # from list for devices to be compatible with dygraph mode @@ -790,7 +564,7 @@ def __next__(self): self._on_output_batch() return data except StopIteration: - self._reader.reset() + self._reader.shutdown() self._try_shutdown_all() six.reraise(*sys.exc_info()) diff --git a/python/paddle/fluid/dataloader/dataset.py b/python/paddle/fluid/dataloader/dataset.py index ac90cbafe1731..e46083295d1ce 100755 --- a/python/paddle/fluid/dataloader/dataset.py +++ b/python/paddle/fluid/dataloader/dataset.py @@ -14,8 +14,8 @@ from __future__ import print_function +import paddle from .. import framework -import paddle.dataset.common __all__ = [ "Dataset", "IterableDataset", "TensorDataset", "ComposeDataset", diff --git a/python/paddle/fluid/dataloader/flat.py b/python/paddle/fluid/dataloader/flat.py new file mode 100644 index 0000000000000..6cccbc7ee4ea7 --- /dev/null +++ b/python/paddle/fluid/dataloader/flat.py @@ -0,0 +1,150 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import paddle +import numbers +import numpy as np + +try: + from collections.abc import Sequence, Mapping +except: + from collections import Sequence, Mapping + +FIELD_PREFIX = "_paddle_field_" + + +def _flatten_batch(batch): + """ + For lod_blocking_queue only receive tensor array, flatten batch + data, extract numpy.array data out as a list of numpy.array to + send to lod_blocking_queue, and save the batch data structure + such as fields in other types (str, int, etc) or key-value map + of dictionaries + """ + + def _flatten(batch, flat_batch, structure, field_idx): + if isinstance(batch, Sequence): + for field in batch: + if isinstance(field, np.ndarray): + structure.append('{}{}'.format(FIELD_PREFIX, field_idx)) + flat_batch.append(field) + field_idx += 1 + elif isinstance(field, paddle.Tensor): + structure.append('{}{}'.format(FIELD_PREFIX, field_idx)) + flat_batch.append(field.numpy()) + field_idx += 1 + elif isinstance(field, (str, bytes, numbers.Number)): + structure.append(field) + elif isinstance(field, Sequence): + field_struct, field_idx = _flatten(field, flat_batch, [], + field_idx) + structure.append(field_struct) + elif isinstance(field, Mapping): + field_struct, field_idx = _flatten(field, flat_batch, {}, + field_idx) + structure.append(field_struct) + else: + structure.append(field) + elif isinstance(batch, Mapping): + for k, field in batch.items(): + if isinstance(field, np.ndarray): + structure[k] = '{}{}'.format(FIELD_PREFIX, field_idx) + flat_batch.append(field) + field_idx += 1 + elif isinstance(field, paddle.Tensor): + structure[k] = '{}{}'.format(FIELD_PREFIX, field_idx) + flat_batch.append(field.numpy()) + field_idx += 1 + elif isinstance(field, (str, bytes, numbers.Number)): + structure[k] = field + elif isinstance(field, Sequence): + field_struct, field_idx = _flatten(field, flat_batch, [], + field_idx) + structure[k] = field_struct + elif isinstance(field, Mapping): + field_struct, field_idx = _flatten(field, flat_batch, {}, + field_idx) + structure[k] = field_struct + else: + structure[k] = field + else: + raise TypeError("wrong flat data type: {}".format(type(batch))) + + return structure, field_idx + + # sample only contains single fields + if not isinstance(batch, Sequence): + flat_batch = [] + structure, _ = _flatten([batch], flat_batch, [], 0) + return flat_batch, structure[0] + flat_batch = [] + structure, _ = _flatten(batch, flat_batch, [], 0) + return flat_batch, structure + + +def _restore_batch(flat_batch, structure): + """ + After reading list of Tensor data from lod_blocking_queue outputs, + use this function to restore the batch data structrue, replace + :attr:`_paddle_field_x` with data from flat_batch + """ + + def _restore(structure, field_idx): + if isinstance(structure, Sequence): + for i, field in enumerate(structure): + if isinstance(field, str) and field.startswith(FIELD_PREFIX): + cur_field_idx = int(field.replace(FIELD_PREFIX, '')) + field_idx = max(field_idx, cur_field_idx) + assert flat_batch[cur_field_idx] is not None, \ + "flat_batch[{}] parsed repeatly" + structure[i] = flat_batch[cur_field_idx] + flat_batch[cur_field_idx] = None + elif isinstance(field, (str, bytes, numbers.Number)): + continue + elif isinstance(field, (Sequence, Mapping)): + field_idx = _restore(structure[i], field_idx) + elif isinstance(structure, Mapping): + for k, field in structure.items(): + if isinstance(field, str) and field.startswith(FIELD_PREFIX): + cur_field_idx = int(field.replace(FIELD_PREFIX, '')) + field_idx = max(field_idx, cur_field_idx) + assert flat_batch[cur_field_idx] is not None, \ + "flat_batch[{}] parsed repeatly" + structure[k] = flat_batch[cur_field_idx] + flat_batch[cur_field_idx] = None + elif isinstance(field, (str, bytes, numbers.Number)): + continue + elif isinstance(field, (Sequence, Mapping)): + field_idx = _restore(structure[k], field_idx) + else: + raise TypeError("wrong flat data type: {}".format(type(batch))) + + return field_idx + + assert isinstance(flat_batch, Sequence), \ + "flat_batch is not a list or tuple" + + # no np.array in dataset, no output tensor from blocking queue + # simply return structure + if len(flat_batch) == 0: + return structure + + # sample only contains single fields + if isinstance(structure, (str, bytes)): + assert structure == '{}{}'.format(FIELD_PREFIX, 0), \ + "invalid structure: {}".format(structure) + return flat_batch[0] + field_idx = _restore(structure, 0) + assert field_idx + 1 == len(flat_batch), "Tensor parse incomplete" + return structure diff --git a/python/paddle/fluid/dataloader/worker.py b/python/paddle/fluid/dataloader/worker.py new file mode 100644 index 0000000000000..2d1b554e53d68 --- /dev/null +++ b/python/paddle/fluid/dataloader/worker.py @@ -0,0 +1,253 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import os +import six +import sys +import paddle +import numpy as np +import traceback +from collections import namedtuple +from .. import core +from .fetcher import _IterableDatasetFetcher, _MapDatasetFetcher +from ..multiprocess_utils import _cleanup_mmap, CleanupFuncRegistrar, MP_STATUS_CHECK_INTERVAL +from ..framework import in_dygraph_mode +from .flat import _flatten_batch + +# NOTE: queue has a different name in python2 and python3 +if six.PY2: + import Queue as queue +else: + import queue + +__all__ = ['get_worker_info'] + + +class _IterableDatasetStopIteration(object): + def __init__(self, worker_id): + self.worker_id = worker_id + + +class _DatasetKind(object): + MAP = 0 + ITER = 1 + + @staticmethod + def create_fetcher(kind, dataset, auto_collate_batch, collate_fn, + drop_last): + if kind == _DatasetKind.MAP: + return _MapDatasetFetcher(dataset, auto_collate_batch, collate_fn, + drop_last) + elif kind == _DatasetKind.ITER: + return _IterableDatasetFetcher(dataset, auto_collate_batch, + collate_fn, drop_last) + else: + raise NotImplementedError("unknown Dataset kind {}".format(kind)) + + +class ParentWatchDog(object): + def __init__(self): + self._parent_pid = os.getppid() + self._parent_alive = True + + def is_alive(self): + if self._parent_alive: + self._parent_alive = os.getppid() == self._parent_pid + return self._parent_alive + + +# worker information for each workers, used for splitting data copy +# for IteratorDataset in worker processes. +_worker_info = None + + +def get_worker_info(): + """ + Get DataLoader worker process information function, this function is + used to split data copy in worker process for IterableDataset + (see :code:`paddle.io.IterableDataset`), worker information contains + following fields: + + :attr:`num_workers`: total worker process number, see `paddle.io.DataLoader` + + :attr:`id`: the worker processs id, count from 0 to :attr:`num_workers - 1` + + :attr:`dataset`: the dataset object in this worker process + + Returns: + WorkerInfo: an instance of WorkerInfo which contains fields above. + + .. note:: + For more usage and examples, please see :code:`paddle.io.IterableDataset` + + Example: + + .. code-block:: python + + import math + import paddle + import numpy as np + from paddle.io import IterableDataset, DataLoader, get_worker_info + + class SplitedIterableDataset(IterableDataset): + def __init__(self, start, end): + self.start = start + self.end = end + + def __iter__(self): + worker_info = get_worker_info() + if worker_info is None: + iter_start = self.start + iter_end = self.end + else: + per_worker = int( + math.ceil((self.end - self.start) / float( + worker_info.num_workers))) + worker_id = worker_info.id + iter_start = self.start + worker_id * per_worker + iter_end = min(iter_start + per_worker, self.end) + + for i in range(iter_start, iter_end): + yield np.array([i]) + + place = paddle.CPUPlace() + dataset = SplitedIterableDataset(start=2, end=9) + dataloader = DataLoader( + dataset, + places=place, + num_workers=2, + batch_size=1, + drop_last=True) + + for data in dataloader: + print(data) + # outputs: [2, 5, 3, 6, 4, 7] + + """ + return _worker_info + + +class WorkerInfo(object): + __initialized = False + + def __init__(self, **kwargs): + for k, v in kwargs.items(): + setattr(self, k, v) + self.__initialized = True + + def __setattr__(self, key, val): + if self.__initialized: + raise RuntimeError("Cannot assign attributes to {} objects".format( + self.__class__.__name__)) + return super(WorkerInfo, self).__setattr__(key, val) + + +class _WorkerException(object): + def __init__(self, worker_id, exc_info=None): + self.worker_id = worker_id + exc_info = exc_info or sys.exc_info() + self.exc_type = exc_info[0] + self.exc_msg = "".join(traceback.format_exception(*exc_info)) + + def reraise(self): + msg = "DataLoader worker({}) caught {} with message:\n{}".format( + self.worker_id, self.exc_type.__name__, self.exc_msg) + if getattr(self.exc_type, "message", None): + raise self.exc_type(message=msg) + raise self.exc_type(msg) + + +def _worker_loop(dataset, dataset_kind, indices_queue, out_queue, done_event, + auto_collate_batch, collate_fn, init_fn, worker_id, + num_workers, use_shared_memory): + try: + # NOTE: [ mmap files clear ] When the child process exits unexpectedly, + # some shared memory objects may have been applied for but have not yet + # been put into the inter-process Queue. This part of the object needs + # to be cleaned up when the process ends. + CleanupFuncRegistrar.register(_cleanup_mmap) + + # set signal handler + core._set_process_signal_handler() + + global _worker_info + _worker_info = WorkerInfo( + id=worker_id, num_workers=num_workers, dataset=dataset) + + init_exception = None + try: + if init_fn is not None: + init_fn(worker_id) + fetcher = _DatasetKind.create_fetcher( + dataset_kind, dataset, auto_collate_batch, collate_fn, True) + except: + init_exception = _WorkerException(worker_id) + + iterator_drained = False + parent_watch_dog = ParentWatchDog() + + while parent_watch_dog.is_alive(): + try: + data = indices_queue.get(MP_STATUS_CHECK_INTERVAL) + except queue.Empty: + continue + + # None as poison piil, so worker event should be set + if data is None: + assert done_event.is_set() or iterator_drained, \ + "get None when worker done_event set" + break + # If worker done event is set but get still get data in + # indices_queue, remaining data should be get and skipped. + if done_event.is_set() or iterator_drained: + continue + + idx, indices = data + try: + if init_exception is not None: + batch = init_exception + init_exception = None + else: + # NOTE: GPU tensor operation is not supported in sub-process + # but default device is GPU in paddle-gpu version, which + # may copy CPU tensor to GPU even if users want to use + # CPU tensor operation, so we add CPUPlace guard here + # to make sure tensor will be operated only on CPU + with paddle.fluid.dygraph.guard(place=paddle.CPUPlace()): + batch = fetcher.fetch(indices) + except Exception as e: + if isinstance( + e, StopIteration) and dataset_kind == _DatasetKind.ITER: + out_queue.put(_IterableDatasetStopIteration(worker_id)) + iterator_drained = True + else: + out_queue.put((idx, _WorkerException(worker_id), None)) + else: + if isinstance(batch, _WorkerException): + out_queue.put((idx, batch, None)) + batch, structure = _flatten_batch(batch) + if use_shared_memory: + tensor_list = core._convert_to_tensor_list(batch) + out_queue.put((idx, tensor_list, structure)) + core._remove_tensor_list_mmap_fds(tensor_list) + else: + out_queue.put((idx, batch, structure)) + except KeyboardInterrupt: + # NOTE: Main process will raise KeyboardInterrupt anyways, ignore it in child process + pass + except: + six.reraise(*sys.exc_info()) + finally: + if use_shared_memory: + _cleanup_mmap() diff --git a/python/paddle/fluid/debugger.py b/python/paddle/fluid/debugger.py index 9110b8daf38e1..75dc14a1d754c 100644 --- a/python/paddle/fluid/debugger.py +++ b/python/paddle/fluid/debugger.py @@ -280,88 +280,3 @@ def add_op_link_var(op, var, op2var=False): add_op_link_var(opn, var, True) graph(path, show=False) - - -def prepare_fast_nan_inf_debug(_program): - """ - Given a program to run, insert a (reduce) sum op for every var in that program. - Instead of checking all vars originally defined in the program, - only those inserted ops will be checked in the c++ end, to detect if it contains NAN or INF. - Thereforce, the speed of nan/inf checking could be improved. - Please set ``FLAGS_fast_check_nan_inf" to open the fast nan/inf feature. - """ - - helper = LayerHelper('reduce_sum', **locals()) - - if _program is None: - _program = default_main_program() - - for _block in _program.blocks: - # fetch vars in the current block - _vars_in_prog = [] - for _var_name in _block.vars: - _vars_in_prog.append((_var_name, _block.vars[_var_name])) - - # append sum_op in the current block - for _var_name, _var in _vars_in_prog: - - try: - - if _var.dtype == -1: - continue - - ## create a var for holding sum output - _output_var = _block.create_var( - name=unique_name.generate("debug_var_" + _var_name), - dtype=_var.dtype, - type=core.VarDesc.VarType.LOD_TENSOR, - persistable=False, - stop_gradient=True) - - ## create a sum op, input each existing var in the block - _block.append_op( - type='sum', - outputs={'Out': _output_var}, - inputs={'X': [_var]}) - except Exception as e: - pass - - -def run_fast_nan_inf_debug(executor, - program=None, - feed=None, - fetch_list=None, - feed_var_name='feed', - fetch_var_name='fetch', - scope=None, - return_numpy=True, - use_program_cache=False, - dump_core=True): - """ - Run a program by the given executor. Catch the exception of NAN and INF, and save persistables into the dumped core. - """ - - assert (executor is not None) - - try: - output = executor.run(program=program, - feed=feed, - fetch_list=fetch_list, - feed_var_name=feed_var_name, - fetch_var_name=fetch_var_name, - scope=scope, - return_numpy=return_numpy, - use_program_cache=use_program_cache) - - return output - - except Exception as e: - - print("catch an exception:") - print(e) - - core_filename = "core" + str(int(random.random() * 10000)) + ".pdckpt" - io.save_persistables( - executor, "./", main_program=program, filename=core_filename) - - print("dumping a core into ./%s" % core_filename) diff --git a/python/paddle/fluid/device_worker.py b/python/paddle/fluid/device_worker.py index 838aea37f1834..b923f36af8d02 100644 --- a/python/paddle/fluid/device_worker.py +++ b/python/paddle/fluid/device_worker.py @@ -413,6 +413,18 @@ def _gen_worker_desc(self, trainer_desc): section_param = trainer_desc.section_param section_param.num_microbatches = pipeline_opt["num_microbatches"] section_param.start_cpu_core_id = pipeline_opt["start_cpu_core_id"] + section_param.pipeline_stage = pipeline_opt["pipeline_stage"] + section_param.num_pipeline_stages = pipeline_opt["num_pipeline_stages"] + schedule_mode_str = pipeline_opt["schedule_mode"] + # F-then-B scheduler which runs Forward phase for all microbatches, + # then runs Backward phase for all microbatches. + # 1F1B scheduler, which runs forward phase and backward phase altertively + # after startup phase. + assert schedule_mode_str in ["F-then-B", "1F1B"], ( + "The schedule mode " + "for pipeline must be one of F-then-B or 1F1B") + schedule_mode = 0 if schedule_mode_str == "F-then-B" else 1 + section_param.schedule_mode = schedule_mode cfg = section_param.section_config program = pipeline_opt["section_program"] cfg.program_desc.ParseFromString(program["program"]._get_desc() diff --git a/python/paddle/fluid/dygraph/dygraph_to_static/convert_operators.py b/python/paddle/fluid/dygraph/dygraph_to_static/convert_operators.py index 779e50c3dc5b5..4126e94225943 100644 --- a/python/paddle/fluid/dygraph/dygraph_to_static/convert_operators.py +++ b/python/paddle/fluid/dygraph/dygraph_to_static/convert_operators.py @@ -302,9 +302,19 @@ def convert_var_shape_simple(x): return x.shape -def eval_if_exist_else_none(name): +def eval_if_exist_else_none(name, global_symbol_table): + """ + Args: + name([str]): Expression passed into `eval`. + local_symbol_table(dict): Specified from `globals()`. DO NOT use `locals()`, + because all STATIC_CONVERT_VAR_SHAPE_SUFFIX vars is + declared with keyword `global`. + + Returns: + Return the variable if found in global_symbol_table else None. + """ try: - return eval(name) + return eval(name, global_symbol_table) except: return None diff --git a/python/paddle/fluid/dygraph/dygraph_to_static/list_transformer.py b/python/paddle/fluid/dygraph/dygraph_to_static/list_transformer.py index 7e4c6ca33cb72..a3311765a996f 100644 --- a/python/paddle/fluid/dygraph/dygraph_to_static/list_transformer.py +++ b/python/paddle/fluid/dygraph/dygraph_to_static/list_transformer.py @@ -18,7 +18,10 @@ import gast from paddle.fluid.dygraph.dygraph_to_static.static_analysis import AstNodeWrapper, StaticAnalysisVisitor -from paddle.fluid.dygraph.dygraph_to_static.utils import ast_to_source_code, is_control_flow_to_transform +from paddle.fluid.dygraph.dygraph_to_static.utils import ast_to_source_code +from paddle.fluid.dygraph.dygraph_to_static.utils import slice_is_num +from paddle.fluid.dygraph.dygraph_to_static.utils import is_control_flow_to_transform + from paddle.fluid.dygraph.dygraph_to_static.utils import SplitAssignTransformer @@ -116,12 +119,13 @@ def _need_to_array_write_node(self, node): def _transform_slice_to_tensor_write(self, node): assert isinstance(node, gast.Assign) target_node = node.targets[0] + target_name = target_node.value.id slice_node = target_node.slice if isinstance(slice_node, gast.Slice): pass - elif isinstance(slice_node, gast.Index): + elif slice_is_num(target_node): value_code = ast_to_source_code(node.value) i = "paddle.cast(" \ "x=paddle.jit.dy2static.to_static_variable({})," \ diff --git a/python/paddle/fluid/dygraph/dygraph_to_static/loop_transformer.py b/python/paddle/fluid/dygraph/dygraph_to_static/loop_transformer.py index 140c57f710a3d..bd89a79c805c9 100644 --- a/python/paddle/fluid/dygraph/dygraph_to_static/loop_transformer.py +++ b/python/paddle/fluid/dygraph/dygraph_to_static/loop_transformer.py @@ -39,8 +39,35 @@ FOR_BODY_PREFIX = 'for_loop_body' GENERATE_VARIABLE_PREFIX = 'generate_variable' +ATTRIBUTE_VARIABLE_PREFIX = '__attribute_variable' -def create_while_node(condition_name, body_name, loop_var_names): + +def create_while_nodes(condition_name, body_name, loop_var_names): + """ + Returns a list of gast.Node which represents the calling of Paddle + controlflow while_loop. + + Usually, the list just contain 1 statement such as: + + [a, b, c] = paddle.jit.dy2static.convert_while_loop( + condition_name, body_name, [a, b, c]) + + where a, b, c are in loop_var_names. + + However, if loop_var_names contains property such as foo.x, we cannot + assign the property as output of convert_while_loop because Python + property is a kind of read-only attribute. To handle the case, we replace + the attributes which are output of convert_while_loop with generated + variables, then if we know the attribute is not read-only at runtime, we + assign the attribute. The created statements are like: + + [a, b, __attribute_variable_1] = paddle.jit.dy2static.convert_while_loop( + condition_name, body_name, [a, b, foo.x]) + if not isinstance(getattr(type(foo), x, None), property): foo.x = __attribute_variable_1 + + The number of above statements is not only 1, that's why the return type is + a list of gast.Node. + """ # NOTE(liym27): # It's better to parse the source code into an AST node than to customize an AST node # including child nodes, because it is easy to mistake the ast node type when customizing the node. @@ -48,14 +75,37 @@ def create_while_node(condition_name, body_name, loop_var_names): # For example: loop_var_names = [a, b, foo.x], the type of `a` or `b` is gast.Name, # but the type of `foo.x` gast.Attribute. + unique_name_to_origin = {} + # We have to make loop_var_names and assign_loop_var_names with same order + # set doesn't have order so we convert it to list + loop_var_names = list(loop_var_names) + assign_loop_var_names = [] + for name in (loop_var_names): + if "." in name: + # name is an attribute variable such as foo.x + tmp_attr_name = unique_name.generate(ATTRIBUTE_VARIABLE_PREFIX) + unique_name_to_origin[tmp_attr_name] = name + assign_loop_var_names.append(tmp_attr_name) + else: + assign_loop_var_names.append(name) + while_func_name = "paddle.jit.dy2static.convert_while_loop" while_node_str = "[{}] = {}({}, {}, [{}])".format( - ",".join(loop_var_names), while_func_name, condition_name, body_name, - ",".join(loop_var_names)) - + ",".join(assign_loop_var_names), while_func_name, condition_name, + body_name, ",".join(loop_var_names)) while_node = gast.parse(while_node_str).body[0] - return while_node + ret = [while_node] + for tmp_attr_name in unique_name_to_origin: + origin_attr_var = unique_name_to_origin[tmp_attr_name] + dot_pos = origin_attr_var.rindex(".") + obj_name = origin_attr_var[0:dot_pos] + attr_name = origin_attr_var[dot_pos + 1:] + assign_if_not_prop_str = "if not isinstance(getattr(type({}), '{}', None), property): {} = {}".format( + obj_name, attr_name, origin_attr_var, tmp_attr_name) + assign_if_not_prop_node = gast.parse(assign_if_not_prop_str).body[0] + ret.append(assign_if_not_prop_node) + return ret class NameVisitor(gast.NodeVisitor): @@ -544,7 +594,7 @@ def get_for_stmt_nodes(self, node): # append return values for loop body body_stmts.append( gast.Return(value=generate_name_node( - loop_var_names, ctx=gast.Load()))) + loop_var_names, ctx=gast.Load(), gen_tuple_if_single=True))) body_func_node = gast.FunctionDef( name=unique_name.generate(FOR_BODY_PREFIX), args=gast.arguments( @@ -573,9 +623,9 @@ def get_for_stmt_nodes(self, node): new_stmts.append(body_func_node) # 7. create & append while loop node - while_loop_node = create_while_node(condition_func_node.name, - body_func_node.name, loop_var_names) - new_stmts.append(while_loop_node) + while_loop_nodes = create_while_nodes( + condition_func_node.name, body_func_node.name, loop_var_names) + new_stmts.extend(while_loop_nodes) return new_stmts @@ -655,7 +705,7 @@ def get_while_stmt_nodes(self, node): name, unique_name.generate(GENERATE_VARIABLE_PREFIX)) new_stmts.append(body_func_node) - while_loop_node = create_while_node(condition_func_node.name, - body_func_node.name, loop_var_names) - new_stmts.append(while_loop_node) + while_loop_nodes = create_while_nodes( + condition_func_node.name, body_func_node.name, loop_var_names) + new_stmts.extend(while_loop_nodes) return new_stmts diff --git a/python/paddle/fluid/dygraph/dygraph_to_static/tensor_shape_transformer.py b/python/paddle/fluid/dygraph/dygraph_to_static/tensor_shape_transformer.py index ddd5d84ef4212..eb53d7ec9bec8 100644 --- a/python/paddle/fluid/dygraph/dygraph_to_static/tensor_shape_transformer.py +++ b/python/paddle/fluid/dygraph/dygraph_to_static/tensor_shape_transformer.py @@ -19,6 +19,7 @@ from paddle.fluid import unique_name from paddle.fluid.dygraph.dygraph_to_static.utils import ast_to_source_code +from paddle.fluid.dygraph.dygraph_to_static.utils import slice_is_num from paddle.fluid.dygraph.dygraph_to_static.utils import is_paddle_api from paddle.fluid.dygraph.dygraph_to_static.utils import SplitAssignTransformer from paddle.fluid.dygraph.dygraph_to_static.static_analysis import AstNodeWrapper @@ -34,42 +35,42 @@ def create_convert_shape_node(var_shape_node, if isinstance(var_shape_node, gast.Attribute): args = [ast_to_source_code(var_shape_node.value).strip()] - # (1) A slice can be a simple number such as 1, -2, i.e. gast.Index - # (2) A slice can also be represented by bounds such as 2:-1, i.e. not gast.Index + # (1) A slice can be a simple number such as 1, -2, i.e. gast.Index or gast.Constant + # (2) A slice can also be represented by bounds such as 2:-1, i.e. not gast.Index or gast.Constant # In (1) case, we pass the number as 'idx' argument in convert_var_shape # In (2) case, we have to make it like `convert_var_shape(x)[slice]` - if slice_node is not None and isinstance(slice_node, gast.Index): - args.append(ast_to_source_code(slice_node).strip()) + if slice_node is not None and slice_is_num(slice_node): + args.append(ast_to_source_code(slice_node.slice).strip()) convert_var_shape_func = "paddle.jit.dy2static.convert_var_shape({}, in_control_flow={})".format( ",".join(args), in_control_flow) api_shape_node = gast.parse(convert_var_shape_func).body[0].value - if slice_node is not None and not isinstance(slice_node, gast.Index): + if slice_node is not None and not slice_is_num(slice_node): return gast.Subscript( - value=api_shape_node, slice=slice_node, ctx=gast.Load()) + value=api_shape_node, slice=slice_node.slice, ctx=gast.Load()) return api_shape_node if isinstance(var_shape_node, gast.Subscript): result_node = copy.deepcopy(var_shape_node) - result_node = create_convert_shape_node( - result_node.value, result_node.slice, in_control_flow) + result_node = create_convert_shape_node(result_node.value, result_node, + in_control_flow) return result_node def create_choose_shape_node(attr_shape_name, api_shape_name, slice_node=None): - eval_exist_func = "paddle.jit.dy2static.eval_if_exist_else_none('{}')".format( + eval_exist_func = "paddle.jit.dy2static.eval_if_exist_else_none('{}', globals())".format( api_shape_name) args = [attr_shape_name, eval_exist_func] - if slice_node is not None and isinstance(slice_node, gast.Index): - args.append(ast_to_source_code(slice_node).strip()) + if slice_node is not None and slice_is_num(slice_node): + args.append(ast_to_source_code(slice_node.slice).strip()) choose_shape_func = "paddle.jit.dy2static.choose_shape_attr_or_api({})".format( ",".join(args)) choose_shape_node = gast.parse(choose_shape_func).body[0].value - if slice_node is not None and not isinstance(slice_node, gast.Index): + if slice_node is not None and not slice_is_num(slice_node): return gast.Subscript( - value=choose_shape_node, slice=slice_node, ctx=gast.Load()) + value=choose_shape_node, slice=slice_node.slice, ctx=gast.Load()) return choose_shape_node @@ -132,17 +133,15 @@ def visit_Subscript(self, node): if value_node.id in self.name_to_var_shape and self._used_by_paddle_api( value_node): return create_choose_shape_node( - value_node.id, self.name_to_var_shape[value_node.id], - slice_node) + value_node.id, self.name_to_var_shape[value_node.id], node) elif isinstance(value_node, gast.Attribute): if self._used_by_paddle_api(value_node): value_name = ast_to_source_code(value_node).strip() if value_name in self.name_to_var_shape: return create_choose_shape_node( - value_name, self.name_to_var_shape[value_name], - slice_node) + value_name, self.name_to_var_shape[value_name], node) if self._is_var_shape(value_node): - return create_convert_shape_node(value_node, slice_node) + return create_convert_shape_node(value_node, node) return node def visit_Attribute(self, node): @@ -294,6 +293,10 @@ def _is_var_shape(self, node): return False def _update_name_to_var_shape(self, node): + def replace_dot(name): + # replace all '.' into '_' + return name.replace('.', '_') + assert isinstance(node, gast.Assign) target_node = node.targets[0] value_node = node.value @@ -308,20 +311,17 @@ def _update_name_to_var_shape(self, node): if value_node.id in self.name_to_var_shape: # TODO(zhhsplendid): is context a problem for the result node of gast.parse? static_shape_var_name = unique_name.generate( - target_id + STATIC_CONVERT_VAR_SHAPE_SUFFIX) + replace_dot(target_id) + + STATIC_CONVERT_VAR_SHAPE_SUFFIX) static_shape_var_node = gast.parse( static_shape_var_name).body[0].value static_shape_value_name = self.name_to_var_shape[ value_node.id] - static_shape_value_node = gast.parse( - static_shape_value_name).body[0].value - index_value_node = gast.Constant(value=idx, kind=None) - slice_index_node = gast.Index(value=index_value_node) - sub_node = gast.Subscript( - value=static_shape_value_node, - slice=slice_index_node, - ctx=gast.Load()) + + sub_node_str = "{}[{}]".format(static_shape_value_name, + idx) + sub_node = gast.parse(sub_node_str).body[0].value update_static_shape_var_node.append( gast.Assign( @@ -333,20 +333,26 @@ def _update_name_to_var_shape(self, node): if isinstance(value_node, gast.Attribute): if self._is_var_shape(value_node): # eg: x.shape static_shape_var_name = unique_name.generate( - target_id + STATIC_CONVERT_VAR_SHAPE_SUFFIX) + replace_dot(target_id) + + STATIC_CONVERT_VAR_SHAPE_SUFFIX) static_shape_var_node = gast.parse( static_shape_var_name).body[0].value static_shape_value_node = copy.deepcopy(value_node) # x.shape becomes convert_var_shape_simple(x) - ShapeAttributeTransformer().visit( - static_shape_value_node) - index_value_node = gast.Constant(value=idx, kind=None) - slice_index_node = gast.Index(value=index_value_node) - sub_node = gast.Subscript( - value=static_shape_value_node, - slice=slice_index_node, - ctx=gast.Load()) + static_shape_value_node = ShapeAttributeTransformer( + ).visit(static_shape_value_node) + + sub_node_str = "{}[{}]".format( + ast_to_source_code(static_shape_value_node).strip(), + idx) + sub_node = gast.parse(sub_node_str).body[0].value + # Note(Aurelius84): Becuase static_shape_var_name is used in + # eval_if_exist_else_none() as plain string, so it will not + # be pasred as argument in convert_loop/ifelse. We delcare it + # as global var because it has unique name. + update_static_shape_var_node.append( + gast.Global(names=[static_shape_var_name])) update_static_shape_var_node.append( gast.Assign( @@ -360,7 +366,8 @@ def _update_name_to_var_shape(self, node): if isinstance(value_node, gast.Name): if value_node.id in self.name_to_var_shape: static_shape_var_name = unique_name.generate( - target_id + STATIC_CONVERT_VAR_SHAPE_SUFFIX) + replace_dot(target_id) + + STATIC_CONVERT_VAR_SHAPE_SUFFIX) static_shape_var_node = gast.parse( static_shape_var_name).body[0].value static_shape_value_name = self.name_to_var_shape[ @@ -376,16 +383,20 @@ def _update_name_to_var_shape(self, node): self.name_to_var_shape[target_id] = static_shape_var_name elif self._is_var_shape(value_node): # eg: x.shape or x.shape[0] static_shape_var_name = unique_name.generate( - target_id + STATIC_CONVERT_VAR_SHAPE_SUFFIX) + replace_dot(target_id) + STATIC_CONVERT_VAR_SHAPE_SUFFIX) static_shape_var_node = gast.parse(static_shape_var_name).body[ 0].value static_shape_value_node = copy.deepcopy(value_node) # x.shape becomes convert_var_shape_simple(x) - ShapeAttributeTransformer().visit(static_shape_value_node) + static_shape_value_node = ShapeAttributeTransformer().visit( + static_shape_value_node) + # Declare static_shape_var_name as global var update_static_shape_var_node = [ + gast.Global(names=[static_shape_var_name]) + ] + update_static_shape_var_node.append( gast.Assign( targets=[static_shape_var_node], - value=static_shape_value_node) - ] + value=static_shape_value_node)) self.name_to_var_shape[target_id] = static_shape_var_name return update_static_shape_var_node diff --git a/python/paddle/fluid/dygraph/dygraph_to_static/utils.py b/python/paddle/fluid/dygraph/dygraph_to_static/utils.py index e9f8afc06c7ca..624ca085ac6c2 100644 --- a/python/paddle/fluid/dygraph/dygraph_to_static/utils.py +++ b/python/paddle/fluid/dygraph/dygraph_to_static/utils.py @@ -381,9 +381,15 @@ def get_attribute_full_name(node): return astor.to_source(gast.gast_to_ast(node)).strip() -def generate_name_node(name_ids, ctx=gast.Load()): +def generate_name_node(name_ids, ctx=gast.Load(), gen_tuple_if_single=False): """ - Generate list or gast.Tuple of ast.Name for Return statement. + If name_ids is list or tuple or set with multiple strings, this function + generates gast.Tuple of gast.Name. + If the name_ids is single string or contains only 1 string, this function + returns gast.Name if gen_tuple_if_single==False else returns gast.Tuple + with only one gast.Name + + This function is used at several gast.Return statements. """ if isinstance(name_ids, six.string_types): name_ids = [name_ids] @@ -395,7 +401,7 @@ def generate_name_node(name_ids, ctx=gast.Load()): id=name_id, ctx=ctx, annotation=None, type_comment=None) for name_id in name_ids ] - if len(gast_names) == 1: + if len(gast_names) == 1 and not gen_tuple_if_single: name_node = gast_names[0] else: name_node = gast.Tuple(elts=gast_names, ctx=ctx) @@ -921,18 +927,15 @@ def visit_For(self, node): def tuple_to_stmts(self, node, tuple_name, idx=[]): if not isinstance(node, (gast.Tuple, gast.List)): - value_node = gast.Name( - id=tuple_name, - ctx=gast.Load(), - annotation=None, - type_comment=None) + value_node_str = tuple_name for i in idx: - value_node = gast.Subscript( - value=value_node, - slice=gast.Index(value=gast.Constant( - value=i, kind=None)), - ctx=gast.Load()) - return [gast.Assign(targets=[node], value=value_node)] + value_node_str = value_node_str + "[{}]".format(i) + + node_str = ast_to_source_code(node).strip() + assign_node_str = "{} = {}".format(node_str, value_node_str) + assign_node = gast.parse(assign_node_str).body[0] + return [assign_node] + # isinstance(node, (gast.Tuple, gast.List)) ret = [] for i, element in enumerate(node.elts): @@ -1240,14 +1243,9 @@ def _build_index_increase_node(self, step_node): value=step_node) def _build_assign_var_slice_node(self): - var_slice_node = gast.Subscript( - value=self.iter_node, - slice=gast.Index(value=gast.Name( - id=self.iter_idx_name, - ctx=gast.Load(), - annotation=None, - type_comment=None)), - ctx=gast.Load(), ) + var_slice_str = "{}[{}]".format( + ast_to_source_code(self.iter_node).strip(), self.iter_idx_name) + var_slice_node = gast.parse(var_slice_str).body[0].value new_iter_var_name = unique_name.generate(FOR_ITER_VAR_NAME_PREFIX) target_node, assign_node = create_assign_node(new_iter_var_name, var_slice_node) @@ -1422,3 +1420,28 @@ def input_specs_compatible(src_input_specs, desired_input_specs): return False return True + + +def slice_is_num(slice_node): + # A slice_node.slice can be a: + # (1) ast.Index, which is a simple number such as [1], [-2] + # (2) ast.Slice, which is represented by bounds such as [2:-1] + # (3) ast.Tuple, which includes the above two cases such as [2:-1, 1] + # If slice node is case (1), return True, Otherwise, return False. + # + # NOTE: In (1) case, when gast>=0.4.0, gast.Index is not used, which is replaced + # other gast node such as gast.Constant, gast.Name, gast.UnaryOp and so on. + # Considering the compatibility of gast, here use ast note to check whether the + # node is a num. For more details, please visit https://github.com/serge-sans-paille/gast + + assert isinstance(slice_node, gast.Subscript) + slice_node_str = ast_to_source_code(slice_node).strip() + ast_node = ast.parse(slice_node_str).body[0].value + + if isinstance(ast_node.slice, (ast.Tuple, ast.Slice)): + return False + + if isinstance(ast_node.slice, ast.Index): + return True + + return False diff --git a/python/paddle/fluid/dygraph/jit.py b/python/paddle/fluid/dygraph/jit.py index 90b0085fe330e..4b35d77845970 100644 --- a/python/paddle/fluid/dygraph/jit.py +++ b/python/paddle/fluid/dygraph/jit.py @@ -25,7 +25,7 @@ from paddle.fluid import core from paddle.fluid.compiler import BuildStrategy, CompiledProgram, ExecutionStrategy from paddle.fluid.data_feeder import check_type -from paddle.fluid.layers.utils import flatten +from paddle.fluid.layers.utils import flatten, pack_sequence_as from paddle.fluid.dygraph.base import program_desc_tracing_guard, switch_to_static_graph from paddle.fluid.dygraph.dygraph_to_static import logging_utils from paddle.fluid.dygraph.dygraph_to_static.convert_call_func import ConversionOptions, CONVERSION_OPTIONS @@ -681,6 +681,11 @@ def train(layer, loader, loss_fn, opt): inner_input_spec) elif 'forward' == attr_func: # transform in jit.save, if input_spec is incomplete, declarative will throw error + # inner_input_spec is list[InputSpec], it should be packed with same sturcture + # as original input_spec here. + if inner_input_spec: + inner_input_spec = pack_sequence_as(input_spec, + inner_input_spec) static_forward = declarative( inner_layer.forward, input_spec=inner_input_spec) concrete_program = static_forward.concrete_program diff --git a/python/paddle/fluid/framework.py b/python/paddle/fluid/framework.py index fd8a39259d9ea..db487128bbe75 100644 --- a/python/paddle/fluid/framework.py +++ b/python/paddle/fluid/framework.py @@ -877,7 +877,7 @@ def get_new_list_tensor(old_list): new_list_tensor.append(dim) else: assert (isinstance(dim, int)) - temp_out = var.block.create_var(dtype='int32') + temp_out = var.block.create_var(dtype='int64') fill_constant([1], dim, force_cpu=True, out=temp_out) new_list_tensor.append(temp_out) return new_list_tensor @@ -2121,7 +2121,8 @@ class Operator(object): 'fl_listen_and_serv', 'ncclInit', 'select', 'checkpoint_notify', 'gen_bkcl_id', 'c_gen_bkcl_id', 'gen_nccl_id', 'c_gen_nccl_id', 'c_comm_init', 'c_sync_calc_stream', 'c_sync_comm_stream', - 'queue_generator', 'dequeue', 'enqueue', 'heter_listen_and_serv' + 'queue_generator', 'dequeue', 'enqueue', 'heter_listen_and_serv', + 'c_wait_comm', 'c_wait_compute' } def __init__(self, @@ -3030,7 +3031,11 @@ def _is_inited_by(block, var): # In startup_program, "c_broadcast" and "c_sync_comm_stream" # are treated as initialization ops that cause error. # Think of "c_broadcast" and "c_sync_comm_stream" as a special case here. - if op.type in ["c_broadcast", "c_sync_comm_stream"]: + # NOTE: "coalesce_tensor" is a special case for rnn with cudnn support + if op.type in [ + "c_broadcast", "c_sync_comm_stream", + "coalesce_tensor" + ]: continue init_ops.append(op) return init_ops diff --git a/python/paddle/fluid/incubate/fleet/base/role_maker.py b/python/paddle/fluid/incubate/fleet/base/role_maker.py index e3c417d4a6257..d3737e742b478 100644 --- a/python/paddle/fluid/incubate/fleet/base/role_maker.py +++ b/python/paddle/fluid/incubate/fleet/base/role_maker.py @@ -1039,11 +1039,17 @@ def generate_role(self): self._node_type = 1 self._cur_endpoint = worker_endpoints[current_id] gloo = fluid.core.Gloo() - gloo.init(current_id, - len(worker_endpoints), - self._hdfs_path.rstrip("/") + "/trainer", - self._hdfs_name, self._hdfs_ugi, self._iface, - self._prefix) + + gloo.set_rank(current_id) + gloo.set_size(len(worker_endpoints)) + gloo.set_prefix(self._prefix) + gloo.set_iface(self._iface) + gloo.set_timeout_seconds(self._init_timeout_seconds, + self._run_timeout_seconds) + gloo.set_hdfs_store( + self._hdfs_path.rstrip("/") + "/trainer", self._hdfs_name, + self._hdfs_ugi) + gloo.init() self._node_type_comm = gloo elif training_role == "XPU": role = Role.XPU @@ -1051,10 +1057,17 @@ def generate_role(self): self._node_type = 2 self._cur_endpoint = xpu_endpoints[current_id] gloo = fluid.core.Gloo() - gloo.init(current_id, - len(xpu_endpoints), - self._hdfs_path.rstrip("/") + "/xpu", self._hdfs_name, - self._hdfs_ugi, self._iface, self._prefix) + + gloo.set_rank(current_id) + gloo.set_size(len(xpu_endpoints)) + gloo.set_prefix(self._prefix) + gloo.set_iface(self._iface) + gloo.set_timeout_seconds(self._init_timeout_seconds, + self._run_timeout_seconds) + gloo.set_hdfs_store( + self._hdfs_path.rstrip("/") + "/xpu", self._hdfs_name, + self._hdfs_ugi) + gloo.init() self._node_type_comm = gloo elif training_role == "PSERVER": role = Role.SERVER @@ -1070,30 +1083,47 @@ def generate_role(self): self._node_type = 0 self._cur_endpoint = cur_endpoint gloo = fluid.core.Gloo() - gloo.init(current_id, - len(eplist), - self._hdfs_path.rstrip("/") + "/pserver", - self._hdfs_name, self._hdfs_ugi, self._iface, - self._prefix) + gloo.set_rank(current_id) + gloo.set_size(len(eplist)) + gloo.set_prefix(self._prefix) + gloo.set_iface(self._iface) + gloo.set_timeout_seconds(self._init_timeout_seconds, + self._run_timeout_seconds) + gloo.set_hdfs_store( + self._hdfs_path.rstrip("/") + "/pserver", self._hdfs_name, + self._hdfs_ugi) + gloo.init() self._node_type_comm = gloo if training_role == "TRAINER" or training_role == "XPU": gloo = fluid.core.Gloo() heter_list = worker_endpoints + xpu_endpoints - gloo.init( - heter_list.index(self._cur_endpoint), - len(heter_list), + + gloo.set_rank(heter_list.index(self._cur_endpoint)) + gloo.set_size(len(heter_list)) + gloo.set_prefix(self._prefix) + gloo.set_iface(self._iface) + gloo.set_timeout_seconds(self._init_timeout_seconds, + self._run_timeout_seconds) + gloo.set_hdfs_store( self._hdfs_path.rstrip("/") + "/heter", self._hdfs_name, - self._hdfs_ugi, self._iface, self._prefix) + self._hdfs_ugi) + gloo.init() self._heter_comm = gloo gloo = fluid.core.Gloo() all_list = worker_endpoints + eplist + xpu_endpoints - gloo.init( - all_list.index(self._cur_endpoint), - len(all_list), + + gloo.set_rank(all_list.index(self._cur_endpoint)) + gloo.set_size(len(all_list)) + gloo.set_prefix(self._prefix) + gloo.set_iface(self._iface) + gloo.set_timeout_seconds(self._init_timeout_seconds, + self._run_timeout_seconds) + gloo.set_hdfs_store( self._hdfs_path.rstrip("/") + "/all", self._hdfs_name, - self._hdfs_ugi, self._iface, self._prefix) + self._hdfs_ugi) + gloo.init() self._all_comm = gloo self._trainers_num = trainers_num diff --git a/python/paddle/fluid/incubate/fleet/parameter_server/ir/public.py b/python/paddle/fluid/incubate/fleet/parameter_server/ir/public.py index b987e01bba46e..baf8add04caad 100644 --- a/python/paddle/fluid/incubate/fleet/parameter_server/ir/public.py +++ b/python/paddle/fluid/incubate/fleet/parameter_server/ir/public.py @@ -31,7 +31,7 @@ from paddle.fluid.transpiler.details.program_utils import delete_ops OP_NAME_SCOPE = "op_namescope" -CLIP_OP_NAME_SCOPE = "@CLIP" +CLIP_OP_NAME_SCOPE = "gradient_clip" STEP_COUNTER = "@PS_STEP_COUNTER@" LEARNING_RATE_DECAY_COUNTER = "@LR_DECAY_COUNTER@" diff --git a/python/paddle/fluid/incubate/fleet/parameter_server/ir/trainer_pass.py b/python/paddle/fluid/incubate/fleet/parameter_server/ir/trainer_pass.py index 2292d4c0a4d6f..08e64c15c483b 100644 --- a/python/paddle/fluid/incubate/fleet/parameter_server/ir/trainer_pass.py +++ b/python/paddle/fluid/incubate/fleet/parameter_server/ir/trainer_pass.py @@ -32,7 +32,7 @@ from paddle.fluid.incubate.fleet.parameter_server.mode import DistributedMode OP_NAME_SCOPE = "op_namescope" -CLIP_OP_NAME_SCOPE = "@CLIP" +CLIP_OP_NAME_SCOPE = "gradient_clip" STEP_COUNTER = "@PS_STEP_COUNTER@" OP_ROLE_VAR_ATTR_NAME = core.op_proto_and_checker_maker.kOpRoleVarAttrName() RPC_OP_ROLE_ATTR_NAME = core.op_proto_and_checker_maker.kOpRoleAttrName() diff --git a/python/paddle/fluid/layers/control_flow.py b/python/paddle/fluid/layers/control_flow.py index b735ae247f94d..3a06b84d111c4 100755 --- a/python/paddle/fluid/layers/control_flow.py +++ b/python/paddle/fluid/layers/control_flow.py @@ -1098,6 +1098,10 @@ def assign_skip_lod_tensor_array(input, output): """ Assign input to output, but skip the process of copying LoDTensorArray unless it's created in while_block. """ + if not isinstance(input, Variable) and not isinstance(input, core.VarBase): + output = input + return + if input.type == core.VarDesc.VarType.LOD_TENSOR_ARRAY: main_program = input.block.program parent_block = main_program.block(main_program.current_block() diff --git a/python/paddle/fluid/layers/nn.py b/python/paddle/fluid/layers/nn.py index 8d96e46f833e4..00d1db19fc2f5 100755 --- a/python/paddle/fluid/layers/nn.py +++ b/python/paddle/fluid/layers/nn.py @@ -6137,9 +6137,9 @@ def reshape(x, shape, actual_shape=None, act=None, inplace=False, name=None): return dygraph_utils._append_activation_in_dygraph(out, act) - check_variable_and_dtype( - x, 'x', ['float16', 'float32', 'float64', 'int32', 'int64', - 'bool'], 'reshape') + check_variable_and_dtype(x, 'x', [ + 'float16', 'float32', 'float64', 'int32', 'int64', 'bool', 'uint16' + ], 'reshape') check_type(shape, 'shape', (list, tuple, Variable), 'reshape') check_type(actual_shape, 'actual_shape', (Variable, type(None)), 'reshape') @@ -11354,9 +11354,11 @@ def _elementwise_op(helper): assert x is not None, 'x cannot be None in {}'.format(op_type) assert y is not None, 'y cannot be None in {}'.format(op_type) check_variable_and_dtype( - x, 'x', ['float16', 'float32', 'float64', 'int32', 'int64'], op_type) + x, 'x', ['float16', 'uint16', 'float32', 'float64', 'int32', 'int64'], + op_type) check_variable_and_dtype( - y, 'y', ['float16', 'float32', 'float64', 'int32', 'int64'], op_type) + y, 'y', ['float16', 'uint16', 'float32', 'float64', 'int32', 'int64'], + op_type) axis = helper.kwargs.get('axis', -1) use_mkldnn = helper.kwargs.get('use_mkldnn', False) @@ -11428,8 +11430,8 @@ def scale(x, scale=1.0, bias=0.0, bias_after_scale=True, act=None, name=None): return dygraph_utils._append_activation_in_dygraph(out) check_variable_and_dtype(x, "x", [ - 'float16', 'float32', 'float64', 'int8', 'int16', 'int32', 'int64', - 'uint8' + 'float16', 'uint16', 'float32', 'float64', 'int8', 'int16', 'int32', + 'int64', 'uint8' ], "scale") inputs = {'X': [x]} attrs = { @@ -13334,7 +13336,7 @@ def shuffle_channel(x, group, name=None): @templatedoc() -def temporal_shift(x, seg_num, shift_ratio=0.25, name=None): +def temporal_shift(x, seg_num, shift_ratio=0.25, name=None, data_format="NCHW"): """ **Temporal Shift Operator** @@ -13348,6 +13350,8 @@ def temporal_shift(x, seg_num, shift_ratio=0.25, name=None): name(str, optional): For detailed information, please refer to :ref:`api_guide_Name`. Usually name is no need to set and None by default. + data_format(str, optional): Data format that specifies the layout of input. + It can be "NCHW" or "NHWC". Default: "NCHW". Returns: out(Tensor): The temporal shifting result is a tensor with the @@ -13365,6 +13369,13 @@ def temporal_shift(x, seg_num, shift_ratio=0.25, name=None): input = paddle.randn([6, 4, 2, 2]) out = F.temporal_shift(x=input, seg_num=2, shift_ratio=0.2) """ + if data_format not in ["NCHW", "NHWC"]: + raise ValueError("Attr(data_format) should be 'NCHW' or 'NHWC'. " + "Received Attr(data_format): {}.".format(data_format)) + if in_dygraph_mode(): + return core.ops.temporal_shift(x, 'seg_num', seg_num, 'shift_ratio', + shift_ratio, 'data_format', data_format) + helper = LayerHelper("temporal_shift", **locals()) check_variable_and_dtype(x, 'x', ['float32', 'float64'], 'temporal_shift') check_type(seg_num, 'seg_num', int, 'temporal_shift') @@ -13375,16 +13386,15 @@ def temporal_shift(x, seg_num, shift_ratio=0.25, name=None): if not isinstance(seg_num, int): raise TypeError("seg_num must be int type.") - if in_dygraph_mode(): - return core.ops.temporal_shift(x, 'seg_num', seg_num, 'shift_ratio', - shift_ratio) - helper.append_op( type="temporal_shift", inputs={"X": x}, outputs={"Out": out}, - attrs={"seg_num": seg_num, - "shift_ratio": shift_ratio}) + attrs={ + "seg_num": seg_num, + "shift_ratio": shift_ratio, + "data_format": data_format + }) return out diff --git a/python/paddle/fluid/layers/tensor.py b/python/paddle/fluid/layers/tensor.py index cd0d652af8495..84f99962e8430 100644 --- a/python/paddle/fluid/layers/tensor.py +++ b/python/paddle/fluid/layers/tensor.py @@ -1374,19 +1374,19 @@ def range(start, end, step, dtype, name=None): if not isinstance(start, Variable): with device_guard("cpu"): - start = fill_constant([1], dtype, start) + start = fill_constant([1], dtype, start, force_cpu=True) elif start.dtype != dtype: start = cast(start, dtype) if not isinstance(end, Variable): with device_guard("cpu"): - end = fill_constant([1], dtype, end) + end = fill_constant([1], dtype, end, force_cpu=True) elif end.dtype != dtype: end = cast(end, dtype) if not isinstance(step, Variable): with device_guard("cpu"): - step = fill_constant([1], dtype, step) + step = fill_constant([1], dtype, step, force_cpu=True) elif step.dtype != dtype: step = cast(step, dtype) diff --git a/python/paddle/fluid/layers/utils.py b/python/paddle/fluid/layers/utils.py index 0d278d493bc11..463d9102660f4 100644 --- a/python/paddle/fluid/layers/utils.py +++ b/python/paddle/fluid/layers/utils.py @@ -23,7 +23,7 @@ from sys import version_info -def convert_to_list(value, n, name, dtype=np.int): +def convert_to_list(value, n, name, dtype=int): """ Converts a single numerical type or iterable of numerical types into an numerical type list. diff --git a/python/paddle/fluid/multiprocess_utils.py b/python/paddle/fluid/multiprocess_utils.py index a63825e73638b..82fb0f60b064f 100644 --- a/python/paddle/fluid/multiprocess_utils.py +++ b/python/paddle/fluid/multiprocess_utils.py @@ -25,6 +25,10 @@ else: import queue +# multi-process worker check indices queue interval, avoid +# hanging in subprocess data loading +MP_STATUS_CHECK_INTERVAL = 5. + # NOTE: [ mmap files clear ] If there is still data in the multiprocess queue when the main process finishes reading, # the data in the queue needs to be popped. Then the LoDTensor read by the main process # from the child process will automatically clear the memory-mapped file. diff --git a/python/paddle/fluid/optimizer.py b/python/paddle/fluid/optimizer.py index 01a0a78fbaa9d..9c724cbfdd4a7 100755 --- a/python/paddle/fluid/optimizer.py +++ b/python/paddle/fluid/optimizer.py @@ -4075,12 +4075,15 @@ def _insert_sendrecv_for_data_var(self, main_block, programs, startup, break source_var = main_program.block(0).var(var_name) new_var = self._create_var(block, source_var, var_name) + new_var_shape = list(new_var.shape) + new_var_shape[0] = self.micro_batch_size if new_var_shape[ + 0] < 0 else new_var_shape[0] block._insert_op( index=index, type='recv_v2', outputs={'Out': [new_var]}, attrs={ - 'out_shape': new_var.shape, + 'out_shape': new_var_shape, 'dtype': new_var.dtype, self._op_device_key: device, self._op_role_key: self._op_role.Forward, @@ -4243,12 +4246,15 @@ def _insert_sendrecv_ops_for_boundaries(self, block): 'peer': cur_device_index, }) extra_index += 1 + var_shape = list(var.shape) + var_shape[0] = self.micro_batch_size if var_shape[ + 0] < 0 else var_shape[0] block._insert_op( index=index + extra_index, type='recv_v2', outputs={'Out': [var]}, attrs={ - 'out_shape': var.shape, + 'out_shape': var_shape, 'dtype': var.dtype, self._op_device_key: cur_device_spec, self._op_role_key: op_role, @@ -4267,6 +4273,7 @@ def _clear_gradients(self, main_block, dev_spec): grad_name = self._append_grad_suffix(param_name) if not main_block.has_var(grad_name): continue grad_var = main_block.vars[grad_name] + grad_var.persistable = True main_block._insert_op( index=0, type='fill_constant', @@ -4455,6 +4462,8 @@ def minimize(self, optimize_ops, params_grads = self._optimizer.minimize( loss, startup_program, parameter_list, no_grad_set) self._param_device_map = self._optimizer._param_device_map + self.micro_batch_size = main_block.program._pipeline_opt[ + 'micro_batch_size'] # Step1: add default op_device attribute for regulization and clip ops self._add_opdevice_attr_for_regularization_clip(main_block) @@ -4509,6 +4518,7 @@ def device_cmp(device1, device2): "You must use pipeline with fleet" local_rank = main_program._pipeline_opt['local_rank'] % len( device_specs) + self.schedule_mode = main_program._pipeline_opt['schedule_mode'] place_list = [] for dev_spec in device_specs: @@ -4535,6 +4545,9 @@ def device_cmp(device1, device2): main_program._pipeline_opt = { "trainer": "PipelineTrainer", "device_worker": "Section", + "pipeline_stage": local_rank, + "num_pipeline_stages": len(device_specs), + "schedule_mode": self.schedule_mode, "inner_parallelism": len(device_specs), "section_program": program_list[local_rank], "place": place_list[local_rank], diff --git a/python/paddle/fluid/tests/book/CMakeLists.txt b/python/paddle/fluid/tests/book/CMakeLists.txt index 6f717302468af..09c650f16e2fb 100644 --- a/python/paddle/fluid/tests/book/CMakeLists.txt +++ b/python/paddle/fluid/tests/book/CMakeLists.txt @@ -9,7 +9,7 @@ endforeach() set_tests_properties(test_word2vec_book PROPERTIES TIMEOUT 120) set_tests_properties(test_recognize_digits PROPERTIES TIMEOUT 120) set_tests_properties(test_image_classification PROPERTIES TIMEOUT 200) -set_tests_properties(test_label_semantic_roles PROPERTIES TIMEOUT 120) +set_tests_properties(test_label_semantic_roles PROPERTIES TIMEOUT 240) set_tests_properties(test_machine_translation PROPERTIES TIMEOUT 120) set_tests_properties(test_rnn_encoder_decoder PROPERTIES TIMEOUT 120) set_tests_properties(test_fit_a_line PROPERTIES TIMEOUT 120) diff --git a/python/paddle/fluid/tests/book/test_fit_a_line.py b/python/paddle/fluid/tests/book/test_fit_a_line.py index 9a2cc4ab1a1b9..df43d9366ff78 100644 --- a/python/paddle/fluid/tests/book/test_fit_a_line.py +++ b/python/paddle/fluid/tests/book/test_fit_a_line.py @@ -26,7 +26,7 @@ paddle.enable_static() -def train(use_cuda, save_dirname, is_local): +def train(use_cuda, save_dirname, is_local, use_bf16): x = fluid.layers.data(name='x', shape=[13], dtype='float32') y_predict = fluid.layers.fc(input=x, size=1, act=None) @@ -37,6 +37,8 @@ def train(use_cuda, save_dirname, is_local): avg_cost = fluid.layers.mean(cost) sgd_optimizer = fluid.optimizer.SGD(learning_rate=0.001) + if use_bf16: + paddle.static.amp.rewrite_program_bf16(fluid.default_main_program()) sgd_optimizer.minimize(avg_cost) BATCH_SIZE = 20 @@ -133,14 +135,17 @@ def infer(use_cuda, save_dirname=None): print("ground truth: ", test_label) -def main(use_cuda, is_local=True): +def main(use_cuda, is_local=True, use_bf16=False): if use_cuda and not fluid.core.is_compiled_with_cuda(): return + if use_bf16 and not fluid.core.is_compiled_with_mkldnn(): + return + # Directory for saving the trained model save_dirname = "fit_a_line.inference.model" - train(use_cuda, save_dirname, is_local) + train(use_cuda, save_dirname, is_local, use_bf16) infer(use_cuda, save_dirname) @@ -153,6 +158,12 @@ def test_cuda(self): with self.program_scope_guard(): main(use_cuda=True) + @unittest.skipIf(not fluid.core.supports_bfloat16(), + "place does not support BF16 evaluation") + def test_bf16(self): + with self.program_scope_guard(): + main(use_cuda=False, use_bf16=True) + @contextlib.contextmanager def program_scope_guard(self): prog = fluid.Program() diff --git a/python/paddle/fluid/tests/book/test_word2vec_book.py b/python/paddle/fluid/tests/book/test_word2vec_book.py index e33b1cc514aa6..ad7550fa9dd96 100644 --- a/python/paddle/fluid/tests/book/test_word2vec_book.py +++ b/python/paddle/fluid/tests/book/test_word2vec_book.py @@ -39,7 +39,12 @@ def get_place(target): format(target)) -def train(target, is_sparse, is_parallel, save_dirname, is_local=True): +def train(target, + is_sparse, + is_parallel, + save_dirname, + is_local=True, + use_bf16=False): PASS_NUM = 100 EMBED_SIZE = 32 HIDDEN_SIZE = 256 @@ -101,6 +106,8 @@ def __network__(words): raise NotImplementedError() sgd_optimizer = fluid.optimizer.SGD(learning_rate=0.001) + if use_bf16: + paddle.static.amp.rewrite_program_bf16(fluid.default_main_program()) sgd_optimizer.minimize(avg_cost) train_reader = paddle.batch( @@ -239,12 +246,15 @@ def to_infer_tensor(lod_tensor): assert np.isclose(a, b, rtol=5e-5), "a: {}, b: {}".format(a, b) -def main(target, is_sparse, is_parallel): +def main(target, is_sparse, is_parallel, use_bf16): if target == "cuda" and not fluid.core.is_compiled_with_cuda(): return if target == "xpu" and not fluid.core.is_compiled_with_xpu(): return + if use_bf16 and not fluid.core.is_compiled_with_mkldnn(): + return + if not is_parallel: save_dirname = "word2vec.inference.model" else: @@ -255,7 +265,7 @@ def main(target, is_sparse, is_parallel): # so only inference is turned on. train("cpu", is_sparse, is_parallel, save_dirname) else: - train(target, is_sparse, is_parallel, save_dirname) + train(target, is_sparse, is_parallel, save_dirname, use_bf16=use_bf16) infer(target, save_dirname) @@ -268,10 +278,11 @@ class W2VTest(unittest.TestCase): pass -def inject_test_method(target, is_sparse, is_parallel): - fn_name = "test_{0}_{1}_{2}".format(target, "sparse" - if is_sparse else "dense", "parallel" - if is_parallel else "normal") +def inject_test_method(target, is_sparse, is_parallel, use_bf16=False): + fn_name = "test_{0}_{1}_{2}{3}".format(target, "sparse" + if is_sparse else "dense", "parallel" + if is_parallel else "normal", "_bf16" + if use_bf16 else "") def __impl__(*args, **kwargs): prog = fluid.Program() @@ -279,8 +290,7 @@ def __impl__(*args, **kwargs): scope = fluid.core.Scope() with fluid.scope_guard(scope): with fluid.program_guard(prog, startup_prog): - main( - target=target, is_sparse=is_sparse, is_parallel=is_parallel) + main(target, is_sparse, is_parallel, use_bf16) if (not fluid.core.is_compiled_with_cuda() or target == "cuda") and is_sparse: @@ -297,6 +307,7 @@ def __impl__(*args, **kwargs): for is_sparse in (False, True): for is_parallel in (False, ): inject_test_method(target, is_sparse, is_parallel) +inject_test_method("cpu", False, False, use_bf16=True) if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/CMakeLists.txt b/python/paddle/fluid/tests/custom_op/CMakeLists.txt index 3f85f4ef50a22..36496ec499fd9 100644 --- a/python/paddle/fluid/tests/custom_op/CMakeLists.txt +++ b/python/paddle/fluid/tests/custom_op/CMakeLists.txt @@ -3,31 +3,31 @@ if(WITH_GPU) # 'test_custom_relu_op_setup/jit' compile .cc and .cu file py_test(test_custom_relu_op_setup SRCS test_custom_relu_op_setup.py) py_test(test_custom_relu_op_jit SRCS test_custom_relu_op_jit.py) + py_test(test_custom_relu_model SRCS test_custom_relu_model.py) # Compiling shared library will cost some time, but running process is very fast. set_tests_properties(test_custom_relu_op_setup PROPERTIES TIMEOUT 250) set_tests_properties(test_custom_relu_op_jit PROPERTIES TIMEOUT 180) + set_tests_properties(test_custom_relu_model PROPERTIES TIMEOUT 180) endif() py_test(test_sysconfig SRCS test_sysconfig.py) -# 'test_dispatch' compile .cc file +# CPU custom op tests: only compile .cc file py_test(test_dispatch_jit SRCS test_dispatch_jit.py) -set_tests_properties(test_dispatch_jit PROPERTIES TIMEOUT 120) - py_test(test_multi_out_jit SRCS test_multi_out_jit.py) -set_tests_properties(test_multi_out_jit PROPERTIES TIMEOUT 120) - py_test(test_custom_attrs_jit SRCS test_custom_attrs_jit.py) -set_tests_properties(test_custom_attrs_jit PROPERTIES TIMEOUT 120) +py_test(test_custom_concat SRCS test_custom_concat.py) +py_test(test_custom_conj SRCS test_custom_conj.py) + +# other tests +py_test(test_check_abi SRCS test_check_abi.py) +cc_test(test_check_error SRCS test_check_error.cc DEPS gtest) if(NOT LINUX) return() endif() -# TODO(zhouwei): support test_check_abi and abi check on Windows -py_test(test_check_abi SRCS test_check_abi.py) - # Old custom OP only support Linux, only run on Linux py_test(test_custom_op SRCS test_custom_op.py) py_test(test_jit_load SRCS test_jit_load.py) @@ -35,7 +35,7 @@ py_test(test_setup_install SRCS test_setup_install.py) py_test(test_setup_build SRCS test_setup_build.py) set_tests_properties(test_jit_load PROPERTIES TIMEOUT 180) -set_tests_properties(test_setup_install PROPERTIES TIMEOUT 180) +set_tests_properties(test_setup_install PROPERTIES TIMEOUT 250) set_tests_properties(test_setup_build PROPERTIES TIMEOUT 180) diff --git a/python/paddle/fluid/tests/custom_op/attr_test_op.cc b/python/paddle/fluid/tests/custom_op/attr_test_op.cc index 474d3d2d4e2b3..1edc10b8a8a98 100644 --- a/python/paddle/fluid/tests/custom_op/attr_test_op.cc +++ b/python/paddle/fluid/tests/custom_op/attr_test_op.cc @@ -27,27 +27,15 @@ void assign_cpu_kernel(const data_t* x_data, } } -std::vector AttrTestForward( - const paddle::Tensor& x, - bool bool_attr, - int int_attr, - float float_attr, - int64_t int64_attr, - std::string str_attr, - std::vector int_vec_attr, - std::vector float_vec_attr, - std::vector int64_vec_attr, - std::vector str_vec_attr) { - auto out = paddle::Tensor(paddle::PlaceType::kCPU); - out.reshape(x.shape()); - - PD_DISPATCH_FLOATING_TYPES( - x.type(), "assign_cpu_kernel", ([&] { - assign_cpu_kernel( - x.data(), out.mutable_data(), x.size()); - })); - - // Check attrs value +void CheckAllForwardAttrs(const bool& bool_attr, + const int& int_attr, + const float& float_attr, + const int64_t& int64_attr, + const std::string& str_attr, + const std::vector& int_vec_attr, + const std::vector& float_vec_attr, + const std::vector& int64_vec_attr, + const std::vector& str_vec_attr) { if (bool_attr != true) { throw std::runtime_error("bool_attr value error."); } @@ -103,26 +91,11 @@ std::vector AttrTestForward( } } } - - return {out}; } -// The attrs of backward op must be the subset of attrs of forward op -std::vector AttrTestBackward( - const paddle::Tensor& grad_out, - int int_attr, - std::vector float_vec_attr, - std::vector str_vec_attr) { - auto grad_x = paddle::Tensor(paddle::PlaceType::kCPU); - grad_x.reshape(grad_out.shape()); - - PD_DISPATCH_FLOATING_TYPES(grad_out.type(), "assign_cpu_kernel", ([&] { - assign_cpu_kernel( - grad_out.data(), - grad_x.mutable_data(), - grad_out.size()); - })); - +void CheckAllBackwardAttrs(const int& int_attr, + const std::vector& float_vec_attr, + const std::vector& str_vec_attr) { if (int_attr != 10) { throw std::runtime_error("int_attr value error."); } @@ -146,19 +119,141 @@ std::vector AttrTestBackward( } } } +} + +std::vector AttrTestForward( + const paddle::Tensor& x, + bool bool_attr, + int int_attr, + float float_attr, + int64_t int64_attr, + std::string str_attr, + std::vector int_vec_attr, + std::vector float_vec_attr, + std::vector int64_vec_attr, + std::vector str_vec_attr) { + auto out = paddle::Tensor(paddle::PlaceType::kCPU); + out.reshape(x.shape()); + + PD_DISPATCH_FLOATING_TYPES( + x.type(), "assign_cpu_kernel", ([&] { + assign_cpu_kernel( + x.data(), out.mutable_data(), x.size()); + })); + + // Check attrs value + CheckAllForwardAttrs(bool_attr, + int_attr, + float_attr, + int64_attr, + str_attr, + int_vec_attr, + float_vec_attr, + int64_vec_attr, + str_vec_attr); + + return {out}; +} + +// The attrs of backward op must be the subset of attrs of forward op +std::vector AttrTestBackward( + const paddle::Tensor& grad_out, + int int_attr, + std::vector float_vec_attr, + std::vector str_vec_attr) { + auto grad_x = paddle::Tensor(paddle::PlaceType::kCPU); + grad_x.reshape(grad_out.shape()); + + PD_DISPATCH_FLOATING_TYPES(grad_out.type(), "assign_cpu_kernel", ([&] { + assign_cpu_kernel( + grad_out.data(), + grad_x.mutable_data(), + grad_out.size()); + })); + + CheckAllBackwardAttrs(int_attr, float_vec_attr, str_vec_attr); return {grad_x}; } -std::vector> InferShape(std::vector x_shape) { - return {x_shape}; +std::vector ConstAttrTestForward( + const paddle::Tensor& x, + const bool& bool_attr, + const int& int_attr, + const float& float_attr, + const int64_t& int64_attr, + const std::string& str_attr, + const std::vector& int_vec_attr, + const std::vector& float_vec_attr, + const std::vector& int64_vec_attr, + const std::vector& str_vec_attr) { + auto out = paddle::Tensor(paddle::PlaceType::kCPU); + out.reshape(x.shape()); + + PD_DISPATCH_FLOATING_TYPES( + x.type(), "assign_cpu_kernel", ([&] { + assign_cpu_kernel( + x.data(), out.mutable_data(), x.size()); + })); + + // Check attrs value + CheckAllForwardAttrs(bool_attr, + int_attr, + float_attr, + int64_attr, + str_attr, + int_vec_attr, + float_vec_attr, + int64_vec_attr, + str_vec_attr); + + return {out}; } -std::vector InferDType(paddle::DataType x_dtype) { - return {x_dtype}; +// The attrs of backward op must be the subset of attrs of forward op +std::vector ConstAttrTestBackward( + const paddle::Tensor& grad_out, + const int& int_attr, + const std::vector& float_vec_attr, + const std::vector& str_vec_attr) { + auto grad_x = paddle::Tensor(paddle::PlaceType::kCPU); + grad_x.reshape(grad_out.shape()); + + PD_DISPATCH_FLOATING_TYPES(grad_out.type(), "assign_cpu_kernel", ([&] { + assign_cpu_kernel( + grad_out.data(), + grad_x.mutable_data(), + grad_out.size()); + })); + + CheckAllBackwardAttrs(int_attr, float_vec_attr, str_vec_attr); + + return {grad_x}; } -PD_BUILD_OP("attr_test") +PD_BUILD_OP(attr_test) + .Inputs({"X"}) + .Outputs({"Out"}) + .Attrs({"bool_attr: bool", + "int_attr: int", + "float_attr: float", + "int64_attr: int64_t", + "str_attr: std::string", + "int_vec_attr: std::vector", + "float_vec_attr: std::vector", + "int64_vec_attr: std::vector", + "str_vec_attr: std::vector"}) + .SetKernelFn(PD_KERNEL(AttrTestForward)); + +PD_BUILD_GRAD_OP(attr_test) + .Inputs({paddle::Grad("Out")}) + .Outputs({paddle::Grad("X")}) + .Attrs({"int_attr: int", + "float_vec_attr: std::vector", + "str_vec_attr: std::vector"}) + .SetKernelFn(PD_KERNEL(AttrTestBackward)); + +PD_BUILD_OP(const_attr_test) .Inputs({"X"}) .Outputs({"Out"}) .Attrs({"bool_attr: bool", @@ -170,10 +265,9 @@ PD_BUILD_OP("attr_test") "float_vec_attr: std::vector", "int64_vec_attr: std::vector", "str_vec_attr: std::vector"}) - .SetKernelFn(PD_KERNEL(AttrTestForward)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)) - .SetBackwardOp("attr_test_grad") + .SetKernelFn(PD_KERNEL(AttrTestForward)); + +PD_BUILD_GRAD_OP(const_attr_test) .Inputs({paddle::Grad("Out")}) .Outputs({paddle::Grad("X")}) .Attrs({"int_attr: int", diff --git a/python/paddle/fluid/tests/custom_op/concat_and_split.h b/python/paddle/fluid/tests/custom_op/concat_and_split.h new file mode 100644 index 0000000000000..9f24cc4369977 --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/concat_and_split.h @@ -0,0 +1,84 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + +#pragma once + +#include +#include +#include "paddle/extension.h" + +int64_t GetRows(std::vector shape, int64_t axis) { + int64_t rows = 1; + for (int64_t i = 0; i < axis; ++i) { + rows *= shape[i]; + } + return rows; +} + +std::vector GetCols(const std::vector& ins, + int64_t rows, + int64_t* cols) { + std::vector cols_vec(ins.size()); + for (size_t i = 0; i < ins.size(); ++i) { + int64_t t_cols = ins[i].size() / rows; + *cols += t_cols; + cols_vec[i] = t_cols; + } + return cols_vec; +} + +template +void ConcatCpuKernel(const std::vector& ins, + paddle::Tensor* out, + int64_t axis) { + size_t num = ins.size(); + int64_t out_rows = GetRows(ins[0].shape(), axis); + int64_t out_cols = 0; + auto ins_cols = GetCols(ins, out_rows, &out_cols); + + auto* out_data = out->mutable_data(); + int64_t col_idx = 0; + for (size_t i = 0; i < num; ++i) { + int64_t col_len = ins_cols[i]; + auto* in_data = ins[i].data(); + for (int j = 0; j < out_rows; ++j) { + std::memcpy(out_data + j * out_cols + col_idx, + in_data + j * col_len, + sizeof(data_t) * col_len); + } + col_idx += col_len; + } +} + +template +void SplitCpuKernel(const paddle::Tensor& in, + const std::vector& ref_ins, + std::vector* outs, + int64_t axis) { + size_t num = outs->size(); + int64_t in_rows = GetRows(ref_ins[0].shape(), axis); + int64_t in_cols = 0; + auto out_cols = GetCols(ref_ins, in_rows, &in_cols); + + for (size_t i = 0; i < in_rows; ++i) { + auto* in_data = in.data() + i * in_cols; + int64_t col_idx = 0; + for (size_t j = 0; j < num; ++j) { + int64_t col_len = out_cols[j]; + auto* out_data = outs->at(j).mutable_data() + i * col_len; + std::memcpy(out_data, in_data + col_idx, sizeof(data_t) * col_len); + col_idx += col_len; + } + } +} diff --git a/python/paddle/fluid/tests/custom_op/custom_concat_op.cc b/python/paddle/fluid/tests/custom_op/custom_concat_op.cc new file mode 100644 index 0000000000000..a01e01f2bc592 --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/custom_concat_op.cc @@ -0,0 +1,236 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include "concat_and_split.h" // NOLINT +#include "paddle/extension.h" + +#define CHECK_INPUT(x) \ + PD_CHECK(x.place() == paddle::PlaceType::kCPU, #x " must be a CPU Tensor.") + +int64_t ComputeAxis(int64_t axis, int64_t rank) { + PD_CHECK(axis >= -rank && axis < rank, + "The axis is excepted to be in range of [", + -rank, + ", ", + rank, + "]."); + if (axis < 0) { + axis = axis + rank; + } + return axis > 0 ? axis : 0; +} + +std::vector ComputeOutShape( + std::vector> in_shapes, int64_t axis) { + size_t n = in_shapes.size(); + auto out_shape = in_shapes[0]; + size_t zero_dim_size = out_shape.size(); + for (size_t i = 1; i < n; ++i) { + PD_CHECK(in_shapes[i].size() == out_shape.size(), + "Input dimension must be same."); + for (size_t j = 0; j < zero_dim_size; ++j) { + if (j == axis) { + out_shape[axis] += in_shapes[i][j]; + } else { + PD_CHECK(in_shapes[0][j] == in_shapes[i][j], + "The ", + j, + "-th dimension of input must be same."); + } + } + } + return out_shape; +} + +std::vector ConcatForwardDynamicAxis( + const std::vector& inputs, const paddle::Tensor& axis_t) { + // check inputs + PD_CHECK(inputs.size() >= 1, "No Tensor need to be concat."); + for (auto& t : inputs) { + CHECK_INPUT(t); + } + CHECK_INPUT(axis_t); + + // compute output shape + int64_t rank = static_cast(inputs[0].shape().size()); + int64_t axis = axis_t.data()[0]; + axis = ComputeAxis(axis, rank); + std::vector> in_shapes; + for (auto& t : inputs) { + in_shapes.emplace_back(t.shape()); + } + auto out_shape = ComputeOutShape(in_shapes, axis); + + // create output + auto out = paddle::Tensor(paddle::PlaceType::kCPU); + out.reshape(out_shape); + + // calc + PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( + inputs[0].type(), "ConcatCpuKernel", ([&] { + ConcatCpuKernel(inputs, &out, axis); + })); + + return {out}; +} + +std::vector ConcatBackwardDynamicAxis( + const std::vector& inputs, + const paddle::Tensor& grad_out, + const paddle::Tensor& axis_t) { + // check input + PD_CHECK(inputs.size() >= 1, "No Tensor need to be concat."); + for (auto& t : inputs) { + CHECK_INPUT(t); + } + CHECK_INPUT(axis_t); + CHECK_INPUT(grad_out); + + // compate axis + int64_t rank = static_cast(inputs[0].shape().size()); + int64_t axis = axis_t.data()[0]; + axis = ComputeAxis(axis, rank); + + // create outputs + std::vector grad_inputs; + for (auto& t : inputs) { + auto grad = paddle::Tensor(paddle::PlaceType::kCPU); + grad.reshape(t.shape()); + grad_inputs.emplace_back(grad); + } + + // calc + PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( + grad_out.type(), "SplitCpuKernel", ([&] { + SplitCpuKernel(grad_out, inputs, &grad_inputs, axis); + })); + + return grad_inputs; +} + +std::vector> ConcatInferShapeDynamicAxis( + const std::vector>& input_shapes, + const std::vector& axis_shape) { + return {std::vector(input_shapes[0].size(), -1)}; +} + +std::vector ConcatInferDtypeDynamicAxis( + const std::vector& input_dtypes, + const paddle::DataType& axis_dtype) { + return {input_dtypes[0]}; +} + +PD_BUILD_OP(custom_concat) + .Inputs({paddle::Vec("X"), "Axis"}) + .Outputs({"Out"}) + .SetKernelFn(PD_KERNEL(ConcatForwardDynamicAxis)) + .SetInferShapeFn(PD_INFER_SHAPE(ConcatInferShapeDynamicAxis)) + .SetInferDtypeFn(PD_INFER_DTYPE(ConcatInferDtypeDynamicAxis)); + +PD_BUILD_GRAD_OP(custom_concat) + .Inputs({paddle::Vec("X"), paddle::Grad("Out"), "Axis"}) + .Outputs({paddle::Grad(paddle::Vec("X"))}) + .SetKernelFn(PD_KERNEL(ConcatBackwardDynamicAxis)); + +std::vector ConcatForwardStaticAxis( + const std::vector& inputs, const int64_t& axis) { + // check inputs + PD_CHECK(inputs.size() >= 1, "No Tensor need to be concat."); + for (auto& t : inputs) { + CHECK_INPUT(t); + } + + // compute output shape + int64_t rank = static_cast(inputs[0].shape().size()); + auto final_axis = ComputeAxis(axis, rank); + std::vector> in_shapes; + for (auto& t : inputs) { + in_shapes.emplace_back(t.shape()); + } + auto out_shape = ComputeOutShape(in_shapes, final_axis); + + // create output + auto out = paddle::Tensor(paddle::PlaceType::kCPU); + out.reshape(out_shape); + + // calc + PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( + inputs[0].type(), "ConcatCpuKernel", ([&] { + ConcatCpuKernel(inputs, &out, final_axis); + })); + + return {out}; +} + +std::vector ConcatBackwardStaticAxis( + const std::vector& inputs, + const paddle::Tensor& grad_out, + const int64_t& axis) { + // check input + PD_CHECK(inputs.size() >= 1, "No Tensor need to be concat."); + for (auto& t : inputs) { + CHECK_INPUT(t); + } + CHECK_INPUT(grad_out); + + // compate axis + int64_t rank = static_cast(inputs[0].shape().size()); + auto final_axis = ComputeAxis(axis, rank); + + // create outputs + std::vector grad_inputs; + for (auto& t : inputs) { + auto grad = paddle::Tensor(paddle::PlaceType::kCPU); + grad.reshape(t.shape()); + grad_inputs.emplace_back(grad); + } + + // calc + PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( + grad_out.type(), "SplitCpuKernel", ([&] { + SplitCpuKernel(grad_out, inputs, &grad_inputs, final_axis); + })); + + return grad_inputs; +} + +std::vector> ConcatInferShapeStaticAxis( + const std::vector>& input_shapes, + const int64_t& axis) { + int64_t rank = static_cast(input_shapes[0].size()); + auto final_axis = ComputeAxis(axis, rank); + auto out_shape = ComputeOutShape(input_shapes, final_axis); + return {out_shape}; +} + +std::vector ConcatInferDtypeStaticAxis( + const std::vector& input_dtypes) { + return {input_dtypes[0]}; +} + +PD_BUILD_OP(custom_concat_with_attr) + .Inputs({paddle::Vec("X")}) + .Outputs({"Out"}) + .Attrs({"axis: int64_t"}) + .SetKernelFn(PD_KERNEL(ConcatForwardStaticAxis)) + .SetInferShapeFn(PD_INFER_SHAPE(ConcatInferShapeStaticAxis)) + .SetInferDtypeFn(PD_INFER_DTYPE(ConcatInferDtypeStaticAxis)); + +PD_BUILD_GRAD_OP(custom_concat_with_attr) + .Inputs({paddle::Vec("X"), paddle::Grad("Out")}) + .Outputs({paddle::Grad(paddle::Vec("X"))}) + .Attrs({"axis: int64_t"}) + .SetKernelFn(PD_KERNEL(ConcatBackwardStaticAxis)); diff --git a/python/paddle/fluid/tests/custom_op/custom_conj_op.cc b/python/paddle/fluid/tests/custom_op/custom_conj_op.cc new file mode 100644 index 0000000000000..4feb887ca036a --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/custom_conj_op.cc @@ -0,0 +1,94 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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, +// WIdata_tHOUdata_t WARRANdata_tIES OR CONDIdata_tIONS OF ANY KIND, either +// express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include "paddle/extension.h" + +#define CHECK_INPUT(x) \ + PD_CHECK(x.place() == paddle::PlaceType::kCPU, #x " must be a CPU Tensor.") + +template +using EnableComplex = typename std::enable_if< + std::is_same::value || + std::is_same::value>::type; + +template +using DisableComplex = typename std::enable_if< + !std::is_same::value && + !std::is_same::value>::type; + +template +struct ConjFunctor; + +template +struct ConjFunctor> { + ConjFunctor(const data_t* input, int64_t numel, data_t* output) + : input_(input), numel_(numel), output_(output) {} + + void operator()(size_t idx) const { + output_[idx] = data_t(input_[idx].real, -input_[idx].imag); + } + + const data_t* input_; + int64_t numel_; + data_t* output_; +}; + +template +struct ConjFunctor> { + ConjFunctor(const data_t* input, int64_t numel, data_t* output) + : input_(input), numel_(numel), output_(output) {} + + void operator()(size_t idx) const { output_[idx] = input_[idx]; } + + const data_t* input_; + int64_t numel_; + data_t* output_; +}; + +template +void ConjCPUKernel(const data_t* x_data, int64_t numel, data_t* out_data) { + ConjFunctor conj(x_data, numel, out_data); + for (int64_t i = 0; i < numel; ++i) { + conj(i); + } +} + +std::vector ConjFunction(const paddle::Tensor& x) { + CHECK_INPUT(x); + + paddle::Tensor out(x.place()); + out.reshape(x.shape()); + + PD_DISPATCH_FLOATING_AND_COMPLEX_TYPES( + x.type(), "ConjCPUKernel", ([&] { + ConjCPUKernel( + x.data(), x.size(), out.mutable_data()); + })); + + return {out}; +} + +PD_BUILD_OP(custom_conj) + .Inputs({"X"}) + .Outputs({"Out"}) + .SetKernelFn(PD_KERNEL(ConjFunction)); + +PD_BUILD_GRAD_OP(custom_conj) + .Inputs({paddle::Grad("Out")}) + .Outputs({paddle::Grad("X")}) + .SetKernelFn(PD_KERNEL(ConjFunction)); diff --git a/python/paddle/fluid/tests/custom_op/custom_relu_op.cc b/python/paddle/fluid/tests/custom_op/custom_relu_op.cc index 0e358e24ae3e8..c0b30a1cb5579 100644 --- a/python/paddle/fluid/tests/custom_op/custom_relu_op.cc +++ b/python/paddle/fluid/tests/custom_op/custom_relu_op.cc @@ -39,8 +39,8 @@ void relu_cpu_backward_kernel(const data_t* grad_out_data, std::vector relu_cpu_forward(const paddle::Tensor& x) { auto out = paddle::Tensor(paddle::PlaceType::kCPU); - out.reshape(x.shape()); + out.reshape(x.shape()); PD_DISPATCH_FLOATING_TYPES( x.type(), "relu_cpu_forward", ([&] { relu_cpu_forward_kernel( @@ -79,7 +79,7 @@ std::vector ReluForward(const paddle::Tensor& x) { } else if (x.place() == paddle::PlaceType::kGPU) { return relu_cuda_forward(x); } else { - throw std::runtime_error("Not implemented."); + PD_THROW("Not implemented."); } } @@ -92,25 +92,16 @@ std::vector ReluBackward(const paddle::Tensor& x, } else if (x.place() == paddle::PlaceType::kGPU) { return relu_cuda_backward(x, out, grad_out); } else { - throw std::runtime_error("Not implemented."); + PD_THROW("Not implemented."); } } -std::vector> ReluInferShape(std::vector x_shape) { - return {x_shape}; -} - -std::vector ReluInferDType(paddle::DataType x_dtype) { - return {x_dtype}; -} - -PD_BUILD_OP("custom_relu") +PD_BUILD_OP(custom_relu) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(ReluForward)) - .SetInferShapeFn(PD_INFER_SHAPE(ReluInferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(ReluInferDType)) - .SetBackwardOp("relu2_grad") + .SetKernelFn(PD_KERNEL(ReluForward)); + +PD_BUILD_GRAD_OP(custom_relu) .Inputs({"X", "Out", paddle::Grad("Out")}) .Outputs({paddle::Grad("X")}) .SetKernelFn(PD_KERNEL(ReluBackward)); diff --git a/python/paddle/fluid/tests/custom_op/custom_relu_op.cu b/python/paddle/fluid/tests/custom_op/custom_relu_op.cu index a9ce517607093..4ec7d0884582e 100644 --- a/python/paddle/fluid/tests/custom_op/custom_relu_op.cu +++ b/python/paddle/fluid/tests/custom_op/custom_relu_op.cu @@ -20,7 +20,7 @@ __global__ void relu_cuda_forward_kernel(const data_t* x, const int num) { int gid = blockIdx.x * blockDim.x + threadIdx.x; for (int i = gid; i < num; i += blockDim.x * gridDim.x) { - y[i] = max(x[i], static_cast(0.)); + y[i] = x[i] > static_cast(0.) ? x[i] : static_cast(0.); } } @@ -31,20 +31,21 @@ __global__ void relu_cuda_backward_kernel(const data_t* dy, const int num) { int gid = blockIdx.x * blockDim.x + threadIdx.x; for (int i = gid; i < num; i += blockDim.x * gridDim.x) { - dx[i] = dy[i] * (y[i] > 0 ? 1. : 0.); + dx[i] = dy[i] * (y[i] > static_cast(0.) ? static_cast(1.) + : static_cast(0.)); } } std::vector relu_cuda_forward(const paddle::Tensor& x) { auto out = paddle::Tensor(paddle::PlaceType::kGPU); - out.reshape(x.shape()); + out.reshape(x.shape()); int numel = x.size(); int block = 512; int grid = (numel + block - 1) / block; - PD_DISPATCH_FLOATING_TYPES( + PD_DISPATCH_FLOATING_AND_HALF_TYPES( x.type(), "relu_cuda_forward_kernel", ([&] { - relu_cuda_forward_kernel<<>>( + relu_cuda_forward_kernel<<>>( x.data(), out.mutable_data(x.place()), numel); })); @@ -60,9 +61,9 @@ std::vector relu_cuda_backward(const paddle::Tensor& x, int numel = out.size(); int block = 512; int grid = (numel + block - 1) / block; - PD_DISPATCH_FLOATING_TYPES( + PD_DISPATCH_FLOATING_AND_HALF_TYPES( out.type(), "relu_cuda_backward_kernel", ([&] { - relu_cuda_backward_kernel<<>>( + relu_cuda_backward_kernel<<>>( grad_out.data(), out.data(), grad_x.mutable_data(x.place()), diff --git a/python/paddle/fluid/tests/custom_op/custom_relu_op_dup.cc b/python/paddle/fluid/tests/custom_op/custom_relu_op_dup.cc index 7319bdd762645..89d14bfa04960 100644 --- a/python/paddle/fluid/tests/custom_op/custom_relu_op_dup.cc +++ b/python/paddle/fluid/tests/custom_op/custom_relu_op_dup.cc @@ -25,19 +25,14 @@ std::vector ReluBackward(const paddle::Tensor& x, const paddle::Tensor& out, const paddle::Tensor& grad_out); -std::vector> ReluInferShape(std::vector x_shape); - -std::vector ReluInferDType(paddle::DataType x_dtype); - // Reuse codes in `custom_relu_op.cc/cu` to register another custom operator // to test jointly compile multi operators at same time. -PD_BUILD_OP("custom_relu_dup") +PD_BUILD_OP(custom_relu_dup) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(ReluForward)) - .SetInferShapeFn(PD_INFER_SHAPE(ReluInferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(ReluInferDType)) - .SetBackwardOp("relu3_grad") + .SetKernelFn(PD_KERNEL(ReluForward)); + +PD_BUILD_GRAD_OP(custom_relu_dup) .Inputs({"X", "Out", paddle::Grad("Out")}) .Outputs({paddle::Grad("X")}) .SetKernelFn(PD_KERNEL(ReluBackward)); diff --git a/python/paddle/fluid/tests/custom_op/dispatch_test_op.cc b/python/paddle/fluid/tests/custom_op/dispatch_test_op.cc index e09ac2f87c806..0435f50b7c701 100644 --- a/python/paddle/fluid/tests/custom_op/dispatch_test_op.cc +++ b/python/paddle/fluid/tests/custom_op/dispatch_test_op.cc @@ -26,14 +26,6 @@ void assign_cpu_kernel(const data_t* x_data, } } -std::vector> InferShape(std::vector x_shape) { - return {x_shape}; -} - -std::vector InferDType(paddle::DataType x_dtype) { - return {x_dtype}; -} - std::vector DispatchTestInterger(const paddle::Tensor& x) { auto out = paddle::Tensor(paddle::PlaceType::kCPU); out.reshape(x.shape()); @@ -47,18 +39,17 @@ std::vector DispatchTestInterger(const paddle::Tensor& x) { return {out}; } -PD_BUILD_OP("dispatch_test_integer") +PD_BUILD_OP(dispatch_test_integer) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(DispatchTestInterger)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)); + .SetKernelFn(PD_KERNEL(DispatchTestInterger)); -std::vector DispatchTestComplex(const paddle::Tensor& x) { +std::vector DispatchTestFloatAndInteger( + const paddle::Tensor& x) { auto out = paddle::Tensor(paddle::PlaceType::kCPU); out.reshape(x.shape()); - PD_DISPATCH_COMPLEX_TYPES( + PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( x.type(), "assign_cpu_kernel", ([&] { assign_cpu_kernel( x.data(), out.mutable_data(), x.size()); @@ -67,19 +58,16 @@ std::vector DispatchTestComplex(const paddle::Tensor& x) { return {out}; } -PD_BUILD_OP("dispatch_test_complex") +PD_BUILD_OP(dispatch_test_float_and_integer) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(DispatchTestComplex)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)); + .SetKernelFn(PD_KERNEL(DispatchTestFloatAndInteger)); -std::vector DispatchTestFloatAndInteger( - const paddle::Tensor& x) { +std::vector DispatchTestComplex(const paddle::Tensor& x) { auto out = paddle::Tensor(paddle::PlaceType::kCPU); out.reshape(x.shape()); - PD_DISPATCH_FLOATING_AND_INTEGRAL_TYPES( + PD_DISPATCH_COMPLEX_TYPES( x.type(), "assign_cpu_kernel", ([&] { assign_cpu_kernel( x.data(), out.mutable_data(), x.size()); @@ -88,12 +76,10 @@ std::vector DispatchTestFloatAndInteger( return {out}; } -PD_BUILD_OP("dispatch_test_float_and_integer") +PD_BUILD_OP(dispatch_test_complex) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(DispatchTestFloatAndInteger)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)); + .SetKernelFn(PD_KERNEL(DispatchTestComplex)); std::vector DispatchTestFloatAndComplex( const paddle::Tensor& x) { @@ -109,12 +95,10 @@ std::vector DispatchTestFloatAndComplex( return {out}; } -PD_BUILD_OP("dispatch_test_float_and_complex") +PD_BUILD_OP(dispatch_test_float_and_complex) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(DispatchTestFloatAndComplex)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)); + .SetKernelFn(PD_KERNEL(DispatchTestFloatAndComplex)); std::vector DispatchTestFloatAndIntegerAndComplex( const paddle::Tensor& x) { @@ -130,9 +114,25 @@ std::vector DispatchTestFloatAndIntegerAndComplex( return {out}; } -PD_BUILD_OP("dispatch_test_float_and_integer_and_complex") +PD_BUILD_OP(dispatch_test_float_and_integer_and_complex) + .Inputs({"X"}) + .Outputs({"Out"}) + .SetKernelFn(PD_KERNEL(DispatchTestFloatAndIntegerAndComplex)); + +std::vector DispatchTestFloatAndHalf(const paddle::Tensor& x) { + auto out = paddle::Tensor(paddle::PlaceType::kCPU); + out.reshape(x.shape()); + + PD_DISPATCH_FLOATING_AND_HALF_TYPES( + x.type(), "assign_cpu_kernel", ([&] { + assign_cpu_kernel( + x.data(), out.mutable_data(), x.size()); + })); + + return {out}; +} + +PD_BUILD_OP(dispatch_test_float_and_half) .Inputs({"X"}) .Outputs({"Out"}) - .SetKernelFn(PD_KERNEL(DispatchTestFloatAndIntegerAndComplex)) - .SetInferShapeFn(PD_INFER_SHAPE(InferShape)) - .SetInferDtypeFn(PD_INFER_DTYPE(InferDType)); + .SetKernelFn(PD_KERNEL(DispatchTestFloatAndHalf)); diff --git a/python/paddle/fluid/tests/custom_op/multi_out_test_op.cc b/python/paddle/fluid/tests/custom_op/multi_out_test_op.cc index bece0f49845a5..17a36df2cde48 100644 --- a/python/paddle/fluid/tests/custom_op/multi_out_test_op.cc +++ b/python/paddle/fluid/tests/custom_op/multi_out_test_op.cc @@ -68,7 +68,7 @@ std::vector InferDtype(paddle::DataType x_dtype) { return {x_dtype, paddle::DataType::FLOAT64, paddle::DataType::INT32}; } -PD_BUILD_OP("multi_out") +PD_BUILD_OP(multi_out) .Inputs({"X"}) .Outputs({"Out", "Fake_float64", "ZFake_int32"}) .SetKernelFn(PD_KERNEL(MultiOutCPU)) diff --git a/python/paddle/fluid/tests/custom_op/test_check_abi.py b/python/paddle/fluid/tests/custom_op/test_check_abi.py index b171fca2076ac..ed2af83b2342b 100644 --- a/python/paddle/fluid/tests/custom_op/test_check_abi.py +++ b/python/paddle/fluid/tests/custom_op/test_check_abi.py @@ -22,10 +22,11 @@ class TestABIBase(unittest.TestCase): def test_environ(self): - compiler = 'gcc' - for flag in ['1', 'True', 'true']: - os.environ['PADDLE_SKIP_CHECK_ABI'] = flag - self.assertTrue(utils.check_abi_compatibility(compiler)) + compiler_list = ['gcc', 'cl'] + for compiler in compiler_list: + for flag in ['1', 'True', 'true']: + os.environ['PADDLE_SKIP_CHECK_ABI'] = flag + self.assertTrue(utils.check_abi_compatibility(compiler)) def del_environ(self): key = 'PADDLE_SKIP_CHECK_ABI' @@ -33,43 +34,49 @@ def del_environ(self): del os.environ[key] -class TestCheckLinux(TestABIBase): +class TestCheckCompiler(TestABIBase): def test_expected_compiler(self): if utils.OS_NAME.startswith('linux'): gt = ['gcc', 'g++', 'gnu-c++', 'gnu-cc'] - self.assertListEqual(utils._expected_compiler_current_platform(), - gt) + elif utils.IS_WINDOWS: + gt = ['cl'] + elif utils.OS_NAME.startswith('darwin'): + gt = ['clang', 'clang++'] + + self.assertListEqual(utils._expected_compiler_current_platform(), gt) - def test_gcc_version(self): + def test_compiler_version(self): # clear environ self.del_environ() - compiler = 'g++' if utils.OS_NAME.startswith('linux'): - # all CI gcc version > 5.4.0 - self.assertTrue( - utils.check_abi_compatibility( - compiler, verbose=True)) + compiler = 'g++' + elif utils.IS_WINDOWS: + compiler = 'cl' + + # Linux: all CI gcc version > 5.4.0 + # Windows: all CI MSVC version > 19.00.24215 + # Mac: clang has no version limitation, always return true + self.assertTrue(utils.check_abi_compatibility(compiler, verbose=True)) def test_wrong_compiler_warning(self): # clear environ self.del_environ() - compiler = 'nvcc' # fake wrong compiler - if utils.OS_NAME.startswith('linux'): - with warnings.catch_warnings(record=True) as error: - flag = utils.check_abi_compatibility(compiler, verbose=True) - # check return False - self.assertFalse(flag) - # check Compiler Compatibility WARNING - self.assertTrue(len(error) == 1) - self.assertTrue( - "Compiler Compatibility WARNING" in str(error[0].message)) + compiler = 'python' # fake wrong compiler + with warnings.catch_warnings(record=True) as error: + flag = utils.check_abi_compatibility(compiler, verbose=True) + # check return False + self.assertFalse(flag) + # check Compiler Compatibility WARNING + self.assertTrue(len(error) == 1) + self.assertTrue( + "Compiler Compatibility WARNING" in str(error[0].message)) def test_exception(self): # clear environ self.del_environ() compiler = 'python' # fake command if utils.OS_NAME.startswith('linux'): - # to skip _expected_compiler_current_platform + def fake(): return [compiler] @@ -89,40 +96,6 @@ def fake(): utils._expected_compiler_current_platform = raw_func -class TestCheckMacOs(TestABIBase): - def test_expected_compiler(self): - if utils.OS_NAME.startswith('darwin'): - gt = ['clang', 'clang++'] - self.assertListEqual(utils._expected_compiler_current_platform(), - gt) - - def test_gcc_version(self): - # clear environ - self.del_environ() - - if utils.OS_NAME.startswith('darwin'): - # clang has no version limitation. - self.assertTrue(utils.check_abi_compatibility()) - - -class TestCheckWindows(TestABIBase): - def test_gcc_version(self): - # clear environ - self.del_environ() - - if utils.IS_WINDOWS: - # we skip windows now - self.assertTrue(utils.check_abi_compatibility()) - - -class TestJITCompilerException(unittest.TestCase): - def test_exception(self): - with self.assertRaisesRegexp(RuntimeError, - "Failed to check Python interpreter"): - file_path = os.path.abspath(__file__) - utils._jit_compile(file_path, interpreter='fake_cmd', verbose=True) - - class TestRunCMDException(unittest.TestCase): def test_exception(self): for verbose in [True, False]: diff --git a/python/paddle/fluid/tests/custom_op/test_check_error.cc b/python/paddle/fluid/tests/custom_op/test_check_error.cc new file mode 100644 index 0000000000000..eda521a89662f --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/test_check_error.cc @@ -0,0 +1,169 @@ +/* Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + +#include +#include +#include "gtest/gtest.h" +#include "paddle/fluid/extension/include/ext_exception.h" + +TEST(PD_THROW, empty) { + bool caught_exception = false; + try { + PD_THROW(); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("An error occurred.") != std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc:20") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc:20") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); +} + +TEST(PD_THROW, non_empty) { + bool caught_exception = false; + try { + PD_THROW("PD_THROW returns ", + false, + ". DataType of ", + 1, + " is INT. ", + "DataType of ", + 0.23, + " is FLOAT. "); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("PD_THROW returns 0. DataType of 1 is INT. ") != + std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); +} + +TEST(PD_CHECK, OK) { + PD_CHECK(true); + PD_CHECK(true, "PD_CHECK returns ", true, "now"); + + const size_t a = 1; + const size_t b = 10; + PD_CHECK(a < b); + PD_CHECK(a < b, "PD_CHECK returns ", true, a, "should < ", b); +} + +TEST(PD_CHECK, FAILED) { + bool caught_exception = false; + try { + PD_CHECK(false); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("Expected false, but it's not satisfied.") != + std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); + + caught_exception = false; + try { + PD_CHECK(false, + "PD_CHECK returns ", + false, + ". DataType of ", + 1, + " is INT. ", + "DataType of ", + 0.23, + " is FLOAT. "); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("PD_CHECK returns 0. DataType of 1 is INT. ") != + std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); + + const size_t a = 1; + const size_t b = 10; + caught_exception = false; + try { + PD_CHECK(a > b); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("Expected a > b, but it's not satisfied.") != + std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); + + const size_t c = 123; + const float d = 0.345; + caught_exception = false; + try { + PD_CHECK(c < d, "PD_CHECK returns ", false, ", because ", c, " > ", d); + } catch (const std::exception& e) { + caught_exception = true; + std::string err_msg = e.what(); + EXPECT_TRUE(err_msg.find("PD_CHECK returns 0, because 123 > 0.345") != + std::string::npos); +#if _WIN32 + EXPECT_TRUE(err_msg.find("tests\\custom_op\\test_check_error.cc") != + std::string::npos); +#else + EXPECT_TRUE( + err_msg.find( + "python/paddle/fluid/tests/custom_op/test_check_error.cc") != + std::string::npos); +#endif + } + EXPECT_TRUE(caught_exception); +} diff --git a/python/paddle/fluid/tests/custom_op/test_custom_attrs_jit.py b/python/paddle/fluid/tests/custom_op/test_custom_attrs_jit.py index 754f76cab86f0..1c9c6eedbaeb8 100644 --- a/python/paddle/fluid/tests/custom_op/test_custom_attrs_jit.py +++ b/python/paddle/fluid/tests/custom_op/test_custom_attrs_jit.py @@ -18,7 +18,7 @@ import paddle from paddle.utils.cpp_extension import load, get_build_directory -from utils import paddle_includes, extra_compile_args +from utils import paddle_includes, extra_cc_args, extra_nvcc_args from paddle.utils.cpp_extension.extension_utils import run_cmd # Because Windows don't use docker, the shared lib already exists in the @@ -34,29 +34,44 @@ name='custom_attrs_jit', sources=['attr_test_op.cc'], extra_include_paths=paddle_includes, # add for Coverage CI - extra_cxx_cflags=extra_compile_args, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cflags + extra_cuda_cflags=extra_nvcc_args, # test for cflags verbose=True) class TestJitCustomAttrs(unittest.TestCase): - def test_attr_value(self): + def setUp(self): paddle.set_device('cpu') # prepare test value - bool_attr = True - int_attr = 10 - float_attr = 3.14 - int64_attr = 10000000000 - str_attr = "StrAttr" - int_vec_attr = [10, 10, 10] - float_vec_attr = [3.14, 3.14, 3.14] - int64_vec_attr = [10000000000, 10000000000, 10000000000] - str_vec_attr = ["StrAttr", "StrAttr", "StrAttr"] + self.bool_attr = True + self.int_attr = 10 + self.float_attr = 3.14 + self.int64_attr = 10000000000 + self.str_attr = "StrAttr" + self.int_vec_attr = [10, 10, 10] + self.float_vec_attr = [3.14, 3.14, 3.14] + self.int64_vec_attr = [10000000000, 10000000000, 10000000000] + self.str_vec_attr = ["StrAttr", "StrAttr", "StrAttr"] + def test_attr_value(self): x = paddle.ones([2, 2], dtype='float32') x.stop_gradient = False out = custom_attrs.attr_test( - x, bool_attr, int_attr, float_attr, int64_attr, str_attr, - int_vec_attr, float_vec_attr, int64_vec_attr, str_vec_attr) + x, self.bool_attr, self.int_attr, self.float_attr, self.int64_attr, + self.str_attr, self.int_vec_attr, self.float_vec_attr, + self.int64_vec_attr, self.str_vec_attr) + out.stop_gradient = False + out.backward() + + self.assertTrue(np.array_equal(x.numpy(), out.numpy())) + + def test_const_attr_value(self): + x = paddle.ones([2, 2], dtype='float32') + x.stop_gradient = False + out = custom_attrs.const_attr_test( + x, self.bool_attr, self.int_attr, self.float_attr, self.int64_attr, + self.str_attr, self.int_vec_attr, self.float_vec_attr, + self.int64_vec_attr, self.str_vec_attr) out.stop_gradient = False out.backward() diff --git a/python/paddle/fluid/tests/custom_op/test_custom_concat.py b/python/paddle/fluid/tests/custom_op/test_custom_concat.py new file mode 100644 index 0000000000000..ea41126c1c471 --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/test_custom_concat.py @@ -0,0 +1,172 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import os +import unittest +import numpy as np + +import paddle +import paddle.static as static +from paddle.utils.cpp_extension import load, get_build_directory +from paddle.utils.cpp_extension.extension_utils import run_cmd +from utils import paddle_includes, extra_cc_args, extra_nvcc_args + +# Because Windows don't use docker, the shared lib already exists in the +# cache dir, it will not be compiled again unless the shared lib is removed. +file = '{}\\custom_relu_module_jit\\custom_relu_module_jit.pyd'.format( + get_build_directory()) +if os.name == 'nt' and os.path.isfile(file): + cmd = 'del {}'.format(file) + run_cmd(cmd, True) + +if os.name == 'nt': + test_include = "..\\python\\paddle\\fluid\\tests\\custom_op" +else: + test_include = "../python/paddle/fluid/tests/custom_op" +paddle_includes.append(test_include) + +custom_ops = load( + name='custom_concat_jit', + sources=['custom_concat_op.cc'], + extra_include_paths=paddle_includes, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cc flags + extra_cuda_cflags=extra_nvcc_args, # test for nvcc flags + verbose=True) + + +def concat_dynamic(func, dtype, np_inputs, axis_v, with_attr=False): + paddle.set_device("cpu") + inputs = [ + paddle.to_tensor( + x, dtype=dtype, stop_gradient=False) for x in np_inputs + ] + if with_attr: + axis = axis_v + else: + axis = paddle.full(shape=[1], dtype='int64', fill_value=axis_v) + out = func(inputs, axis) + out.stop_gradient = False + out.backward() + grad_inputs = [x.grad for x in inputs] + return out.numpy(), grad_inputs + + +def concat_static(func, dtype, np_inputs, axis_v, with_attr=False): + paddle.enable_static() + paddle.set_device("cpu") + with static.scope_guard(static.Scope()): + with static.program_guard(static.Program()): + x1 = static.data(name="x1", shape=[2, 3], dtype=dtype) + x2 = static.data(name="x2", shape=[2, 3], dtype=dtype) + if with_attr: + axis = axis_v + else: + axis = paddle.full(shape=[1], dtype='int64', fill_value=axis_v) + x1.stop_gradient = False + x2.stop_gradient = False + out = func([x1, x2], axis) + # mean only support float, so here use sum + sum_out = paddle.sum(out) + static.append_backward(sum_out) + + exe = static.Executor() + exe.run(static.default_startup_program()) + + if with_attr: + feed_dict = { + "x1": np_inputs[0].astype(dtype), + "x2": np_inputs[1].astype(dtype) + } + else: + feed_dict = { + "x1": np_inputs[0].astype(dtype), + "x2": np_inputs[1].astype(dtype), + "axis": axis + } + out_v, x1_grad_v, x2_grad_v = exe.run( + static.default_main_program(), + feed=feed_dict, + fetch_list=[out.name, x1.name + "@GRAD", x2.name + "@GRAD"]) + paddle.disable_static() + return out_v, x1_grad_v, x2_grad_v + + +class TestCustomConcatDynamicAxisJit(unittest.TestCase): + def setUp(self): + self.dtypes = ['float32', 'float64', 'int32', 'int64'] + self.np_inputs = [ + np.array([[1, 2, 3], [4, 5, 6]]), + np.array([[11, 12, 13], [14, 15, 16]]) + ] + self.axises = [0, 1] + + def check_output(self, out, pd_out, name): + self.assertTrue( + np.array_equal(out, pd_out), + "custom op {}: {},\n paddle api {}: {}".format(name, out, name, + pd_out)) + + def test_dynamic(self): + for dtype in self.dtypes: + for axis in self.axises: + out, grad_inputs = concat_dynamic(custom_ops.custom_concat, + dtype, self.np_inputs, axis) + pd_out, pd_grad_inputs = concat_dynamic(paddle.concat, dtype, + self.np_inputs, axis) + + self.check_output(out, pd_out, "out") + for x_grad, pd_x_grad in zip(grad_inputs, pd_grad_inputs): + self.check_output(x_grad, pd_x_grad, "x_grad") + + def test_static(self): + for dtype in self.dtypes: + for axis in self.axises: + out, x1_grad, x2_grad = concat_static( + custom_ops.custom_concat, dtype, self.np_inputs, axis) + pd_out, pd_x1_grad, pd_x2_grad = concat_static( + paddle.concat, dtype, self.np_inputs, axis) + + self.check_output(out, pd_out, "out") + self.check_output(x1_grad, pd_x1_grad, "x1_grad") + self.check_output(x2_grad, pd_x2_grad, "x2_grad") + + def test_dynamic_with_attr(self): + for dtype in self.dtypes: + for axis in self.axises: + out, grad_inputs = concat_dynamic( + custom_ops.custom_concat_with_attr, dtype, self.np_inputs, + axis, True) + pd_out, pd_grad_inputs = concat_dynamic( + paddle.concat, dtype, self.np_inputs, axis, True) + + self.check_output(out, pd_out, "out") + for x_grad, pd_x_grad in zip(grad_inputs, pd_grad_inputs): + self.check_output(x_grad, pd_x_grad, "x_grad") + + def test_static_with_attr(self): + for dtype in self.dtypes: + for axis in self.axises: + out, x1_grad, x2_grad = concat_static( + custom_ops.custom_concat_with_attr, dtype, self.np_inputs, + axis, True) + pd_out, pd_x1_grad, pd_x2_grad = concat_static( + paddle.concat, dtype, self.np_inputs, axis, True) + + self.check_output(out, pd_out, "out") + self.check_output(x1_grad, pd_x1_grad, "x1_grad") + self.check_output(x2_grad, pd_x2_grad, "x2_grad") + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/test_custom_conj.py b/python/paddle/fluid/tests/custom_op/test_custom_conj.py new file mode 100644 index 0000000000000..3a8f79a06fc0b --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/test_custom_conj.py @@ -0,0 +1,136 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import os +import unittest +import numpy as np + +import paddle +import paddle.static as static +from paddle.utils.cpp_extension import load, get_build_directory +from paddle.utils.cpp_extension.extension_utils import run_cmd +from utils import paddle_includes, extra_cc_args, extra_nvcc_args + +# Because Windows don't use docker, the shared lib already exists in the +# cache dir, it will not be compiled again unless the shared lib is removed. +file = '{}\\custom_relu_module_jit\\custom_relu_module_jit.pyd'.format( + get_build_directory()) +if os.name == 'nt' and os.path.isfile(file): + cmd = 'del {}'.format(file) + run_cmd(cmd, True) + +custom_ops = load( + name='custom_conj_jit', + sources=['custom_conj_op.cc'], + extra_include_paths=paddle_includes, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cc flags + extra_cuda_cflags=extra_nvcc_args, # test for nvcc flags + verbose=True) + + +def is_complex(dtype): + return dtype == paddle.fluid.core.VarDesc.VarType.COMPLEX64 or \ + dtype == paddle.fluid.core.VarDesc.VarType.COMPLEX128 + + +def to_complex(dtype): + if dtype == "float32": + return np.complex64 + elif dtype == "float64": + return np.complex128 + else: + return dtype + + +def conj_dynamic(func, dtype, np_input): + paddle.set_device("cpu") + x = paddle.to_tensor(np_input) + out = func(x) + out.stop_gradient = False + sum_out = paddle.sum(out) + if is_complex(sum_out.dtype): + sum_out.real().backward() + else: + sum_out.backward() + return out.numpy(), x.grad + + +def conj_static(func, shape, dtype, np_input): + paddle.enable_static() + paddle.set_device("cpu") + with static.scope_guard(static.Scope()): + with static.program_guard(static.Program()): + x = static.data(name="x", shape=shape, dtype=dtype) + x.stop_gradient = False + out = func(x) + sum_out = paddle.sum(out) + static.append_backward(sum_out) + + exe = static.Executor() + exe.run(static.default_startup_program()) + + out_v, x_grad_v = exe.run(static.default_main_program(), + feed={"x": np_input}, + fetch_list=[out.name, x.name + "@GRAD"]) + paddle.disable_static() + return out_v, x_grad_v + + +class TestCustomConjJit(unittest.TestCase): + def setUp(self): + self.dtypes = ['float32', 'float64'] + self.shape = [2, 20, 2, 3] + + def check_output(self, out, pd_out, name): + self.assertTrue( + np.array_equal(out, pd_out), + "custom op {}: {},\n paddle api {}: {}".format(name, out, name, + pd_out)) + + def run_dynamic(self, dtype, np_input): + out, x_grad = conj_dynamic(custom_ops.custom_conj, dtype, np_input) + pd_out, pd_x_grad = conj_dynamic(paddle.conj, dtype, np_input) + + self.check_output(out, pd_out, "out") + self.check_output(x_grad, pd_x_grad, "x's grad") + + def run_static(self, dtype, np_input): + out, x_grad = conj_static(custom_ops.custom_conj, self.shape, dtype, + np_input) + pd_out, pd_x_grad = conj_static(paddle.conj, self.shape, dtype, + np_input) + + self.check_output(out, pd_out, "out") + self.check_output(x_grad, pd_x_grad, "x's grad") + + def test_dynamic(self): + for dtype in self.dtypes: + np_input = np.random.random(self.shape).astype(dtype) + self.run_dynamic(dtype, np_input) + + def test_static(self): + for dtype in self.dtypes: + np_input = np.random.random(self.shape).astype(dtype) + self.run_static(dtype, np_input) + + # complex only used in dynamic mode now + def test_complex_dynamic(self): + for dtype in self.dtypes: + np_input = np.random.random(self.shape).astype( + dtype) + 1j * np.random.random(self.shape).astype(dtype) + self.run_dynamic(to_complex(dtype), np_input) + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/test_custom_relu_model.py b/python/paddle/fluid/tests/custom_op/test_custom_relu_model.py new file mode 100644 index 0000000000000..1d4b2ae161eda --- /dev/null +++ b/python/paddle/fluid/tests/custom_op/test_custom_relu_model.py @@ -0,0 +1,310 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +import os +import unittest +import numpy as np + +import paddle +from paddle import nn +from paddle.utils.cpp_extension import load, get_build_directory +from paddle.utils.cpp_extension.extension_utils import run_cmd + +from utils import paddle_includes, extra_cc_args, extra_nvcc_args + +# Because Windows don't use docker, the shared lib already exists in the +# cache dir, it will not be compiled again unless the shared lib is removed. +file = '{}\\custom_relu_for_model_jit\\custom_relu_for_model_jit.pyd'.format( + get_build_directory()) +if os.name == 'nt' and os.path.isfile(file): + cmd = 'del {}'.format(file) + run_cmd(cmd, True) + +# Compile and load custom op Just-In-Time. +# custom_relu_op_dup.cc is only used for multi ops test, +# not a new op, if you want to test only one op, remove this +# source file +custom_module = load( + name='custom_relu_for_model_jit', + sources=['custom_relu_op.cc', 'custom_relu_op.cu'], + extra_include_paths=paddle_includes, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cc flags + extra_cuda_cflags=extra_nvcc_args, # test for nvcc flags + verbose=True) + + +class Net(nn.Layer): + """ + A simple exmaple for Regression Model. + """ + + def __init__(self, in_dim, out_dim, use_custom_op=False): + super(Net, self).__init__() + self.fc1 = nn.Linear(in_dim, in_dim) + self.fc2 = nn.Linear(in_dim, out_dim) + self.relu_act = custom_module.custom_relu if use_custom_op else nn.functional.relu + + def forward(self, x): + out = self.fc1(x) + out = self.relu_act(out) + out = self.fc2(out) + out = self.relu_act(out) + + out = paddle.mean(out, axis=-1) + + return out + + +class TestDygraphModel(unittest.TestCase): + def setUp(self): + + self.seed = 2021 + self.in_dim = 10 + self.out_dim = 64 + self.batch_num = 10 + self.batch_size = 4 + self.datas = [ + np.random.uniform( + size=[self.batch_size, self.in_dim]).astype('float32') + for i in range(self.batch_num) + ] + self.labels = [ + np.random.uniform(size=[self.batch_size, 1]).astype('float32') + for i in range(self.batch_num) + ] + + self.devices = ['cpu', 'gpu'] + + # for saving model + self.model_path_template = "infer_model/custom_relu_dygaph_model_{}.pdparams" + self.model_dy2stat_path = "infer_model/custom_relu_model_dy2sta" + + # for dy2stat + self.x_spec = paddle.static.InputSpec( + shape=[None, self.in_dim], dtype='float32', name='x') + + def test_train_eval(self): + for device in self.devices: + # set device + paddle.set_device(device) + + # for train + origin_relu_train_out = self.train_model(use_custom_op=False) + custom_relu_train_out = self.train_model(use_custom_op=True) + custom_relu_dy2stat_train_out = self.train_model( + use_custom_op=True, dy2stat=True) # for to_static + + self.assertTrue( + np.array_equal(origin_relu_train_out, custom_relu_train_out)) + self.assertTrue( + np.array_equal(origin_relu_train_out, + custom_relu_dy2stat_train_out)) + + # for eval + origin_relu_eval_out = self.eval_model(use_custom_op=False) + custom_relu_eval_out = self.eval_model(use_custom_op=True) + custom_relu_dy2stat_eval_out = self.eval_model( + use_custom_op=True, dy2stat=True) # for to_static + + self.assertTrue( + np.array_equal(origin_relu_eval_out, custom_relu_eval_out)) + self.assertTrue( + np.array_equal(origin_relu_eval_out, + custom_relu_dy2stat_eval_out)) + + def train_model(self, use_custom_op=False, dy2stat=False): + # reset random seed + paddle.seed(self.seed) + np.random.seed(self.seed) + # paddle.framework.random._manual_program_seed(SEED) + + net = Net(self.in_dim, self.out_dim, use_custom_op) + if dy2stat: + net = paddle.jit.to_static(net, input_spec=[self.x_spec]) + mse_loss = paddle.nn.MSELoss() + sgd = paddle.optimizer.SGD(learning_rate=0.1, + parameters=net.parameters()) + + for batch_id in range(self.batch_num): + x = paddle.to_tensor(self.datas[batch_id]) + y = paddle.to_tensor(self.labels[batch_id]) + + out = net(x) + loss = mse_loss(out, y) + + loss.backward() + sgd.minimize(loss) + net.clear_gradients() + + # save inference model + net.eval() + if dy2stat: + paddle.jit.save(net, self.model_dy2stat_path) + else: + paddle.save(net.state_dict(), + self.model_path_template.format(use_custom_op)) + + return out.numpy() + + def eval_model(self, use_custom_op=False, dy2stat=False): + net = Net(self.in_dim, self.out_dim, use_custom_op) + + if dy2stat: + net = paddle.jit.load(self.model_dy2stat_path) + else: + state_dict = paddle.load( + self.model_path_template.format(use_custom_op)) + net.set_state_dict(state_dict) + + sample_x = paddle.to_tensor(self.datas[0]) + net.eval() + out = net(sample_x) + + return out.numpy() + + +class TestStaticModel(unittest.TestCase): + def setUp(self): + self.seed = 2021 + self.in_dim = 10 + self.out_dim = 64 + self.batch_num = 10 + self.batch_size = 8 + self.datas = [ + np.random.uniform( + size=[self.batch_size, self.in_dim]).astype('float32') + for i in range(self.batch_num) + ] + self.labels = [ + np.random.uniform(size=[self.batch_size, 1]).astype('float32') + for i in range(self.batch_num) + ] + + self.devices = ['cpu', 'gpu'] + + # for saving model + self.model_path_template = "infer_model/custom_relu_static_model_{}_{}" + + paddle.enable_static() + + def tearDown(self): + paddle.disable_static() + + def test_train_eval(self): + for device in self.devices: + # for train + original_relu_train_out = self.train_model( + device, use_custom_op=False) + custom_relu_train_out = self.train_model(device, use_custom_op=True) + # using PE + original_relu_train_pe_out = self.train_model( + device, use_custom_op=False, use_pe=True) + custom_relu_train_pe_out = self.train_model( + device, use_custom_op=True, use_pe=True) + + self.assertTrue( + np.array_equal(original_relu_train_out, custom_relu_train_out)) + self.assertTrue( + np.array_equal(original_relu_train_pe_out, + custom_relu_train_pe_out)) + + # for eval + original_relu_eval_out = self.eval_model( + device, use_custom_op=False) + custom_relu_eval_out = self.eval_model(device, use_custom_op=True) + # using PE + original_relu_eval_pe_out = self.eval_model( + device, use_custom_op=False, use_pe=True) + custom_relu_eval_pe_out = self.eval_model( + device, use_custom_op=True, use_pe=True) + + self.assertTrue( + np.array_equal(original_relu_eval_out, custom_relu_eval_out)) + self.assertTrue( + np.array_equal(original_relu_eval_pe_out, + custom_relu_eval_pe_out)) + + def train_model(self, device, use_custom_op=False, use_pe=False): + # reset random seed + paddle.seed(self.seed) + np.random.seed(self.seed) + # set device + paddle.set_device(device) + + with paddle.static.scope_guard(paddle.static.Scope()): + with paddle.static.program_guard(paddle.static.Program()): + x = paddle.static.data( + shape=[None, self.in_dim], name='x', dtype='float32') + y = paddle.static.data( + shape=[None, 1], name='y', dtype='float32') + + net = Net(self.in_dim, self.out_dim, use_custom_op) + out = net(x) + + loss = nn.functional.mse_loss(out, y) + sgd = paddle.optimizer.SGD(learning_rate=0.01) + sgd.minimize(loss) + + exe = exe = paddle.static.Executor() + exe.run(paddle.static.default_startup_program()) + + # For PE + if use_pe: + places = paddle.static.cpu_places( + ) if device is 'cpu' else paddle.static.cuda_places() + main_program = paddle.static.CompiledProgram( + paddle.static.default_main_program( + )).with_data_parallel( + loss_name=loss.name, places=places) + else: + main_program = paddle.static.default_main_program() + + for batch_id in range(self.batch_num): + x_data = self.datas[batch_id] + y_data = self.labels[batch_id] + + res = exe.run(main_program, + feed={'x': x_data, + 'y': y_data}, + fetch_list=[out]) + + # save model + paddle.static.save_inference_model( + self.model_path_template.format(use_custom_op, use_pe), + [x], [out], exe) + + return res[0] + + def eval_model(self, device, use_custom_op=False, use_pe=False): + paddle.set_device(device) + + with paddle.static.scope_guard(paddle.static.Scope()): + with paddle.static.program_guard(paddle.static.Program()): + exe = paddle.static.Executor() + + [inference_program, feed_target_names, + fetch_targets] = paddle.static.load_inference_model( + self.model_path_template.format(use_custom_op, use_pe), + exe) + + x_data = self.datas[0] + results = exe.run(inference_program, + feed={feed_target_names[0]: x_data}, + fetch_list=fetch_targets) + + return results[0] + + +if __name__ == '__main__': + unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/test_custom_relu_op_jit.py b/python/paddle/fluid/tests/custom_op/test_custom_relu_op_jit.py index 9c108a799d955..23733d20841b3 100644 --- a/python/paddle/fluid/tests/custom_op/test_custom_relu_op_jit.py +++ b/python/paddle/fluid/tests/custom_op/test_custom_relu_op_jit.py @@ -13,13 +13,12 @@ # limitations under the License. import os -import subprocess import unittest import paddle import numpy as np from paddle.utils.cpp_extension import load, get_build_directory from paddle.utils.cpp_extension.extension_utils import run_cmd -from utils import paddle_includes, extra_compile_args +from utils import paddle_includes, extra_cc_args, extra_nvcc_args, IS_WINDOWS from test_custom_relu_op_setup import custom_relu_dynamic, custom_relu_static # Because Windows don't use docker, the shared lib already exists in the @@ -40,8 +39,8 @@ 'custom_relu_op.cc', 'custom_relu_op.cu', 'custom_relu_op_dup.cc' ], extra_include_paths=paddle_includes, # add for Coverage CI - extra_cxx_cflags=extra_compile_args, # add for Coverage CI - extra_cuda_cflags=extra_compile_args, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cc flags + extra_cuda_cflags=extra_nvcc_args, # test for nvcc flags verbose=True) @@ -51,11 +50,17 @@ def setUp(self): custom_module.custom_relu, custom_module.custom_relu_dup ] self.dtypes = ['float32', 'float64'] - self.devices = ['cpu', 'gpu'] + if paddle.is_compiled_with_cuda(): + self.dtypes.append('float16') + self.devices = ['cpu'] + if paddle.is_compiled_with_cuda(): + self.devices.append('gpu') def test_static(self): for device in self.devices: for dtype in self.dtypes: + if device == 'cpu' and dtype == 'float16': + continue x = np.random.uniform(-1, 1, [4, 8]).astype(dtype) for custom_op in self.custom_ops: out = custom_relu_static(custom_op, device, dtype, x) @@ -69,6 +74,8 @@ def test_static(self): def test_dynamic(self): for device in self.devices: for dtype in self.dtypes: + if device == 'cpu' and dtype == 'float16': + continue x = np.random.uniform(-1, 1, [4, 8]).astype(dtype) for custom_op in self.custom_ops: out, x_grad = custom_relu_dynamic(custom_op, device, dtype, @@ -84,6 +91,40 @@ def test_dynamic(self): "custom op x grad: {},\n paddle api x grad: {}".format( x_grad, pd_x_grad)) + def test_exception(self): + caught_exception = False + try: + x = np.random.uniform(-1, 1, [4, 8]).astype('int32') + custom_relu_dynamic(custom_module.custom_relu, 'cpu', 'int32', x) + except OSError as e: + caught_exception = True + self.assertTrue( + "function \"relu_cpu_forward\" is not implemented for data type `int32_t`" + in str(e)) + if IS_WINDOWS: + self.assertTrue( + r"python\paddle\fluid\tests\custom_op\custom_relu_op.cc:48" + in str(e)) + else: + self.assertTrue( + "python/paddle/fluid/tests/custom_op/custom_relu_op.cc:48" + in str(e)) + self.assertTrue(caught_exception) + + caught_exception = False + try: + x = np.random.uniform(-1, 1, [4, 8]).astype('int32') + custom_relu_dynamic(custom_module.custom_relu, 'gpu', 'int32', x) + except OSError as e: + caught_exception = True + self.assertTrue( + "function \"relu_cuda_forward_kernel\" is not implemented for data type `int32_t`" + in str(e)) + self.assertTrue( + "python/paddle/fluid/tests/custom_op/custom_relu_op.cu:50" in + str(e)) + self.assertTrue(caught_exception) + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/test_custom_relu_op_setup.py b/python/paddle/fluid/tests/custom_op/test_custom_relu_op_setup.py index 6781915e021c9..5c5c2d65a5957 100644 --- a/python/paddle/fluid/tests/custom_op/test_custom_relu_op_setup.py +++ b/python/paddle/fluid/tests/custom_op/test_custom_relu_op_setup.py @@ -26,7 +26,7 @@ def custom_relu_dynamic(func, device, dtype, np_x, use_func=True): paddle.set_device(device) - t = paddle.to_tensor(np_x) + t = paddle.to_tensor(np_x, dtype=dtype) t.stop_gradient = False out = func(t) if use_func else paddle.nn.functional.relu(t) @@ -171,7 +171,11 @@ def setUp(self): ] self.dtypes = ['float32', 'float64'] - self.devices = ['cpu', 'gpu'] + if paddle.is_compiled_with_cuda(): + self.dtypes.append('float16') + self.devices = ['cpu'] + if paddle.is_compiled_with_cuda(): + self.devices.append('gpu') # config seed SEED = 2021 @@ -181,6 +185,8 @@ def setUp(self): def test_static(self): for device in self.devices: for dtype in self.dtypes: + if device == 'cpu' and dtype == 'float16': + continue x = np.random.uniform(-1, 1, [4, 8]).astype(dtype) for custom_op in self.custom_ops: out = custom_relu_static(custom_op, device, dtype, x) @@ -194,6 +200,8 @@ def test_static(self): def test_static_pe(self): for device in self.devices: for dtype in self.dtypes: + if device == 'cpu' and dtype == 'float16': + continue x = np.random.uniform(-1, 1, [4, 8]).astype(dtype) for custom_op in self.custom_ops: out = custom_relu_static_pe(custom_op, device, dtype, x) @@ -207,6 +215,8 @@ def test_static_pe(self): def test_dynamic(self): for device in self.devices: for dtype in self.dtypes: + if device == 'cpu' and dtype == 'float16': + continue x = np.random.uniform(-1, 1, [4, 8]).astype(dtype) for custom_op in self.custom_ops: out, x_grad = custom_relu_dynamic(custom_op, device, dtype, diff --git a/python/paddle/fluid/tests/custom_op/test_dispatch_jit.py b/python/paddle/fluid/tests/custom_op/test_dispatch_jit.py index 54d317c37faa9..12e9f50a5e409 100644 --- a/python/paddle/fluid/tests/custom_op/test_dispatch_jit.py +++ b/python/paddle/fluid/tests/custom_op/test_dispatch_jit.py @@ -17,7 +17,7 @@ import paddle import numpy as np from paddle.utils.cpp_extension import load, get_build_directory -from utils import paddle_includes, extra_compile_args +from utils import paddle_includes, extra_cc_args from paddle.utils.cpp_extension.extension_utils import run_cmd # Because Windows don't use docker, the shared lib already exists in the @@ -31,7 +31,7 @@ name='dispatch_op', sources=['dispatch_test_op.cc'], extra_include_paths=paddle_includes, # add for Coverage CI - extra_cxx_cflags=extra_compile_args, + extra_cxx_cflags=extra_cc_args, verbose=True) @@ -83,6 +83,12 @@ def test_dispatch_float_and_integer_and_complex(self): self.run_dispatch_test( dispatch_op.dispatch_test_float_and_integer_and_complex, dtype) + def test_dispatch_float_and_half(self): + dtypes = ["float32", "float64", "float16"] + for dtype in dtypes: + self.run_dispatch_test(dispatch_op.dispatch_test_float_and_half, + dtype) + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/custom_op/test_jit_load.py b/python/paddle/fluid/tests/custom_op/test_jit_load.py index ccb9544433488..4e6d74b7d6099 100644 --- a/python/paddle/fluid/tests/custom_op/test_jit_load.py +++ b/python/paddle/fluid/tests/custom_op/test_jit_load.py @@ -17,7 +17,7 @@ import paddle import numpy as np from paddle.utils.cpp_extension import load -from utils import paddle_includes, extra_compile_args +from utils import paddle_includes, extra_cc_args, extra_nvcc_args from paddle.utils.cpp_extension.extension_utils import use_new_custom_op_load_method # switch to old custom op method @@ -27,10 +27,9 @@ custom_module = load( name='custom_relu2', sources=['relu_op.cc', 'relu_op.cu', 'relu_op3.cc', 'relu_op3.cu'], - interpreter='python', # add for unittest extra_include_paths=paddle_includes, # add for Coverage CI - extra_cxx_cflags=extra_compile_args, # add for Coverage CI, - extra_cuda_cflags=extra_compile_args, # add for split cpp/cuda flags + extra_cxx_cflags=extra_cc_args, # test for cc flags + extra_cuda_cflags=extra_nvcc_args, # test for nvcc flags verbose=True # add for unittest ) diff --git a/python/paddle/fluid/tests/custom_op/test_multi_out_jit.py b/python/paddle/fluid/tests/custom_op/test_multi_out_jit.py index 79d366cc4af44..97b37498c4d3d 100644 --- a/python/paddle/fluid/tests/custom_op/test_multi_out_jit.py +++ b/python/paddle/fluid/tests/custom_op/test_multi_out_jit.py @@ -21,7 +21,7 @@ from paddle.utils.cpp_extension import load from paddle.utils.cpp_extension import load, get_build_directory from paddle.utils.cpp_extension.extension_utils import run_cmd -from utils import paddle_includes, extra_compile_args +from utils import paddle_includes, extra_cc_args # Because Windows don't use docker, the shared lib already exists in the # cache dir, it will not be compiled again unless the shared lib is removed. @@ -35,7 +35,7 @@ name='multi_out_jit', sources=['multi_out_test_op.cc'], extra_include_paths=paddle_includes, # add for Coverage CI - extra_cxx_cflags=extra_compile_args, # add for Coverage CI + extra_cxx_cflags=extra_cc_args, # test for cflags verbose=True) diff --git a/python/paddle/fluid/tests/custom_op/utils.py b/python/paddle/fluid/tests/custom_op/utils.py index 52b294dc72b4b..57ce79b1f3055 100644 --- a/python/paddle/fluid/tests/custom_op/utils.py +++ b/python/paddle/fluid/tests/custom_op/utils.py @@ -27,7 +27,7 @@ os.path.join(site_packages_path, 'paddle', 'include', 'third_party') ] -# TODO(Aurelius84): Memory layout is different if build paddle with PADDLE_WITH_MKLDNN=ON, -# and will lead to ABI problem on Coverage CI. We will handle it in next PR. -extra_compile_args = ['-DPADDLE_WITH_MKLDNN' - ] if six.PY2 and not IS_WINDOWS else [] +# Test for extra compile args +extra_cc_args = ['-w', '-g'] if not IS_WINDOWS else ['/w'] +extra_nvcc_args = ['-O3'] +extra_compile_args = {'cc': extra_cc_args, 'nvcc': extra_nvcc_args} diff --git a/python/paddle/fluid/tests/unittests/CMakeLists.txt b/python/paddle/fluid/tests/unittests/CMakeLists.txt index 796331e7a5a5c..b5c554a58cbbd 100644 --- a/python/paddle/fluid/tests/unittests/CMakeLists.txt +++ b/python/paddle/fluid/tests/unittests/CMakeLists.txt @@ -84,6 +84,7 @@ if(((NOT WITH_ROCM) AND (NOT WITH_GPU)) OR WIN32) LIST(REMOVE_ITEM TEST_OPS test_collective_allreduce_api) LIST(REMOVE_ITEM TEST_OPS test_collective_broadcast_api) LIST(REMOVE_ITEM TEST_OPS test_collective_allgather_api) + LIST(REMOVE_ITEM TEST_OPS test_collective_wait) LIST(REMOVE_ITEM TEST_OPS test_memcpy_op) endif() diff --git a/python/paddle/fluid/tests/unittests/collective_allreduce_op_wait.py b/python/paddle/fluid/tests/unittests/collective_allreduce_op_wait.py new file mode 100644 index 0000000000000..61a0ad3bd7636 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/collective_allreduce_op_wait.py @@ -0,0 +1,114 @@ +# Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import numpy as np +import argparse +import os +import sys +import signal +import time +import socket +from contextlib import closing +from six import string_types +import math +import paddle +import paddle.fluid as fluid +import paddle.fluid.profiler as profiler +import paddle.fluid.unique_name as nameGen +from paddle.fluid import core +import unittest +from multiprocessing import Process +import paddle.fluid.layers as layers +from functools import reduce +from test_collective_base import TestCollectiveRunnerBase, runtime_main + +paddle.enable_static() + + +class TestCollectiveAllreduce(TestCollectiveRunnerBase): + def __init__(self): + self.global_ring_id = 0 + + def get_model(self, main_prog, startup_program): + ring_id = 0 + with fluid.program_guard(main_prog, startup_program): + tindata = layers.data( + name="tindata", shape=[10, 1000], dtype='float32') + toutdata = main_prog.current_block().create_var( + name="outofallreduce", + dtype='float32', + type=core.VarDesc.VarType.LOD_TENSOR, + persistable=False, + stop_gradient=False) + + # tout = tin + tin - tin = tin + if True: + main_prog.global_block().append_op( + type="elementwise_add", + inputs={ + 'X': tindata, + 'Y': tindata, + }, + outputs={'Out': toutdata}, ) + main_prog.global_block().append_op( + type="elementwise_sub", + inputs={ + 'X': toutdata, + 'Y': tindata, + }, + outputs={'Out': toutdata}, ) + + main_prog.global_block().append_op( + type='c_wait_compute', + inputs={'X': toutdata}, + outputs={'Out': toutdata}, + attrs={'ring_id': ring_id}) + + main_prog.global_block().append_op( + type="c_allreduce_sum", + inputs={'X': toutdata}, + attrs={'ring_id': ring_id}, + outputs={'Out': toutdata}, + attr={'use_calc_stream': False}) + + main_prog.global_block().append_op( + type="c_wait_comm", + inputs={'X': toutdata}, + outputs={'Out': toutdata}, + attrs={'ring_id': ring_id}) + + # tout = tin + tout - tin = tout + if True: + main_prog.global_block().append_op( + type="elementwise_add", + inputs={ + 'X': tindata, + 'Y': toutdata, + }, + outputs={'Out': toutdata}, ) + main_prog.global_block().append_op( + type="elementwise_sub", + inputs={ + 'X': toutdata, + 'Y': tindata, + }, + outputs={'Out': toutdata}, ) + + return toutdata + + +if __name__ == "__main__": + runtime_main(TestCollectiveAllreduce, "allreduce", 0) diff --git a/python/paddle/fluid/tests/unittests/collective_sendrecv_op.py b/python/paddle/fluid/tests/unittests/collective_sendrecv_op.py index 0a1967aa658ed..18a7aeccf4c15 100644 --- a/python/paddle/fluid/tests/unittests/collective_sendrecv_op.py +++ b/python/paddle/fluid/tests/unittests/collective_sendrecv_op.py @@ -46,7 +46,10 @@ def get_model(self, main_prog, startup_program): ring_id = self.global_ring_id with fluid.program_guard(main_prog, startup_program): tindata = layers.data( - name="tindata", shape=[10, 1000], dtype='float64') + name="tindata", + shape=[10, 1000], + dtype='float64', + append_batch_size=False) if self.rank == 0: main_prog.global_block().append_op( type="send_v2", diff --git a/python/paddle/fluid/tests/unittests/dist_test.sh b/python/paddle/fluid/tests/unittests/dist_test.sh index d5a6490042b20..69a893a7ddc13 100644 --- a/python/paddle/fluid/tests/unittests/dist_test.sh +++ b/python/paddle/fluid/tests/unittests/dist_test.sh @@ -1,4 +1,19 @@ #!/bin/bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + unset https_proxy http_proxy export FLAGS_rpc_disable_reuse_port=1 @@ -50,14 +65,30 @@ do cat -n ${log} done +# check CUDA or ROCM env +GPU_SYS_INFO_CMD=nvidia-smi + +which ${GPU_SYS_INFO_CMD} +exit_code=$? +if [[ $exit_code -ne 0 ]]; then + GPU_SYS_INFO_CMD=rocm-smi +fi + +which ${GPU_SYS_INFO_CMD} +exit_code=$? +if [[ $exit_code -ne 0 ]]; then + echo "nvidia-smi or rocm-smi faild with ${exit_code}" + exit ${exit_code} +fi + #display system context for i in {1..2}; do sleep 3 ps -aux netstat -anlp - if hash "nvidia-smi" > /dev/null; then - nvidia-smi + if hash "${GPU_SYS_INFO_CMD}" > /dev/null; then + ${GPU_SYS_INFO_CMD} fi done diff --git a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_convert_operators.py b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_convert_operators.py index 631cd426b32b8..54dcc152fd6b2 100644 --- a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_convert_operators.py +++ b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_convert_operators.py @@ -15,6 +15,7 @@ import numpy as np import paddle import unittest +from paddle.jit.dy2static.convert_operators import eval_if_exist_else_none class CallNotExist(paddle.nn.Layer): @@ -189,5 +190,76 @@ def test_negative_attr_shape(self): paddle.shape(x)) +class TestEvaIfExistElseNone(unittest.TestCase): + def test_globals(self): + global x_shape + x_shape = [1, 2, 3] + self.assertEqual(eval_if_exist_else_none('x_shape', locals()), None) + self.assertEqual(eval_if_exist_else_none('x_shape', globals()), x_shape) + + del x_shape + + def test_enclosing_scope(self): + global x_shape + x_shape = [1, 2, 3] + + def foo(): + y_shape = [2, 3, 4] + self.assertEqual( + eval_if_exist_else_none('x_shape', globals()), [1, 2, 3]) + self.assertEqual( + eval_if_exist_else_none('y_shape', locals()), [2, 3, 4]) + + foo() + del x_shape + + def test_global_in_func(self): + x_shape = [1, 2, 3] + + def foo(): + global y_shape + y_shape = [2, 3, 4] + + self.assertEqual( + eval_if_exist_else_none('y_shape', globals()), [2, 3, 4]) + self.assertEqual(eval_if_exist_else_none('x_shape', locals()), None) + self.assertEqual( + eval_if_exist_else_none('x_shape', globals()), None) + + del y_shape + + foo() + + def test_none(self): + def foo(): + x_shape = [2, 3, 4] + return x_shape + + self.assertEqual(eval_if_exist_else_none('x_shape', locals()), None) + + +class ShapeLayer(paddle.nn.Layer): + def __init__(self): + super(ShapeLayer, self).__init__() + + @paddle.jit.to_static(input_spec=[paddle.static.InputSpec(shape=[None, 1])]) + def forward(self, x): + x = paddle.reshape(x, [-1, x.shape[1]]) + bs = x.shape[0] # -1 + + # for trigger choos_shape_attr_or_api + out = paddle.zeros([bs, 1], dtype='float32') + return out + + +class TestChooseShapeAttrOrApiWithLayer(unittest.TestCase): + def test_tensor_shape(self): + x = paddle.zeros(shape=[4, 1], dtype='float32') + net = ShapeLayer() + out = net(x) + + self.assertTrue(np.array_equal(out.numpy(), x.numpy())) + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_for_enumerate.py b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_for_enumerate.py index c28997c5c1c67..517cff39a276f 100644 --- a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_for_enumerate.py +++ b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_for_enumerate.py @@ -233,6 +233,7 @@ def for_iter_var_idx(x_array): return z +# 17. for a,b,c in z: (a, b, c) is a tuple @paddle.jit.to_static def for_tuple_as_iter_var(x_array): x = paddle.to_tensor(x_array) @@ -250,6 +251,7 @@ def for_tuple_as_iter_var(x_array): return a_result, b_result, c_result +# 18. for t in enumerate(collection): t is tuple of (idx, element) @paddle.jit.to_static def for_tuple_as_enumerate_iter(x_array): x = paddle.to_tensor(x_array) @@ -263,6 +265,7 @@ def for_tuple_as_enumerate_iter(x_array): return a_result +# 19. for i, (a, b, c, d, e) in enumerate(collection): (a, b, c, d, e) is a tuple @paddle.jit.to_static def for_tuple_as_enumerate_value(x_array): x = paddle.to_tensor(x_array) @@ -284,6 +287,23 @@ def for_tuple_as_enumerate_value(x_array): return a_result +# 20. test for function in a class +class ForwardContainsForLayer(paddle.nn.Layer): + def __init__(self): + super(ForwardContainsForLayer, self).__init__() + self.high = 5 + self.low = 3 + + @paddle.jit.to_static + def forward(self, x): + # just for test case, x is useless in this method + y = paddle.zeros([10, 2, 3]) + z = [] + for i in range(self.high - self.low): + z.append(y[i].clone()) + return z + + class TestTransformBase(unittest.TestCase): def setUp(self): self.place = fluid.CUDAPlace(0) if fluid.is_compiled_with_cuda( @@ -313,11 +333,11 @@ def get_static_output(self): class TestTransform(TestTransformBase): def transformed_result_compare(self): dy_outs = self.get_dygraph_output() - if not isinstance(dy_outs, tuple): + if not isinstance(dy_outs, (tuple, list)): dy_outs = (dy_outs, ) st_outs = self.get_static_output() - if not isinstance(st_outs, tuple): + if not isinstance(st_outs, (tuple, list)): st_outs = (st_outs, ) for x, y in zip(dy_outs, st_outs): @@ -446,5 +466,10 @@ def set_test_func(self): self.dygraph_func = for_tuple_as_enumerate_value +class TestForwardContainsForLayer(TestForIterVarNumpy): + def set_test_func(self): + self.dygraph_func = ForwardContainsForLayer() + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_loop.py b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_loop.py index bc235ca860649..fe86d5d636811 100644 --- a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_loop.py +++ b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_loop.py @@ -17,6 +17,7 @@ import gast import inspect import numpy as np +import paddle import paddle.fluid as fluid import unittest @@ -157,6 +158,16 @@ def __init__(self): return foo.c +def loop_var_contains_property(x): + a = paddle.zeros(shape=[1], dtype='float32') + i = paddle.to_tensor(x) + s = i.shape + while i < 10 and s[0] >= 1: + a += i.shape[0] + i += 1 + return a + + def for_loop_class_var(max_len): class Foo(object): def __init__(self): @@ -240,9 +251,7 @@ def test_nested_loop_vars(self): name_visitor = NameVisitor(gast_root) self.loop_var_names = [ - set(["j", "two"]), - set(["i", "three", "b"]), - set(["i", "j"]), + set(["j", "two"]), set(["i", "three", "b"]), set(["i", "j"]) ] self.create_var_names = [set(), set(["b"]), set()] @@ -326,6 +335,11 @@ def _init_dyfunc(self): self.dyfunc = while_loop_class_var +class TestLoopVarContainsProperty(TestTransformWhileLoop): + def _init_dyfunc(self): + self.dyfunc = loop_var_contains_property + + class TestTransformForLoop(unittest.TestCase): def setUp(self): self.place = fluid.CUDAPlace(0) if fluid.is_compiled_with_cuda( diff --git a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_tensor_shape.py b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_tensor_shape.py index d28864aade5ce..70749c2e24447 100644 --- a/python/paddle/fluid/tests/unittests/dygraph_to_static/test_tensor_shape.py +++ b/python/paddle/fluid/tests/unittests/dygraph_to_static/test_tensor_shape.py @@ -144,11 +144,6 @@ def dyfunc_with_for_2(x): def dyfunc_with_for_3(x): - # TODO(liym27): - # It will fail to run because `for i in range(len(x.shape))` will be transformed into Paddle while_loop. - # Here the python list x.shape will be added to loop_vars. However, loop_vars doesn't support python list. - # And the condition of `for i in range(len(x.shape))` only uses the length of x.shape, so it doesn't have to be transformed into Paddle while_loop. - # After the AST tranformation of for loop is improved, add TestTensorShapeInFor3. x = fluid.dygraph.to_variable(x) res = fluid.layers.fill_constant(value=0, shape=[1], dtype="int32") # `len(x.shape)` is not transformed. @@ -282,6 +277,11 @@ class TestTensorShapeBasic2(TestTensorShapeBasic): def init_test_func(self): self.dygraph_func = dyfunc_tensor_shape_2 + def _set_expected_op_num(self): + self.expected_op_num = 3 + self.expected_shape_op_num = 1 + self.expected_slice_op_num = 0 + class TestTensorShapeBasic3(TestTensorShapeBasic): def init_test_func(self): @@ -319,6 +319,11 @@ def init_test_func(self): self.input_spec = [paddle.static.InputSpec(shape=[5, 7], dtype="int32")] self.dygraph_func = dyfunc_tuple_shape_1 + def _set_expected_op_num(self): + self.expected_op_num = 6 + self.expected_shape_op_num = 2 + self.expected_slice_op_num = 2 + class TestTupleShape2(TestTensorShapeBasic): def init_test_func(self): @@ -326,6 +331,11 @@ def init_test_func(self): self.input_spec = [paddle.static.InputSpec(shape=[5, 7], dtype="int32")] self.dygraph_func = dyfunc_tuple_shape_2 + def _set_expected_op_num(self): + self.expected_op_num = 5 + self.expected_shape_op_num = 1 + self.expected_slice_op_num = 2 + class TestPaddleShapeApi(TestTensorShapeBasic): def init_test_func(self): @@ -381,6 +391,16 @@ def _set_expected_op_num(self): self.expected_slice_op_num = 1 +class TestTensorShapeInFor3(TestTensorShapeInFor1): + def init_test_func(self): + self.dygraph_func = dyfunc_with_for_3 + + def _set_expected_op_num(self): + self.expected_op_num = 25 + self.expected_shape_op_num = 6 + self.expected_slice_op_num = 3 + + # 4. Tests with control flow while loop class TestTensorShapeInWhile1(TestTensorShapeInFor1): def init_test_func(self): @@ -402,8 +422,8 @@ def init_test_func(self): self.dygraph_func = dyfunc_with_while_3 def _set_expected_op_num(self): - self.expected_op_num = 2 - self.expected_shape_op_num = 0 + self.expected_op_num = 3 + self.expected_shape_op_num = 1 self.expected_slice_op_num = 0 @@ -474,9 +494,9 @@ def _set_test_func(self): self.dygraph_func = dyfunc_tuple_shape_1 def _set_expected_op_num(self): - self.expected_op_num = 2 - self.expected_shape_op_num = 0 - self.expected_slice_op_num = 0 + self.expected_op_num = 7 + self.expected_shape_op_num = 2 + self.expected_slice_op_num = 2 class TestOpNumWithTensorShapeInIf1(TestOpNumBasicWithTensorShape): @@ -484,7 +504,7 @@ def _set_test_func(self): self.dygraph_func = dyfunc_with_if_1 def _set_expected_op_num(self): - self.expected_op_num = 19 + self.expected_op_num = 28 self.expected_shape_op_num = 4 self.expected_slice_op_num = 2 @@ -516,9 +536,31 @@ def init_test_func(self): self.dygraph_func = dyfunc_change_shape_after_assign def _set_expected_op_num(self): - self.expected_op_num = 3 - self.expected_shape_op_num = 0 - self.expected_slice_op_num = 0 + self.expected_op_num = 7 + self.expected_shape_op_num = 2 + self.expected_slice_op_num = 2 + + +def dyfunc_with_static_convert_var_shape(x): + # Note: this will create `batch_size__static_convert_var_shape_suffix_0` firstly. + batch_size = x.shape[0] + if len(x.shape) < 1: + res = x + else: + # Test for correctly to find `batch_size__static_convert_var_shape_suffix_0` in + # deeply nested scope. + res = fluid.layers.fill_constant( + value=8, shape=[batch_size], dtype="int32") + + return res + + +class TestFindStatiConvertVarShapeSuffixVar(unittest.TestCase): + def test(self): + x_spec = paddle.static.InputSpec(shape=[None, 10]) + func = paddle.jit.to_static(dyfunc_with_if_2, input_spec=[x_spec]) + # Call this function to trigger program translation. + func.concrete_program if __name__ == '__main__': diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_conv_pass.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_conv_pass.py index 0de37fce0ae1a..0821b390e5e6a 100644 --- a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_conv_pass.py +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_conv_pass.py @@ -67,15 +67,12 @@ def set_params(self): self.conv_padding = 'VALID' -''' -# conv2d padded in 'SAME' mode is not yet supported in TRT, reopen this when support is complete. class TensorRTSubgraphPassConvSamePaddingTest(InferencePassTest): def set_params(self): self.conv_num_filters = 6 self.conv_filter_size = 6 self.conv_groups = 3 self.conv_padding = 'SAME' -''' class TensorRTSubgraphPassDepthwiseConvTest(TensorRTSubgraphPassConvTest): @@ -131,15 +128,13 @@ def set_params(self): self.conv_padding = 'VALID' -''' -# conv2d_transpose padded in 'SAME' mode is not yet supported in TRT, reopen this when support is complete. -class TensorRTSubgraphPassConvTransposeSamePaddingTest(TensorRTSubgraphPassConvTransposeTest): +class TensorRTSubgraphPassConvTransposeSamePaddingTest( + TensorRTSubgraphPassConvTransposeTest): def set_params(self): self.conv_num_filters = 6 self.conv_filter_size = 6 self.conv_groups = 1 self.conv_padding = 'SAME' -''' class TensorRTSubgraphPassDepthwiseConvTransposeTest( diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_gather_op.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_gather_op.py new file mode 100644 index 0000000000000..fec15ea7295a0 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_gather_op.py @@ -0,0 +1,70 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from inference_pass_test import InferencePassTest +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.core import PassVersionChecker +from paddle.fluid.core import AnalysisConfig + + +class TRTGatherTest(InferencePassTest): + def setUp(self): + self.set_params() + with fluid.program_guard(self.main_program, self.startup_program): + data = fluid.data(name='data', shape=[-1, 512], dtype='float32') + index = fluid.data(name='index', shape=[-1], dtype='int32') + scale_out = self.append_gather(data, index) + out = fluid.layers.batch_norm(scale_out, is_test=True) + + index = np.arange(self.num_gather, dtype='int32') + np.random.shuffle(index) + + self.feeds = { + "data": np.random.random([self.bs, 512]).astype("float32"), + "index": index, + } + + self.enable_trt = True + self.trt_parameters = TRTGatherTest.TensorRTParam( + 1 << 30, self.bs, 1, AnalysisConfig.Precision.Float32, False, False) + self.fetch_list = [out] + + def set_params(self): + self.num_gather = 16 + self.bs = 32 + + def append_gather(self, data, index): + return fluid.layers.gather(data, index=index) + + def test_check_output(self): + if core.is_compiled_with_cuda(): + use_gpu = True + self.check_output_with_option(use_gpu, flatten=True) + self.assertTrue( + PassVersionChecker.IsCompatible('tensorrt_subgraph_pass')) + + +class TRTGatherTest1(TRTGatherTest): + def set_params(self): + self.num_gather = 32 + self.bs = 32 + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_nearest_interp_op.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_nearest_interp_op.py new file mode 100644 index 0000000000000..1a58a6c9dda7d --- /dev/null +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_nearest_interp_op.py @@ -0,0 +1,192 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from inference_pass_test import InferencePassTest +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.core import PassVersionChecker +from paddle.fluid.core import AnalysisConfig + + +class TRTNearestInterpTest(InferencePassTest): + def setUp(self): + self.set_params() + + with fluid.program_guard(self.main_program, self.startup_program): + if self.data_layout == 'NCHW': + shape = [ + -1, self.channels, self.origin_shape[0], + self.origin_shape[1] + ] + else: + shape = [ + -1, self.origin_shape[0], self.origin_shape[1], + self.channels + ] + data = fluid.data(name='data', shape=shape, dtype='float32') + resize_out = self.append_nearest_interp(data) + out = fluid.layers.batch_norm(resize_out, is_test=True) + + if self.data_layout == 'NCHW': + shape = [ + self.bs, self.channels, self.origin_shape[0], + self.origin_shape[1] + ] + else: + shape = [ + self.bs, self.origin_shape[0], self.origin_shape[1], + self.channels + ] + + self.feeds = {'data': np.random.random(shape).astype('float32'), } + self.enable_trt = True + self.trt_parameters = TRTNearestInterpTest.TensorRTParam( + 1 << 30, self.bs, 1, AnalysisConfig.Precision.Float32, False, False) + self.fetch_list = [out] + + def set_params(self): + self.bs = 4 + self.scale = 1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = True + self.data_layout = 'NCHW' + + def append_nearest_interp(self, data): + if self.scale > 0.: + return fluid.layers.resize_nearest( + data, + scale=self.scale, + align_corners=self.align_corners, + data_format=self.data_layout) + return fluid.layers.resize_nearest( + data, + out_shape=self.resize_shape, + align_corners=self.align_corners, + data_format=self.data_layout) + + def test_check_output(self): + if core.is_compiled_with_cuda(): + use_gpu = True + self.check_output_with_option(use_gpu, flatten=True) + self.assertTrue( + PassVersionChecker.IsCompatible('tensorrt_subgraph_pass')) + + +class TRTNearestInterpTest1(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = True + self.data_layout = 'NCHW' + + +class TRTNearestInterpTest2(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = 2. + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = False + self.data_layout = 'NCHW' + + +class TRTNearestInterpTest3(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = False + self.data_layout = 'NCHW' + + +class TRTNearestInterpTest4(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (47, 48) # HW + self.align_corners = False + self.data_layout = 'NCHW' + + +class TRTNearestInterpTest5(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = True + self.data_layout = 'NHWC' + + +class TRTNearestInterpTest6(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = 2. + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = False + self.data_layout = 'NHWC' + + +class TRTNearestInterpTest7(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (64, 64) # HW + self.align_corners = False + self.data_layout = 'NHWC' + + +class TRTNearestInterpTest8(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (47, 48) # HW + self.align_corners = False + self.data_layout = 'NHWC' + + +class TRTNearestInterpTest9(TRTNearestInterpTest): + def set_params(self): + self.bs = 4 + self.scale = -1 + self.channels = 3 + self.origin_shape = (32, 32) # HW + self.resize_shape = (47, 48) # HW + self.align_corners = False + self.data_layout = 'NHWC' + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_scale_op.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_scale_op.py new file mode 100644 index 0000000000000..67a1253b2cd02 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_scale_op.py @@ -0,0 +1,52 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from inference_pass_test import InferencePassTest +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.core import PassVersionChecker +from paddle.fluid.core import AnalysisConfig + + +class TRTScaleTest(InferencePassTest): + def setUp(self): + with fluid.program_guard(self.main_program, self.startup_program): + data = fluid.data(name="data", shape=[-1, 512], dtype="float32") + scale_out = self.append_scale(data) + out = fluid.layers.batch_norm(scale_out, is_test=True) + + self.feeds = {"data": np.random.random([1, 512]).astype("float32"), } + self.enable_trt = True + self.trt_parameters = TRTScaleTest.TensorRTParam( + 1 << 30, 32, 1, AnalysisConfig.Precision.Float32, False, False) + self.fetch_list = [out] + + def append_scale(self, data): + return fluid.layers.scale( + x=data, scale=2.0, bias=-1.0, bias_after_scale=False) + + def test_check_output(self): + if core.is_compiled_with_cuda(): + use_gpu = True + self.check_output_with_option(use_gpu, flatten=True) + self.assertTrue( + PassVersionChecker.IsCompatible('tensorrt_subgraph_pass')) + + +if __name__ == "__main__": + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_subgraph_pass.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_subgraph_pass.py index 2c77ce1723129..bdcdeee8dcb66 100644 --- a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_subgraph_pass.py +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_subgraph_pass.py @@ -414,6 +414,58 @@ def append_eltwise(self, data1, data2): return fluid.layers.elementwise_mul(x=data1, y=data2) +class TensorRTSubgraphPassElementwiseSerializeTest( + TensorRTSubgraphPassElementwiseTest): + def setUp(self): + super(TensorRTSubgraphPassElementwiseSerializeTest, self).setUp() + self.trt_parameters = TensorRTSubgraphPassElementwiseTest.TensorRTParam( + 1 << 30, 32, 0, AnalysisConfig.Precision.Float32, True, False) + + def test_check_output(self): + if os.path.exists(self.path + "_opt_cache"): + shutil.rmtree(self.path + "_opt_cache") + super(TensorRTSubgraphPassElementwiseSerializeTest, + self).test_check_output() + + +class TensorRTSubgraphPassElementwiseBroadcastDynamicTest(InferencePassTest): + def setUp(self): + with fluid.program_guard(self.main_program, self.startup_program): + data1 = fluid.data( + name="data1", shape=[-1, 3, 64, 64], dtype="float32") + data2 = fluid.data(name="data2", shape=[64, 64], dtype="float32") + eltwise_out = self.append_eltwise(data1, data2) + out = fluid.layers.batch_norm(eltwise_out, is_test=True) + self.feeds = { + "data1": np.random.random([1, 3, 64, 64]).astype("float32"), + "data2": np.random.random([64, 64]).astype("float32"), + } + self.enable_trt = True + self.trt_parameters = TensorRTSubgraphPassElementwiseBroadcastDynamicTest.TensorRTParam( + 1 << 30, 32, 0, AnalysisConfig.Precision.Float32, True, False) + self.dynamic_shape_params = TensorRTSubgraphPassElementwiseBroadcastDynamicTest.DynamicShapeParam( + { + 'data1': [1, 3, 8, 64], + 'data2': [8, 64] + }, {'data1': [1, 3, 512, 64], + 'data2': + [512, 64]}, {'data1': [1, 3, 256, 64], + 'data2': [256, 64]}, False) + self.fetch_list = [out] + + def append_eltwise(self, data1, data2): + return fluid.layers.elementwise_add(x=data1, y=data2) + + def test_check_output(self): + if os.path.exists(self.path + "_opt_cache"): + shutil.rmtree(self.path + "_opt_cache") + if core.is_compiled_with_cuda(): + use_gpu = True + self.check_output_with_option(use_gpu) + self.assertTrue( + PassVersionChecker.IsCompatible('tensorrt_subgraph_pass')) + + class TensorRTSubgraphPassShuffleChannelTest(InferencePassTest): def setUp(self): with fluid.program_guard(self.main_program, self.startup_program): diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_bf16_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_bf16_mkldnn_op.py index ac235e00755e9..3a20ffde7a1b2 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_bf16_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_bf16_mkldnn_op.py @@ -45,13 +45,13 @@ def generate_data(self): def test_check_output(self): self.check_output_with_place(core.CPUPlace()) - # elementwise_add grad is just passing upper gradients to either X or Y or both + # elementwise_add grad (no braodcasting) is just passing upper gradients to either X or Y or both def test_check_grad_normal(self): self.check_grad_with_place( core.CPUPlace(), ["X", "Y"], "Out", check_dygraph=False, - user_defined_grads=[self.x_bf16, self.x_bf16], + user_defined_grads=[self.x, self.x], user_defined_grad_outputs=[self.x_bf16]) def test_check_grad_ingore_x(self): @@ -59,7 +59,7 @@ def test_check_grad_ingore_x(self): core.CPUPlace(), ["Y"], "Out", check_dygraph=False, - user_defined_grads=[self.y_bf16], + user_defined_grads=[self.y], user_defined_grad_outputs=[self.y_bf16]) def test_check_grad_ingore_y(self): @@ -67,7 +67,40 @@ def test_check_grad_ingore_y(self): core.CPUPlace(), ["X"], "Out", check_dygraph=False, - user_defined_grads=[self.x_bf16], + user_defined_grads=[self.x], + user_defined_grad_outputs=[self.x_bf16]) + + +class TestElementwiseAddBroadCastingBf16MklDNNOp( + TestElementwiseAddBf16MklDNNOp): + def generate_data(self): + self.x = np.random.uniform(1, 2, [2, 3, 4, 100]).astype(np.float32) + self.y = np.random.uniform(1, 2, [100]).astype(np.float32) + self.out = np.add(self.x, self.y) + + # Compute partial sums along all axes but last one + def compute_reduced_gradients(self, out_grads): + part_sum = np.add.reduceat(out_grads, [0], axis=0) + part_sum = np.add.reduceat(part_sum, [0], axis=1) + part_sum = np.add.reduceat(part_sum, [0], axis=2) + return part_sum.flatten() + + def test_check_grad_normal(self): + self.check_grad_with_place( + core.CPUPlace(), ["X", "Y"], + "Out", + check_dygraph=False, + user_defined_grads=[ + self.x, self.compute_reduced_gradients(self.x) + ], + user_defined_grad_outputs=[self.x_bf16]) + + def test_check_grad_ingore_x(self): + self.check_grad_with_place( + core.CPUPlace(), ["Y"], + "Out", + check_dygraph=False, + user_defined_grads=[self.compute_reduced_gradients(self.x)], user_defined_grad_outputs=[self.x_bf16]) diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_mkldnn_op.py index 532c6a606d1cc..28456a3e91dca 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_add_mkldnn_op.py @@ -17,6 +17,7 @@ import numpy as np from paddle.fluid.tests.unittests.op_test import skip_check_grad_ci from paddle.fluid.tests.unittests.test_elementwise_add_op import TestElementwiseAddOp +from paddle import enable_static class TestMKLDNNElementwiseAddOp(TestElementwiseAddOp): @@ -51,13 +52,17 @@ def init_input_output(self): def test_check_grad_normal(self): pass - def test_check_grad_ingore_x(self): - pass - def test_check_grad_ingore_y(self): pass +class TestMKLDNNElementwiseAddOp5(TestMKLDNNElementwiseAddOp): + def init_input_output(self): + self.x = np.random.uniform(1, 2, [2, 3, 4, 100]).astype(self.dtype) + self.y = np.random.uniform(1, 2, [100]).astype(self.dtype) + self.out = np.add(self.x, self.y) + + class TestMKLDNNElementwiseAddOp_broadcast_3(TestMKLDNNElementwiseAddOp): def init_input_output(self): self.x = np.random.rand(2, 10, 12, 3).astype(self.dtype) @@ -150,4 +155,5 @@ def init_dtype(self): if __name__ == '__main__': + enable_static() unittest.main() diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_bf16_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_bf16_mkldnn_op.py index c2716420fba37..9b7f4b9b860de 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_bf16_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_bf16_mkldnn_op.py @@ -30,10 +30,9 @@ def setUp(self): self.axis = -1 self.generate_data() - self.inputs = { - 'X': convert_float_to_uint16(self.x), - 'Y': convert_float_to_uint16(self.y) - } + self.x_bf16 = convert_float_to_uint16(self.x) + self.y_bf16 = convert_float_to_uint16(self.y) + self.inputs = {'X': self.x_bf16, 'Y': self.y_bf16} self.attrs = {'axis': self.axis, 'use_mkldnn': self.use_mkldnn} self.outputs = {'Out': convert_float_to_uint16(self.out)} @@ -46,13 +45,66 @@ def test_check_output(self): self.check_output_with_place(core.CPUPlace()) def test_check_grad_normal(self): - pass + self.check_grad_with_place( + core.CPUPlace(), ["X", "Y"], + "Out", + check_dygraph=False, + user_defined_grads=[ + np.multiply(self.x, self.y), np.multiply(self.x, self.x) + ], + user_defined_grad_outputs=[self.x_bf16]) def test_check_grad_ingore_x(self): - pass + self.check_grad_with_place( + core.CPUPlace(), ["Y"], + "Out", + check_dygraph=False, + user_defined_grads=[np.multiply(self.y, self.x)], + user_defined_grad_outputs=[self.y_bf16]) def test_check_grad_ingore_y(self): - pass + self.check_grad_with_place( + core.CPUPlace(), ["X"], + "Out", + check_dygraph=False, + user_defined_grads=[np.multiply(self.x, self.y)], + user_defined_grad_outputs=[self.x_bf16]) + + +class TestElementwiseMulBroadcastingBf16MklDNNOp( + TestElementwiseMulBf16MklDNNOp): + def generate_data(self): + self.x = np.random.uniform(1, 2, [1, 2, 3, 100]).astype(np.float32) + self.y = np.random.uniform(1, 2, [100]).astype(np.float32) + self.out = np.multiply(self.x, self.y) + + # Compute partial sums along all axes but last one + def compute_reduced_gradients(self, out_grads): + part_sum = np.add.reduceat(out_grads, [0], axis=0) + part_sum = np.add.reduceat(part_sum, [0], axis=1) + part_sum = np.add.reduceat(part_sum, [0], axis=2) + return part_sum.flatten() + + def test_check_grad_normal(self): + self.check_grad_with_place( + core.CPUPlace(), ["X", "Y"], + "Out", + check_dygraph=False, + user_defined_grads=[ + np.multiply(self.x, self.y), + self.compute_reduced_gradients(np.multiply(self.x, self.x)) + ], + user_defined_grad_outputs=[self.x_bf16]) + + def test_check_grad_ingore_x(self): + self.check_grad_with_place( + core.CPUPlace(), ["Y"], + "Out", + check_dygraph=False, + user_defined_grads=[ + self.compute_reduced_gradients(np.multiply(self.x, self.x)) + ], + user_defined_grad_outputs=[self.x_bf16]) if __name__ == '__main__': diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_mkldnn_op.py index d66f3dfb89185..03dc2421b65b0 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_elementwise_mul_mkldnn_op.py @@ -17,6 +17,7 @@ import numpy as np from paddle.fluid.tests.unittests.op_test import skip_check_grad_ci from paddle.fluid.tests.unittests.test_elementwise_mul_op import ElementwiseMulOp +from paddle import enable_static class TestMKLDNNElementwiseMulOp(ElementwiseMulOp): @@ -51,13 +52,17 @@ def init_input_output(self): def test_check_grad_normal(self): pass - def test_check_grad_ingore_x(self): - pass - def test_check_grad_ingore_y(self): pass +class TestMKLDNNElementwiseMulOp5(TestMKLDNNElementwiseMulOp): + def init_input_output(self): + self.x = np.random.uniform(1, 2, [2, 3, 4, 100]).astype(self.dtype) + self.y = np.random.uniform(1, 2, [100]).astype(self.dtype) + self.out = np.multiply(self.x, self.y) + + ''' INT8 Tests ''' @@ -140,4 +145,5 @@ def init_dtype(self): if __name__ == '__main__': + enable_static() unittest.main() diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_bf16_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_bf16_mkldnn_op.py index 90140a3474fed..c024ffbdb4b6a 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_bf16_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_bf16_mkldnn_op.py @@ -30,6 +30,11 @@ class TestFusionGRUBF16MKLDNNOp(OpTest): def set_confs(self): self.mkldnn_data_type = False + def test_check_output(self): + for use_seq in {True, False}: + self.attrs['use_seq'] = use_seq + self.check_output(check_dygraph=False) + def setUp(self): self.op_type = "fusion_gru" self.lod = [[2, 4, 3]] @@ -45,6 +50,7 @@ def setUp(self): self.origin_mode = False self.use_mkldnn = True self.force_fp32_output = False + self.weights_dtype = 'fp32' self.set_confs() T = sum(self.lod[0]) @@ -58,6 +64,9 @@ def setUp(self): wx_fp32 = np.random.rand(self.M, 3 * self.D).astype('float32') wh_fp32 = np.random.rand(self.D, 3 * self.D).astype('float32') + wx_bf16 = convert_float_to_uint16(wx_fp32) + wh_bf16 = convert_float_to_uint16(wh_fp32) + # bias is fp32 despite other inputs being in bf16 bias = np.random.rand( 1, 3 * self.D).astype('float32') if self.with_bias else np.zeros( @@ -74,20 +83,30 @@ def setUp(self): hidden_bf16 = convert_float_to_uint16(hidden) - self.inputs = { - 'X': (x_bf16, self.lod), - 'WeightX': wx_fp32, - 'WeightH': wh_fp32 - } + if self.weights_dtype == 'bf16': + self.inputs = { + 'X': (x_bf16, self.lod), + 'WeightX': wx_bf16, + 'WeightH': wh_bf16 + } + elif self.weights_dtype == 'fp32': + self.inputs = { + 'X': (x_bf16, self.lod), + 'WeightX': wx_fp32, + 'WeightH': wh_fp32 + } if self.with_bias: self.inputs['Bias'] = bias if self.with_h0: - self.inputs['H0'] = h0_bf16 + if self.weights_dtype == 'bf16': + self.inputs['H0'] = h0_bf16 + elif self.weights_dtype == 'fp32': + self.inputs['H0'] = h0_fp32 h0_bf16 = convert_float_to_uint16(h0_fp32) - self.outputs = {'Hidden': (hidden_bf16, self.lod)} + self.outputs = {'Hidden': (hidden, self.lod)} self.attrs = { 'activation': self.act_state, @@ -109,6 +128,11 @@ def set_confs(self): self.with_bias = False +class TestFusionGRUINT8MKLDNNBF16WeightsOp(TestFusionGRUBF16MKLDNNOp): + def set_confs(self): + self.weights_dtype = 'bf16' + + if __name__ == "__main__": from paddle import enable_static enable_static() diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_int8_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_int8_mkldnn_op.py index 89343c9fae459..2d3caf0be97c9 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_int8_mkldnn_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_gru_int8_mkldnn_op.py @@ -146,4 +146,6 @@ def set_confs(self): if __name__ == "__main__": + from paddle import enable_static + enable_static() unittest.main() diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_lstm_bf16_mkldnn_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_lstm_bf16_mkldnn_op.py new file mode 100644 index 0000000000000..46bdbb1a420af --- /dev/null +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_fusion_lstm_bf16_mkldnn_op.py @@ -0,0 +1,159 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +import struct +import paddle.fluid.core as core +from paddle.fluid.tests.unittests.op_test import OpTest, convert_float_to_uint16, convert_uint16_to_float +from paddle.fluid.tests.unittests.test_fusion_lstm_op import TestFusionLSTMOp, fc, ACTIVATION, fusion_lstm +from paddle.fluid.tests.unittests.test_fusion_gru_op import fusion_gru + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestFusionLSTMBF16ONEDNNOp(OpTest): + def set_confs(self): + self.mkldnn_data_type = False + + def test_check_output(self): + for use_seq in {True, False}: + self.attrs['use_seq'] = use_seq + self.check_output(check_dygraph=False, no_check_set=["Cell"]) + + def setUp(self): + self.op_type = 'fusion_lstm' + self.lod = [[2, 3, 5, 4]] + self.M = 8 + self.D = 16 + self.has_initial_state = False + self.use_peepholes = False + self.is_reverse = False + self._cpu_only = True + self.act_gate = 'sigmoid' + self.act_cell = 'tanh' + self.act_cand = 'tanh' + self.use_mkldnn = True + self.force_fp32_output = False + self.weights_dtype = 'fp32' + self.set_confs() + + T = sum(self.lod[0]) + bs = len(self.lod[0]) + + # fp32 X input for reference implementation and + # corressponding bf16 data as input to LSTM oneDNN bf16 kernel + x = np.random.normal(size=(T, self.M)).astype('float32') + + x_bf16 = convert_float_to_uint16(x) + + if self.has_initial_state: + h0 = np.random.normal(size=(bs, self.D)).astype('float32') + c0 = np.random.normal(size=(bs, self.D)).astype('float32') + else: + h0 = np.zeros((bs, self.D)).astype('float32') + c0 = np.zeros((bs, self.D)).astype('float32') + + wh = np.random.normal(size=(self.D, 4 * self.D)).astype('float32') + + h0_bf16 = convert_float_to_uint16(h0) + + if self.use_peepholes: + b = np.random.normal(size=(1, 7 * self.D)).astype('float32') + else: + b = np.random.normal(size=(1, 4 * self.D)).astype('float32') + w_b = np.copy(b[:, 0:4 * self.D]) + w_c = b[:, 4 * self.D:] if self.use_peepholes else None + + wx = np.random.normal(size=(self.M, 4 * self.D)).astype('float32') + + wx_bf16 = convert_float_to_uint16(wx) + wh_bf16 = convert_float_to_uint16(wh) + + bx = np.random.normal(size=(1, 4 * self.D)).astype('float32') + b[0, 0:4 * self.D] += bx[0, :] + + hidden, c = fusion_lstm(x, self.lod, wx, bx, h0, c0, wh, w_b, w_c, + self.is_reverse, ACTIVATION[self.act_gate], + ACTIVATION[self.act_cell], + ACTIVATION[self.act_cand]) + + hidden = hidden.astype('float32') + hidden_bf16 = convert_float_to_uint16(hidden) + + if self.weights_dtype == 'bf16': + self.inputs = { + 'X': (x_bf16, self.lod), + 'WeightX': wx_bf16, + 'WeightH': wh_bf16, + 'Bias': b + } + elif self.weights_dtype == 'fp32': + self.inputs = { + 'X': (x_bf16, self.lod), + 'WeightX': wx, + 'WeightH': wh, + 'Bias': b + } + + if self.has_initial_state: + if self.weights_dtype == 'bf16': + self.inputs['H0'] = h0_bf16 + elif self.weights_dtype == 'fp32': + self.inputs['H0'] = h0 + + self.inputs['C0'] = c0 + + self.outputs = { + 'Hidden': (hidden, self.lod), + 'Cell': (c, self.lod), + } + + self.attrs = { + 'use_peepholes': self.use_peepholes, + 'is_reverse': self.is_reverse, + 'gate_activation': self.act_gate, + 'cell_activation': self.act_cell, + 'candidate_activation': self.act_cand, + 'force_fp32_output': self.force_fp32_output, + 'use_mkldnn': self.use_mkldnn + } + + +class TestFusionLSTMBF16ONEDNNPeepholesOp(TestFusionLSTMBF16ONEDNNOp): + def set_confs(self): + self.use_peepholes = True + + +class TestFusionLSTMBF16ONEDNNInitializedStateOp(TestFusionLSTMBF16ONEDNNOp): + def set_confs(self): + self.has_initial_state = True + + +class TestFusionLSTMBF16ONEDNNReverseOp(TestFusionLSTMBF16ONEDNNOp): + def set_confs(self): + self.is_reverse = True + + +class TestFusionLSTMBF16ONEDNNBF16WeightsOp(TestFusionLSTMBF16ONEDNNOp): + def set_confs(self): + self.weights_dtype = 'bf16' + + +if __name__ == "__main__": + from paddle import enable_static + enable_static() + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/mkldnn/test_reshape_bf16_op.py b/python/paddle/fluid/tests/unittests/mkldnn/test_reshape_bf16_op.py index ac9b881313a31..ae844834154fb 100644 --- a/python/paddle/fluid/tests/unittests/mkldnn/test_reshape_bf16_op.py +++ b/python/paddle/fluid/tests/unittests/mkldnn/test_reshape_bf16_op.py @@ -50,8 +50,9 @@ def init_data(self): self.infered_shape = (10, 2, 3, -1) def init_input_data(self): - self.input_data = convert_float_to_uint16( - np.random.random(self.ori_shape).astype(np.float32)) + self.input_data_fp32 = np.random.random(self.ori_shape).astype( + np.float32) + self.input_data = convert_float_to_uint16(self.input_data_fp32) def test_check_output(self): self.check_output_with_place(core.CPUPlace(), no_check_set=['XShape']) @@ -61,7 +62,7 @@ def test_check_grad(self): core.CPUPlace(), ["X"], "Out", check_dygraph=False, - user_defined_grads=[self.inputs["X"]], + user_defined_grads=[self.input_data_fp32], user_defined_grad_outputs=[ self.inputs["X"].reshape(self.infered_shape) ]) diff --git a/python/paddle/fluid/tests/unittests/op_test.py b/python/paddle/fluid/tests/unittests/op_test.py index 8bb0779bc0499..dff96a8cbc3c4 100644 --- a/python/paddle/fluid/tests/unittests/op_test.py +++ b/python/paddle/fluid/tests/unittests/op_test.py @@ -33,10 +33,19 @@ from paddle.fluid.op import Operator from paddle.fluid.executor import Executor from paddle.fluid.framework import Program, OpProtoHolder, Variable -from testsuite import create_op, set_input, append_input_output, append_loss_ops +from paddle.fluid.tests.unittests.testsuite import ( + create_op, + set_input, + append_input_output, + append_loss_ops, ) from paddle.fluid import unique_name -from white_list import op_accuracy_white_list, check_shape_white_list, compile_vs_runtime_white_list, no_check_set_white_list -from white_list import op_threshold_white_list, no_grad_set_white_list +from paddle.fluid.tests.unittests.white_list import ( + op_accuracy_white_list, + check_shape_white_list, + compile_vs_runtime_white_list, + no_check_set_white_list, + op_threshold_white_list, + no_grad_set_white_list, ) def check_out_dtype(api_fn, in_specs, expect_dtypes, target_index=0, **configs): @@ -235,6 +244,14 @@ def convert_float_to_uint16(float_list, data_format="NCHW"): return new_output +def convert_uint16_to_float(in_list): + in_list = np.asarray(in_list) + out = np.vectorize( + lambda x: struct.unpack(' _b) diff --git a/python/paddle/fluid/tests/unittests/test_conv2d_op.py b/python/paddle/fluid/tests/unittests/test_conv2d_op.py index d2c2d2cecdda7..29c35d28d4d2e 100644 --- a/python/paddle/fluid/tests/unittests/test_conv2d_op.py +++ b/python/paddle/fluid/tests/unittests/test_conv2d_op.py @@ -17,6 +17,7 @@ import unittest import numpy as np +import paddle import paddle.fluid.core as core import paddle.fluid as fluid from op_test import OpTest @@ -127,6 +128,8 @@ def create_test_cudnn_class(parent): class TestCUDNNCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 cls_name = "{0}_{1}".format(parent.__name__, "CUDNN") TestCUDNNCase.__name__ = cls_name @@ -184,6 +187,8 @@ def create_test_cudnn_channel_last_class(parent): class TestCudnnChannelLastCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_data_format(self): self.data_format = "NHWC" @@ -263,6 +268,8 @@ def create_test_cudnn_padding_SAME_class(parent): class TestCUDNNPaddingSMAECase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_paddings(self): self.pad = [1, 1] @@ -279,6 +286,8 @@ def create_test_cudnn_padding_VALID_class(parent): class TestCUDNNPaddingVALIDCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_paddings(self): self.pad = [1, 1] @@ -691,6 +700,7 @@ class TestCUDNNExhaustiveSearch(TestConv2DOp): def init_kernel_type(self): self.use_cudnn = True self.exhaustive_search = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 class TestConv2DOpError(unittest.TestCase): @@ -1326,6 +1336,16 @@ def test_api(self): groups=1, data_format="NCHW") + def test_depthwise_conv2d(self): + x_var = paddle.uniform((2, 8, 8, 4), dtype='float32', min=-1., max=1.) + conv = paddle.nn.Conv2D( + in_channels=4, + out_channels=4, + kernel_size=(3, 3), + groups=4, + data_format='NHWC') + y_var = conv(x_var) + class TestConv2DAPI_Error(unittest.TestCase): def test_api(self): diff --git a/python/paddle/fluid/tests/unittests/test_conv2d_transpose_op.py b/python/paddle/fluid/tests/unittests/test_conv2d_transpose_op.py index fb6058c0f036b..4e582d74c24a2 100644 --- a/python/paddle/fluid/tests/unittests/test_conv2d_transpose_op.py +++ b/python/paddle/fluid/tests/unittests/test_conv2d_transpose_op.py @@ -116,7 +116,7 @@ def _get_padding_with_SAME(input_shape, kernel_size, kernel_stride): class TestConv2DTransposeOp(OpTest): def setUp(self): # init as conv transpose - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.need_check_grad = True self.is_test = False self.use_cudnn = False diff --git a/python/paddle/fluid/tests/unittests/test_conv3d_op.py b/python/paddle/fluid/tests/unittests/test_conv3d_op.py index 1636019a6252c..59d1f3216e17e 100644 --- a/python/paddle/fluid/tests/unittests/test_conv3d_op.py +++ b/python/paddle/fluid/tests/unittests/test_conv3d_op.py @@ -135,6 +135,8 @@ def create_test_cudnn_class(parent): class TestCUDNNCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 cls_name = "{0}_{1}".format(parent.__name__, "CUDNN") TestCUDNNCase.__name__ = cls_name @@ -169,6 +171,8 @@ def create_test_cudnn_padding_SAME_class(parent): class TestCUDNNPaddingSMAECase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_paddings(self): self.pad = [1, 1, 1] @@ -185,6 +189,8 @@ def create_test_cudnn_padding_VALID_class(parent): class TestCUDNNPaddingVALIDCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_paddings(self): self.pad = [1, 1, 1] @@ -215,6 +221,8 @@ def create_test_cudnn_channel_last_class(parent): class TestCudnnChannelLastCase(parent): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 def init_data_format(self): self.data_format = "NDHWC" @@ -410,6 +418,7 @@ def init_group(self): class TestCUDNN(TestConv3DOp): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -431,6 +440,7 @@ def test_check_output(self): class TestWithGroup1CUDNN(TestWithGroup1): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -452,6 +462,7 @@ def test_check_output(self): class TestWithGroup2CUDNN(TestWithGroup2): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -473,6 +484,7 @@ def test_check_output(self): class TestWith1x1CUDNN(TestWith1x1): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -494,6 +506,7 @@ def test_check_output(self): class TestWithInput1x1Filter1x1CUDNN(TestWithInput1x1Filter1x1): def init_kernel_type(self): self.use_cudnn = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -514,6 +527,7 @@ class TestCUDNNExhaustiveSearch(TestCUDNN): def init_kernel_type(self): self.use_cudnn = True self.exhaustive_search = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 # ---- test asymmetric padding ---- diff --git a/python/paddle/fluid/tests/unittests/test_deform_conv2d.py b/python/paddle/fluid/tests/unittests/test_deform_conv2d.py index dc57e87f94022..508fc1705218a 100644 --- a/python/paddle/fluid/tests/unittests/test_deform_conv2d.py +++ b/python/paddle/fluid/tests/unittests/test_deform_conv2d.py @@ -32,6 +32,7 @@ def setUp(self): self.padding = [0, 0] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = True @@ -67,11 +68,11 @@ def out_size(in_size, pad_size, dilation_size, kernel_size, self.input_shape = (self.batch_size, self.in_channels ) + self.spatial_shape - self.offset_shape = (self.batch_size, 2 * filter_shape[0] * - filter_shape[1]) + out_shape + self.offset_shape = (self.batch_size, self.deformable_groups * 2 * + filter_shape[0] * filter_shape[1]) + out_shape - self.mask_shape = (self.batch_size, filter_shape[0] * filter_shape[1] - ) + out_shape + self.mask_shape = (self.batch_size, self.deformable_groups * + filter_shape[0] * filter_shape[1]) + out_shape self.input = np.random.uniform(-1, 1, self.input_shape).astype(self.dtype) @@ -89,12 +90,12 @@ def static_graph_case_dcn(self): x = paddle.static.data( "input", (-1, self.in_channels, -1, -1), dtype=self.dtype) offset = paddle.static.data( - "offset", - (-1, 2 * self.filter_shape[0] * self.filter_shape[1], -1, -1), + "offset", (-1, self.deformable_groups * 2 * + self.filter_shape[0] * self.filter_shape[1], -1, -1), dtype=self.dtype) mask = paddle.static.data( - "mask", - (-1, self.filter_shape[0] * self.filter_shape[1], -1, -1), + "mask", (-1, self.deformable_groups * self.filter_shape[0] * + self.filter_shape[1], -1, -1), dtype=self.dtype) y_v1 = paddle.fluid.layers.deformable_conv( @@ -107,7 +108,7 @@ def static_graph_case_dcn(self): padding=self.padding, dilation=self.dilation, groups=self.groups, - deformable_groups=1, + deformable_groups=self.deformable_groups, im2col_step=1, param_attr=I.Assign(self.weight), bias_attr=False if self.no_bias else I.Assign(self.bias), @@ -123,7 +124,7 @@ def static_graph_case_dcn(self): padding=self.padding, dilation=self.dilation, groups=self.groups, - deformable_groups=1, + deformable_groups=self.deformable_groups, im2col_step=1, param_attr=I.Assign(self.weight), bias_attr=False if self.no_bias else I.Assign(self.bias)) @@ -154,6 +155,7 @@ def dygraph_case_dcn(self): stride=self.stride, padding=self.padding, dilation=self.dilation, + deformable_groups=self.deformable_groups, groups=self.groups, weight_attr=I.Assign(self.weight), bias_attr=False if self.no_bias else I.Assign(self.bias)) @@ -194,6 +196,7 @@ def setUp(self): self.padding = [0, 0] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = True @@ -229,11 +232,11 @@ def out_size(in_size, pad_size, dilation_size, kernel_size, self.input_shape = (self.batch_size, self.in_channels ) + self.spatial_shape - self.offset_shape = (self.batch_size, 2 * filter_shape[0] * - filter_shape[1]) + out_shape + self.offset_shape = (self.batch_size, self.deformable_groups * 2 * + filter_shape[0] * filter_shape[1]) + out_shape - self.mask_shape = (self.batch_size, filter_shape[0] * filter_shape[1] - ) + out_shape + self.mask_shape = (self.batch_size, self.deformable_groups * + filter_shape[0] * filter_shape[1]) + out_shape self.input = np.random.uniform(-1, 1, self.input_shape).astype(self.dtype) @@ -251,12 +254,12 @@ def static_graph_case_dcn(self): x = paddle.static.data( "input", (-1, self.in_channels, -1, -1), dtype=self.dtype) offset = paddle.static.data( - "offset", - (-1, 2 * self.filter_shape[0] * self.filter_shape[1], -1, -1), + "offset", (-1, self.deformable_groups * 2 * + self.filter_shape[0] * self.filter_shape[1], -1, -1), dtype=self.dtype) mask = paddle.static.data( - "mask", - (-1, self.filter_shape[0] * self.filter_shape[1], -1, -1), + "mask", (-1, self.deformable_groups * self.filter_shape[0] * + self.filter_shape[1], -1, -1), dtype=self.dtype) y_v1 = paddle.fluid.layers.deformable_conv( @@ -269,7 +272,7 @@ def static_graph_case_dcn(self): padding=self.padding, dilation=self.dilation, groups=self.groups, - deformable_groups=1, + deformable_groups=self.deformable_groups, im2col_step=1, param_attr=I.Assign(self.weight), bias_attr=False if self.no_bias else I.Assign(self.bias), @@ -285,7 +288,7 @@ def static_graph_case_dcn(self): padding=self.padding, dilation=self.dilation, groups=self.groups, - deformable_groups=1, + deformable_groups=self.deformable_groups, im2col_step=1, param_attr=I.Assign(self.weight), bias_attr=False if self.no_bias else I.Assign(self.bias)) @@ -317,6 +320,7 @@ def dygraph_case_dcn(self): stride=self.stride, padding=self.padding, dilation=self.dilation, + deformable_groups=self.deformable_groups, groups=self.groups, ) y_v2 = paddle.vision.ops.deform_conv2d( @@ -328,6 +332,7 @@ def dygraph_case_dcn(self): stride=self.stride, padding=self.padding, dilation=self.dilation, + deformable_groups=self.deformable_groups, groups=self.groups, ) out_v1 = y_v1.numpy() @@ -343,12 +348,12 @@ def new_api_static_graph_case_dcn(self): x = paddle.static.data( "input", (-1, self.in_channels, -1, -1), dtype=self.dtype) offset = paddle.static.data( - "offset", - (-1, 2 * self.filter_shape[0] * self.filter_shape[1], -1, -1), + "offset", (-1, self.deformable_groups * 2 * + self.filter_shape[0] * self.filter_shape[1], -1, -1), dtype=self.dtype) mask = paddle.static.data( - "mask", - (-1, self.filter_shape[0] * self.filter_shape[1], -1, -1), + "mask", (-1, self.deformable_groups * self.filter_shape[0] * + self.filter_shape[1], -1, -1), dtype=self.dtype) weight = paddle.static.data( @@ -365,6 +370,7 @@ def new_api_static_graph_case_dcn(self): stride=self.stride, padding=self.padding, dilation=self.dilation, + deformable_groups=self.deformable_groups, groups=self.groups, ) y_v2 = paddle.vision.ops.deform_conv2d( @@ -376,6 +382,7 @@ def new_api_static_graph_case_dcn(self): stride=self.stride, padding=self.padding, dilation=self.dilation, + deformable_groups=self.deformable_groups, groups=self.groups, ) exe = paddle.static.Executor(self.place) @@ -421,6 +428,7 @@ def setUp(self): self.padding = [2, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = True @@ -433,6 +441,7 @@ def setUp(self): self.padding = [2, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -445,6 +454,7 @@ def setUp(self): self.padding = [1, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -457,6 +467,7 @@ def setUp(self): self.padding = [1, 1] self.stride = [1, 1] self.dilation = [3, 3] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -469,6 +480,20 @@ def setUp(self): self.padding = [1, 1] self.stride = [2, 2] self.dilation = [1, 1] + self.deformable_groups = 1 + self.groups = 1 + self.no_bias = False + + +class TestDeformConv2DWithDeformable_Groups(TestDeformConv2D): + def setUp(self): + self.in_channels = 5 + self.out_channels = 5 + self.kernel_size = [3, 3] + self.padding = [1, 1] + self.stride = [1, 1] + self.dilation = [1, 1] + self.deformable_groups = 5 self.groups = 1 self.no_bias = False @@ -481,6 +506,7 @@ def setUp(self): self.padding = [1, 1] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 5 self.no_bias = False @@ -494,6 +520,7 @@ def setUp(self): self.padding = [2, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = True @@ -506,6 +533,7 @@ def setUp(self): self.padding = [2, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -518,6 +546,7 @@ def setUp(self): self.padding = [1, 2] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -530,6 +559,7 @@ def setUp(self): self.padding = [1, 1] self.stride = [1, 1] self.dilation = [3, 3] + self.deformable_groups = 1 self.groups = 1 self.no_bias = False @@ -542,6 +572,21 @@ def setUp(self): self.padding = [1, 1] self.stride = [2, 2] self.dilation = [1, 1] + self.deformable_groups = 1 + self.groups = 1 + self.no_bias = False + + +class TestDeformConv2DFunctionalWithDeformable_Groups( + TestDeformConv2DFunctional): + def setUp(self): + self.in_channels = 5 + self.out_channels = 5 + self.kernel_size = [3, 3] + self.padding = [1, 1] + self.stride = [1, 1] + self.dilation = [1, 1] + self.deformable_groups = 5 self.groups = 1 self.no_bias = False @@ -554,6 +599,7 @@ def setUp(self): self.padding = [1, 1] self.stride = [1, 1] self.dilation = [1, 1] + self.deformable_groups = 1 self.groups = 5 self.no_bias = False diff --git a/python/paddle/fluid/tests/unittests/test_dist_fleet_base.py b/python/paddle/fluid/tests/unittests/test_dist_fleet_base.py index 03d7251f8292f..e84e91de0ba79 100644 --- a/python/paddle/fluid/tests/unittests/test_dist_fleet_base.py +++ b/python/paddle/fluid/tests/unittests/test_dist_fleet_base.py @@ -18,6 +18,7 @@ import paddle.distributed.fleet as fleet import paddle.distributed.fleet.base.role_maker as role_maker import paddle.fluid as fluid +import paddle """ high level unit test for distribute fleet. """ @@ -112,23 +113,21 @@ def build_strategy(self, args): def build_optimizer(self, avg_cost, strategy): use_grad_clip = int(os.getenv('GRAD_CLIP', 0)) + grad_clip = None if use_grad_clip: # 1: clip_by_value; 2: clip_by_norm; 3:clip_by_global_norm if use_grad_clip == 1: - fluid.clip.set_gradient_clip( - clip=fluid.clip.GradientClipByValue(2.0)) + grad_clip = paddle.nn.ClipGradByValue(min=-5.0, max=5.0) elif use_grad_clip == 2: - fluid.clip.set_gradient_clip( - clip=fluid.clip.GradientClipByNorm(2.0)) + grad_clip = paddle.nn.ClipGradByNorm(2.0) elif use_grad_clip == 3: - fluid.clip.set_gradient_clip( - clip=fluid.clip.GradientClipByGlobalNorm(2.0)) + grad_clip = paddle.nn.ClipGradByGlobalNorm(2.0) use_decay = int(os.getenv("USE_DECAY", "0")) if use_decay: scheduler = paddle.optimizer.lr.ExponentialDecay( learning_rate=LEARNING_RATE, gamma=0.999, verbose=True) - optimizer = fluid.optimizer.SGD(scheduler) + optimizer = fluid.optimizer.SGD(scheduler, grad_clip=grad_clip) """ # learning rate decay method before 2.0 optimizer = fluid.optimizer.SGD( @@ -139,7 +138,7 @@ def build_optimizer(self, avg_cost, strategy): staircase=True)) """ else: - optimizer = fluid.optimizer.SGD(LEARNING_RATE) + optimizer = fluid.optimizer.SGD(LEARNING_RATE, grad_clip=grad_clip) optimizer = fleet.distributed_optimizer(optimizer, strategy=strategy) optimizer.minimize(avg_cost) diff --git a/python/paddle/fluid/tests/unittests/test_dist_fleet_grad_clip.py b/python/paddle/fluid/tests/unittests/test_dist_fleet_grad_clip.py index 3c68af474cf7c..f9509d60072f8 100644 --- a/python/paddle/fluid/tests/unittests/test_dist_fleet_grad_clip.py +++ b/python/paddle/fluid/tests/unittests/test_dist_fleet_grad_clip.py @@ -16,53 +16,66 @@ import os import unittest -import paddle.fluid as fluid -import paddle.fluid.incubate.fleet.base.role_maker as role_maker -from paddle.fluid.incubate.fleet.parameter_server.distribute_transpiler import fleet -from paddle.fluid.transpiler.distribute_transpiler import DistributeTranspilerConfig from test_dist_fleet_base import TestFleetBase -from dist_fleet_simnet_bow import train_network -@unittest.skip(reason="Skip unstable ut, add it after PR 22957 merged") -class TestDistGeoClipByGlobalNormTranspiler(unittest.TestCase): - def test_pserver(self): - role = role_maker.UserDefinedRoleMaker( - current_id=0, - role=role_maker.Role.SERVER, - worker_num=2, - server_endpoints=["127.0.0.1:36011", "127.0.0.1:36012"]) +class TestDistGeoClipByGlobalNorm(TestFleetBase): + def _setup_config(self): + self._mode = "geo" + self._reader = "dataset" + self._geo_sgd_need_push_nums = 5 + self._grad_clip_mode = 3 - fleet.init(role) + def check_with_place(self, + model_file, + delta=1e-3, + check_error_log=False, + need_envs={}): + required_envs = { + "PATH": os.getenv("PATH", ""), + "PYTHONPATH": os.getenv("PYTHONPATH", ""), + "LD_LIBRARY_PATH": os.getenv("LD_LIBRARY_PATH", ""), + "FLAGS_rpc_deadline": "5000", # 5sec to fail fast + "http_proxy": "" + } + required_envs.update(need_envs) - batch_size = 128 - is_sparse = True - is_distribute = False + tr0_losses, tr1_losses = self._run_cluster(model_file, required_envs) - strategy = DistributeTranspilerConfig() - strategy.sync_mode = False - strategy.geo_sgd_mode = True - strategy.geo_sgd_need_push_nums = 5 + def test_dist_train(self): + self.check_with_place( + "dist_fleet_ctr.py", delta=1e-5, check_error_log=True) - avg_cost, _, _, _ = train_network(batch_size, is_distribute, is_sparse) - fluid.clip.set_gradient_clip( - clip=fluid.clip.GradientClipByGlobalNorm(2.0)) + def _setup_config(self): + self._sync_mode = False + self._grad_clip_mode = 2 - optimizer = fluid.optimizer.SGD(0.1) - optimizer = fleet.distributed_optimizer(optimizer, strategy) - optimizer.minimize(avg_cost) + def check_with_place(self, + model_file, + delta=1e-3, + check_error_log=False, + need_envs={}): + required_envs = { + "PATH": os.getenv("PATH", ""), + "PYTHONPATH": os.getenv("PYTHONPATH", ""), + "LD_LIBRARY_PATH": os.getenv("LD_LIBRARY_PATH", ""), + "FLAGS_rpc_deadline": "5000", # 5sec to fail fast + "http_proxy": "" + } + required_envs.update(need_envs) + + tr0_losses, tr1_losses = self._run_cluster(model_file, required_envs) - pserver_startup_program = fleet.startup_program - pserver_mian_program = fleet.main_program + def test_dist_train(self): + self.check_with_place( + "dist_fleet_ctr.py", delta=1e-5, check_error_log=True) -@unittest.skip(reason="Skip unstable ut, add it after PR 22957 merged") -class TestDistGeoClipByGlobalNorm(TestFleetBase): +class TestDistASyncClipByValue(TestFleetBase): def _setup_config(self): - self._mode = "geo" + self._mode = "async" self._reader = "dataset" - self._geo_sgd_need_push_nums = 5 - self._grad_clip_mode = 3 + self._grad_clip_mode = 1 def check_with_place(self, model_file, @@ -84,8 +97,11 @@ def test_dist_train(self): self.check_with_place( "dist_fleet_ctr.py", delta=1e-5, check_error_log=True) + +class TestDistASyncClipByNorm(TestFleetBase): def _setup_config(self): - self._sync_mode = False + self._mode = "async" + self._reader = "dataset" self._grad_clip_mode = 2 def check_with_place(self, @@ -109,7 +125,6 @@ def test_dist_train(self): "dist_fleet_ctr.py", delta=1e-5, check_error_log=True) -@unittest.skip(reason="Skip unstable ut, add it after PR 22957 merged") class TestDistASyncClipByGlobalNorm(TestFleetBase): def _setup_config(self): self._mode = "async" diff --git a/python/paddle/fluid/tests/unittests/test_dist_op.py b/python/paddle/fluid/tests/unittests/test_dist_op.py index 0f71027d27401..b9b8ea92cb3a8 100644 --- a/python/paddle/fluid/tests/unittests/test_dist_op.py +++ b/python/paddle/fluid/tests/unittests/test_dist_op.py @@ -39,9 +39,10 @@ def setUp(self): self.op_type = 'dist' self.attrs = {} self.init_case() + self.init_data_type() self.inputs = { - "X": np.random.random(self.x_shape).astype("float64"), - "Y": np.random.random(self.y_shape).astype("float64") + "X": np.random.random(self.x_shape).astype(self.data_type), + "Y": np.random.random(self.y_shape).astype(self.data_type) } self.attrs["p"] = self.p @@ -55,6 +56,10 @@ def init_case(self): self.y_shape = (120) self.p = 0. + def init_data_type(self): + self.data_type = np.float32 if core.is_compiled_with_rocm( + ) else np.float64 + def calc_gradient(self): x = self.inputs["X"] y = self.inputs["Y"] @@ -143,15 +148,20 @@ def init_case(self): class TestDistAPI(unittest.TestCase): + def init_data_type(self): + self.data_type = 'float32' if core.is_compiled_with_rocm( + ) else 'float64' + def test_api(self): + self.init_data_type() main_program = fluid.Program() startup_program = fluid.Program() with fluid.program_guard(main_program, startup_program): - x = fluid.data(name='x', shape=[2, 3, 4, 5], dtype='float64') - y = fluid.data(name='y', shape=[3, 1, 5], dtype='float64') + x = fluid.data(name='x', shape=[2, 3, 4, 5], dtype=self.data_type) + y = fluid.data(name='y', shape=[3, 1, 5], dtype=self.data_type) p = 2 - x_i = np.random.random((2, 3, 4, 5)).astype("float64") - y_i = np.random.random((3, 1, 5)).astype("float64") + x_i = np.random.random((2, 3, 4, 5)).astype(self.data_type) + y_i = np.random.random((3, 1, 5)).astype(self.data_type) result = paddle.dist(x, y, p) place = fluid.CUDAPlace(0) if core.is_compiled_with_cuda( ) else fluid.CPUPlace() diff --git a/python/paddle/fluid/tests/unittests/test_fleet_distributed_strategy.py b/python/paddle/fluid/tests/unittests/test_fleet_distributed_strategy.py index 7375049b3c864..31771ddbd6874 100644 --- a/python/paddle/fluid/tests/unittests/test_fleet_distributed_strategy.py +++ b/python/paddle/fluid/tests/unittests/test_fleet_distributed_strategy.py @@ -66,9 +66,12 @@ def test_pipeline(self): def test_pipeline_configs(self): strategy = paddle.distributed.fleet.DistributedStrategy() - configs = {"micro_batch": 4} + configs = {"micro_batch_size": 4} strategy.pipeline_configs = configs - self.assertEqual(strategy.pipeline_configs["micro_batch"], 4) + self.assertEqual(strategy.pipeline_configs["micro_batch_size"], 4) + configs = {"accumulate_steps": 2} + strategy.pipeline_configs = configs + self.assertEqual(strategy.pipeline_configs["accumulate_steps"], 2) def test_localsgd(self): strategy = paddle.distributed.fleet.DistributedStrategy() diff --git a/python/paddle/fluid/tests/unittests/test_fleet_launch_ps.sh b/python/paddle/fluid/tests/unittests/test_fleet_launch_ps.sh index 21875851bf530..0f28be614c085 100644 --- a/python/paddle/fluid/tests/unittests/test_fleet_launch_ps.sh +++ b/python/paddle/fluid/tests/unittests/test_fleet_launch_ps.sh @@ -16,27 +16,25 @@ set -e -function test_launch_ps(){ - server_port_0=${PADDLE_DIST_UT_PORT} - server_port_1=$(( PADDLE_DIST_UT_PORT + 1 )) - echo "server_port_0:${server_port_0} server_port_1=${server_port_1}" - python -m paddle.distributed.fleet.launch --server_num=2 --worker_num=2 fleet_ps_training.py 2> ut.elog - if grep -q "server are killed" ut.elog; then - echo "test pserver launch succeed" - else - echo "test pserver launch failed" - exit -1 - fi +server_port_00=${PADDLE_DIST_UT_PORT} +server_port_10=$(( PADDLE_DIST_UT_PORT + 1 )) +worker_port_00=$(( PADDLE_DIST_UT_PORT + 2 )) +worker_port_10=$(( PADDLE_DIST_UT_PORT + 3 )) - python -m paddle.distributed.fleet.launch --servers="127.0.0.1:${server_port_0},127.0.0.1:${server_port_1}" --workers="127.0.0.1:6782,127.0.0.1:6783" fleet_ps_training.py 2> ut.elog - if grep -q "server are killed" ut.elog; then - echo "test pserver launch succeed" - else - echo "test pserver launch failed" - exit -1 - fi +server_port_01=$(( PADDLE_DIST_UT_PORT + 4 )) +server_port_11=$(( PADDLE_DIST_UT_PORT + 5 )) +worker_port_01=$(( PADDLE_DIST_UT_PORT + 6 )) +worker_port_11=$(( PADDLE_DIST_UT_PORT + 7 )) + +heter_worker_port_0=$(( PADDLE_DIST_UT_PORT + 8 )) +heter_worker_port_1=$(( PADDLE_DIST_UT_PORT + 9 )) + +function test_launch_ps(){ - python -m paddle.distributed.fleet.launch --servers="127.0.0.1:${server_port_0},127.0.0.1:${server_port_1}" --workers="127.0.0.1,127.0.0.1" fleet_ps_training.py 2> ut.elog + python -m paddle.distributed.fleet.launch \ + --servers="127.0.0.1:${server_port_00},127.0.0.1:${server_port_10}" \ + --workers="127.0.0.1:${worker_port_00},127.0.0.1:${worker_port_10}" \ + fleet_ps_training.py 2> ut.elog if grep -q "server are killed" ut.elog; then echo "test pserver launch succeed" else @@ -46,7 +44,11 @@ function test_launch_ps(){ } function test_launch_ps_heter(){ - python -m paddle.distributed.fleet.launch --server_num=2 --worker_num=2 --heter_worker_num=2 fleet_ps_training.py 2> ut.elog + python -m paddle.distributed.fleet.launch \ + --servers="127.0.0.1:${server_port_01},127.0.0.1:${server_port_11}" \ + --workers="127.0.0.1:${worker_port_01},127.0.0.1:${worker_port_11}" \ + --heter_workers="127.0.0.1:${heter_worker_port_0},127.0.0.1:${heter_worker_port_1}" \ + fleet_ps_training.py 2> ut.elog if grep -q "server are killed" ut.elog; then echo "test heter pserver launch succeed" else diff --git a/python/paddle/fluid/tests/unittests/test_fleet_pipeline_meta_optimizer.py b/python/paddle/fluid/tests/unittests/test_fleet_pipeline_meta_optimizer.py index 68702562dde4a..a9c37d78537ee 100644 --- a/python/paddle/fluid/tests/unittests/test_fleet_pipeline_meta_optimizer.py +++ b/python/paddle/fluid/tests/unittests/test_fleet_pipeline_meta_optimizer.py @@ -48,7 +48,10 @@ def test_pipeline_optimizer(self): strategy = paddle.distributed.fleet.DistributedStrategy() strategy.pipeline = True - strategy.pipeline_configs = {'micro_batch': 2} + strategy.pipeline_configs = { + 'micro_batch_size': 1, + 'accumulate_steps': 2 + } optimizer = paddle.fluid.optimizer.Adam(0.01) optimizer = fleet.distributed_optimizer(optimizer, strategy=strategy) diff --git a/python/paddle/fluid/tests/unittests/test_functional_conv2d.py b/python/paddle/fluid/tests/unittests/test_functional_conv2d.py index 68be0bf5d561e..766e1bb1d34af 100644 --- a/python/paddle/fluid/tests/unittests/test_functional_conv2d.py +++ b/python/paddle/fluid/tests/unittests/test_functional_conv2d.py @@ -442,5 +442,20 @@ def setUp(self): self.data_format = "NHWC" +class TestFunctionalConv2DErrorCase11(TestFunctionalConv2DError): + def setUp(self): + self.in_channels = 3 + self.out_channels = 5 + self.filter_shape = 3 + self.padding = 0 + self.stride = 1 + self.dilation = 1 + self.groups = 1 + self.no_bias = False + self.act = "sigmoid" + self.use_cudnn = False + self.data_format = "NHCW" + + if __name__ == "__main__": unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_fuse_all_reduce_pass.py b/python/paddle/fluid/tests/unittests/test_fuse_all_reduce_pass.py index 881b9d905799f..e3a2566133742 100644 --- a/python/paddle/fluid/tests/unittests/test_fuse_all_reduce_pass.py +++ b/python/paddle/fluid/tests/unittests/test_fuse_all_reduce_pass.py @@ -22,6 +22,8 @@ import unittest import os +paddle.enable_static() + class TestFuseAllReduceOpsBase(TestParallelExecutorBase): @classmethod @@ -37,6 +39,8 @@ def compare_fuse_all_reduce_ops(self, fuse_all_optimizer_ops=False): if use_device == DeviceType.CUDA and not core.is_compiled_with_cuda(): return + if use_device == DeviceType.XPU and not core.is_compiled_with_xpu(): + return feed_dict_data = None if init_feed_dict is not None: @@ -83,11 +87,15 @@ def _decorate_compare_fused_all_reduce(self, model, use_device): def test_simple_fc_with_fuse_all_reduce(self): self._decorate_compare_fused_all_reduce(simple_fc_net, DeviceType.CUDA) + self._decorate_compare_fused_all_reduce(simple_fc_net, DeviceType.XPU) self._decorate_compare_fused_all_reduce(simple_fc_net, DeviceType.CPU) def test_batchnorm_fc_with_fuse_all_reduce(self): self._decorate_compare_fused_all_reduce(fc_with_batchnorm, DeviceType.CUDA) + # TODO(wangxi): xpu batch_norm op only support dim = 4 + # self._decorate_compare_fused_all_reduce(fc_with_batchnorm, + # DeviceType.XPU) self._decorate_compare_fused_all_reduce(fc_with_batchnorm, DeviceType.CPU) @@ -127,6 +135,8 @@ def _decorate_compare_fused_all_reduce(self, model, use_device): def test_simple_bow_net_with_fuse_all_reduce(self): model = partial(bow_net, dict_dim=self.word_dict_len, is_sparse=True) self._decorate_compare_fused_all_reduce(model, DeviceType.CUDA) + # TODO(wangxi): xpu sum op only support LodTensor for now + # self._decorate_compare_fused_all_reduce(model, DeviceType.XPU) self._decorate_compare_fused_all_reduce(model, DeviceType.CPU) diff --git a/python/paddle/fluid/tests/unittests/test_gast_with_compatibility.py b/python/paddle/fluid/tests/unittests/test_gast_with_compatibility.py index c176ff09e024d..17ba6869534fe 100644 --- a/python/paddle/fluid/tests/unittests/test_gast_with_compatibility.py +++ b/python/paddle/fluid/tests/unittests/test_gast_with_compatibility.py @@ -97,7 +97,6 @@ def visit_Subscript(self, node): It will be generally represented by gast.Index or gast.Slice in gast. Note: Paddle doesn't support PY3.8 currently. """ - assert isinstance(node.slice, (gast.Index, gast.Slice)) self.generic_visit(node) return node diff --git a/python/paddle/fluid/tests/unittests/test_imperative_ocr_attention_model.py b/python/paddle/fluid/tests/unittests/test_imperative_ocr_attention_model.py index f256e97e83795..973c559857974 100644 --- a/python/paddle/fluid/tests/unittests/test_imperative_ocr_attention_model.py +++ b/python/paddle/fluid/tests/unittests/test_imperative_ocr_attention_model.py @@ -29,19 +29,19 @@ class Config(object): config for training ''' # encoder rnn hidden_size - encoder_size = 16 + encoder_size = 8 # decoder size for decoder stage - decoder_size = 16 + decoder_size = 8 # size for word embedding - word_vector_dim = 16 + word_vector_dim = 8 # max length for label padding - max_length = 5 + max_length = 3 # optimizer setting LR = 1.0 learning_rate_decay = None # batch size to train - batch_size = 8 + batch_size = 2 # class number to classify num_classes = 64 @@ -55,7 +55,7 @@ class Config(object): TRAIN_LIST_FILE_NAME = "train.list" # data shape for input image - DATA_SHAPE = [1, 48, 384] + DATA_SHAPE = [1, 16, 64] class ConvBNPool(fluid.dygraph.Layer): @@ -124,13 +124,13 @@ class OCRConv(fluid.dygraph.Layer): def __init__(self, is_test=False, use_cudnn=True): super(OCRConv, self).__init__() self.conv_bn_pool_1 = ConvBNPool( - 2, [16, 16], [1, 16], is_test=is_test, use_cudnn=use_cudnn) + 2, [8, 8], [1, 8], is_test=is_test, use_cudnn=use_cudnn) self.conv_bn_pool_2 = ConvBNPool( - 2, [32, 32], [16, 32], is_test=is_test, use_cudnn=use_cudnn) + 2, [8, 8], [8, 8], is_test=is_test, use_cudnn=use_cudnn) self.conv_bn_pool_3 = ConvBNPool( - 2, [64, 64], [32, 64], is_test=is_test, use_cudnn=use_cudnn) + 2, [8, 8], [8, 8], is_test=is_test, use_cudnn=use_cudnn) self.conv_bn_pool_4 = ConvBNPool( - 2, [128, 128], [64, 128], + 2, [16, 16], [8, 16], is_test=is_test, pool=False, use_cudnn=use_cudnn) @@ -212,9 +212,9 @@ def __init__(self, self.ocr_convs = OCRConv(is_test=is_test, use_cudnn=use_cudnn) self.fc_1_layer = Linear( - 768, rnn_hidden_size * 3, param_attr=para_attr, bias_attr=False) + 32, rnn_hidden_size * 3, param_attr=para_attr, bias_attr=False) self.fc_2_layer = Linear( - 768, rnn_hidden_size * 3, param_attr=para_attr, bias_attr=False) + 32, rnn_hidden_size * 3, param_attr=para_attr, bias_attr=False) self.gru_forward_layer = DynamicGRU( size=rnn_hidden_size, h_0=h_0, @@ -241,10 +241,9 @@ def forward(self, inputs): transpose_conv_features = fluid.layers.transpose( conv_features, perm=[0, 3, 1, 2]) - sliced_feature = fluid.layers.reshape( transpose_conv_features, [ - -1, 48, transpose_conv_features.shape[2] * + -1, 8, transpose_conv_features.shape[2] * transpose_conv_features.shape[3] ], inplace=False) @@ -376,9 +375,9 @@ def test_while_op(self): seed = 90 epoch_num = 1 if core.is_compiled_with_cuda(): - batch_num = 6 + batch_num = 3 else: - batch_num = 4 + batch_num = 2 np.random.seed = seed image_np = np.random.randn(Config.batch_size, Config.DATA_SHAPE[0], Config.DATA_SHAPE[1], @@ -536,8 +535,9 @@ def test_while_op(self): self.assertTrue(np.array_equal(value, dy_param_init_value[key])) for key, value in six.iteritems(static_param_value): - self.assertTrue(np.allclose(value, dy_param_value[key])) + self.assertTrue(np.allclose(value, dy_param_value[key], rtol=1e-05)) if __name__ == '__main__': + paddle.enable_static() unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_jit_save_load.py b/python/paddle/fluid/tests/unittests/test_jit_save_load.py index a43918765d44f..bf9912c89cb87 100644 --- a/python/paddle/fluid/tests/unittests/test_jit_save_load.py +++ b/python/paddle/fluid/tests/unittests/test_jit_save_load.py @@ -222,6 +222,16 @@ def forward(self, img, label): return out +class LinearNetWithDictInputNoPrune(paddle.nn.Layer): + def __init__(self, in_size, out_size): + super(LinearNetWithDictInputNoPrune, self).__init__() + self._linear = Linear(in_size, out_size) + + def forward(self, img): + out = self._linear(img['img'] + img['img2']) + return out + + class EmptyLayer(paddle.nn.Layer): def __init__(self): super(EmptyLayer, self).__init__() @@ -443,6 +453,30 @@ def test_dict_input(self): self.assertEqual(len(loaded_net._input_spec()), 1) +class TestSaveLoadWithDictInputNoPrune(unittest.TestCase): + def test_dict_input(self): + net = LinearNetWithDictInputNoPrune(8, 8) + + path = "test_jit_save_load_with_dict_input_no_prune/model" + # prune inputs + paddle.jit.save( + layer=net, + path=path, + input_spec=[{ + 'img': InputSpec( + shape=[None, 8], dtype='float32', name='img'), + 'img2': InputSpec( + shape=[None, 8], dtype='float32', name='img2') + }]) + + img = paddle.randn(shape=[4, 8], dtype='float32') + img2 = paddle.randn(shape=[4, 8], dtype='float32') + loaded_net = paddle.jit.load(path) + loaded_out = loaded_net(img, img2) + + self.assertEqual(len(loaded_net._input_spec()), 2) + + class TestSaveLoadWithInputSpec(unittest.TestCase): def setUp(self): # enable dygraph mode diff --git a/python/paddle/fluid/tests/unittests/test_lookup_table_bf16_op.py b/python/paddle/fluid/tests/unittests/test_lookup_table_bf16_op.py new file mode 100644 index 0000000000000..13c4aa6d767a6 --- /dev/null +++ b/python/paddle/fluid/tests/unittests/test_lookup_table_bf16_op.py @@ -0,0 +1,176 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import numpy as np +from paddle.fluid.tests.unittests.op_test import ( + OpTest, convert_float_to_uint16, convert_uint16_to_float, + skip_check_grad_ci) +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.op import Operator +from paddle import enable_static + + +def _lookup(weights, ids, flat_ids): + w_shape = weights.shape + out_shape = list(ids.shape[:-1]) + out_shape.append(w_shape[-1]) + out = weights[flat_ids].reshape(out_shape) + return out + + +def _get_grad(weights, ids, flat_ids): + w_shape = weights.shape + w_grad = np.zeros((w_shape), dtype=weights.dtype) + out_grad_shape = (np.prod(ids.shape[:-1]), w_shape[-1]) + out_grad = weights[flat_ids].reshape(out_grad_shape) + for i, idx in enumerate(flat_ids): + w_grad[idx, :] += out_grad[i] + return w_grad + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16Op(OpTest): + def setUp(self): + self.op_type = "lookup_table" + self.dtype = np.uint16 + + table = np.random.random((17, 31)).astype("float32") + self.ids = np.random.randint(0, 17, (4, 1)).astype("int64") + self.flat_ids = self.ids.flatten() + + self.w_bf16 = convert_float_to_uint16(table) + self.out_bf16 = _lookup(self.w_bf16, self.ids, self.flat_ids) + self.out_fp32 = _lookup(table, self.ids, self.flat_ids) + self.w_grad_fp32 = _get_grad(table, self.ids, self.flat_ids) + + self.inputs = {'W': self.w_bf16, 'Ids': self.ids} + self.outputs = {'Out': self.out_fp32} + + def test_check_output(self): + self.check_output_with_place(core.CPUPlace(), check_dygraph=False) + + def test_check_grad(self): + self.check_grad_with_place( + core.CPUPlace(), ['W'], + 'Out', + no_grad_set=set('Ids'), + check_dygraph=False, + max_relative_error=1.5e-2, + user_defined_grads=[self.w_grad_fp32], + user_defined_grad_outputs=[self.out_bf16]) + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16OpIds4D(TestLookupTableBF16Op): + def setUp(self): + super(TestLookupTableBF16OpIds4D, self).setUp() + self.ids = np.random.randint(0, 17, (2, 4, 5, 1)).astype("int64") + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16OpWIsSelectedRows(unittest.TestCase): + def setUp(self): + self.ids = np.random.randint( + low=0, high=15, size=(10, 1)).astype("int64") + self.flat_ids = self.ids.flatten() + self.w_fp32 = np.random.random((15, 32)).astype("float32") + self.w_bf16 = convert_float_to_uint16(self.w_fp32) + self.scope = core.Scope() + self.place = core.CPUPlace() + + def prepare_w(self): + rows = [a for a in range(self.w_bf16.shape[0])] + row_numel = self.w_bf16.shape[1] + + w_selected_rows = self.scope.var('W').get_selected_rows() + w_selected_rows.set_height(len(rows)) + w_selected_rows.set_rows(rows) + w_tensor = w_selected_rows.get_tensor() + w_tensor.set(self.w_bf16, self.place) + + def prepare_ids(self): + ids_tensor = self.scope.var('Ids').get_tensor() + ids_tensor.set(self.ids, self.place) + + def _check_output(self, reference, result_array): + result_array_fp32 = convert_uint16_to_float(result_array) + np.testing.assert_allclose(result_array_fp32, reference, rtol=1.5e-2) + + def test_check_output(self): + self.prepare_ids() + self.prepare_w() + out_tensor = self.scope.var('Out').get_tensor() + + # create and run lookup_table operator + lookup_table = Operator("lookup_table", W='W', Ids='Ids', Out='Out') + lookup_table.run(self.scope, self.place) + + # get result from Out + result_array = np.array(out_tensor) + ref = _lookup(self.w_fp32, self.ids, self.flat_ids) + self._check_output(ref, result_array) + + +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16OpWIsSelectedRows4DIds( + TestLookupTableBF16OpWIsSelectedRows): + def setUp(self): + super(TestLookupTableBF16OpWIsSelectedRows4DIds, self).setUp() + self.ids = np.random.randint( + low=0, high=15, size=(3, 4, 5, 1)).astype("int64") + self.flat_ids = self.ids.flatten() + + +@skip_check_grad_ci( + reason="Since paddings are not trainable and fixed in forward," + "the gradient of paddings makes no sense and we don't " + "test the gradient here.") +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16OpWithPadding(TestLookupTableBF16Op): + def test_check_output(self): + ids = np.squeeze(self.inputs['Ids']) + padding_idx = np.random.choice(ids, 1)[0] + self.outputs['Out'][ids == padding_idx] = np.zeros(31) + self.attrs = {'padding_idx': int(padding_idx)} + self.check_output_with_place(core.CPUPlace(), check_dygraph=False) + + +@skip_check_grad_ci( + reason="Since paddings are not trainable and fixed in forward," + "the gradient of paddings makes no sense and we don't " + "test the gradient here.") +@unittest.skipIf(not core.supports_bfloat16(), + "place does not support BF16 evaluation") +class TestLookupTableBF16OpIds4DPadding(TestLookupTableBF16OpIds4D): + def test_check_output(self): + ids = self.inputs['Ids'] + flatten_idx = ids.flatten() + padding_idx = np.random.choice(flatten_idx, 1)[0] + self.outputs['Out'][np.squeeze(ids == padding_idx)] = np.zeros(31) + self.attrs = {'padding_idx': int(padding_idx)} + self.check_output_with_place(core.CPUPlace(), check_dygraph=False) + + +if __name__ == "__main__": + enable_static() + unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_math_op_patch_var_base.py b/python/paddle/fluid/tests/unittests/test_math_op_patch_var_base.py index e908f1a60a002..4b097f6359f88 100644 --- a/python/paddle/fluid/tests/unittests/test_math_op_patch_var_base.py +++ b/python/paddle/fluid/tests/unittests/test_math_op_patch_var_base.py @@ -354,8 +354,11 @@ def test_tensor_patch_method(self): [1.30058, 1.0688717, 1.4928783], [1.0958099, 1.3724753, 1.8926544]]) d = d.matmul(d.t()) - self.assertTrue( - np.array_equal(d.cholesky().numpy(), paddle.cholesky(d).numpy())) + # ROCM not support cholesky + if not fluid.core.is_compiled_with_rocm(): + self.assertTrue( + np.array_equal(d.cholesky().numpy(), paddle.cholesky(d).numpy( + ))) self.assertTrue( np.array_equal(x.is_empty().numpy(), paddle.is_empty(x).numpy())) diff --git a/python/paddle/fluid/tests/unittests/test_matmul_v2_op.py b/python/paddle/fluid/tests/unittests/test_matmul_v2_op.py index 761d318d7b8a3..efcc0e4cfe323 100644 --- a/python/paddle/fluid/tests/unittests/test_matmul_v2_op.py +++ b/python/paddle/fluid/tests/unittests/test_matmul_v2_op.py @@ -67,7 +67,7 @@ def config(self): self.trans_y = False def init_kernel_type(self): - self.dtype = "float64" + self.dtype = "float32" if core.is_compiled_with_rocm() else "float64" def setUp(self): self.init_kernel_type() @@ -91,7 +91,10 @@ def test_check_output(self): self.check_output() def test_check_grad(self): - self.check_grad(['X', 'Y'], 'Out') + if core.is_compiled_with_rocm(): + self.check_grad(['X', 'Y'], 'Out', max_relative_error=1e-2) + else: + self.check_grad(['X', 'Y'], 'Out') class TestMatMuklOp2(TestMatMulV2Op): diff --git a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_dataset.py b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_dataset.py index 39fc965e5ede3..977882543a888 100755 --- a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_dataset.py +++ b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_dataset.py @@ -273,5 +273,62 @@ def run_main(self, num_workers, places): assert isinstance(label, paddle.Tensor) +class ComplextDataset(Dataset): + def __init__(self, sample_num): + self.sample_num = sample_num + + def __len__(self): + return self.sample_num + + def __getitem__(self, idx): + return (3.1, 'abc', paddle.to_tensor( + np.random.random([IMAGE_SIZE]).astype('float32'), + place=paddle.CPUPlace()), + [1, np.random.random([2]).astype('float32')], { + 'a': 2.0, + 'b': np.random.random([2]).astype('float32') + }) + + +class TestComplextDataset(unittest.TestCase): + def run_main(self, num_workers): + paddle.static.default_startup_program().random_seed = 1 + paddle.static.default_main_program().random_seed = 1 + place = paddle.CPUPlace() + with fluid.dygraph.guard(place): + dataset = ComplextDataset(16) + assert len(dataset) == 16 + dataloader = DataLoader( + dataset, + places=place, + num_workers=num_workers, + batch_size=2, + drop_last=True) + + for i, data in enumerate(dataloader()): + assert len(data) == 5 + # data[0]: collate 3.1 + assert data[0].shape == [2] + assert isinstance(data[1], list) + # data[1]: collate 'abc' + assert len(data[1]) == 2 + assert isinstance(data[1][0], str) + assert isinstance(data[1][1], str) + # data[2]: collate tensor + assert data[2].shape == [2, IMAGE_SIZE] + # data[3]: collate list + assert isinstance(data[3], list) + assert data[3][0].shape == [2] + assert data[3][1].shape == [2, 2] + # data[4]: collate dict + assert isinstance(data[4], dict) + assert data[4]['a'].shape == [2] + assert data[4]['b'].shape == [2, 2] + + def test_main(self): + for num_workers in [0, 2]: + self.run_main(num_workers) + + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_split.py b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_split.py index 562051335850a..d2b7971a85dd0 100644 --- a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_split.py +++ b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_split.py @@ -58,7 +58,7 @@ def test_main(self): rets = [] for d in dataloader: - rets.append(d[0].numpy()[0][0]) + rets.append(d.numpy()[0][0]) assert tuple(sorted(rets)) == tuple(range(0, 10)) @@ -102,7 +102,7 @@ def worker_spliter(worker_id): rets = [] for d in dataloader: - rets.append(d[0].numpy()[0][0]) + rets.append(d.numpy()[0][0]) assert tuple(sorted(rets)) == tuple(range(0, 10)) diff --git a/python/paddle/fluid/tests/unittests/test_pad3d_op.py b/python/paddle/fluid/tests/unittests/test_pad3d_op.py index 88d3d80a14c78..8dc825e60bc4d 100644 --- a/python/paddle/fluid/tests/unittests/test_pad3d_op.py +++ b/python/paddle/fluid/tests/unittests/test_pad3d_op.py @@ -467,12 +467,15 @@ def test_class(self): for place in self.places: input_shape = (3, 4, 5) pad = [1, 2] + pad_int = 1 value = 100 input_data = np.random.rand(*input_shape).astype(np.float32) pad_reflection = nn.Pad1D(padding=pad, mode="reflect") pad_replication = nn.Pad1D(padding=pad, mode="replicate") pad_constant = nn.Pad1D(padding=pad, mode="constant", value=value) + pad_constant_int = nn.Pad1D( + padding=pad_int, mode="constant", value=value) pad_circular = nn.Pad1D(padding=pad, mode="circular") data = paddle.to_tensor(input_data) @@ -492,6 +495,14 @@ def test_class(self): input_data, pad, "constant", value=value, data_format="NCL") self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_constant_int(data) + np_out = self._get_numpy_out( + input_data, [pad_int] * 2, + "constant", + value=value, + data_format="NCL") + self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_circular(data) np_out = self._get_numpy_out( input_data, pad, "circular", value=value, data_format="NCL") @@ -541,12 +552,15 @@ def test_class(self): for place in self.places: input_shape = (3, 4, 5, 6) pad = [1, 2, 2, 1] + pad_int = 1 value = 100 input_data = np.random.rand(*input_shape).astype(np.float32) pad_reflection = nn.Pad2D(padding=pad, mode="reflect") pad_replication = nn.Pad2D(padding=pad, mode="replicate") pad_constant = nn.Pad2D(padding=pad, mode="constant", value=value) + pad_constant_int = nn.Pad2D( + padding=pad_int, mode="constant", value=value) pad_circular = nn.Pad2D(padding=pad, mode="circular") data = paddle.to_tensor(input_data) @@ -566,6 +580,14 @@ def test_class(self): input_data, pad, "constant", value=value, data_format="NCHW") self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_constant_int(data) + np_out = self._get_numpy_out( + input_data, [pad_int] * 4, + "constant", + value=value, + data_format="NCHW") + self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_circular(data) np_out = self._get_numpy_out( input_data, pad, "circular", data_format="NCHW") @@ -617,12 +639,15 @@ def test_class(self): for place in self.places: input_shape = (3, 4, 5, 6, 7) pad = [1, 2, 2, 1, 1, 0] + pad_int = 1 value = 100 input_data = np.random.rand(*input_shape).astype(np.float32) pad_reflection = nn.Pad3D(padding=pad, mode="reflect") pad_replication = nn.Pad3D(padding=pad, mode="replicate") pad_constant = nn.Pad3D(padding=pad, mode="constant", value=value) + pad_constant_int = nn.Pad3D( + padding=pad_int, mode="constant", value=value) pad_circular = nn.Pad3D(padding=pad, mode="circular") data = paddle.to_tensor(input_data) @@ -642,6 +667,14 @@ def test_class(self): input_data, pad, "constant", value=value, data_format="NCDHW") self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_constant_int(data) + np_out = self._get_numpy_out( + input_data, [pad_int] * 6, + "constant", + value=value, + data_format="NCDHW") + self.assertTrue(np.allclose(output.numpy(), np_out)) + output = pad_circular(data) np_out = self._get_numpy_out( input_data, pad, "circular", data_format="NCDHW") diff --git a/python/paddle/fluid/tests/unittests/test_pipeline.py b/python/paddle/fluid/tests/unittests/test_pipeline.py index e6d585e5bc176..cd592416c1a51 100644 --- a/python/paddle/fluid/tests/unittests/test_pipeline.py +++ b/python/paddle/fluid/tests/unittests/test_pipeline.py @@ -34,9 +34,13 @@ def _setup_config(self): def test_dist_train(self): import paddle.fluid as fluid if fluid.core.is_compiled_with_cuda(): + # TODO (sandyhouse) fix the delta value. + # Now pipeline only gets the loss value of the last + # microbatch, so it is not consistable with the + # non-pipeline one. self.check_with_place( "pipeline_mnist.py", - delta=1e-5, + delta=1e0, check_error_log=True, log_name=flag_name) diff --git a/python/paddle/fluid/tests/unittests/test_pool2d_op.py b/python/paddle/fluid/tests/unittests/test_pool2d_op.py index e6d41902a7c6d..d66bdd2948d46 100644 --- a/python/paddle/fluid/tests/unittests/test_pool2d_op.py +++ b/python/paddle/fluid/tests/unittests/test_pool2d_op.py @@ -41,6 +41,8 @@ def max_pool2D_forward_naive(x, exclusive=True, adaptive=False, data_type=np.float64): + if data_type == np.float64 and core.is_compiled_with_rocm(): + data_type = np.float32 N, C, H, W = x.shape if global_pool == 1: ksize = [H, W] @@ -81,6 +83,8 @@ def avg_pool2D_forward_naive(x, exclusive=True, adaptive=False, data_type=np.float64): + if data_type == np.float64 and core.is_compiled_with_rocm(): + data_type = np.float32 N, C, H, W = x.shape if global_pool == 1: ksize = [H, W] @@ -340,7 +344,7 @@ def init_kernel_type(self): self.use_cudnn = False def init_data_type(self): - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 def init_pool_type(self): self.pool_type = "avg" diff --git a/python/paddle/fluid/tests/unittests/test_reduce_op.py b/python/paddle/fluid/tests/unittests/test_reduce_op.py index e549a2eca2d7d..912df563fcdbf 100644 --- a/python/paddle/fluid/tests/unittests/test_reduce_op.py +++ b/python/paddle/fluid/tests/unittests/test_reduce_op.py @@ -156,9 +156,14 @@ def test_check_output(self): class TestProdOp(OpTest): def setUp(self): self.op_type = "reduce_prod" - self.inputs = {'X': np.random.random((5, 6, 10)).astype("float64")} + self.init_data_type() + self.inputs = {'X': np.random.random((5, 6, 10)).astype(self.data_type)} self.outputs = {'Out': self.inputs['X'].prod(axis=0)} + def init_data_type(self): + self.data_type = "float32" if core.is_compiled_with_rocm( + ) else "float64" + def test_check_output(self): self.check_output() @@ -169,14 +174,19 @@ def test_check_grad(self): class TestProd6DOp(OpTest): def setUp(self): self.op_type = "reduce_prod" + self.init_data_type() self.inputs = { - 'X': np.random.random((5, 6, 2, 3, 4, 2)).astype("float64") + 'X': np.random.random((5, 6, 2, 3, 4, 2)).astype(self.data_type) } self.attrs = {'dim': [2, 3, 4]} self.outputs = { 'Out': self.inputs['X'].prod(axis=tuple(self.attrs['dim'])) } + def init_data_type(self): + self.data_type = "float32" if core.is_compiled_with_rocm( + ) else "float64" + def test_check_output(self): self.check_output() @@ -187,14 +197,20 @@ def test_check_grad(self): class TestProd8DOp(OpTest): def setUp(self): self.op_type = "reduce_prod" + self.init_data_type() self.inputs = { - 'X': np.random.random((2, 5, 3, 2, 2, 3, 4, 2)).astype("float64") + 'X': np.random.random( + (2, 5, 3, 2, 2, 3, 4, 2)).astype(self.data_type) } self.attrs = {'dim': [2, 3, 4]} self.outputs = { 'Out': self.inputs['X'].prod(axis=tuple(self.attrs['dim'])) } + def init_data_type(self): + self.data_type = "float32" if core.is_compiled_with_rocm( + ) else "float64" + def test_check_output(self): self.check_output() diff --git a/python/paddle/fluid/tests/unittests/test_rnn_op.py b/python/paddle/fluid/tests/unittests/test_rnn_op.py index 5ad2ffec98247..22e07b0bc48c0 100644 --- a/python/paddle/fluid/tests/unittests/test_rnn_op.py +++ b/python/paddle/fluid/tests/unittests/test_rnn_op.py @@ -47,8 +47,10 @@ def get_weight_names(self): def setUp(self): self.op_type = "rnn" - self.dtype = np.float64 - self.sequence_length = np.array([12, 11, 10, 9, 8], dtype=np.int32) + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.sequence_length = None if core.is_compiled_with_rocm( + ) else np.array( + [12, 11, 10, 9, 8], dtype=np.int32) self.num_layers = 1 self.is_bidirec = False self.mode = "LSTM" @@ -78,12 +80,31 @@ def setUp(self): num_layers=self.num_layers, time_major=True, direction=direction, - dropout=self.dropout) + dropout=self.dropout, + dtype=self.dtype) flat_w = get_params_for_net(rnn1) output, (last_hidden, last_cell) = rnn1( input, sequence_length=self.sequence_length) + if core.is_compiled_with_rocm(): + + def rocm_rnn_get_place(): + places = [core.CUDAPlace(0)] + return places + + self._get_places = rocm_rnn_get_place + + if self.is_bidirec: + for i in range(0, len(flat_w), 4): + flat_w[i + 1], flat_w[i + 2] = flat_w[i + 2], flat_w[i + 1] + + for i in range(len(flat_w)): + w = np.split(flat_w[i][1], 4, 0) + w = [w[0], w[1], w[3], w[2]] + w = np.concatenate(w) + flat_w[i] = (flat_w[i][0], w) + init_h = np.zeros((self.num_layers * self.direction_num, batch_size, hidden_size)).astype(self.dtype) init_c = np.zeros((self.num_layers * self.direction_num, batch_size, diff --git a/python/paddle/fluid/tests/unittests/test_roi_align_op.py b/python/paddle/fluid/tests/unittests/test_roi_align_op.py index 940a3e9f9605b..7d030855d114e 100644 --- a/python/paddle/fluid/tests/unittests/test_roi_align_op.py +++ b/python/paddle/fluid/tests/unittests/test_roi_align_op.py @@ -129,9 +129,9 @@ def calc_roi_align(self): roi_width = roi_xmax - roi_xmin roi_height = roi_ymax - roi_ymin - if not self.aligned: - roi_width = max(roi_width, 1) - roi_height = max(roi_height, 1) + roi_width = max(roi_width, 1) + roi_height = max(roi_height, 1) + bin_size_h = float(roi_height) / float(self.pooled_height) bin_size_w = float(roi_width) / float(self.pooled_width) roi_bin_grid_h = self.sampling_ratio if self.sampling_ratio > 0 else \ diff --git a/python/paddle/fluid/tests/unittests/test_softmax_op.py b/python/paddle/fluid/tests/unittests/test_softmax_op.py index 9b0de4e59b4f0..a1cbefa40f307 100644 --- a/python/paddle/fluid/tests/unittests/test_softmax_op.py +++ b/python/paddle/fluid/tests/unittests/test_softmax_op.py @@ -55,7 +55,8 @@ def setUp(self): self.op_type = "softmax" self.use_cudnn = False self.use_mkldnn = False - self.dtype = np.float64 + # explicilty use float32 for ROCm, as MIOpen does not yet support float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.init_kernel_type() self.shape = self.get_x_shape() self.axis = self.get_axis() @@ -338,8 +339,13 @@ def test_dygraph_check(self): for r in [out1, out2]: self.assertEqual(np.allclose(out_ref, r.numpy()), True) - out = self.softmax(x, dtype=np.float64) - out_ref = ref_softmax(self.x_np, axis=-1, dtype=np.float64) + # explicilty use float32 for ROCm, as MIOpen does not yet support float64 + if core.is_compiled_with_rocm(): + out = self.softmax(x, dtype=np.float32) + out_ref = ref_softmax(self.x_np, axis=-1, dtype=np.float32) + else: + out = self.softmax(x, dtype=np.float64) + out_ref = ref_softmax(self.x_np, axis=-1, dtype=np.float64) self.assertEqual(np.allclose(out_ref, out.numpy()), True) paddle.enable_static() diff --git a/python/paddle/fluid/tests/unittests/test_softmax_with_cross_entropy_op.py b/python/paddle/fluid/tests/unittests/test_softmax_with_cross_entropy_op.py index 0ee58d5be15e6..e1f5ecf268304 100644 --- a/python/paddle/fluid/tests/unittests/test_softmax_with_cross_entropy_op.py +++ b/python/paddle/fluid/tests/unittests/test_softmax_with_cross_entropy_op.py @@ -51,10 +51,12 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = False self.soft_label = False - self.dtype = np.float64 + # explicilty use float32 for ROCm, as MIOpen does not yet support float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = -1 self.ignore_index = -1 self.shape = [41, 37] + self.softmax_switch = True def setUp(self): self.initParams() @@ -75,7 +77,11 @@ def setUp(self): loss = cross_entropy(softmax, labels, self.soft_label, self.axis, self.ignore_index) - self.inputs = {"Logits": logits, "Label": labels} + if self.softmax_switch == False: + self.inputs = {"Logits": softmax, "Label": labels} + else: + self.inputs = {"Logits": logits, "Label": labels} + self.outputs = { "Softmax": softmax.astype(self.dtype), "Loss": loss.astype(self.dtype) @@ -84,6 +90,7 @@ def setUp(self): "numeric_stable_mode": self.numeric_stable_mode, "soft_label": self.soft_label, "ignore_index": self.ignore_index, + "softmax_switch": self.softmax_switch, } if self.axis != -1: @@ -93,7 +100,219 @@ def test_check_output(self): self.check_output() def test_check_grad(self): - self.check_grad(["Logits"], "Loss", max_relative_error=5e-5) + if core.is_compiled_with_rocm(): + # HIP will have accuracy fail when using float32 in CPU place + self.check_grad(["Logits"], "Loss", max_relative_error=5e-1) + else: + self.check_grad(["Logits"], "Loss", numeric_grad_delta=0.001) + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_SoftLabel_1D( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = True + self.shape = [13, 8] + self.axis = -1 + self.ignore_index = -1 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_1D( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.shape = [13, 8] + self.axis = -1 + self.ignore_index = -1 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +############################################################################## +#NotWithSoftmax_SoftLabel_2D start +############################################################################## +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_SoftLabel_2D( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = True + self.shape = [3, 5, 7, 11] + self.axis = -1 + self.ignore_index = -1 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_SoftLabel_2D_Axis2( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 1 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_SoftLabel_2D_Axis3( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 2 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_SoftLabel_2D_Axis4( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = True + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 3 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +############################################################################## +#NotWithSoftmax_SoftLabel_2D end +############################################################################## + +############################################################################## +#NotWithSoftmax_HardLabel_2D start +############################################################################## + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.shape = [3, 5, 7, 11] + self.axis = -1 + self.ignore_index = -1 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D_Axis2( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 1 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D_Axis3( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 2 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D_Axis4( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 3 + self.ignore_index = -1 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +############################################################################## +#NotWithSoftmax_HardLabel_2D end +############################################################################## + +############################################################################## +#NotWithSoftmax_HardLabel_2D_Ignore start +############################################################################## + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_Ignore( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = False + self.soft_label = False + self.shape = [13, 8] + self.axis = -1 + self.ignore_index = 2 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_Ignore_Axis( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = False + self.soft_label = False + self.shape = [13, 8] + self.axis = 1 + self.ignore_index = 2 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D_Ignore( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.shape = [3, 5, 7, 11] + self.axis = -1 + self.ignore_index = 2 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = False #default is true, means "with softmax" + + +class TestSoftmaxWithCrossEntropyOp_NotWithSoftmax_HardLabel_2D_Ignore_Axis3( + TestSoftmaxWithCrossEntropyOp): + def initParams(self): + self.op_type = "softmax_with_cross_entropy" + self.numeric_stable_mode = True + self.soft_label = False + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.axis = 2 + self.ignore_index = 2 + self.shape = [3, 5, 7, 11] + self.softmax_switch = False #default is true, means "with softmax" + + +############################################################################## +#NotWithSoftmax_HardLabel_2D_Ignore end +############################################################################## class TestSoftmaxWithCrossEntropyOpNoCudnn(TestSoftmaxWithCrossEntropyOp): @@ -104,7 +323,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = -1 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True @unittest.skipIf(not core.is_compiled_with_cuda(), @@ -124,9 +344,10 @@ def setUp(self): self.op_type = "softmax_with_cross_entropy" # NOTE: numpy float16 have very low accuracy, use float32 for numpy check. + date_type = np.float32 if core.is_compiled_with_rocm() else np.float64 logits = getattr( self, "logits", - np.random.uniform(0.1, 1.0, self.shape).astype(np.float64)) + np.random.uniform(0.1, 1.0, self.shape).astype(date_type)) softmax = np.apply_along_axis(stable_softmax, self.axis, logits) axis_dim = self.shape[self.axis] @@ -178,16 +399,21 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = True - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = -1 self.ignore_index = -1 self.shape = [41, 37] + self.softmax_switch = True def test_check_output(self): self.check_output() def test_check_grad(self): - self.check_grad(["Logits"], "Loss") + if core.is_compiled_with_rocm(): + # HIP will have accuracy fail when using float32 in CPU place + self.check_grad(["Logits"], "Loss", max_relative_error=0.1) + else: + self.check_grad(["Logits"], "Loss") class TestSoftmaxWithCrossEntropyOp3(TestSoftmaxWithCrossEntropyOp): @@ -202,7 +428,8 @@ def initParams(self): self.shape = [41, 37] self.ignore_index = 5 self.axis = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOp3NoCudnn(TestSoftmaxWithCrossEntropyOp3): @@ -213,7 +440,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.ignore_index = 4 self.axis = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpAxis1(TestSoftmaxWithCrossEntropyOp): @@ -226,10 +454,11 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = False - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = 0 self.ignore_index = -1 self.shape = [3, 5, 7, 11] + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpAxis2(TestSoftmaxWithCrossEntropyOp): @@ -242,10 +471,11 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = False - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = 1 self.ignore_index = -1 self.shape = [3, 5, 7, 11] + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpAxis3(TestSoftmaxWithCrossEntropyOp): @@ -258,10 +488,11 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = False - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = 2 self.ignore_index = -1 self.shape = [3, 5, 7, 11] + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpAxis4(TestSoftmaxWithCrossEntropyOp): @@ -274,10 +505,11 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = False - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = 3 self.ignore_index = -1 self.shape = [3, 5, 7, 11] + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpAxisDimEqualOne( @@ -291,10 +523,11 @@ def initParams(self): self.op_type = "softmax_with_cross_entropy" self.numeric_stable_mode = True self.soft_label = False - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.axis = -1 self.ignore_index = -1 self.shape = [3, 5, 7, 1] + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpNoCudnnFp16Axis1( @@ -307,6 +540,7 @@ def initParams(self): self.axis = 0 self.ignore_index = -1 self.dtype = np.float16 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpNoCudnnFp16Axis2( @@ -319,6 +553,7 @@ def initParams(self): self.axis = 1 self.ignore_index = -1 self.dtype = np.float16 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpNoCudnnFp16Axis3( @@ -331,6 +566,7 @@ def initParams(self): self.axis = 2 self.ignore_index = -1 self.dtype = np.float16 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpSoftLabelAxis1( @@ -342,7 +578,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = 0 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpSoftLabelAxis2( @@ -354,7 +591,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = 1 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpSoftLabelAxis3( @@ -366,7 +604,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = 2 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpSoftLabelAxis4( @@ -378,7 +617,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = 3 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpIgnoreIndexNoCudnnAxis1( @@ -390,7 +630,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.ignore_index = 1 self.axis = 0 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpIgnoreIndexNoCudnnAxis2( @@ -402,7 +643,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.ignore_index = 0 self.axis = 1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpIgnoreIndexNoCudnnAxis3( @@ -414,7 +656,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.ignore_index = 3 self.axis = 2 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpIgnoreIndexNoCudnnAxis4( @@ -426,7 +669,8 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.ignore_index = 3 self.axis = 3 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpBoundary0(TestSoftmaxWithCrossEntropyOp): @@ -442,8 +686,9 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = -1 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.logits = np.full(self.shape, -500.0).astype(self.dtype) + self.softmax_switch = True class TestSoftmaxWithCrossEntropyOpBoundary1(TestSoftmaxWithCrossEntropyOp): @@ -459,9 +704,10 @@ def initParams(self): self.shape = [3, 5, 7, 11] self.axis = -1 self.ignore_index = -1 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.logits = np.full(self.shape, 1000.0).astype(self.dtype) self.logits[:, :, 0, :] = -1000.0 + self.softmax_switch = True if __name__ == "__main__": diff --git a/python/paddle/fluid/tests/unittests/test_spawn_and_init_parallel_env.py b/python/paddle/fluid/tests/unittests/test_spawn_and_init_parallel_env.py index 53efa186d1993..6efab81a265ea 100644 --- a/python/paddle/fluid/tests/unittests/test_spawn_and_init_parallel_env.py +++ b/python/paddle/fluid/tests/unittests/test_spawn_and_init_parallel_env.py @@ -59,10 +59,10 @@ def test_nprocs_greater_than_device_num_error(self): with self.assertRaises(RuntimeError): _get_subprocess_env_list(nprocs=100, options=dict()) - def test_selected_gpus_error(self): + def test_selected_devices_error(self): with self.assertRaises(ValueError): options = dict() - options['selected_gpus'] = "100,101" + options['selected_devices'] = "100,101" _get_subprocess_env_list(nprocs=2, options=options) def test_get_correct_env(self): @@ -72,15 +72,15 @@ def test_get_correct_env(self): self.assertEqual(env_dict['PADDLE_TRAINER_ID'], '0') self.assertEqual(env_dict['PADDLE_TRAINERS_NUM'], '1') - def test_nprocs_not_equal_to_selected_gpus(self): + def test_nprocs_not_equal_to_selected_devices(self): with self.assertRaises(ValueError): options = dict() - options['selected_gpus'] = "100,101,102" + options['selected_devices'] = "100,101,102" _get_subprocess_env_list(nprocs=2, options=options) def test_options_valid_check(self): options = dict() - options['selected_gpus'] = "100,101,102" + options['selected_devices'] = "100,101,102" _options_valid_check(options) with self.assertRaises(ValueError): diff --git a/python/paddle/fluid/tests/unittests/test_sync_batch_norm_op.py b/python/paddle/fluid/tests/unittests/test_sync_batch_norm_op.py index 4649323b5b395..13aa7d3d37dd4 100644 --- a/python/paddle/fluid/tests/unittests/test_sync_batch_norm_op.py +++ b/python/paddle/fluid/tests/unittests/test_sync_batch_norm_op.py @@ -50,7 +50,7 @@ class TestSyncBatchNormOpTraining(unittest.TestCase): def setUp(self): """Setup.""" #self.dtype = np.float32 - self.dtype = np.float64 + self.dtype = np.float32 if core.is_compiled_with_rocm() else np.float64 self.N = 8 self.C = 16 self.H = 32 @@ -92,7 +92,10 @@ def _build_program(self, moving_variance_name='bn_moving_variance', data_layout=layout, is_test=only_forward) - bn = fluid.layers.cast(bn, 'float64') + if core.is_compiled_with_rocm(): + bn = fluid.layers.cast(bn, 'float32') + else: + bn = fluid.layers.cast(bn, 'float64') sigmoid = fluid.layers.sigmoid(bn) out = fluid.layers.reduce_sum(sigmoid) if not sync_bn: diff --git a/python/paddle/fluid/tests/unittests/test_temporal_shift_op.py b/python/paddle/fluid/tests/unittests/test_temporal_shift_op.py index 12eec2073b3d0..5bab4a52bf05a 100644 --- a/python/paddle/fluid/tests/unittests/test_temporal_shift_op.py +++ b/python/paddle/fluid/tests/unittests/test_temporal_shift_op.py @@ -22,7 +22,9 @@ from paddle.fluid import core -def temporal_shift(x, seg_num, shift_ratio): +def temporal_shift(x, seg_num, shift_ratio, data_format): + if data_format == "NHWC": + x = np.transpose(x, (0, 3, 1, 2)) shape = x.shape reshape_x = x.reshape((-1, seg_num, shape[1], shape[2], shape[3])) pad_x = np.pad(reshape_x, ((0, 0), (1, 1), (0, 0), (0, 0), (0, 0)), @@ -33,23 +35,28 @@ def temporal_shift(x, seg_num, shift_ratio): slice2 = pad_x[:, 2:seg_num + 2, c1:c2, :, :] slice3 = pad_x[:, 1:seg_num + 1, c2:, :, :] concat_x = np.concatenate([slice1, slice2, slice3], axis=2) - return concat_x.reshape(shape) + out = concat_x.reshape(shape) + if data_format == "NHWC": + out = np.transpose(out, (0, 2, 3, 1)) + return out class TestTemporalShift(OpTest): def setUp(self): self.initTestCase() self.op_type = 'temporal_shift' - x = np.random.random(self.x_shape).astype('float64') + x = np.random.random(self.x_shape).astype(self.dtype) self.attrs = { "seg_num": self.seg_num, "shift_ratio": self.shift_ratio, + "data_format": self.data_format } self.inputs = {"X": x, } - output = temporal_shift(x, self.seg_num, self.shift_ratio) + output = temporal_shift(x, self.seg_num, self.shift_ratio, + self.data_format) self.outputs = {"Out": output} def test_check_output(self): @@ -62,6 +69,8 @@ def initTestCase(self): self.x_shape = (6, 4, 4, 4) self.seg_num = 3 self.shift_ratio = 0.25 + self.dtype = 'float64' + self.data_format = 'NCHW' class TestTemporalShift2(TestTemporalShift): @@ -69,6 +78,7 @@ def initTestCase(self): self.x_shape = (4, 9, 7, 7) self.seg_num = 2 self.shift_ratio = 0.2 + self.data_format = 'NCHW' class TestTemporalShift3(TestTemporalShift): @@ -76,6 +86,36 @@ def initTestCase(self): self.x_shape = (3, 10, 5, 5) self.seg_num = 1 self.shift_ratio = 0.3 + self.data_format = 'NCHW' + + +class TestTemporalShift4(TestTemporalShift): + def initTestCase(self): + self.x_shape = (6, 5, 5, 4) + self.seg_num = 3 + self.shift_ratio = 0.25 + self.data_format = 'NHWC' + + +@unittest.skipIf(not core.is_compiled_with_cuda(), + "core is not compiled with CUDA") +class TestTemporalShiftFP16(TestTemporalShift): + def initTestCase(self): + self.x_shape = (3, 10, 5, 5) + self.seg_num = 1 + self.shift_ratio = 0.3 + self.dtype = 'float16' + self.data_format = 'NCHW' + + def test_check_output(self): + place = core.CUDAPlace(0) + if core.is_float16_supported(place): + self.check_output_with_place(place) + + def test_check_grad_ignore_uv(self): + place = core.CUDAPlace(0) + if core.is_float16_supported(place): + self.check_grad_with_place(place, ['X'], 'Out') class TestTemporalShiftAPI(unittest.TestCase): @@ -93,6 +133,14 @@ def test_api(self): out = paddle.nn.functional.temporal_shift( x=input, seg_num=2, shift_ratio=0.2) + def test_error(self): + def attr_data_format(): + input = paddle.randn([6, 4, 2, 2]) + out = paddle.nn.functional.temporal_shift( + x=input, seg_num=2, shift_ratio=0.2, data_format="HWC") + + self.assertRaises(ValueError, attr_data_format) + if __name__ == "__main__": unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_var_base.py b/python/paddle/fluid/tests/unittests/test_var_base.py index 6c5458c1a2cb9..b0c9dda7a3098 100644 --- a/python/paddle/fluid/tests/unittests/test_var_base.py +++ b/python/paddle/fluid/tests/unittests/test_var_base.py @@ -631,10 +631,14 @@ def test_print_tensor_dtype(self): class TestVarBaseSetitem(unittest.TestCase): def setUp(self): paddle.disable_static() - self.tensor_x = paddle.to_tensor(np.ones((4, 2, 3)).astype(np.float32)) - self.np_value = np.random.random((2, 3)).astype(np.float32) + self.set_dtype() + self.tensor_x = paddle.to_tensor(np.ones((4, 2, 3)).astype(self.dtype)) + self.np_value = np.random.random((2, 3)).astype(self.dtype) self.tensor_value = paddle.to_tensor(self.np_value) + def set_dtype(self): + self.dtype = "int32" + def _test(self, value): paddle.disable_static() self.assertEqual(self.tensor_x.inplace_version, 0) @@ -644,7 +648,7 @@ def _test(self, value): self.assertEqual(self.tensor_x.inplace_version, 1) if isinstance(value, (six.integer_types, float)): - result = np.zeros((2, 3)).astype(np.float32) + value + result = np.zeros((2, 3)).astype(self.dtype) + value else: result = self.np_value @@ -674,11 +678,26 @@ def test_value_int(self): paddle.disable_static() self._test(10) + +class TestVarBaseSetitemInt64(TestVarBaseSetitem): + def set_dtype(self): + self.dtype = "int64" + + +class TestVarBaseSetitemFp32(TestVarBaseSetitem): + def set_dtype(self): + self.dtype = "float32" + def test_value_float(self): paddle.disable_static() self._test(3.3) +class TestVarBaseSetitemFp64(TestVarBaseSetitem): + def set_dtype(self): + self.dtype = "float64" + + class TestVarBaseInplaceVersion(unittest.TestCase): def test_setitem(self): paddle.disable_static() diff --git a/python/paddle/fluid/tests/unittests/xpu/test_lamb_op_xpu.py b/python/paddle/fluid/tests/unittests/xpu/test_lamb_op_xpu.py new file mode 100644 index 0000000000000..0e1714f1922de --- /dev/null +++ b/python/paddle/fluid/tests/unittests/xpu/test_lamb_op_xpu.py @@ -0,0 +1,121 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function +import sys +sys.path.append("..") +import unittest +import numpy as np +from op_test_xpu import XPUOpTest +from paddle.fluid import core +from paddle.fluid.op import Operator +import paddle.fluid as fluid +import paddle + + +class TestLambOp1(XPUOpTest): + def set_attrs(self): + self.attrs = { + 'epsilon': 1e-6, + 'beta1': 0.9, + 'beta2': 0.999, + 'weight_decay': 0.01 + } + + def setUp(self): + '''Test Lamb Op with supplied attributes + ''' + self.op_type = "lamb" + param = np.random.uniform(-1, 1, 5000).astype("float32") + grad = np.random.uniform(-1, 1, 5000).astype("float32") + moment1 = np.random.uniform(-1, 1, 5000).astype("float32") + moment2 = np.random.random(5000).astype("float32") + + self.set_attrs() + learning_rate = 0.001 + beta1_pow = self.attrs['beta1'] + beta2_pow = self.attrs['beta2'] + + self.inputs = { + 'Param': param, + 'Grad': grad, + 'Moment1': moment1, + 'Moment2': moment2, + 'LearningRate': np.array([learning_rate]).astype("float32"), + 'Beta1Pow': np.array([beta1_pow]).astype("float32"), + 'Beta2Pow': np.array([beta2_pow]).astype("float32") + } + + param_out, moment1_out, moment2_out, \ + beta1_pow_out, beta2_pow_out = lamb_step(self.inputs, self.attrs) + + self.outputs = { + 'Moment1Out': moment1_out, + 'Moment2Out': moment2_out, + 'ParamOut': param_out, + 'Beta1PowOut': beta1_pow_out, + 'Beta2PowOut': beta2_pow_out + } + + def test_check_output(self): + self.check_output_with_place(paddle.XPUPlace(0)) + + +def lamb_step(inputs, attributes): + ''' + Simulate one step of the lamb optimizer + :param inputs: dict of inputs + :param attributes: dict of attributes + :return tuple: tuple of output param, moment1, moment2, + beta1 power accumulator and beta2 power accumulator + ''' + param = inputs['Param'] + grad = inputs['Grad'] + moment1 = inputs['Moment1'] + moment2 = inputs['Moment2'] + lr = inputs['LearningRate'] + beta1_pow = inputs['Beta1Pow'] + beta2_pow = inputs['Beta2Pow'] + + beta1 = attributes['beta1'] + beta2 = attributes['beta2'] + epsilon = attributes['epsilon'] + weight_decay = attributes['weight_decay'] + + moment1_out = beta1 * moment1 + (1 - beta1) * grad + moment2_out = beta2 * moment2 + (1 - beta2) * np.square(grad) + + moment1_unbiased = moment1_out / (1 - beta1_pow) + moment2_unbiased = moment2_out / (1 - beta2_pow) + + r_1 = np.linalg.norm(param) + r_2 = np.linalg.norm(moment1_unbiased / (np.sqrt(moment2_unbiased) + epsilon + ) + weight_decay * param) + if r_1 > 0.0 and r_2 > 0.0: + lr_t = lr * r_1 / r_2 + else: + lr_t = 1.0 + + param_out = param - lr_t * (moment1_unbiased / ( + np.sqrt(moment2_unbiased) + epsilon) + weight_decay * param) + + beta1_pow_out = beta1_pow * beta1 + beta2_pow_out = beta2_pow * beta2 + + return param_out, moment1_out, moment2_out, beta1_pow_out, beta2_pow_out + + +if __name__ == "__main__": + paddle.enable_static() + unittest.main() diff --git a/python/paddle/hapi/model_summary.py b/python/paddle/hapi/model_summary.py index babbe962a9525..9f2769e1ca285 100644 --- a/python/paddle/hapi/model_summary.py +++ b/python/paddle/hapi/model_summary.py @@ -341,10 +341,12 @@ def _get_str_length(summary): total_params += summary[layer]["nb_params"] try: - total_output += np.prod(summary[layer]["output_shape"]) + total_output += np.sum( + np.prod( + summary[layer]["output_shape"], axis=-1)) except: for output_shape in summary[layer]["output_shape"]: - total_output += np.prod(output_shape) + total_output += np.sum(np.prod(output_shape, axis=-1)) if "trainable" in summary[layer]: if summary[layer]["trainable"] == True: diff --git a/python/paddle/nn/functional/conv.py b/python/paddle/nn/functional/conv.py index eaa4dc4d4f2cd..75dc62e530d0d 100644 --- a/python/paddle/nn/functional/conv.py +++ b/python/paddle/nn/functional/conv.py @@ -110,6 +110,12 @@ def _conv_nd(x, use_mkldnn=False, name=None): + # Due to the poor performance of NHWC, we transpose the input to NCHW. + origin_format = data_format + if origin_format == "NHWC" and op_type == "depthwise_conv2d": + x = nn.transpose(x, perm=[0, 3, 1, 2]) + data_format = "NCHW" + channel_dim = 1 if in_dygraph_mode(): attrs = ('strides', stride, 'paddings', padding, 'dilations', dilation, 'groups', groups, 'use_cudnn', use_cudnn, 'use_mkldnn', @@ -154,6 +160,9 @@ def _conv_nd(x, else: out = pre_bias + if origin_format == "NHWC" and op_type == "depthwise_conv2d": + out = nn.transpose(out, perm=[0, 2, 3, 1]) + return out diff --git a/python/paddle/nn/functional/norm.py b/python/paddle/nn/functional/norm.py index 03ba78e12f637..54824233f7076 100644 --- a/python/paddle/nn/functional/norm.py +++ b/python/paddle/nn/functional/norm.py @@ -188,10 +188,10 @@ def batch_norm(x, if in_dygraph_mode(): # for dygraph need tuple - attrs = ("momentum", momentum, "epsilon", epsilon, "data_layout", - data_format, "use_mkldnn", False, "fuse_with_relu", False, - "use_global_stats", use_global_stats, "trainable_statistics", - trainable_statistics) + attrs = ("momentum", momentum, "epsilon", epsilon, "is_test", + not training, "data_layout", data_format, "use_mkldnn", False, + "fuse_with_relu", False, "use_global_stats", use_global_stats, + "trainable_statistics", trainable_statistics) batch_norm_out, _, _, _, _, _ = core.ops.batch_norm( x, weight, bias, running_mean, running_var, mean_out, variance_out, *attrs) @@ -205,6 +205,7 @@ def batch_norm(x, attrs = { "momentum": momentum, "epsilon": epsilon, + "is_test": not training, "data_layout": data_format, "use_mkldnn": False, "fuse_with_relu": False, diff --git a/python/paddle/nn/layer/common.py b/python/paddle/nn/layer/common.py index 05d619bd729d8..d0f97625bcba7 100644 --- a/python/paddle/nn/layer/common.py +++ b/python/paddle/nn/layer/common.py @@ -38,6 +38,13 @@ ] +def _npairs(x, n): + if isinstance(x, (paddle.Tensor, list)): + return x + x = [x] * (n * 2) + return x + + class Linear(layers.Layer): r""" @@ -915,7 +922,8 @@ class Pad1D(layers.Layer): If mode is 'reflect', pad[0] and pad[1] must be no greater than width-1. Parameters: - padding (Tensor | List[int32]): The padding size with data type int32. [len(padding)/2] dimensions + padding (Tensor | List[int] | int): The padding size with data type int. If is int, use the + same padding in both dimensions. Else [len(padding)/2] dimensions of input will be padded. The pad has the form (pad_left, pad_right). mode (str): Four modes: 'constant' (default), 'reflect', 'replicate', 'circular'. When in 'constant' mode, this op uses a constant value to pad the input tensor. @@ -968,7 +976,7 @@ def __init__(self, data_format="NCL", name=None): super(Pad1D, self).__init__() - self._pad = padding + self._pad = _npairs(padding, 1) self._mode = mode self._value = value self._data_format = data_format @@ -996,8 +1004,9 @@ class Pad2D(layers.Layer): than width-1. The height dimension has the same condition. Parameters: - padding (Tensor | List[int32]): The padding size with data type int32. [len(padding)/2] dimensions - of input will be padded. The pad has the form (pad_left, pad_right, pad_top, pad_bottom). + padding (Tensor | List[int] | int): The padding size with data type int. If is int, use the + same padding in all dimensions. Else [len(padding)/2] dimensions of input will be padded. + The pad has the form (pad_left, pad_right, pad_top, pad_bottom). mode (str): Four modes: 'constant' (default), 'reflect', 'replicate', 'circular'. When in 'constant' mode, this op uses a constant value to pad the input tensor. When in 'reflect' mode, uses reflection of the input boundaries to pad the input tensor. @@ -1051,7 +1060,7 @@ def __init__(self, data_format="NCHW", name=None): super(Pad2D, self).__init__() - self._pad = padding + self._pad = _npairs(padding, 2) self._mode = mode self._value = value self._data_format = data_format @@ -1079,7 +1088,8 @@ class Pad3D(layers.Layer): than width-1. The height and depth dimension has the same condition. Parameters: - padding (Tensor | List[int32]): The padding size with data type int32. [len(padding)/2] dimensions + padding (Tensor | List[int] | int): The padding size with data type int. If is int, use the + same padding in all dimensions. Else [len(padding)/2] dimensions of input will be padded. The pad has the form (pad_left, pad_right, pad_top, pad_bottom, pad_front, pad_back). mode (str): Four modes: 'constant' (default), 'reflect', 'replicate', 'circular'. When in 'constant' mode, this op uses a constant value to pad the input tensor. @@ -1134,7 +1144,7 @@ def __init__(self, data_format="NCDHW", name=None): super(Pad3D, self).__init__() - self._pad = padding + self._pad = _npairs(padding, 3) self._mode = mode self._value = value self._data_format = data_format diff --git a/python/paddle/nn/layer/conv.py b/python/paddle/nn/layer/conv.py index 2c6308d112925..389920b923876 100644 --- a/python/paddle/nn/layer/conv.py +++ b/python/paddle/nn/layer/conv.py @@ -85,6 +85,12 @@ def __init__(self, "when padding_mode in ['reflect', 'replicate', 'circular'], type of padding must be int" ) + valid_format = {'NHWC', 'NCHW', 'NDHWC', 'NCDHW', 'NLC', 'NCL'} + if data_format not in valid_format: + raise ValueError( + "data_format must be one of {}, but got data_format='{}'". + format(valid_format, data_format)) + channel_last = (data_format == "NHWC") or (data_format == "NDHWC") or ( data_format == "NLC") if channel_last: diff --git a/python/paddle/nn/layer/pooling.py b/python/paddle/nn/layer/pooling.py index 0f3c4449a3f20..5830af3a182d4 100755 --- a/python/paddle/nn/layer/pooling.py +++ b/python/paddle/nn/layer/pooling.py @@ -589,8 +589,8 @@ class MaxPool3D(layers.Layer): def __init__(self, kernel_size, - stride, - padding, + stride=None, + padding=0, return_mask=False, ceil_mode=False, data_format="NCDHW", diff --git a/python/paddle/optimizer/adam.py b/python/paddle/optimizer/adam.py index b0c05cf8de76c..0cafbda893dd2 100644 --- a/python/paddle/optimizer/adam.py +++ b/python/paddle/optimizer/adam.py @@ -351,7 +351,7 @@ def step(self): """ params_grads = [] for param in self._parameter_list: - if not param.trainable: + if param.stop_gradient: continue if param._grad_ivar() is not None: grad_var = param._grad_ivar() diff --git a/python/paddle/optimizer/adamax.py b/python/paddle/optimizer/adamax.py index bd65fc19c32aa..4a6c2278a46f4 100644 --- a/python/paddle/optimizer/adamax.py +++ b/python/paddle/optimizer/adamax.py @@ -184,7 +184,7 @@ def _finish_update(self, block, parameters_and_grads): """ assert isinstance(block, framework.Block) for param, grad in parameters_and_grads: - if grad is None or param.trainable is False: + if grad is None or param.stop_gradient is True: continue with param.block.program._optimized_guard( [param, grad]), name_scope('adamax'): diff --git a/python/paddle/optimizer/optimizer.py b/python/paddle/optimizer/optimizer.py index 212dad7c77cb4..b37d172606411 100644 --- a/python/paddle/optimizer/optimizer.py +++ b/python/paddle/optimizer/optimizer.py @@ -542,7 +542,7 @@ def _get_accumulator(self, name, param): def _update_param_device_map(self, parameters_and_grads, target_block): for param_and_grad in parameters_and_grads: - if param_and_grad[0].trainable is True: + if param_and_grad[0].stop_gradient is False: param_name = param_and_grad[0].name ops = target_block.ops device_attr_name = core.op_proto_and_checker_maker.kOpDeviceAttrName( @@ -598,14 +598,14 @@ def _create_optimization_pass(self, parameters_and_grads): self._update_param_device_map(parameters_and_grads, target_block) self._create_accumulators( target_block, - [p[0] for p in parameters_and_grads if p[0].trainable]) + [p[0] for p in parameters_and_grads if not p[0].stop_gradient]) self._create_global_learning_rate() if framework.in_dygraph_mode(): for param_and_grad in parameters_and_grads: if param_and_grad[1] is None: continue - if param_and_grad[0].trainable is True: + if param_and_grad[0].stop_gradient is False: self._append_optimize_op(target_block, param_and_grad) else: for param_and_grad in parameters_and_grads: @@ -613,7 +613,7 @@ def _create_optimization_pass(self, parameters_and_grads): continue with param_and_grad[0].block.program._optimized_guard( param_and_grad), name_scope("optimizer"): - if param_and_grad[0].trainable is True: + if param_and_grad[0].stop_gradient is False: device = self._get_device_for_param(param_and_grad[0] .name) with device_guard(device): @@ -689,7 +689,7 @@ def backward(self, params_grads = [] for param in parameter_list: - if not param.trainable: + if param.stop_gradient: continue if param._grad_ivar() is not None: # create gradient tensor @@ -789,8 +789,9 @@ def _apply_optimize(self, loss, startup_program, params_grads): def _get_no_grad_set(self, loss, no_grad_set=None): no_grad_set = _get_no_grad_set_name(no_grad_set) parameters = loss.block.program.global_block().all_parameters() - param_no_trainable = set( - [param.name for param in parameters if param.trainable is False]) + param_no_trainable = set([ + param.name for param in parameters if param.stop_gradient is True + ]) # If the parameter is no trainable, it should not have a gradient. no_grad_set.update(param_no_trainable) @@ -825,7 +826,7 @@ def clear_grad(self): """ for p in self._parameter_list: - if p.trainable: + if not p.stop_gradient: p.clear_gradient() @imperative_base.no_grad @@ -920,7 +921,7 @@ def step(self): """ params_grads = [] for param in self._parameter_list: - if not param.trainable: + if param.stop_gradient: continue if param._grad_ivar() is not None: grad_var = param._grad_ivar() diff --git a/python/paddle/static/amp/__init__.py b/python/paddle/static/amp/__init__.py index 604c7c3d2b490..bfc1beed55297 100644 --- a/python/paddle/static/amp/__init__.py +++ b/python/paddle/static/amp/__init__.py @@ -14,5 +14,8 @@ from ...fluid.contrib import mixed_precision from ...fluid.contrib.mixed_precision import * +from ...fluid.contrib.mixed_precision import bf16 +from ...fluid.contrib.mixed_precision.bf16 import * __all__ = mixed_precision.__all__ +__all__ += bf16.__all__ diff --git a/python/paddle/utils/cpp_extension/cpp_extension.py b/python/paddle/utils/cpp_extension/cpp_extension.py index 57bcea658b53c..ea4c85e20db76 100644 --- a/python/paddle/utils/cpp_extension/cpp_extension.py +++ b/python/paddle/utils/cpp_extension/cpp_extension.py @@ -22,11 +22,16 @@ from setuptools.command.build_ext import build_ext from distutils.command.build import build -from .extension_utils import find_cuda_home, normalize_extension_kwargs, add_compile_flag, bootstrap_context -from .extension_utils import is_cuda_file, prepare_unix_cudaflags, prepare_win_cudaflags, add_std_without_repeat, get_build_directory -from .extension_utils import _import_module_from_library, CustomOpInfo, _write_setup_file, _jit_compile, parse_op_name_from -from .extension_utils import check_abi_compatibility, log_v, IS_WINDOWS, OS_NAME -from .extension_utils import use_new_custom_op_load_method, MSVC_COMPILE_FLAGS +from .extension_utils import find_cuda_home, find_rocm_home, normalize_extension_kwargs, add_compile_flag +from .extension_utils import is_cuda_file, prepare_unix_cudaflags, prepare_win_cudaflags +from .extension_utils import _import_module_from_library, _write_setup_file, _jit_compile +from .extension_utils import check_abi_compatibility, log_v, CustomOpInfo, parse_op_name_from +from .extension_utils import use_new_custom_op_load_method, clean_object_if_change_cflags +from .extension_utils import bootstrap_context, get_build_directory, add_std_without_repeat + +from .extension_utils import IS_WINDOWS, OS_NAME, MSVC_COMPILE_FLAGS, MSVC_COMPILE_FLAGS + +from ...fluid import core # Note(zhouwei): On windows, it will export function 'PyInit_[name]' by default, # The solution is: 1.User add function PyInit_[name] 2. set not to export @@ -36,7 +41,10 @@ from unittest.mock import Mock _du_build_ext.get_export_symbols = Mock(return_value=None) -CUDA_HOME = find_cuda_home() +if core.is_compiled_with_rocm(): + ROCM_HOME = find_rocm_home() +else: + CUDA_HOME = find_cuda_home() def setup(**attr): @@ -49,24 +57,28 @@ def setup(**attr): It encapsulates the python built-in ``setuptools.setup`` function and keeps arguments and usage same as the native interface. Meanwhile, it hiddens Paddle inner framework concepts, such as necessary compiling flags, included paths of head files, and linking - flags. It also will automatically search and valid local enviromment and versions of ``cc`` and - ``nvcc`` , then compiles customized operators supporting CPU or GPU device according to - the specified Extension type. + flags. It also will automatically search and valid local enviromment and versions of + ``cc(Linux)`` , ``cl.exe(Windows)`` and ``nvcc`` , then compiles customized operators + supporting CPU or GPU device according to the specified Extension type. Moreover, `ABI compatibility `_ - will be checked to ensure that compiler version from ``cc`` + will be checked to ensure that compiler version from ``cc(Linux)`` , ``cl.exe(Windows)`` on local machine is compatible with pre-installed Paddle whl in python site-packages. - For example if Paddle with CUDA 10.1 is built with GCC 8.2, then the version of user's - local machine should satisfy GCC >= 8.2. Otherwise, a fatal error will occur because of - ABI compatibility. - .. note:: + For Linux, GCC version will be checked . For example if Paddle with CUDA 10.1 is built with GCC 8.2, + then the version of user's local machine should satisfy GCC >= 8.2. + For Windows, Visual Studio version will be checked, and it shoule be greater than or equal to that of + PaddlePaddle (Visual Studio 2015 update3). + If the above conditions are not met, the corresponding warning will be printed, and a fatal error may + occur because of ABI compatibility. - 1. Compiler ABI compatibility is forward compatible. On Linux platform, - we recommend to use GCC 8.2 as soft linking condidate of ``/usr/bin/cc`` . - 2. Using ``which cc`` to ensure location of ``cc`` and using ``cc --version`` - to ensure linking GCC version on Linux. - 3. Currently we support Linux and Windows platfrom. MacOS is supporting... + .. note:: + + 1. Currently we support Linux and Windows platfrom. MacOS is supporting... + 2. On Linux platform, we recommend to use GCC 8.2 as soft linking condidate of ``/usr/bin/cc`` . + Then, Use ``which cc`` to ensure location of ``cc`` and using ``cc --version`` to ensure linking + GCC version. + 3. On Windows platform, we recommend to install `` Visual Studio`` (>=2015 update3). Compared with Just-In-Time ``load`` interface, it only compiles once by executing @@ -357,6 +369,13 @@ def finalize_options(self): def build_extensions(self): self._check_abi() + # Note(Aurelius84): If already compiling source before, we should check whether + # cflags have changed and delete the built shared library to re-compile the source + # even though source file content keep unchanaged. + so_name = self.get_ext_fullpath(self.extensions[0].name) + clean_object_if_change_cflags( + os.path.abspath(so_name), self.extensions[0]) + # Consider .cu, .cu.cc as valid source extensions. self.compiler.src_extensions += ['.cu', '.cu.cc'] # Save the original _compile method for later. @@ -380,12 +399,20 @@ def unix_custom_single_compiler(obj, src, ext, cc_args, extra_postargs, original_compiler = self.compiler.compiler_so # ncvv compile CUDA source if is_cuda_file(src): - assert CUDA_HOME is not None - nvcc_cmd = os.path.join(CUDA_HOME, 'bin', 'nvcc') - self.compiler.set_executable('compiler_so', nvcc_cmd) - # {'nvcc': {}, 'cxx: {}} - if isinstance(cflags, dict): - cflags = cflags['nvcc'] + if core.is_compiled_with_rocm(): + assert ROCM_HOME is not None, "Not found ROCM runtime, please use `export ROCM_PATH= XXX` to specific it." + hipcc_cmd = os.path.join(ROCM_HOME, 'bin', 'hipcc') + self.compiler.set_executable('compiler_so', hipcc_cmd) + # {'nvcc': {}, 'cxx: {}} + if isinstance(cflags, dict): + cflags = cflags['hipcc'] + else: + assert CUDA_HOME is not None, "Not found CUDA runtime, please use `export CUDA_HOME= XXX` to specific it." + nvcc_cmd = os.path.join(CUDA_HOME, 'bin', 'nvcc') + self.compiler.set_executable('compiler_so', nvcc_cmd) + # {'nvcc': {}, 'cxx: {}} + if isinstance(cflags, dict): + cflags = cflags['nvcc'] cflags = prepare_unix_cudaflags(cflags) # cxx compile Cpp source @@ -443,7 +470,7 @@ def win_custom_spawn(cmd): src = src_list[0] obj = obj_list[0] if is_cuda_file(src): - assert CUDA_HOME is not None + assert CUDA_HOME is not None, "Not found CUDA runtime, please use `export CUDA_HOME= XXX` to specific it." nvcc_cmd = os.path.join(CUDA_HOME, 'bin', 'nvcc') if isinstance(self.cflags, dict): cflags = self.cflags['nvcc'] @@ -655,7 +682,6 @@ def load(name, extra_ldflags=None, extra_include_paths=None, build_directory=None, - interpreter=None, verbose=False): """ An Interface to automatically compile C++/CUDA source files Just-In-Time @@ -663,19 +689,23 @@ def load(name, append user defined custom operators in background while building models. It will perform compiling, linking, Python API generation and module loading - processes under a individual subprocess. It does not require CMake or Ninja environment - and only ``g++/nvcc`` on Linux and clang++ on MacOS. For example it requires - GCC compiler with version is greater than 5.4 and linked into ``/usr/bin/cc`` . - If compiling Operators supporting GPU device, please make sure ``nvcc`` compiler - is installed in local environment. - + processes under a individual subprocess. It does not require CMake or Ninja + environment. On Linux platform, it requires GCC compiler whose version is + greater than 5.4 and it should be soft linked to ``/usr/bin/cc`` . On Windows + platform, it requires Visual Studio whose version is greater than 2015 update3. + On MacOS, clang++ is requited. In addition, if compiling Operators supporting + GPU device, please make sure ``nvcc`` compiler is installed in local environment. Moreover, `ABI compatibility `_ - will be checked to ensure that compiler version from ``cc`` + will be checked to ensure that compiler version from ``cc(Linux)`` , ``cl.exe(Windows)`` on local machine is compatible with pre-installed Paddle whl in python site-packages. - For example if Paddle with CUDA 10.1 is built with GCC 8.2, then the version of user's - local machine should satisfy GCC >= 8.2. Otherwise, a fatal error will occur because of - ABI compatibility. + + For Linux, GCC version will be checked . For example if Paddle with CUDA 10.1 is built with GCC 8.2, + then the version of user's local machine should satisfy GCC >= 8.2. + For Windows, Visual Studio version will be checked, and it shoule be greater than or equal to that of + PaddlePaddle (Visual Studio 2015 update3). + If the above conditions are not met, the corresponding warning will be printed, and a fatal error may + occur because of ABI compatibility. Compared with ``setup`` interface, it doesn't need extra ``setup.py`` and excute ``python setup.py install`` command. The interface contains all compiling and installing @@ -683,11 +713,11 @@ def load(name, .. note:: - 1. Compiler ABI compatibility is forward compatible. On Linux platform, - we recommend to use GCC 8.2 as soft linking condidate of ``/usr/bin/cc`` . - 2. Using ``which cc`` to ensure location of ``cc`` and using ``cc --version`` - to ensure linking GCC version on Linux. - 3. Currenly we support Linux and Windows platfrom. MacOS is supporting... + 1. Currently we support Linux and Windows platfrom. MacOS is supporting... + 2. On Linux platform, we recommend to use GCC 8.2 as soft linking condidate of ``/usr/bin/cc`` . + Then, Use ``which cc`` to ensure location of ``cc`` and using ``cc --version`` to ensure linking + GCC version. + 3. On Windows platform, we recommend to install `` Visual Studio`` (>=2015 update3). **A simple example:** @@ -700,10 +730,9 @@ def load(name, custom_op_module = load( name="op_shared_libary_name", # name of shared library sources=['relu_op.cc', 'relu_op.cu'], # source files of cusomized op - extra_cxx_cflags=['-DPADDLE_WITH_MKLDNN'], # need to specify the flag if pre-installed Paddle supports MKLDNN - extra_cuda_cflags=['-DPADDLE_WITH_MKLDNN'], # need to specify the flag if pre-installed Paddle supports MKLDNN - interpreter='python3.7', # optional, specify another python interpreter - verbose=True # output log information + extra_cxx_cflags=['-g', '-w'], # optional, specify extra flags to compile .cc/.cpp file + extra_cuda_cflags=['-O2'], # optional, specify extra flags to compile .cu file + verbose=True # optional, specify to output log information ) x = paddle.randn([4, 10], dtype='float32') @@ -716,11 +745,9 @@ def load(name, and ``.cu`` for CUDA file. extra_cxx_cflags(list[str], optional): Specify additional flags used to compile CPP files. By default all basic and framework related flags have been included. - If your pre-insall Paddle supported MKLDNN, please add - ``-DPADDLE_WITH_MKLDNN`` . Default is None. extra_cuda_cflags(list[str], optional): Specify additional flags used to compile CUDA files. By default - all basic and framework related flags have been included. If your pre-insall Paddle supported MKLDNN, - please add ``-DPADDLE_WITH_MKLDNN`` . Default None. See `Cuda Compiler Driver NVCC `_ + all basic and framework related flags have been included. + See `Cuda Compiler Driver NVCC `_ for details. Default is None. extra_ldflags(list[str], optional): Specify additional flags used to link shared library. See `GCC Link Options `_ for details. @@ -731,10 +758,6 @@ def load(name, build_directory(str, optional): Specify root directory path to put shared library file. If set None, it will use ``PADDLE_EXTENSION_DIR`` from os.environ. Use ``paddle.utils.cpp_extension.get_build_directory()`` to see the location. Default is None. - interpreter(str, optional): Specify nterpreter path, supporting alias and full path. - If set None, it will use `python` as default interpreter. If local environment contains - more than one python interpreters and want to use new interpreter to apply compilation, - please specify this parameter, such as ``python3.7`` . Default is None. verbose(bool, optional): whether to verbose compiled log information. Default is False Returns: @@ -775,7 +798,7 @@ def load(name, _write_setup_file(name, sources, file_path, build_base_dir, extra_include_paths, extra_cxx_cflags, extra_cuda_cflags, extra_ldflags, verbose) - _jit_compile(file_path, interpreter, verbose) + _jit_compile(file_path, verbose) # import as callable python api custom_op_api = _import_module_from_library(name, build_base_dir, verbose) diff --git a/python/paddle/utils/cpp_extension/extension_utils.py b/python/paddle/utils/cpp_extension/extension_utils.py index 896293246a275..7d6bcc4d564c9 100644 --- a/python/paddle/utils/cpp_extension/extension_utils.py +++ b/python/paddle/utils/cpp_extension/extension_utils.py @@ -16,7 +16,9 @@ import re import six import sys +import json import glob +import hashlib import logging import collections import textwrap @@ -46,9 +48,10 @@ MSVC_LINK_FLAGS = ['/MACHINE:X64', 'paddle_custom_op.lib'] -COMMON_NVCC_FLAGS = ['-DPADDLE_WITH_CUDA', '-DEIGEN_USE_GPU', '-O3'] +COMMON_NVCC_FLAGS = ['-DPADDLE_WITH_CUDA', '-DEIGEN_USE_GPU'] GCC_MINI_VERSION = (5, 4, 0) +MSVC_MINI_VERSION = (19, 0, 24215) # Give warning if using wrong compiler WRONG_COMPILER_WARNING = ''' ************************************* @@ -62,7 +65,7 @@ use {paddle_compiler} to compile your custom op. Or you may compile Paddle from source using {user_compiler}, and then also use it compile your custom op. -See https://www.paddlepaddle.org.cn/install/quick?docurl=/documentation/docs/zh/2.0/install/compile/linux-compile.html +See https://www.paddlepaddle.org.cn/documentation/docs/zh/install/compile/fromsource.html for help with compiling Paddle from source. !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!! @@ -219,12 +222,112 @@ def last(self): return next(reversed(self.op_info_map.items())) +VersionFields = collections.namedtuple('VersionFields', [ + 'sources', + 'extra_compile_args', + 'extra_link_args', + 'library_dirs', + 'runtime_library_dirs', + 'include_dirs', + 'define_macros', + 'undef_macros', +]) + + +class VersionManager: + def __init__(self, version_field): + self.version_field = version_field + self.version = self.hasher(version_field) + + def hasher(self, version_field): + from paddle.fluid.layers.utils import flatten + + md5 = hashlib.md5() + for field in version_field._fields: + elem = getattr(version_field, field) + if not elem: continue + if isinstance(elem, (list, tuple, dict)): + flat_elem = flatten(elem) + md5 = combine_hash(md5, tuple(flat_elem)) + else: + raise RuntimeError( + "Support types with list, tuple and dict, but received {} with {}.". + format(type(elem), elem)) + + return md5.hexdigest() + + @property + def details(self): + return self.version_field._asdict() + + +def combine_hash(md5, value): + """ + Return new hash value. + DO NOT use `hash()` beacuse it doesn't generate stable value between different process. + See https://stackoverflow.com/questions/27522626/hash-function-in-python-3-3-returns-different-results-between-sessions + """ + md5.update(repr(value).encode()) + return md5 + + +def clean_object_if_change_cflags(so_path, extension): + """ + If already compiling source before, we should check whether cflags + have changed and delete the built object to re-compile the source + even though source file content keeps unchanaged. + """ + + def serialize(path, version_info): + assert isinstance(version_info, dict) + with open(path, 'w') as f: + f.write(json.dumps(version_info, indent=4, sort_keys=True)) + + def deserialize(path): + assert os.path.exists(path) + with open(path, 'r') as f: + content = f.read() + return json.loads(content) + + # version file + VERSION_FILE = "version.txt" + base_dir = os.path.dirname(so_path) + so_name = os.path.basename(so_path) + version_file = os.path.join(base_dir, VERSION_FILE) + + # version info + args = [getattr(extension, field, None) for field in VersionFields._fields] + version_field = VersionFields._make(args) + versioner = VersionManager(version_field) + + if os.path.exists(so_path) and os.path.exists(version_file): + old_version_info = deserialize(version_file) + so_version = old_version_info.get(so_name, None) + # delete shared library file if versison is changed to re-compile it. + if so_version is not None and so_version != versioner.version: + log_v( + "Re-Compiling {}, because specified cflags have been changed. New signature {} has been saved into {}.". + format(so_name, versioner.version, version_file)) + os.remove(so_path) + # upate new version information + new_version_info = versioner.details + new_version_info[so_name] = versioner.version + serialize(version_file, new_version_info) + else: + # If compile at first time, save compiling detail information for debug. + if not os.path.exists(base_dir): + os.makedirs(base_dir) + details = versioner.details + details[so_name] = versioner.version + serialize(version_file, details) + + def prepare_unix_cudaflags(cflags): """ Prepare all necessary compiled flags for nvcc compiling CUDA files. """ cflags = COMMON_NVCC_FLAGS + [ - '-ccbin', 'cc', '-Xcompiler', '-fPIC', '-w', '--expt-relaxed-constexpr', + '-ccbin', 'cc', '-Xcompiler', '-fPIC', '--expt-relaxed-constexpr', '-DNVCC' ] + cflags + get_cuda_arch_flags(cflags) @@ -295,8 +398,11 @@ def normalize_extension_kwargs(kwargs, use_cuda=False): extra_link_args.extend(['cudadevrt.lib', 'cudart_static.lib']) kwargs['extra_link_args'] = extra_link_args else: - # append compile flags - add_compile_flag(extra_compile_args, ['-g', '-w']) # disable warnings + add_compile_flag(extra_compile_args, ['-w']) # disable warning + # Note(Aurelius84): This marco will impact memory layout of `Tensor`. + # We align it automatially with pre-installed Paddle. + if core.is_compiled_with_mkldnn(): + add_compile_flag(extra_compile_args, ['-DPADDLE_WITH_MKLDNN']) # append link flags extra_link_args = kwargs.get('extra_link_args', []) @@ -336,7 +442,8 @@ def find_cuda_home(): [which_cmd, 'nvcc'], stderr=devnull) if six.PY3: nvcc_path = nvcc_path.decode() - nvcc_path = nvcc_path.rstrip('\r\n') + # Multi CUDA, select the first + nvcc_path = nvcc_path.split('\r\n')[0] # for example: /usr/local/cuda/bin/nvcc cuda_home = os.path.dirname(os.path.dirname(nvcc_path)) @@ -354,13 +461,40 @@ def find_cuda_home(): if cuda_home and not os.path.exists( cuda_home) and core.is_compiled_with_cuda(): cuda_home = None - warnings.warn( - "Not found CUDA runtime, please use `export CUDA_HOME= XXX` to specific it." - ) return cuda_home +def find_rocm_home(): + """ + Use heuristic method to find rocm path + """ + # step 1. find in $ROCM_HOME or $ROCM_PATH + rocm_home = os.environ.get('ROCM_HOME') or os.environ.get('ROCM_PATH') + + # step 2. find path by `which nvcc` + if rocm_home is None: + which_cmd = 'where' if IS_WINDOWS else 'which' + try: + with open(os.devnull, 'w') as devnull: + hipcc_path = subprocess.check_output( + [which_cmd, 'hipcc'], stderr=devnull) + if six.PY3: + hipcc_path = hipcc_path.decode() + hipcc_path = hipcc_path.rstrip('\r\n') + + # for example: /opt/rocm/bin/hipcc + rocm_home = os.path.dirname(os.path.dirname(hipcc_path)) + except: + rocm_home = "/opt/rocm" + # step 3. check whether path is valid + if rocm_home and not os.path.exists( + rocm_home) and core.is_compiled_with_rocm(): + rocm_home = None + + return rocm_home + + def find_cuda_includes(): """ Use heuristic method to find cuda include path @@ -374,6 +508,19 @@ def find_cuda_includes(): return [os.path.join(cuda_home, 'include')] +def find_rocm_includes(): + """ + Use heuristic method to find rocm include path + """ + rocm_home = find_rocm_home() + if rocm_home is None: + raise ValueError( + "Not found ROCM runtime, please use `export ROCM_PATH= XXX` to specific it." + ) + + return [os.path.join(rocm_home, 'include')] + + def find_paddle_includes(use_cuda=False): """ Return Paddle necessary include dir path. @@ -384,8 +531,12 @@ def find_paddle_includes(use_cuda=False): include_dirs = [paddle_include_dir, third_party_dir] if use_cuda: - cuda_include_dir = find_cuda_includes() - include_dirs.extend(cuda_include_dir) + if core.is_compiled_with_rocm(): + rocm_include_dir = find_rocm_includes() + include_dirs.extend(rocm_include_dir) + else: + cuda_include_dir = find_cuda_includes() + include_dirs.extend(cuda_include_dir) return include_dirs @@ -407,6 +558,20 @@ def find_cuda_libraries(): return cuda_lib_dir +def find_rocm_libraries(): + """ + Use heuristic method to find rocm dynamic lib path + """ + rocm_home = find_rocm_home() + if rocm_home is None: + raise ValueError( + "Not found ROCM runtime, please use `export ROCM_PATH=XXX` to specific it." + ) + rocm_lib_dir = [os.path.join(rocm_home, 'lib')] + + return rocm_lib_dir + + def find_paddle_libraries(use_cuda=False): """ Return Paddle necessary library dir path. @@ -415,8 +580,12 @@ def find_paddle_libraries(use_cuda=False): paddle_lib_dirs = [get_lib()] if use_cuda: - cuda_lib_dir = find_cuda_libraries() - paddle_lib_dirs.extend(cuda_lib_dir) + if core.is_compiled_with_rocm(): + rocm_lib_dir = find_rocm_libraries() + paddle_lib_dirs.extend(rocm_lib_dir) + else: + cuda_lib_dir = find_cuda_libraries() + paddle_lib_dirs.extend(cuda_lib_dir) return paddle_lib_dirs @@ -607,13 +776,18 @@ def _get_api_inputs_str(op_name): in_names, out_names, attr_names = parse_op_info(op_name) # e.g: x, y, z param_names = in_names + attr_names - params_str = ','.join([p.lower() for p in param_names]) + # NOTE(chenweihang): we add suffix `@VECTOR` for std::vector input, + # but the string contains `@` cannot used as argument name, so we split + # input name by `@`, and only use first substr as argument + params_str = ','.join([p.split("@")[0].lower() for p in param_names]) # e.g: {'X': x, 'Y': y, 'Z': z} - ins_str = "{%s}" % ','.join( - ["'{}' : {}".format(in_name, in_name.lower()) for in_name in in_names]) + ins_str = "{%s}" % ','.join([ + "'{}' : {}".format(in_name, in_name.split("@")[0].lower()) + for in_name in in_names + ]) # e.g: {'num': n} attrs_str = "{%s}" % ",".join([ - "'{}' : {}".format(attr_name, attr_name.lower()) + "'{}' : {}".format(attr_name, attr_name.split("@")[0].lower()) for attr_name in attr_names ]) # e.g: ['Out', 'Index'] @@ -685,24 +859,22 @@ def list2str(args): return repr(args) -def _jit_compile(file_path, interpreter=None, verbose=False): +def _jit_compile(file_path, verbose=False): """ Build shared library in subprocess """ ext_dir = os.path.dirname(file_path) setup_file = os.path.basename(file_path) - if interpreter is None: - interpreter = 'python' + # Using interpreter same with current process. + interpreter = sys.executable + try: - which = 'where' if IS_WINDOWS else 'which' - py_path = subprocess.check_output([which, interpreter]) py_version = subprocess.check_output([interpreter, '-V']) if six.PY3: - py_path = py_path.decode() py_version = py_version.decode() log_v("Using Python interpreter: {}, version: {}".format( - py_path.strip(), py_version.strip()), verbose) + interpreter, py_version.strip()), verbose) except Exception: _, error, _ = sys.exc_info() raise RuntimeError( @@ -775,13 +947,12 @@ def check_abi_compatibility(compiler, verbose=False): Check whether GCC version on user local machine is compatible with Paddle in site-packages. """ - # TODO(Aurelius84): After we support windows, remove IS_WINDOWS in following code. - if os.environ.get('PADDLE_SKIP_CHECK_ABI') in ['True', 'true', '1' - ] or IS_WINDOWS: + if os.environ.get('PADDLE_SKIP_CHECK_ABI') in ['True', 'true', '1']: return True + which = 'where' if IS_WINDOWS else 'which' cmd_out = subprocess.check_output( - ['which', compiler], stderr=subprocess.STDOUT) + [which, compiler], stderr=subprocess.STDOUT) compiler_path = os.path.realpath(cmd_out.decode() if six.PY3 else cmd_out).strip() # step 1. if not found any suitable compiler, raise error @@ -794,32 +965,44 @@ def check_abi_compatibility(compiler, verbose=False): platform=OS_NAME)) return False + version = (0, 0, 0) # clang++ have no ABI compatibility problem if OS_NAME.startswith('darwin'): return True try: if OS_NAME.startswith('linux'): + mini_required_version = GCC_MINI_VERSION version_info = subprocess.check_output( [compiler, '-dumpfullversion', '-dumpversion']) if six.PY3: version_info = version_info.decode() version = version_info.strip().split('.') - assert len(version) == 3 - # check version compatibility - if tuple(map(int, version)) >= GCC_MINI_VERSION: - return True - else: - warnings.warn( - ABI_INCOMPATIBILITY_WARNING.format( - user_compiler=compiler, version=version_info.strip())) elif IS_WINDOWS: - # TODO(zhouwei): support check abi compatibility on windows - warnings.warn("We don't support Windows now.") + mini_required_version = MSVC_MINI_VERSION + compiler_info = subprocess.check_output( + compiler, stderr=subprocess.STDOUT) + if six.PY3: + try: + compiler_info = compiler_info.decode('UTF-8') + except UnicodeDecodeError: + compiler_info = compiler_info.decode('gbk') + match = re.search(r'(\d+)\.(\d+)\.(\d+)', compiler_info.strip()) + if match is not None: + version = match.groups() except Exception: + # check compiler version failed _, error, _ = sys.exc_info() warnings.warn('Failed to check compiler version for {}: {}'.format( compiler, error)) + return False + # check version compatibility + assert len(version) == 3 + if tuple(map(int, version)) >= mini_required_version: + return True + warnings.warn( + ABI_INCOMPATIBILITY_WARNING.format( + user_compiler=compiler, version='.'.join(version))) return False @@ -827,8 +1010,12 @@ def _expected_compiler_current_platform(): """ Returns supported compiler string on current platform """ - expect_compilers = ['clang', 'clang++'] if OS_NAME.startswith( - 'darwin') else ['gcc', 'g++', 'gnu-c++', 'gnu-cc'] + if OS_NAME.startswith('darwin'): + expect_compilers = ['clang', 'clang++'] + elif OS_NAME.startswith('linux'): + expect_compilers = ['gcc', 'g++', 'gnu-c++', 'gnu-cc'] + elif IS_WINDOWS: + expect_compilers = ['cl'] return expect_compilers diff --git a/python/paddle/utils/download.py b/python/paddle/utils/download.py index 3af9a83f6a212..b7d7d0b5adb54 100644 --- a/python/paddle/utils/download.py +++ b/python/paddle/utils/download.py @@ -61,44 +61,6 @@ def __exit__(self, exc_type, exc_val, exc_tb): DOWNLOAD_RETRY_LIMIT = 3 -nlp_models = OrderedDict(( - ('RoBERTa-zh-base', - 'https://bert-models.bj.bcebos.com/chinese_roberta_wwm_ext_L-12_H-768_A-12.tar.gz' - ), - ('RoBERTa-zh-large', - 'https://bert-models.bj.bcebos.com/chinese_roberta_wwm_large_ext_L-24_H-1024_A-16.tar.gz' - ), - ('ERNIE-v2-en-base', - 'https://ernie.bj.bcebos.com/ERNIE_Base_en_stable-2.0.0.tar.gz'), - ('ERNIE-v2-en-large', - 'https://ernie.bj.bcebos.com/ERNIE_Large_en_stable-2.0.0.tar.gz'), - ('XLNet-cased-base', - 'https://xlnet.bj.bcebos.com/xlnet_cased_L-12_H-768_A-12.tgz'), - ('XLNet-cased-large', - 'https://xlnet.bj.bcebos.com/xlnet_cased_L-24_H-1024_A-16.tgz'), - ('ERNIE-v1-zh-base', - 'https://baidu-nlp.bj.bcebos.com/ERNIE_stable-1.0.1.tar.gz'), - ('ERNIE-v1-zh-base-max-len-512', - 'https://ernie.bj.bcebos.com/ERNIE_1.0_max-len-512.tar.gz'), - ('BERT-en-uncased-large-whole-word-masking', - 'https://bert-models.bj.bcebos.com/wwm_uncased_L-24_H-1024_A-16.tar.gz'), - ('BERT-en-cased-large-whole-word-masking', - 'https://bert-models.bj.bcebos.com/wwm_cased_L-24_H-1024_A-16.tar.gz'), - ('BERT-en-uncased-base', - 'https://bert-models.bj.bcebos.com/uncased_L-12_H-768_A-12.tar.gz'), - ('BERT-en-uncased-large', - 'https://bert-models.bj.bcebos.com/uncased_L-24_H-1024_A-16.tar.gz'), - ('BERT-en-cased-base', - 'https://bert-models.bj.bcebos.com/cased_L-12_H-768_A-12.tar.gz'), - ('BERT-en-cased-large', - 'https://bert-models.bj.bcebos.com/cased_L-24_H-1024_A-16.tar.gz'), - ('BERT-multilingual-uncased-base', - 'https://bert-models.bj.bcebos.com/multilingual_L-12_H-768_A-12.tar.gz'), - ('BERT-multilingual-cased-base', - 'https://bert-models.bj.bcebos.com/multi_cased_L-12_H-768_A-12.tar.gz'), - ('BERT-zh-base', - 'https://bert-models.bj.bcebos.com/chinese_L-12_H-768_A-12.tar.gz'), )) - def is_url(path): """ diff --git a/python/paddle/vision/ops.py b/python/paddle/vision/ops.py index 4b4e2088708bb..079aa086f2b3b 100644 --- a/python/paddle/vision/ops.py +++ b/python/paddle/vision/ops.py @@ -398,6 +398,7 @@ def deform_conv2d(x, stride=1, padding=0, dilation=1, + deformable_groups=1, groups=1, mask=None, name=None): @@ -462,6 +463,8 @@ def deform_conv2d(x, dilation (int|list|tuple, optional): The dilation size. If dilation is a tuple, it must contain two integers, (dilation_H, dilation_W). Otherwise, the dilation_H = dilation_W = dilation. Default: dilation = 1. + deformable_groups (int): The number of deformable group partitions. + Default: deformable_groups = 1. groups (int, optonal): The groups number of the deformable conv layer. According to grouped convolution in Alex Krizhevsky's Deep CNN paper: when group=2, the first half of the filters is only connected to the first half @@ -521,7 +524,8 @@ def deform_conv2d(x, if in_dygraph_mode(): attrs = ('strides', stride, 'paddings', padding, 'dilations', dilation, - 'groups', groups, 'im2col_step', 1) + 'deformable_groups', deformable_groups, 'groups', groups, + 'im2col_step', 1) if use_deform_conv2d_v1: op_type = 'deformable_conv_v1' pre_bias = getattr(core.ops, op_type)(x, offset, weight, *attrs) @@ -572,7 +576,7 @@ def deform_conv2d(x, 'paddings': padding, 'dilations': dilation, 'groups': groups, - 'deformable_groups': 1, + 'deformable_groups': deformable_groups, 'im2col_step': 1, } helper.append_op( @@ -649,6 +653,8 @@ class DeformConv2D(Layer): dilation(int|list|tuple, optional): The dilation size. If dilation is a tuple, it must contain three integers, (dilation_D, dilation_H, dilation_W). Otherwise, the dilation_D = dilation_H = dilation_W = dilation. The default value is 1. + deformable_groups (int): The number of deformable group partitions. + Default: deformable_groups = 1. groups(int, optional): The groups number of the Conv3D Layer. According to grouped convolution in Alex Krizhevsky's Deep CNN paper: when group=2, the first half of the filters is only connected to the first half @@ -726,6 +732,7 @@ def __init__(self, stride=1, padding=0, dilation=1, + deformable_groups=1, groups=1, weight_attr=None, bias_attr=None): @@ -733,6 +740,7 @@ def __init__(self, assert weight_attr is not False, "weight_attr should not be False in Conv." self._weight_attr = weight_attr self._bias_attr = bias_attr + self._deformable_groups = deformable_groups self._groups = groups self._in_channels = in_channels self._out_channels = out_channels @@ -770,6 +778,7 @@ def forward(self, x, offset, mask=None): stride=self._stride, padding=self._padding, dilation=self._dilation, + deformable_groups=self._deformable_groups, groups=self._groups, mask=mask) return out diff --git a/python/paddle/vision/transforms/transforms.py b/python/paddle/vision/transforms/transforms.py index 4101c41f2aa30..7d3d5f525c2c7 100644 --- a/python/paddle/vision/transforms/transforms.py +++ b/python/paddle/vision/transforms/transforms.py @@ -407,7 +407,8 @@ class RandomResizedCrop(BaseTransform): Args: size (int|list|tuple): Target size of output image, with (height, width) shape. - scale (list|tuple): Range of size of the origin size cropped. Default: (0.08, 1.0) + scale (list|tuple): Scale range of the cropped image before resizing, relatively to the origin + image. Default: (0.08, 1.0) ratio (list|tuple): Range of aspect ratio of the origin aspect ratio cropped. Default: (0.75, 1.33) interpolation (int|str, optional): Interpolation method. Default: 'bilinear'. when use pil backend, support method are as following: @@ -537,7 +538,7 @@ class RandomHorizontalFlip(BaseTransform): """Horizontally flip the input data randomly with a given probability. Args: - prob (float, optional): Probability of the input data being flipped. Default: 0.5 + prob (float, optional): Probability of the input data being flipped. Should be in [0, 1]. Default: 0.5 keys (list[str]|tuple[str], optional): Same as ``BaseTransform``. Default: None. Examples: @@ -548,7 +549,7 @@ class RandomHorizontalFlip(BaseTransform): from PIL import Image from paddle.vision.transforms import RandomHorizontalFlip - transform = RandomHorizontalFlip(224) + transform = RandomHorizontalFlip(0.5) fake_img = Image.fromarray((np.random.rand(300, 320, 3) * 255.).astype(np.uint8)) diff --git a/python/setup.py.in b/python/setup.py.in index 0e214c5c65fbe..69a8bc771aefb 100644 --- a/python/setup.py.in +++ b/python/setup.py.in @@ -179,6 +179,7 @@ packages=['paddle', 'paddle.fluid.contrib.utils', 'paddle.fluid.contrib.extend_optimizer', 'paddle.fluid.contrib.mixed_precision', + 'paddle.fluid.contrib.mixed_precision.bf16', 'paddle.fluid.contrib.layers', 'paddle.fluid.transpiler', 'paddle.fluid.transpiler.details', @@ -335,6 +336,17 @@ if '${WITH_XPU_BKCL}' == 'ON': shutil.copy('${XPU_BKCL_LIB}', libs_path) package_data['paddle.libs']+=['${XPU_BKCL_LIB_NAME}'] +# Only for lite xpu inference. +if '${WITH_XPU}' == 'OFF' and '${XPU_SDK_ROOT}' != '': + xpu_api_lib = os.path.join('${XPU_SDK_ROOT}', 'XTDK/shlib/', 'libxpuapi.so') + xpu_rt_lib = os.path.join('${XPU_SDK_ROOT}', 'XTDK/runtime/shlib/', 'libxpurt.so') + if os.path.exists(xpu_api_lib): + shutil.copy(xpu_api_lib, libs_path) + package_data['paddle.libs']+=['libxpuapi.so'] + if os.path.exists(xpu_rt_lib): + shutil.copy(xpu_rt_lib, libs_path) + package_data['paddle.libs']+=['libxpurt.so'] + ### Old custom op extension mechanism related, will be removed in 2.1.0 ### # copy libpaddle_framework.so to libs on linux if sys.platform.startswith('linux'): @@ -451,12 +463,27 @@ class InstallHeaders(Command): ('install_headers', 'install_dir'), ('force', 'force')) + def copy_data_type_headers(self, header): + if os.name == 'nt': + data_type_headers = ['platform\\complex64.h', 'platform\\complex128.h', 'platform\\float16.h'] + else: + data_type_headers = ['platform/complex64.h', 'platform/complex128.h', 'platform/float16.h'] + for dtype_header in data_type_headers: + if dtype_header in header: + install_dir = os.path.join(self.install_dir, "paddle/fluid/extension/include") + if not os.path.exists(install_dir): + self.mkpath(install_dir) + return self.copy_file(header, install_dir) + def mkdir_and_copy_file(self, header): if 'pb.h' in header: install_dir = re.sub('${PADDLE_BINARY_DIR}/', '', header) elif 'third_party' not in header: - # framework + # paddle headers install_dir = re.sub('@PADDLE_SOURCE_DIR@/', '', header) + # For paddle data type headers, we also need to copy to `extension/incude`, + # used for new custom operator + self.copy_data_type_headers(header) else: # third_party install_dir = re.sub('${THIRD_PARTY_PATH}', 'third_party', header) diff --git a/tools/check_api_approvals.sh b/tools/check_api_approvals.sh index 1db3f6d3d27ec..eb05468eda6ca 100644 --- a/tools/check_api_approvals.sh +++ b/tools/check_api_approvals.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + if [ -z ${BRANCH} ]; then BRANCH="develop" fi @@ -61,8 +75,8 @@ DEV_OP_USE_DEFAULT_GRAD_MAKER_SPEC=${PADDLE_ROOT}/paddle/fluid/op_use_default_gr PR_OP_USE_DEFAULT_GRAD_MAKER_SPEC=${PADDLE_ROOT}/paddle/fluid/op_use_default_grad_maker_PR.spec ADDED_OP_USE_DEFAULT_GRAD_MAKER=`python ${PADDLE_ROOT}/tools/diff_use_default_grad_op_maker.py ${DEV_OP_USE_DEFAULT_GRAD_MAKER_SPEC} ${PR_OP_USE_DEFAULT_GRAD_MAKER_SPEC}` if [ "${ADDED_OP_USE_DEFAULT_GRAD_MAKER}" != "" ]; then - echo_line="You must have one RD (sneaxiy (Recommend) or luotao1) approval because you use DefaultGradOpMaker for ${ADDED_OP_USE_DEFAULT_GRAD_MAKER}, which manages the grad_op memory optimization.\n" - check_approval 1 32832641 6836917 + echo_line="You must have one RD (zhiqiu (Recommend) or zhhsplendid) approval because you use DefaultGradOpMaker for ${ADDED_OP_USE_DEFAULT_GRAD_MAKER}, which manages the grad_op memory optimization.\n" + check_approval 1 6888866 7913861 fi if [ -n "${echo_list}" ];then diff --git a/tools/check_file_diff_approvals.sh b/tools/check_file_diff_approvals.sh index fd3175a5729da..f3bf3ea508ba7 100644 --- a/tools/check_file_diff_approvals.sh +++ b/tools/check_file_diff_approvals.sh @@ -92,11 +92,11 @@ for API_FILE in ${API_FILES[*]}; do # You can use http://caius.github.io/github_id/ to find Github user id. # approval_user_list: XiaoguangHu01 46782768,Xreki 12538138,luotao1 6836917,qingqing01 7845005,guoshengCS 14105589,heavengate 12605721,kuke 3064195,Superjomn 328693,lanxianghit 47554610,cyj1986 39645414,hutuxian 11195205,frankwhzhang 20274488,nepeplwu 45024560,Dianhai 38231817,chenwhql 22561442,zhiqiu 6888866,seiriosPlus 5442383,gongweibao 10721757,saxon-zh 2870059, zhouwei25 52485244, Aurelius84 9301846, liym27 33742067, zhhsplendid 7913861, kolinwei 22165420, liuwei1031 46661762, swtkiwi 27208573, juncaipeng 52520497, zhangting2020 26615455, Shixiaowei02 39303645, Heeenrrry 28379894,XieYunshen 32428676, Dong Daxiang 35550832, phlrain 43953930. if [ "${API_FILE}" == "CMakeLists.txt" ];then - echo_line="You must have one RD (luotao1 or XiaoguangHu01) approval for CMakeLists.txt, which manages the compilation parameter.\n" - check_approval 1 6836917 46782768 + echo_line="You must have one RD (wanghuancoder, luotao1 or XiaoguangHu01) approval for CMakeLists.txt, which manages the compilation parameter.\n" + check_approval 1 6836917 46782768 26922892 elif [ "${API_FILE}" == "python/paddle/fluid/__init__.py" ];then - echo_line="You must have one RD (lanxianghit (Recommend) or luotao1) approval for the python/paddle/fluid/init.py, which manages the environment variables.\n" - check_approval 1 6836917 47554610 + echo_line="You must have one RD (lanxianghit (Recommend), phlrain or luotao1) approval for the python/paddle/fluid/init.py, which manages the environment variables.\n" + check_approval 1 6836917 47554610 43953930 elif [ "${API_FILE}" == "python/requirements.txt" ];then echo_line="You must have one RD (phlrain) and one TPM (swtkiwi) and one QA (kolinwei) approval for python/requirements.txt, which manages the third-party python package.\n" check_approval 3 43953930 27208573 22165420 @@ -104,8 +104,8 @@ for API_FILE in ${API_FILES[*]}; do echo_line="You must have one RD (gongweibao or seiriosPlus) approval for the paddle/fluid/operators/distributed/send_recv.proto.in, which manages the environment variables.\n" check_approval 1 10721757 5442383 elif [ "${API_FILE}" == "paddle/fluid/framework/unused_var_check.cc" ];then - echo_line="You must have one RD (zhiqiu (Recommend) or luotao1) approval for the changes of paddle/fluid/framework/unused_var_check.cc, which manages the allow list of operators that have unused input variables. Before change the allow list, please read the specification [https://github.com/PaddlePaddle/Paddle/wiki/OP-Should-Not-Have-Unused-Input] and try to refine code first. \n" - check_approval 1 6888866 6836917 + echo_line="You must have one RD (zhiqiu (Recommend) or chenwhql) approval for the changes of paddle/fluid/framework/unused_var_check.cc, which manages the allow list of operators that have unused input variables. Before change the allow list, please read the specification [https://github.com/PaddlePaddle/Paddle/wiki/OP-Should-Not-Have-Unused-Input] and try to refine code first. \n" + check_approval 1 6888866 22561442 elif [ "${API_FILE}" == "paddle/fluid/pybind/op_function_generator.cc" ];then echo_line="You must have one RD (zhiqiu (Recommend) , phlrain) approval for the changes of paddle/fluid/pybind/op_function_generator.cc, which manages the logic of automatic generating op functions for dygraph. \n" check_approval 1 6888866 43953930 @@ -122,14 +122,14 @@ for API_FILE in ${API_FILES[*]}; do echo_line="You must have one RD (cryoco (Recommend), luotao1 or phlrain) approval for the python/paddle/fluid/tests/unittests/white_list/no_check_set_white_list.py, which manages the white list of setting no_check_set of check_output. \n" check_approval 1 12407750 6836917 43953930 elif [ "${API_FILE}" == "python/paddle/fluid/tests/unittests/white_list/check_op_sequence_instance_0_input_white_list.py" ]; then - echo_line="You must have one RD (luotao1, phlrain) approval for the ${API_FILE}, which manages the white list of instance size 0 input for sequence op test. For more information, please refer to [https://github.com/PaddlePaddle/Paddle/wiki/It-is-required-to-include-LoDTensor-input-with-instance_size=0-in-sequence-OP-test]. \n" - check_approval 1 6836917 43953930 + echo_line="You must have one RD (luotao1, lanxianghit, phlrain) approval for the ${API_FILE}, which manages the white list of instance size 0 input for sequence op test. For more information, please refer to [https://github.com/PaddlePaddle/Paddle/wiki/It-is-required-to-include-LoDTensor-input-with-instance_size=0-in-sequence-OP-test]. \n" + check_approval 1 6836917 43953930 47554610 elif [ "${API_FILE}" == "python/paddle/fluid/tests/unittests/white_list/op_threshold_white_list.py" ];then echo_line="It is an Op accuracy problem, please take care of it. You must have one RD (juncaipeng (Recommend), zhangting2020 or luotao1) approval for the python/paddle/fluid/tests/unittests/white_list/op_threshold_white_list.py, which manages the white list of error threshold for op test with float64 precision. For more information, please refer to: https://github.com/PaddlePaddle/Paddle/wiki/Upgrade-OP-Precision-to-Float64. \n" check_approval 1 52520497 26615455 6836917 elif [ "${API_FILE}" == "python/paddle/fluid/tests/unittests/white_list/check_op_sequence_batch_1_input_white_list.py" ];then - echo_line="You must have one RD (luotao1 or phlrain) approval for ${API_FILE}, which manages the white list of batch size 1 input for sequence op test. For more information, please refer to [https://github.com/PaddlePaddle/Paddle/wiki/It-is-required-to-include-LoDTensor-input-with-batch_size=1-in-sequence-OP-test]. \n" - check_approval 1 6836917 43953930 + echo_line="You must have one RD (luotao1, lanxianghit or phlrain) approval for ${API_FILE}, which manages the white list of batch size 1 input for sequence op test. For more information, please refer to [https://github.com/PaddlePaddle/Paddle/wiki/It-is-required-to-include-LoDTensor-input-with-batch_size=1-in-sequence-OP-test]. \n" + check_approval 1 6836917 43953930 47554610 elif [ "${API_FILE}" == "python/paddle/fluid/tests/unittests/white_list/no_grad_set_white_list.py" ];then echo_line="You must have one RD (Shixiaowei02 (Recommend), luotao1 or phlrain) approval for the python/paddle/fluid/tests/unittests/white_list/no_grad_set_white_list.py, which manages the white list of no_grad_set without value in operators. For more information, please refer to[https://github.com/PaddlePaddle/Paddle/wiki/It's-recommend-to-set-no_grad_set-to-be-None].\n" check_approval 1 39303645 6836917 43953930 @@ -143,17 +143,17 @@ for API_FILE in ${API_FILES[*]}; do echo_line="You must have (fuyinno4 (Recommend), raindrops2sea) approval for ${API_FILE} changes" check_approval 1 35824027 38231817 elif [ "${API_FILE}" == "paddle/scripts/paddle_build.bat" ] || [ "${API_FILE}" == "tools/windows/run_unittests.sh" ]; then - echo_line="You must have one RD (zhouwei25 (Recommend), luotao1) approval for ${API_FILE} changes, which manages the Paddle CI task on Windows.\n" - check_approval 1 52485244 6836917 + echo_line="You must have one RD (zhouwei25 (Recommend), wanghuancoder, luotao1) approval for ${API_FILE} changes, which manages the Paddle CI task on Windows.\n" + check_approval 1 52485244 6836917 26922892 elif [ "${API_FILE}" == "tools/parallel_UT_rule.py" ]; then - echo_line="You must have one RD (zhouwei25 (Recommend), luotao1) approval for ${API_FILE} changes, which manages the rule of running unittest with a same GPU. If the unittest failed due to Insufficient GPU memory or CUBLAS_STATUS_ALLOC_FAILED, you can remove it from ${API_FILE}.\n" - check_approval 1 52485244 6836917 + echo_line="You must have one RD (zhouwei25 (Recommend), wanghuancoder, luotao1) approval for ${API_FILE} changes, which manages the rule of running unittest with a same GPU. If the unittest failed due to Insufficient GPU memory or CUBLAS_STATUS_ALLOC_FAILED, you can remove it from ${API_FILE}.\n" + check_approval 1 52485244 6836917 26922892 elif [ "${API_FILE}" == "python/paddle/fluid/parallel_executor.py" ]; then echo_line="You must have one RD (Xreki,luotao1,zhhsplendid) approval for ${API_FILE}, which manages the underlying code for PaddlePaddle.\n" check_approval 1 12538138 6836917 7913861 else - echo_line="You must have one RD (XiaoguangHu01,Xreki,luotao1) approval for ${API_FILE}, which manages the underlying code for fluid.\n" - check_approval 1 46782768 12538138 6836917 + echo_line="You must have one RD (XiaoguangHu01,chenwhql,zhiqiu,Xreki,luotao1) approval for ${API_FILE}, which manages the underlying code for fluid.\n" + check_approval 1 46782768 12538138 6836917 22561442 6888866 fi fi done @@ -161,8 +161,8 @@ done FILTER=`git diff --name-only upstream/develop | grep -v "tools/"` HAS_CONST_CAST=`git diff -U0 upstream/$BRANCH $FILTER | grep '^\+' | grep -o -m 1 "const_cast" || true` if [ ${HAS_CONST_CAST} ] && [ "${GIT_PR_ID}" != "" ]; then - echo_line="You must have one RD (XiaoguangHu01,Xreki,luotao1) approval for the usage of const_cast.\n" - check_approval 1 46782768 12538138 6836917 + echo_line="You must have one RD (XiaoguangHu01,chenwhql,zhiqiu,Xreki,luotao1) approval for the usage of const_cast.\n" + check_approval 1 46782768 12538138 6836917 22561442 6888866 fi HAS_BOOST_GET=`git diff -U0 upstream/$BRANCH $FILTER |grep "^+" |grep -o -m 1 "boost::get" || true` @@ -185,14 +185,14 @@ fi HAS_UNITTEST_SKIP=`git diff -U0 upstream/$BRANCH | grep "^+[[:space:]]\{0,\}@unittest.skip" || true` if [ "${HAS_UNITTEST_SKIP}" != "" ] && [ "${GIT_PR_ID}" != "" ]; then - echo_line="Unittest is not allowed to be disabled.\nYou must have one RD (kolinwei(Recommend), or luotao1) approval for the usage of @unittest.skip or @unittest.skipIf.\n${HAS_UNITTEST_SKIP}\n" - check_approval 1 22165420 6836917 46661762 + echo_line="Unittest is not allowed to be disabled.\nYou must have one RD (kolinwei(Recommend), wanghuancoder or luotao1) approval for the usage of @unittest.skip or @unittest.skipIf.\n${HAS_UNITTEST_SKIP}\n" + check_approval 1 22165420 6836917 46661762 26922892 fi HAS_MODIFIED_DEMO_CMAKE=`git diff --name-only upstream/$BRANCH | grep "paddle/fluid/inference/api/demo_ci/CMakeLists.txt" || true` if [ "${HAS_MODIFIED_DEMO_CMAKE}" != "" ] && [ "${GIT_PR_ID}" != "" ]; then - echo_line="You must have one RD (Superjomn (Recommend), luotao1) approval for paddle/fluid/inference/api/demo_ci/CMakeLists.txt.\nwhich manages the compilation parameter of inference demo\n" - check_approval 1 328693 6836917 + echo_line="You must have one RD (Superjomn (Recommend), Shixiaowei02, luotao1) approval for paddle/fluid/inference/api/demo_ci/CMakeLists.txt.\nwhich manages the compilation parameter of inference demo\n" + check_approval 1 328693 6836917 39303645 fi ALL_PADDLE_ENFORCE=`git diff -U0 upstream/$BRANCH |grep "^+" |grep -zoE "PADDLE_ENFORCE\(.[^,\);]+.[^;]*\);\s" || true` diff --git a/tools/dockerfile/Dockerfile.rocm b/tools/dockerfile/Dockerfile.rocm index 6ae6b8963b7f5..5df66b9ea633a 100644 --- a/tools/dockerfile/Dockerfile.rocm +++ b/tools/dockerfile/Dockerfile.rocm @@ -1,16 +1,15 @@ # A image for building paddle binaries # Use rocm-terminal base image for both rocm environment # When you modify it, please be aware of rocm version -# -# Build: ROCM 3.9 +# +# Build: ROCM 4.0.1 # cd Paddle/tools/dockerfile # docker build -f Dockerfile.rocm \ -# --build-arg ROCM_VERSION=3.9 \ -# -t paddlepaddle/paddle-centos-rocm39-dev:latest . -# +# -t paddlepaddle/paddle-centos-rocm401-dev:latest . +# # docker run -it --device=/dev/kfd --device=/dev/dri \ # --security-opt seccomp=unconfined --group-add video \ -# paddlepaddle/paddle-centos-rocm39-dev:latest /bin/bash +# paddlepaddle/paddle-centos-rocm401-dev:latest /bin/bash FROM centos:7.8.2003 MAINTAINER PaddlePaddle Authors @@ -21,7 +20,8 @@ ENV LANGUAGE en_US.UTF-8 RUN yum install -y epel-release deltarpm sudo openssh-server gettext-devel sqlite-devel \ zlib-devel openssl-devel pcre-devel vim tk-devel tkinter libtool xz graphviz wget curl-devel \ - make bzip2 git patch unzip bison yasm diffutils automake which file kernel-headers kernel-devel + make bzip2 git patch unzip bison yasm diffutils automake which file kernel-headers kernel-devel \ + net-tools numactl-devel chrpath screen initscripts # Install devtoolset-7 RUN yum install -y yum-utils centos-release-scl && \ @@ -44,11 +44,10 @@ RUN wget -q https://cmake.org/files/v3.16/cmake-3.16.0-Linux-x86_64.tar.gz && \ ENV PATH=/opt/cmake-3.16/bin:${PATH} # ROCM -ARG ROCM_VERSION RUN yum install -y kmod wget openblas-devel epel-release RUN echo "[ROCm]" > /etc/yum.repos.d/rocm.repo && \ echo "name=ROCm" >> /etc/yum.repos.d/rocm.repo && \ - echo "baseurl=http://repo.radeon.com/rocm/yum/${ROCM_VERSION}" >> /etc/yum.repos.d/rocm.repo && \ + echo "baseurl=http://repo.radeon.com/rocm/yum/4.0.1" >> /etc/yum.repos.d/rocm.repo && \ echo "enabled=1" >> /etc/yum.repos.d/rocm.repo && \ echo "gpgcheck=0" >> /etc/yum.repos.d/rocm.repo RUN yum install -y rocm-dev rocm-utils rocfft miopen-hip rocblas hipsparse rocrand rccl hipcub rocthrust rocprofiler-dev roctracer-dev @@ -70,7 +69,7 @@ RUN cd /opt && wget -q https://paddle-ci.gz.bcebos.com/git-2.17.1.tar.gz && \ make -j8 && make install && \ cd .. && rm -rf git-2.17.1.tar.gz && rm -rf git-2.17.1 -ENV GOROOT=/usr/local/go +ENV GOROOT=/usr/local/go ENV GOPATH=/root/gopath ENV PATH=${GOROOT}/bin:${GOPATH}/bin:${PATH} @@ -82,37 +81,47 @@ RUN wget --no-check-certificate -qO- https://storage.googleapis.com/golang/go1.8 mkdir /root/gopath/src # protobuf 3.6.1 -RUN cd /opt && wget -q --no-check-certificate https://paddle-ci.cdn.bcebos.com/protobuf-cpp-3.6.1.tar.gz && \ +RUN cd /opt && wget -q --no-check-certificate https://paddle-ci.cdn.bcebos.com/protobuf-cpp-3.6.1.tar.gz && \ tar xzf protobuf-cpp-3.6.1.tar.gz && \ cd protobuf-3.6.1 && ./configure && make -j4 && make install && \ cd .. && rm -f protobuf-cpp-3.6.1.tar.gz && rm -rf protobuf-3.6.1 # conda -RUN cd /opt && wget https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86_64.sh && chmod +x Miniconda3-latest-Linux-x86_64.sh -RUN mkdir /opt/conda && ./Miniconda3-latest-Linux-x86_64.sh -b -f -p "/opt/conda" && rm -rf Miniconda3-latest-Linux-x86_64.sh +ENV CONDA_FILE=Miniconda3-py37_4.9.2-Linux-x86_64.sh +RUN cd /opt && wget https://repo.anaconda.com/miniconda/${CONDA_FILE} && chmod +x ${CONDA_FILE} +RUN mkdir /opt/conda && ./${CONDA_FILE} -b -f -p "/opt/conda" && rm -rf ${CONDA_FILE} ENV PATH=/opt/conda/bin:${PATH} -RUN conda init bash && \ - conda create -n python2.7 python=2.7 && \ - conda create -n python3.7 python=3.7 +RUN conda init bash && conda install -n base jupyter jupyterlab -# install paddle requirement +# install pylint and pre-commit +RUN /opt/conda/bin/pip install pre-commit pylint pytest astroid isort protocol PyGithub + +# install Paddle requirement RUN wget https://raw.githubusercontent.com/PaddlePaddle/Paddle/develop/python/requirements.txt -O /root/requirements.txt RUN /opt/conda/bin/pip install -r /root/requirements.txt && \ - /opt/conda/envs/python2.7/bin/pip install -r /root/requirements.txt && \ - /opt/conda/envs/python3.7/bin/pip install -r /root/requirements.txt && \ rm -rf /root/requirements.txt RUN wget https://raw.githubusercontent.com/PaddlePaddle/Paddle/develop/python/unittest_py/requirements.txt -O /root/requirements.txt -RUN /opt/conda/bin/pip install -r /root/requirements.txt && \ - /opt/conda/envs/python2.7/bin/pip install -r /root/requirements.txt && \ - /opt/conda/envs/python3.7/bin/pip install -r /root/requirements.txt && \ - rm -rf /root/requirements.txt +RUN /opt/conda/bin/pip install -r /root/requirements.txt && rm -rf /root/requirements.txt + +# install PaddleClas requirement +RUN wget https://raw.githubusercontent.com/PaddlePaddle/PaddleClas/develop/requirements.txt -O /root/requirements.txt +RUN /opt/conda/bin/pip install -r /root/requirements.txt && rm -rf /root/requirements.txt + +# install PaddleDetection requirement +RUN wget https://raw.githubusercontent.com/PaddlePaddle/PaddleDetection/develop/requirements.txt -O /root/requirements.txt +RUN /opt/conda/bin/pip install -r /root/requirements.txt && rm -rf /root/requirements.txt # configure ssh RUN sed -i "s/^#PermitRootLogin/PermitRootLogin/" /etc/ssh/sshd_config && \ sed -i "s/^#PubkeyAuthentication/PubkeyAuthentication/" /etc/ssh/sshd_config && \ sed -i "s/^#RSAAuthentication/RSAAuthentication/" /etc/ssh/sshd_config +# clang-format 3.8 +RUN wget https://copr.fedorainfracloud.org/coprs/alonid/llvm-3.8.0/repo/epel-7/alonid-llvm-3.8.0-epel-7.repo -P /etc/yum.repos.d/ +RUN yum install -y clang-3.8.0 +ENV PATH=/opt/llvm-3.8.0/bin:${PATH} + # patchelf RUN yum install -y patchelf && \ yum clean all && \ diff --git a/tools/get_pr_ut.py b/tools/get_pr_ut.py index 6b26ede908e48..58d7d2c0d6bc7 100644 --- a/tools/get_pr_ut.py +++ b/tools/get_pr_ut.py @@ -20,11 +20,15 @@ import time import subprocess import requests +import urllib.request +import ssl +import platform from github import Github PADDLE_ROOT = os.getenv('PADDLE_ROOT', '/paddle/') PADDLE_ROOT += '/' PADDLE_ROOT = PADDLE_ROOT.replace('//', '/') +ssl._create_default_https_context = ssl._create_unverified_context class PRChecker(object): @@ -74,7 +78,10 @@ def __wget_with_retry(self, url): if ix // 2 == 0: proxy = '' else: - proxy = '--no-proxy' + if platform.system() == 'Windows': + proxy = '-Y off' + else: + proxy = '--no-proxy' code = subprocess.call( 'wget -q {} --no-check-certificate {}'.format(proxy, url), shell=True) @@ -87,6 +94,33 @@ def __wget_with_retry(self, url): ix += 1 return False + def __urlretrieve(self, url, filename): + ix = 1 + with_proxy = urllib.request.getproxies() + without_proxy = {'http': '', 'http': ''} + while ix < 6: + if ix // 2 == 0: + cur_proxy = urllib.request.ProxyHandler(without_proxy) + else: + cur_proxy = urllib.request.ProxyHandler(with_proxy) + opener = urllib.request.build_opener(cur_proxy, + urllib.request.HTTPHandler) + urllib.request.install_opener(opener) + try: + urllib.request.urlretrieve(url, filename) + except Exception as e: + print(e) + print( + 'PREC download {} error, retry {} time(s) after {} secs.[proxy_option={}]'. + format(url, ix, ix * 10, proxy)) + continue + else: + return True + time.sleep(ix * 10) + ix += 1 + + return False + def get_pr_files(self): """ Get files in pull request. """ page = 0 @@ -201,16 +235,22 @@ def get_pr_ut(self): check_added_ut = False ut_list = [] file_ut_map = None - ret = self.__wget_with_retry( + ret = self.__urlretrieve( 'https://sys-p0.bj.bcebos.com/prec/file_ut.json{}'.format( - self.suffix)) + self.suffix), 'file_ut.json{}'.format(self.suffix)) if not ret: print('PREC download file_ut.json failed') exit(1) with open('file_ut.json' + self.suffix) as jsonfile: file_ut_map = json.load(jsonfile) for f in self.get_pr_files(): - if f not in file_ut_map: + current_system = platform.system() + if current_system == "Darwin" or current_system == "Windows": + f_judge = f.replace(PADDLE_ROOT, '/paddle/', 1) + f_judge = f_judge.replace('//', '/') + else: + f_judge = f + if f_judge not in file_ut_map: if f.endswith('.md'): ut_list.append('md_placeholder') elif f.endswith('.h') or f.endswith('.cu'): @@ -225,7 +265,8 @@ def get_pr_ut(self): '.cu'): if f.find('test_') != -1 or f.find('_test') != -1: print('PREC {} need check new ut'.format(f)) - check_added_ut = True + if current_system != "Windows": + check_added_ut = True elif self.is_only_comment(f): ut_list.append('nomap_comment_placeholder') else: @@ -240,7 +281,7 @@ def get_pr_ut(self): if self.is_only_comment(f): ut_list.append('map_comment_placeholder') else: - ut_list.extend(file_ut_map.get(f)) + ut_list.extend(file_ut_map.get(f_judge)) ut_list = list(set(ut_list)) if check_added_ut: @@ -250,9 +291,9 @@ def get_pr_ut(self): ut_list.append(ut.rstrip('\r\n')) if ut_list: - ret = self.__wget_with_retry( + ret = self.__urlretrieve( 'https://sys-p0.bj.bcebos.com/prec/prec_delta{}'.format( - self.suffix)) + self.suffix), 'prec_delta{}'.format(self.suffix)) if ret: with open('prec_delta' + self.suffix) as delta: for ut in delta: diff --git a/tools/parallel_UT_rule.py b/tools/parallel_UT_rule.py index a5239e534e2f5..3fb78b0d0a19a 100644 --- a/tools/parallel_UT_rule.py +++ b/tools/parallel_UT_rule.py @@ -219,6 +219,7 @@ 'test_full_op', 'test_framework_debug_str', 'test_fp16_utils', + 'test_bf16_utils', 'test_fleet_rolemaker_4', 'test_flags_use_mkldnn', 'test_filter_by_instag_op', diff --git a/tools/remove_grad_op_and_kernel.py b/tools/remove_grad_op_and_kernel.py new file mode 100644 index 0000000000000..85bbf8cdddc29 --- /dev/null +++ b/tools/remove_grad_op_and_kernel.py @@ -0,0 +1,177 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. +""" +This script simply removes all grad ops and kernels. You should use this script +when cmake ON_INFER=ON, which can greatly reduce the volume of the prediction library. +""" + +import os +import sys +import re +import glob + + +def find_type_files(cur_dir, file_type, file_list=[]): + next_level_dirs = os.listdir(cur_dir) + for next_level_name in next_level_dirs: + next_level_dir = os.path.join(cur_dir, next_level_name) + if os.path.isfile(next_level_dir): + if os.path.splitext(next_level_dir)[1] == file_type: + file_list.append(next_level_dir) + elif os.path.isdir(next_level_dir): + find_type_files(next_level_dir, file_type, file_list) + return file_list + + +def remove_grad_op_and_kernel(content, pattern1, pattern2): + res = [] + first_match = re.findall(pattern1, content, flags=re.DOTALL) + for match in first_match: + res.extend(re.findall(pattern2, match, flags=re.DOTALL)) + return res, len(res) + + +def update_operator_cmake(cmake_file): + pat1 = 'add_subdirectory(optimizers)' + pat2 = 'register_operators\(EXCLUDES.*?py_func_op.*?\)' + + code1 = 'if(ON_INFER)\nadd_subdirectory(optimizers)\nendif()' + code2 = 'if(ON_INFER)\nfile(GLOB LOSS_OPS RELATIVE "${CMAKE_CURRENT_SOURCE_DIR}" "*loss_op.cc")\nstring(REPLACE ".cc" "" LOSS_OPS "${LOSS_OPS}")\nendif()' + + with open(cmake_file, 'r') as f: + content = ''.join(f.readlines()) + content = content.replace(pat1, code1) + + match = re.findall(pat2, content, flags=re.DOTALL) + content = content.replace(match[0], code2 + '\n' + match[0].replace( + 'py_func_op', 'py_func_op ${LOSS_OPS}')) + + with open(cmake_file, 'w') as f: + f.write(content) + + +if __name__ == '__main__': + + tool_dir = os.path.dirname(os.path.abspath(__file__)) + + if sys.version_info[0] == 3: + all_op = glob.glob( + os.path.join(tool_dir, '../paddle/fluid/operators/**/*.cc'), + recursive=True) + all_op += glob.glob( + os.path.join(tool_dir, '../paddle/fluid/operators/**/*.cu'), + recursive=True) + elif sys.version_info[0] == 2: + all_op = find_type_files( + os.path.join(tool_dir, '../paddle/fluid/operators/'), '.cc') + all_op = find_type_files( + os.path.join(tool_dir, '../paddle/fluid/operators/'), '.cu', all_op) + + spec_ops = ['activation_op.cc'] + + register_op_count, register_op_cpu_kernel_count, register_op_cuda_kernel_count, register_op_xpu_kernel_count = 0, 0, 0, 0 + register_op_kernel_count, register_op_kernel_with_custom_type_count = 0, 0 + + # 1. remove all grad op and kernel + for op_file in all_op: + # remove all grad op + op_pattern1 = 'REGISTER_OPERATOR\(.*?\);?' + op_pattern2 = 'REGISTER_OPERATOR\(.*?_grad,.*?\);?' + + # remove all cpu grad kernel + cpu_kernel_pattern1 = 'REGISTER_OP_CPU_KERNEL\(.*?\);?' + cpu_kernel_pattern2 = 'REGISTER_OP_CPU_KERNEL\(.*?_grad,.*?\);?' + + # remove all gpu grad kernel + gpu_kernel_pattern1 = 'REGISTER_OP_CUDA_KERNEL\(.*?\);?' + gpu_kernel_pattern2 = 'REGISTER_OP_CUDA_KERNEL\(.*?_grad,.*?\);?' + + # remove all xpu grad kernel + xpu_kernel_pattern1 = 'REGISTER_OP_XPU_KERNEL\(.*?\);?' + xpu_kernel_pattern2 = 'REGISTER_OP_XPU_KERNEL\(.*?_grad,.*?\);?' + + # remove custom grad kernel, mkldnn or cudnn etc. + op_kernel_pattern1 = 'REGISTER_OP_KERNEL\(.*?\);?' + op_kernel_pattern2 = 'REGISTER_OP_KERNEL\(.*?_grad,.*?\);?' + + custom_pattern1 = 'REGISTER_OP_KERNEL_WITH_CUSTOM_TYPE\(.*?\);?' + custom_pattern2 = 'REGISTER_OP_KERNEL_WITH_CUSTOM_TYPE\(.*?_grad,.*?\);?' + + op_name = os.path.split(op_file)[1] + if op_name in spec_ops: + op_pattern1 = op_pattern1[:-1] + op_pattern2 = op_pattern2[:-1] + cpu_kernel_pattern1 = cpu_kernel_pattern1[:-1] + cpu_kernel_pattern2 = cpu_kernel_pattern2[:-1] + gpu_kernel_pattern1 = gpu_kernel_pattern1[:-1] + gpu_kernel_pattern2 = gpu_kernel_pattern2[:-1] + xpu_kernel_pattern1 = xpu_kernel_pattern1[:-1] + xpu_kernel_pattern2 = xpu_kernel_pattern2[:-1] + op_kernel_pattern1 = op_kernel_pattern1[:-1] + op_kernel_pattern2 = op_kernel_pattern2[:-1] + custom_pattern1 = custom_pattern1[:-1] + custom_pattern2 = custom_pattern2[:-1] + + all_matches = [] + with open(op_file, 'r') as f: + content = ''.join(f.readlines()) + + op, op_count = remove_grad_op_and_kernel(content, op_pattern1, + op_pattern2) + cpu_kernel, cpu_kernel_count = remove_grad_op_and_kernel( + content, cpu_kernel_pattern1, cpu_kernel_pattern2) + gpu_kernel, gpu_kernel_count = remove_grad_op_and_kernel( + content, gpu_kernel_pattern1, gpu_kernel_pattern2) + xpu_kernel, xpu_kernel_count = remove_grad_op_and_kernel( + content, xpu_kernel_pattern1, xpu_kernel_pattern2) + op_kernel, op_kernel_count = remove_grad_op_and_kernel( + content, op_kernel_pattern1, op_kernel_pattern2) + custom_kernel, custom_kernel_count = remove_grad_op_and_kernel( + content, custom_pattern1, custom_pattern2) + + register_op_count += op_count + register_op_cpu_kernel_count += cpu_kernel_count + register_op_cuda_kernel_count += gpu_kernel_count + register_op_xpu_kernel_count += xpu_kernel_count + register_op_kernel_count += op_kernel_count + register_op_kernel_with_custom_type_count += custom_kernel_count + + all_matches.extend(op) + all_matches.extend(cpu_kernel) + all_matches.extend(gpu_kernel) + all_matches.extend(xpu_kernel) + all_matches.extend(op_kernel) + all_matches.extend(custom_kernel) + + for i in all_matches: + content = content.replace(i, '') + + with open(op_file, 'w') as f: + f.write(content) + + # 2. update operators/CMakeLists.txt + cmake_file = os.path.join(tool_dir, + '../paddle/fluid/operators/CMakeLists.txt') + update_operator_cmake(cmake_file) + + print('We erase all grad op and kernel for Paddle-Inference lib.') + print('%50s%10s' % ('type', 'count')) + print('%50s%10s' % ('REGISTER_OPERATOR', register_op_count)) + print('%50s%10s' % ('REGISTER_OP_CPU_KERNEL', register_op_cpu_kernel_count)) + print('%50s%10s' % + ('REGISTER_OP_CUDA_KERNEL', register_op_cuda_kernel_count)) + print('%50s%10s' % ('REGISTER_OP_XPU_KERNEL', register_op_xpu_kernel_count)) + print('%50s%10s' % ('REGISTER_OP_KERNEL', register_op_kernel_count)) + print('%50s%10s' % ('REGISTER_OP_KERNEL_WITH_CUSTOM_TYPE', + register_op_kernel_with_custom_type_count)) diff --git a/tools/static_mode_white_list.py b/tools/static_mode_white_list.py index 872fd857381d0..6453eb48d7004 100644 --- a/tools/static_mode_white_list.py +++ b/tools/static_mode_white_list.py @@ -21,6 +21,7 @@ 'test_linear_chain_crf_op', 'test_lod_reset_op', 'test_lookup_table_op', + 'test_lookup_table_bf16_op', 'test_pad2d_op', 'test_scatter_op', 'test_sequence_concat', @@ -602,8 +603,10 @@ 'test_nearest_interp_mkldnn_op', 'test_bilinear_interp_mkldnn_op', 'test_fusion_gru_int8_mkldnn_op', + 'test_fusion_gru_bf16_mkldnn_op', 'test_fusion_gru_mkldnn_op', 'test_fusion_lstm_mkldnn_op', + 'test_fusion_lstm_bf16_mkldnn_op', 'test_gaussian_random_mkldnn_op', 'test_lrn_mkldnn_op', 'test_matmul_mkldnn_op', @@ -695,4 +698,6 @@ 'test_shape_op_xpu', 'test_slice_op_xpu', 'test_generate_proposals_v2_op', + 'test_lamb_op_xpu', + 'test_model_cast_to_bf16', ] diff --git a/tools/timeout_debug_help.sh b/tools/timeout_debug_help.sh new file mode 100644 index 0000000000000..45de2db87e853 --- /dev/null +++ b/tools/timeout_debug_help.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. +set +e +failed_uts=$1 +need_debug_ut_re='test_dist_fleet' +cat_log_judge=$(echo "${failed_uts}" | grep 'Timeout' | grep -oEi "$need_debug_ut_re" ) +if [[ "$cat_log_judge" != "" ]];then + echo "==============================================" + echo "show timeout ut logs" + echo "==============================================" + cat /tmp/tr0_err.log /tmp/tr1_err.log /tmp/ps0_err.log /tmp/ps1_err.log + cat /tmp/heter0_err.log /tmp/heter1_err.log +fi +set -e diff --git a/tools/windows/run_unittests.sh b/tools/windows/run_unittests.sh index e2a8e8b618379..dd4b21c80d910 100644 --- a/tools/windows/run_unittests.sh +++ b/tools/windows/run_unittests.sh @@ -15,8 +15,9 @@ set -e set +x NIGHTLY_MODE=$1 +PRECISION_TEST=$2 -PADDLE_ROOT="$(cd "$PWD/../" && pwd )" +export PADDLE_ROOT="$(cd "$PWD/../" && pwd )" if [ ${NIGHTLY_MODE:-OFF} == "ON" ]; then nightly_label="" else @@ -210,6 +211,36 @@ export CUDA_VISIBLE_DEVICES=0 UT_list=$(ctest -N | awk -F ': ' '{print $2}' | sed '/^$/d' | sed '$d') num=$(ctest -N | awk -F ': ' '{print $2}' | sed '/^$/d' | sed '$d' | wc -l) echo "Windows 1 card TestCases count is $num" +if [ ${PRECISION_TEST:-OFF} == "ON" ]; then + python ${PADDLE_ROOT}/tools/get_pr_ut.py + if [[ -f "ut_list" ]]; then + set +x + echo "PREC length: "`wc -l ut_list` + precision_cases=`cat ut_list` + set -x + fi +fi + +set +e +if [ ${PRECISION_TEST:-OFF} == "ON" ] && [[ "$precision_cases" != "" ]];then + UT_list_prec='' + re=$(cat ut_list|awk -F ' ' '{print }' | awk 'BEGIN{ all_str=""}{if (all_str==""){all_str=$1}else{all_str=all_str"$|^"$1}} END{print "^"all_str"$"}') + for case in $UT_list; do + flag=$(echo $case|grep -oE $re) + if [ -n "$flag" ];then + if [ -z "$UT_list_prec" ];then + UT_list_prec=$case + else + UT_list_prec=$UT_list_prec'\n'$case + fi + else + echo $case "won't run in PRECISION_TEST mode." + fi + done + UT_list=$UT_list_prec +fi +set -e + output=$(python ${PADDLE_ROOT}/tools/parallel_UT_rule.py "${UT_list}") eight_parallel_job=$(echo $output | cut -d ";" -f 1) tetrad_parallel_jog=$(echo $output | cut -d ";" -f 2) @@ -221,6 +252,7 @@ non_parallel_job_2=$(echo $non_parallel_job | cut -d "," -f 2) failed_test_lists='' tmp_dir=`mktemp -d` function collect_failed_tests() { + set +e for file in `ls $tmp_dir`; do grep -q 'The following tests FAILED:' $tmp_dir/$file exit_code=$? @@ -232,6 +264,7 @@ function collect_failed_tests() { ${failuretest}" fi done + set -e } function run_unittest() { @@ -247,10 +280,90 @@ function run_unittest() { echo "************************************************************************" export CUDA_VISIBLE_DEVICES=0 tmpfile=$tmp_dir/$RANDOM - (ctest -R "$test_case" -E "$disable_ut_quickly|$diable_wingpu_test|$long_time_test" -LE "${nightly_label}" --output-on-failure -C Release -j $parallel_job --repeat until-pass:4 after-timeout:4 | tee $tmpfile ) & + (ctest -R "$test_case" -E "$disable_ut_quickly|$diable_wingpu_test|$long_time_test" -LE "${nightly_label}" --output-on-failure -C Release -j $parallel_job | tee $tmpfile ) & wait; } +function unittests_retry(){ + parallel_job=1 + is_retry_execuate=0 + wintest_error=1 + retry_time=3 + exec_times=0 + exec_retry_threshold=10 + retry_unittests=$(echo "${failed_test_lists}" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) + need_retry_ut_counts=$(echo "$retry_unittests" |awk -F ' ' '{print }'| sed '/^$/d' | wc -l) + retry_unittests_regular=$(echo "$retry_unittests" |awk -F ' ' '{print }' | awk 'BEGIN{ all_str=""}{if (all_str==""){all_str=$1}else{all_str=all_str"$|^"$1}} END{print "^"all_str"$"}') + tmpfile=$tmp_dir/$RANDOM + + if [ $need_retry_ut_counts -lt $exec_retry_threshold ];then + retry_unittests_record='' + while ( [ $exec_times -lt $retry_time ] ) + do + retry_unittests_record="$retry_unittests_record$failed_test_lists" + if ( [[ "$exec_times" == "0" ]] );then + cur_order='first' + elif ( [[ "$exec_times" == "1" ]] );then + cur_order='second' + if [[ "$failed_test_lists" == "" ]]; then + break + else + retry_unittests=$(echo "${failed_test_lists}" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) + retry_unittests_regular=$(echo "$retry_unittests" |awk -F ' ' '{print }' | awk 'BEGIN{ all_str=""}{if (all_str==""){all_str=$1}else{all_str=all_str"$|^"$1}} END{print "^"all_str"$"}') + fi + elif ( [[ "$exec_times" == "2" ]] );then + cur_order='third' + fi + echo "=========================================" + echo "This is the ${cur_order} time to re-run" + echo "=========================================" + echo "The following unittest will be re-run:" + echo "${retry_unittests}" + echo "=========================================" + rm -f $tmp_dir/* + failed_test_lists='' + (ctest -R "($retry_unittests_regular)" --output-on-failure -C Release -j $parallel_job| tee $tmpfile ) & + wait; + collect_failed_tests + exec_times=$(echo $exec_times | awk '{print $0+1}') + done + else + # There are more than 10 failed unit tests, so no unit test retry + is_retry_execuate=1 + fi + rm -f $tmp_dir/* +} + +function show_ut_retry_result() { + if [[ "$is_retry_execuate" != "0" ]];then + failed_test_lists_ult=`echo "${failed_test_lists}" | grep -Po '[^ ].*$'` + echo "=========================================" + echo "There are more than 10 failed unit tests, so no unit test retry!!!" + echo "=========================================" + echo "${failed_test_lists_ult}" + exit 8; + else + retry_unittests_ut_name=$(echo "$retry_unittests_record" | grep -oEi "\-.+\(" | sed 's/(//' | sed 's/- //' ) + retry_unittests_record_judge=$(echo ${retry_unittests_ut_name}| tr ' ' '\n' | sort | uniq -c | awk '{if ($1 >=3) {print $2}}') + if [ -z "${retry_unittests_record_judge}" ];then + echo "========================================" + echo "There are failed tests, which have been successful after re-run:" + echo "========================================" + echo "The following tests have been re-ran:" + echo "${retry_unittests_record}" + else + failed_ut_re=$(echo "${retry_unittests_record_judge}" | awk 'BEGIN{ all_str=""}{if (all_str==""){all_str=$1}else{all_str=all_str"|"$1}} END{print all_str}') + echo "========================================" + echo "There are failed tests, which have been executed re-run,but success rate is less than 50%:" + echo "Summary Failed Tests... " + echo "========================================" + echo "The following tests FAILED: " + echo "${retry_unittests_record}" | grep -E "$failed_ut_re" + exit 8; + fi + fi +} + set +e run_unittest $eight_parallel_job 8 run_unittest $tetrad_parallel_jog 4 @@ -260,10 +373,6 @@ collect_failed_tests set -e rm -f $tmp_dir/* if [[ "$failed_test_lists" != "" ]]; then - echo "========================================" - echo "Summary Failed Tests... " - echo "========================================" - echo "The following tests FAILED: " - echo "${failed_test_lists}" - exit 8 + unittests_retry + show_ut_retry_result fi diff --git a/tools/wlist.json b/tools/wlist.json index e8ec83b49db82..cd9f2a7ca661e 100644 --- a/tools/wlist.json +++ b/tools/wlist.json @@ -286,9 +286,7 @@ "BilinearTensorProduct", "GroupNorm", "SpectralNorm", - "TreeConv" - ], - "wlist_temp":[ + "TreeConv", "prroi_pool", "ChunkEvaluator", "EditDistance", From 1181212b4f23a5bfdc455c7fb7d20c8ff41292ea Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 25 Mar 2021 08:07:58 +0000 Subject: [PATCH 070/106] fix threadpool.h head --- paddle/fluid/distributed/service/CMakeLists.txt | 4 ++-- paddle/fluid/distributed/service/graph_brpc_client.h | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/service/CMakeLists.txt b/paddle/fluid/distributed/service/CMakeLists.txt index b211e474d00a9..b888de998f98e 100644 --- a/paddle/fluid/distributed/service/CMakeLists.txt +++ b/paddle/fluid/distributed/service/CMakeLists.txt @@ -73,8 +73,8 @@ set_source_files_properties(graph_brpc_server.cc PROPERTIES COMPILE_FLAGS ${DIST set_source_files_properties(graph_brpc_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) -cc_library(downpour_server SRCS graph_brpc_server.cc brpc_ps_server.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) -cc_library(downpour_client SRCS graph_brpc_client.cc brpc_ps_client.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) +cc_library(downpour_server SRCS graph_brpc_server.cc brpc_ps_server.cc DEPS boost eigen3 table brpc_utils simple_threadpool ${RPC_DEPS}) +cc_library(downpour_client SRCS graph_brpc_client.cc brpc_ps_client.cc DEPS boost eigen3 table brpc_utils simple_threadpool ${RPC_DEPS}) # cc_library(downpour_server1 SRCS graph_brpc_server.cc DEPS downpour_server ${RPC_DEPS}) # cc_library(downpour_client1 SRCS graph_brpc_client.cc DEPS downpour_client ${RPC_DEPS}) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 60bbe42d32ccd..12f10234430a0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -19,6 +19,7 @@ #include #include +#include "ThreadPool.h" #include "brpc/channel.h" #include "brpc/controller.h" #include "brpc/server.h" From ffd07a20fc24002df344aa6b2a3dab3837c0af00 Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Fri, 26 Mar 2021 11:24:44 +0800 Subject: [PATCH 071/106] fix --- .../distributed/service/graph_brpc_client.h | 1 + .../distributed/service/graph_py_service.cc | 1 + .../distributed/table/common_graph_table.cc | 19 ++++++++++++++++++- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index b4850c75259ae..fcaf8fb0c9c8a 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include "brpc/channel.h" diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 5c8ab2d02138b..00972f250be2d 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -246,6 +246,7 @@ void GraphPyClient::load_edge_file(std::string name, std::string filepath, params += ">"; } if (this->table_id_map.count(name)) { + VLOG(0) << "loadding data with type " << name << " from " << filepath; uint32_t table_id = this->table_id_map[name]; auto status = get_ps_client()->load(table_id, std::string(filepath), params); diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 5ab5d7e2f1f23..c8c04ae26de2a 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -118,10 +118,13 @@ int32_t GraphTable::get_nodes_ids_by_ranges( int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { auto paths = paddle::string::split_string(path, ";"); + int64_t count = 0; + int64_t valid_count = 0; for (auto path : paths) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { + count ++; auto values = paddle::string::split_string(line, "\t"); if (values.size() < 2) continue; auto id = std::stoull(values[1]); @@ -133,6 +136,10 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { continue; } + if (count % 1000000 == 0) { + VLOG(0) << count << " nodes are loaded from filepath"; + } + std::string nt = values[0]; if (nt != node_type) { continue; @@ -152,8 +159,12 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { VLOG(4) << "Node feature: " << values[slice] << " not in feature_map."; } } + valid_count ++; } } + + VLOG(0) << valid_count << "/" << count << " nodes in type " << + node_type << " are loaded successfully in " << path; return 0; } @@ -162,6 +173,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { int count = 0; std::string sample_type = "random"; bool is_weighted = false; + int valid_count = 0; for (auto path : paths) { std::ifstream file(path); @@ -189,13 +201,18 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { << ", please check id distribution"; continue; } + if (count % 1000000 == 0) { + VLOG(0) << count << " edges are loaded from filepath"; + } size_t index = src_shard_id - shard_start; shards[index].add_graph_node(src_id)->build_edges(is_weighted); shards[index].add_neighboor(src_id, dst_id, weight); + valid_count ++; } } - VLOG(0) << "Load Finished Total Edge Count " << count; + VLOG(0) << valid_count << "/" << count << " edges in type " << + node_type << " are loaded successfully in " << path; // Build Sampler j From 1d87155c1ceb2f6c470f4cf3519af3e1f60017cc Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Fri, 26 Mar 2021 11:37:17 +0800 Subject: [PATCH 072/106] fix typo --- paddle/fluid/distributed/table/common_graph_table.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index c8c04ae26de2a..6b73aef2458c6 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -211,8 +211,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { valid_count ++; } } - VLOG(0) << valid_count << "/" << count << " edges in type " << - node_type << " are loaded successfully in " << path; + VLOG(0) << valid_count << "/" << count << " edges are loaded successfully in " << path; // Build Sampler j From 49e21548867a669b31361c3d49a8db3579f37515 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sat, 27 Mar 2021 02:57:46 +0000 Subject: [PATCH 073/106] resolve conflict --- .../distributed/table/common_graph_table.cc | 24 +------------------ 1 file changed, 1 insertion(+), 23 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 49400ebb3d1fe..63ce6a25ccd26 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -361,27 +361,6 @@ int32_t GraphTable::get_node_feat(const std::vector &node_ids, return 0; } -if (node == nullptr) { - return 0; -} -for (int feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { - const std::string &feature_name = feature_names[feat_idx]; - if (feat_id_map.find(feature_name) != feat_id_map.end()) { - // res[feat_idx][idx] = - // node->get_feature(feat_id_map[feature_name]); - auto feat = node->get_feature(feat_id_map[feature_name]); - res[feat_idx][idx] = feat; - } -} -return 0; -})); -} -for (size_t idx = 0; idx < node_num; ++idx) { - tasks[idx].get(); -} -return 0; -} - std::pair GraphTable::parse_feature( std::string feat_str) { // Return (feat_id, btyes) if name are in this->feat_name, else return (-1, @@ -507,5 +486,4 @@ int32_t GraphTable::initialize() { return 0; } } -} -; +}; From 0f2e0b12384e982ccffc5dfc21404e8b91910942 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sat, 27 Mar 2021 03:26:38 +0000 Subject: [PATCH 074/106] fix conflict --- paddle/fluid/pybind/pybind.cc | 8 -------- 1 file changed, 8 deletions(-) diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 0381d54737fe1..264ccc4b1a1c0 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -142,14 +142,6 @@ bool IsCompiledWithROCM() { #endif } -bool IsCompiledWithROCM() { -#ifndef PADDLE_WITH_HIP - return false; -#else - return true; -#endif -} - bool IsCompiledWithXPU() { #ifndef PADDLE_WITH_XPU return false; From 2c3515a39eda68bf21fa4331750dd854e2d8c855 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sat, 27 Mar 2021 14:51:19 +0000 Subject: [PATCH 075/106] recover lost content --- .../fluid/distributed/service/CMakeLists.txt | 50 --- paddle/fluid/distributed/table/table.cc | 2 - paddle/fluid/distributed/test/CMakeLists.txt | 21 -- paddle/fluid/operators/activation_op.cu | 284 +----------------- paddle/fluid/pybind/CMakeLists.txt | 5 + paddle/fluid/pybind/pybind.cc | 88 ++++-- 6 files changed, 64 insertions(+), 386 deletions(-) diff --git a/paddle/fluid/distributed/service/CMakeLists.txt b/paddle/fluid/distributed/service/CMakeLists.txt index b888de998f98e..843dea9eea6ef 100644 --- a/paddle/fluid/distributed/service/CMakeLists.txt +++ b/paddle/fluid/distributed/service/CMakeLists.txt @@ -1,48 +1,3 @@ -# set(BRPC_SRCS ps_client.cc server.cc) -# set_source_files_properties(${BRPC_SRCS}) - -# set(BRPC_DEPS brpc ssl crypto protobuf gflags glog zlib leveldb snappy gflags glog device_context) - -# brpc_library(sendrecv_rpc SRCS -# ${BRPC_SRCS} -# PROTO sendrecv.proto -# DEPS ${BRPC_DEPS} ) - -# set_property(GLOBAL PROPERTY RPC_DEPS sendrecv_rpc ${BRPC_DEPS} string_helper) - -# get_property(RPC_DEPS GLOBAL PROPERTY RPC_DEPS) - -# set_source_files_properties(communicator.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(service.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(brpc_ps_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(brpc_ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) - -# set_source_files_properties(brpc_utils.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(heter_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(heter_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) - -# set_source_files_properties(client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(ps_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(graph_brpc_server.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# set_source_files_properties(graph_brpc_client.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) - -# cc_library(downpour_server SRCS brpc_ps_server.cc graph_brpc_server.cc DEPS boost eigen3 table ${RPC_DEPS}) -# cc_library(downpour_client SRCS brpc_ps_client.cc graph_brpc_client.cc DEPS boost eigen3 table ${RPC_DEPS}) - -# cc_library(downpour_server SRCS brpc_ps_server.cc DEPS boost eigen3 table ${RPC_DEPS}) -# cc_library(downpour_client SRCS brpc_ps_client.cc DEPS boost eigen3 table ${RPC_DEPS}) - -# cc_library(client SRCS ps_client.cc DEPS downpour_client boost ${RPC_DEPS}) -# cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) - -# cc_library(communicator SRCS communicator.cc DEPS scope client boost table math_function selected_rows_functor ${RPC_DEPS}) -# cc_library(ps_service SRCS service.cc DEPS communicator client server boost ${RPC_DEPS}) - -# cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEPS} ${RPC_DEPS}) -# cc_library(heter_server SRCS heter_server.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) -# cc_library(heter_client SRCS heter_client.cc DEPS brpc_utils ${COMMON_DEPS} ${RPC_DEPS}) - set(BRPC_SRCS ps_client.cc server.cc) set_source_files_properties(${BRPC_SRCS}) @@ -76,11 +31,6 @@ cc_library(brpc_utils SRCS brpc_utils.cc DEPS tensor device_context ${COMMON_DEP cc_library(downpour_server SRCS graph_brpc_server.cc brpc_ps_server.cc DEPS boost eigen3 table brpc_utils simple_threadpool ${RPC_DEPS}) cc_library(downpour_client SRCS graph_brpc_client.cc brpc_ps_client.cc DEPS boost eigen3 table brpc_utils simple_threadpool ${RPC_DEPS}) -# cc_library(downpour_server1 SRCS graph_brpc_server.cc DEPS downpour_server ${RPC_DEPS}) -# cc_library(downpour_client1 SRCS graph_brpc_client.cc DEPS downpour_client ${RPC_DEPS}) -#cc_library(downpour_server SRCS brpc_ps_server.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) -#cc_library(downpour_client SRCS brpc_ps_client.cc DEPS boost eigen3 table brpc_utils ${RPC_DEPS}) - cc_library(client SRCS ps_client.cc DEPS downpour_client boost ${RPC_DEPS}) cc_library(server SRCS server.cc DEPS downpour_server boost ${RPC_DEPS}) diff --git a/paddle/fluid/distributed/table/table.cc b/paddle/fluid/distributed/table/table.cc index 4ec4d9faaa4a7..600be954cb596 100644 --- a/paddle/fluid/distributed/table/table.cc +++ b/paddle/fluid/distributed/table/table.cc @@ -14,8 +14,6 @@ #include "paddle/fluid/distributed/table/table.h" -#include -#include #include "glog/logging.h" #include "paddle/fluid/distributed/common/registerer.h" diff --git a/paddle/fluid/distributed/test/CMakeLists.txt b/paddle/fluid/distributed/test/CMakeLists.txt index 69cb3bfc36d5f..b756c740ac764 100644 --- a/paddle/fluid/distributed/test/CMakeLists.txt +++ b/paddle/fluid/distributed/test/CMakeLists.txt @@ -1,24 +1,3 @@ -# set_source_files_properties(table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(table_test SRCS table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) - -# set_source_files_properties(dense_table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(dense_table_test SRCS dense_table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) - -# set_source_files_properties(barrier_table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(barrier_table_test SRCS barrier_table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) - -# set_source_files_properties(brpc_service_dense_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(brpc_service_dense_sgd_test SRCS brpc_service_dense_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) - -# set_source_files_properties(brpc_service_sparse_sgd_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(brpc_service_sparse_sgd_test SRCS brpc_service_sparse_sgd_test.cc DEPS scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) - -# set_source_files_properties(brpc_utils_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(brpc_utils_test SRCS brpc_utils_test.cc DEPS brpc_utils scope math_function ${COMMON_DEPS} ${RPC_DEPS}) - -# set_source_files_properties(graph_node_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -# cc_test(graph_node_test SRCS graph_node_test.cc DEPS graph_node scope server client communicator ps_service boost table ps_framework_proto ${COMMON_DEPS}) - set_source_files_properties(table_test.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_test(table_test SRCS table_test.cc DEPS common_table table tensor_accessor ps_framework_proto ${COMMON_DEPS}) diff --git a/paddle/fluid/operators/activation_op.cu b/paddle/fluid/operators/activation_op.cu index 29498da0f026f..2033081af224a 100644 --- a/paddle/fluid/operators/activation_op.cu +++ b/paddle/fluid/operators/activation_op.cu @@ -10,276 +10,8 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "paddle/fluid/operators/activation_op.h" -#include "paddle/fluid/operators/math/math_cuda_utils.h" -#include "paddle/fluid/platform/cuda_device_function.h" #include "paddle/fluid/platform/float16.h" -namespace paddle { -namespace operators { - -using Tensor = framework::Tensor; -using float16 = paddle::platform::float16; - -template -struct CudaVecType { - using type = T; - static constexpr int vecsize = 1; -}; - -template <> -struct CudaVecType { - using type = __half2; - static constexpr int vecsize = 2; -}; - -template <> -struct CudaVecType { - using type = float4; - static constexpr int vecsize = 4; -}; - -template -class BaseGPUFunctor { - public: - using ELEMENT_TYPE = T; -}; - -/* ========================================================================== */ - -/* =========================== relu forward ============================ */ -template -class ReluGPUFuctor : public BaseGPUFunctor { - private: - T zero_; - - public: - ReluGPUFuctor() { zero_ = static_cast(0.0f); } - - // for relu forward when T is double - __device__ __forceinline__ typename CudaVecType::type Compute( - const typename CudaVecType::type* x); - - // when num % vecsize != 0 this func will be used - __device__ __forceinline__ T ComputeRemainder(const T x) { - return x > zero_ ? x : zero_; - } -}; - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGPUFuctor::Compute(const CudaVecType::type* x) { -// relu forward : out = max(x, 0) -#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 - return __ldg(x) > zero_ ? __ldg(x) : zero_; -#else - return (*x) > zero_ ? (*x) : zero_; -#endif -} - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGPUFuctor::Compute(const CudaVecType::type* xx) { - // relu forward : out = max(xx, 0) - return make_float4((xx->x > zero_) * (xx->x), (xx->y > zero_) * (xx->y), - (xx->z > zero_) * (xx->z), (xx->w > zero_) * (xx->w)); -} - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGPUFuctor::Compute(const CudaVecType::type* in) { -// relu forward : out = max(in, 0) -#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 - const half2 kzero = __float2half2_rn(0.0f); - return __hmul2(__hgt2(__ldg(in), kzero), __ldg(in)); -#else - const float2 xx = __half22float2(*in); - return __floats2half2_rn((xx.x > 0.0f) * static_cast(xx.x), - (xx.y > 0.0f) * static_cast(xx.y)); -#endif -} -/* ========================================================================== */ - -/* =========================== relu backward ============================ - */ - -template -class ReluGradGPUFunctor : public BaseGPUFunctor { - private: - T zero_; - - public: - ReluGradGPUFunctor() { zero_ = static_cast(0.0f); } - - // for relu backward when T is double - __device__ __forceinline__ typename CudaVecType::type Compute( - const typename CudaVecType::type* out, - const typename CudaVecType::type* dout); - - // when num % vecsize != 0 this func will be used - __device__ __forceinline__ T ComputeRemainder(const T out, const T dout) { - // relu backward : dx = out > 0 ? dout : 0; - return out > zero_ ? dout : zero_; - } - - static constexpr ActBwdOpFwdDeps FwdDeps() { return kDepOut; } -}; - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGradGPUFunctor::Compute(const CudaVecType::type* out, - const CudaVecType::type* dout) { -// relu backward : dx = out > 0 ? dout : 0; -#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 - return __ldg(out) > zero_ ? __ldg(dout) : zero_; -#else - return (*out) > zero_ ? (*dout) : zero_; -#endif -} - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGradGPUFunctor::Compute(const CudaVecType::type* out, - const CudaVecType::type* dout) { - // relu backward : dx = out > 0 ? dout : 0; - return make_float4((out->x > zero_) * (dout->x), (out->y > zero_) * (dout->y), - (out->z > zero_) * (dout->z), - (out->w > zero_) * (dout->w)); -} - -template <> -__device__ __forceinline__ CudaVecType::type -ReluGradGPUFunctor::Compute(const CudaVecType::type* out, - const CudaVecType::type* dout) { -// relu backward : dx = out > 0 ? dout : 0; -#ifdef __HIPCC__ || __CUDA_ARCH__ >= 350 || CUDA_VERSION >= 300 - const half2 kzero = __float2half2_rn(0.0f); - return __hmul2(__hgt2(__ldg(out), kzero), __ldg(dout)); -#else - const float2 xx = __half22float2(*out); - const float2 yy = __half22float2(*dout); - return __floats2half2_rn((xx.x > 0.0f) * static_cast(yy.x), - (xx.y > 0.0f) * static_cast(yy.y)); -#endif -} - -/* ========================================================================== */ - -template -__global__ void ActivationGradKernelVec(const T* forward_data, const T* dout, - T* dx, int num, Functor functor) { - using VecType = typename CudaVecType::type; - constexpr int vecsize = CudaVecType::vecsize; - int idx = threadIdx.x + blockIdx.x * blockDim.x; - int stride = blockDim.x * gridDim.x; - int loop = num / vecsize; - int tail = num % vecsize; - const VecType* in_forward = reinterpret_cast(forward_data); - const VecType* in_dout = reinterpret_cast(dout); - VecType* out = reinterpret_cast(dx); - - for (int i = idx; i < loop; i += stride) { - out[i] = functor.Compute((in_forward + i), (in_dout + i)); - } - - while (idx == loop && tail) { - dx[num - tail] = - functor.ComputeRemainder(forward_data[num - tail], dout[num - tail]); - --tail; - } -} - -template -__global__ void ActivationkernelVec(const T* src, T* dst, int num, - Functor functor) { - constexpr int vecsize = CudaVecType::vecsize; - using VecType = typename CudaVecType::type; - int idx = threadIdx.x + blockIdx.x * blockDim.x; - int stride = blockDim.x * gridDim.x; - int loop = num / vecsize; - int tail = num % vecsize; - const VecType* in = reinterpret_cast(src); - VecType* out = reinterpret_cast(dst); - - for (int i = idx; i < loop; i += stride) { - out[i] = functor.Compute((in + i)); - } - - while (idx == loop && tail) { - dst[num - tail] = functor.ComputeRemainder(src[num - tail]); - --tail; - } -} - -template -class ActivationGPUKernel - : public framework::OpKernel { - public: - using T = typename Functor::ELEMENT_TYPE; - void Compute(const framework::ExecutionContext& context) const override { - const framework::Tensor* in_x = nullptr; - framework::Tensor* out = nullptr; - ExtractActivationTensor(context, &in_x, &out); - auto& dev_ctx = context.template device_context(); - - int num = in_x->numel(); - const T* input_data = in_x->data(); - T* output_data = out->mutable_data(dev_ctx.GetPlace(), - static_cast(num * sizeof(T))); - - int block = 512; -#ifdef __HIPCC__ - block = 256; -#endif - Functor functor; - constexpr int vecsize = CudaVecType::vecsize; - int grid = max((num / vecsize + block - 1) / block, 1); - ActivationkernelVec<<>>(input_data, output_data, - num, functor); - } -}; - -template -class ActivationGradGPUKernel - : public framework::OpKernel { - public: - using T = typename Functor::ELEMENT_TYPE; - void Compute(const framework::ExecutionContext& context) const override { - const framework::Tensor *x, *out, *d_out; - framework::Tensor* d_x = nullptr; - x = out = d_out = nullptr; - ExtractActivationGradTensor(context, &x, &out, &d_out, - &d_x); - int numel = d_out->numel(); - auto& dev_ctx = context.template device_context(); - auto* dx_data = d_x->mutable_data( - dev_ctx.GetPlace(), static_cast(numel * sizeof(T))); - auto* dout_data = d_out->data(); - - auto* forward_data = dout_data; - if (static_cast(Functor::FwdDeps()) == static_cast(kDepOut)) { - // Only need forward output Out - forward_data = out->data(); - } else if (static_cast(Functor::FwdDeps()) == - static_cast(kDepX)) { - // Only need forward input X - forward_data = x->data(); - } - - int block = 512; -#ifdef __HIPCC__ - block = 256; -#endif - Functor functor; - constexpr int vecsize = CudaVecType::vecsize; - int grid = max((numel / vecsize + block - 1) / block, 1); - ActivationGradKernelVec<<>>( - forward_data, dout_data, dx_data, numel, functor); - } -}; - -} // namespace operators -} // namespace paddle - namespace ops = paddle::operators; namespace plat = paddle::platform; @@ -328,21 +60,7 @@ REGISTER_OP_CUDA_KERNEL( /* ========================================================================== */ /* =========================== relu register ============================ */ -REGISTER_OP_CUDA_KERNEL( - relu, ops::ActivationGPUKernel>, - ops::ActivationGPUKernel>, - ops::ActivationGPUKernel>); - -REGISTER_OP_CUDA_KERNEL( - relu_grad, ops::ActivationGradGPUKernel>, - ops::ActivationGradGPUKernel>, - ops::ActivationGradGPUKernel>); +REGISTER_ACTIVATION_CUDA_KERNEL(relu, Relu, ReluCUDAFunctor, ReluGradFunctor); REGISTER_OP_CUDA_KERNEL( relu_grad_grad, diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index baee3291da8ef..45e0594ceabfc 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -17,6 +17,11 @@ if (WITH_NCCL OR WITH_RCCL) set(PYBIND_DEPS ${PYBIND_DEPS} reducer) endif() +if (WITH_XPU_BKCL) + set(PYBIND_DEPS ${PYBIND_DEPS} reducer) + set(PYBIND_DEPS ${PYBIND_DEPS} bkcl_context) +endif() + if(NOT WIN32) set(PYBIND_DEPS ${PYBIND_DEPS} data_loader) set(PYBIND_DEPS ${PYBIND_DEPS} mmap_allocator) diff --git a/paddle/fluid/pybind/pybind.cc b/paddle/fluid/pybind/pybind.cc index 264ccc4b1a1c0..1f8427fb47d9d 100644 --- a/paddle/fluid/pybind/pybind.cc +++ b/paddle/fluid/pybind/pybind.cc @@ -24,6 +24,7 @@ limitations under the License. */ #include #include +#include "paddle/fluid/framework/custom_operator.h" #include "paddle/fluid/framework/data_layout.h" #include "paddle/fluid/framework/executor.h" #include "paddle/fluid/framework/feed_fetch_method.h" @@ -57,6 +58,7 @@ limitations under the License. */ #include "paddle/fluid/operators/py_func_op.h" #include "paddle/fluid/platform/cpu_helper.h" #include "paddle/fluid/platform/cpu_info.h" +#include "paddle/fluid/platform/device_context.h" #include "paddle/fluid/platform/dynload/dynamic_loader.h" #include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/init.h" @@ -169,6 +171,17 @@ bool SupportsBfloat16() { #endif } +bool SupportsBfloat16FastPerformance() { +#ifndef PADDLE_WITH_MKLDNN + return false; +#else + if (platform::MayIUse(platform::cpu_isa_t::avx512_bf16)) + return true; + else + return false; +#endif +} + bool IsCompiledWithBrpc() { #ifndef PADDLE_WITH_DISTRIBUTE return false; @@ -394,7 +407,7 @@ PYBIND11_MODULE(core_noavx, m) { PyCapsule_GetPointer(dltensor->ptr(), "dltensor")); PyCapsule_SetName(dltensor->ptr(), "used_dltensor"); DLTensor dl = dmt->dl_tensor; - Tensor tensor; + framework::Tensor tensor; if (dl.ctx.device_type == kDLCPU) { paddle::framework::TensorFromDLPack(dl, &tensor); @@ -532,77 +545,80 @@ PYBIND11_MODULE(core_noavx, m) { BindImperative(&m); - py::class_(m, "Tensor", py::buffer_protocol()) - .def("__array__", [](Tensor &self) { return TensorToPyArray(self); }) + py::class_(m, "Tensor", py::buffer_protocol()) + .def("__array__", + [](framework::Tensor &self) { return TensorToPyArray(self); }) .def("_is_initialized", - [](const Tensor &self) { return self.IsInitialized(); }) + [](const framework::Tensor &self) { return self.IsInitialized(); }) .def("_get_dims", - [](const Tensor &self) { return vectorize(self.dims()); }) + [](const framework::Tensor &self) { return vectorize(self.dims()); }) .def("_set_dims", - [](Tensor &self, const std::vector &dim) { + [](framework::Tensor &self, const std::vector &dim) { self.Resize(make_ddim(dim)); }) .def("_set_layout", - [](Tensor &self, const std::string &layout) { + [](framework::Tensor &self, const std::string &layout) { self.set_layout(StringToDataLayout(layout)); }) .def("_alloc_float", - [](Tensor &self, paddle::platform::CUDAPlace &place) { + [](framework::Tensor &self, paddle::platform::CUDAPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](Tensor &self, paddle::platform::XPUPlace &place) { + [](framework::Tensor &self, paddle::platform::XPUPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](Tensor &self, paddle::platform::CPUPlace &place) { + [](framework::Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_double", - [](Tensor &self, paddle::platform::CPUPlace &place) { + [](framework::Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](Tensor &self, paddle::platform::CPUPlace &place) { + [](framework::Tensor &self, paddle::platform::CPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](Tensor &self, paddle::platform::XPUPlace &place) { + [](framework::Tensor &self, paddle::platform::XPUPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](Tensor &self, paddle::platform::CUDAPlace &place) { + [](framework::Tensor &self, paddle::platform::CUDAPlace &place) { self.mutable_data(place); }) .def("_alloc_int", - [](Tensor &self, paddle::platform::CUDAPinnedPlace &place) { + [](framework::Tensor &self, + paddle::platform::CUDAPinnedPlace &place) { self.mutable_data(place); }) .def("_alloc_float", - [](Tensor &self, paddle::platform::CUDAPinnedPlace &place) { + [](framework::Tensor &self, + paddle::platform::CUDAPinnedPlace &place) { self.mutable_data(place); }) .def("_mutable_data", - [](Tensor &self, paddle::platform::CPUPlace &place, + [](framework::Tensor &self, paddle::platform::CPUPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](Tensor &self, paddle::platform::XPUPlace &place, + [](framework::Tensor &self, paddle::platform::XPUPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](Tensor &self, paddle::platform::CUDAPlace &place, + [](framework::Tensor &self, paddle::platform::CUDAPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) .def("_mutable_data", - [](Tensor &self, paddle::platform::CUDAPinnedPlace &place, + [](framework::Tensor &self, paddle::platform::CUDAPinnedPlace &place, paddle::framework::proto::VarType::Type type) { return reinterpret_cast(self.mutable_data(place, type)); }) - .def("_clear", &Tensor::clear) + .def("_clear", &framework::Tensor::clear) .def("set", SetTensorFromPyArray, py::arg("array"), py::arg("place"), py::arg("zero_copy") = false) .def("set", SetTensorFromPyArray, @@ -634,7 +650,9 @@ PYBIND11_MODULE(core_noavx, m) { t.set(np.ndarray([5, 30]), fluid.CPUPlace()) )DOC") - .def("shape", [](Tensor &self) { return vectorize(self.dims()); }, R"DOC( + .def("shape", + [](framework::Tensor &self) { return vectorize(self.dims()); }, + R"DOC( Return the shape of LoDTensor. Returns: @@ -652,7 +670,7 @@ PYBIND11_MODULE(core_noavx, m) { print(t.shape()) # [5, 30] )DOC") .def("_to_dlpack", - [](Tensor &self) { + [](framework::Tensor &self) { DLPackTensor dlpack_tensor(self, 1); DLManagedTensor *dmt = dlpack_tensor.ToCudfCompatibleDLManagedTensor(); @@ -677,20 +695,22 @@ PYBIND11_MODULE(core_noavx, m) { .def("_get_float_element", TensorGetElement) .def("_set_double_element", TensorSetElement) .def("_get_double_element", TensorGetElement) - .def("_place", [](Tensor &self) { return self.place(); }) - .def("_dtype", [](Tensor &self) { return self.type(); }) + .def("_place", [](framework::Tensor &self) { return self.place(); }) + .def("_dtype", [](framework::Tensor &self) { return self.type(); }) .def("_layout", - [](Tensor &self) { return DataLayoutToString(self.layout()); }) - .def("_share_data_with", &Tensor::ShareDataWith) + [](framework::Tensor &self) { + return DataLayoutToString(self.layout()); + }) + .def("_share_data_with", &framework::Tensor::ShareDataWith) .def("__getitem__", PySliceTensor, py::return_value_policy::reference) - .def("__str__", [](const Tensor &self) { + .def("__str__", [](const framework::Tensor &self) { std::stringstream ostr; ostr << self; return ostr.str(); }); // TODO(cql): add reference: en_user_guide_lod_tensor - py::class_(m, "LoDTensor", R"DOC( + py::class_(m, "LoDTensor", R"DOC( LoDTensor is a Tensor with optional LoD (Level of Details) information, it can be used for variable-length sequences, see :ref:`user_guide_lod_tensor` for details. @@ -774,7 +794,8 @@ PYBIND11_MODULE(core_noavx, m) { t = fluid.LoDTensor() )DOC") - .def("__array__", [](Tensor &self) { return TensorToPyArray(self); }) + .def("__array__", + [](framework::Tensor &self) { return TensorToPyArray(self); }) .def("__init__", [](LoDTensor &instance, const std::vector> &recursive_sequence_lengths) { @@ -1732,6 +1753,8 @@ All parameter, weight, gradient are variables in Paddle. m.def("init_gflags", framework::InitGflags); m.def("init_glog", framework::InitGLOG); m.def("load_op_library", framework::LoadOpLib); + m.def("load_op_meta_info_and_register_op", + framework::LoadOpMetaInfoAndRegisterOp); m.def("init_devices", []() { framework::InitDevices(); }); m.def("is_compiled_with_cuda", IsCompiledWithCUDA); @@ -1739,6 +1762,7 @@ All parameter, weight, gradient are variables in Paddle. m.def("is_compiled_with_xpu", IsCompiledWithXPU); m.def("is_compiled_with_mkldnn", IsCompiledWithMKLDNN); m.def("supports_bfloat16", SupportsBfloat16); + m.def("supports_bfloat16_fast_performance", SupportsBfloat16FastPerformance); m.def("is_compiled_with_brpc", IsCompiledWithBrpc); m.def("is_compiled_with_dist", IsCompiledWithDIST); m.def("_cuda_synchronize", [](const platform::CUDAPlace &place) { @@ -1960,6 +1984,10 @@ All parameter, weight, gradient are variables in Paddle. m.def("nvprof_init", platform::CudaProfilerInit); m.def("nvprof_start", platform::CudaProfilerStart); m.def("nvprof_stop", platform::CudaProfilerStop); + m.def("nvprof_nvtx_push", platform::CudaNvtxRangePush); + m.def("nvprof_nvtx_pop", platform::CudaNvtxRangePop); + m.def("nvprof_enable_record_event", platform::NvprofEnableRecordEvent); + m.def("nvprof_disable_record_event", platform::NvprofDisableRecordEvent); #endif #endif From 8826468545ea98cf031bc8de8fc56332c5d9f30c Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Sun, 28 Mar 2021 09:26:48 +0800 Subject: [PATCH 076/106] fix pybind of FeatureNode --- paddle/fluid/pybind/fleet_py.cc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index 97949c48ce908..774cae8cb2561 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -166,7 +166,10 @@ void BindGraphNode(py::module* m) { .def("get_feature", &GraphNode::get_feature); } void BindGraphPyFeatureNode(py::module* m) { - py::class_(*m, "FeatureNode").def(py::init<>()); + py::class_(*m, "FeatureNode") + .def(py::init<>()) + .def("get_id", &GraphNode::get_id) + .def("get_feature", &GraphNode::get_feature); } void BindGraphPyService(py::module* m) { From 5b2393b88bb4fa0921adae6e2df4b6ab0f873832 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sun, 28 Mar 2021 02:21:37 +0000 Subject: [PATCH 077/106] recover cmake --- cmake/external/pslib_brpc.cmake | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cmake/external/pslib_brpc.cmake b/cmake/external/pslib_brpc.cmake index e5a85a47a317b..7b00474a65070 100644 --- a/cmake/external/pslib_brpc.cmake +++ b/cmake/external/pslib_brpc.cmake @@ -41,8 +41,7 @@ FILE(WRITE ${PSLIB_BRPC_DOWNLOAD_DIR}/CMakeLists.txt "cmake_minimum_required(VERSION 3.0)\n" "install(DIRECTORY ${PSLIB_BRPC_NAME}/include ${PSLIB_BRPC_NAME}/lib \n" " DESTINATION ${PSLIB_BRPC_DST_DIR})\n") -MESSAGE("DOWNLAODING PATH FOR BRPC ---------------",${PSLIB_BRPC_DOWNLOAD_DIR}) -MESSAGE("PREFIX----------",${PSLIB_BRPC_SOURCE_DIR}) + ExternalProject_Add( ${PSLIB_BRPC_PROJECT} ${EXTERNAL_PROJECT_LOG_ARGS} From bdd9404a583dbb0452134cad61548e1c90c5cf15 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sun, 28 Mar 2021 02:26:02 +0000 Subject: [PATCH 078/106] recover tools --- tools/check_api_approvals.sh | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/tools/check_api_approvals.sh b/tools/check_api_approvals.sh index eb05468eda6ca..4e8ea25715451 100644 --- a/tools/check_api_approvals.sh +++ b/tools/check_api_approvals.sh @@ -1,19 +1,5 @@ #!/bin/bash -# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - if [ -z ${BRANCH} ]; then BRANCH="develop" fi From 6535661f7ddd4976f8bf18441ca00d4ad28aaa9d Mon Sep 17 00:00:00 2001 From: seemingwang Date: Mon, 29 Mar 2021 03:18:23 +0000 Subject: [PATCH 079/106] resolve conflict --- .../fluid/inference/api/analysis_predictor.cc | 2 +- .../inference/tensorrt/convert/CMakeLists.txt | 1 + .../tensorrt/convert/affine_channel_op.cc | 94 +++++++++++++++++++ paddle/fluid/inference/tensorrt/op_teller.cc | 10 +- 4 files changed, 105 insertions(+), 2 deletions(-) create mode 100644 paddle/fluid/inference/tensorrt/convert/affine_channel_op.cc diff --git a/paddle/fluid/inference/api/analysis_predictor.cc b/paddle/fluid/inference/api/analysis_predictor.cc index 7789e9d75f4e1..9613955c6b50b 100644 --- a/paddle/fluid/inference/api/analysis_predictor.cc +++ b/paddle/fluid/inference/api/analysis_predictor.cc @@ -1192,7 +1192,7 @@ USE_TRT_CONVERTER(scale); USE_TRT_CONVERTER(stack); USE_TRT_CONVERTER(clip); USE_TRT_CONVERTER(gather); - +USE_TRT_CONVERTER(affine_channel); USE_TRT_CONVERTER(multiclass_nms); USE_TRT_CONVERTER(nearest_interp); diff --git a/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt b/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt index be7fa0548d9f3..6af76bd11cd59 100644 --- a/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt +++ b/paddle/fluid/inference/tensorrt/convert/CMakeLists.txt @@ -6,6 +6,7 @@ nv_library(tensorrt_converter shuffle_channel_op.cc swish_op.cc instance_norm_op.cc stack_op.cc transpose_op.cc flatten_op.cc emb_eltwise_layernorm.cc skip_layernorm.cc scale_op.cc slice_op.cc hard_sigmoid_op.cc hard_swish_op.cc clip_op.cc gather_op.cc + affine_channel_op.cc multiclass_nms_op.cc nearest_interp_op.cc DEPS tensorrt_engine tensorrt_plugin operator scope framework_proto op_registry) diff --git a/paddle/fluid/inference/tensorrt/convert/affine_channel_op.cc b/paddle/fluid/inference/tensorrt/convert/affine_channel_op.cc new file mode 100644 index 0000000000000..813342c08483b --- /dev/null +++ b/paddle/fluid/inference/tensorrt/convert/affine_channel_op.cc @@ -0,0 +1,94 @@ +/* Copyright (c) 2018 PaddlePaddle Authors. All Rights Reserved. + +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 "paddle/fluid/framework/data_layout.h" +#include "paddle/fluid/inference/tensorrt/convert/op_converter.h" + +namespace paddle { +namespace framework { +class Scope; + +namespace proto { +class OpDesc; +} // namespace proto +} // namespace framework +} // namespace paddle + +namespace paddle { +namespace inference { +namespace tensorrt { + +/* + * Affine Channel Op + */ +class AffineChannelOpConverter : public OpConverter { + public: + void operator()(const framework::proto::OpDesc& op, + const framework::Scope& scope, bool test_mode) override { + VLOG(3) << "convert a fluid affine_channel op to tensorrt scale nd layer"; + + framework::OpDesc op_desc(op, nullptr); + std::string input_name = op_desc.Input("X").front(); + std::string scale_name = op_desc.Input("Scale").front(); + std::string bias_name = op_desc.Input("Bias").front(); + std::string output_name = op_desc.Output("Out").front(); + + auto input_tensor = engine_->GetITensor(input_name); + auto idim = input_tensor->getDimensions(); + + auto* scale_v = scope.FindVar(scale_name); + auto* scale_t = scale_v->GetMutable(); + float* scale_ptr = engine_->GetWeightCPUData(scale_name, scale_t, false); + + auto* bias_v = scope.FindVar(bias_name); + auto* bias_t = bias_v->GetMutable(); + float* bias_ptr = engine_->GetWeightCPUData(bias_name, bias_t, false); + + auto data_layout = framework::StringToDataLayout( + BOOST_GET_CONST(std::string, op_desc.GetAttr("data_layout"))); + + PADDLE_ENFORCE_EQ( + data_layout, framework::DataLayout::kNCHW, + platform::errors::InvalidArgument( + "TensorRT affine channel converter can only convert NCHW format. " + "Other format should be run in fluid mode. Report a bug on github " + "issue if you see this line.")); + + // tensorrt scalend layer only support spatial dims >= 2, + // so nhwc is not availabe (spatial dims == 0) + const int channel_axis = engine_->with_dynamic_shape(); + + TensorRTEngine::Weight scale_weights{nvinfer1::DataType::kFLOAT, + static_cast(scale_ptr), + (size_t)idim.d[channel_axis]}; + TensorRTEngine::Weight bias_weights{nvinfer1::DataType::kFLOAT, + static_cast(bias_ptr), + (size_t)idim.d[channel_axis]}; + TensorRTEngine::Weight power_weights{nvinfer1::DataType::kFLOAT, nullptr, + 0}; + + auto layer = TRT_ENGINE_ADD_LAYER(engine_, ScaleNd, *input_tensor, + nvinfer1::ScaleMode::kCHANNEL, + bias_weights.get(), scale_weights.get(), + power_weights.get(), channel_axis); + + RreplenishLayerAndOutput(layer, "affine_channel", {output_name}, test_mode); + } +}; + +} // namespace tensorrt +} // namespace inference +} // namespace paddle + +REGISTER_TRT_OP_CONVERTER(affine_channel, AffineChannelOpConverter); diff --git a/paddle/fluid/inference/tensorrt/op_teller.cc b/paddle/fluid/inference/tensorrt/op_teller.cc index 82f58254fe8e0..eb429405d18ae 100644 --- a/paddle/fluid/inference/tensorrt/op_teller.cc +++ b/paddle/fluid/inference/tensorrt/op_teller.cc @@ -111,6 +111,7 @@ struct SimpleOpTypeSetTeller : public Teller { "flatten2", "flatten", "gather", + "affine_channel", "multiclass_nms", "nearest_interp", }; @@ -196,6 +197,13 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, if (!with_dynamic_shape || desc.Input("Axis").size() > 0) return false; } + if (op_type == "affine_channel") { + if (!desc.HasAttr("data_layout")) return false; + auto data_layout = framework::StringToDataLayout( + BOOST_GET_CONST(std::string, desc.GetAttr("data_layout"))); + if (data_layout != framework::DataLayout::kNCHW) return false; + } + if (op_type == "multiclass_nms") { if (with_dynamic_shape) return false; auto* block = desc.Block(); @@ -238,6 +246,7 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, return false; } } + if (op_type == "nearest_interp") { std::vector attrs{"data_layout", "interp_method", "align_corners", "scale", @@ -254,7 +263,6 @@ bool OpTeller::Tell(const framework::ir::Node* node, bool use_no_calib_int8, BOOST_GET_CONST(std::string, desc.GetAttr("interp_method")); if (interp_method != "nearest") return false; } - if ((*teller)(op_type, desc, use_no_calib_int8)) return true; } return false; From 63d7fdc0ca4bf1340e7235c2eefbea3b55f072a3 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Mon, 29 Mar 2021 06:32:46 +0000 Subject: [PATCH 080/106] resolve linking problem --- paddle/fluid/pybind/CMakeLists.txt | 6 +- .../inference/test_trt_affine_channel_op.py | 141 +++++++++++++++ scripts/paddle | 169 ++++++++++++++++++ 3 files changed, 315 insertions(+), 1 deletion(-) create mode 100644 python/paddle/fluid/tests/unittests/ir/inference/test_trt_affine_channel_op.py create mode 100644 scripts/paddle diff --git a/paddle/fluid/pybind/CMakeLists.txt b/paddle/fluid/pybind/CMakeLists.txt index 45e0594ceabfc..8a83512131a3f 100644 --- a/paddle/fluid/pybind/CMakeLists.txt +++ b/paddle/fluid/pybind/CMakeLists.txt @@ -5,8 +5,12 @@ include_directories(${PADDLE_SOURCE_DIR}/paddle/fluid/platform) set(PYBIND_DEPS pybind python proto_desc memory executor fleet_wrapper box_wrapper prune feed_fetch_method pass_builder parallel_executor profiler layer tracer engine scope_pool analysis_predictor imperative_profiler imperative_flag save_load_util dlpack_tensor device_context - gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper custom_operator ps_service graph_py_service) + gloo_wrapper infer_io_utils heter_wrapper generator op_version_registry ps_gpu_wrapper custom_operator) +if (WITH_PSCORE) + set(PYBIND_DEPS ${PYBIND_DEPS} ps_service) + set(PYBIND_DEPS ${PYBIND_DEPS} graph_py_service) +endif() if (WITH_GPU OR WITH_ROCM) set(PYBIND_DEPS ${PYBIND_DEPS} dynload_cuda) set(PYBIND_DEPS ${PYBIND_DEPS} cuda_device_guard) diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_affine_channel_op.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_affine_channel_op.py new file mode 100644 index 0000000000000..8bbba7c8b55fe --- /dev/null +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_affine_channel_op.py @@ -0,0 +1,141 @@ +# Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + +from __future__ import print_function + +import unittest +import itertools +import numpy as np +from inference_pass_test import InferencePassTest +import paddle.fluid as fluid +import paddle.fluid.core as core +from paddle.fluid.core import PassVersionChecker +from paddle.fluid.core import AnalysisConfig + + +class TRTAffineChannelTest(InferencePassTest): + def setUp(self): + self.bs = 2 + self.channel = 8 + self.height = 16 + self.width = 16 + self.data_layout = 'NCHW' + self.precision = AnalysisConfig.Precision.Float32 + self.serialize = False + self.enable_trt = True + + def build(self): + # set min_graph_size to 2, + # because affine channel doesn't support nhwc format + self.trt_parameters = InferencePassTest.TensorRTParam( + 1 << 30, self.bs, 2, self.precision, self.serialize, False) + + with fluid.program_guard(self.main_program, self.startup_program): + if self.data_layout == 'NCHW': + shape = [-1, self.channel, self.height, self.width] + else: + shape = [-1, self.height, self.width, self.channel] + + data = fluid.data(name='in', shape=shape, dtype='float32') + # set scale, bias by constant + scale = fluid.layers.create_parameter( + shape=[self.channel], + dtype='float32', + default_initializer=fluid.initializer.Constant(2.)) + bias = fluid.layers.create_parameter( + shape=[self.channel], + dtype='float32', + default_initializer=fluid.initializer.Constant(.5)) + affine_channel_out = fluid.layers.affine_channel( + data, scale=scale, bias=bias, data_layout=self.data_layout) + out = fluid.layers.batch_norm(affine_channel_out, is_test=True) + + shape[0] = self.bs + self.feeds = {'in': np.random.random(shape).astype('float32'), } + self.fetch_list = [out] + + def check_output(self): + if core.is_compiled_with_cuda(): + use_gpu = True + atol = 1e-5 + if self.trt_parameters.precision == AnalysisConfig.Precision.Half: + atol = 1e-3 + self.check_output_with_option(use_gpu, atol, flatten=True) + self.assertTrue( + PassVersionChecker.IsCompatible('tensorrt_subgraph_pass')) + + def run_test(self): + self.build() + self.check_output() + + def run_test_all(self): + precision_opt = [ + AnalysisConfig.Precision.Float32, AnalysisConfig.Precision.Half + ] + serialize_opt = [False, True] + + if self.data_layout == 'NCHW': + min_shape = [ + self.bs, self.channel, self.height // 2, self.width // 2 + ] + max_shape = [self.bs, self.channel, self.height * 2, self.width * 2] + opt_shape = [self.bs, self.channel, self.height, self.width] + + if self.data_layout == 'NHWC': + min_shape = [ + self.bs, self.height // 2, self.width // 2, self.channel + ] + max_shape = [self.bs, self.height * 2, self.width * 2, self.channel] + opt_shape = [self.bs, self.height, self.width, self.channel] + + dynamic_shape_profile = InferencePassTest.DynamicShapeParam({ + 'in': min_shape + }, {'in': max_shape}, {'in': opt_shape}, False) + dynamic_shape_opt = [None, dynamic_shape_profile] + + for precision, serialize, dynamic_shape in itertools.product( + precision_opt, serialize_opt, dynamic_shape_opt): + self.precision = precision + self.serialize = serialize + self.dynamic_shape_params = dynamic_shape + self.run_test() + + def test_base(self): + self.run_test() + + def test_fp16(self): + self.precision = AnalysisConfig.Precision.Half + self.run_test() + + def test_serialize(self): + self.serialize = True + self.run_test() + + def test_dynamic(self): + self.dynamic_shape_params = InferencePassTest.DynamicShapeParam({ + 'in': [self.bs, self.channel, self.height // 2, self.width // 2] + }, {'in': [self.bs, self.channel, self.height * 2, self.width * 2] + }, {'in': [self.bs, self.channel, self.height, self.width]}, False) + self.run_test() + + def test_nchw_all(self): + self.run_test_all() + + def test_nhwc(self): + self.data_layout = 'NHWC' + self.run_test_all() + + +if __name__ == "__main__": + unittest.main() diff --git a/scripts/paddle b/scripts/paddle new file mode 100644 index 0000000000000..5f256ccf15791 --- /dev/null +++ b/scripts/paddle @@ -0,0 +1,169 @@ +#!/bin/bash + +function version(){ + echo "PaddlePaddle , compiled with" + echo " with_avx: ON" + echo " with_gpu: OFF" + echo " with_mkl: ON" + echo " with_mkldnn: " + echo " with_python: ON" +} + +function ver2num() { + set -e + # convert version to number. + if [ -z "$1" ]; then # empty argument + printf "%03d%03d%03d%03d%03d" 0 + else + local VERN=$(echo $1 | sed 's#v##g' | sed 's#\.# #g' \ + | sed 's#a# 0 #g' | sed 's#b# 1 #g' | sed 's#rc# 2 #g') + if [ `echo $VERN | wc -w` -eq 3 ] ; then + printf "%03d%03d%03d%03d%03d" $VERN 999 999 + else + printf "%03d%03d%03d%03d%03d" $VERN + fi + fi + set +e +} + +function cpu_config() { + # auto set KMP_AFFINITY and OMP_DYNAMIC from Hyper Threading Status + # only when MKL enabled + if [ "ON" == "OFF" ]; then + return 0 + fi + platform="`uname -s`" + ht=0 + if [ $platform == "Linux" ]; then + ht=`lscpu |grep "per core"|awk -F':' '{print $2}'|xargs` + elif [ $platform == "Darwin" ]; then + if [ `sysctl -n hw.physicalcpu` -eq `sysctl -n hw.logicalcpu` ]; then + # HT is OFF + ht=1 + fi + else + return 0 + fi + if [ $ht -eq 1 ]; then # HT is OFF + if [ -z "$KMP_AFFINITY" ]; then + export KMP_AFFINITY="granularity=fine,compact,0,0" + fi + if [ -z "$OMP_DYNAMIC" ]; then + export OMP_DYNAMIC="FALSE" + fi + else # HT is ON + if [ -z "$KMP_AFFINITY" ]; then + export KMP_AFFINITY="granularity=fine,compact,1,0" + fi + if [ -z "$OMP_DYNAMIC" ]; then + export OMP_DYNAMIC="True" + fi + fi +} + +function threads_config() { + # auto set OMP_NUM_THREADS and MKL_NUM_THREADS + # according to trainer_count and total processors + # only when MKL enabled + # auto set OPENBLAS_NUM_THREADS when do not use MKL + platform="`uname -s`" + processors=0 + if [ $platform == "Linux" ]; then + processors=`grep "processor" /proc/cpuinfo|sort -u|wc -l` + elif [ $platform == "Darwin" ]; then + processors=`sysctl -n hw.logicalcpu` + else + return 0 + fi + trainers=`grep -Eo 'trainer_count.[0-9]+' <<< "$@" |grep -Eo '[0-9]+'|xargs` + if [ -z $trainers ]; then + trainers=1 + fi + threads=$((processors / trainers)) + if [ $threads -eq 0 ]; then + threads=1 + fi + if [ "ON" == "ON" ]; then + if [ -z "$OMP_NUM_THREADS" ]; then + export OMP_NUM_THREADS=$threads + fi + if [ -z "$MKL_NUM_THREADS" ]; then + export MKL_NUM_THREADS=$threads + fi + else + if [ -z "$OPENBLAS_NUM_THREADS" ]; then + export OPENBLAS_NUM_THREADS=$threads + fi + if [ $threads -gt 1 ] && [ -z "$OPENBLAS_MAIN_FREE" ]; then + export OPENBLAS_MAIN_FREE=1 + fi + fi + +} + +PADDLE_CONF_HOME="$HOME/.config/paddle" +mkdir -p ${PADDLE_CONF_HOME} + +if [ -z "${PADDLE_NO_STAT+x}" ]; then + SERVER_VER=`curl -m 5 -X POST --data content="{ \"version\": \"\" }"\ + -b ${PADDLE_CONF_HOME}/paddle.cookie \ + -c ${PADDLE_CONF_HOME}/paddle.cookie \ + http://api.paddlepaddle.org/version 2>/dev/null` + if [ $? -eq 0 ] && [ "$(ver2num )" -lt $(ver2num $SERVER_VER) ]; then + echo "Paddle release a new version ${SERVER_VER}, you can get the install package in http://www.paddlepaddle.org" + fi +fi + +PADDLE_BIN_PATH="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" + +if [ ! -z "${DEBUGGER}" ]; then + echo "Using debug command ${DEBUGGER}" +fi + +CUDNN_LIB_PATH="" + +if [ ! -z "${CUDNN_LIB_PATH}" ]; then + export LD_LIBRARY_PATH=${CUDNN_LIB_PATH}:${LD_LIBRARY_PATH} +fi + +export PYTHONPATH=${PWD}:${PYTHONPATH} + + +# Check python lib installed or not. +pip --help > /dev/null +if [ $? -ne 0 ]; then + echo "pip should be installed to run paddle." + exit 1 +fi + +if [ "OFF" == "ON" ]; then + PADDLE_NAME="paddlepaddle-gpu" +else + PADDLE_NAME="paddlepaddle" +fi + +INSTALLED_VERSION=`pip freeze 2>/dev/null | grep "^${PADDLE_NAME}==" | sed 's/.*==//g'` + +if [ -z "${INSTALLED_VERSION}" ]; then + INSTALLED_VERSION="0.0.0" # not installed +fi +cat < Date: Mon, 29 Mar 2021 07:44:38 +0000 Subject: [PATCH 081/106] code style --- .github/ISSUE_TEMPLATE/---document-issue-.md | 2 +- .../distributed/service/graph_brpc_client.h | 2 +- .../distributed/table/common_graph_table.cc | 13 +- paddle/fluid/distributed/table/graph_edge.cc | 5 +- paddle/fluid/distributed/table/graph_node.cc | 37 ++-- .../distributed/table/weighted_sampler.cc | 37 ++-- .../distributed/table/weighted_sampler.h | 27 ++- paddle/fluid/inference/api/demo_ci/clean.sh | 14 ++ paddle/fluid/train/demo/run.sh | 14 ++ paddle/fluid/train/imdb_demo/run.sh | 13 ++ paddle/scripts/build_docker_images.sh | 15 ++ .../docker/root/.scripts/git-completion.sh | 15 ++ paddle/scripts/fast_install.sh | 14 ++ patches/eigen/TensorBlock.h | 180 ++++++++++++------ python/paddle/fluid/dataloader/fetcher.py | 7 +- .../incubate/fleet/tests/cluster_train.sh | 14 ++ .../test_squared_mat_sub_fuse_pass.py | 6 +- .../unittests/ir/inference/test_trt_matmul.py | 23 +-- .../fluid/tests/unittests/parallel_test.sh | 15 ++ .../fluid/tests/unittests/test_bce_loss.py | 12 +- .../unittests/test_bce_with_logits_loss.py | 6 +- .../tests/unittests/test_c_comm_init_op.sh | 15 ++ .../tests/unittests/test_dist_fleet_ps10.py | 1 - .../test_flatten_contiguous_range_op.py | 3 +- .../fluid/tests/unittests/test_l1_loss.py | 12 +- .../tests/unittests/test_listen_and_serv.sh | 15 ++ .../fluid/tests/unittests/test_mse_loss.py | 18 +- ...ess_dataloader_iterable_dataset_dynamic.py | 1 + .../tests/unittests/test_pixel_shuffle.py | 12 +- .../fluid/tests/unittests/test_prod_op.py | 6 +- .../fluid/tests/unittests/test_selu_op.py | 9 +- .../unittests/test_sigmoid_focal_loss.py | 6 +- .../tests/unittests/test_transpose_op.py | 8 +- tools/check_api_approvals.sh | 14 ++ tools/check_sequence_op.sh | 14 ++ tools/cudaError/start.sh | 15 ++ tools/diff_api.py | 15 ++ tools/diff_unittest.py | 15 ++ tools/dockerfile/icode.sh | 14 ++ tools/document_preview.sh | 15 ++ tools/get_cpu_info.sh | 14 ++ 41 files changed, 520 insertions(+), 173 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/---document-issue-.md b/.github/ISSUE_TEMPLATE/---document-issue-.md index 7c464ac584bc8..ffc2fcd7817b6 100644 --- a/.github/ISSUE_TEMPLATE/---document-issue-.md +++ b/.github/ISSUE_TEMPLATE/---document-issue-.md @@ -56,4 +56,4 @@ For example: no sample code; The sample code is not helpful; The sample code not For example:Chinese API in this doc is inconsistent with English API, including params, description, sample code, formula, etc. #### Other -For example: The doc link is broken; The doc page is missing; Dead link in docs. \ No newline at end of file +For example: The doc link is broken; The doc page is missing; Dead link in docs. diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index c0d7910851f91..854e0f6ca4ac1 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -14,10 +14,10 @@ #pragma once +#include #include #include #include -#include #include #include "ThreadPool.h" diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 9776e4775bdde..995a39a654312 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -124,7 +124,7 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { std::ifstream file(path); std::string line; while (std::getline(file, line)) { - count ++; + count++; auto values = paddle::string::split_string(line, "\t"); if (values.size() < 2) continue; auto id = std::stoull(values[1]); @@ -160,12 +160,12 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { << " not in feature_map."; } } - valid_count ++; + valid_count++; } } - VLOG(0) << valid_count << "/" << count << " nodes in type " << - node_type << " are loaded successfully in " << path; + VLOG(0) << valid_count << "/" << count << " nodes in type " << node_type + << " are loaded successfully in " << path; return 0; } @@ -209,10 +209,11 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { size_t index = src_shard_id - shard_start; shards[index].add_graph_node(src_id)->build_edges(is_weighted); shards[index].add_neighboor(src_id, dst_id, weight); - valid_count ++; + valid_count++; } } - VLOG(0) << valid_count << "/" << count << " edges are loaded successfully in " << path; + VLOG(0) << valid_count << "/" << count << " edges are loaded successfully in " + << path; // Build Sampler j diff --git a/paddle/fluid/distributed/table/graph_edge.cc b/paddle/fluid/distributed/table/graph_edge.cc index 56ce7852484ec..cc90f4c6516c1 100644 --- a/paddle/fluid/distributed/table/graph_edge.cc +++ b/paddle/fluid/distributed/table/graph_edge.cc @@ -17,14 +17,13 @@ namespace paddle { namespace distributed { -void GraphEdgeBlob::add_edge(uint64_t id, float weight=1){ +void GraphEdgeBlob::add_edge(uint64_t id, float weight = 1) { id_arr.push_back(id); } -void WeightedGraphEdgeBlob::add_edge(uint64_t id, float weight=1){ +void WeightedGraphEdgeBlob::add_edge(uint64_t id, float weight = 1) { id_arr.push_back(id); weight_arr.push_back(weight); } - } } diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc index 8c20fc302f8d7..27a2cafaf4f0f 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph_node.cc @@ -17,13 +17,12 @@ namespace paddle { namespace distributed { - GraphNode::~GraphNode() { - if (sampler != nullptr){ + if (sampler != nullptr) { delete sampler; sampler = nullptr; } - if (edges != nullptr){ + if (edges != nullptr) { delete edges; edges = nullptr; } @@ -33,9 +32,7 @@ int Node::weight_size = sizeof(float); int Node::id_size = sizeof(uint64_t); int Node::int_size = sizeof(int); -int Node::get_size(bool need_feature) { - return id_size + int_size; -} +int Node::get_size(bool need_feature) { return id_size + int_size; } void Node::to_buffer(char* buffer, bool need_feature) { memcpy(buffer, &id, id_size); @@ -45,15 +42,13 @@ void Node::to_buffer(char* buffer, bool need_feature) { memcpy(buffer, &feat_num, sizeof(int)); } -void Node::recover_from_buffer(char* buffer) { - memcpy(&id, buffer, id_size); -} +void Node::recover_from_buffer(char* buffer) { memcpy(&id, buffer, id_size); } int FeatureNode::get_size(bool need_feature) { - int size = id_size + int_size; // id, feat_num - if (need_feature){ + int size = id_size + int_size; // id, feat_num + if (need_feature) { size += feature.size() * int_size; - for (const std::string& fea: feature){ + for (const std::string& fea : feature) { size += fea.size(); } } @@ -61,8 +56,8 @@ int FeatureNode::get_size(bool need_feature) { } void GraphNode::build_edges(bool is_weighted) { - if (edges == nullptr){ - if (is_weighted == true){ + if (edges == nullptr) { + if (is_weighted == true) { edges = new WeightedGraphEdgeBlob(); } else { edges = new GraphEdgeBlob(); @@ -70,11 +65,11 @@ void GraphNode::build_edges(bool is_weighted) { } } void GraphNode::build_sampler(std::string sample_type) { - if (sample_type == "random"){ + if (sample_type == "random") { sampler = new RandomSampler(); - } else if (sample_type == "weighted"){ + } else if (sample_type == "weighted") { sampler = new WeightedSampler(); - } + } sampler->build(edges); } void FeatureNode::to_buffer(char* buffer, bool need_feature) { @@ -87,7 +82,7 @@ void FeatureNode::to_buffer(char* buffer, bool need_feature) { feat_num += feature.size(); memcpy(buffer, &feat_num, sizeof(int)); buffer += sizeof(int); - for (int i = 0; i < feat_num; ++i){ + for (int i = 0; i < feat_num; ++i) { feat_len = feature[i].size(); memcpy(buffer, &feat_len, sizeof(int)); buffer += sizeof(int); @@ -99,14 +94,13 @@ void FeatureNode::to_buffer(char* buffer, bool need_feature) { } } void FeatureNode::recover_from_buffer(char* buffer) { - int feat_num, feat_len; memcpy(&id, buffer, id_size); buffer += id_size; - + memcpy(&feat_num, buffer, sizeof(int)); buffer += sizeof(int); - + feature.clear(); for (int i = 0; i < feat_num; ++i) { memcpy(&feat_len, buffer, sizeof(int)); @@ -118,7 +112,6 @@ void FeatureNode::recover_from_buffer(char* buffer) { str[feat_len] = '\0'; feature.push_back(std::string(str)); } - } } } diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/weighted_sampler.cc index 9dc9064742dee..69f845843007c 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/weighted_sampler.cc @@ -14,17 +14,15 @@ #include "paddle/fluid/distributed/table/weighted_sampler.h" #include -#include +#include namespace paddle { namespace distributed { -void RandomSampler::build(GraphEdgeBlob* edges) { - this->edges = edges; -} +void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } std::vector RandomSampler::sample_k(int k) { int n = edges->size(); - if (k > n){ + if (k > n) { k = n; } struct timespec tn; @@ -32,19 +30,19 @@ std::vector RandomSampler::sample_k(int k) { srand(tn.tv_nsec); std::vector sample_result; std::unordered_map replace_map; - while(k--){ + while (k--) { int rand_int = rand() % n; auto iter = replace_map.find(rand_int); - if(iter == replace_map.end()){ + if (iter == replace_map.end()) { sample_result.push_back(rand_int); - }else{ + } else { sample_result.push_back(iter->second); } iter = replace_map.find(n - 1); - if(iter == replace_map.end()){ + if (iter == replace_map.end()) { replace_map[rand_int] = n - 1; - }else{ + } else { replace_map[rand_int] = iter->second; } --n; @@ -52,36 +50,37 @@ std::vector RandomSampler::sample_k(int k) { return sample_result; } -WeightedSampler::WeightedSampler(){ +WeightedSampler::WeightedSampler() { left = nullptr; right = nullptr; edges = nullptr; } WeightedSampler::~WeightedSampler() { - if(left != nullptr){ + if (left != nullptr) { delete left; left = nullptr; } - if(right != nullptr){ + if (right != nullptr) { delete right; right = nullptr; } } -void WeightedSampler::build(GraphEdgeBlob* edges) { - if(left != nullptr){ +void WeightedSampler::build(GraphEdgeBlob *edges) { + if (left != nullptr) { delete left; left = nullptr; } - if(right != nullptr){ + if (right != nullptr) { delete right; right = nullptr; } - return build_one((WeightedGraphEdgeBlob*)edges, 0, edges->size()); + return build_one((WeightedGraphEdgeBlob *)edges, 0, edges->size()); } -void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, int end) { +void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, + int end) { count = 0; this->edges = edges; if (start + 1 == end) { @@ -137,7 +136,7 @@ int WeightedSampler::sample( if (right_count == 0 || left_count > 0 && left->weight - left_subtract >= query_weight) { return_idx = left->sample(query_weight, subtract_weight_map, - subtract_count_map, subtract); + subtract_count_map, subtract); } else { return_idx = right->sample(query_weight - (left->weight - left_subtract), diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/weighted_sampler.h index 82e5109e8f34b..cfc341d27c6b7 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.h +++ b/paddle/fluid/distributed/table/weighted_sampler.h @@ -21,21 +21,21 @@ namespace paddle { namespace distributed { class Sampler { -public: + public: virtual ~Sampler() {} - virtual void build(GraphEdgeBlob* edges) = 0; + virtual void build(GraphEdgeBlob *edges) = 0; virtual std::vector sample_k(int k) = 0; }; -class RandomSampler: public Sampler { -public: +class RandomSampler : public Sampler { + public: virtual ~RandomSampler() {} - virtual void build(GraphEdgeBlob* edges); + virtual void build(GraphEdgeBlob *edges); virtual std::vector sample_k(int k); - GraphEdgeBlob* edges; + GraphEdgeBlob *edges; }; -class WeightedSampler: public Sampler { +class WeightedSampler : public Sampler { public: WeightedSampler(); virtual ~WeightedSampler(); @@ -43,17 +43,16 @@ class WeightedSampler: public Sampler { float weight; int count; int idx; - GraphEdgeBlob * edges; - virtual void build(GraphEdgeBlob* edges); + GraphEdgeBlob *edges; + virtual void build(GraphEdgeBlob *edges); virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); virtual std::vector sample_k(int k); private: - int sample( - float query_weight, - std::unordered_map &subtract_weight_map, - std::unordered_map &subtract_count_map, - float &subtract); + int sample(float query_weight, + std::unordered_map &subtract_weight_map, + std::unordered_map &subtract_count_map, + float &subtract); }; } } diff --git a/paddle/fluid/inference/api/demo_ci/clean.sh b/paddle/fluid/inference/api/demo_ci/clean.sh index 0d9f3d2aa237a..c265721db5775 100755 --- a/paddle/fluid/inference/api/demo_ci/clean.sh +++ b/paddle/fluid/inference/api/demo_ci/clean.sh @@ -1,3 +1,17 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + set -x cd `dirname $0` rm -rf build/ data/ diff --git a/paddle/fluid/train/demo/run.sh b/paddle/fluid/train/demo/run.sh index 2955e7574daa2..c45a3528febdd 100755 --- a/paddle/fluid/train/demo/run.sh +++ b/paddle/fluid/train/demo/run.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + set -x PADDLE_ROOT=$1 diff --git a/paddle/fluid/train/imdb_demo/run.sh b/paddle/fluid/train/imdb_demo/run.sh index f71b4bac602a9..6de1df27e0035 100644 --- a/paddle/fluid/train/imdb_demo/run.sh +++ b/paddle/fluid/train/imdb_demo/run.sh @@ -1,3 +1,16 @@ +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. set -exu build/demo_trainer --flagfile="train.cfg" diff --git a/paddle/scripts/build_docker_images.sh b/paddle/scripts/build_docker_images.sh index a90f0885294a9..2b584cdca6b4c 100644 --- a/paddle/scripts/build_docker_images.sh +++ b/paddle/scripts/build_docker_images.sh @@ -1,4 +1,19 @@ #!/bin/sh + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + set -xe REPO="${REPO:-paddlepaddle}" diff --git a/paddle/scripts/docker/root/.scripts/git-completion.sh b/paddle/scripts/docker/root/.scripts/git-completion.sh index bdddef5ac2faf..c43e88a4acd73 100755 --- a/paddle/scripts/docker/root/.scripts/git-completion.sh +++ b/paddle/scripts/docker/root/.scripts/git-completion.sh @@ -1,4 +1,19 @@ #!bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + # # bash/zsh completion support for core Git. # diff --git a/paddle/scripts/fast_install.sh b/paddle/scripts/fast_install.sh index 1034b1c5c1043..cacec55d3bc22 100644 --- a/paddle/scripts/fast_install.sh +++ b/paddle/scripts/fast_install.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + ## purple to echo function purple(){ echo -e "\033[35m$1\033[0m" diff --git a/patches/eigen/TensorBlock.h b/patches/eigen/TensorBlock.h index 1e55d12c42fc2..be0a02f53d1c5 100644 --- a/patches/eigen/TensorBlock.h +++ b/patches/eigen/TensorBlock.h @@ -1,3 +1,17 @@ +// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +// +// 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. + // This file is part of Eigen, a lightweight C++ template library // for linear algebra. // @@ -82,15 +96,16 @@ struct TensorBlockResourceRequirements { // which is implicitly invoked in the "merge" / "any" routines. else HIPCC // errors out complaining about the lack of a matching constructor EIGEN_DEVICE_FUNC - TensorBlockResourceRequirements(TensorBlockShapeType shape_type_, size_t size_, - TensorOpCost cost_) - : shape_type(shape_type_), size(size_), cost_per_coeff(cost_) - {} + TensorBlockResourceRequirements(TensorBlockShapeType shape_type_, + size_t size_, + TensorOpCost cost_) + : shape_type(shape_type_), size(size_), cost_per_coeff(cost_) {} #endif template EIGEN_DEVICE_FUNC static TensorBlockResourceRequirements withShapeAndSize( - TensorBlockShapeType shape_type, size_t size_in_bytes, + TensorBlockShapeType shape_type, + size_t size_in_bytes, TensorOpCost cost) { const size_t size = numext::maxi(size_t(1), size_in_bytes / sizeof(Scalar)); return {shape_type, size, cost}; @@ -113,7 +128,8 @@ struct TensorBlockResourceRequirements { // cost approximation (e.g. shuffling inner dimension has a much higher cost // because it reads memory randomly, although the total number of moved // bytes is the same). - return withShapeAndSize(shape_type, size_in_bytes, + return withShapeAndSize(shape_type, + size_in_bytes, {/*bytes_loaded=*/sizeof(Scalar), /*bytes_stored=*/sizeof(Scalar), /*compute_cycles=*/0}); @@ -247,7 +263,8 @@ class TensorBlockDescriptor { DestinationBuffer() : m_data(NULL), m_data_type_size(0), m_kind(kEmpty) {} template - DestinationBuffer(Scalar* data, const Dimensions& strides, + DestinationBuffer(Scalar* data, + const Dimensions& strides, DestinationBufferKind kind) : m_data(static_cast(data)), m_data_type_size(sizeof(Scalar)), @@ -256,7 +273,8 @@ class TensorBlockDescriptor { template static DestinationBuffer make(const TensorBlockDescriptor& desc, - Scalar* data, const Dimensions& strides) { + Scalar* data, + const Dimensions& strides) { return DestinationBuffer(data, strides, kind(desc, strides)); } @@ -284,7 +302,8 @@ class TensorBlockDescriptor { DestinationBufferKind m_kind; }; - TensorBlockDescriptor(const IndexType offset, const Dimensions& dimensions, + TensorBlockDescriptor(const IndexType offset, + const Dimensions& dimensions, const DestinationBuffer& destination) : m_offset(offset), m_dimensions(dimensions), @@ -651,15 +670,19 @@ struct XprScalar { // be invalid, and should never be used in block assignment or any other tensor // expression. -template class TensorMaterializedBlock { public: typedef DSizes Dimensions; - typedef TensorMap > XprType; + typedef TensorMap> XprType; - TensorMaterializedBlock(TensorBlockKind kind, const Scalar* data, - const Dimensions& dimensions, bool valid_expr = true) + TensorMaterializedBlock(TensorBlockKind kind, + const Scalar* data, + const Dimensions& dimensions, + bool valid_expr = true) : m_kind(kind), m_data(data), m_dimensions(dimensions), @@ -702,14 +725,18 @@ class TensorMaterializedBlock { m_materialized_in_output ? internal::TensorBlockKind::kMaterializedInOutput : internal::TensorBlockKind::kMaterializedInScratch, - m_data, m_dimensions, !m_strided_storage); + m_data, + m_dimensions, + !m_strided_storage); } private: friend class TensorMaterializedBlock; - Storage(Scalar* data, const Dimensions& dimensions, - const Dimensions& strides, bool materialized_in_output, + Storage(Scalar* data, + const Dimensions& dimensions, + const Dimensions& strides, + bool materialized_in_output, bool strided_storage) : m_data(data), m_dimensions(dimensions), @@ -728,7 +755,8 @@ class TensorMaterializedBlock { // destination buffer, or allocates a new buffer with scratch allocator. template EIGEN_STRONG_INLINE static Storage prepareStorage( - TensorBlockDesc& desc, TensorBlockScratch& scratch, + TensorBlockDesc& desc, + TensorBlockScratch& scratch, bool allow_strided_storage = false) { // Try to reuse destination as an output block buffer. typedef typename TensorBlockDesc::DestinationBuffer DestinationBuffer; @@ -736,7 +764,8 @@ class TensorMaterializedBlock { if (desc.destination().kind() == DestinationBuffer::kContiguous) { Scalar* buffer = desc.destination().template data(); desc.DropDestinationBuffer(); - return Storage(buffer, desc.dimensions(), + return Storage(buffer, + desc.dimensions(), internal::strides(desc.dimensions()), /*materialized_in_output=*/true, /*strided_storage=*/false); @@ -745,12 +774,16 @@ class TensorMaterializedBlock { allow_strided_storage) { Scalar* buffer = desc.destination().template data(); desc.DropDestinationBuffer(); - return Storage(buffer, desc.dimensions(), desc.destination().strides(), - /*materialized_in_output=*/true, /*strided_storage=*/true); + return Storage(buffer, + desc.dimensions(), + desc.destination().strides(), + /*materialized_in_output=*/true, + /*strided_storage=*/true); } else { void* mem = scratch.allocate(desc.size() * sizeof(Scalar)); - return Storage(static_cast(mem), desc.dimensions(), + return Storage(static_cast(mem), + desc.dimensions(), internal::strides(desc.dimensions()), /*materialized_in_output=*/false, /*strided_storage=*/false); @@ -760,8 +793,10 @@ class TensorMaterializedBlock { // Creates a materialized block for the given descriptor from a memory buffer. template EIGEN_STRONG_INLINE static TensorMaterializedBlock materialize( - const Scalar* data, const DataDimensions& data_dims, - TensorBlockDesc& desc, TensorBlockScratch& scratch) { + const Scalar* data, + const DataDimensions& data_dims, + TensorBlockDesc& desc, + TensorBlockScratch& scratch) { eigen_assert(array_size::value == desc.dimensions().size()); // If a tensor block dimensions covers a contiguous block of the underlying @@ -797,8 +832,8 @@ class TensorMaterializedBlock { if (can_use_direct_access) { const Scalar* block_start = data + desc.offset(); - return TensorMaterializedBlock(internal::TensorBlockKind::kView, - block_start, desc.dimensions()); + return TensorMaterializedBlock( + internal::TensorBlockKind::kView, block_start, desc.dimensions()); } else { // Reuse destination buffer or allocate new buffer with scratch allocator. @@ -810,9 +845,10 @@ class TensorMaterializedBlock { typedef typename TensorBlockIO::Src TensorBlockIOSrc; TensorBlockIOSrc src(internal::strides(Dimensions(data_dims)), - data, desc.offset()); - TensorBlockIODst dst(storage.dimensions(), storage.strides(), - storage.data()); + data, + desc.offset()); + TensorBlockIODst dst( + storage.dimensions(), storage.strides(), storage.data()); TensorBlockIO::Copy(dst, src); return storage.AsTensorMaterializedBlock(); @@ -838,9 +874,10 @@ class TensorCwiseUnaryBlock { public: typedef typename conditional< - NoArgBlockAccess, void, - TensorCwiseUnaryOp >:: - type XprType; + NoArgBlockAccess, + void, + TensorCwiseUnaryOp>::type + XprType; typedef typename XprScalar::type Scalar; @@ -870,9 +907,11 @@ class TensorCwiseBinaryBlock { public: typedef typename conditional< - NoArgBlockAccess, void, - TensorCwiseBinaryOp >::type + NoArgBlockAccess, + void, + TensorCwiseBinaryOp>::type XprType; typedef typename XprScalar::type Scalar; @@ -915,7 +954,8 @@ class TensorUnaryExprBlock { public: typedef typename conditional< - NoArgBlockAccess, void, + NoArgBlockAccess, + void, typename BlockFactory::template XprType::type>::type XprType; typedef typename XprScalar::type Scalar; @@ -938,8 +978,10 @@ class TensorUnaryExprBlock { // TensorTernaryExprBlock is a lazy tensor expression block that can construct // an arbitrary tensor expression from three blocks of the underlying type. -template +template class TensorTernaryExprBlock { typedef typename Arg1TensorBlock::XprType Arg1XprType; typedef typename Arg2TensorBlock::XprType Arg2XprType; @@ -951,8 +993,10 @@ class TensorTernaryExprBlock { public: typedef typename conditional< - NoArgBlockAccess, void, - typename BlockFactory::template XprType::type>::type XprType; typedef typename XprScalar::type Scalar; @@ -968,8 +1012,8 @@ class TensorTernaryExprBlock { TensorBlockKind kind() const { return internal::TensorBlockKind::kExpr; } XprType expr() const { - return m_factory.expr(m_arg1_block.expr(), m_arg2_block.expr(), - m_arg3_block.expr()); + return m_factory.expr( + m_arg1_block.expr(), m_arg2_block.expr(), m_arg3_block.expr()); } const Scalar* data() const { return NULL; } void cleanup() { @@ -1029,16 +1073,24 @@ class StridedLinearBufferCopy { static EIGEN_DEVICE_FUNC EIGEN_STRONG_INLINE void Run(const Dst& dst, const Src& src, const size_t count) { - Run(count, dst.offset, dst.stride, dst.data, src.offset, src.stride, + Run(count, + dst.offset, + dst.stride, + dst.data, + src.offset, + src.stride, src.data); } private: template static EIGEN_DEVICE_FUNC EIGEN_STRONG_INLINE void Run( - const IndexType count, const IndexType dst_offset, - const IndexType dst_stride, Scalar* EIGEN_RESTRICT dst_data, - const IndexType src_offset, const IndexType src_stride, + const IndexType count, + const IndexType dst_offset, + const IndexType dst_stride, + Scalar* EIGEN_RESTRICT dst_data, + const IndexType src_offset, + const IndexType src_stride, const Scalar* EIGEN_RESTRICT src_data) { const Scalar* src = &src_data[src_offset]; Scalar* dst = &dst_data[dst_offset]; @@ -1150,7 +1202,9 @@ class TensorBlockIO { typedef DSizes DimensionsMap; struct Dst { - Dst(const Dimensions& dst_dims, const Dimensions& dst_strides, Scalar* dst, + Dst(const Dimensions& dst_dims, + const Dimensions& dst_strides, + Scalar* dst, IndexType dst_offset = 0) : dims(dst_dims), strides(dst_strides), data(dst), offset(dst_offset) {} @@ -1161,7 +1215,8 @@ class TensorBlockIO { }; struct Src { - Src(const Dimensions& src_strides, const Scalar* src, + Src(const Dimensions& src_strides, + const Scalar* src, IndexType src_offset = 0) : strides(src_strides), data(src), offset(src_offset) {} @@ -1376,7 +1431,9 @@ class TensorBlockIO { // where `src` is a tensor expression. Explore if it is possible to rewrite IO // to use expressions instead of pointers, and after that TensorBlockAssignment // will become an alias to IO. -template class TensorBlockAssignment { // We will use coeff/packet path to evaluate block expressions. @@ -1392,7 +1449,8 @@ class TensorBlockAssignment { template struct InnerDimAssign { - EIGEN_ALWAYS_INLINE static void Run(Scalar* target, IndexType count, + EIGEN_ALWAYS_INLINE static void Run(Scalar* target, + IndexType count, const Evaluator& eval, IndexType eval_offset) { for (IndexType i = 0; i < count; ++i) { @@ -1403,7 +1461,8 @@ class TensorBlockAssignment { template struct InnerDimAssign { - EIGEN_ALWAYS_INLINE static void Run(Scalar* target, IndexType count, + EIGEN_ALWAYS_INLINE static void Run(Scalar* target, + IndexType count, const Evaluator& eval, IndexType eval_offset) { typedef typename packet_traits::type Packet; @@ -1433,8 +1492,10 @@ class TensorBlockAssignment { public: struct Target { - Target(const Dimensions& target_dims, const Dimensions& target_strides, - Scalar* target_data, IndexType target_offset = 0) + Target(const Dimensions& target_dims, + const Dimensions& target_strides, + Scalar* target_data, + IndexType target_offset = 0) : dims(target_dims), strides(target_strides), data(target_data), @@ -1447,7 +1508,8 @@ class TensorBlockAssignment { }; static Target target(const Dimensions& target_dims, - const Dimensions& target_strides, Scalar* target_data, + const Dimensions& target_strides, + Scalar* target_data, IndexType target_offset = 0) { return Target(target_dims, target_strides, target_data, target_offset); } @@ -1456,10 +1518,13 @@ class TensorBlockAssignment { static Target target( const DSizes& target_dims, const DSizes& target_strides, - Scalar* target_data, IndexType target_offset = 0) { + Scalar* target_data, + IndexType target_offset = 0) { // DSizes constructor will do index type promotion if it's safe. - return Target(Dimensions(target_dims), Dimensions(target_strides), - target_data, target_offset); + return Target(Dimensions(target_dims), + Dimensions(target_strides), + target_data, + target_offset); } static EIGEN_DEVICE_FUNC EIGEN_STRONG_INLINE void Run( @@ -1521,7 +1586,8 @@ class TensorBlockAssignment { // Assign to `target` at current offset. InnerDimAssign::Run(target.data + output_offset, - output_inner_dim_size, eval, + output_inner_dim_size, + eval, input_offset); // Move input offset forward by the number of assigned coefficients. diff --git a/python/paddle/fluid/dataloader/fetcher.py b/python/paddle/fluid/dataloader/fetcher.py index 9382a70422370..41e12fbc68ec1 100644 --- a/python/paddle/fluid/dataloader/fetcher.py +++ b/python/paddle/fluid/dataloader/fetcher.py @@ -27,8 +27,8 @@ def fetch(self, batch_indices): class _IterableDatasetFetcher(_DatasetFetcher): def __init__(self, dataset, auto_collate_batch, collate_fn, drop_last): - super(_IterableDatasetFetcher, self).__init__(dataset, auto_collate_batch, - collate_fn, drop_last) + super(_IterableDatasetFetcher, self).__init__( + dataset, auto_collate_batch, collate_fn, drop_last) self.dataset_iter = iter(dataset) def fetch(self, batch_indices): @@ -53,7 +53,8 @@ def fetch(self, batch_indices): class _MapDatasetFetcher(_DatasetFetcher): def __init__(self, dataset, auto_collate_batch, collate_fn, drop_last): - super(_MapDatasetFetcher, self).__init__(dataset, auto_collate_batch, collate_fn, drop_last) + super(_MapDatasetFetcher, self).__init__(dataset, auto_collate_batch, + collate_fn, drop_last) def fetch(self, batch_indices): if self.auto_collate_batch: diff --git a/python/paddle/fluid/incubate/fleet/tests/cluster_train.sh b/python/paddle/fluid/incubate/fleet/tests/cluster_train.sh index 1df6b0618de8d..cac2f7234bdf2 100644 --- a/python/paddle/fluid/incubate/fleet/tests/cluster_train.sh +++ b/python/paddle/fluid/incubate/fleet/tests/cluster_train.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + # start pserver0 python fleet_deep_ctr.py \ --role pserver \ diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_squared_mat_sub_fuse_pass.py b/python/paddle/fluid/tests/unittests/ir/inference/test_squared_mat_sub_fuse_pass.py index 95cff4de6f6b0..69a9ae3c0ad2c 100644 --- a/python/paddle/fluid/tests/unittests/ir/inference/test_squared_mat_sub_fuse_pass.py +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_squared_mat_sub_fuse_pass.py @@ -40,9 +40,11 @@ def setUp(self): matmul_ab_square = paddle.square(matmul_ab) matmul_square_ab = paddle.matmul(data_a_square, data_b_square) - scale = paddle.fluid.layers.fill_constant(shape=[1], value=0.5, dtype='float32') + scale = paddle.fluid.layers.fill_constant( + shape=[1], value=0.5, dtype='float32') - sub_val = paddle.fluid.layers.elementwise_sub(matmul_ab_square, matmul_square_ab) + sub_val = paddle.fluid.layers.elementwise_sub(matmul_ab_square, + matmul_square_ab) squared_mat_sub_out = fluid.layers.elementwise_mul(sub_val, scale) self.feeds = { diff --git a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_matmul.py b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_matmul.py index 94434f4043448..080d1ccc9054b 100644 --- a/python/paddle/fluid/tests/unittests/ir/inference/test_trt_matmul.py +++ b/python/paddle/fluid/tests/unittests/ir/inference/test_trt_matmul.py @@ -25,19 +25,16 @@ class TensorRTMatMulDims2Test(InferencePassTest): def setUp(self): self.set_params() with fluid.program_guard(self.main_program, self.startup_program): - data = fluid.data( - name="data", shape=[24, 24], dtype="float32") + data = fluid.data(name="data", shape=[24, 24], dtype="float32") matmul_out = fluid.layers.matmul( x=data, y=data, - transpose_x = self.transpose_x, - transpose_y = self.transpose_y, - alpha = self.alpha) + transpose_x=self.transpose_x, + transpose_y=self.transpose_y, + alpha=self.alpha) out = fluid.layers.batch_norm(matmul_out, is_test=True) - self.feeds = { - "data": np.ones([24, 24]).astype("float32"), - } + self.feeds = {"data": np.ones([24, 24]).astype("float32"), } self.enable_trt = True self.trt_parameters = TensorRTMatMulDims2Test.TensorRTParam( 1 << 30, 32, 0, AnalysisConfig.Precision.Float32, False, False) @@ -65,14 +62,12 @@ def setUp(self): matmul_out = fluid.layers.matmul( x=data, y=data, - transpose_x = self.transpose_x, - transpose_y = self.transpose_y, - alpha = self.alpha) + transpose_x=self.transpose_x, + transpose_y=self.transpose_y, + alpha=self.alpha) out = fluid.layers.batch_norm(matmul_out, is_test=True) - self.feeds = { - "data": np.ones([1, 6, 24, 24]).astype("float32"), - } + self.feeds = {"data": np.ones([1, 6, 24, 24]).astype("float32"), } self.enable_trt = True self.trt_parameters = TensorRTMatMulTest.TensorRTParam( 1 << 30, 32, 0, AnalysisConfig.Precision.Float32, False, False) diff --git a/python/paddle/fluid/tests/unittests/parallel_test.sh b/python/paddle/fluid/tests/unittests/parallel_test.sh index 9da4f035345d7..551b7cdb7a43c 100644 --- a/python/paddle/fluid/tests/unittests/parallel_test.sh +++ b/python/paddle/fluid/tests/unittests/parallel_test.sh @@ -1,4 +1,19 @@ #!/bin/bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + unset https_proxy http_proxy export FLAGS_rpc_disable_reuse_port=1 diff --git a/python/paddle/fluid/tests/unittests/test_bce_loss.py b/python/paddle/fluid/tests/unittests/test_bce_loss.py index 4b39436842b89..ea1a22780f093 100644 --- a/python/paddle/fluid/tests/unittests/test_bce_loss.py +++ b/python/paddle/fluid/tests/unittests/test_bce_loss.py @@ -27,8 +27,10 @@ def test_static_layer(place, prog = paddle.static.Program() startup_prog = paddle.static.Program() with paddle.static.program_guard(prog, startup_prog): - input = paddle.fluid.data(name='input', shape=input_np.shape, dtype='float64') - label = paddle.fluid.data(name='label', shape=label_np.shape, dtype='float64') + input = paddle.fluid.data( + name='input', shape=input_np.shape, dtype='float64') + label = paddle.fluid.data( + name='label', shape=label_np.shape, dtype='float64') if weight_np is not None: weight = paddle.fluid.data( name='weight', shape=weight_np.shape, dtype='float64') @@ -58,8 +60,10 @@ def test_static_functional(place, prog = paddle.static.Program() startup_prog = paddle.static.Program() with paddle.static.program_guard(prog, startup_prog): - input = paddle.fluid.data(name='input', shape=input_np.shape, dtype='float64') - label = paddle.fluid.data(name='label', shape=label_np.shape, dtype='float64') + input = paddle.fluid.data( + name='input', shape=input_np.shape, dtype='float64') + label = paddle.fluid.data( + name='label', shape=label_np.shape, dtype='float64') if weight_np is not None: weight = paddle.fluid.data( name='weight', shape=weight_np.shape, dtype='float64') diff --git a/python/paddle/fluid/tests/unittests/test_bce_with_logits_loss.py b/python/paddle/fluid/tests/unittests/test_bce_with_logits_loss.py index a6175aa471d69..153b8fd3e7f6b 100644 --- a/python/paddle/fluid/tests/unittests/test_bce_with_logits_loss.py +++ b/python/paddle/fluid/tests/unittests/test_bce_with_logits_loss.py @@ -48,8 +48,10 @@ def test_static(place, prog = paddle.static.Program() startup_prog = paddle.static.Program() with paddle.static.program_guard(prog, startup_prog): - logit = paddle.fluid.data(name='logit', shape=logit_np.shape, dtype='float64') - label = paddle.fluid.data(name='label', shape=label_np.shape, dtype='float64') + logit = paddle.fluid.data( + name='logit', shape=logit_np.shape, dtype='float64') + label = paddle.fluid.data( + name='label', shape=label_np.shape, dtype='float64') feed_dict = {"logit": logit_np, "label": label_np} pos_weight = None diff --git a/python/paddle/fluid/tests/unittests/test_c_comm_init_op.sh b/python/paddle/fluid/tests/unittests/test_c_comm_init_op.sh index a9d450e223f1e..aba95a68ab790 100644 --- a/python/paddle/fluid/tests/unittests/test_c_comm_init_op.sh +++ b/python/paddle/fluid/tests/unittests/test_c_comm_init_op.sh @@ -1,4 +1,19 @@ #!/bin/bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + set -e # use default values # FIXME: random fails on Unknown command lines -c (or -m). diff --git a/python/paddle/fluid/tests/unittests/test_dist_fleet_ps10.py b/python/paddle/fluid/tests/unittests/test_dist_fleet_ps10.py index 16584ee50081a..a82866a797db1 100644 --- a/python/paddle/fluid/tests/unittests/test_dist_fleet_ps10.py +++ b/python/paddle/fluid/tests/unittests/test_dist_fleet_ps10.py @@ -23,7 +23,6 @@ paddle.enable_static() - # For Net base_lr = 0.2 emb_lr = base_lr * 3 diff --git a/python/paddle/fluid/tests/unittests/test_flatten_contiguous_range_op.py b/python/paddle/fluid/tests/unittests/test_flatten_contiguous_range_op.py index aa85eb3df3527..28803f5ac6232 100644 --- a/python/paddle/fluid/tests/unittests/test_flatten_contiguous_range_op.py +++ b/python/paddle/fluid/tests/unittests/test_flatten_contiguous_range_op.py @@ -170,7 +170,8 @@ def test_type(): x2 = np.arange(image_shape[0] * image_shape[1] * image_shape[2] * image_shape[3]).reshape(image_shape) / 100. x2 = x2.astype('float16') - x2_var = paddle.fluid.data(name='x2', shape=[3, 2, 4, 5], dtype='float16') + x2_var = paddle.fluid.data( + name='x2', shape=[3, 2, 4, 5], dtype='float16') paddle.flatten(x2_var) self.assertRaises(TypeError, test_type) diff --git a/python/paddle/fluid/tests/unittests/test_l1_loss.py b/python/paddle/fluid/tests/unittests/test_l1_loss.py index fba16959901a8..c35188623b440 100644 --- a/python/paddle/fluid/tests/unittests/test_l1_loss.py +++ b/python/paddle/fluid/tests/unittests/test_l1_loss.py @@ -44,8 +44,10 @@ def run_imperative(self): self.assertTrue(dy_result.shape, [10, 10, 5]) def run_static(self, use_gpu=False): - input = paddle.fluid.data(name='input', shape=[10, 10, 5], dtype='float32') - label = paddle.fluid.data(name='label', shape=[10, 10, 5], dtype='float32') + input = paddle.fluid.data( + name='input', shape=[10, 10, 5], dtype='float32') + label = paddle.fluid.data( + name='label', shape=[10, 10, 5], dtype='float32') result0 = paddle.nn.functional.l1_loss(input, label) result1 = paddle.nn.functional.l1_loss(input, label, reduction='sum') result2 = paddle.nn.functional.l1_loss(input, label, reduction='none') @@ -127,8 +129,10 @@ def run_imperative(self): self.assertTrue(dy_result.shape, [10, 10, 5]) def run_static(self, use_gpu=False): - input = paddle.fluid.data(name='input', shape=[10, 10, 5], dtype='float32') - label = paddle.fluid.data(name='label', shape=[10, 10, 5], dtype='float32') + input = paddle.fluid.data( + name='input', shape=[10, 10, 5], dtype='float32') + label = paddle.fluid.data( + name='label', shape=[10, 10, 5], dtype='float32') l1_loss = paddle.nn.loss.L1Loss() result0 = l1_loss(input, label) l1_loss = paddle.nn.loss.L1Loss(reduction='sum') diff --git a/python/paddle/fluid/tests/unittests/test_listen_and_serv.sh b/python/paddle/fluid/tests/unittests/test_listen_and_serv.sh index bee230fba5a7e..d9d64e4dfa693 100644 --- a/python/paddle/fluid/tests/unittests/test_listen_and_serv.sh +++ b/python/paddle/fluid/tests/unittests/test_listen_and_serv.sh @@ -1,4 +1,19 @@ #!/bin/bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + unset https_proxy http_proxy nohup python -u test_listen_and_serv_op.py > test_listen_and_serv_op.log 2>&1 & diff --git a/python/paddle/fluid/tests/unittests/test_mse_loss.py b/python/paddle/fluid/tests/unittests/test_mse_loss.py index bc5d35d3254bc..89eef6ca24243 100644 --- a/python/paddle/fluid/tests/unittests/test_mse_loss.py +++ b/python/paddle/fluid/tests/unittests/test_mse_loss.py @@ -191,8 +191,10 @@ def test_NNFunctionalMseLoss_mean(self): place = paddle.CUDAPlace(0) if core.is_compiled_with_cuda( ) else paddle.CPUPlace() with paddle.static.program_guard(prog, startup_prog): - input = paddle.fluid.data(name='input', shape=dim, dtype='float32') - target = paddle.fluid.data(name='target', shape=dim, dtype='float32') + input = paddle.fluid.data( + name='input', shape=dim, dtype='float32') + target = paddle.fluid.data( + name='target', shape=dim, dtype='float32') mse_loss = paddle.nn.functional.mse_loss(input, target, 'mean') exe = paddle.static.Executor(place) @@ -225,8 +227,10 @@ def test_NNFunctionalMseLoss_sum(self): place = paddle.CUDAPlace(0) if core.is_compiled_with_cuda( ) else paddle.CPUPlace() with paddle.static.program_guard(prog, startup_prog): - input = paddle.fluid.data(name='input', shape=dim, dtype='float32') - target = paddle.fluid.data(name='target', shape=dim, dtype='float32') + input = paddle.fluid.data( + name='input', shape=dim, dtype='float32') + target = paddle.fluid.data( + name='target', shape=dim, dtype='float32') mse_loss = paddle.nn.functional.mse_loss(input, target, 'sum') exe = paddle.static.Executor(place) @@ -259,8 +263,10 @@ def test_NNFunctionalMseLoss_none(self): place = paddle.CUDAPlace(0) if core.is_compiled_with_cuda( ) else paddle.CPUPlace() with paddle.static.program_guard(prog, startup_prog): - input = paddle.fluid.data(name='input', shape=dim, dtype='float32') - target = paddle.fluid.data(name='target', shape=dim, dtype='float32') + input = paddle.fluid.data( + name='input', shape=dim, dtype='float32') + target = paddle.fluid.data( + name='target', shape=dim, dtype='float32') mse_loss = paddle.nn.functional.mse_loss(input, target, 'none') exe = paddle.static.Executor(place) diff --git a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_dynamic.py b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_dynamic.py index 0533a0d09fa0d..3bb3e843b1b11 100644 --- a/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_dynamic.py +++ b/python/paddle/fluid/tests/unittests/test_multiprocess_dataloader_iterable_dataset_dynamic.py @@ -160,5 +160,6 @@ def run_main(self, num_workers, places): print("time cost", ret['time'], 'step_list', ret['step']) return ret + if __name__ == '__main__': unittest.main() diff --git a/python/paddle/fluid/tests/unittests/test_pixel_shuffle.py b/python/paddle/fluid/tests/unittests/test_pixel_shuffle.py index f75d6e9df540b..f1a409c712fc3 100644 --- a/python/paddle/fluid/tests/unittests/test_pixel_shuffle.py +++ b/python/paddle/fluid/tests/unittests/test_pixel_shuffle.py @@ -97,8 +97,10 @@ def test_static_graph_functional(self): place = paddle.CUDAPlace(0) if use_cuda else paddle.CPUPlace() paddle.enable_static() - x_1 = paddle.fluid.data(name="x", shape=[2, 9, 4, 4], dtype="float64") - x_2 = paddle.fluid.data(name="x2", shape=[2, 4, 4, 9], dtype="float64") + x_1 = paddle.fluid.data( + name="x", shape=[2, 9, 4, 4], dtype="float64") + x_2 = paddle.fluid.data( + name="x2", shape=[2, 4, 4, 9], dtype="float64") out_1 = F.pixel_shuffle(x_1, 3) out_2 = F.pixel_shuffle(x_2, 3, "NHWC") @@ -123,8 +125,10 @@ def test_static_graph_layer(self): place = paddle.CUDAPlace(0) if use_cuda else paddle.CPUPlace() paddle.enable_static() - x_1 = paddle.fluid.data(name="x", shape=[2, 9, 4, 4], dtype="float64") - x_2 = paddle.fluid.data(name="x2", shape=[2, 4, 4, 9], dtype="float64") + x_1 = paddle.fluid.data( + name="x", shape=[2, 9, 4, 4], dtype="float64") + x_2 = paddle.fluid.data( + name="x2", shape=[2, 4, 4, 9], dtype="float64") # init instance ps_1 = paddle.nn.PixelShuffle(3) ps_2 = paddle.nn.PixelShuffle(3, "NHWC") diff --git a/python/paddle/fluid/tests/unittests/test_prod_op.py b/python/paddle/fluid/tests/unittests/test_prod_op.py index 15fd79542d608..cdfcbb4e4e735 100644 --- a/python/paddle/fluid/tests/unittests/test_prod_op.py +++ b/python/paddle/fluid/tests/unittests/test_prod_op.py @@ -55,7 +55,8 @@ def run_imperative(self): self.assertTrue(np.allclose(dy_result.numpy(), expected_result)) def run_static(self, use_gpu=False): - input = paddle.fluid.data(name='input', shape=[10, 10, 5], dtype='float32') + input = paddle.fluid.data( + name='input', shape=[10, 10, 5], dtype='float32') result0 = paddle.prod(input) result1 = paddle.prod(input, axis=1) result2 = paddle.prod(input, axis=-1) @@ -114,7 +115,8 @@ def test_error(self): with paddle.static.program_guard(paddle.static.Program(), paddle.static.Program()): x = paddle.fluid.data(name='x', shape=[2, 2, 4], dtype='float32') - bool_x = paddle.fluid.data(name='bool_x', shape=[2, 2, 4], dtype='bool') + bool_x = paddle.fluid.data( + name='bool_x', shape=[2, 2, 4], dtype='bool') # The argument x shoule be a Tensor self.assertRaises(TypeError, paddle.prod, [1]) diff --git a/python/paddle/fluid/tests/unittests/test_selu_op.py b/python/paddle/fluid/tests/unittests/test_selu_op.py index 95ae1eecc6614..e71adae8d9b6e 100644 --- a/python/paddle/fluid/tests/unittests/test_selu_op.py +++ b/python/paddle/fluid/tests/unittests/test_selu_op.py @@ -128,15 +128,18 @@ def test_errors(self): # The input type must be Variable. self.assertRaises(TypeError, F.selu, 1) # The input dtype must be float16, float32, float64. - x_int32 = paddle.fluid.data(name='x_int32', shape=[12, 10], dtype='int32') + x_int32 = paddle.fluid.data( + name='x_int32', shape=[12, 10], dtype='int32') self.assertRaises(TypeError, F.selu, x_int32) # The scale must be greater than 1.0 - x_fp32 = paddle.fluid.data(name='x_fp32', shape=[12, 10], dtype='float32') + x_fp32 = paddle.fluid.data( + name='x_fp32', shape=[12, 10], dtype='float32') self.assertRaises(ValueError, F.selu, x_fp32, -1.0) # The alpha must be no less than 0 self.assertRaises(ValueError, F.selu, x_fp32, 1.6, -1.0) # support the input dtype is float16 - x_fp16 = paddle.fluid.data(name='x_fp16', shape=[12, 10], dtype='float16') + x_fp16 = paddle.fluid.data( + name='x_fp16', shape=[12, 10], dtype='float16') F.selu(x_fp16) diff --git a/python/paddle/fluid/tests/unittests/test_sigmoid_focal_loss.py b/python/paddle/fluid/tests/unittests/test_sigmoid_focal_loss.py index 85f9501e53f4a..2ef04d9cbfa73 100644 --- a/python/paddle/fluid/tests/unittests/test_sigmoid_focal_loss.py +++ b/python/paddle/fluid/tests/unittests/test_sigmoid_focal_loss.py @@ -42,8 +42,10 @@ def test_static(place, prog = paddle.static.Program() startup_prog = paddle.static.Program() with paddle.static.program_guard(prog, startup_prog): - logit = paddle.fluid.data(name='logit', shape=logit_np.shape, dtype='float64') - label = paddle.fluid.data(name='label', shape=label_np.shape, dtype='float64') + logit = paddle.fluid.data( + name='logit', shape=logit_np.shape, dtype='float64') + label = paddle.fluid.data( + name='label', shape=label_np.shape, dtype='float64') feed_dict = {"logit": logit_np, "label": label_np} normalizer = None diff --git a/python/paddle/fluid/tests/unittests/test_transpose_op.py b/python/paddle/fluid/tests/unittests/test_transpose_op.py index f72df8cbe4640..59b4afdf8b02d 100644 --- a/python/paddle/fluid/tests/unittests/test_transpose_op.py +++ b/python/paddle/fluid/tests/unittests/test_transpose_op.py @@ -23,6 +23,7 @@ paddle.enable_static() + class TestTransposeOp(OpTest): def setUp(self): self.init_op_type() @@ -151,6 +152,7 @@ def test_each_elem_value_check(): self.assertRaises(ValueError, test_each_elem_value_check) + class TestTransposeApi(unittest.TestCase): def test_static_out(self): paddle.enable_static() @@ -161,10 +163,11 @@ def test_static_out(self): place = paddle.CPUPlace() exe = paddle.static.Executor(place) x_np = np.random.random([2, 3, 4]).astype("float32") - result1, result2 = exe.run(feed={"x": x_np}, fetch_list=[x_trans1, x_trans2]) + result1, result2 = exe.run(feed={"x": x_np}, + fetch_list=[x_trans1, x_trans2]) expected_result1 = np.transpose(x_np, [1, 0, 2]) expected_result2 = np.transpose(x_np, (2, 1, 0)) - + np.testing.assert_array_equal(result1, expected_result1) np.testing.assert_array_equal(result2, expected_result2) @@ -185,6 +188,7 @@ def test_dygraph_out(self): # dygraph test paddle.enable_static() + class TestTAPI(unittest.TestCase): def test_out(self): with fluid.program_guard(fluid.Program()): diff --git a/tools/check_api_approvals.sh b/tools/check_api_approvals.sh index 4e8ea25715451..eb05468eda6ca 100644 --- a/tools/check_api_approvals.sh +++ b/tools/check_api_approvals.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + if [ -z ${BRANCH} ]; then BRANCH="develop" fi diff --git a/tools/check_sequence_op.sh b/tools/check_sequence_op.sh index ada96750eaad8..a263b046b258b 100644 --- a/tools/check_sequence_op.sh +++ b/tools/check_sequence_op.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + PADDLE_ROOT="$( cd "$( dirname "${BASH_SOURCE[0]}")/../" && pwd )" function check_sequnece_op_unitests(){ diff --git a/tools/cudaError/start.sh b/tools/cudaError/start.sh index 3c0e57ffe7ec1..66e56b8485d8c 100644 --- a/tools/cudaError/start.sh +++ b/tools/cudaError/start.sh @@ -1,4 +1,19 @@ #!/usr/bin/env bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + set -ex SYSTEM=`uname -s` rm -f protoc-3.11.3-linux-x86_64.* diff --git a/tools/diff_api.py b/tools/diff_api.py index 8a2acbb3d0acc..f086598945afe 100644 --- a/tools/diff_api.py +++ b/tools/diff_api.py @@ -1,4 +1,19 @@ #!/usr/bin/env python + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + from __future__ import print_function import difflib import sys diff --git a/tools/diff_unittest.py b/tools/diff_unittest.py index 382fbdd0b0c29..fa70be0990ec0 100644 --- a/tools/diff_unittest.py +++ b/tools/diff_unittest.py @@ -1,4 +1,19 @@ #!/usr/bin/env python + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + import difflib import sys diff --git a/tools/dockerfile/icode.sh b/tools/dockerfile/icode.sh index da3ffb8c77db7..973975fe7f737 100755 --- a/tools/dockerfile/icode.sh +++ b/tools/dockerfile/icode.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + function install_gcc(){ sed -i 's##RUN apt-get update \ diff --git a/tools/document_preview.sh b/tools/document_preview.sh index 10f486f8fd4f6..83c758d0aa8b8 100755 --- a/tools/document_preview.sh +++ b/tools/document_preview.sh @@ -1,4 +1,19 @@ #!/bin/bash + +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# 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. + PADDLE_ROOT=/home mkdir ${PADDLE_ROOT} cd ${PADDLE_ROOT} diff --git a/tools/get_cpu_info.sh b/tools/get_cpu_info.sh index 81eb19dc0661e..bce338a8619e6 100755 --- a/tools/get_cpu_info.sh +++ b/tools/get_cpu_info.sh @@ -1,5 +1,19 @@ #!/bin/bash +# Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + if [ "`uname -s`" != "Linux" ]; then echo "Current scenario only support in Linux yet!" exit 0 From 6f0b2e18463d06e33d5abfe466491c75d928b03d Mon Sep 17 00:00:00 2001 From: seemingwang Date: Mon, 29 Mar 2021 11:23:31 +0000 Subject: [PATCH 082/106] change test_server port --- paddle/fluid/distributed/test/graph_node_test.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 7bf2733e42293..64b822f44b2c5 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -289,7 +289,7 @@ ::paddle::distributed::PSParameter GetWorkerProto() { /*-------------------------------------------------------------------------*/ std::string ip_ = "127.0.0.1", ip2 = "127.0.0.1"; -uint32_t port_ = 4209, port2 = 4210; +uint32_t port_ = 5209, port2 = 5210; std::vector host_sign_list_; @@ -413,7 +413,7 @@ void RunBrpcPushSparse() { } distributed::GraphPyServer server1, server2; distributed::GraphPyClient client1, client2; - std::string ips_str = "127.0.0.1:4211;127.0.0.1:4212"; + std::string ips_str = "127.0.0.1:5211;127.0.0.1:5212"; std::vector edge_types = {std::string("user2item")}; std::vector node_types = {std::string("user"), std::string("item")}; From 771be003643e1b3e41a62554bd2be3025cb106d2 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 30 Mar 2021 09:55:55 +0000 Subject: [PATCH 083/106] fix code problems --- .../distributed/service/brpc_ps_client.h | 12 ------ .../distributed/service/graph_brpc_client.h | 2 + .../distributed/service/graph_brpc_server.h | 17 -------- paddle/fluid/distributed/service/ps_client.h | 39 ------------------- paddle/fluid/distributed/table/CMakeLists.txt | 4 +- paddle/fluid/distributed/table/graph_node.h | 2 +- ...d_sampler.cc => graph_weighted_sampler.cc} | 2 +- ...ted_sampler.h => graph_weighted_sampler.h} | 0 8 files changed, 6 insertions(+), 72 deletions(-) rename paddle/fluid/distributed/table/{weighted_sampler.cc => graph_weighted_sampler.cc} (98%) rename paddle/fluid/distributed/table/{weighted_sampler.h => graph_weighted_sampler.h} (100%) diff --git a/paddle/fluid/distributed/service/brpc_ps_client.h b/paddle/fluid/distributed/service/brpc_ps_client.h index 5b6911564419d..84a31fdbd5df8 100644 --- a/paddle/fluid/distributed/service/brpc_ps_client.h +++ b/paddle/fluid/distributed/service/brpc_ps_client.h @@ -197,16 +197,6 @@ class BrpcPsClient : public PSClient { std::future send_save_cmd(uint32_t table_id, int cmd_id, const std::vector ¶m); - // inline brpc::Channel *get_sparse_channel(size_t server_id) { - // return _server_channels[server_id][0].get(); - // } - // inline brpc::Channel *get_dense_channel(size_t server_id) { - // return _server_channels[server_id][1].get(); - // } - // inline brpc::Channel *get_cmd_channel(size_t server_id) { - // return _server_channels[server_id][2].get(); - // } - bool _running = false; bool _flushing = false; std::atomic _async_call_num; //异步请求计数 @@ -233,8 +223,6 @@ class BrpcPsClient : public PSClient { size_t num, void *done) override; - // virtual size_t get_server_nums() { return _server_channels.size(); } - private: int32_t start_client_service(); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 854e0f6ca4ac1..5d89e0fb94ad2 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -62,9 +62,11 @@ class GraphBrpcClient : public BrpcPsClient { public: GraphBrpcClient() {} virtual ~GraphBrpcClient() {} + // given a batch of nodes, sample graph_neighboors for each of them virtual std::future batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, std::vector>>& res); + virtual std::future pull_graph_list(uint32_t table_id, int server_index, int start, int size, int step, diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index 0badfd4313eb0..657c16e835dc0 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -51,14 +51,7 @@ class GraphBrpcServer : public PSServer { std::shared_ptr _service; std::vector> _pserver_channels; }; -// class GraphBrpcServer : public GraphBrpcServer { -// public: -// GraphBrpcServer() {} -// virtual ~GraphBrpcServer() {} -// private: -// virtual int32_t initialize(); -// }; class GraphBrpcService; typedef int32_t (GraphBrpcService::*serviceFunc)( @@ -113,16 +106,6 @@ class GraphBrpcService : public PsBaseService { std::vector _ori_values; const int sample_nodes_ranges = 3; }; -// class GraphBrpcService : public BrpcPsService { -// public: -// virtual int32_t initialize() override; -// int32_t pull_graph_list(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller -// *cntl); -// int32_t graph_random_sample(Table *table, const PsRequestMessage &request, -// PsResponseMessage &response, brpc::Controller -// *cntl); -// }; } // namespace distributed } // namespace paddle diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 9fd0bedf23ee9..7b698afa72645 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -155,46 +155,7 @@ class PSClient { promise.set_value(-1); return fut; } - virtual std::future batch_sample_neighboors( - uint32_t table_id, std::vector node_ids, int sample_size, - std::vector>> &res) { - LOG(FATAL) << "Did not implement"; - std::promise promise; - std::future fut = promise.get_future(); - promise.set_value(-1); - return fut; - } - virtual std::future pull_graph_list(uint32_t table_id, - int server_index, int start, - int size, int step, - std::vector &res) { - LOG(FATAL) << "Did not implement"; - std::promise promise; - std::future fut = promise.get_future(); - promise.set_value(-1); - return fut; - } - virtual std::future random_sample_nodes(uint32_t table_id, - int server_index, - int sample_size, - std::vector &ids) { - LOG(FATAL) << "Did not implement"; - std::promise promise; - std::future fut = promise.get_future(); - promise.set_value(-1); - return fut; - } - virtual std::future get_node_feat( - const uint32_t &table_id, const std::vector &node_ids, - const std::vector &feature_names, - std::vector> &res) { - LOG(FATAL) << "Did not implement"; - std::promise promise; - std::future fut = promise.get_future(); - promise.set_value(-1); - return fut; - } // client2client消息处理,std::function ret (msg_type, from_client_id, msg) typedef std::function MsgHandlerFunc; diff --git a/paddle/fluid/distributed/table/CMakeLists.txt b/paddle/fluid/distributed/table/CMakeLists.txt index ede1c88d757bc..33873abc5f7f5 100644 --- a/paddle/fluid/distributed/table/CMakeLists.txt +++ b/paddle/fluid/distributed/table/CMakeLists.txt @@ -3,8 +3,8 @@ set_property(GLOBAL PROPERTY TABLE_DEPS string_helper) get_property(TABLE_DEPS GLOBAL PROPERTY TABLE_DEPS) set_source_files_properties(graph_edge.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_library(graph_edge SRCS graph_edge.cc) -set_source_files_properties(weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(WeightedSampler SRCS weighted_sampler.cc DEPS graph_edge) +set_source_files_properties(graph_weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(WeightedSampler SRCS graph_weighted_sampler.cc DEPS graph_edge) set_source_files_properties(graph_node.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) cc_library(graph_node SRCS graph_node.cc DEPS WeightedSampler) set_source_files_properties(common_dense_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h index 2e1b84a2af5be..c3e8e3ce5b50d 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph_node.h @@ -17,7 +17,7 @@ #include #include #include -#include "paddle/fluid/distributed/table/weighted_sampler.h" +#include "paddle/fluid/distributed/table/graph_weighted_sampler.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/weighted_sampler.cc b/paddle/fluid/distributed/table/graph_weighted_sampler.cc similarity index 98% rename from paddle/fluid/distributed/table/weighted_sampler.cc rename to paddle/fluid/distributed/table/graph_weighted_sampler.cc index 69f845843007c..059a1d64bc392 100644 --- a/paddle/fluid/distributed/table/weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph_weighted_sampler.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/distributed/table/weighted_sampler.h" +#include "paddle/fluid/distributed/table/graph_weighted_sampler.h" #include #include namespace paddle { diff --git a/paddle/fluid/distributed/table/weighted_sampler.h b/paddle/fluid/distributed/table/graph_weighted_sampler.h similarity index 100% rename from paddle/fluid/distributed/table/weighted_sampler.h rename to paddle/fluid/distributed/table/graph_weighted_sampler.h From 1a3013aca908c811f50570ff5c180ea03bc012b4 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 30 Mar 2021 12:08:10 +0000 Subject: [PATCH 084/106] remove shard_num config --- paddle/fluid/distributed/ps.proto | 1 - .../distributed/service/graph_brpc_client.cc | 2 +- .../distributed/service/graph_py_service.cc | 3 +-- .../fluid/distributed/test/graph_node_test.cc | 19 +------------------ 4 files changed, 3 insertions(+), 22 deletions(-) diff --git a/paddle/fluid/distributed/ps.proto b/paddle/fluid/distributed/ps.proto index 665098cd7d547..862ae4a504d9b 100644 --- a/paddle/fluid/distributed/ps.proto +++ b/paddle/fluid/distributed/ps.proto @@ -40,7 +40,6 @@ message PSParameter { optional ServerParameter server_param = 102; repeated DownpourTrainerParameter trainer_param = 301; optional FsClientParameter fs_client_param = 501; - optional int32 shard_num = 502; } message WorkerParameter { diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 14bf6a03763d8..dd056bb098e86 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -324,7 +324,7 @@ std::future GraphBrpcClient::pull_graph_list( return fut; } int32_t GraphBrpcClient::initialize() { - set_shard_num(_config.shard_num()); + // set_shard_num(_config.shard_num()); BrpcPsClient::initialize(); server_size = get_server_nums(); graph_service = NULL; diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 7ed50dbde763b..c096b0453e5e8 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -90,6 +90,7 @@ void GraphPyClient::start_client() { (paddle::distributed::GraphBrpcClient*) paddle::distributed::PSClientFactory::create(worker_proto)); worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); + worker_ptr->set_shard_num(get_shard_num()); } void GraphPyServer::start_server() { std::string ip = server_list[rank]; @@ -119,7 +120,6 @@ void GraphPyServer::start_server() { ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { // Generate server proto desc ::paddle::distributed::PSParameter server_fleet_desc; - server_fleet_desc.set_shard_num(get_shard_num()); ::paddle::distributed::ServerParameter* server_proto = server_fleet_desc.mutable_server_param(); ::paddle::distributed::DownpourServerParameter* downpour_server_proto = @@ -162,7 +162,6 @@ ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { ::paddle::distributed::PSParameter GraphPyClient::GetWorkerProto() { ::paddle::distributed::PSParameter worker_fleet_desc; - worker_fleet_desc.set_shard_num(get_shard_num()); ::paddle::distributed::WorkerParameter* worker_proto = worker_fleet_desc.mutable_worker_param(); diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 64b822f44b2c5..63af2b21395a3 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -214,11 +214,6 @@ void prepare_file(char file_name[], bool load_edge) { ofile << x << std::endl; } } - // for(int i = 0;i < 10;i++){ - // for(int j = 0;j < 10;j++){ - // ofile<configure(worker_proto, dense_regions, _ps_env, 0); + worker_ptr_->set_shard_num(127); worker_ptr_->set_local_channel(index); worker_ptr_->set_local_graph_service( (paddle::distributed::GraphBrpcService*)service); @@ -379,16 +373,6 @@ void RunBrpcPushSparse() { srand(time(0)); pull_status.wait(); std::vector>> vs; - // for(int i = 0;i < 100000000;i++){ - // std::vector nodes; - // pull_status = worker_ptr_->pull_graph_list(0, 0, 0, 1, nodes); - // pull_status.wait(); - // pull_status = worker_ptr_->batch_sample(0, std::vector(1, 37), 4, - // vs); - // pull_status.wait(); - // } - // std::vector> v; - // pull_status = worker_ptr_->sample(0, 37, 4, v); testSampleNodes(worker_ptr_); sleep(5); testSingleSampleNeighboor(worker_ptr_); @@ -560,7 +544,6 @@ void RunBrpcPushSparse() { void testGraphToBuffer() { ::paddle::distributed::GraphNode s, s1; - // s.add_feature("hhhh"); s.set_feature_size(1); s.set_feature(0, std::string("hhhh")); s.set_id(65); From bd6b545b626db99064f91037db3ab8c22b848fc4 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 31 Mar 2021 12:44:30 +0000 Subject: [PATCH 085/106] remove redundent threads --- .../distributed/service/graph_py_service.cc | 34 ++++++++----------- .../distributed/service/graph_py_service.h | 4 +-- 2 files changed, 15 insertions(+), 23 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index c096b0453e5e8..9aea281f945fa 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -72,9 +72,6 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, host_sign_list.push_back(ph_host.serialize_to_string()); index++; } - // VLOG(0) << "IN set up rank = " << rank; - // start_client(); - // start_server(server_list[rank], std::stoul(port_list[rank])); } void GraphPyClient::start_client() { std::map> dense_regions; @@ -96,25 +93,22 @@ void GraphPyServer::start_server() { std::string ip = server_list[rank]; uint32_t port = std::stoul(port_list[rank]); server_thread = new std::thread([this, &ip, &port]() { - std::function func = [this, &ip, &port]() { - ::paddle::distributed::PSParameter server_proto = this->GetServerProto(); + ::paddle::distributed::PSParameter server_proto = this->GetServerProto(); - auto _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&this->host_sign_list, - this->host_sign_list.size()); // test - pserver_ptr = std::shared_ptr( - (paddle::distributed::GraphBrpcServer*) - paddle::distributed::PSServerFactory::create(server_proto)); - VLOG(0) << "pserver-ptr created "; - std::vector empty_vec; - framework::ProgramDesc empty_prog; - empty_vec.push_back(empty_prog); - pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); - pserver_ptr->start(ip, port); - }; - std::thread t1(func); - t1.join(); + auto _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&this->host_sign_list, + this->host_sign_list.size()); // test + pserver_ptr = std::shared_ptr( + (paddle::distributed::GraphBrpcServer*) + paddle::distributed::PSServerFactory::create(server_proto)); + VLOG(0) << "pserver-ptr created "; + std::vector empty_vec; + framework::ProgramDesc empty_prog; + empty_vec.push_back(empty_prog); + pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); + pserver_ptr->start(ip, port); }); + server_thread->detach(); sleep(3); } ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 375f8f4560ea8..2637db9a7b607 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -151,9 +151,7 @@ class GraphPyClient : public GraphPyService { (paddle::distributed::GraphBrpcService*)server.get_ps_server() ->get_service()); } - // void set_local_graph_service(GraphBrpcService *service){ - // worker_ptr->set_local_graph_service(service); - // } + void load_edge_file(std::string name, std::string filepath, bool reverse); void load_node_file(std::string name, std::string filepath); int get_client_id() { return client_id; } From a768899ac9555b8f2cad761219f41b1a2c11b413 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 1 Apr 2021 07:52:58 +0000 Subject: [PATCH 086/106] optimize start server --- .../distributed/service/graph_brpc_server.cc | 15 ++---- .../distributed/service/graph_brpc_server.h | 8 +-- .../distributed/service/graph_py_service.cc | 53 ++++++++++++------- .../distributed/service/graph_py_service.h | 11 ++-- .../fluid/distributed/test/graph_node_test.cc | 32 +++++------ paddle/fluid/pybind/fleet_py.cc | 1 + 6 files changed, 65 insertions(+), 55 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 9585134d8637e..f2d02f91e9d43 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -68,17 +68,9 @@ uint64_t GraphBrpcServer::start(const std::string &ip, uint32_t port) { LOG(ERROR) << "GraphBrpcServer start failed, ip_port=" << ip_port; return 0; } - VLOG(0) << "GraphBrpcServer::start registe_ps_server"; _environment->registe_ps_server(ip, port, _rank); - VLOG(0) << "GraphBrpcServer::start wait"; - cv_.wait(lock, [&] { return stoped_; }); - - PSHost host; - host.ip = ip; - host.port = port; - host.rank = _rank; - VLOG(0) << "GraphBrpcServer::start return host.rank"; - return host.rank; + // cv_.wait(lock, [&] { return stoped_; }); + return 0; } int32_t GraphBrpcServer::port() { return _server.listen_address().port; } @@ -232,11 +224,12 @@ int32_t GraphBrpcService::stop_server(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl) { - auto *p_server = _server; + GraphBrpcServer *p_server = (GraphBrpcServer *)_server; std::thread t_stop([p_server]() { p_server->stop(); LOG(INFO) << "Server Stoped"; }); + p_server->export_cv()->notify_all(); t_stop.detach(); return 0; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index 657c16e835dc0..af63bf5d99ef2 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -33,15 +33,17 @@ class GraphBrpcServer : public PSServer { virtual uint64_t start(const std::string &ip, uint32_t port); virtual int32_t stop() { std::unique_lock lock(mutex_); + if (stoped_) return 0; stoped_ = true; - cv_.notify_all(); - + // cv_.notify_all(); _server.Stop(1000); _server.Join(); return 0; } virtual int32_t port(); + std::condition_variable *export_cv() { return &cv_; } + private: virtual int32_t initialize(); mutable std::mutex mutex_; @@ -104,7 +106,7 @@ class GraphBrpcService : public PsBaseService { std::mutex _initialize_shard_mutex; std::unordered_map _msg_handler_map; std::vector _ori_values; - const int sample_nodes_ranges = 3; + const int sample_nodes_ranges = 23; }; } // namespace distributed diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 9aea281f945fa..6a281918ed7c6 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -49,8 +49,6 @@ void GraphPyService::set_up(std::string ips_str, int shard_num, std::vector edge_types) { set_shard_num(shard_num); set_num_node_types(node_types.size()); - // set_client_Id(client_id); - // set_rank(rank); for (size_t table_id = 0; table_id < node_types.size(); table_id++) { this->table_id_map[node_types[table_id]] = this->table_id_map.size(); @@ -89,27 +87,33 @@ void GraphPyClient::start_client() { worker_ptr->configure(worker_proto, dense_regions, _ps_env, client_id); worker_ptr->set_shard_num(get_shard_num()); } -void GraphPyServer::start_server() { +void GraphPyServer::start_server(bool block) { std::string ip = server_list[rank]; uint32_t port = std::stoul(port_list[rank]); - server_thread = new std::thread([this, &ip, &port]() { - ::paddle::distributed::PSParameter server_proto = this->GetServerProto(); + ::paddle::distributed::PSParameter server_proto = this->GetServerProto(); - auto _ps_env = paddle::distributed::PaddlePSEnvironment(); - _ps_env.set_ps_servers(&this->host_sign_list, - this->host_sign_list.size()); // test - pserver_ptr = std::shared_ptr( - (paddle::distributed::GraphBrpcServer*) - paddle::distributed::PSServerFactory::create(server_proto)); - VLOG(0) << "pserver-ptr created "; - std::vector empty_vec; - framework::ProgramDesc empty_prog; - empty_vec.push_back(empty_prog); - pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); - pserver_ptr->start(ip, port); - }); - server_thread->detach(); - sleep(3); + auto _ps_env = paddle::distributed::PaddlePSEnvironment(); + _ps_env.set_ps_servers(&this->host_sign_list, + this->host_sign_list.size()); // test + pserver_ptr = std::shared_ptr( + (paddle::distributed::GraphBrpcServer*) + paddle::distributed::PSServerFactory::create(server_proto)); + VLOG(0) << "pserver-ptr created "; + std::vector empty_vec; + framework::ProgramDesc empty_prog; + empty_vec.push_back(empty_prog); + pserver_ptr->configure(server_proto, _ps_env, rank, empty_vec); + pserver_ptr->start(ip, port); + std::condition_variable* cv_ = pserver_ptr->export_cv(); + if (block) { + std::mutex mutex_; + std::unique_lock lock(mutex_); + cv_->wait(lock); + } + + // }); + // server_thread->detach(); + // sleep(3); } ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { // Generate server proto desc @@ -312,5 +316,14 @@ std::vector GraphPyClient::pull_graph_list(std::string name, } return res; } + +void GraphPyClient::stop_server() { + VLOG(0) << "going to stop server"; + std::unique_lock lock(mutex_); + if (stoped_) return; + auto status = this->worker_ptr->stop_server(); + if (status.get() == 0) stoped_ = true; +} +void GraphPyClient::finalize_worker() { this->worker_ptr->finalize_worker(); } } } diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index 2637db9a7b607..e185f23e3d240 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -120,10 +120,8 @@ class GraphPyServer : public GraphPyService { } int get_rank() { return rank; } void set_rank(int rank) { this->rank = rank; } - // paddle::distributed::GraphBrpcService * get_service(){ - // return pserver_ptr->get_service(); - // } - void start_server(); + + void start_server(bool block = true); ::paddle::distributed::PSParameter GetServerProto(); std::shared_ptr get_ps_server() { return pserver_ptr; @@ -151,7 +149,8 @@ class GraphPyClient : public GraphPyService { (paddle::distributed::GraphBrpcService*)server.get_ps_server() ->get_service()); } - + void stop_server(); + void finalize_worker(); void load_edge_file(std::string name, std::string filepath, bool reverse); void load_node_file(std::string name, std::string filepath); int get_client_id() { return client_id; } @@ -169,9 +168,11 @@ class GraphPyClient : public GraphPyService { ::paddle::distributed::PSParameter GetWorkerProto(); protected: + mutable std::mutex mutex_; int client_id; std::shared_ptr worker_ptr; std::thread* client_thread; + bool stoped_ = false; }; } } diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 63af2b21395a3..92faf9cc96c54 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -356,8 +356,8 @@ void RunBrpcPushSparse() { host_sign_list_.push_back(ph_host2.serialize_to_string()); // test-end // Srart Server - std::thread server_thread(RunServer); - std::thread server_thread2(RunServer2); + std::thread* server_thread = new std::thread(RunServer); + std::thread* server_thread2 = new std::thread(RunServer2); sleep(1); std::map> dense_regions; @@ -433,9 +433,9 @@ void RunBrpcPushSparse() { client2.add_table_feat_conf("user", "d", "string", 1); client2.add_table_feat_conf("item", "a", "float32", 1); - server1.start_server(); + server1.start_server(false); std::cout << "first server done" << std::endl; - server2.start_server(); + server2.start_server(false); std::cout << "second server done" << std::endl; client1.start_client(); std::cout << "first client done" << std::endl; @@ -451,8 +451,6 @@ void RunBrpcPushSparse() { client1.load_node_file(std::string("item"), std::string(node_file_name)); client1.load_edge_file(std::string("user2item"), std::string(edge_file_name), 0); - // client2.load_edge_file(std::string("user2item"), std::string(file_name), - // 0); nodes.clear(); nodes = client1.pull_graph_list(std::string("user"), 0, 1, 4, 1); @@ -505,10 +503,10 @@ void RunBrpcPushSparse() { client1.get_node_feat(std::string("user"), node_ids, feature_names); ASSERT_EQ(node_feat.size(), 2); ASSERT_EQ(node_feat[0].size(), 2); - std::cout << "get_node_feat: " << node_feat[0][0] << std::endl; - std::cout << "get_node_feat: " << node_feat[0][1] << std::endl; - std::cout << "get_node_feat: " << node_feat[1][0] << std::endl; - std::cout << "get_node_feat: " << node_feat[1][1] << std::endl; + VLOG(0) << "get_node_feat: " << node_feat[0][0]; + VLOG(0) << "get_node_feat: " << node_feat[0][1]; + VLOG(0) << "get_node_feat: " << node_feat[1][0]; + VLOG(0) << "get_node_feat: " << node_feat[1][1]; // Test string node_ids.clear(); @@ -522,10 +520,10 @@ void RunBrpcPushSparse() { client1.get_node_feat(std::string("user"), node_ids, feature_names); ASSERT_EQ(node_feat.size(), 2); ASSERT_EQ(node_feat[0].size(), 2); - std::cout << "get_node_feat: " << node_feat[0][0].size() << std::endl; - std::cout << "get_node_feat: " << node_feat[0][1].size() << std::endl; - std::cout << "get_node_feat: " << node_feat[1][0].size() << std::endl; - std::cout << "get_node_feat: " << node_feat[1][1].size() << std::endl; + VLOG(0) << "get_node_feat: " << node_feat[0][0].size(); + VLOG(0) << "get_node_feat: " << node_feat[0][1].size(); + VLOG(0) << "get_node_feat: " << node_feat[1][0].size(); + VLOG(0) << "get_node_feat: " << node_feat[1][1].size(); std::remove(edge_file_name); std::remove(node_file_name); @@ -533,13 +531,15 @@ void RunBrpcPushSparse() { worker_ptr_->stop_server(); LOG(INFO) << "Run finalize_worker"; worker_ptr_->finalize_worker(); - server_thread.join(); - server_thread2.join(); + + // server_thread.join(); + // server_thread2.join(); testFeatureNodeSerializeInt(); testFeatureNodeSerializeInt64(); testFeatureNodeSerializeFloat32(); testFeatureNodeSerializeFloat64(); testGraphToBuffer(); + client1.stop_server(); } void testGraphToBuffer() { diff --git a/paddle/fluid/pybind/fleet_py.cc b/paddle/fluid/pybind/fleet_py.cc index a5a5db65bc335..0a2159667f352 100644 --- a/paddle/fluid/pybind/fleet_py.cc +++ b/paddle/fluid/pybind/fleet_py.cc @@ -194,6 +194,7 @@ void BindGraphPyClient(py::module* m) { .def("start_client", &GraphPyClient::start_client) .def("batch_sample_neighboors", &GraphPyClient::batch_sample_neighboors) .def("random_sample_nodes", &GraphPyClient::random_sample_nodes) + .def("stop_server", &GraphPyClient::stop_server) .def("get_node_feat", [](GraphPyClient& self, std::string node_type, std::vector node_ids, From 085bca68dba71624fcf9364dcc1894d28bdeef35 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 1 Apr 2021 07:56:02 +0000 Subject: [PATCH 087/106] remove logs --- paddle/fluid/distributed/service/graph_brpc_server.cc | 1 - paddle/fluid/distributed/service/graph_py_service.cc | 4 ---- paddle/fluid/distributed/test/graph_node_test.cc | 3 --- 3 files changed, 8 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index f2d02f91e9d43..4f6cc1143e96b 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -69,7 +69,6 @@ uint64_t GraphBrpcServer::start(const std::string &ip, uint32_t port) { return 0; } _environment->registe_ps_server(ip, port, _rank); - // cv_.wait(lock, [&] { return stoped_; }); return 0; } diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 6a281918ed7c6..61e4e0cf7bb91 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -110,10 +110,6 @@ void GraphPyServer::start_server(bool block) { std::unique_lock lock(mutex_); cv_->wait(lock); } - - // }); - // server_thread->detach(); - // sleep(3); } ::paddle::distributed::PSParameter GraphPyServer::GetServerProto() { // Generate server proto desc diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 92faf9cc96c54..79ab279596384 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -531,9 +531,6 @@ void RunBrpcPushSparse() { worker_ptr_->stop_server(); LOG(INFO) << "Run finalize_worker"; worker_ptr_->finalize_worker(); - - // server_thread.join(); - // server_thread2.join(); testFeatureNodeSerializeInt(); testFeatureNodeSerializeInt64(); testFeatureNodeSerializeFloat32(); From 0f85b95c2f6666424188f91bee7eca7b173e7ad8 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 1 Apr 2021 08:31:34 +0000 Subject: [PATCH 088/106] fix code problems by reviewers' suggestions --- paddle/fluid/distributed/service/graph_brpc_client.cc | 1 - 1 file changed, 1 deletion(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index 81089302f45ee..a6271cac83c9a 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -26,7 +26,6 @@ #include "paddle/fluid/string/string_helper.h" namespace paddle { namespace distributed { -int GraphPsService_Stub::thread_num = 3; void GraphPsService_Stub::service( ::google::protobuf::RpcController *controller, From 2afea69b51088b99a3d26e9f0decb8ad6ab9ce23 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 6 Apr 2021 12:21:06 +0000 Subject: [PATCH 089/106] move graph files into a folder --- paddle/fluid/distributed/service/ps_client.h | 2 +- paddle/fluid/distributed/table/CMakeLists.txt | 14 +++++++------- .../fluid/distributed/table/common_graph_table.cc | 2 +- .../fluid/distributed/table/common_graph_table.h | 2 +- .../distributed/table/{ => graph}/graph_edge.cc | 2 +- .../distributed/table/{ => graph}/graph_edge.h | 0 .../distributed/table/{ => graph}/graph_node.cc | 2 +- .../distributed/table/{ => graph}/graph_node.h | 2 +- .../table/{ => graph}/graph_weighted_sampler.cc | 2 +- .../table/{ => graph}/graph_weighted_sampler.h | 2 +- paddle/fluid/distributed/table/table.h | 2 +- paddle/fluid/distributed/test/graph_node_test.cc | 2 +- 12 files changed, 17 insertions(+), 17 deletions(-) rename paddle/fluid/distributed/table/{ => graph}/graph_edge.cc (93%) rename paddle/fluid/distributed/table/{ => graph}/graph_edge.h (100%) rename paddle/fluid/distributed/table/{ => graph}/graph_node.cc (98%) rename paddle/fluid/distributed/table/{ => graph}/graph_node.h (98%) rename paddle/fluid/distributed/table/{ => graph}/graph_weighted_sampler.cc (98%) rename paddle/fluid/distributed/table/{ => graph}/graph_weighted_sampler.h (96%) diff --git a/paddle/fluid/distributed/service/ps_client.h b/paddle/fluid/distributed/service/ps_client.h index 7b698afa72645..cc332b5659ed9 100644 --- a/paddle/fluid/distributed/service/ps_client.h +++ b/paddle/fluid/distributed/service/ps_client.h @@ -24,7 +24,7 @@ #include "paddle/fluid/distributed/service/env.h" #include "paddle/fluid/distributed/service/sendrecv.pb.h" #include "paddle/fluid/distributed/table/accessor.h" -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/CMakeLists.txt b/paddle/fluid/distributed/table/CMakeLists.txt index 33873abc5f7f5..dde1f5ae8ee3a 100644 --- a/paddle/fluid/distributed/table/CMakeLists.txt +++ b/paddle/fluid/distributed/table/CMakeLists.txt @@ -1,12 +1,12 @@ set_property(GLOBAL PROPERTY TABLE_DEPS string_helper) - +set(graphDir graph) get_property(TABLE_DEPS GLOBAL PROPERTY TABLE_DEPS) -set_source_files_properties(graph_edge.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(graph_edge SRCS graph_edge.cc) -set_source_files_properties(graph_weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(WeightedSampler SRCS graph_weighted_sampler.cc DEPS graph_edge) -set_source_files_properties(graph_node.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) -cc_library(graph_node SRCS graph_node.cc DEPS WeightedSampler) +set_source_files_properties(${graphDir}/graph_edge.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(graph_edge SRCS ${graphDir}/graph_edge.cc) +set_source_files_properties(${graphDir}/graph_weighted_sampler.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(WeightedSampler SRCS ${graphDir}/graph_weighted_sampler.cc DEPS graph_edge) +set_source_files_properties(${graphDir}/graph_node.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) +cc_library(graph_node SRCS ${graphDir}/graph_node.cc DEPS WeightedSampler) set_source_files_properties(common_dense_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(common_sparse_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) set_source_files_properties(sparse_geo_table.cc PROPERTIES COMPILE_FLAGS ${DISTRIBUTE_COMPILE_FLAGS}) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 995a39a654312..020bcdcc52ef4 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -18,7 +18,7 @@ #include #include #include "paddle/fluid/distributed/common/utils.h" -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/string/printf.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index de3cac134cd51..0e3efbbd0c661 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -26,7 +26,7 @@ #include #include "paddle/fluid/distributed/table/accessor.h" #include "paddle/fluid/distributed/table/common_table.h" -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/framework/rw_lock.h" #include "paddle/fluid/string/string_helper.h" namespace paddle { diff --git a/paddle/fluid/distributed/table/graph_edge.cc b/paddle/fluid/distributed/table/graph/graph_edge.cc similarity index 93% rename from paddle/fluid/distributed/table/graph_edge.cc rename to paddle/fluid/distributed/table/graph/graph_edge.cc index cc90f4c6516c1..0ab0d5a76d671 100644 --- a/paddle/fluid/distributed/table/graph_edge.cc +++ b/paddle/fluid/distributed/table/graph/graph_edge.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/distributed/table/graph_edge.h" +#include "paddle/fluid/distributed/table/graph/graph_edge.h" #include namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/graph_edge.h b/paddle/fluid/distributed/table/graph/graph_edge.h similarity index 100% rename from paddle/fluid/distributed/table/graph_edge.h rename to paddle/fluid/distributed/table/graph/graph_edge.h diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph/graph_node.cc similarity index 98% rename from paddle/fluid/distributed/table/graph_node.cc rename to paddle/fluid/distributed/table/graph/graph_node.cc index 27a2cafaf4f0f..816d31b979072 100644 --- a/paddle/fluid/distributed/table/graph_node.cc +++ b/paddle/fluid/distributed/table/graph/graph_node.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph/graph_node.h similarity index 98% rename from paddle/fluid/distributed/table/graph_node.h rename to paddle/fluid/distributed/table/graph/graph_node.h index c3e8e3ce5b50d..8ad795ac97b54 100644 --- a/paddle/fluid/distributed/table/graph_node.h +++ b/paddle/fluid/distributed/table/graph/graph_node.h @@ -17,7 +17,7 @@ #include #include #include -#include "paddle/fluid/distributed/table/graph_weighted_sampler.h" +#include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc similarity index 98% rename from paddle/fluid/distributed/table/graph_weighted_sampler.cc rename to paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc index 059a1d64bc392..3a680875e3df4 100644 --- a/paddle/fluid/distributed/table/graph_weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "paddle/fluid/distributed/table/graph_weighted_sampler.h" +#include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" #include #include namespace paddle { diff --git a/paddle/fluid/distributed/table/graph_weighted_sampler.h b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h similarity index 96% rename from paddle/fluid/distributed/table/graph_weighted_sampler.h rename to paddle/fluid/distributed/table/graph/graph_weighted_sampler.h index cfc341d27c6b7..1787ab23b0431 100644 --- a/paddle/fluid/distributed/table/graph_weighted_sampler.h +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h @@ -16,7 +16,7 @@ #include #include #include -#include "paddle/fluid/distributed/table/graph_edge.h" +#include "paddle/fluid/distributed/table/graph/graph_edge.h" namespace paddle { namespace distributed { diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index d64e805af40e6..b2cb1d7c2621b 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -21,7 +21,7 @@ #include #include #include "paddle/fluid/distributed/table/accessor.h" -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/platform/device_context.h" diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index 79ab279596384..b268bb449e146 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -33,7 +33,7 @@ limitations under the License. */ #include "paddle/fluid/distributed/service/ps_client.h" #include "paddle/fluid/distributed/service/sendrecv.pb.h" #include "paddle/fluid/distributed/service/service.h" -#include "paddle/fluid/distributed/table/graph_node.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/framework/lod_tensor.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" From 06c6c9b299413bb0cc014b89fb28caf7802f7076 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 7 Apr 2021 07:02:14 +0000 Subject: [PATCH 090/106] code style change --- paddle/fluid/distributed/table/table.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 6d0704cfbeabf..7d49afd92a4bd 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -21,8 +21,8 @@ #include #include #include "paddle/fluid/distributed/table/accessor.h" -#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/distributed/table/depends/sparse_utils.h" +#include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/framework/program_desc.h" #include "paddle/fluid/framework/scope.h" #include "paddle/fluid/platform/device_context.h" From f291ea05e785d4004412f4debc86a34e09dfb0d9 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 7 Apr 2021 07:30:01 +0000 Subject: [PATCH 091/106] remove graph operations from base table --- .../distributed/service/graph_brpc_server.cc | 13 +++++----- .../distributed/service/graph_brpc_server.h | 3 ++- paddle/fluid/distributed/table/table.h | 25 ------------------- 3 files changed, 9 insertions(+), 32 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index 4f6cc1143e96b..bdd926278b624 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -19,7 +19,6 @@ #include "butil/endpoint.h" #include "iomanip" #include "paddle/fluid/distributed/service/brpc_ps_client.h" -#include "paddle/fluid/distributed/table/table.h" #include "paddle/fluid/framework/archive.h" #include "paddle/fluid/platform/profiler.h" namespace paddle { @@ -265,7 +264,8 @@ int32_t GraphBrpcService::pull_graph_list(Table *table, int step = *(int *)(request.params(2).c_str()); std::unique_ptr buffer; int actual_size; - table->pull_graph_list(start, size, buffer, actual_size, false, step); + ((GraphTable *)table) + ->pull_graph_list(start, size, buffer, actual_size, false, step); cntl->response_attachment().append(buffer.get(), actual_size); return 0; } @@ -284,8 +284,8 @@ int32_t GraphBrpcService::graph_random_sample_neighboors( int sample_size = *(uint64_t *)(request.params(1).c_str()); std::vector> buffers(node_num); std::vector actual_sizes(node_num, 0); - table->random_sample_neighboors(node_data, sample_size, buffers, - actual_sizes); + ((GraphTable *)table) + ->random_sample_neighboors(node_data, sample_size, buffers, actual_sizes); cntl->response_attachment().append(&node_num, sizeof(size_t)); cntl->response_attachment().append(actual_sizes.data(), @@ -301,7 +301,8 @@ int32_t GraphBrpcService::graph_random_sample_nodes( size_t size = *(uint64_t *)(request.params(0).c_str()); std::unique_ptr buffer; int actual_size; - if (table->random_sample_nodes(size, buffer, actual_size) == 0) { + if (((GraphTable *)table)->random_sample_nodes(size, buffer, actual_size) == + 0) { cntl->response_attachment().append(buffer.get(), actual_size); } else cntl->response_attachment().append(NULL, 0); @@ -330,7 +331,7 @@ int32_t GraphBrpcService::graph_get_node_feat(Table *table, std::vector> feature( feature_names.size(), std::vector(node_num)); - table->get_node_feat(node_ids, feature_names, feature); + ((GraphTable *)table)->get_node_feat(node_ids, feature_names, feature); for (size_t feat_idx = 0; feat_idx < feature_names.size(); ++feat_idx) { for (size_t node_idx = 0; node_idx < node_num; ++node_idx) { diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index af63bf5d99ef2..32c572f9e6c2b 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -22,7 +22,8 @@ #include #include "paddle/fluid/distributed/service/brpc_ps_server.h" #include "paddle/fluid/distributed/service/server.h" - +#include "paddle/fluid/distributed/table/common_graph_table.h" +#include "paddle/fluid/distributed/table/table.h" namespace paddle { namespace distributed { class GraphBrpcServer : public PSServer { diff --git a/paddle/fluid/distributed/table/table.h b/paddle/fluid/distributed/table/table.h index 7d49afd92a4bd..5bc818ff4741f 100644 --- a/paddle/fluid/distributed/table/table.h +++ b/paddle/fluid/distributed/table/table.h @@ -88,31 +88,6 @@ class Table { return 0; } - // only for graph table - virtual int32_t pull_graph_list(int start, int total_size, - std::unique_ptr &buffer, - int &actual_size, bool need_feature, - int step = 1) { - return 0; - } - // only for graph table - virtual int32_t random_sample_neighboors( - uint64_t *node_ids, int sample_size, - std::vector> &buffers, - std::vector &actual_sizes) { - return 0; - } - - virtual int32_t random_sample_nodes(int sample_size, - std::unique_ptr &buffers, - int &actual_sizes) { - return 0; - } - virtual int32_t get_node_feat(const std::vector &node_ids, - const std::vector &feature_names, - std::vector> &res) { - return 0; - } virtual int32_t pour() { return 0; } virtual void clear() = 0; From 59fa61447a75a9e339c947e147b09778623a6441 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 14 Apr 2021 07:06:15 +0000 Subject: [PATCH 092/106] optimize get_feat function of graph engine --- paddle/fluid/distributed/service/graph_brpc_client.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index a6271cac83c9a..eafb4d596cc16 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -135,7 +135,8 @@ std::future GraphBrpcClient::get_node_feat( closure->request(request_idx) ->add_params(joint_feature_name.c_str(), joint_feature_name.size()); - PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = + getServiceStub(get_cmd_channel(server_index)); closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), closure->response(request_idx), closure); From faadcbadebcfffbd8330661d855780882b21c694 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 21 Apr 2021 07:39:17 +0000 Subject: [PATCH 093/106] fix long long count problem --- .../distributed/service/graph_py_service.h | 12 ----------- .../distributed/table/common_graph_table.cc | 2 +- .../distributed/table/common_graph_table.h | 20 ++----------------- 3 files changed, 3 insertions(+), 31 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index e185f23e3d240..c6657be96ba44 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -54,19 +54,7 @@ class GraphPyService { std::vector table_feat_conf_feat_dtype; std::vector table_feat_conf_feat_shape; - // std::thread *server_thread, *client_thread; - - // std::shared_ptr pserver_ptr; - - // std::shared_ptr worker_ptr; - public: - // std::shared_ptr get_ps_server() { - // return pserver_ptr; - // } - // std::shared_ptr get_ps_client() { - // return worker_ptr; - // } int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } void GetDownpourSparseTableProto( diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 020bcdcc52ef4..0dc99de1bfe82 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -171,7 +171,7 @@ int32_t GraphTable::load_nodes(const std::string &path, std::string node_type) { int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { auto paths = paddle::string::split_string(path, ";"); - int count = 0; + int64_t count = 0; std::string sample_type = "random"; bool is_weighted = false; int valid_count = 0; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 8ddf3c8f904a6..b18da82abe61c 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -33,26 +33,11 @@ namespace paddle { namespace distributed { class GraphShard { public: - // static int bucket_low_bound; - // static int gcd(int s, int t) { - // if (s % t == 0) return t; - // return gcd(t, s % t); - // } size_t get_size(); GraphShard() {} - GraphShard(int shard_num) { - this->shard_num = shard_num; - // bucket_size = init_bucket_size(shard_num); - // bucket.resize(bucket_size); - } + GraphShard(int shard_num) { this->shard_num = shard_num; } std::vector &get_bucket() { return bucket; } std::vector get_batch(int start, int end, int step); - // int init_bucket_size(int shard_num) { - // for (int i = bucket_low_bound;; i++) { - // if (gcd(i, shard_num) == 1) return i; - // } - // return -1; - // } std::vector get_ids_by_range(int start, int end) { std::vector res; for (int i = start; i < end && i < bucket.size(); i++) { @@ -64,7 +49,6 @@ class GraphShard { FeatureNode *add_feature_node(uint64_t id); Node *find_node(uint64_t id); void add_neighboor(uint64_t id, uint64_t dst_id, float weight); - // std::unordered_map::iterator> std::unordered_map get_node_location() { return node_location; } @@ -131,7 +115,7 @@ class GraphTable : public SparseTable { protected: std::vector shards; size_t shard_start, shard_end, server_num, shard_num_per_table, shard_num; - const int task_pool_size_ = 11; + const int task_pool_size_ = 24; const int random_sample_nodes_ranges = 3; std::vector feat_name; From 7ced0155829912cb4a4e6aa36f659dd26acfdb2a Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sat, 15 May 2021 07:05:14 +0000 Subject: [PATCH 094/106] remove redandunt graph files --- paddle/fluid/distributed/table/graph_edge.cc | 29 ---- paddle/fluid/distributed/table/graph_edge.h | 46 ------ paddle/fluid/distributed/table/graph_node.cc | 117 -------------- paddle/fluid/distributed/table/graph_node.h | 127 --------------- .../table/graph_weighted_sampler.cc | 150 ------------------ .../table/graph_weighted_sampler.h | 58 ------- 6 files changed, 527 deletions(-) delete mode 100644 paddle/fluid/distributed/table/graph_edge.cc delete mode 100644 paddle/fluid/distributed/table/graph_edge.h delete mode 100644 paddle/fluid/distributed/table/graph_node.cc delete mode 100644 paddle/fluid/distributed/table/graph_node.h delete mode 100644 paddle/fluid/distributed/table/graph_weighted_sampler.cc delete mode 100644 paddle/fluid/distributed/table/graph_weighted_sampler.h diff --git a/paddle/fluid/distributed/table/graph_edge.cc b/paddle/fluid/distributed/table/graph_edge.cc deleted file mode 100644 index cc90f4c6516c1..0000000000000 --- a/paddle/fluid/distributed/table/graph_edge.cc +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/graph_edge.h" -#include -namespace paddle { -namespace distributed { - -void GraphEdgeBlob::add_edge(uint64_t id, float weight = 1) { - id_arr.push_back(id); -} - -void WeightedGraphEdgeBlob::add_edge(uint64_t id, float weight = 1) { - id_arr.push_back(id); - weight_arr.push_back(weight); -} -} -} diff --git a/paddle/fluid/distributed/table/graph_edge.h b/paddle/fluid/distributed/table/graph_edge.h deleted file mode 100644 index 3dfe5a6f357a7..0000000000000 --- a/paddle/fluid/distributed/table/graph_edge.h +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - -#pragma once -#include -#include -#include -namespace paddle { -namespace distributed { - -class GraphEdgeBlob { - public: - GraphEdgeBlob() {} - virtual ~GraphEdgeBlob() {} - size_t size() { return id_arr.size(); } - virtual void add_edge(uint64_t id, float weight); - uint64_t get_id(int idx) { return id_arr[idx]; } - virtual float get_weight(int idx) { return 1; } - - protected: - std::vector id_arr; -}; - -class WeightedGraphEdgeBlob : public GraphEdgeBlob { - public: - WeightedGraphEdgeBlob() {} - virtual ~WeightedGraphEdgeBlob() {} - virtual void add_edge(uint64_t id, float weight); - virtual float get_weight(int idx) { return weight_arr[idx]; } - - protected: - std::vector weight_arr; -}; -} -} diff --git a/paddle/fluid/distributed/table/graph_node.cc b/paddle/fluid/distributed/table/graph_node.cc deleted file mode 100644 index 27a2cafaf4f0f..0000000000000 --- a/paddle/fluid/distributed/table/graph_node.cc +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/graph_node.h" -#include -namespace paddle { -namespace distributed { - -GraphNode::~GraphNode() { - if (sampler != nullptr) { - delete sampler; - sampler = nullptr; - } - if (edges != nullptr) { - delete edges; - edges = nullptr; - } -} - -int Node::weight_size = sizeof(float); -int Node::id_size = sizeof(uint64_t); -int Node::int_size = sizeof(int); - -int Node::get_size(bool need_feature) { return id_size + int_size; } - -void Node::to_buffer(char* buffer, bool need_feature) { - memcpy(buffer, &id, id_size); - buffer += id_size; - - int feat_num = 0; - memcpy(buffer, &feat_num, sizeof(int)); -} - -void Node::recover_from_buffer(char* buffer) { memcpy(&id, buffer, id_size); } - -int FeatureNode::get_size(bool need_feature) { - int size = id_size + int_size; // id, feat_num - if (need_feature) { - size += feature.size() * int_size; - for (const std::string& fea : feature) { - size += fea.size(); - } - } - return size; -} - -void GraphNode::build_edges(bool is_weighted) { - if (edges == nullptr) { - if (is_weighted == true) { - edges = new WeightedGraphEdgeBlob(); - } else { - edges = new GraphEdgeBlob(); - } - } -} -void GraphNode::build_sampler(std::string sample_type) { - if (sample_type == "random") { - sampler = new RandomSampler(); - } else if (sample_type == "weighted") { - sampler = new WeightedSampler(); - } - sampler->build(edges); -} -void FeatureNode::to_buffer(char* buffer, bool need_feature) { - memcpy(buffer, &id, id_size); - buffer += id_size; - - int feat_num = 0; - int feat_len; - if (need_feature) { - feat_num += feature.size(); - memcpy(buffer, &feat_num, sizeof(int)); - buffer += sizeof(int); - for (int i = 0; i < feat_num; ++i) { - feat_len = feature[i].size(); - memcpy(buffer, &feat_len, sizeof(int)); - buffer += sizeof(int); - memcpy(buffer, feature[i].c_str(), feature[i].size()); - buffer += feature[i].size(); - } - } else { - memcpy(buffer, &feat_num, sizeof(int)); - } -} -void FeatureNode::recover_from_buffer(char* buffer) { - int feat_num, feat_len; - memcpy(&id, buffer, id_size); - buffer += id_size; - - memcpy(&feat_num, buffer, sizeof(int)); - buffer += sizeof(int); - - feature.clear(); - for (int i = 0; i < feat_num; ++i) { - memcpy(&feat_len, buffer, sizeof(int)); - buffer += sizeof(int); - - char str[feat_len + 1]; - memcpy(str, buffer, feat_len); - buffer += feat_len; - str[feat_len] = '\0'; - feature.push_back(std::string(str)); - } -} -} -} diff --git a/paddle/fluid/distributed/table/graph_node.h b/paddle/fluid/distributed/table/graph_node.h deleted file mode 100644 index c3e8e3ce5b50d..0000000000000 --- a/paddle/fluid/distributed/table/graph_node.h +++ /dev/null @@ -1,127 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - -#pragma once -#include -#include -#include -#include -#include "paddle/fluid/distributed/table/graph_weighted_sampler.h" -namespace paddle { -namespace distributed { - -class Node { - public: - Node() {} - Node(uint64_t id) : id(id) {} - virtual ~Node() {} - static int id_size, int_size, weight_size; - uint64_t get_id() { return id; } - void set_id(uint64_t id) { this->id = id; } - - virtual void build_edges(bool is_weighted) {} - virtual void build_sampler(std::string sample_type) {} - virtual void add_edge(uint64_t id, float weight) {} - virtual std::vector sample_k(int k) { return std::vector(); } - virtual uint64_t get_neighbor_id(int idx) { return 0; } - virtual float get_neighbor_weight(int idx) { return 1.; } - - virtual int get_size(bool need_feature); - virtual void to_buffer(char *buffer, bool need_feature); - virtual void recover_from_buffer(char *buffer); - virtual std::string get_feature(int idx) { return std::string(""); } - virtual void set_feature(int idx, std::string str) {} - virtual void set_feature_size(int size) {} - virtual int get_feature_size() { return 0; } - - protected: - uint64_t id; -}; - -class GraphNode : public Node { - public: - GraphNode() : Node(), sampler(nullptr), edges(nullptr) {} - GraphNode(uint64_t id) : Node(id), sampler(nullptr), edges(nullptr) {} - virtual ~GraphNode(); - virtual void build_edges(bool is_weighted); - virtual void build_sampler(std::string sample_type); - virtual void add_edge(uint64_t id, float weight) { - edges->add_edge(id, weight); - } - virtual std::vector sample_k(int k) { return sampler->sample_k(k); } - virtual uint64_t get_neighbor_id(int idx) { return edges->get_id(idx); } - virtual float get_neighbor_weight(int idx) { return edges->get_weight(idx); } - - protected: - Sampler *sampler; - GraphEdgeBlob *edges; -}; - -class FeatureNode : public Node { - public: - FeatureNode() : Node() {} - FeatureNode(uint64_t id) : Node(id) {} - virtual ~FeatureNode() {} - virtual int get_size(bool need_feature); - virtual void to_buffer(char *buffer, bool need_feature); - virtual void recover_from_buffer(char *buffer); - virtual std::string get_feature(int idx) { - if (idx < (int)this->feature.size()) { - return this->feature[idx]; - } else { - return std::string(""); - } - } - - virtual void set_feature(int idx, std::string str) { - if (idx >= (int)this->feature.size()) { - this->feature.resize(idx + 1); - } - this->feature[idx] = str; - } - virtual void set_feature_size(int size) { this->feature.resize(size); } - virtual int get_feature_size() { return this->feature.size(); } - - template - static std::string parse_value_to_bytes(std::vector feat_str) { - T v; - size_t Tsize = sizeof(T) * feat_str.size(); - char buffer[Tsize]; - for (size_t i = 0; i < feat_str.size(); i++) { - std::stringstream ss(feat_str[i]); - ss >> v; - std::memcpy(buffer + sizeof(T) * i, (char *)&v, sizeof(T)); - } - return std::string(buffer, Tsize); - } - - template - static std::vector parse_bytes_to_array(std::string feat_str) { - T v; - std::vector out; - size_t start = 0; - const char *buffer = feat_str.data(); - while (start < feat_str.size()) { - std::memcpy((char *)&v, buffer + start, sizeof(T)); - start += sizeof(T); - out.push_back(v); - } - return out; - } - - protected: - std::vector feature; -}; -} -} diff --git a/paddle/fluid/distributed/table/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph_weighted_sampler.cc deleted file mode 100644 index 059a1d64bc392..0000000000000 --- a/paddle/fluid/distributed/table/graph_weighted_sampler.cc +++ /dev/null @@ -1,150 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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 "paddle/fluid/distributed/table/graph_weighted_sampler.h" -#include -#include -namespace paddle { -namespace distributed { - -void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } - -std::vector RandomSampler::sample_k(int k) { - int n = edges->size(); - if (k > n) { - k = n; - } - struct timespec tn; - clock_gettime(CLOCK_REALTIME, &tn); - srand(tn.tv_nsec); - std::vector sample_result; - std::unordered_map replace_map; - while (k--) { - int rand_int = rand() % n; - auto iter = replace_map.find(rand_int); - if (iter == replace_map.end()) { - sample_result.push_back(rand_int); - } else { - sample_result.push_back(iter->second); - } - - iter = replace_map.find(n - 1); - if (iter == replace_map.end()) { - replace_map[rand_int] = n - 1; - } else { - replace_map[rand_int] = iter->second; - } - --n; - } - return sample_result; -} - -WeightedSampler::WeightedSampler() { - left = nullptr; - right = nullptr; - edges = nullptr; -} - -WeightedSampler::~WeightedSampler() { - if (left != nullptr) { - delete left; - left = nullptr; - } - if (right != nullptr) { - delete right; - right = nullptr; - } -} - -void WeightedSampler::build(GraphEdgeBlob *edges) { - if (left != nullptr) { - delete left; - left = nullptr; - } - if (right != nullptr) { - delete right; - right = nullptr; - } - return build_one((WeightedGraphEdgeBlob *)edges, 0, edges->size()); -} - -void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, - int end) { - count = 0; - this->edges = edges; - if (start + 1 == end) { - left = right = nullptr; - idx = start; - count = 1; - weight = edges->get_weight(idx); - - } else { - left = new WeightedSampler(); - right = new WeightedSampler(); - left->build_one(edges, start, start + (end - start) / 2); - right->build_one(edges, start + (end - start) / 2, end); - weight = left->weight + right->weight; - count = left->count + right->count; - } -} -std::vector WeightedSampler::sample_k(int k) { - if (k > count) { - k = count; - } - std::vector sample_result; - float subtract; - std::unordered_map subtract_weight_map; - std::unordered_map subtract_count_map; - struct timespec tn; - clock_gettime(CLOCK_REALTIME, &tn); - srand(tn.tv_nsec); - while (k--) { - float query_weight = rand() % 100000 / 100000.0; - query_weight *= weight - subtract_weight_map[this]; - sample_result.push_back(sample(query_weight, subtract_weight_map, - subtract_count_map, subtract)); - } - return sample_result; -} - -int WeightedSampler::sample( - float query_weight, - std::unordered_map &subtract_weight_map, - std::unordered_map &subtract_count_map, - float &subtract) { - if (left == nullptr) { - subtract_weight_map[this] = weight; - subtract = weight; - subtract_count_map[this] = 1; - return idx; - } - int left_count = left->count - subtract_count_map[left]; - int right_count = right->count - subtract_count_map[right]; - float left_subtract = subtract_weight_map[left]; - int return_idx; - if (right_count == 0 || - left_count > 0 && left->weight - left_subtract >= query_weight) { - return_idx = left->sample(query_weight, subtract_weight_map, - subtract_count_map, subtract); - } else { - return_idx = - right->sample(query_weight - (left->weight - left_subtract), - subtract_weight_map, subtract_count_map, subtract); - } - subtract_weight_map[this] += subtract; - subtract_count_map[this]++; - return return_idx; -} -} -} diff --git a/paddle/fluid/distributed/table/graph_weighted_sampler.h b/paddle/fluid/distributed/table/graph_weighted_sampler.h deleted file mode 100644 index cfc341d27c6b7..0000000000000 --- a/paddle/fluid/distributed/table/graph_weighted_sampler.h +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright (c) 2021 PaddlePaddle Authors. All Rights Reserved. -// -// 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. - -#pragma once -#include -#include -#include -#include "paddle/fluid/distributed/table/graph_edge.h" -namespace paddle { -namespace distributed { - -class Sampler { - public: - virtual ~Sampler() {} - virtual void build(GraphEdgeBlob *edges) = 0; - virtual std::vector sample_k(int k) = 0; -}; - -class RandomSampler : public Sampler { - public: - virtual ~RandomSampler() {} - virtual void build(GraphEdgeBlob *edges); - virtual std::vector sample_k(int k); - GraphEdgeBlob *edges; -}; - -class WeightedSampler : public Sampler { - public: - WeightedSampler(); - virtual ~WeightedSampler(); - WeightedSampler *left, *right; - float weight; - int count; - int idx; - GraphEdgeBlob *edges; - virtual void build(GraphEdgeBlob *edges); - virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); - virtual std::vector sample_k(int k); - - private: - int sample(float query_weight, - std::unordered_map &subtract_weight_map, - std::unordered_map &subtract_count_map, - float &subtract); -}; -} -} From 20023e911c13cdff646ac01209227d041324a40d Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 18 May 2021 05:56:01 +0000 Subject: [PATCH 095/106] remove unused shell --- scripts/paddle | 169 ------------------------------------------------- 1 file changed, 169 deletions(-) delete mode 100644 scripts/paddle diff --git a/scripts/paddle b/scripts/paddle deleted file mode 100644 index 5f256ccf15791..0000000000000 --- a/scripts/paddle +++ /dev/null @@ -1,169 +0,0 @@ -#!/bin/bash - -function version(){ - echo "PaddlePaddle , compiled with" - echo " with_avx: ON" - echo " with_gpu: OFF" - echo " with_mkl: ON" - echo " with_mkldnn: " - echo " with_python: ON" -} - -function ver2num() { - set -e - # convert version to number. - if [ -z "$1" ]; then # empty argument - printf "%03d%03d%03d%03d%03d" 0 - else - local VERN=$(echo $1 | sed 's#v##g' | sed 's#\.# #g' \ - | sed 's#a# 0 #g' | sed 's#b# 1 #g' | sed 's#rc# 2 #g') - if [ `echo $VERN | wc -w` -eq 3 ] ; then - printf "%03d%03d%03d%03d%03d" $VERN 999 999 - else - printf "%03d%03d%03d%03d%03d" $VERN - fi - fi - set +e -} - -function cpu_config() { - # auto set KMP_AFFINITY and OMP_DYNAMIC from Hyper Threading Status - # only when MKL enabled - if [ "ON" == "OFF" ]; then - return 0 - fi - platform="`uname -s`" - ht=0 - if [ $platform == "Linux" ]; then - ht=`lscpu |grep "per core"|awk -F':' '{print $2}'|xargs` - elif [ $platform == "Darwin" ]; then - if [ `sysctl -n hw.physicalcpu` -eq `sysctl -n hw.logicalcpu` ]; then - # HT is OFF - ht=1 - fi - else - return 0 - fi - if [ $ht -eq 1 ]; then # HT is OFF - if [ -z "$KMP_AFFINITY" ]; then - export KMP_AFFINITY="granularity=fine,compact,0,0" - fi - if [ -z "$OMP_DYNAMIC" ]; then - export OMP_DYNAMIC="FALSE" - fi - else # HT is ON - if [ -z "$KMP_AFFINITY" ]; then - export KMP_AFFINITY="granularity=fine,compact,1,0" - fi - if [ -z "$OMP_DYNAMIC" ]; then - export OMP_DYNAMIC="True" - fi - fi -} - -function threads_config() { - # auto set OMP_NUM_THREADS and MKL_NUM_THREADS - # according to trainer_count and total processors - # only when MKL enabled - # auto set OPENBLAS_NUM_THREADS when do not use MKL - platform="`uname -s`" - processors=0 - if [ $platform == "Linux" ]; then - processors=`grep "processor" /proc/cpuinfo|sort -u|wc -l` - elif [ $platform == "Darwin" ]; then - processors=`sysctl -n hw.logicalcpu` - else - return 0 - fi - trainers=`grep -Eo 'trainer_count.[0-9]+' <<< "$@" |grep -Eo '[0-9]+'|xargs` - if [ -z $trainers ]; then - trainers=1 - fi - threads=$((processors / trainers)) - if [ $threads -eq 0 ]; then - threads=1 - fi - if [ "ON" == "ON" ]; then - if [ -z "$OMP_NUM_THREADS" ]; then - export OMP_NUM_THREADS=$threads - fi - if [ -z "$MKL_NUM_THREADS" ]; then - export MKL_NUM_THREADS=$threads - fi - else - if [ -z "$OPENBLAS_NUM_THREADS" ]; then - export OPENBLAS_NUM_THREADS=$threads - fi - if [ $threads -gt 1 ] && [ -z "$OPENBLAS_MAIN_FREE" ]; then - export OPENBLAS_MAIN_FREE=1 - fi - fi - -} - -PADDLE_CONF_HOME="$HOME/.config/paddle" -mkdir -p ${PADDLE_CONF_HOME} - -if [ -z "${PADDLE_NO_STAT+x}" ]; then - SERVER_VER=`curl -m 5 -X POST --data content="{ \"version\": \"\" }"\ - -b ${PADDLE_CONF_HOME}/paddle.cookie \ - -c ${PADDLE_CONF_HOME}/paddle.cookie \ - http://api.paddlepaddle.org/version 2>/dev/null` - if [ $? -eq 0 ] && [ "$(ver2num )" -lt $(ver2num $SERVER_VER) ]; then - echo "Paddle release a new version ${SERVER_VER}, you can get the install package in http://www.paddlepaddle.org" - fi -fi - -PADDLE_BIN_PATH="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" - -if [ ! -z "${DEBUGGER}" ]; then - echo "Using debug command ${DEBUGGER}" -fi - -CUDNN_LIB_PATH="" - -if [ ! -z "${CUDNN_LIB_PATH}" ]; then - export LD_LIBRARY_PATH=${CUDNN_LIB_PATH}:${LD_LIBRARY_PATH} -fi - -export PYTHONPATH=${PWD}:${PYTHONPATH} - - -# Check python lib installed or not. -pip --help > /dev/null -if [ $? -ne 0 ]; then - echo "pip should be installed to run paddle." - exit 1 -fi - -if [ "OFF" == "ON" ]; then - PADDLE_NAME="paddlepaddle-gpu" -else - PADDLE_NAME="paddlepaddle" -fi - -INSTALLED_VERSION=`pip freeze 2>/dev/null | grep "^${PADDLE_NAME}==" | sed 's/.*==//g'` - -if [ -z "${INSTALLED_VERSION}" ]; then - INSTALLED_VERSION="0.0.0" # not installed -fi -cat < Date: Tue, 18 May 2021 06:54:30 +0000 Subject: [PATCH 096/106] recover dropout_op_pass.h --- paddle/fluid/framework/ir/delete_dropout_op_pass.h | 1 + 1 file changed, 1 insertion(+) diff --git a/paddle/fluid/framework/ir/delete_dropout_op_pass.h b/paddle/fluid/framework/ir/delete_dropout_op_pass.h index 1ba539beaa1c8..c49abf3c871ce 100644 --- a/paddle/fluid/framework/ir/delete_dropout_op_pass.h +++ b/paddle/fluid/framework/ir/delete_dropout_op_pass.h @@ -14,6 +14,7 @@ #pragma once #include + #include "paddle/fluid/framework/ir/fuse_pass_base.h" #include "paddle/fluid/framework/ir/graph_pattern_detector.h" From 5b07c6cd72ff00f97381a95390cce1b1926c147e Mon Sep 17 00:00:00 2001 From: seemingwang Date: Sat, 22 May 2021 12:19:13 +0000 Subject: [PATCH 097/106] fix potential stack overflow when request number is too large & node add & node clear & node remove --- .../distributed/service/graph_brpc_client.cc | 173 +++++++++++++++++- .../distributed/service/graph_brpc_client.h | 7 + .../distributed/service/graph_brpc_server.cc | 74 +++++++- .../distributed/service/graph_brpc_server.h | 7 + .../distributed/service/graph_py_service.cc | 31 ++++ .../distributed/service/graph_py_service.h | 4 + .../fluid/distributed/service/sendrecv.proto | 3 + .../distributed/table/common_graph_table.cc | 106 +++++++++-- .../distributed/table/common_graph_table.h | 13 +- .../fluid/distributed/test/graph_node_test.cc | 53 +++++- 10 files changed, 442 insertions(+), 29 deletions(-) diff --git a/paddle/fluid/distributed/service/graph_brpc_client.cc b/paddle/fluid/distributed/service/graph_brpc_client.cc index eafb4d596cc16..70f2da6d7252c 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.cc +++ b/paddle/fluid/distributed/service/graph_brpc_client.cc @@ -80,11 +80,11 @@ std::future GraphBrpcClient::get_node_feat( [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; - int fail_num = 0; + size_t fail_num = 0; for (int request_idx = 0; request_idx < request_call_num; ++request_idx) { - if (closure->check_response(request_idx, - PS_GRAPH_SAMPLE_NEIGHBOORS) != 0) { + if (closure->check_response(request_idx, PS_GRAPH_GET_NODE_FEAT) != + 0) { ++fail_num; } else { auto &res_io_buffer = @@ -144,6 +144,163 @@ std::future GraphBrpcClient::get_node_feat( return fut; } + +std::future GraphBrpcClient::clear_nodes(uint32_t table_id) { + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + server_size, [&, server_size = this->server_size ](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + size_t fail_num = 0; + for (size_t request_idx = 0; request_idx < server_size; ++request_idx) { + if (closure->check_response(request_idx, PS_GRAPH_CLEAR) != 0) { + ++fail_num; + break; + } + } + ret = fail_num == 0 ? 0 : -1; + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + for (size_t i = 0; i < server_size; i++) { + int server_index = i; + closure->request(server_index)->set_cmd_id(PS_GRAPH_CLEAR); + closure->request(server_index)->set_table_id(table_id); + closure->request(server_index)->set_client_id(_client_id); + + GraphPsService_Stub rpc_stub = + getServiceStub(get_cmd_channel(server_index)); + closure->cntl(server_index)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(server_index), + closure->request(server_index), + closure->response(server_index), closure); + } + return fut; +} +std::future GraphBrpcClient::add_graph_node( + uint32_t table_id, std::vector &node_id_list, + std::vector &is_weighted_list) { + std::vector> request_bucket; + std::vector> is_weighted_bucket; + bool add_weight = is_weighted_list.size() > 0; + std::vector server_index_arr; + std::vector index_mapping(server_size, -1); + for (size_t query_idx = 0; query_idx < node_id_list.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_id_list[query_idx]); + if (index_mapping[server_index] == -1) { + index_mapping[server_index] = request_bucket.size(); + server_index_arr.push_back(server_index); + request_bucket.push_back(std::vector()); + if (add_weight) is_weighted_bucket.push_back(std::vector()); + } + request_bucket[index_mapping[server_index]].push_back( + node_id_list[query_idx]); + if (add_weight) + is_weighted_bucket[index_mapping[server_index]].push_back( + query_idx < is_weighted_list.size() ? is_weighted_list[query_idx] + : false); + } + size_t request_call_num = request_bucket.size(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + request_call_num, [&, request_call_num](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + size_t fail_num = 0; + for (size_t request_idx = 0; request_idx < request_call_num; + ++request_idx) { + if (closure->check_response(request_idx, PS_GRAPH_ADD_GRAPH_NODE) != + 0) { + ++fail_num; + } + } + ret = fail_num == request_call_num ? -1 : 0; + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + + for (size_t request_idx = 0; request_idx < request_call_num; ++request_idx) { + int server_index = server_index_arr[request_idx]; + closure->request(request_idx)->set_cmd_id(PS_GRAPH_ADD_GRAPH_NODE); + closure->request(request_idx)->set_table_id(table_id); + closure->request(request_idx)->set_client_id(_client_id); + size_t node_num = request_bucket[request_idx].size(); + closure->request(request_idx) + ->add_params((char *)request_bucket[request_idx].data(), + sizeof(uint64_t) * node_num); + if (add_weight) { + bool weighted[is_weighted_bucket[request_idx].size() + 1]; + for (size_t j = 0; j < is_weighted_bucket[request_idx].size(); j++) + weighted[j] = is_weighted_bucket[request_idx][j]; + closure->request(request_idx) + ->add_params((char *)weighted, + sizeof(bool) * is_weighted_bucket[request_idx].size()); + } + // PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = + getServiceStub(get_cmd_channel(server_index)); + closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), + closure->response(request_idx), closure); + } + return fut; +} +std::future GraphBrpcClient::remove_graph_node( + uint32_t table_id, std::vector &node_id_list) { + std::vector> request_bucket; + std::vector server_index_arr; + std::vector index_mapping(server_size, -1); + for (size_t query_idx = 0; query_idx < node_id_list.size(); ++query_idx) { + int server_index = get_server_index_by_id(node_id_list[query_idx]); + if (index_mapping[server_index] == -1) { + index_mapping[server_index] = request_bucket.size(); + server_index_arr.push_back(server_index); + request_bucket.push_back(std::vector()); + } + request_bucket[index_mapping[server_index]].push_back( + node_id_list[query_idx]); + } + size_t request_call_num = request_bucket.size(); + DownpourBrpcClosure *closure = new DownpourBrpcClosure( + request_call_num, [&, request_call_num](void *done) { + int ret = 0; + auto *closure = (DownpourBrpcClosure *)done; + int fail_num = 0; + for (size_t request_idx = 0; request_idx < request_call_num; + ++request_idx) { + if (closure->check_response(request_idx, + PS_GRAPH_REMOVE_GRAPH_NODE) != 0) { + ++fail_num; + } + } + ret = fail_num == request_call_num ? -1 : 0; + closure->set_promise_value(ret); + }); + auto promise = std::make_shared>(); + closure->add_promise(promise); + std::future fut = promise->get_future(); + + for (size_t request_idx = 0; request_idx < request_call_num; ++request_idx) { + int server_index = server_index_arr[request_idx]; + closure->request(request_idx)->set_cmd_id(PS_GRAPH_REMOVE_GRAPH_NODE); + closure->request(request_idx)->set_table_id(table_id); + closure->request(request_idx)->set_client_id(_client_id); + size_t node_num = request_bucket[request_idx].size(); + + closure->request(request_idx) + ->add_params((char *)request_bucket[request_idx].data(), + sizeof(uint64_t) * node_num); + // PsService_Stub rpc_stub(get_cmd_channel(server_index)); + GraphPsService_Stub rpc_stub = + getServiceStub(get_cmd_channel(server_index)); + closure->cntl(request_idx)->set_log_id(butil::gettimeofday_ms()); + rpc_stub.service(closure->cntl(request_idx), closure->request(request_idx), + closure->response(request_idx), closure); + } + return fut; +} // char* &buffer,int &actual_size std::future GraphBrpcClient::batch_sample_neighboors( uint32_t table_id, std::vector node_ids, int sample_size, @@ -174,8 +331,8 @@ std::future GraphBrpcClient::batch_sample_neighboors( [&, node_id_buckets, query_idx_buckets, request_call_num](void *done) { int ret = 0; auto *closure = (DownpourBrpcClosure *)done; - int fail_num = 0; - for (int request_idx = 0; request_idx < request_call_num; + size_t fail_num = 0; + for (size_t request_idx = 0; request_idx < request_call_num; ++request_idx) { if (closure->check_response(request_idx, PS_GRAPH_SAMPLE_NEIGHBOORS) != 0) { @@ -254,13 +411,14 @@ std::future GraphBrpcClient::random_sample_nodes( auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); - char buffer[bytes_size]; + char *buffer = new char[bytes_size]; auto size = io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); int index = 0; while (index < bytes_size) { ids.push_back(*(uint64_t *)(buffer + index)); index += GraphNode::id_size; } + delete[] buffer; } closure->set_promise_value(ret); }); @@ -292,7 +450,7 @@ std::future GraphBrpcClient::pull_graph_list( auto &res_io_buffer = closure->cntl(0)->response_attachment(); butil::IOBufBytesIterator io_buffer_itr(res_io_buffer); size_t bytes_size = io_buffer_itr.bytes_left(); - char buffer[bytes_size]; + char *buffer = new char[bytes_size]; io_buffer_itr.copy_and_forward((void *)(buffer), bytes_size); int index = 0; while (index < bytes_size) { @@ -301,6 +459,7 @@ std::future GraphBrpcClient::pull_graph_list( index += node.get_size(false); res.push_back(node); } + delete buffer; } closure->set_promise_value(ret); }); diff --git a/paddle/fluid/distributed/service/graph_brpc_client.h b/paddle/fluid/distributed/service/graph_brpc_client.h index 4e6775a4bedaf..5696e8b08037b 100644 --- a/paddle/fluid/distributed/service/graph_brpc_client.h +++ b/paddle/fluid/distributed/service/graph_brpc_client.h @@ -78,6 +78,13 @@ class GraphBrpcClient : public BrpcPsClient { const uint32_t& table_id, const std::vector& node_ids, const std::vector& feature_names, std::vector>& res); + + virtual std::future clear_nodes(uint32_t table_id); + virtual std::future add_graph_node( + uint32_t table_id, std::vector& node_id_list, + std::vector& is_weighted_list); + virtual std::future remove_graph_node( + uint32_t table_id, std::vector& node_id_list); virtual int32_t initialize(); int get_shard_num() { return shard_num; } void set_shard_num(int shard_num) { this->shard_num = shard_num; } diff --git a/paddle/fluid/distributed/service/graph_brpc_server.cc b/paddle/fluid/distributed/service/graph_brpc_server.cc index bdd926278b624..52ac8c5d688a4 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.cc +++ b/paddle/fluid/distributed/service/graph_brpc_server.cc @@ -24,6 +24,14 @@ namespace paddle { namespace distributed { +#define CHECK_TABLE_EXIST(table, request, response) \ + if (table == NULL) { \ + std::string err_msg("table not found with table_id:"); \ + err_msg.append(std::to_string(request.table_id())); \ + set_response_code(response, -1, err_msg.c_str()); \ + return -1; \ + } + int32_t GraphBrpcServer::initialize() { auto &service_config = _config.downpour_server_param().service_param(); if (!service_config.has_service_class()) { @@ -71,6 +79,58 @@ uint64_t GraphBrpcServer::start(const std::string &ip, uint32_t port) { return 0; } +int32_t GraphBrpcService::clear_nodes(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + ((GraphTable *)table)->clear_nodes(); + return 0; +} + +int32_t GraphBrpcService::add_graph_node(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 1) { + set_response_code( + response, -1, + "graph_get_node_feat request requires at least 2 arguments"); + return 0; + } + + size_t node_num = request.params(0).size() / sizeof(uint64_t); + uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); + std::vector node_ids(node_data, node_data + node_num); + std::vector is_weighted_list; + if (request.params_size() == 2) { + size_t weight_list_size = request.params(1).size() / sizeof(bool); + bool *is_weighted_buffer = (bool *)(request.params(1).c_str()); + is_weighted_list = std::vector(is_weighted_buffer, + is_weighted_buffer + weight_list_size); + } + + ((GraphTable *)table)->add_graph_node(node_ids, is_weighted_list); + return 0; +} +int32_t GraphBrpcService::remove_graph_node(Table *table, + const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl) { + CHECK_TABLE_EXIST(table, request, response) + if (request.params_size() < 1) { + set_response_code( + response, -1, + "graph_get_node_feat request requires at least 1 argument"); + return 0; + } + size_t node_num = request.params(0).size() / sizeof(uint64_t); + uint64_t *node_data = (uint64_t *)(request.params(0).c_str()); + std::vector node_ids(node_data, node_data + node_num); + + ((GraphTable *)table)->remove_graph_node(node_ids); + return 0; +} int32_t GraphBrpcServer::port() { return _server.listen_address().port; } int32_t GraphBrpcService::initialize() { @@ -92,21 +152,17 @@ int32_t GraphBrpcService::initialize() { &GraphBrpcService::graph_random_sample_nodes; _service_handler_map[PS_GRAPH_GET_NODE_FEAT] = &GraphBrpcService::graph_get_node_feat; - + _service_handler_map[PS_GRAPH_CLEAR] = &GraphBrpcService::clear_nodes; + _service_handler_map[PS_GRAPH_ADD_GRAPH_NODE] = + &GraphBrpcService::add_graph_node; + _service_handler_map[PS_GRAPH_REMOVE_GRAPH_NODE] = + &GraphBrpcService::remove_graph_node; // shard初始化,server启动后才可从env获取到server_list的shard信息 initialize_shard_info(); return 0; } -#define CHECK_TABLE_EXIST(table, request, response) \ - if (table == NULL) { \ - std::string err_msg("table not found with table_id:"); \ - err_msg.append(std::to_string(request.table_id())); \ - set_response_code(response, -1, err_msg.c_str()); \ - return -1; \ - } - int32_t GraphBrpcService::initialize_shard_info() { if (!_is_initialize_shard_info) { std::lock_guard guard(_initialize_shard_mutex); diff --git a/paddle/fluid/distributed/service/graph_brpc_server.h b/paddle/fluid/distributed/service/graph_brpc_server.h index 32c572f9e6c2b..47c370572826a 100644 --- a/paddle/fluid/distributed/service/graph_brpc_server.h +++ b/paddle/fluid/distributed/service/graph_brpc_server.h @@ -86,6 +86,13 @@ class GraphBrpcService : public PsBaseService { int32_t graph_get_node_feat(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); + int32_t clear_nodes(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t add_graph_node(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, brpc::Controller *cntl); + int32_t remove_graph_node(Table *table, const PsRequestMessage &request, + PsResponseMessage &response, + brpc::Controller *cntl); int32_t barrier(Table *table, const PsRequestMessage &request, PsResponseMessage &response, brpc::Controller *cntl); int32_t load_one_table(Table *table, const PsRequestMessage &request, diff --git a/paddle/fluid/distributed/service/graph_py_service.cc b/paddle/fluid/distributed/service/graph_py_service.cc index 61e4e0cf7bb91..39befb1a112c8 100644 --- a/paddle/fluid/distributed/service/graph_py_service.cc +++ b/paddle/fluid/distributed/service/graph_py_service.cc @@ -44,6 +44,9 @@ void GraphPyService::add_table_feat_conf(std::string table_name, } } +void add_graph_node(std::vector node_ids, + std::vector weight_list) {} +void remove_graph_node(std::vector node_ids) {} void GraphPyService::set_up(std::string ips_str, int shard_num, std::vector node_types, std::vector edge_types) { @@ -247,6 +250,34 @@ void GraphPyClient::load_edge_file(std::string name, std::string filepath, } } +void GraphPyClient::clear_nodes(std::string name) { + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = get_ps_client()->clear_nodes(table_id); + status.wait(); + } +} + +void GraphPyClient::add_graph_node(std::string name, + std::vector& node_ids, + std::vector& weight_list) { + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = + get_ps_client()->add_graph_node(table_id, node_ids, weight_list); + status.wait(); + } +} + +void GraphPyClient::remove_graph_node(std::string name, + std::vector& node_ids) { + if (this->table_id_map.count(name)) { + uint32_t table_id = this->table_id_map[name]; + auto status = get_ps_client()->remove_graph_node(table_id, node_ids); + status.wait(); + } +} + void GraphPyClient::load_node_file(std::string name, std::string filepath) { // 'n' means load nodes and 'node_type' follows std::string params = "n" + name; diff --git a/paddle/fluid/distributed/service/graph_py_service.h b/paddle/fluid/distributed/service/graph_py_service.h index c6657be96ba44..da027fbae3e6f 100644 --- a/paddle/fluid/distributed/service/graph_py_service.h +++ b/paddle/fluid/distributed/service/graph_py_service.h @@ -141,6 +141,10 @@ class GraphPyClient : public GraphPyService { void finalize_worker(); void load_edge_file(std::string name, std::string filepath, bool reverse); void load_node_file(std::string name, std::string filepath); + void clear_nodes(std::string name); + void add_graph_node(std::string name, std::vector& node_ids, + std::vector& weight_list); + void remove_graph_node(std::string name, std::vector& node_ids); int get_client_id() { return client_id; } void set_client_id(int client_id) { this->client_id = client_id; } void start_client(); diff --git a/paddle/fluid/distributed/service/sendrecv.proto b/paddle/fluid/distributed/service/sendrecv.proto index d908c26da9870..a4b811e950a3b 100644 --- a/paddle/fluid/distributed/service/sendrecv.proto +++ b/paddle/fluid/distributed/service/sendrecv.proto @@ -52,6 +52,9 @@ enum PsCmdID { PS_GRAPH_SAMPLE_NEIGHBOORS = 31; PS_GRAPH_SAMPLE_NODES = 32; PS_GRAPH_GET_NODE_FEAT = 33; + PS_GRAPH_CLEAR = 34; + PS_GRAPH_ADD_GRAPH_NODE = 35; + PS_GRAPH_REMOVE_GRAPH_NODE = 36; } message PsRequestMessage { diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 0dc99de1bfe82..92f8304a8bf62 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -35,6 +35,77 @@ std::vector GraphShard::get_batch(int start, int end, int step) { size_t GraphShard::get_size() { return bucket.size(); } +int32_t GraphTable::add_graph_node(std::vector &id_list, + std::vector &is_weight_list) { + size_t node_size = id_list.size(); + std::vector>> batch(task_pool_size_); + for (size_t i = 0; i < node_size; i++) { + size_t shard_id = id_list[i] % shard_num; + if (shard_id >= shard_end || shard_id < shard_start) { + continue; + } + batch[get_thread_pool_index(id_list[i])].push_back( + {id_list[i], i < is_weight_list.size() ? is_weight_list[i] : false}); + } + std::vector> tasks; + for (size_t i = 0; i < batch.size(); ++i) { + if (!batch[i].size()) continue; + tasks.push_back(_shards_task_pool[i]->enqueue([&batch, i, this]() -> int { + for (auto &p : batch[i]) { + size_t index = p.first % this->shard_num - this->shard_start; + this->shards[index].add_graph_node(p.first)->build_edges(p.second); + } + return 0; + })); + } + for (size_t i = 0; i < tasks.size(); i++) tasks[i].get(); + return 0; +} + +int32_t GraphTable::remove_graph_node(std::vector &id_list) { + size_t node_size = id_list.size(); + std::vector> batch(task_pool_size_); + for (size_t i = 0; i < node_size; i++) { + size_t shard_id = id_list[i] % shard_num; + if (shard_id >= shard_end || shard_id < shard_start) continue; + batch[get_thread_pool_index(id_list[i])].push_back(id_list[i]); + } + std::vector> tasks; + for (size_t i = 0; i < batch.size(); ++i) { + if (!batch[i].size()) continue; + tasks.push_back(_shards_task_pool[i]->enqueue([&batch, i, this]() -> int { + for (auto &p : batch[i]) { + size_t index = p % this->shard_num - this->shard_start; + this->shards[index].delete_node(p); + } + return 0; + })); + } + for (size_t i = 0; i < tasks.size(); i++) tasks[i].get(); + return 0; +} + +void GraphShard::clear() { + for (size_t i = 0; i < bucket.size(); i++) { + delete bucket[i]; + } + bucket.clear(); + node_location.clear(); +} + +GraphShard::~GraphShard() { clear(); } +void GraphShard::delete_node(uint64_t id) { + auto iter = node_location.find(id); + if (iter == node_location.end()) return; + int pos = iter->second; + delete bucket[pos]; + if (pos != (int)bucket.size() - 1) { + bucket[pos] = bucket.back(); + node_location[bucket.back()->get_id()] = pos; + } + node_location.erase(id); + bucket.pop_back(); +} GraphNode *GraphShard::add_graph_node(uint64_t id) { if (node_location.find(id) == node_location.end()) { node_location[id] = bucket.size(); @@ -79,11 +150,7 @@ int32_t GraphTable::get_nodes_ids_by_ranges( int start = 0, end, index = 0, total_size = 0; res.clear(); std::vector>> tasks; - // std::string temp = ""; - // for(int i = 0;i < shards.size();i++) - // temp+= std::to_string((int)shards[i].get_size()) + " "; - // VLOG(0)<<"range distribution "<enqueue( [this, first, second, i]() -> std::vector { return shards[i].get_ids_by_range(first, second); @@ -106,7 +172,7 @@ int32_t GraphTable::get_nodes_ids_by_ranges( } total_size += shards[i].get_size(); } - for (int i = 0; i < tasks.size(); i++) { + for (size_t i = 0; i < tasks.size(); i++) { auto vec = tasks[i].get(); for (auto &id : vec) { res.push_back(id); @@ -219,7 +285,7 @@ int32_t GraphTable::load_edges(const std::string &path, bool reverse_edge) { for (auto &shard : shards) { auto bucket = shard.get_bucket(); - for (int i = 0; i < bucket.size(); i++) { + for (size_t i = 0; i < bucket.size(); i++) { bucket[i]->build_sampler(sample_type); } } @@ -238,10 +304,29 @@ Node *GraphTable::find_node(uint64_t id) { uint32_t GraphTable::get_thread_pool_index(uint64_t node_id) { return node_id % shard_num % shard_num_per_table % task_pool_size_; } + +uint32_t GraphTable::get_thread_pool_index_by_shard_index( + uint64_t shard_index) { + return shard_index % shard_num_per_table % task_pool_size_; +} + +int32_t GraphTable::clear_nodes() { + std::vector> tasks; + for (size_t i = 0; i < shards.size(); i++) { + tasks.push_back( + _shards_task_pool[get_thread_pool_index_by_shard_index(i)]->enqueue( + [this, i]() -> int { + this->shards[i].clear(); + return 0; + })); + } + for (size_t i = 0; i < tasks.size(); i++) tasks[i].get(); + return 0; +} + int32_t GraphTable::random_sample_nodes(int sample_size, std::unique_ptr &buffer, int &actual_size) { - bool need_feature = false; int total_size = 0; for (int i = 0; i < shards.size(); i++) { total_size += shards[i].get_size(); @@ -281,7 +366,7 @@ int32_t GraphTable::random_sample_nodes(int sample_size, } std::vector> first_half, second_half; int start_index = rand() % total_size; - for (int i = 0; i < ranges_len.size() && i < ranges_pos.size(); i++) { + for (size_t i = 0; i < ranges_len.size() && i < ranges_pos.size(); i++) { if (ranges_pos[i] + ranges_len[i] - 1 + start_index < total_size) first_half.push_back({ranges_pos[i] + start_index, ranges_pos[i] + ranges_len[i] + start_index}); @@ -386,7 +471,6 @@ std::pair GraphTable::parse_feature( if (this->feat_id_map.count(fields[0])) { int32_t id = this->feat_id_map[fields[0]]; std::string dtype = this->feat_dtype[id]; - int32_t shape = this->feat_shape[id]; std::vector values(fields.begin() + 1, fields.end()); if (dtype == "feasign") { return std::make_pair( diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index b18da82abe61c..5eeb3915f5b1f 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -36,11 +36,12 @@ class GraphShard { size_t get_size(); GraphShard() {} GraphShard(int shard_num) { this->shard_num = shard_num; } + ~GraphShard(); std::vector &get_bucket() { return bucket; } std::vector get_batch(int start, int end, int step); std::vector get_ids_by_range(int start, int end) { std::vector res; - for (int i = start; i < end && i < bucket.size(); i++) { + for (int i = start; i < end && i < (int)bucket.size(); i++) { res.push_back(bucket[i]->get_id()); } return res; @@ -48,6 +49,8 @@ class GraphShard { GraphNode *add_graph_node(uint64_t id); FeatureNode *add_feature_node(uint64_t id); Node *find_node(uint64_t id); + void delete_node(uint64_t id); + void clear(); void add_neighboor(uint64_t id, uint64_t dst_id, float weight); std::unordered_map get_node_location() { return node_location; @@ -85,6 +88,11 @@ class GraphTable : public SparseTable { int32_t load_nodes(const std::string &path, std::string node_type); + int32_t add_graph_node(std::vector &id_list, + std::vector &is_weight_list); + + int32_t remove_graph_node(std::vector &id_list); + Node *find_node(uint64_t id); virtual int32_t pull_sparse(float *values, @@ -97,6 +105,7 @@ class GraphTable : public SparseTable { return 0; } + virtual int32_t clear_nodes(); virtual void clear() {} virtual int32_t flush() { return 0; } virtual int32_t shrink(const std::string ¶m) { return 0; } @@ -105,6 +114,7 @@ class GraphTable : public SparseTable { return 0; } virtual int32_t initialize_shard() { return 0; } + virtual uint32_t get_thread_pool_index_by_shard_index(uint64_t shard_index); virtual uint32_t get_thread_pool_index(uint64_t node_id); virtual std::pair parse_feature(std::string feat_str); @@ -128,4 +138,5 @@ class GraphTable : public SparseTable { std::vector> _shards_task_pool; }; } // namespace distributed + }; // namespace paddle diff --git a/paddle/fluid/distributed/test/graph_node_test.cc b/paddle/fluid/distributed/test/graph_node_test.cc index b268bb449e146..b8630aed02ffe 100644 --- a/paddle/fluid/distributed/test/graph_node_test.cc +++ b/paddle/fluid/distributed/test/graph_node_test.cc @@ -124,7 +124,6 @@ void testSingleSampleNeighboor( for (auto g : s) { ASSERT_EQ(true, s1.find(g) != s1.end()); } - VLOG(0) << "test single done"; s.clear(); s1.clear(); vs.clear(); @@ -141,6 +140,57 @@ void testSingleSampleNeighboor( } } +void testAddNode( + std::shared_ptr& worker_ptr_) { + worker_ptr_->clear_nodes(0); + int total_num = 270000; + uint64_t id; + std::unordered_set id_set; + for (int i = 0; i < total_num; i++) { + while (id_set.find(id = rand()) != id_set.end()) + ; + id_set.insert(id); + } + std::vector id_list(id_set.begin(), id_set.end()); + std::vector weight_list; + auto status = worker_ptr_->add_graph_node(0, id_list, weight_list); + status.wait(); + std::vector ids[2]; + for (int i = 0; i < 2; i++) { + auto sample_status = + worker_ptr_->random_sample_nodes(0, i, total_num, ids[i]); + sample_status.wait(); + } + std::unordered_set id_set_check(ids[0].begin(), ids[0].end()); + for (auto x : ids[1]) id_set_check.insert(x); + ASSERT_EQ(id_set.size(), id_set_check.size()); + for (auto x : id_set) { + ASSERT_EQ(id_set_check.find(x) != id_set_check.end(), true); + } + std::vector remove_ids; + for (auto p : id_set_check) { + if (remove_ids.size() == 0) + remove_ids.push_back(p); + else if (remove_ids.size() < total_num / 2 && rand() % 2 == 1) { + remove_ids.push_back(p); + } + } + for (auto p : remove_ids) id_set_check.erase(p); + status = worker_ptr_->remove_graph_node(0, remove_ids); + status.wait(); + for (int i = 0; i < 2; i++) ids[i].clear(); + for (int i = 0; i < 2; i++) { + auto sample_status = + worker_ptr_->random_sample_nodes(0, i, total_num, ids[i]); + sample_status.wait(); + } + std::unordered_set id_set_check1(ids[0].begin(), ids[0].end()); + for (auto x : ids[1]) id_set_check1.insert(x); + ASSERT_EQ(id_set_check1.size(), id_set_check.size()); + for (auto x : id_set_check1) { + ASSERT_EQ(id_set_check.find(x) != id_set_check.end(), true); + } +} void testBatchSampleNeighboor( std::shared_ptr& worker_ptr_) { std::vector>> vs; @@ -527,6 +577,7 @@ void RunBrpcPushSparse() { std::remove(edge_file_name); std::remove(node_file_name); + testAddNode(worker_ptr_); LOG(INFO) << "Run stop_server"; worker_ptr_->stop_server(); LOG(INFO) << "Run finalize_worker"; From d66d6486af69b48f0531f422443f3009bd5d6174 Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Thu, 8 Jul 2021 16:16:11 +0800 Subject: [PATCH 098/106] when sample k is larger than neigbor num, return directly --- .../table/graph/graph_weighted_sampler.cc | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc index 3a680875e3df4..307f7f80c2f51 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc @@ -22,8 +22,13 @@ void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } std::vector RandomSampler::sample_k(int k) { int n = edges->size(); - if (k > n) { + if (k >= n) { k = n; + std::vector sample_result; + for (int i = 0; i < k; i++) { + sample_result.push_back(i); + } + return sample_result; } struct timespec tn; clock_gettime(CLOCK_REALTIME, &tn); @@ -99,8 +104,13 @@ void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, } } std::vector WeightedSampler::sample_k(int k) { - if (k > count) { + if (k >= count) { k = count; + std::vector sample_result; + for (int i = 0; i < k; i++) { + sample_result.push_back(i); + } + return sample_result; } std::vector sample_result; float subtract; From 2037c51f244de048786b83556aaeb29bc974a50a Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Fri, 9 Jul 2021 10:24:53 +0800 Subject: [PATCH 099/106] using random seed generator of paddle to speed up --- .../distributed/table/common_graph_table.cc | 12 +++- .../distributed/table/common_graph_table.h | 1 + .../distributed/table/graph/graph_node.h | 5 +- .../table/graph/graph_weighted_sampler.cc | 68 +++++++++++-------- .../table/graph/graph_weighted_sampler.h | 8 ++- 5 files changed, 59 insertions(+), 35 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 92f8304a8bf62..997c395f5147d 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -20,7 +20,10 @@ #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/distributed/table/graph/graph_node.h" #include "paddle/fluid/string/printf.h" +#include #include "paddle/fluid/string/string_helper.h" +#include "paddle/fluid/framework/generator.h" + namespace paddle { namespace distributed { @@ -399,7 +402,11 @@ int32_t GraphTable::random_sample_neighboors( uint64_t &node_id = node_ids[idx]; std::unique_ptr &buffer = buffers[idx]; int &actual_size = actual_sizes[idx]; - tasks.push_back(_shards_task_pool[get_thread_pool_index(node_id)]->enqueue( + + int thread_pool_index = get_thread_pool_index(node_id); + auto rng = _shards_task_rng_pool[thread_pool_index]; + + tasks.push_back(_shards_task_pool[thread_pool_index]->enqueue( [&]() -> int { Node *node = find_node(node_id); @@ -407,7 +414,7 @@ int32_t GraphTable::random_sample_neighboors( actual_size = 0; return 0; } - std::vector res = node->sample_k(sample_size); + std::vector res = node->sample_k(sample_size, rng); actual_size = res.size() * (Node::id_size + Node::weight_size); int offset = 0; uint64_t id; @@ -546,6 +553,7 @@ int32_t GraphTable::initialize() { _shards_task_pool.resize(task_pool_size_); for (size_t i = 0; i < _shards_task_pool.size(); ++i) { _shards_task_pool[i].reset(new ::ThreadPool(1)); + _shards_task_rng_pool.push_back(paddle::framework::GetCPURandomEngine(0)); } server_num = _shard_num; // VLOG(0) << "in init graph table server num = " << server_num; diff --git a/paddle/fluid/distributed/table/common_graph_table.h b/paddle/fluid/distributed/table/common_graph_table.h index 5eeb3915f5b1f..6ccce44c7ead6 100644 --- a/paddle/fluid/distributed/table/common_graph_table.h +++ b/paddle/fluid/distributed/table/common_graph_table.h @@ -136,6 +136,7 @@ class GraphTable : public SparseTable { std::string table_type; std::vector> _shards_task_pool; + std::vector> _shards_task_rng_pool; }; } // namespace distributed diff --git a/paddle/fluid/distributed/table/graph/graph_node.h b/paddle/fluid/distributed/table/graph/graph_node.h index 8ad795ac97b54..940896188bf60 100644 --- a/paddle/fluid/distributed/table/graph/graph_node.h +++ b/paddle/fluid/distributed/table/graph/graph_node.h @@ -18,6 +18,7 @@ #include #include #include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" +#include namespace paddle { namespace distributed { @@ -33,7 +34,7 @@ class Node { virtual void build_edges(bool is_weighted) {} virtual void build_sampler(std::string sample_type) {} virtual void add_edge(uint64_t id, float weight) {} - virtual std::vector sample_k(int k) { return std::vector(); } + virtual std::vector sample_k(int k, const std::shared_ptr rng) { return std::vector(); } virtual uint64_t get_neighbor_id(int idx) { return 0; } virtual float get_neighbor_weight(int idx) { return 1.; } @@ -59,7 +60,7 @@ class GraphNode : public Node { virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } - virtual std::vector sample_k(int k) { return sampler->sample_k(k); } + virtual std::vector sample_k(int k, const std::shared_ptr rng) { return sampler->sample_k(k, rng); } virtual uint64_t get_neighbor_id(int idx) { return edges->get_id(idx); } virtual float get_neighbor_weight(int idx) { return edges->get_weight(idx); } diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc index 307f7f80c2f51..4520513588267 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc @@ -13,44 +13,58 @@ // limitations under the License. #include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" +#include "paddle/fluid/framework/generator.h" #include #include +#include namespace paddle { namespace distributed { void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } -std::vector RandomSampler::sample_k(int k) { +std::vector RandomSampler::sample_k(int k, const std::shared_ptr rng) { int n = edges->size(); - if (k >= n) { + if (k > n) { k = n; - std::vector sample_result; - for (int i = 0; i < k; i++) { - sample_result.push_back(i); - } - return sample_result; } - struct timespec tn; - clock_gettime(CLOCK_REALTIME, &tn); - srand(tn.tv_nsec); std::vector sample_result; + for(int i = 0;i < k;i ++ ) { + sample_result.push_back(i); + } + if (k == n) { + return sample_result; + } + + std::uniform_int_distribution distrib(0, n - 1); std::unordered_map replace_map; - while (k--) { - int rand_int = rand() % n; - auto iter = replace_map.find(rand_int); - if (iter == replace_map.end()) { - sample_result.push_back(rand_int); - } else { - sample_result.push_back(iter->second); - } - iter = replace_map.find(n - 1); - if (iter == replace_map.end()) { - replace_map[rand_int] = n - 1; + for(int i = 0; i < k; i ++) { + int j = distrib(*rng); + if (j >= i) { + // buff_nid[offset + i] = nid[j] if m.find(j) == m.end() else nid[m[j]] + auto iter_j = replace_map.find(j); + if(iter_j == replace_map.end()) { + sample_result[i] = j; + } else { + sample_result[i] = iter_j -> second; + } + // m[j] = i if m.find(i) == m.end() else m[i] + auto iter_i = replace_map.find(i); + if(iter_i == replace_map.end()) { + replace_map[j] = i; + } else { + replace_map[j] = (iter_i -> second); + } } else { - replace_map[rand_int] = iter->second; + sample_result[i] = sample_result[j]; + // buff_nid[offset + j] = nid[i] if m.find(i) == m.end() else nid[m[i]] + auto iter_i = replace_map.find(i); + if(iter_i == replace_map.end()) { + sample_result[j] = i; + } else { + sample_result[j] = (iter_i -> second); + } } - --n; } return sample_result; } @@ -103,7 +117,7 @@ void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, count = left->count + right->count; } } -std::vector WeightedSampler::sample_k(int k) { +std::vector WeightedSampler::sample_k(int k, const std::shared_ptr rng) { if (k >= count) { k = count; std::vector sample_result; @@ -116,11 +130,9 @@ std::vector WeightedSampler::sample_k(int k) { float subtract; std::unordered_map subtract_weight_map; std::unordered_map subtract_count_map; - struct timespec tn; - clock_gettime(CLOCK_REALTIME, &tn); - srand(tn.tv_nsec); + std::uniform_real_distribution distrib(0, 1.0); while (k--) { - float query_weight = rand() % 100000 / 100000.0; + float query_weight = distrib(*rng); query_weight *= weight - subtract_weight_map[this]; sample_result.push_back(sample(query_weight, subtract_weight_map, subtract_count_map, subtract)); diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h index 1787ab23b0431..a23207cc31eab 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h @@ -17,6 +17,8 @@ #include #include #include "paddle/fluid/distributed/table/graph/graph_edge.h" +#include +#include namespace paddle { namespace distributed { @@ -24,14 +26,14 @@ class Sampler { public: virtual ~Sampler() {} virtual void build(GraphEdgeBlob *edges) = 0; - virtual std::vector sample_k(int k) = 0; + virtual std::vector sample_k(int k, const std::shared_ptr rng) = 0; }; class RandomSampler : public Sampler { public: virtual ~RandomSampler() {} virtual void build(GraphEdgeBlob *edges); - virtual std::vector sample_k(int k); + virtual std::vector sample_k(int k, const std::shared_ptr rng); GraphEdgeBlob *edges; }; @@ -46,7 +48,7 @@ class WeightedSampler : public Sampler { GraphEdgeBlob *edges; virtual void build(GraphEdgeBlob *edges); virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); - virtual std::vector sample_k(int k); + virtual std::vector sample_k(int k, const std::shared_ptr rng); private: int sample(float query_weight, From d743606ed65834bdb408b6fdd53e22368a40e227 Mon Sep 17 00:00:00 2001 From: liweibin02 Date: Mon, 12 Jul 2021 10:57:00 +0800 Subject: [PATCH 100/106] fix bug of random sample k --- .../table/graph/graph_weighted_sampler.cc | 54 +++++++------------ 1 file changed, 20 insertions(+), 34 deletions(-) diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc index 4520513588267..a26017443fc99 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc @@ -24,47 +24,33 @@ void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } std::vector RandomSampler::sample_k(int k, const std::shared_ptr rng) { int n = edges->size(); - if (k > n) { + if (k >= n) { k = n; - } - std::vector sample_result; - for(int i = 0;i < k;i ++ ) { + std::vector sample_result; + for (int i = 0; i < k; i++) { sample_result.push_back(i); + } + return sample_result; } - if (k == n) { - return sample_result; - } - - std::uniform_int_distribution distrib(0, n - 1); + std::vector sample_result; std::unordered_map replace_map; + while (k--) { + std::uniform_int_distribution distrib(0, n - 1); + int rand_int = distrib(*rng); + auto iter = replace_map.find(rand_int); + if (iter == replace_map.end()) { + sample_result.push_back(rand_int); + } else { + sample_result.push_back(iter->second); + } - for(int i = 0; i < k; i ++) { - int j = distrib(*rng); - if (j >= i) { - // buff_nid[offset + i] = nid[j] if m.find(j) == m.end() else nid[m[j]] - auto iter_j = replace_map.find(j); - if(iter_j == replace_map.end()) { - sample_result[i] = j; - } else { - sample_result[i] = iter_j -> second; - } - // m[j] = i if m.find(i) == m.end() else m[i] - auto iter_i = replace_map.find(i); - if(iter_i == replace_map.end()) { - replace_map[j] = i; - } else { - replace_map[j] = (iter_i -> second); - } + iter = replace_map.find(n - 1); + if (iter == replace_map.end()) { + replace_map[rand_int] = n - 1; } else { - sample_result[i] = sample_result[j]; - // buff_nid[offset + j] = nid[i] if m.find(i) == m.end() else nid[m[i]] - auto iter_i = replace_map.find(i); - if(iter_i == replace_map.end()) { - sample_result[j] = i; - } else { - sample_result[j] = (iter_i -> second); - } + replace_map[rand_int] = iter->second; } + --n; } return sample_result; } From 4cfea1faede1b6b6c9b9a1d37d67548e06cedde1 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 13 Jul 2021 01:49:31 +0800 Subject: [PATCH 101/106] fix code style --- .../table/graph/graph_weighted_sampler.cc | 14 ++++++++------ .../table/graph/graph_weighted_sampler.h | 17 ++++++++++------- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc index a26017443fc99..7a46433e3defb 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.cc @@ -13,16 +13,17 @@ // limitations under the License. #include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" -#include "paddle/fluid/framework/generator.h" #include -#include #include +#include +#include "paddle/fluid/framework/generator.h" namespace paddle { namespace distributed { void RandomSampler::build(GraphEdgeBlob *edges) { this->edges = edges; } -std::vector RandomSampler::sample_k(int k, const std::shared_ptr rng) { +std::vector RandomSampler::sample_k( + int k, const std::shared_ptr rng) { int n = edges->size(); if (k >= n) { k = n; @@ -103,7 +104,8 @@ void WeightedSampler::build_one(WeightedGraphEdgeBlob *edges, int start, count = left->count + right->count; } } -std::vector WeightedSampler::sample_k(int k, const std::shared_ptr rng) { +std::vector WeightedSampler::sample_k( + int k, const std::shared_ptr rng) { if (k >= count) { k = count; std::vector sample_result; @@ -154,5 +156,5 @@ int WeightedSampler::sample( subtract_count_map[this]++; return return_idx; } -} -} +} // namespace distributed +} // namespace paddle diff --git a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h index a23207cc31eab..4a75a112697d3 100644 --- a/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h +++ b/paddle/fluid/distributed/table/graph/graph_weighted_sampler.h @@ -14,11 +14,11 @@ #pragma once #include +#include +#include #include #include #include "paddle/fluid/distributed/table/graph/graph_edge.h" -#include -#include namespace paddle { namespace distributed { @@ -26,14 +26,16 @@ class Sampler { public: virtual ~Sampler() {} virtual void build(GraphEdgeBlob *edges) = 0; - virtual std::vector sample_k(int k, const std::shared_ptr rng) = 0; + virtual std::vector sample_k( + int k, const std::shared_ptr rng) = 0; }; class RandomSampler : public Sampler { public: virtual ~RandomSampler() {} virtual void build(GraphEdgeBlob *edges); - virtual std::vector sample_k(int k, const std::shared_ptr rng); + virtual std::vector sample_k(int k, + const std::shared_ptr rng); GraphEdgeBlob *edges; }; @@ -48,7 +50,8 @@ class WeightedSampler : public Sampler { GraphEdgeBlob *edges; virtual void build(GraphEdgeBlob *edges); virtual void build_one(WeightedGraphEdgeBlob *edges, int start, int end); - virtual std::vector sample_k(int k, const std::shared_ptr rng); + virtual std::vector sample_k(int k, + const std::shared_ptr rng); private: int sample(float query_weight, @@ -56,5 +59,5 @@ class WeightedSampler : public Sampler { std::unordered_map &subtract_count_map, float &subtract); }; -} -} +} // namespace distributed +} // namespace paddle From 6087b289d9e17e38a94e5183a63882cd10f415dd Mon Sep 17 00:00:00 2001 From: seemingwang Date: Tue, 13 Jul 2021 02:19:39 +0800 Subject: [PATCH 102/106] fix code style --- .../distributed/table/common_graph_table.cc | 56 +++++++++---------- .../distributed/table/graph/graph_node.cc | 4 +- .../distributed/table/graph/graph_node.h | 16 ++++-- 3 files changed, 40 insertions(+), 36 deletions(-) diff --git a/paddle/fluid/distributed/table/common_graph_table.cc b/paddle/fluid/distributed/table/common_graph_table.cc index 997c395f5147d..29bcc04d9c1df 100644 --- a/paddle/fluid/distributed/table/common_graph_table.cc +++ b/paddle/fluid/distributed/table/common_graph_table.cc @@ -15,14 +15,14 @@ #include "paddle/fluid/distributed/table/common_graph_table.h" #include #include +#include #include #include #include "paddle/fluid/distributed/common/utils.h" #include "paddle/fluid/distributed/table/graph/graph_node.h" +#include "paddle/fluid/framework/generator.h" #include "paddle/fluid/string/printf.h" -#include #include "paddle/fluid/string/string_helper.h" -#include "paddle/fluid/framework/generator.h" namespace paddle { namespace distributed { @@ -406,31 +406,30 @@ int32_t GraphTable::random_sample_neighboors( int thread_pool_index = get_thread_pool_index(node_id); auto rng = _shards_task_rng_pool[thread_pool_index]; - tasks.push_back(_shards_task_pool[thread_pool_index]->enqueue( - [&]() -> int { - Node *node = find_node(node_id); + tasks.push_back(_shards_task_pool[thread_pool_index]->enqueue([&]() -> int { + Node *node = find_node(node_id); - if (node == nullptr) { - actual_size = 0; - return 0; - } - std::vector res = node->sample_k(sample_size, rng); - actual_size = res.size() * (Node::id_size + Node::weight_size); - int offset = 0; - uint64_t id; - float weight; - char *buffer_addr = new char[actual_size]; - buffer.reset(buffer_addr); - for (int &x : res) { - id = node->get_neighbor_id(x); - weight = node->get_neighbor_weight(x); - memcpy(buffer_addr + offset, &id, Node::id_size); - offset += Node::id_size; - memcpy(buffer_addr + offset, &weight, Node::weight_size); - offset += Node::weight_size; - } - return 0; - })); + if (node == nullptr) { + actual_size = 0; + return 0; + } + std::vector res = node->sample_k(sample_size, rng); + actual_size = res.size() * (Node::id_size + Node::weight_size); + int offset = 0; + uint64_t id; + float weight; + char *buffer_addr = new char[actual_size]; + buffer.reset(buffer_addr); + for (int &x : res) { + id = node->get_neighbor_id(x); + weight = node->get_neighbor_weight(x); + memcpy(buffer_addr + offset, &id, Node::id_size); + offset += Node::id_size; + memcpy(buffer_addr + offset, &weight, Node::weight_size); + offset += Node::weight_size; + } + return 0; + })); } for (size_t idx = 0; idx < node_num; ++idx) { tasks[idx].get(); @@ -519,7 +518,6 @@ int32_t GraphTable::pull_graph_list(int start, int total_size, int end = start + (count - 1) * step + 1; tasks.push_back(_shards_task_pool[i % task_pool_size_]->enqueue( [this, i, start, end, step, size]() -> std::vector { - return this->shards[i].get_batch(start - size, end - size, step); })); start += count * step; @@ -594,5 +592,5 @@ int32_t GraphTable::initialize() { shards = std::vector(shard_num_per_table, GraphShard(shard_num)); return 0; } -} -}; +} // namespace distributed +}; // namespace paddle diff --git a/paddle/fluid/distributed/table/graph/graph_node.cc b/paddle/fluid/distributed/table/graph/graph_node.cc index 816d31b979072..e2311cc307b60 100644 --- a/paddle/fluid/distributed/table/graph/graph_node.cc +++ b/paddle/fluid/distributed/table/graph/graph_node.cc @@ -113,5 +113,5 @@ void FeatureNode::recover_from_buffer(char* buffer) { feature.push_back(std::string(str)); } } -} -} +} // namespace distributed +} // namespace paddle diff --git a/paddle/fluid/distributed/table/graph/graph_node.h b/paddle/fluid/distributed/table/graph/graph_node.h index 940896188bf60..62c101ec02a93 100644 --- a/paddle/fluid/distributed/table/graph/graph_node.h +++ b/paddle/fluid/distributed/table/graph/graph_node.h @@ -15,10 +15,10 @@ #pragma once #include #include +#include #include #include #include "paddle/fluid/distributed/table/graph/graph_weighted_sampler.h" -#include namespace paddle { namespace distributed { @@ -34,7 +34,10 @@ class Node { virtual void build_edges(bool is_weighted) {} virtual void build_sampler(std::string sample_type) {} virtual void add_edge(uint64_t id, float weight) {} - virtual std::vector sample_k(int k, const std::shared_ptr rng) { return std::vector(); } + virtual std::vector sample_k( + int k, const std::shared_ptr rng) { + return std::vector(); + } virtual uint64_t get_neighbor_id(int idx) { return 0; } virtual float get_neighbor_weight(int idx) { return 1.; } @@ -60,7 +63,10 @@ class GraphNode : public Node { virtual void add_edge(uint64_t id, float weight) { edges->add_edge(id, weight); } - virtual std::vector sample_k(int k, const std::shared_ptr rng) { return sampler->sample_k(k, rng); } + virtual std::vector sample_k( + int k, const std::shared_ptr rng) { + return sampler->sample_k(k, rng); + } virtual uint64_t get_neighbor_id(int idx) { return edges->get_id(idx); } virtual float get_neighbor_weight(int idx) { return edges->get_weight(idx); } @@ -124,5 +130,5 @@ class FeatureNode : public Node { protected: std::vector feature; }; -} -} +} // namespace distributed +} // namespace paddle From a0982aef88dba498e3c580aa8ab81291d65600f0 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 28 Jul 2021 18:03:18 +0800 Subject: [PATCH 103/106] fix blocking_queue problem --- .../fluid/distributed/service/communicator.h | 75 +++++++++++++------ 1 file changed, 53 insertions(+), 22 deletions(-) diff --git a/paddle/fluid/distributed/service/communicator.h b/paddle/fluid/distributed/service/communicator.h index fa60cab2b5877..4c7bbabcb104d 100644 --- a/paddle/fluid/distributed/service/communicator.h +++ b/paddle/fluid/distributed/service/communicator.h @@ -61,38 +61,65 @@ using Variable = framework::Variable; template class BlockingQueue { public: - explicit BlockingQueue(size_t capacity) : capacity_(capacity) { - PADDLE_ENFORCE_GT(capacity_, 0, - platform::errors::InvalidArgument( - "The capacity must be greater than 0.")); - } + explicit BlockingQueue(size_t capacity) : capacity_(capacity) {} bool Push(const T &elem) { - { - std::unique_lock lock(mutex_); - cv_.wait(lock, [&] { return queue_.size() < capacity_; }); - queue_.push_back(elem); + std::unique_lock lock(mutex_); + WaitForWrite(lock); + + queue_.push_back(elem); + + Notify(); + return true; + } + bool WaitForWrite(std::unique_lock &lock) { // NOLINT + while (FullUnlocked()) { + if (empty_waiters_ != 0) { + empty_cond_.notify_one(); + } + full_waiters_++; + full_cond_.wait(lock); + full_waiters_--; } - cv_.notify_one(); return true; } - - bool Push(T &&elem) { - { - std::unique_lock lock(mutex_); - cv_.wait(lock, [&] { return queue_.size() < capacity_; }); - queue_.emplace_back(std::move(elem)); + bool WaitForRead(std::unique_lock &lock) { // NOLINT + while (EmptyUnlocked()) { + if (full_waiters_ != 0) { + full_cond_.notify_one(); + } + empty_waiters_++; + empty_cond_.wait(lock); + empty_waiters_--; } - cv_.notify_one(); return true; } + bool EmptyUnlocked() { return queue_.empty(); } + bool FullUnlocked() { return queue_.size() >= capacity_; } + void Notify() { + if (empty_waiters_ != 0 && (!EmptyUnlocked())) { + empty_cond_.notify_one(); + } + if (full_waiters_ != 0 && (!FullUnlocked())) { + full_cond_.notify_one(); + } + } + + bool Push(T &&elem) { + std::unique_lock lock(mutex_); + WaitForWrite(lock); + queue_.emplace_back(std::move(elem)); + + Notify(); + return true; + } T Pop() { std::unique_lock lock(mutex_); - cv_.wait(lock, [=] { return !queue_.empty(); }); + WaitForRead(lock); T rc(std::move(queue_.front())); queue_.pop_front(); - cv_.notify_one(); + Notify(); return rc; } @@ -107,11 +134,14 @@ class BlockingQueue { } private: + int empty_waiters_ = 0; + int full_waiters_ = 0; + std::condition_variable empty_cond_; + std::condition_variable full_cond_; const size_t capacity_; std::deque queue_; mutable std::mutex mutex_; - std::condition_variable cv_; }; template inline void MergeVars(const std::string &var_name, const std::vector> &vars, Scope *scope, bool merge_add = true) { - PADDLE_ENFORCE_NE(vars.empty(), true, platform::errors::InvalidArgument( - "vector vars are empty.")); + PADDLE_ENFORCE_NE( + vars.empty(), true, + platform::errors::InvalidArgument("vector vars are empty.")); auto cpu_place = platform::CPUPlace(); auto &var0 = vars[0]; auto *out_var = scope->Var(var_name); From fdac4b62c7580a43e82e84accbd08ef87c1434a1 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 28 Jul 2021 19:00:54 +0800 Subject: [PATCH 104/106] fix style --- paddle/fluid/distributed/service/communicator.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/paddle/fluid/distributed/service/communicator.h b/paddle/fluid/distributed/service/communicator.h index 4c7bbabcb104d..a54e6737f6f06 100644 --- a/paddle/fluid/distributed/service/communicator.h +++ b/paddle/fluid/distributed/service/communicator.h @@ -152,9 +152,8 @@ template inline void MergeVars(const std::string &var_name, const std::vector> &vars, Scope *scope, bool merge_add = true) { - PADDLE_ENFORCE_NE( - vars.empty(), true, - platform::errors::InvalidArgument("vector vars are empty.")); + PADDLE_ENFORCE_NE(vars.empty(), true, platform::errors::InvalidArgument( + "vector vars are empty.")); auto cpu_place = platform::CPUPlace(); auto &var0 = vars[0]; auto *out_var = scope->Var(var_name); From dfdfc0558af5f3fbf0ed928b3b1713df51318aba Mon Sep 17 00:00:00 2001 From: seemingwang Date: Wed, 28 Jul 2021 21:55:50 +0800 Subject: [PATCH 105/106] fix --- paddle/fluid/distributed/service/communicator.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/service/communicator.h b/paddle/fluid/distributed/service/communicator.h index a54e6737f6f06..9caec79e58b01 100644 --- a/paddle/fluid/distributed/service/communicator.h +++ b/paddle/fluid/distributed/service/communicator.h @@ -153,7 +153,7 @@ inline void MergeVars(const std::string &var_name, const std::vector> &vars, Scope *scope, bool merge_add = true) { PADDLE_ENFORCE_NE(vars.empty(), true, platform::errors::InvalidArgument( - "vector vars are empty.")); + "vector vars are empty.")); auto cpu_place = platform::CPUPlace(); auto &var0 = vars[0]; auto *out_var = scope->Var(var_name); From 4308f91dac4e6d5ae697f40354d7519136948432 Mon Sep 17 00:00:00 2001 From: seemingwang Date: Thu, 29 Jul 2021 05:08:10 +0000 Subject: [PATCH 106/106] recover capacity check --- paddle/fluid/distributed/service/communicator.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/paddle/fluid/distributed/service/communicator.h b/paddle/fluid/distributed/service/communicator.h index 9caec79e58b01..dd9a81dc5bb38 100644 --- a/paddle/fluid/distributed/service/communicator.h +++ b/paddle/fluid/distributed/service/communicator.h @@ -61,7 +61,12 @@ using Variable = framework::Variable; template class BlockingQueue { public: - explicit BlockingQueue(size_t capacity) : capacity_(capacity) {} + explicit BlockingQueue(size_t capacity) : capacity_(capacity) { + PADDLE_ENFORCE_GT(capacity_, 0, + + platform::errors::InvalidArgument( + "The capacity must be greater than 0.")); + } bool Push(const T &elem) { std::unique_lock lock(mutex_);