interpretercore_util.cc 43.0 KB
Newer Older
W
wanghuancoder 已提交
1 2 3 4 5 6 7 8 9 10 11 12 13 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/framework/new_executor/interpretercore_util.h"
15

16 17
#include <algorithm>

18
#include "paddle/fluid/framework/details/nan_inf_utils.h"
W
wanghuancoder 已提交
19
#include "paddle/fluid/framework/executor_gc_helper.h"
20
#include "paddle/fluid/framework/new_executor/data_transfer.h"
21
#include "paddle/fluid/framework/new_executor/interpreter/dependency_utils.h"
X
xiongkun 已提交
22 23 24
#include "paddle/fluid/operators/controlflow/conditional_block_op_helper.h"
#include "paddle/fluid/operators/controlflow/recurrent_op_helper.h"
#include "paddle/fluid/operators/controlflow/while_op_helper.h"
25
#include "paddle/phi/core/kernel_context.h"
26
#include "paddle/phi/core/kernel_factory.h"
W
wanghuancoder 已提交
27

L
Leo Chen 已提交
28 29 30 31
#ifdef PADDLE_WITH_MKLDNN
#include "paddle/fluid/platform/mkldnn_helper.h"
#endif

32 33 34 35 36
// The difference between "sequential_run" and "serial_run":
// "sequential_run" dispatches OPs one by one according to the sequence in the
// Program, while "serial_run" ensures that all Ops are scheduled in a singal
// thread. In standalone executor, "sequential_run" is also "serial_run", while
// "serial_run" is not necessarily "sequential_run".
37 38
PADDLE_DEFINE_EXPORTED_bool(new_executor_sequential_run,
                            false,
39 40 41
                            "Enable sequential execution for standalone "
                            "executor, only applied to GPU OPs.");

42
PADDLE_DEFINE_EXPORTED_bool(
43 44
    new_executor_serial_run,
    false,
45
    "Enable serial execution for standalone executor, used for debug.");
46

47
DECLARE_bool(use_mkldnn);
48
DECLARE_bool(check_nan_inf);
49

