未验证 提交 ce0c5c27 编写于 作者: H hong 提交者: GitHub

[IR] [Baby step] New interprector support new ir (#54570)

* add kernel dialect

* change DenseTensorTypeStorage to DenseTensorType

* add test case`

* add first pd_op to kernel dialect

* lower pd op to kernel dialect

* update

* update

* remove useless code

* add attrite print test

* fix bug

* update

* update

* update

* update

* polish code

* fix bug

* polish  code  and add python test

* add test

* fix test error

* add env flag

* fix bug

* revert test env

* change cc_test_old to cc_test

* fix build_static bug

* fix type test error

* udpate cmake

* disable test in windows

* fix inference compile
上级 c7ba811d
......@@ -5,8 +5,15 @@ add_subdirectory(workqueue)
set(STANDALONE_EXECUTOR_SRCS interpretercore.cc new_executor_defs.cc
standalone_executor.cc)
set(STANDALONE_EXECUTOR_DEPS interpreter interpretercore_garbage_collector
workqueue)
set(STANDALONE_EXECUTOR_DEPS
interpreter
interpretercore_garbage_collector
workqueue
pd_dialect
pd_op_to_kernel_pass
phi_kernel_adaptor
program_translator
new_ir)
cc_library(
standalone_executor
......
......@@ -117,7 +117,8 @@ const std::map<size_t, std::set<size_t>>& DependencyBuilder::OpDownstreamMap()
void DependencyBuilder::AddDependencyForCoalesceTensorOp() {
for (size_t op_idx = 0; op_idx < op_num_; ++op_idx) {
if (instructions_->at(op_idx).OpBase()->Type() == kCoalesceTensor) {
if (instructions_->at(op_idx).OpBaseValid() &&
instructions_->at(op_idx).OpBase()->Type() == kCoalesceTensor) {
VLOG(4) << "Add depend for " << kCoalesceTensor << " " << op_idx;
auto fused_out = instructions_->at(op_idx).Outputs().at("FusedOutput")[0];
auto outputs = instructions_->at(op_idx).Outputs().at("Output");
......@@ -224,7 +225,8 @@ void DependencyBuilder::AddDependencyForCommunicationOp() {
const std::string kSyncComm = "c_sync_comm_stream";
dependence_op_idx = ULLONG_MAX;
for (size_t op_idx = 0; op_idx < op_num_; ++op_idx) {
if (instructions_->at(op_idx).OpBase()->Type() == kSyncComm) {
if (instructions_->at(op_idx).OpBaseValid() &&
instructions_->at(op_idx).OpBase()->Type() == kSyncComm) {
dependence_op_idx = op_idx;
} else {
if (dependence_op_idx != ULLONG_MAX) {
......@@ -251,7 +253,8 @@ void DependencyBuilder::AddDependencyForRandomOp() {
size_t dependence_op_idx = ULLONG_MAX;
for (size_t op_idx = 0; op_idx < op_num_; ++op_idx) {
if (random_op_set.count(instructions_->at(op_idx).OpBase()->Type())) {
if (instructions_->at(op_idx).OpBaseValid() &&
random_op_set.count(instructions_->at(op_idx).OpBase()->Type())) {
if (dependence_op_idx != ULLONG_MAX) {
AddDownstreamOp(dependence_op_idx, op_idx);
}
......@@ -275,7 +278,8 @@ void DependencyBuilder::AddDependencyForReadOp() {
std::vector<size_t> read_ops;
std::vector<size_t> startup_ops;
for (size_t op_idx = 0; op_idx < op_num_; ++op_idx) {
if (instructions_->at(op_idx).OpBase()->Type() == "read") {
if (instructions_->at(op_idx).OpBaseValid() &&
instructions_->at(op_idx).OpBase()->Type() == "read") {
read_ops.push_back(op_idx);
}
......
......@@ -22,6 +22,9 @@
#include "paddle/fluid/framework/new_executor/interpreter/data_transfer.h"
#include "paddle/fluid/framework/new_executor/interpreter/execution_config.h"
#include "paddle/fluid/framework/new_executor/interpreter/static_build.h"
#include "paddle/fluid/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/interface/op_yaml_info.h"
#include "paddle/fluid/ir/phi_kernel_adaptor/phi_kernel_util.h"
#include "paddle/fluid/memory/stats.h"
#include "paddle/fluid/operators/controlflow/conditional_block_op_helper.h"
#include "paddle/fluid/operators/controlflow/recurrent_op_helper.h"
......@@ -138,6 +141,9 @@ bool IsCommunicationOp(const std::string& op_name) {
}
bool IsCommunicationOp(const Instruction& instr) {
if (!instr.OpBaseValid()) {
return false;
}
return IsCommunicationOp(instr.OpBase()->Type());
}
......@@ -924,6 +930,78 @@ void BuildOpFuncList(const platform::Place& place,
delete garbages;
}
void BuildOpFuncList(
const platform::Place& place,
::ir::Block* block,
std::vector<OpFuncNode>* vec_func_list,
framework::Scope* scope,
const std::unordered_map<::ir::Value, std::string>& value_2_name_map,
const ExecutionConfig& execution_config) {
vec_func_list->reserve(block->size());
::ir::IrContext* ctx = ir::IrContext::Instance();
ctx->GetOrRegisterDialect<paddle::dialect::PaddleDialect>();
for (auto it = block->begin(); it != block->end(); ++it) {
OpFuncNode op_func_node;
auto attr_map = (*it)->attributes();
auto op_name = attr_map.at("op_name").dyn_cast<::ir::StrAttribute>().data();
if (op_name == "pd.fetch") {
VLOG(6) << "skip process pd.fetch op";
continue;
}
op_func_node.phi_op_name_ = op_name;
::ir::OpInfo op_info = ctx->GetRegisteredOpInfo(op_name);
auto impl =
op_info.GetInterfaceImpl<paddle::dialect::OpYamlInfoInterface>();
auto yaml_info = impl->get_op_info_();
auto attr_info = std::get<1>(yaml_info);
op_func_node.infer_shape_interface_ =
op_info.GetInterfaceImpl<paddle::dialect::InferShapeInterface>();
::ir::BuildInferMetaContext((*it),
value_2_name_map,
scope,
yaml_info,
&(op_func_node.infer_meta_context_));
auto kernel_name =
attr_map.at("kernel_name").dyn_cast<ir::StrAttribute>().data();
auto kernel_key = attr_map.at("kernel_key")
.dyn_cast<paddle::dialect::KernelAttribute>()
.data();
auto t1 =
phi::KernelFactory::Instance().SelectKernel(kernel_name, kernel_key);
op_func_node.phi_kernel_ = new phi::Kernel(t1);
PADDLE_ENFORCE_EQ(op_func_node.phi_kernel_->IsValid(),
true,
"not found kernel for [%s]",
kernel_name);
::ir::BuildPhiKernelContext((*it),
value_2_name_map,
scope,
yaml_info,
&(op_func_node.kernel_context_),
&(op_func_node.input_index),
&(op_func_node.output_index));
op_func_node.kernel_context_.SetDeviceContext(
phi::DeviceContextPool::Instance().Get(
phi::TransToPhiPlace(kernel_key.backend())));
op_func_node.dev_ctx_ = phi::DeviceContextPool::Instance().Get(
phi::TransToPhiPlace(kernel_key.backend()));
vec_func_list->emplace_back(op_func_node);
}
}
void BuildVariableScope(const framework::BlockDesc& block,
const ExecutionConfig& execution_config,
VariableScope* var_scope) {
......
......@@ -38,6 +38,7 @@
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/fluid/platform/device_context.h"
#include "paddle/fluid/platform/init.h"
#include "paddle/ir/core/program.h"
using AtomicVectorSizeT = std::vector<std::atomic<size_t>>;
......@@ -93,6 +94,14 @@ void BuildOpFuncList(const platform::Place& place,
bool use_local_scope = true,
bool static_build = false);
void BuildOpFuncList(
const platform::Place& place,
::ir::Block* block,
std::vector<OpFuncNode>* vec_func_list,
framework::Scope* scope,
const std::unordered_map<::ir::Value, std::string>& value_2_name_map,
const ExecutionConfig& execution_config);
void BuildVariableScope(const framework::BlockDesc& block,
const ExecutionConfig& execution_config,
VariableScope* var_scope);
......
......@@ -143,6 +143,9 @@ void StreamAnalyzer::ConstructEvents(
DeviceContext* StreamAnalyzer::ParseDeviceContext(
const OpFuncNode& op_func_node) const {
auto& op = op_func_node.operator_base_;
if (op == nullptr) {
return op_func_node.dev_ctx_;
}
auto& op_type = op->Type();
const std::string& execution_stream = op_func_node.execution_stream_;
const int stream_priority = op_func_node.stream_priority_;
......@@ -201,7 +204,9 @@ DeviceContext* StreamAnalyzer::ParseDeviceContext(
#endif
}
SetDeviceCommContext(op.get(), op_func_node.dev_ctx_);
if (op != nullptr) {
SetDeviceCommContext(op.get(), op_func_node.dev_ctx_);
}
return op_func_node.dev_ctx_;
}
......
......@@ -32,6 +32,7 @@
#ifdef PADDLE_WITH_MKLDNN
#include "paddle/fluid/platform/mkldnn_helper.h"
#endif
#include "paddle/fluid/ir/phi_kernel_adaptor/phi_kernel_util.h"
#include "paddle/fluid/platform/cuda_graph_with_memory_pool.h"
#include "paddle/fluid/platform/flags.h"
#include "paddle/phi/backends/device_manager.h"
......@@ -60,6 +61,8 @@ PHI_DECLARE_bool(new_executor_use_cuda_graph);
PHI_DECLARE_bool(sync_nccl_allreduce);
#endif
PHI_DECLARE_bool(enable_new_ir_in_executor);
constexpr const char* kExceptionCaught = "ExceptionCaught";
constexpr const char* kTaskCompletion = "TaskCompletion";
......@@ -145,6 +148,58 @@ InterpreterCore::InterpreterCore(const platform::Place& place,
PrepareForCUDAGraphCapture();
}
InterpreterCore::InterpreterCore(const platform::Place& place,
const BlockDesc& block,
framework::Scope* scope,
::ir::Program* ir_prog,
const ExecutionConfig& execution_config)
: place_(place),
block_(block),
stream_analyzer_(place),
execution_config_(execution_config),
var_scope_(scope),
ir_program_(ir_prog) {
VLOG(4) << "InterpreterCore(): " << this << " on " << place_;
static_build_ = FLAGS_new_executor_static_build &&
!FLAGS_new_executor_use_cuda_graph &&
!execution_config.used_for_control_flow_op &&
interpreter::BlockCanBeStaticBuilt(block);
exception_notifier_ = main_thread_blocker_.RegisterEvent(kExceptionCaught);
completion_notifier_ = main_thread_blocker_.RegisterEvent(kTaskCompletion);
if (!FLAGS_new_executor_use_local_scope) {
execution_config_.create_local_scope = false;
}
execution_config_.AnalyzeThreadPoolConfig(place, block.OpSize());
execution_config_.Log(/*log_level=*/8);
if (execution_config_.create_local_scope) {
auto local_scope = &var_scope_.GetMutableScope()->NewScope();
local_scope_ = local_scope;
}
// force use outer scope for now
local_scope_ = scope;
static_build_ = true;
var_scope_.SetLocalScope(local_scope_);
instruction_scheduling_priority_less = [this](size_t lhs, size_t rhs) {
SchedulingPriority lhs_scheduling_priority =
vec_instruction_[lhs].GetSchedulingPriority();
SchedulingPriority rhs_scheduling_priority =
vec_instruction_[rhs].GetSchedulingPriority();
if (lhs_scheduling_priority == rhs_scheduling_priority) {
return lhs < rhs;
}
return lhs_scheduling_priority > rhs_scheduling_priority;
};
PrepareForCUDAGraphCapture();
}
InterpreterCore::~InterpreterCore() {
// cancle gc's thread
gc_.reset(nullptr);
......@@ -166,8 +221,9 @@ void InterpreterCore::RunImpl() {
interpreter::ResetAtomicGuard guard(&deps_, &refs_);
if ((execution_config_.used_for_jit || execution_config_.used_for_cinn) &&
(sync_op_num_ == 0)) {
if (FLAGS_enable_new_ir_in_executor ||
((execution_config_.used_for_jit || execution_config_.used_for_cinn) &&
(sync_op_num_ == 0))) {
VLOG(4) << "Tracing Instruction List";
TraceInstructionList(vec_instruction_);
} else {
......@@ -196,6 +252,7 @@ paddle::framework::FetchList InterpreterCore::Run(
#endif
bool is_build = is_build_;
Prepare(feed_names, feed_tensors, is_build);
if (is_build) {
......@@ -235,19 +292,31 @@ paddle::framework::FetchList InterpreterCore::Run(
if (!is_build_) {
LOG_FIRST_N(INFO, 1) << "New Executor is Running.";
paddle::framework::interpreter::BuildVariableScope(
block_, execution_config_, &var_scope_);
if (FLAGS_enable_new_ir_in_executor) {
::ir::BuildScope(
ir_program_->block(), local_scope_, &value_2_var_name_map_);
} else {
interpreter::BuildVariableScope(block_, execution_config_, &var_scope_);
}
std::vector<paddle::framework::OpFuncNode> op_func_nodes;
paddle::framework::interpreter::BuildOpFuncList(
place_,
block_,
execution_config_.skip_gc_vars,
&op_func_nodes,
&var_scope_,
execution_config_,
HasLocalScope(),
static_build_);
if (FLAGS_enable_new_ir_in_executor) {
interpreter::BuildOpFuncList(place_,
ir_program_->block(),
&op_func_nodes,
local_scope_,
value_2_var_name_map_,
execution_config_);
} else {
interpreter::BuildOpFuncList(place_,
block_,
execution_config_.skip_gc_vars,
&op_func_nodes,
&var_scope_,
execution_config_,
HasLocalScope(),
static_build_);
}
SetFeedVarsInplaceSkip(feed_names);
// convert vec func_list to graph
Convert(&op_func_nodes);
......@@ -656,7 +725,6 @@ void InterpreterCore::Convert(
}
#endif
}
BuildOperatorDependences();
// NOTE(Ruibiao): For cross-step stream synchronization, an event may be
......@@ -666,13 +734,13 @@ void InterpreterCore::Convert(
// work and WaitEvent always return succeed immediately, we omit the
// prelude-record for the first step here.
stream_analyzer_.ConstructEvents(&vec_instruction_);
// add event for the input var of jit program, since there are async copied
// from gpu_pinned place to gpu place on compute stream.
for (size_t i = 0; i < dependecy_count_.size(); ++i) {
if (dependecy_count_[i] == 0) {
auto& inst = vec_instruction_[i];
if (inst.OpBase()->Type() == interpreter::kMemcpyD2H &&
if (inst.OpBaseValid() &&
inst.OpBase()->Type() == interpreter::kMemcpyD2H &&
platform::is_gpu_place(place_)) {
for (auto& item : inst.Inputs()) {
for (auto var_id : item.second) {
......@@ -692,61 +760,65 @@ void InterpreterCore::Convert(
}
// calculate last_live_ops_
for (size_t op_idx = 0; op_idx < op_nums; ++op_idx) {
Instruction& instr = vec_instruction_[op_idx];
OpInOutInfo info;
info.Build(instr.OpBase());
if (!FLAGS_enable_new_ir_in_executor) {
for (size_t op_idx = 0; op_idx < op_nums; ++op_idx) {
Instruction& instr = vec_instruction_[op_idx];
OpInOutInfo info;
if (instr.OpBaseValid()) {
info.Build(instr.OpBase());
}
std::set<size_t> gc_check_vars;
std::set<size_t> gc_check_vars;
const std::map<std::string, std::vector<int>>& ins = instr.Inputs();
const std::map<std::string, std::vector<int>>& outs = instr.Outputs();
std::multimap<std::string, std::vector<int>> ins_and_outs{ins.begin(),
ins.end()};
ins_and_outs.insert(outs.begin(), outs.end());
const std::map<std::string, std::vector<int>>& ins = instr.Inputs();
const std::map<std::string, std::vector<int>>& outs = instr.Outputs();
std::multimap<std::string, std::vector<int>> ins_and_outs{ins.begin(),
ins.end()};
ins_and_outs.insert(outs.begin(), outs.end());
for (auto& item : ins_and_outs) {
for (auto id : item.second) {
if (id == kEmptyVarIndex) {
continue;
}
auto* var_desc = var_scope_.VarDesc(id);
// skip no_need_buffer input vars
if (var_desc && ins.count(item.first) &&
!info.IsInArgBufferNeeded(var_desc->Name())) {
continue;
}
// skip when this var is not in block and not a data_transferred var,
// which means this var is managed by other block
const auto& var_name = var_scope_.GetNameById(id);
bool not_owned = !block_.HasVar(var_name);
const auto& transferred_vars = var_scope_.DataTransferAddedVars();
bool not_transferred =
std::all_of(transferred_vars.begin(),
transferred_vars.end(),
[&](const std::pair<std::string, int>& elem) {
return elem.first != var_name;
});
if (not_owned && not_transferred) {
VLOG(10) << "[gc_check_inputs] skip gc: " << var_name;
continue;
for (auto& item : ins_and_outs) {
for (auto id : item.second) {
if (id == kEmptyVarIndex) {
continue;
}
auto* var_desc = var_scope_.VarDesc(id);
// skip no_need_buffer input vars
if (var_desc && ins.count(item.first) &&
!info.IsInArgBufferNeeded(var_desc->Name())) {
continue;
}
// skip when this var is not in block and not a data_transferred var,
// which means this var is managed by other block
const auto& var_name = var_scope_.GetNameById(id);
bool not_owned = !block_.HasVar(var_name);
const auto& transferred_vars = var_scope_.DataTransferAddedVars();
bool not_transferred =
std::all_of(transferred_vars.begin(),
transferred_vars.end(),
[&](const std::pair<std::string, int>& elem) {
return elem.first != var_name;
});
if (not_owned && not_transferred) {
VLOG(10) << "[gc_check_inputs] skip gc: " << var_name;
continue;
}
gc_check_vars.insert(id);
}
gc_check_vars.insert(id);
}
}
for (auto var_id : gc_check_vars) {
Scope* inner_scope =
HasLocalScope() ? local_scope_ : var_scope_.GetMutableScope();
paddle::framework::Variable* var =
inner_scope->FindVar(var_scope_.GetNameById(var_id));
if (var->IsType<phi::DenseTensor>() || var->IsType<phi::SelectedRows>() ||
var->IsType<LoDTensorArray>()) {
last_live_ops_[var_id].insert(op_idx);
} else {
VLOG(4) << "not clear " << var_scope_.GetNameById(var_id) << " after "
<< instr.OpBase()->Type() << " because its type is "
<< framework::ToTypeName(var->Type());
for (auto var_id : gc_check_vars) {
Scope* inner_scope =
HasLocalScope() ? local_scope_ : var_scope_.GetMutableScope();
paddle::framework::Variable* var =
inner_scope->FindVar(var_scope_.GetNameById(var_id));
if (var->IsType<phi::DenseTensor>() ||
var->IsType<phi::SelectedRows>() || var->IsType<LoDTensorArray>()) {
last_live_ops_[var_id].insert(op_idx);
} else {
VLOG(4) << "not clear " << var_scope_.GetNameById(var_id) << " after "
<< instr.OpBase()->Type() << " because its type is "
<< framework::ToTypeName(var->Type());
}
}
}
}
......@@ -791,21 +863,22 @@ void InterpreterCore::Convert(
last_live_ops_[i] = minumum_last_live_ops;
vec_meta_info[i].var_ref_count_ = last_live_ops_[i].size();
}
for (size_t i = 0; i < vec_instruction_.size(); ++i) {
BuildAndCacheInstructionCtx(&vec_instruction_[i]);
if (!FLAGS_enable_new_ir_in_executor) {
for (size_t i = 0; i < vec_instruction_.size(); ++i) {
BuildAndCacheInstructionCtx(&vec_instruction_[i]);
}
}
bool inplaced = false;
for (const Instruction& inst : vec_instruction_) {
if (inst.OpBase()->Type() == "share_buffer" ||
inst.OpBase()->Type() == "share_data") {
if (inst.OpBaseValid() && (inst.OpBase()->Type() == "share_buffer" ||
inst.OpBase()->Type() == "share_data")) {
VLOG(4) << "Already inplaced, skip inplace now.";
inplaced = true;
}
}
if (FLAGS_new_executor_use_inplace && !inplaced) {
if (!FLAGS_enable_new_ir_in_executor && FLAGS_new_executor_use_inplace &&
!inplaced) {
BuildInplace();
}
......@@ -996,22 +1069,30 @@ void InterpreterCore::RunInstruction(const Instruction& instr_node) {
? "kGpuSync"
: "kGpuAsync"))
<< " runs on " << platform::GetCurrentThreadName();
auto* op = instr_node.OpBase();
platform::RecordEvent instruction_event(
op->Type(), platform::TracerEventType::Operator, 1);
OperatorBase* op = nullptr;
if (instr_node.OpBaseValid()) {
op = instr_node.OpBase();
platform::RecordEvent instruction_event(
op->Type(), platform::TracerEventType::Operator, 1);
}
SetDeviceId(instr_node.DeviceContext().GetPlace());
try {
instr_node.WaitEvent(place_);
if (!instr_node.IsArtificial()) {
if (instr_node.PreDefineContext()) {
VLOG(5) << "run new ir selected kernel";
auto op_func_node = const_cast<OpFuncNode*>((instr_node.OpFunc()));
op_func_node->infer_shape_interface_->infer_shape_(
&(op_func_node->infer_meta_context_));
(*(op_func_node->phi_kernel_))(&(op_func_node->kernel_context_));
} else if (!instr_node.IsArtificial()) {
RunOperator(instr_node);
CheckGC(instr_node);
interpreter::LogDeviceMemoryStats(place_);
}
instr_node.RecordEvent(place_);
} catch (platform::EnforceNotMet& ex) {
framework::InsertCallStackInfo(op->Type(), op->Attrs(), &ex);
......@@ -1437,7 +1518,8 @@ void InterpreterCore::TraceInstructionList(
void InterpreterCore::RecordMemcpyD2H(const Instruction& instr_node) {
// NOTE(zhiqiu): hot fix for jit input var
if (instr_node.OpBase()->Type() == interpreter::kMemcpyD2H) {
if (instr_node.OpBaseValid() &&
instr_node.OpBase()->Type() == interpreter::kMemcpyD2H) {
platform::DeviceContextPool& pool = platform::DeviceContextPool::Instance();
auto* default_dev_ctx = pool.Get(place_);
for (auto& event : instr_node.EventsToWait()) {
......
......@@ -32,6 +32,8 @@
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/memory/allocation/spin_lock.h"
#include "paddle/fluid/platform/device_event.h"
#include "paddle/ir/core/program.h"
#include "paddle/ir/core/value.h"
DECLARE_bool(new_executor_use_local_scope);
......@@ -52,6 +54,12 @@ class InterpreterCore {
Scope* scope,
const ExecutionConfig& execution_config = ExecutionConfig());
InterpreterCore(const platform::Place& place,
const BlockDesc& block,
Scope* scope,
::ir::Program* ir_prog,
const ExecutionConfig& execution_config = ExecutionConfig());
~InterpreterCore();
paddle::framework::FetchList Run(
......@@ -191,6 +199,11 @@ class InterpreterCore {
InstructionSchedulingPriorityLess instruction_scheduling_priority_less;
std::vector<HookFunc> hookfuncs_;
// The next only for new IR
::ir::Program* ir_program_{nullptr};
std::unordered_map<::ir::Value, std::string> value_2_var_name_map_;
};
} // namespace framework
......
......@@ -152,10 +152,18 @@ void VariableScope::CheckExist(const std::string& name) const {
Instruction::Instruction(size_t id,
OpFuncNode&& op_func_node,
const platform::DeviceContext& dev_ctx)
: is_artificial_(op_func_node.operator_base_->Type() == "depend"),
: is_artificial_(false),
id_(id),
op_func_node_(op_func_node),
dev_ctx_(dev_ctx) {
if (op_func_node.operator_base_ != nullptr &&
op_func_node.operator_base_->Type() == "depend") {
is_artificial_ = true;
}
if (op_func_node_.infer_shape_interface_ != nullptr) {
pre_define_context_ = true;
}
PADDLE_ENFORCE_GE(id,
0,
platform::errors::PreconditionNotMet(
......@@ -218,6 +226,10 @@ OperatorBase* Instruction::OpBase() const {
return op_base.get();
}
bool Instruction::OpBaseValid() const {
return op_func_node_.operator_base_ != nullptr;
}
void Instruction::AddGCCheckVar(size_t id) { gc_check_vars_.push_back(id); }
const std::vector<size_t>& Instruction::GCCheckVars() const {
......
......@@ -20,6 +20,7 @@
#include "paddle/fluid/framework/operator.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/fluid/ir/interface/infershape.h"
#include "paddle/fluid/platform/device_event_base.h"
#include "paddle/fluid/platform/event.h"
#include "paddle/phi/core/utils/rw_lock.h"
......@@ -164,13 +165,20 @@ struct OpFuncNode {
std::map<std::string, std::vector<int>> output_index;
// TODO(zhiqiu): Better make it unique_ptr
std::shared_ptr<OperatorBase> operator_base_;
std::shared_ptr<OperatorBase> operator_base_{nullptr};
std::string execution_stream_{kDefaultStream};
OpFuncType type_;
OpKernelComputeFunc kernel_func_;
SchedulingPriority scheduling_priority_{0}; // lower value, higher priority
// the next only for new IR
phi::KernelContext kernel_context_;
phi::InferMetaContext infer_meta_context_;
std::string phi_op_name_;
paddle::dialect::InferShapeInterface::Concept* infer_shape_interface_{
nullptr};
};
class Instruction {
......@@ -234,6 +242,8 @@ class Instruction {
OperatorBase* OpBase() const;
bool OpBaseValid() const;
void AddGCCheckVar(size_t id);
const std::vector<size_t>& GCCheckVars() const;
......@@ -263,6 +273,10 @@ class Instruction {
return op_func_node_.scheduling_priority_;
}
bool PreDefineContext() const { return pre_define_context_; }
const OpFuncNode* OpFunc() const { return &op_func_node_; }
private:
bool is_artificial_; // Instruction is artificial means that it is only used
// to assist scheduling and no need to be executed.
......@@ -285,6 +299,8 @@ class Instruction {
std::vector<size_t> gc_check_vars_;
std::vector<std::pair<Variable*, Variable*>> vec_inplace_in_to_out_;
bool pre_define_context_{false};
};
namespace interpreter {
......
......@@ -16,11 +16,29 @@
#include "paddle/fluid/framework/new_executor/interpreter/interpreter_util.h"
#include "paddle/fluid/platform/profiler/event_tracing.h"
#include "paddle/fluid/ir/pass/pd_op_to_kernel_pass.h"
#include "paddle/fluid/ir_adaptor/translator/translate.h"
PHI_DECLARE_bool(enable_new_ir_in_executor);
namespace paddle {
namespace framework {
StandaloneExecutor::StandaloneExecutor(const platform::Place& place,
const std::vector<ProgramDesc>& programs)
: place_(place), programs_(programs) {}
: place_(place), programs_(programs) {
if (FLAGS_enable_new_ir_in_executor) {
for (size_t i = 0; i < programs_.size(); ++i) {
VLOG(6) << "begin to translate" << std::endl;
auto base_progrm = paddle::TranslateLegacyProgramToProgram(programs_[i]);
auto kernel_program =
paddle::dialect::PdOpLowerToKernelPass(base_progrm.get());
ir_programs_.emplace_back(std::move(kernel_program));
}
}
}
paddle::framework::FetchList StandaloneExecutor::Run(
Scope* scope,
......@@ -39,6 +57,7 @@ paddle::framework::FetchList StandaloneExecutor::Run(
0,
interpreter::ExecutionConfig());
VLOG(4) << "StandaloneExecutor: " << this << ", InterpreterCore: " << core;
return core->Run(feed_names);
} else { // run multiple programs
VLOG(6) << "Run multiple program, programs_.size() " << programs_.size();
......@@ -97,8 +116,17 @@ std::shared_ptr<InterpreterCore> StandaloneExecutor::GetInterpreterCore(
if (iter == interpretercores_.end()) {
VLOG(3) << "create interpreter_core for " << oss.str() << " on place "
<< place_;
std::shared_ptr<InterpreterCore> core = std::make_shared<InterpreterCore>(
place_, program.Block(0), scope, execution_config);
std::shared_ptr<InterpreterCore> core = nullptr;
if (FLAGS_enable_new_ir_in_executor) {
core = std::make_shared<InterpreterCore>(place_,
program.Block(0),
scope,
ir_programs_[program_idx].get(),
execution_config);
} else {
core = std::make_shared<InterpreterCore>(
place_, program.Block(0), scope, execution_config);
}
interpretercores_.emplace(oss.str(), core);
return core;
} else {
......
......@@ -23,6 +23,7 @@
#include "paddle/fluid/framework/new_executor/new_executor_defs.h"
#include "paddle/fluid/framework/program_desc.h"
#include "paddle/fluid/platform/place.h"
#include "paddle/ir/core/program.h"
namespace paddle {
namespace framework {
......@@ -54,6 +55,7 @@ class StandaloneExecutor {
const platform::Place place_;
const std::vector<ProgramDesc> programs_;
std::vector<std::unique_ptr<::ir::Program>> ir_programs_;
std::vector<framework::Scope*> microbatch_scopes_;
std::unordered_map<std::string, std::shared_ptr<InterpreterCore>>
......
......@@ -58,14 +58,14 @@ set(KERNEL_LIST
#windows GPU static library over the limit, so not create_static_lib, and cc_library is dummy
if(WIN32 AND WITH_GPU)
cc_library(paddle_inference DEPS ${fluid_modules} ${STATIC_INFERENCE_API}
${utils_modules})
cc_library(paddle_inference DEPS ${fluid_modules} new_ir
${STATIC_INFERENCE_API} ${utils_modules})
else()
# message("${fluid_modules}")
# message("${STATIC_INFERENCE_API}")
# message("${utils_modules}")
# message("${phi_modules}")
create_static_lib(paddle_inference ${phi_modules} ${fluid_modules}
create_static_lib(paddle_inference ${phi_modules} ${fluid_modules} new_ir
${STATIC_INFERENCE_API} ${utils_modules})
endif()
......@@ -96,7 +96,7 @@ set(SHARED_INFERENCE_SRCS
# shared inference library deps
list(REMOVE_ITEM fluid_modules standalone_executor
interpretercore_garbage_collector)
set(SHARED_INFERENCE_DEPS phi ${fluid_modules} analysis_predictor
set(SHARED_INFERENCE_DEPS phi new_ir ${fluid_modules} analysis_predictor
${utils_modules})
if(WITH_CRYPTO)
......
add_subdirectory(interface)
add_subdirectory(dialect)
add_subdirectory(pass)
add_subdirectory(phi_kernel_adaptor)
......@@ -24,9 +24,12 @@
#include "paddle/fluid/ir/dialect/kernel_type_storage.h"
#include "paddle/fluid/ir/dialect/pd_op.h"
#include "paddle/fluid/ir/dialect/utils.h"
#include "paddle/fluid/platform/init_phi.h"
#include "paddle/ir/core/dialect_interface.h"
#include "paddle/phi/core/dense_tensor.h"
REGISTER_FILE_SYMBOLS(kernel_dialect);
namespace paddle {
namespace dialect {
......
......@@ -16,6 +16,8 @@
#include "paddle/ir/core/op_base.h"
#include "paddle/phi/core/infermeta_utils.h"
namespace paddle {
namespace dialect {
class InferShapeInterface : public ir::OpInterfaceBase<InferShapeInterface> {
public:
struct Concept {
......@@ -43,3 +45,6 @@ class InferShapeInterface : public ir::OpInterfaceBase<InferShapeInterface> {
private:
Concept *impl_;
};
} // namespace dialect
} // namespace paddle
......@@ -160,12 +160,16 @@ std::unique_ptr<ir::Program> PdOpLowerToKernelPass(ir::Program* prog) {
// only for single output
// need update new kernel key layout and data tyep
auto allocated_dense_tensor_dtype =
paddle::dialect::AllocatedDenseTensorType::get(
ctx,
phi::TransToPhiPlace(kernel_key.backend()),
(*it)->result(0).type().dyn_cast<dialect::DenseTensorType>());
std::vector<ir::Type> op_output_types;
if ((*it)->num_results() > 0) {
auto allocated_dense_tensor_dtype =
paddle::dialect::AllocatedDenseTensorType::get(
ctx,
phi::TransToPhiPlace(kernel_key.backend()),
(*it)->result(0).type().dyn_cast<dialect::DenseTensorType>());
op_output_types.push_back(allocated_dense_tensor_dtype);
}
// constuct input
std::vector<ir::OpResult> vec_inputs;
if ((*it)->name() != "pd.full_" && (*it)->num_operands() > 0) {
......@@ -195,10 +199,14 @@ std::unique_ptr<ir::Program> PdOpLowerToKernelPass(ir::Program* prog) {
}
ir::Operation* op1 = ir::Operation::Create(
vec_inputs, op1_attribute, {allocated_dense_tensor_dtype}, op1_info);
vec_inputs, op1_attribute, op_output_types, op1_info);
map_op_pair[*it] = op1;
map_value_pair[(*it)->result(0)] = op1->result(0);
// only deal with single output
if ((*it)->num_results() > 0) {
map_value_pair[(*it)->result(0)] = op1->result(0);
}
program->block()->push_back(op1);
}
......
# All source files of pd_dialect, except for the source file of op, which is generated in the compilation directory.
file(GLOB PHI_KERNEL_ADAPTOR_SRCS "*.cc")
cc_library(
phi_kernel_adaptor
SRCS ${PHI_KERNEL_ADAPTOR_SRCS}
DEPS new_ir phi_utils)
// Copyright (c) 2023 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/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/dialect/pd_op.h"
#include "paddle/fluid/ir/dialect/pd_type.h"
#include "paddle/fluid/ir/dialect/utils.h"
#include "paddle/fluid/ir/interface/infershape.h"
#include "paddle/fluid/ir/interface/op_yaml_info.h"
#include "paddle/ir/core/builtin_attribute.h"
#include "paddle/ir/core/builtin_dialect.h"
#include "paddle/ir/core/builtin_op.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/ir/core/utils.h"
#include "paddle/phi/core/meta_tensor.h"
#include "paddle/phi/infermeta/binary.h"
#include "paddle/phi/kernels/elementwise_add_kernel.h"
#include "paddle/fluid/framework/scope.h"
#include "paddle/fluid/framework/tensor.h"
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/phi/common/place.h"
#include "paddle/phi/core/kernel_context.h"
#include "paddle/phi/core/kernel_factory.h"
#include "paddle/fluid/platform/init.h"
#include "paddle/fluid/ir/dialect/kernel_attribute.h"
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "glog/logging.h"
#include "paddle/fluid/ir/phi_kernel_adaptor/phi_kernel_util.h"
class PhiKernelAdaptor {
public:
explicit PhiKernelAdaptor(paddle::framework::Scope* scope) : scope_(scope) {}
void run(ir::Program* program) {
auto block = program->block();
std::unordered_map<ir::Value, std::string> name_map;
std::cerr << "run here" << std::endl;
ir::BuildScope(block, scope_, &name_map);
std::cerr << "after buid scope" << std::endl;
auto* dev_ctx = phi::DeviceContextPool::Instance().Get(phi::CPUPlace());
phi::Place cpu_place(phi::AllocationType::CPU);
for (auto it = block->begin(); it != block->end(); ++it) {
VLOG(6) << "begin to run op " << (*it)->name();
std::cerr << (*it)->name() << std::endl;
auto attr_map = (*it)->attributes();
paddle::dialect::OpYamlInfoInterface op_info_interface =
(*it)->dyn_cast<paddle::dialect::OpYamlInfoInterface>();
auto op_info_res = op_info_interface.GetOpInfo();
paddle::dialect::InferShapeInterface interface =
(*it)->dyn_cast<paddle::dialect::InferShapeInterface>();
phi::InferMetaContext ctx;
ir::BuildInferMetaContext((*it), name_map, scope_, op_info_res, &ctx);
interface.InferShape(&ctx);
auto runtime_info = std::get<3>(op_info_res);
auto phi_kernels = phi::KernelFactory::Instance().SelectKernelMap(
runtime_info.kernel_func[0]);
phi::KernelKey kernel_key(phi::TransToPhiBackend(cpu_place),
phi::DataLayout::ANY,
phi::DataType::FLOAT32);
if (runtime_info.kernel_func[0] == "full_int_array") {
kernel_key.set_dtype(phi::DataType::INT64);
}
auto found_it = phi_kernels.find(kernel_key);
if (found_it == phi_kernels.end()) {
std::cerr << "kernel name " << runtime_info.kernel_func[0] << std::endl;
std::cerr << "kernel key " << kernel_key.backend() << "\t"
<< kernel_key.dtype() << "\t" << kernel_key.layout()
<< std::endl;
PADDLE_THROW(paddle::platform::errors::NotFound(
"can not found kerenl for [%s]", (*it)->name()));
} else {
phi::KernelContext kernel_ctx(dev_ctx);
ir::BuildPhiKernelContext(
(*it), name_map, scope_, op_info_res, &kernel_ctx);
found_it->second(&kernel_ctx);
auto out_value = (*it)->result(0);
out_name = name_map[out_value];
}
}
}
void run_kernel_prog(ir::Program* program) {
auto block = program->block();
std::unordered_map<ir::Value, std::string> name_map;
BuildScope(block, scope_, &name_map);
ir::IrContext* ctx = ir::IrContext::Instance();
ctx->GetOrRegisterDialect<paddle::dialect::PaddleDialect>();
auto* dev_ctx = phi::DeviceContextPool::Instance().Get(phi::CPUPlace());
phi::Place cpu_place(phi::AllocationType::CPU);
for (auto it = block->begin(); it != block->end(); ++it) {
auto attr_map = (*it)->attributes();
auto op_name = attr_map.at("op_name").dyn_cast<ir::StrAttribute>().data();
ir::OpInfo op1_info = ctx->GetRegisteredOpInfo(op_name);
auto impl =
op1_info.GetInterfaceImpl<paddle::dialect::OpYamlInfoInterface>();
auto yaml_info = impl->get_op_info_();
auto attr_info = std::get<1>(yaml_info);
auto infer_shape_impl =
op1_info.GetInterfaceImpl<paddle::dialect::InferShapeInterface>();
phi::InferMetaContext ctx;
ir::BuildInferMetaContext((*it), name_map, scope_, yaml_info, &ctx);
infer_shape_impl->infer_shape_(&ctx);
auto kernel_name =
attr_map.at("kernel_name").dyn_cast<ir::StrAttribute>().data();
auto kernel_key = attr_map.at("kernel_key")
.dyn_cast<paddle::dialect::KernelAttribute>()
.data();
auto kernel_fn =
phi::KernelFactory::Instance().SelectKernel(kernel_name, kernel_key);
phi::KernelContext kernel_ctx(dev_ctx);
ir::BuildPhiKernelContext(
(*it), name_map, scope_, yaml_info, &kernel_ctx);
kernel_fn(&kernel_ctx);
auto out_value = (*it)->result(0);
out_name = name_map[out_value];
}
}
std::string out_name;
private:
paddle::framework::Scope* scope_;
};
// Copyright (c) 2023 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/ir/phi_kernel_adaptor/phi_kernel_util.h"
#include "paddle/fluid/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/dialect/pd_type.h"
#include "paddle/fluid/ir/dialect/utils.h"
#include "paddle/fluid/ir/interface/op_yaml_info.h"
#include "paddle/ir/core/builtin_attribute.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/ir/core/utils.h"
#include "paddle/phi/core/meta_tensor.h"
#include "paddle/fluid/framework/scope.h"
#include "paddle/fluid/framework/tensor.h"
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/phi/core/kernel_context.h"
#include "paddle/fluid/ir/dialect/kernel_attribute.h"
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "glog/logging.h"
namespace ir {
void BuildScope(ir::Block* block,
paddle::framework::Scope* scope,
std::unordered_map<ir::Value, std::string>* name_map) {
std::unordered_map<ir::Value, int> map_test;
// int count = name_map->size();
int count = 0;
for (auto it = block->begin(); it != block->end(); ++it) {
size_t input_num = (*it)->num_operands();
auto attr_map = (*it)->attributes();
std::string op_name = (*it)->name();
if (attr_map.count("op_name")) {
auto op_name = attr_map.at("op_name").dyn_cast<ir::StrAttribute>().data();
}
if (op_name == "pd.fetch") {
// fetch is a very special op, with no output
for (size_t i = 0; i < input_num; ++i) {
auto ptr = (*it)->operand(i).source();
auto var_name = attr_map.at("name").dyn_cast<ir::StrAttribute>().data();
PADDLE_ENFORCE_EQ(
name_map->count(ptr),
true,
phi::errors::PreconditionNotMet(
"input of fetch op should in name mape, var_name is [%s]",
var_name));
scope->Rename(name_map->at(ptr), var_name);
(*name_map)[ptr] = var_name;
}
continue;
}
if (input_num > 0) {
for (size_t i = 0; i < input_num; ++i) {
auto ptr = (*it)->operand(i).source();
std::string name;
if (name_map->find(ptr) != name_map->end()) {
name = name_map->at(ptr);
} else {
PADDLE_THROW(phi::errors::PreconditionNotMet(
"input should in name map, [%d] 'th input of [%s] op",
i,
op_name));
}
}
}
int out_num = (*it)->num_results();
if (out_num > 0) {
for (int i = 0; i < out_num; ++i) {
ir::Value ptr = (*it)->result(i);
std::string name;
if (name_map->find(ptr) != name_map->end()) {
name = name_map->at(ptr);
} else {
name = "inner_var_" + std::to_string(count++);
name_map->emplace(ptr, name);
}
auto var = scope->Var(name);
// need to update here, only support DenseTensor
var->GetMutable<phi::DenseTensor>();
}
}
}
}
void BuildInferMetaContext(
ir::Operation* op,
const std::unordered_map<ir::Value, std::string>& name_map,
paddle::framework::Scope* scope,
const OpInfoTuple& op_yaml_info,
phi::InferMetaContext* ctx) {
// inputs include input and mutable attributes
auto input_info = std::get<0>(op_yaml_info);
std::map<std::string, size_t> input_index_map;
std::map<std::string, std::string> mutable_attr_type_map;
int input_index = 0;
for (auto& t : input_info) {
VLOG(6) << t.name << "\t" << t.type_name;
input_index_map[t.name] = input_index++;
if (t.is_mutable_attribute) {
mutable_attr_type_map[t.name] = t.type_name;
}
}
auto attr_info = std::get<1>(op_yaml_info);
std::map<std::string, std::string> attr_type_map;
for (auto& t : attr_info) {
VLOG(6) << t.name << "\t" << t.type_name;
attr_type_map[t.name] = t.type_name;
}
auto attr_map = op->attributes();
auto runtime_info = std::get<3>(op_yaml_info);
// int input_index = 0;
std::vector<std::string> vec_param_list = runtime_info.infer_meta_param;
for (auto& t : vec_param_list) {
if (input_index_map.count(t)) {
// get information from input
ir::Value ptr = op->operand(input_index_map[t]).source();
auto in_var_name = name_map.at(ptr);
if (mutable_attr_type_map.count(t)) {
VLOG(6) << "ctx->EmplaceBack mutable attr: " << t << "\t"
<< in_var_name;
if (mutable_attr_type_map[t] == "paddle::dialect::IntArrayAttribute") {
ctx->EmplaceBackAttr(phi::IntArray(
*(scope->Var(in_var_name)->GetMutable<phi::DenseTensor>())));
} else if (mutable_attr_type_map[t] ==
"paddle::dialect::ScalarAttribute") {
ctx->EmplaceBackAttr(phi::Scalar(
*(scope->Var(in_var_name)->GetMutable<phi::DenseTensor>())));
} else {
PADDLE_THROW(phi::errors::Unimplemented("attr type not support [%s] ",
mutable_attr_type_map[t]));
}
} else {
VLOG(6) << "ctx->EmplaceBackInput: " << t << "\t" << in_var_name;
auto var = scope->Var(in_var_name);
const phi::TensorBase* tensor_in = &(var->Get<phi::DenseTensor>());
ctx->EmplaceBackInput(const_cast<phi::TensorBase*>(tensor_in));
}
}
if (attr_type_map.count(t)) {
auto type_name = attr_type_map[t];
if (type_name == "paddle::dialect::IntArrayAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::IntArrayAttribute>().data());
} else if (type_name == "paddle::dialect::DataTypeAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::DataTypeAttribute>().data());
} else if (type_name == "ir::Int32Attribute") {
ctx->EmplaceBackAttr(attr_map[t].dyn_cast<ir::Int32Attribute>().data());
} else if (type_name == "paddle::dialect::PlaceAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::PlaceAttribute>().data());
} else if (type_name == "paddle::dialect::ScalarAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::ScalarAttribute>().data());
} else {
PADDLE_THROW(phi::errors::Unimplemented("attr type not support [%s] ",
type_name));
}
VLOG(6) << "ctx->EmplaceBackAttr: " << t;
}
}
ir::Value out_ptr = op->result(0);
auto name = name_map.at(out_ptr);
ctx->EmplaceBackOutput(scope->Var(name)->Get<phi::DenseTensor>());
}
void BuildPhiKernelContext(
ir::Operation* op,
const std::unordered_map<ir::Value, std::string>& name_map,
paddle::framework::Scope* scope,
const OpInfoTuple& op_yaml_info,
phi::KernelContext* ctx,
std::map<std::string, std::vector<int>>* input_map,
std::map<std::string, std::vector<int>>* output_map) {
// inputs include input and mutable attributes
auto input_info = std::get<0>(op_yaml_info);
std::map<std::string, size_t> input_index_map;
std::map<std::string, std::string> mutable_attr_type_map;
int input_index = 0;
for (auto& t : input_info) {
VLOG(6) << t.name << "\t" << t.type_name;
input_index_map[t.name] = input_index++;
if (t.is_mutable_attribute) {
mutable_attr_type_map[t.name] = t.type_name;
}
}
auto attr_info = std::get<1>(op_yaml_info);
std::map<std::string, std::string> attr_type_map;
for (auto& t : attr_info) {
VLOG(6) << t.name << "\t" << t.type_name;
attr_type_map[t.name] = t.type_name;
}
auto attr_map = op->attributes();
auto runtime_info = std::get<3>(op_yaml_info);
// int input_index = 0;
std::vector<std::string> vec_param_list = runtime_info.kernel_param;
for (auto& t : vec_param_list) {
if (input_index_map.count(t)) {
// get information from input
ir::Value ptr = op->operand(input_index_map[t]).source();
auto in_var_name = name_map.at(ptr);
if (input_map != nullptr) {
// only deal with single input for now, [todo] need support multi input
// like concat
size_t tmp_id = std::atol(in_var_name.substr(4, 100).c_str());
(*input_map)[std::to_string(input_index_map.at(t))].push_back(tmp_id);
}
if (mutable_attr_type_map.count(t)) {
VLOG(6) << "ctx->EmplaceBack mutable attr: " << t << "\t"
<< in_var_name;
if (mutable_attr_type_map[t] == "paddle::dialect::IntArrayAttribute") {
ctx->EmplaceBackAttr(phi::IntArray(
*(scope->Var(in_var_name)->GetMutable<phi::DenseTensor>())));
} else if (mutable_attr_type_map[t] ==
"paddle::dialect::ScalarAttribute") {
ctx->EmplaceBackAttr(phi::Scalar(
*(scope->Var(in_var_name)->GetMutable<phi::DenseTensor>())));
} else {
PADDLE_THROW(phi::errors::Unimplemented("attr type not support [%s] ",
mutable_attr_type_map[t]));
}
} else {
VLOG(6) << "ctx->EmplaceBackInput: " << t << "\t" << in_var_name;
PADDLE_ENFORCE_NOT_NULL(
scope->FindLocalVar(in_var_name),
phi::errors::PreconditionNotMet("can not find var[%s] in scope",
in_var_name));
auto var = scope->Var(in_var_name);
const phi::TensorBase* tensor_in = &(var->Get<phi::DenseTensor>());
ctx->EmplaceBackInput(tensor_in);
}
}
if (attr_type_map.count(t)) {
auto type_name = attr_type_map[t];
if (type_name == "paddle::dialect::IntArrayAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::IntArrayAttribute>().data());
} else if (type_name == "paddle::dialect::DataTypeAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::DataTypeAttribute>().data());
} else if (type_name == "ir::Int32Attribute") {
ctx->EmplaceBackAttr(attr_map[t].dyn_cast<ir::Int32Attribute>().data());
} else if (type_name == "paddle::dialect::PlaceAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::PlaceAttribute>().data());
} else if (type_name == "paddle::dialect::ScalarAttribute") {
ctx->EmplaceBackAttr(
attr_map[t].dyn_cast<paddle::dialect::ScalarAttribute>().data());
} else {
PADDLE_THROW(phi::errors::Unimplemented("attr type not support [%s] ",
type_name));
}
VLOG(6) << "ctx->EmplaceBackAttr: " << t;
}
}
ir::Value out_ptr = op->result(0);
auto name = name_map.at(out_ptr);
ctx->EmplaceBackOutput(const_cast<phi::DenseTensor*>(
&(scope->Var(name)->Get<phi::DenseTensor>())));
if (output_map != nullptr) {
// only deal with single input for now, [todo] need support multi input like
// concat
size_t tmp_id = std::atol(name.substr(4, 100).c_str());
(*output_map)["out"].push_back(tmp_id);
}
}
} // namespace ir
// Copyright (c) 2023 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/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/dialect/pd_type.h"
#include "paddle/fluid/ir/dialect/utils.h"
#include "paddle/fluid/ir/interface/op_yaml_info.h"
#include "paddle/ir/core/builtin_attribute.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/ir/core/utils.h"
#include "paddle/phi/core/meta_tensor.h"
#include "paddle/fluid/framework/scope.h"
#include "paddle/fluid/framework/tensor.h"
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/phi/core/kernel_context.h"
#include "paddle/fluid/ir/dialect/kernel_attribute.h"
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "paddle/phi/core/infermeta_utils.h"
#include "glog/logging.h"
namespace ir {
void BuildScope(ir::Block* block,
paddle::framework::Scope* scope,
std::unordered_map<ir::Value, std::string>* name_map);
void BuildInferMetaContext(
ir::Operation* op,
const std::unordered_map<ir::Value, std::string>& name_map,
paddle::framework::Scope* scope,
const OpInfoTuple& op_yaml_info,
phi::InferMetaContext* ctx);
void BuildPhiKernelContext(
ir::Operation* op,
const std::unordered_map<ir::Value, std::string>& name_map,
paddle::framework::Scope* scope,
const OpInfoTuple& op_yaml_info,
phi::KernelContext* ctx,
std::map<std::string, std::vector<int>>* input_map = nullptr,
std::map<std::string, std::vector<int>>* output_map = nullptr);
} // namespace ir
......@@ -24,6 +24,7 @@
#include "paddle/phi/common/layout.h"
#include "paddle/phi/common/place.h"
#include "paddle/phi/common/scalar.h"
#include "paddle/phi/core/utils/data_type.h"
#include "paddle/utils/variant.h"
namespace paddle {
......@@ -166,8 +167,9 @@ class DataTypeAttributeVisitor : public AttributeVisitor {
using AttributeVisitor::AttributeVisitor;
ir::Attribute operator()(int i) override {
VLOG(10) << "translating int to DataType: " << i;
phi::DataType data = static_cast<phi::DataType>(i);
return paddle::dialect::DataTypeAttribute::get(ctx, data);
auto phi_dtype = phi::TransToPhiDataType(i);
return paddle::dialect::DataTypeAttribute::get(ctx, phi_dtype);
}
};
......
......@@ -104,7 +104,7 @@ endif()
cc_library(
init
SRCS init.cc
DEPS device_context phi memcpy)
DEPS device_context phi memcpy pd_dialect new_ir)
# memcpy depends on device_context, here add deps individually for
# avoiding cycle dependencies
......
......@@ -51,9 +51,13 @@ limitations under the License. */
#include "paddle/fluid/platform/device/ipu/ipu_info.h"
#endif
#include "paddle/fluid/ir/dialect/pd_dialect.h"
#include "paddle/fluid/memory/allocation/allocator_facade.h"
#include "paddle/fluid/memory/memory.h"
#include "paddle/fluid/platform/flags.h"
#include "paddle/ir/core/builtin_dialect.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/phi/common/memory_utils.h"
#include "paddle/phi/core/custom_kernel.h"
......@@ -198,6 +202,9 @@ void InitDevices() {
}
void InitDevices(const std::vector<int> devices) {
ir::IrContext *ctx = ir::IrContext::Instance();
ctx->GetOrRegisterDialect<paddle::dialect::PaddleDialect>();
std::vector<platform::Place> places;
for (size_t i = 0; i < devices.size(); ++i) {
......
......@@ -38,6 +38,10 @@ set(PYBIND_DEPS
global_utils
phi_utils
phi
phi_kernel_adaptor
pd_dialect
new_ir
program_translator
new_profiler
jit_layer
jit_property
......
......@@ -210,6 +210,7 @@ PYBIND11_MAKE_OPAQUE(paddle::framework::FetchList);
PYBIND11_MAKE_OPAQUE(paddle::framework::FetchType);
DECLARE_FILE_SYMBOLS(init_phi);
DECLARE_FILE_SYMBOLS(kernel_dialect);
namespace paddle {
namespace pybind {
......
......@@ -1256,3 +1256,15 @@ PHI_DEFINE_EXPORTED_bool(use_shm_cache,
PHI_DEFINE_EXPORTED_string(tensor_operants_mode,
"eager",
"Tensor operants mode");
/**
* Using new IR in executor FLAG
* Name: enable_new_ir_in_executor
* Since Version: 2.6.0
* Value Range: bool, default=false
* Example:
* Note: If Ture, executor will use new IR
*/
PHI_DEFINE_EXPORTED_bool(enable_new_ir_in_executor,
false,
"Enable new IR in executor");
......@@ -214,6 +214,14 @@ if(TARGET standalone_executor_test)
endif()
endif()
if(TARGET standalone_executor_new_ir_test)
if(NOT WIN32)
set_tests_properties(
standalone_executor_new_ir_test
PROPERTIES ENVIRONMENT "FLAGS_enable_new_ir_in_executor=true")
endif()
endif()
if(TARGET layer_test)
add_dependencies(layer_test jit_download_program)
add_dependencies(layer_test_new jit_download_program)
......
......@@ -39,6 +39,7 @@ cc_test_old(
DEPS
new_ir
pd_dialect
phi_kernel_adaptor
phi
gtest)
......
......@@ -41,8 +41,8 @@
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "paddle/fluid/ir/phi_kernel_adaptor/phi_kernel_adaptor.h"
#include "paddle/phi/core/kernel_registry.h"
#include "test/cpp/ir/core/phi_kernel_adaptor.h"
PD_DECLARE_KERNEL(full, CPU, ALL_LAYOUT);
PD_DECLARE_KERNEL(full_int_array, CPU, ALL_LAYOUT);
......
......@@ -38,7 +38,8 @@
#include "paddle/phi/infermeta/nullary.h"
// Define op
class OperationTest : public ir::Op<OperationTest, InferShapeInterface> {
class OperationTest
: public ir::Op<OperationTest, paddle::dialect::InferShapeInterface> {
public:
using Op::Op;
static const char *name() { return "test.operation2"; }
......@@ -84,7 +85,8 @@ TEST(infershape_test, infershape_test) {
ir::Operation *op =
ir::Operation::Create(op_inputs, {}, op_output_types, op_info);
InferShapeInterface interface = op->dyn_cast<InferShapeInterface>();
paddle::dialect::InferShapeInterface interface =
op->dyn_cast<paddle::dialect::InferShapeInterface>();
phi::InferMetaContext infer_meta_ctx;
infer_meta_ctx.EmplaceBackAttr(phi::IntArray({5, 6}));
infer_meta_ctx.EmplaceBackAttr(phi::DataType::FLOAT32);
......
......@@ -46,9 +46,9 @@
#include "glog/logging.h"
void build_scope(ir::Block* block,
paddle::framework::Scope* scope,
std::unordered_map<ir::Value, std::string>* name_map) {
void BuildScope(ir::Block* block,
paddle::framework::Scope* scope,
std::unordered_map<ir::Value, std::string>* name_map) {
std::unordered_map<ir::Value, int> map_test;
int count = 0;
......@@ -192,7 +192,7 @@ class PhiKernelAdaptor {
void run(ir::Program* program) {
auto block = program->block();
std::unordered_map<ir::Value, std::string> name_map;
build_scope(block, scope_, &name_map);
BuildScope(block, scope_, &name_map);
auto* dev_ctx = phi::DeviceContextPool::Instance().Get(phi::CPUPlace());
phi::Place cpu_place(phi::AllocationType::CPU);
......@@ -248,7 +248,7 @@ class PhiKernelAdaptor {
void run_kernel_prog(ir::Program* program) {
auto block = program->block();
std::unordered_map<ir::Value, std::string> name_map;
build_scope(block, scope_, &name_map);
BuildScope(block, scope_, &name_map);
ir::IrContext* ctx = ir::IrContext::Instance();
ctx->GetOrRegisterDialect<paddle::dialect::PaddleDialect>();
......
......@@ -210,7 +210,7 @@ TEST(type_test, custom_type_dialect) {
EXPECT_EQ(int8.dialect().id(), ir::TypeId::get<IntegerDialect>());
std::vector<ir::Dialect *> dialect_list = ctx->GetRegisteredDialects();
EXPECT_EQ(dialect_list.size() == 3, 1); // integer, builtin, fake
EXPECT_EQ(dialect_list.size() == 4, 1); // integer, builtin, fake
ir::Dialect *dialect_builtin1 = ctx->GetRegisteredDialect("builtin");
ir::Dialect *dialect_builtin2 =
......
......@@ -6,5 +6,6 @@ cc_test_old(
pd_op_to_kernel_pass
new_ir
pd_dialect
phi_kernel_adaptor
phi
gtest)
......@@ -15,38 +15,32 @@
#include <gtest/gtest.h>
#include <sstream>
#include "paddle/fluid/framework/scope.h"
#include "paddle/fluid/framework/tensor.h"
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/fluid/ir/dialect/kernel_dialect.h"
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "paddle/fluid/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/dialect/pd_type.h"
#include "paddle/fluid/ir/dialect/utils.h"
#include "paddle/fluid/ir/interface/op_yaml_info.h"
#include "paddle/fluid/ir/pass/pd_op_to_kernel_pass.h"
#include "paddle/fluid/ir/phi_kernel_adaptor/phi_kernel_adaptor.h"
#include "paddle/fluid/platform/init.h"
#include "paddle/ir/core/builtin_attribute.h"
#include "paddle/ir/core/builtin_dialect.h"
#include "paddle/ir/core/builtin_op.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/ir/core/utils.h"
#include "paddle/phi/core/meta_tensor.h"
#include "paddle/phi/infermeta/binary.h"
#include "paddle/phi/kernels/elementwise_add_kernel.h"
#include "paddle/fluid/framework/scope.h"
#include "paddle/fluid/framework/tensor.h"
#include "paddle/fluid/framework/variable.h"
#include "paddle/fluid/framework/variable_helper.h"
#include "paddle/phi/common/place.h"
#include "paddle/phi/core/kernel_context.h"
#include "paddle/phi/core/kernel_factory.h"
#include "paddle/fluid/platform/init.h"
#include "paddle/fluid/ir/dialect/pd_attribute.h"
#include "test/cpp/ir/core/phi_kernel_adaptor.h"
#include "paddle/phi/core/kernel_registry.h"
#include "paddle/fluid/ir/dialect/kernel_dialect.h"
#include "paddle/phi/core/meta_tensor.h"
#include "paddle/phi/infermeta/binary.h"
#include "paddle/phi/kernels/elementwise_add_kernel.h"
PD_DECLARE_KERNEL(full, CPU, ALL_LAYOUT);
PD_DECLARE_KERNEL(full_int_array, CPU, ALL_LAYOUT);
......
if(NOT WIN32)
cc_test(
standalone_executor_new_ir_test
SRCS standalone_executor_new_ir_test.cc
DEPS phi_kernel_adaptor new_ir pd_dialect)
endif()
# skip win32 since wget is not installed by default on windows machine.
set(OPS
fill_constant_op
uniform_random_op
lookup_table_op
transpose_op
reshape_op
split_op
slice_op
concat_op
matmul_op
elementwise_add_op
elementwise_mul_op
softmax_with_cross_entropy_op
reduce_mean_op
activation_op
sum_op
elementwise_div_op
generated_op
generated_static_op
squared_l2_norm_op
memcpy_h2d_op
memcpy_d2h_op
fetch_v2_op)
if(WITH_GPU
AND WITH_TESTING
AND NOT WIN32)
......@@ -11,29 +42,6 @@ if(WITH_GPU
WORKING_DIRECTORY "${CC_TESTS_DIR}")
# all operators used in the program
set(OPS
fill_constant_op
uniform_random_op
lookup_table_op
transpose_op
reshape_op
split_op
slice_op
concat_op
matmul_op
elementwise_add_op
elementwise_mul_op
softmax_with_cross_entropy_op
reduce_mean_op
activation_op
sum_op
elementwise_div_op
generated_op
generated_static_op
squared_l2_norm_op
memcpy_h2d_op
memcpy_d2h_op
fetch_v2_op)
# All deps of the operators above, part of GLOB_OPERATOR_DEPS.
set(OP_DEPS phi concat_and_split cross_entropy)
......
// 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/framework/new_executor/standalone_executor.h"
#include <gtest/gtest.h>
#include <chrono>
#include <iostream>
#include <string>
#include "paddle/phi/core/kernel_registry.h"
#include "paddle/fluid/ir/dialect/pd_dialect.h"
#include "paddle/fluid/ir/dialect/pd_op.h"
#include "paddle/fluid/ir/pass/pd_op_to_kernel_pass.h"
#include "paddle/ir/core/builder.h"
#include "paddle/ir/core/ir_context.h"
#include "paddle/ir/core/program.h"
#include "paddle/fluid/platform/init_phi.h"
DECLARE_FILE_SYMBOLS(kernel_dialect);
PD_DECLARE_KERNEL(full, CPU, ALL_LAYOUT);
PD_DECLARE_KERNEL(full_int_array, CPU, ALL_LAYOUT);
PD_DECLARE_KERNEL(uniform, CPU, ALL_LAYOUT);
PD_DECLARE_KERNEL(add, CPU, ALL_LAYOUT);
namespace paddle {
namespace framework {
TEST(StandaloneExecutor, run) {
std::cerr << "here" << std::endl;
ir::IrContext* ctx = ir::IrContext::Instance();
ir::Program program((ctx));
ctx->GetOrRegisterDialect<paddle::dialect::PaddleDialect>();
ir::Builder builder = ir::Builder(ctx, program.block());
paddle::dialect::FullOp op1 = builder.Build<paddle::dialect::FullOp>(
std::vector<int64_t>{2, 2}, 1.0, phi::DataType::FLOAT32, phi::CPUPlace());
paddle::dialect::FullOp op2 = builder.Build<paddle::dialect::FullOp>(
std::vector<int64_t>{2, 2}, 1.0, phi::DataType::FLOAT32, phi::CPUPlace());
builder.Build<paddle::dialect::AddOp>(op1->result(0), op2->result(0));
program.Print(std::cout);
auto kernel_program = paddle::dialect::PdOpLowerToKernelPass(&program);
kernel_program->Print(std::cout);
auto place = platform::CPUPlace();
Scope scope;
ProgramDesc prog_desc;
InterpreterCore test_core(
place, prog_desc.Block(0), &scope, kernel_program.get());
test_core.Run({});
auto tensor = scope.Var("inner_var_2")->Get<phi::DenseTensor>();
std::cerr << "uot" << tensor << std::endl;
}
} // namespace framework
} // namespace paddle
......@@ -16,6 +16,7 @@ foreach(target ${TEST_IR_PASSES})
endforeach()
add_subdirectory(inference)
add_subdirectory(new_ir)
set_tests_properties(test_fuse_resnet_unit PROPERTIES TIMEOUT 120)
set_tests_properties(test_convert_to_mixed_precision PROPERTIES TIMEOUT 300)
file(
GLOB TEST_INTERP_CASES
RELATIVE "${CMAKE_CURRENT_SOURCE_DIR}"
"test_*.py")
string(REPLACE ".py" "" TEST_INTERP_CASES "${TEST_INTERP_CASES}")
foreach(target ${TEST_INTERP_CASES})
py_test_modules(${target} MODULES ${target} ENVS GLOG_v=1
FLAGS_enable_new_ir_in_executor=true)
endforeach()
# Copyright (c) 2023 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 unittest
import numpy as np
import paddle
paddle.enable_static()
class TestNewIr(unittest.TestCase):
def test_with_new_ir(self):
place = paddle.CPUPlace()
exe = paddle.static.Executor(place)
x = paddle.ones([2, 2], dtype="float32")
y = paddle.ones([2, 2], dtype="float32")
z = x + y
out = exe.run(
paddle.static.default_main_program(), {}, fetch_list=[z.name]
)
gold_res = np.ones([2, 2], dtype="float32") * 2
self.assertEqual(
np.array_equal(
np.array(
paddle.static.global_scope()
.find_var("inner_var_2")
.get_tensor()
),
gold_res,
),
True,
)
if __name__ == "__main__":
unittest.main()
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册