W
wanghuancoder 已提交
50 51
namespace paddle {
namespace framework {
52
namespace interpreter {
53

54
using VariableIdMap = std::map<std::string, std::vector<int>>;
L
liutiexing 已提交
55
constexpr size_t kPrepareWorkQueueIdx = 2;
56
const char blocking_queue_prefix[] = "lod_tensor_blocking_queue";
L
liutiexing 已提交
57

58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95
const std::vector<WorkQueueOptions> ConstructWorkQueueOptions(
    size_t host_num_threads, size_t device_num_threads, EventsWaiter* waiter) {
  std::vector<WorkQueueOptions> group_options;
  // for execute host Kernel
  group_options.emplace_back(/*name*/ "HostTasks",
                             /*num_threads*/ host_num_threads,
                             /*allow_spinning*/ true,
                             /*always_spinning*/ false,
                             /*track_task*/ false,
                             /*detached*/ true,
                             /*events_waiter*/ waiter);
  // for launch device Kernel
  group_options.emplace_back(/*name*/ "DeviceKernelLaunch",
                             /*num_threads*/ device_num_threads,
                             /*allow_spinning*/ true,
                             /*always_spinning*/ true,
                             /*track_task*/ false,
                             /*detached*/ true,
                             /*events_waiter*/ waiter);
  // for prepare deps and others
  group_options.emplace_back(/*name*/ "Prepare",
                             /*num_threads*/ 1,
                             /*allow_spinning*/ true,
                             /*always_spinning*/ false,
                             /*track_task*/ false,
                             /*detached*/ true,
                             /*events_waiter*/ waiter);
  return group_options;
}

AsyncWorkQueue::AsyncWorkQueue(size_t host_num_threads,
                               size_t device_num_threads,
                               EventsWaiter* waiter)
    : host_num_thread_(host_num_threads) {
  queue_group_ = CreateWorkQueueGroup(
      ConstructWorkQueueOptions(host_num_threads, device_num_threads, waiter));
}

96 97
void AsyncWorkQueue::AddTask(const OpFuncType& op_func_type,
                             std::function<void()> fn) {
98
  VLOG(4) << "Add task: " << static_cast<size_t>(op_func_type) << " ";
99 100
  // NOTE(zhiqiu): use the second queue of size of, so only one thread is used.
  if (FLAGS_new_executor_serial_run) {
101 102 103 104 105 106 107
    queue_group_->AddTask(static_cast<size_t>(OpFuncType::kQueueAsync),
                          std::move(fn));
  } else {
    queue_group_->AddTask(static_cast<size_t>(op_func_type), std::move(fn));
  }
}

108 109 110 111 112 113 114 115 116 117 118 119 120 121
std::future<std::unique_ptr<AtomicVectorSizeT>>
AsyncWorkQueue::PrepareAtomicDeps(const std::vector<size_t>& dependecy_count) {
  VLOG(4) << "PrepareAtomicDeps";
  return queue_group_->AddAwaitableTask(
      kPrepareWorkQueueIdx, interpreter::PrepareAtomicDeps, dependecy_count);
}

std::future<std::unique_ptr<AtomicVectorSizeT>>
AsyncWorkQueue::PrepareAtomicVarRef(
    const std::vector<VariableMetaInfo>& vec_meta_info) {
  VLOG(4) << "PrepareAtomicVarRef";
  return queue_group_->AddAwaitableTask(
      kPrepareWorkQueueIdx, interpreter::PrepareAtomicVarRef, vec_meta_info);
}
W
wanghuancoder 已提交
122

123
std::unique_ptr<AtomicVectorSizeT> PrepareAtomicDeps(
124
    const std::vector<size_t>& dependecy_count) {
125
  VLOG(4) << "PrepareAtomicDeps";
126 127 128 129 130 131 132

  auto op_deps = std::make_unique<AtomicVectorSizeT>(dependecy_count.size());
  for (size_t i = 0; i < dependecy_count.size(); ++i) {
    (*op_deps)[i] = dependecy_count[i];
  }
  VLOG(4) << "AtomicDeps:" << op_deps.get() << " " << op_deps->size();
  return op_deps;
133 134
}

135
std::unique_ptr<AtomicVectorSizeT> PrepareAtomicVarRef(
136
    const std::vector<VariableMetaInfo>& vec_meta_info) {
137
  VLOG(4) << "PrepareAtomicVarRef";
138 139 140 141 142 143
  auto var_ref = std::make_unique<AtomicVectorSizeT>(vec_meta_info.size());
  for (size_t i = 0; i < vec_meta_info.size(); ++i) {
    (*var_ref)[i] = vec_meta_info[i].var_ref_count_;
  }
  VLOG(4) << "AtomicVarRef:" << var_ref.get() << " " << var_ref->size();
  return var_ref;
144 145
}

W
wanghuancoder 已提交
146 147 148 149 150 151 152 153 154 155 156 157 158 159 160
bool var_can_be_deleted(const std::string& name, const BlockDesc& block) {
  auto* var_desc = block.FindVar(name);
  if (var_desc == nullptr || var_desc->Persistable()) {
    return false;
  }

  auto type = var_desc->Proto()->type().type();

  return type == proto::VarType::LOD_TENSOR ||
         type == proto::VarType::SELECTED_ROWS ||
         type == proto::VarType::LOD_TENSOR_ARRAY;
}

std::unordered_map<const paddle::framework::OperatorBase*,
                   std::vector<std::string>>
L
Leo Chen 已提交
161 162
get_unused_vars(const BlockDesc& block,
                const std::vector<std::shared_ptr<OperatorBase>>& ops) {
W
wanghuancoder 已提交
163 164 165
  std::unordered_map<std::string, size_t> var_op_idx_map;

  for (size_t i = 0; i < ops.size(); ++i) {
L
Leo Chen 已提交
166
    const auto& op = ops[i];
W
wanghuancoder 已提交
167 168 169 170 171 172 173 174 175 176

    OpInOutInfo info;
    for (auto& name_pair : op->Inputs()) {
      for (auto& name : name_pair.second) {
        if (!var_can_be_deleted(name, block)) {
          continue;
        }

        // var can be gc-ed
        if (!info.IsBuilt()) {
L
Leo Chen 已提交
177
          info.Build(op.get());
W
wanghuancoder 已提交
178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204
        }

        if (info.IsInArgBufferNeeded(name)) {
          // Update the last living op of variable to current op
          var_op_idx_map[name] = i;
        } else {
          VLOG(10) << "Skip reference count computing of variable "
                   << name_pair.first << "(" << name << ") in Operator "
                   << op->Type();
        }
      }
    }

    for (auto& name_pair : op->Outputs()) {
      for (auto& name : name_pair.second) {
        if (var_can_be_deleted(name, block)) {
          // Update the last living op of variable to current op
          var_op_idx_map[name] = i;
        }
      }
    }
  }

  std::unordered_map<const OperatorBase*, std::vector<std::string>> result;
  for (auto& name_op_idx_pair : var_op_idx_map) {
    auto& name = name_op_idx_pair.first;
    size_t op_idx = name_op_idx_pair.second;
L
Leo Chen 已提交
205 206

    result[ops[op_idx].get()].emplace_back(name);
207
    VLOG(4) << ops[op_idx].get()->Type() << " " << name;
W
wanghuancoder 已提交
208
  }
209
  VLOG(4) << "gc map size:" << result.size();
W
wanghuancoder 已提交
210 211 212
  return result;
}

213
void build_variable_scope(const framework::BlockDesc& block,
214 215
                          VariableScope* var_scope,
                          bool use_local_scope) {
216 217 218 219 220 221 222 223
  VLOG(3) << "Creating Variables";
  auto inner_scope = var_scope->GetMutableScope();

  // NOTE(zhiqiu): if create_local_scope_ is true, the persistable is
  // created in var_scope.scope_ , and other scope is created in local scope.
  Scope* local_scope = use_local_scope ? var_scope->GetMutableLocalScope()
                                       : var_scope->GetMutableScope();

224
  for (auto& var_desc : block.AllVars()) {
225
    auto var_name = var_desc->Name();
X
xiongkun 已提交
226 227 228
    // TODO(xiongkun): user may create a variable with name that exists before.
    // under such circumstances, we should raise a error. Currently we can't
    // get the var_desc of startup_program, so leave it later.
229
    if (var_name == framework::kEmptyVarName) {
W
wanghuancoder 已提交
230 231
      continue;
    }
232

233 234
    if (var_desc->Persistable()) {
      auto* ptr = inner_scope->Var(var_name);
W
wanghuancoder 已提交
235

236
      VLOG(3) << "Initialize Variable " << var_name;
237 238
      // NOTE(zhiqiu): if var exists in scope and the type is right,
      // InitializeVariable will not create a new variable.
239 240 241
      InitializeVariable(ptr, var_desc->GetType());
      VLOG(3) << "Create Variable " << var_name << " global, which pointer is "
              << ptr << " type is " << static_cast<int>(var_desc->GetType());
242
    } else {
243 244 245 246 247
      auto* ptr = local_scope->Var(var_name);
      InitializeVariable(ptr, var_desc->GetType());
      VLOG(3) << "Create Variable " << var_name << " locally, which pointer is "
              << ptr << "Variable Type "
              << static_cast<int>(var_desc->GetType());
W
wanghuancoder 已提交
248
    }
249
    var_scope->AddVar(var_name, var_desc);
W
wanghuancoder 已提交
250 251 252
  }
}

L
Leo Chen 已提交
253
void create_all_ops(const framework::BlockDesc& block,
X
xiongkun 已提交
254
                    std::vector<std::unique_ptr<OperatorBase>>* ops) {
255 256
  for (auto& op : block.AllOps()) {
    VLOG(3) << "CreateOp from : " << op->Type();
W
wanghuancoder 已提交
257 258 259 260 261

    auto& info = OpInfoMap::Instance().Get(op->Type());

    const VariableNameMap& inputs_names = op->Inputs();
    const VariableNameMap& outputs_names = op->Outputs();
262

W
wanghuancoder 已提交
263 264 265 266 267 268 269
    AttributeMap op_attr_map = op->GetAttrMap();

    if (info.Checker() != nullptr) {
      info.Checker()->Check(&op_attr_map);
    }
    auto op_base =
        info.Creator()(op->Type(), inputs_names, outputs_names, op_attr_map);
270 271 272 273 274 275 276 277 278 279

#ifdef PADDLE_WITH_MKLDNN
    if (FLAGS_use_mkldnn) {
      if (op->HasAttr("use_mkldnn")) {
        VLOG(4) << "Set use_mkldnn=True for " << op_base->Type();
        op_base->SetAttr("use_mkldnn", true);
      }
    }
#endif

X
xiongkun 已提交
280
    ops->emplace_back(std::unique_ptr<OperatorBase>(op_base));
W
wanghuancoder 已提交
281
  }
282 283 284
}

std::tuple<VariableValueMap, VariableIdMap> build_variable_map(
285 286
    const VariableNameMap& var_name_map,
    VariableScope* var_scope,
287
    Scope* local_scope,
288
    bool enforce_exist = true) {
289 290 291 292 293 294 295 296
  VariableValueMap name2var;
  VariableIdMap name2id;
  for (auto& item : var_name_map) {
    std::vector<Variable*> vars;
    std::vector<int> ids;
    vars.reserve(item.second.size());

    for (auto& var_name : item.second) {
297 298 299 300 301 302 303 304 305 306 307 308
      if (!var_scope->HasVar(var_name)) {
        // Hot fix for variables used in dataloader, like
        // 'lod_tensor_blocking_queue_0' These variables may be created in
        // scope, and it is not existed as variable in program.
        if (var_name.find(blocking_queue_prefix) != std::string::npos &&
            local_scope->FindVar(var_name)) {
          var_scope->AddVar(var_name, nullptr);
        } else if (!enforce_exist) {
          // skip the non-exist variable: such as recurrent_grad
          VLOG(4) << var_name << " don't exist in variable scope, skip it!";
          continue;
        }
309
      }
310
      auto* var = local_scope->FindVar(var_name);
311
      auto var_id = var_scope->VarId(var_name);
312
      vars.push_back(var);
313 314 315 316 317 318 319
      ids.push_back(var_id);
    }
    name2var[item.first] = std::move(vars);
    name2id[item.first] = std::move(ids);
  }
  return std::make_tuple(name2var, name2id);
}
W
wanghuancoder 已提交
320

321 322 323 324 325 326 327 328 329 330 331 332
void apply_device_guard(const OperatorBase* op_base,
                        const platform::Place& place,
                        OpKernelType* expected_kernel_key) {
  bool need_change_place =
      (op_base->HasAttr("op_device") &&
       (op_base->Attr<std::string>("op_device").length() > 0));
  if (need_change_place) {
    auto& op_device = op_base->Attr<std::string>("op_device");
    if (op_device == "cpu" || platform::is_cpu_place(place)) {
      VLOG(3) << "Switch into CPUPlace by device_guard.";
      expected_kernel_key->place_ = platform::CPUPlace();
    } else if (op_device.find("gpu") != std::string::npos &&
333 334 335 336
               platform::is_gpu_place(place)) {
      // when the Op that does not have GPUKernel is assigned to GPU, the
      // CPUKernel will be executed and a warning will be given at the same
      // time.
337 338 339 340 341 342 343 344 345 346
      if (op_base->SupportGPU()) {
        expected_kernel_key->place_ = place;
      } else {
        expected_kernel_key->place_ = platform::CPUPlace();
        LOG_FIRST_N(WARNING, 1)
            << "Op(" << op_base->Type()
            << ") has no CUDA implementation. It will be assigned to CPUPlace.";
      }
      VLOG(3) << "Switch into " << expected_kernel_key->place_
              << " by device_guard.";
347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376
    } else if (op_device.find("npu") != std::string::npos &&
               platform::is_npu_place(place)) {
      // when the Op that does not have NPUKernel is assigned to NPU, the
      // CPUKernel will be executed and a warning will be given at the same
      // time.
      if (op_base->SupportNPU()) {
        expected_kernel_key->place_ = place;
      } else {
        expected_kernel_key->place_ = platform::CPUPlace();
        LOG_FIRST_N(WARNING, 1)
            << "Op(" << op_base->Type()
            << ") has no NPU implementation. It will be assigned to CPUPlace.";
      }
      VLOG(3) << "Switch into " << expected_kernel_key->place_
              << " by device_guard.";
    } else if (op_device.find("xpu") != std::string::npos &&
               platform::is_xpu_place(place)) {
      // when the Op that does not have XPUKernel is assigned to XPU, the
      // CPUKernel will be executed and a warning will be given at the same
      // time.
      if (op_base->SupportXPU()) {
        expected_kernel_key->place_ = place;
      } else {
        expected_kernel_key->place_ = platform::CPUPlace();
        LOG_FIRST_N(WARNING, 1)
            << "Op(" << op_base->Type()
            << ") has no XPU implementation. It will be assigned to CPUPlace.";
      }
      VLOG(3) << "Switch into " << expected_kernel_key->place_
              << " by device_guard.";
377 378 379 380 381 382 383
    } else {
      PADDLE_THROW(
          platform::errors::Fatal("Unsupported current place %s", op_device));
    }
  }
}

384
void deal_operator_base(const platform::Place& place,
L
Leo Chen 已提交
385 386
                        const VariableScope* var_scope,
                        std::shared_ptr<OperatorBase> op_base,
387 388
                        OpFuncNode* op_func_node,
                        Scope* local_scope) {
389 390 391 392
  platform::DeviceContextPool& pool = platform::DeviceContextPool::Instance();
  auto* dev_ctx = pool.Get(place);
  // input, output is prepared. set the other attributes.
  op_func_node->operator_base_ = op_base;
393
  if (IsSupportedHetePlace(place)) {
394 395 396 397 398 399 400 401
    op_func_node->type_ = OpFuncType::kQueueAsync;
  } else if (platform::is_cpu_place(place)) {
    op_func_node->type_ = OpFuncType::kQueueSync;
  } else {
    PADDLE_THROW(
        platform::errors::Fatal("Unsupported current place %s", place));
  }

402
  op_func_node->kernel_func_ = nullptr;
403
  op_base->Run(*local_scope, place);  // Run without data transformer.
404 405 406 407 408 409 410 411 412 413 414 415

  std::unordered_set<int> no_data_transform_index;
  for (auto& it : op_func_node->input_index) {
    for (auto& id : it.second) {
      no_data_transform_index.emplace(id);
    }
  }
  op_func_node->no_data_transform_index =
      no_data_transform_index;  // all index is no-need-transform
  op_func_node->dev_ctx_ = dev_ctx;
}

416
void build_op_func_list(const platform::Place& place,
417
                        const framework::BlockDesc& block,
418
                        const std::set<std::string>& skip_gc_vars,
419
                        std::vector<OpFuncNode>* vec_func_list,
420 421
                        VariableScope* var_scope,
                        bool use_local_scope) {
422 423
  Scope* local_scope = use_local_scope ? var_scope->GetMutableLocalScope()
                                       : var_scope->GetMutableScope();
X
xiongkun 已提交
424 425 426 427 428 429 430 431 432 433 434 435 436
  std::vector<std::unique_ptr<OperatorBase>>
      ops_unique;  // its elements will be moved to vec_func_list
  // Step 1: create all ops for current block.
  create_all_ops(block, &ops_unique);
  // If gc is enabled and block size > 1
  const ProgramDesc& main_program = *block.Program();
  operators::PrepareSafeEagerDeletionOnConditionalOpAndConditionalGradOp(
      main_program, block.ID(), ops_unique);
  operators::PrepareSafeEagerDeletionOnWhileOpAndWhileGradOp(
      main_program, block.ID(), ops_unique);
  operators::PrepareSafeEagerDeletionOnRecurrentOpAndRecurrentGradOp(
      main_program, block.ID(), ops_unique);

L
Leo Chen 已提交
437 438 439 440
#ifdef PADDLE_WITH_MKLDNN
  platform::RegisterModelLayout(ops_unique, place);
#endif

441 442
  // its elements will be moved to vec_func_list
  std::vector<std::shared_ptr<OperatorBase>> ops;
X
xiongkun 已提交
443 444 445
  for (auto& op_unique : ops_unique) {
    ops.emplace_back(std::move(op_unique));
  }
446
  auto unused_var_map = get_unused_vars(block, ops);
W
wanghuancoder 已提交
447

L
Leo Chen 已提交
448 449
  for (size_t i = 0; i < ops.size(); ++i) {
    auto op = ops[i].get();
450
    VLOG(6) << "Build OpFuncNode from : " << op->Type();
W
wanghuancoder 已提交
451 452 453 454 455

    auto inputs_names = op->Inputs();
    auto outputs_names = op->Outputs();

    VariableValueMap ins_map;
456
    VariableIdMap ins_name2id;
457
    bool enforce_exist = true;
W
wanghuancoder 已提交
458 459 460 461 462 463 464
    if (op->Type() == "recurrent_grad" || op->Type() == "rnn_memory_helper" ||
        op->Type() == "rnn_memory_helper_grad" ||
        op->Type() == "conditional_block" ||
        op->Type() == "conditional_block_grad" || op->Type() == "while" ||
        op->Type() == "while_grad") {
      enforce_exist = false;
    }
465
    std::tie(ins_map, ins_name2id) =
466
        build_variable_map(inputs_names, var_scope, local_scope, enforce_exist);
W
wanghuancoder 已提交
467 468

    VariableValueMap outs_map;
469
    VariableIdMap outs_name2id;
470 471
    std::tie(outs_map, outs_name2id) = build_variable_map(
        outputs_names, var_scope, local_scope, enforce_exist);
W
wanghuancoder 已提交
472

473
    // step 1: build OpFuncNode
W
wanghuancoder 已提交
474
    OpFuncNode op_func_node;
475
    op_func_node.operator_base_ = ops[i];
W
wanghuancoder 已提交
476 477
    op_func_node.input_index = ins_name2id;
    op_func_node.output_index = outs_name2id;
478
    VLOG(4) << "Start run " << place << " " << op->DebugStringEx(local_scope);
479

480 481 482 483 484 485 486 487 488 489
#ifdef PADDLE_WITH_ASCEND_CL
    // NOTE(wangxi): nan/inf cannot be detected on NPU by checking the variable
    // values, but only through special `float_status` to checks whether
    // the operation is overflow. More about `float_status`, see:
    // https://gitee.com/ascend/modelzoo/issues/I3NF8V?from=project-issue
    if (FLAGS_check_nan_inf) {
      framework::details::NPUAllocAndClearFloatStatus(*op, *local_scope, place);
    }
#endif

490
    if (dynamic_cast<framework::OperatorWithKernel*>(op) == nullptr) {
491
      // op is not a operatorwithkernel, so direcly run OperatorBase::Run()
492
      deal_operator_base(place, var_scope, ops[i], &op_func_node, local_scope);
W
wanghuancoder 已提交
493
    } else {
494 495
      auto op_with_kernel = const_cast<framework::OperatorWithKernel*>(
          static_cast<const framework::OperatorWithKernel*>(op));
496 497 498 499
      // construct RuntimeContext and analysis KernelType
      RuntimeContext runtime_context({}, {});
      runtime_context.inputs.swap(ins_map);
      runtime_context.outputs.swap(outs_map);
500

501
      Scope scope, *runtime_scope = &scope;
502 503 504 505
      // NOTE(Ruibiao): We do not encourage directly using scope in OP kernel.
      // But some OPs do have such behavior (e.g., cinn_launch OP). Here special
      // treatment for them.
      if (op_with_kernel->Type() == "cinn_launch") {
506 507 508 509
        VLOG(6) << "OP(" << op_with_kernel->Type()
                << ") use scope in kernel, "
                   "so pass a real scope to "
                   "ExecutionContext";
510 511 512
        runtime_scope = local_scope;
      }

513 514
      auto& pool = platform::DeviceContextPool::Instance();
      auto* dev_ctx = pool.Get(place);
515 516
      auto exec_ctx = ExecutionContext(
          *op_with_kernel, *runtime_scope, *dev_ctx, runtime_context);
517 518
      auto expected_kernel_key =
          op_with_kernel->GetExpectedKernelType(exec_ctx);
519 520
      // change device by the device_guard()
      apply_device_guard(op, place, &expected_kernel_key);
521
      VLOG(4) << "expected_kernel_key : " << expected_kernel_key;
522

523
      // step 2. select op kernel
524 525
      auto run_phi_kernel = false;
      if (phi::KernelFactory::Instance().HasCompatiblePhiKernel(
526
              op_with_kernel->Type())) {
527 528
        auto pt_kernel_key = op_with_kernel->ChoosePhiKernel(exec_ctx);
        auto pt_kernel_name = op_with_kernel->PhiKernelSignature()->name;
529

530 531
        if (op_with_kernel->PhiKernel()->IsValid()) {
          run_phi_kernel = true;
532
        } else {
533
          if (!op_with_kernel->SupportsKernelType(expected_kernel_key)) {
534 535
            auto pt_cpu_kernel_key = FallBackToCpu(
                expected_kernel_key, pt_kernel_key, *op_with_kernel);
536
            op_with_kernel->ResetPhiKernel(
537
                new phi::Kernel(phi::KernelFactory::Instance().SelectKernel(
538
                    pt_kernel_name, pt_cpu_kernel_key)));
539
            if (op_with_kernel->PhiKernel()->IsValid()) {
540 541 542
              VLOG(6) << "Static mode PrepareImpl - kernel name: "
                      << pt_kernel_name
                      << " | kernel key: " << pt_cpu_kernel_key
543
                      << " | kernel: " << *(op_with_kernel->PhiKernel());
544 545
              op_with_kernel->ResetKernelType(new OpKernelType(
                  TransPhiKernelKeyToOpKernelType(pt_cpu_kernel_key)));
546
              run_phi_kernel = true;
547 548 549 550
            }
          }
        }
      }
551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569
      if (!run_phi_kernel) {
        op_with_kernel->ChooseKernel(exec_ctx);
        op_func_node.kernel_func_ = *op_with_kernel->kernel_func();
      } else {
        op_func_node.pt_kernel_ = op_with_kernel->PhiKernel();
      }
      auto kernel_type = *(op_with_kernel->kernel_type());
      if (kernel_type.place_ != dev_ctx->GetPlace()) {
        dev_ctx = pool.Get(kernel_type.place_);
      }
      op_func_node.dev_ctx_ = dev_ctx;
      if (IsSupportedHetePlace(kernel_type.place_)) {
        op_func_node.type_ = OpFuncType::kQueueAsync;
      } else if (platform::is_cpu_place(kernel_type.place_)) {
        op_func_node.type_ = OpFuncType::kQueueSync;
      } else {
        PADDLE_THROW(platform::errors::Fatal("Unsupported current place %s",
                                             kernel_type.place_));
      }
570
      VLOG(3) << op_with_kernel->Type()
571 572 573 574 575
              << " : finally selected kernel_key: " << kernel_type;

      // step 3. data transform
      VariableValueMap& ins_map_temp = runtime_context.inputs;
      VariableValueMap& outs_map_temp = runtime_context.outputs;
576 577 578 579 580 581 582
      ApplyDataTransform(kernel_type,
                         place,
                         &ins_map_temp,
                         &outs_map_temp,
                         var_scope,
                         &op_func_node,
                         vec_func_list,
583 584 585 586 587 588 589 590 591 592 593 594 595
                         use_local_scope);

      // step 4. infershape, see OperatorWithKernel::RunImpl in operator.cc for
      // why.
      if (!(op->HasAttr(kAllKernelsMustComputeRuntimeShape) &&
            op->Attr<bool>(kAllKernelsMustComputeRuntimeShape))) {
        InterpretercoreInferShapeContext infer_shape_ctx(*op, runtime_context);
        // TODO(Aurelius84): In case of control flow ops, they are NOT
        // inheritted from OperatorWithKernel.
        op_with_kernel->Info().infer_shape_(&infer_shape_ctx);
      }

      // step 5. run kernel
596
      if (run_phi_kernel) {
597
        phi::KernelContext pt_kernel_context;
598 599
        op_with_kernel->BuildPhiKernelContext(
            runtime_context, dev_ctx, &pt_kernel_context);
600
        (*op_func_node.pt_kernel_)(&pt_kernel_context);
601
      } else {
602 603 604
        // the place of exec_ctx maybe has changed.
        op_func_node.kernel_func_(ExecutionContext(
            *op_with_kernel, *runtime_scope, *dev_ctx, runtime_context));
605
      }
606

607 608
      // post-process grad_op.outputs if need cast complex grad into real
      // grad.
609
      // NOTE(Aurelius84): insert a transfer_dtype_op inplacely to cast it.
610
      if (framework::IsComplexType(kernel_type.data_type_)) {
611 612 613 614 615 616 617
        interpreter::HandleComplexGradToRealGrad(op_func_node,
                                                 place,
                                                 outputs_names,
                                                 &runtime_context.outputs,
                                                 var_scope,
                                                 vec_func_list,
                                                 local_scope);
618
      }
619 620
      if (!op_func_node.inplace_back_map.empty()) {
        auto& m = op_func_node.inplace_back_map;
621 622
        // NOTE(zhiqiu): same logic as TransferInplaceVarsBack() in
        // operator.cc
623 624 625
        for (auto& p : m) {
          auto* transformed_tensor =
              GetMutableLoDTensorOrSelectedRowsValueFromVar(
626
                  local_scope->FindVar(var_scope->GetNameById(p.first)));
627
          auto* original_tensor = GetMutableLoDTensorOrSelectedRowsValueFromVar(
628
              local_scope->FindVar(var_scope->GetNameById(p.second)));
629 630 631 632 633 634
          original_tensor->ShareDataWith(*transformed_tensor);
          VLOG(4) << "Transfer inplace variable back form "
                  << var_scope->GetNameById(p.first) << " to "
                  << var_scope->GetNameById(p.second);
        }
      }
635 636 637 638 639 640

      // for debug nan/inf
      if (FLAGS_check_nan_inf) {
        VLOG(4) << "Check nan/inf";
        framework::details::CheckOpHasNanOrInf(*op, *runtime_scope, place);
      }
641
    }
W
wanghuancoder 已提交
642

643 644 645
    VLOG(4) << "End run " << place << " "
            << op_func_node.operator_base_->DebugStringEx(local_scope);

L
Leo Chen 已提交
646
    vec_func_list->emplace_back(op_func_node);
647

W
wanghuancoder 已提交
648
    // gc---------------------------------------------------------------------------
L
Leo Chen 已提交
649
    auto iter = unused_var_map.find(op);
W
wanghuancoder 已提交
650 651 652 653 654 655 656 657 658
    if (iter == unused_var_map.end()) {
      continue;
    }

    auto& delete_vars = iter->second;
    std::deque<std::shared_ptr<memory::Allocation>>* garbages =
        new std::deque<std::shared_ptr<memory::Allocation>>();

    for (auto& var_name : delete_vars) {
659
      auto* var = local_scope->FindVar(var_name);
660
      if (var == nullptr || skip_gc_vars.find(var_name) != skip_gc_vars.end()) {
W
wanghuancoder 已提交
661 662 663
        continue;
      }

664
      VLOG(6) << "Erase variable " << var_name;
W
wanghuancoder 已提交
665 666 667
      if (var->IsType<LoDTensor>()) {
        garbages->emplace_back(
            var->GetMutable<LoDTensor>()->MoveMemoryHolder());
668 669
      } else if (var->IsType<phi::SelectedRows>()) {
        garbages->emplace_back(var->GetMutable<phi::SelectedRows>()
W
wanghuancoder 已提交
670 671 672 673 674 675 676 677 678 679
                                   ->mutable_value()
                                   ->MoveMemoryHolder());
      } else if (var->IsType<LoDTensorArray>()) {
        auto* lod_tensor_arr = var->GetMutable<LoDTensorArray>();
        for (auto& t : *lod_tensor_arr) {
          garbages->emplace_back(t.MoveMemoryHolder());
        }
      } else {
        PADDLE_THROW(platform::errors::Unimplemented(
            "Type %s of variable %s is not supported eager deletion.",
680 681
            framework::ToTypeName(var->Type()),
            var_name));
W
wanghuancoder 已提交
682 683 684 685 686 687
      }
    }
    delete garbages;  // free mem
  }
}

688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706
void add_fetch(const std::vector<std::string>& fetch_names,
               framework::BlockDesc* block) {
  auto* fetch_holder = block->Var(kFetchVarName);
  fetch_holder->SetType(proto::VarType::FETCH_LIST);
  fetch_holder->SetPersistable(true);

  int i = 0;
  for (auto& fetch_name : fetch_names) {
    // append fetch op
    auto* op = block->AppendOp();
    op->SetType("fetch_v2");
    op->SetInput("X", {fetch_name});
    op->SetOutput("Out", {kFetchVarName});
    op->SetAttr("col", {static_cast<int>(i)});
    op->CheckAttrs();
    i++;
  }
}

W
wanghuancoder 已提交
707 708 709
std::vector<size_t> merge_vector(const std::vector<size_t>& first,
                                 const std::vector<size_t>& second) {
  std::vector<size_t> out(first.size() + second.size());
710 711
  std::merge(
      first.begin(), first.end(), second.begin(), second.end(), out.begin());
W
wanghuancoder 已提交
712 713 714 715 716 717 718 719 720

  std::vector<size_t>::iterator it;
  it = std::unique(out.begin(), out.end());

  out.resize(std::distance(out.begin(), it));

  return out;
}

X
xiongkun 已提交
721
void update_var_min_rw_op(const std::map<int, std::set<int>>& op2dependences,
722
                          std::map<int, std::list<int>>* var2min_rw_op,
723 724
                          int cur_op,
                          int rw_var) {
X
xiongkun 已提交
725 726
  // rw_var is inputs or outputs of cur_op
  // this function update the var2min_rw_op set .
727
  if (var2min_rw_op->find(rw_var) == var2min_rw_op->end()) {
728
    (*var2min_rw_op)[rw_var] = std::list<int>();
729
  }
X
xiongkun 已提交
730
  for (auto dep_op : op2dependences.at(cur_op)) {
731
    var2min_rw_op->at(rw_var).remove(dep_op);
X
xiongkun 已提交
732
  }
733
  var2min_rw_op->at(rw_var).push_back(cur_op);
X
xiongkun 已提交
734 735
}

736 737 738 739 740 741 742 743 744 745 746 747 748
size_t CountDownstreamMap(const std::map<int, std::list<int>>& downstream_map) {
  size_t count = 0;
  for (auto pair : downstream_map) {
    count += pair.second.size();
  }
  return count;
}

const std::string StringizeDownstreamMap(
    const std::map<int, std::list<int>>& downstream_map) {
  std::ostringstream oss;
  for (auto pair : downstream_map) {
    oss << pair.first << " -> ";
749 750
    std::copy(pair.second.begin(),
              pair.second.end(),
751 752 753 754 755 756 757 758 759 760 761 762
              std::ostream_iterator<int>(oss, " "));
    oss << std::endl;
  }
  return oss.str();
}

// convert op2dependences to downstream_map directly. op2dependences is op ->
// it's dependences, we want to get op -> [next ops] map, where ops is the next
// instruction of op.
std::map<int, std::list<int>> GetDownstreamMap(
    const std::map<int, std::set<int>>& op2dependences) {
  std::map<int, std::list<int>> downstream_map;
X
xiongkun 已提交
763 764 765
  for (auto& item : op2dependences) {
    int op = item.first;
    for (auto dep_op : item.second) {
766
      AddDownstreamOp(dep_op, op, &downstream_map);
X
xiongkun 已提交
767 768
    }
  }
769

770 771 772
  VLOG(6) << "downstream count: " << CountDownstreamMap(downstream_map);
  VLOG(6) << "downstream_map: " << std::endl
          << StringizeDownstreamMap(downstream_map);
773

774 775
  return downstream_map;
}
776

777 778 779 780
void ShrinkDownstreamMap(std::map<int, std::list<int>>* downstream_map,
                         std::vector<std::vector<bool>>* op_happens_before,
                         size_t op_num) {
  // remove unnecessary downstream ops
781 782 783 784 785 786 787 788
  // for example, a->b->c
  // a: b, c
  // b: c
  // =>
  // a: b
  // b: c

  // happens_before[i][j] means i should be executed before j
789
  op_happens_before->assign(op_num, std::vector<bool>(op_num, false));
790 791 792 793 794 795 796 797 798 799

  // bfs to get all next ops
  auto bfs = [&](size_t op_idx) {
    std::queue<size_t> q;
    std::vector<bool> visited(op_num, false);
    q.push(op_idx);
    while (!q.empty()) {
      size_t op = q.front();
      q.pop();
      visited[op] = true;
800
      if (!downstream_map->count(op)) {
801 802
        continue;
      }
803
      for (auto next : downstream_map->at(op)) {
804
        if (!visited[next]) {
805 806
          PADDLE_ENFORCE_EQ((*op_happens_before)[next][op_idx],
                            false,
807 808 809
                            paddle::platform::errors::AlreadyExists(
                                "There exists circle in graph, expected "
                                "%d->%d, but already got %d->%d",
810 811 812 813
                                op_idx,
                                next,
                                next,
                                op_idx));
814 815 816 817 818 819 820 821 822 823 824 825 826 827 828
          (*op_happens_before)[op_idx][next] = true;
          VLOG(8) << "happens before: " << op_idx << " " << next;
          q.push(next);
        }
      }
    }
  };

  for (size_t i = 0; i < op_num; ++i) {
    bfs(i);
  }

  // shrink, find the downstream op that has no other op in the
  // downstream list happens before it
  for (size_t i = 0; i < op_num; ++i) {
829 830 831 832
    if (downstream_map->find(i) == downstream_map->end()) {
      continue;
    }

833
    std::list<int> minumum_nexts;
834
    for (size_t item : downstream_map->at(i)) {
835 836
      bool not_after_any = true;
      // find the op that is not executed after any
837
      for (size_t other_item : downstream_map->at(i)) {
838 839 840 841 842 843 844 845 846 847 848 849
        if ((*op_happens_before)[other_item][item]) {
          VLOG(8) << "happens_before: " << other_item << "->" << item
                  << ", so skip " << item;
          not_after_any = false;
          break;
        }
      }
      if (not_after_any) {
        VLOG(8) << "downstream op of " << i << ": " << item;
        minumum_nexts.push_back(item);
      }
    }
850
    downstream_map->at(i) = minumum_nexts;
851
  }
852 853 854
  VLOG(6) << "downstream count: " << CountDownstreamMap(*downstream_map);
  VLOG(6) << "downstream_map: " << std::endl
          << StringizeDownstreamMap(*downstream_map);
X
xiongkun 已提交
855 856 857
}

std::map<int, std::list<int>> build_op_downstream_map(
858 859
    const std::vector<Instruction>& vec_instruction,
    std::vector<std::vector<bool>>* op_happens_before) {
860 861 862
  auto var2min_rw_op =
      std::map<int, std::list<int>>();  // # map from variable id to read /
                                        // write op id.
X
xiongkun 已提交
863 864 865 866 867 868 869 870
  auto var2recent_write_op =
      std::map<int, int>();  // # map from variable to recent write op.
  auto op2dependences =
      std::map<int, std::set<int>>();  //# map from op to the dependence list,
                                       // op must run after the dependence.
  std::set<int>
      remove_duplicate;  // remove the duplicate between inputs and outputs

871 872
  size_t op_num = vec_instruction.size();

X
xiongkun 已提交
873
  // reserve
874
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
X
xiongkun 已提交
875 876 877
    op2dependences[op_idx] = std::set<int>();
  }

878
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
X
xiongkun 已提交
879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898
    remove_duplicate.clear();
    // step1: update the op2dependences structure
    for (auto& item :
         vec_instruction[op_idx].Inputs()) {  // for all inputs(read only)
      for (auto var : item.second) {
        if (var2recent_write_op.count(var))
          op2dependences[op_idx].insert(var2recent_write_op[var]);
      }
    }

    for (auto& item :
         vec_instruction[op_idx].Outputs()) {  // for all write vars
      for (auto var : item.second) {
        if (var2min_rw_op.count(var)) {
          for (auto dep_op : var2min_rw_op[var]) {
            op2dependences[op_idx].insert(dep_op);
          }
        }
      }
    }
899 900 901 902 903 904 905 906 907 908 909 910
    // the original output of inplace op is also change.
    if (!vec_instruction[op_idx].InplaceBackMap().empty()) {
      auto& m = vec_instruction[op_idx].InplaceBackMap();
      for (auto& p : m) {
        auto& var = p.second;
        if (var2min_rw_op.count(var)) {
          for (auto dep_op : var2min_rw_op[var]) {
            op2dependences[op_idx].insert(dep_op);
          }
        }
      }
    }
X
xiongkun 已提交
911 912 913

    // step2: update 2 var2xxxx data structure
    for (auto& item :
914
         vec_instruction[op_idx].Outputs()) {  // for all write vars
X
xiongkun 已提交
915
      for (auto var : item.second) {
916 917
        var2recent_write_op[var] = op_idx;
        var2min_rw_op[var] = {static_cast<int>(op_idx)};
X
xiongkun 已提交
918 919 920 921
        remove_duplicate.insert(var);
      }
    }

922 923 924 925 926 927 928 929 930 931
    // NOTE(zhiqiu): The inplace op with `transfer` also changes
    // original output after that so add original output as well
    // original: a->op->a
    // after: a->data_transfer->a'->op->a'->transfer_back->a
    // which means op writes a and a'
    if (!vec_instruction[op_idx].InplaceBackMap().empty()) {
      auto& m = vec_instruction[op_idx].InplaceBackMap();
      for (auto& p : m) {
        auto var = p.second;
        var2recent_write_op[var] = op_idx;
932 933 934 935 936 937 938 939 940 941
        var2min_rw_op[var] = {static_cast<int>(op_idx)};
        remove_duplicate.insert(var);
      }
    }

    for (auto& item :
         vec_instruction[op_idx].Inputs()) {  // for all inputs(read only)
      for (auto var : item.second) {
        if (remove_duplicate.count(var) ==
            0) {  // var in input list and in output list, so remove it.
942 943 944 945
          update_var_min_rw_op(op2dependences, &var2min_rw_op, op_idx, var);
        }
      }
    }
X
xiongkun 已提交
946
  }
947

948 949 950 951 952
  // NOTE(zhiqiu): the size of downstream != size of op2dependences since there
  // are some ops that have no downstream-op.
  std::map<int, std::list<int>> op_downstream_map =
      GetDownstreamMap(op2dependences);

953
  ShrinkDownstreamMap(&op_downstream_map, op_happens_before, op_num);
954

955 956 957
  // add dependences for random op, make sure that the random op is scheduled
  // sequentially
  const std::set<std::string> random_op_set = {
958 959 960 961 962 963 964 965
      "bernoulli",
      "poisson",
      "multinomial",
      "gaussian_random",
      "truncated_gaussian_random",
      "uniform_random",
      "randint",
      "randperm",
966 967 968 969 970
      "exponential",
      "sampling_id"
      "dropout",
      "class_center_sample",
  };
971

972
  int dependence_op_idx = -1;
973
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
974 975
    if (random_op_set.count(vec_instruction[op_idx].OpBase()->Type())) {
      if (dependence_op_idx != -1) {
976
        AddDownstreamOp(
977
            dependence_op_idx, op_idx, &op_downstream_map, op_happens_before);
978 979 980 981 982
      }
      dependence_op_idx = op_idx;
    }
  }

983
  // add dependency for communication op
984 985
  auto is_comm_op = [](std::string op) -> bool {
    const std::set<std::string> special_comm_op_set = {
986 987 988 989
        "send",
        "recv",
        "send_v2",
        "recv_v2",
990 991 992 993 994 995 996 997 998
    };
    const std::string communication_op_prefix = "c_";
    if (op.find(communication_op_prefix) != std::string::npos ||
        special_comm_op_set.count(op)) {
      return true;
    }
    return false;
  };

999
  dependence_op_idx = -1;
1000
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
1001
    if (is_comm_op(vec_instruction[op_idx].OpBase()->Type())) {
1002
      if (dependence_op_idx != -1) {
1003
        AddDownstreamOp(
1004
            dependence_op_idx, op_idx, &op_downstream_map, op_happens_before);
1005 1006 1007
        VLOG(4) << "Add depend from "
                << vec_instruction[dependence_op_idx].OpBase()->Type() << " to "
                << vec_instruction[op_idx].OpBase()->Type();
1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023
      }
      dependence_op_idx = op_idx;
    }
  }

  // TODO(zhiqiu): there still some cases not handled
  // add dependency for c_sync_comm_stream

  // in program, we can add only one c_sync_comm_stream to sync all
  // communication ops.
  // c_allreduce_sum(a)
  // c_allreduce_sum(b)
  // c_allreduce_sum(c)
  // c_sync_comm_stream(a)
  const std::string kSyncComm = "c_sync_comm_stream";
  dependence_op_idx = -1;
1024
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
1025 1026 1027 1028 1029 1030 1031
    if (vec_instruction[op_idx].OpBase()->Type() == kSyncComm) {
      dependence_op_idx = op_idx;
    } else {
      if (dependence_op_idx != -1) {
        VLOG(4) << "Add depend from "
                << vec_instruction[dependence_op_idx].OpBase()->Type() << " to "
                << vec_instruction[op_idx].OpBase()->Type();
1032
        AddDownstreamOp(
1033
            dependence_op_idx, op_idx, &op_downstream_map, op_happens_before);
1034 1035 1036 1037 1038 1039
      }
    }
  }

  // add dependency for coalesce_tensor
  const std::string kCoalesceTensor = "coalesce_tensor";
1040
  for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069
    if (vec_instruction[op_idx].OpBase()->Type() == kCoalesceTensor) {
      VLOG(4) << "Add depend for " << kCoalesceTensor << " " << op_idx;
      auto fused_out = vec_instruction[op_idx].Outputs().at("FusedOutput")[0];
      auto outputs = vec_instruction[op_idx].Outputs().at("Output");

      auto is_read = [](const Instruction& inst, int var_id) -> bool {
        for (auto pair : inst.Inputs()) {
          for (auto item : pair.second) {
            if (item == var_id) {
              return true;
            }
          }
        }
        return false;
      };

      auto is_write = [](const Instruction& inst, int var_id) -> bool {
        for (auto pair : inst.Outputs()) {
          for (auto item : pair.second) {
            if (item == var_id) {
              return true;
            }
          }
        }
        return false;
      };

      // find first op that reads fused_out
      auto first_read_fused_out_op = -1;
1070
      for (auto j = op_idx + 1; j < op_num; ++j) {
1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085
        if (is_read(vec_instruction[j], fused_out)) {
          first_read_fused_out_op = j;
          break;
        }
      }

      if (UNLIKELY(first_read_fused_out_op == -1)) {
        VLOG(4) << "No op read FusedOutput";
        continue;
      }

      // find ops that write 'outputs' between (op_index,
      // first_read_fused_out_op)
      // add depend: them->first_read_fused_out_op
      for (auto j = op_idx + 1;
1086 1087
           j < static_cast<size_t>(first_read_fused_out_op);
           ++j) {
1088 1089
        for (auto var_id : outputs) {
          if (is_write(vec_instruction[j], var_id)) {
1090 1091 1092
            AddDownstreamOp(j,
                            first_read_fused_out_op,
                            &op_downstream_map,
1093
                            op_happens_before);
1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112
            VLOG(4) << j << " -> " << first_read_fused_out_op;
            VLOG(4)
                << "Add depend from " << vec_instruction[j].OpBase()->Type()
                << " to "
                << vec_instruction[first_read_fused_out_op].OpBase()->Type();
          }
        }
      }

      // find first op read 'outputs' between (first_read_fused_out_op, end)
      // add depned:  first_read_fused_out_op -> first op that reads 'outputs'

      // special case for consecutive communication ops, for example,
      // FusedOutput = c_sync_calc_stream(FusedOutput)
      // FusedOutput= c_allreduce_sum(FusedOutput)
      // FusedOutput = c_sync_comm_stream(FusedOutput)
      // we should take the last one to add depned instead of
      // 'first_read_fused_out_op'
      size_t target = first_read_fused_out_op;
1113
      for (size_t j = first_read_fused_out_op + 1; j < op_num; ++j) {
1114
        if (j == target + 1 &&
1115 1116
            is_comm_op(vec_instruction[target].OpBase()->Type()) &&
            is_comm_op(vec_instruction[j].OpBase()->Type())) {
1117 1118 1119 1120 1121 1122 1123 1124 1125
          VLOG(4) << "Found consecutive communication ops, "
                  << vec_instruction[target].OpBase()->Type() << " -> "
                  << vec_instruction[j].OpBase()->Type();
          target = j;
          continue;
        }

        for (auto var_id : outputs) {
          if (is_read(vec_instruction[j], var_id)) {
1126
            AddDownstreamOp(target, j, &op_downstream_map, op_happens_before);
1127 1128 1129 1130 1131 1132 1133 1134 1135
            VLOG(4) << target << " -> " << j;
            VLOG(4) << "Add depend from "
                    << vec_instruction[target].OpBase()->Type() << " to "
                    << vec_instruction[j].OpBase()->Type();
          }
        }
      }
    }
  }
1136

1137 1138 1139 1140 1141
  if (FLAGS_new_executor_sequential_run) {
    dependence_op_idx = -1;
    for (size_t op_idx = 0; op_idx < op_num; ++op_idx) {
      if (!IsCpuOp(vec_instruction[op_idx])) {
        if (dependence_op_idx != -1) {
1142 1143
          AddDownstreamOp(
              dependence_op_idx, op_idx, &op_downstream_map, op_happens_before);
1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154
          VLOG(4) << "Add depend from "
                  << vec_instruction[dependence_op_idx].OpBase()->Type() << "("
                  << dependence_op_idx << ") to "
                  << vec_instruction[op_idx].OpBase()->Type() << "(" << op_idx
                  << ")";
        }
        dependence_op_idx = op_idx;
      }
    }
  }

1155 1156 1157 1158
  AddDependencyForReadOp(
      vec_instruction, &op_downstream_map, op_happens_before);

  VLOG(8) << "build_op_downstream_map finished";
1159 1160 1161 1162 1163
  VLOG(8) << "downstream count: " << CountDownstreamMap(op_downstream_map);
  VLOG(8) << "downstream_map: " << std::endl
          << StringizeDownstreamMap(op_downstream_map);

  return op_downstream_map;
X
xiongkun 已提交
1164 1165
}

1166
}  // namespace interpreter
W
wanghuancoder 已提交
1167 1168
}  // namespace framework
}  // namespace paddle