From 0bae07a44f3df20464631baff33dadb16ce98569 Mon Sep 17 00:00:00 2001 From: daquexian Date: Thu, 3 Mar 2022 19:19:05 +0800 Subject: [PATCH 01/54] remove IsMultiClient() and single client logic Signed-off-by: daquexian --- ci/test/distributed_run.py | 85 +------- docs/source/env.rst | 1 - oneflow/api/cpp/env.cpp | 8 +- oneflow/api/python/eager/single_client.cpp | 25 --- oneflow/api/python/env/env.cpp | 2 - oneflow/api/python/env/env.h | 10 +- oneflow/api/python/env/env_api.h | 10 +- oneflow/core/common/multi_client.h | 36 ---- oneflow/core/device/cuda_util.cpp | 9 +- .../core/framework/instructions_builder.cpp | 1 - .../multi_client_session_context.cpp | 2 - oneflow/core/framework/sbp_infer_util.cpp | 5 +- .../core/functional/impl/random_functor.cpp | 1 - oneflow/core/graph/task_graph.cpp | 1 - oneflow/core/job/job_build_and_infer_ctx.cpp | 2 - .../core/job/job_build_and_infer_ctx_mgr.cpp | 12 +- oneflow/core/job/parallel_desc.cpp | 50 +++-- oneflow/core/job/plan_util.cpp | 46 +--- oneflow/core/job_rewriter/autotick.cpp | 49 ----- oneflow/core/job_rewriter/autotick.h | 5 - .../gradient_accumulation_rewrite_pass.cpp | 197 ------------------ oneflow/core/job_rewriter/job_completer.cpp | 22 +- .../set_default_variable_conf.cpp | 85 -------- .../core/kernel/callback_notify_kernel.cpp | 10 +- .../critical_section_callback_tick_kernel.cpp | 3 - .../critical_section_wait_tick_kernel.cpp | 3 - oneflow/core/kernel/input_kernel.cpp | 3 - oneflow/core/kernel/output_kernel.cpp | 11 +- oneflow/core/kernel/return_kernel.cpp | 12 +- .../core/kernel/wait_and_send_ids_kernel.cpp | 15 +- oneflow/core/operator/output_op.cpp | 28 +-- oneflow/core/rpc/lib/grpc.cpp | 1 - oneflow/core/vm/id_generator.cpp | 22 +- oneflow/core/vm/virtual_machine.cpp | 5 - oneflow/core/vm/vm_util.cpp | 2 +- oneflow/user/data/distributed_util.h | 3 +- oneflow/user/data/ofrecord_dataset.h | 3 +- python/oneflow/__init__.py | 6 +- python/oneflow/__main__.py | 18 -- python/oneflow/env.py | 9 - python/oneflow/framework/c_api_util.py | 4 +- python/oneflow/framework/env_util.py | 15 +- python/oneflow/framework/unittest.py | 14 +- python/oneflow/test/graph/test_graph.py | 1 - .../oneflow/test/graph/test_input_op_expr.py | 1 - .../test/graph/test_multi_client_session.py | 2 - .../oneflow/test/graph/test_output_op_expr.py | 1 - .../oneflow/test/graph/test_user_op_expr.py | 1 - .../test/graph/test_variable_op_expr.py | 1 - python/oneflow/utils/data/distributed.py | 3 - 50 files changed, 70 insertions(+), 791 deletions(-) delete mode 100644 oneflow/api/python/eager/single_client.cpp delete mode 100644 oneflow/core/common/multi_client.h delete mode 100644 oneflow/core/job_rewriter/gradient_accumulation_rewrite_pass.cpp delete mode 100644 oneflow/core/job_rewriter/set_default_variable_conf.cpp diff --git a/ci/test/distributed_run.py b/ci/test/distributed_run.py index 55e97d9edde..5b95fc644f5 100644 --- a/ci/test/distributed_run.py +++ b/ci/test/distributed_run.py @@ -130,11 +130,7 @@ async def launch_remote_container( assert img_tag multi_client_args = [node_rank, master_addr] multi_client_arg_has_value = [x is not None for x in multi_client_args] - if any(multi_client_arg_has_value): - assert all(multi_client_arg_has_value) - is_multi_client = True - else: - is_multi_client = False + assert all(multi_client_arg_has_value) pythonpath_args = None if oneflow_wheel_path: pythonpath_args = "" @@ -157,13 +153,10 @@ async def launch_remote_container( f"ssh {remote_host} docker exec {container_name} python3 -m oneflow --doctor" ) if cmd: - if is_multi_client: - multi_client_docker_args = ( - # Use _MASTER_ADDR to avoid name conflict with OneFlow's built-in MASTER_ADDR - f"--env NODE_RANK={node_rank} --env _MASTER_ADDR={master_addr}" - ) - else: - multi_client_docker_args = "" + multi_client_docker_args = ( + # Use _MASTER_ADDR to avoid name conflict with OneFlow's built-in MASTER_ADDR + f"--env NODE_RANK={node_rank} --env _MASTER_ADDR={master_addr}" + ) await spawn_shell( f"ssh {remote_host} docker exec {multi_client_docker_args} {container_name} {cmd}" ) @@ -187,35 +180,6 @@ def handle_call(conn=None, cmd=None, response=None): return msg -def wait_for_env_proto_and_launch_workers( - agent_port=None, agent_authkey=None, remote_hosts=None -): - listener = Listener(("localhost", agent_port), authkey=agent_authkey) - while True: - conn = listener.accept() - remote_docker_proc = {} - for remote_host in remote_hosts: - assert handle_cast(conn=conn, cmd="host"), remote_host - env_proto_txt = handle_cast(conn=conn, cmd="env_proto") - print("[docker agent]", f"[{remote_host}]", env_proto_txt) - f = tempfile.NamedTemporaryFile(mode="wb+", delete=True) - f.write(env_proto_txt.encode()) - f.flush() - subprocess.check_call( - f"rsync -azPq --omit-dir-times --no-perms --no-group {f.name} {remote_host}:{workspace_dir}/env.prototxt", - shell=True, - ) - run_docker_cmd = f"ssh {remote_host} docker exec {container_name}" - run_docker_cmd += f" python3 -m oneflow.compatible.single_client --start_worker --env_proto={workspace_dir}/env.prototxt" - print("[docker agent]", run_docker_cmd) - remote_docker_proc[remote_host] = subprocess.Popen( - run_docker_cmd, shell=True - ) - handle_call(conn=conn, cmd="start_worker", response="ok") - for k, v in remote_docker_proc.items(): - assert v.wait() == 0 - - class DockerAgent: def __init__( self, @@ -301,24 +265,6 @@ def get_docker_cmd(f, cmd): self.bash_tmp_file = f self.bash_proc = subprocess.Popen(run_docker_cmd, shell=True) - def block(self): - from multiprocessing import Process - - p = None - kwargs = { - "agent_port": self.agent_port, - "agent_authkey": self.agent_authkey, - "remote_hosts": self.remote_hosts, - } - p = Process(target=wait_for_env_proto_and_launch_workers, kwargs=kwargs,) - p.start() - print("[docker agent]", "blocking") - while self.bash_proc.poll() is None and p.is_alive() == True: - pass - p.terminate() - assert self.bash_proc.returncode == 0 - print("[docker agent]", "bash execution done") - def __exit__(self, exc_type, exc_val, exc_tb): pass @@ -450,7 +396,7 @@ def get_remote_hosts(args): parser.add_argument("--copy_files", action="append", default=[]) args = parser.parse_args() - assert args.mode in ["multi_client", "single_client"] + assert args.mode in ["multi_client"] assert bool(args.oneflow_wheel_path) != bool(args.oneflow_python_path) assert bool(args.bash_script) != bool(args.cmd) if args.skip_libs: @@ -701,22 +647,3 @@ def exit_handler(): ) ) - if args.mode == "single_client": - with DockerAgent( - port=agent_port, - authkey=agent_authkey.encode(), - this_host=this_host, - remote_hosts=remote_hosts, - container_name=container_name, - workspace_dir=workspace_dir, - oneflow_wheel_path=oneflow_wheel_path, - oneflow_python_path=args.oneflow_python_path, - img_tag=img_tag, - oneflow_test_tmp_dir=args.oneflow_test_tmp_dir, - extra_docker_args=" ".join(main_node_extra_docker_args), - ) as agent: - if args.bash_script: - agent.run_bash_script_async(bash_script=args.bash_script,) - elif args.cmd: - agent.run_bash_script_async(cmd=args.cmd,) - agent.block() diff --git a/docs/source/env.rst b/docs/source/env.rst index 3425a329a20..fdf298b8578 100644 --- a/docs/source/env.rst +++ b/docs/source/env.rst @@ -8,4 +8,3 @@ Environment .. autofunction:: oneflow.env.get_rank .. autofunction:: oneflow.env.get_local_rank .. autofunction:: oneflow.env.get_node_size -.. autofunction:: oneflow.env.is_multi_client diff --git a/oneflow/api/cpp/env.cpp b/oneflow/api/cpp/env.cpp index ea31af15a69..6573eb06e36 100644 --- a/oneflow/api/cpp/env.cpp +++ b/oneflow/api/cpp/env.cpp @@ -26,7 +26,6 @@ limitations under the License. #include "oneflow/api/cpp/env.h" #include "oneflow/core/common/global.h" #include "oneflow/core/common/just.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/optional.h" #include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/framework/shut_down_util.h" @@ -129,7 +128,6 @@ of::Maybe initEnv() { } // namespace void initialize() { - of::SetIsMultiClient(true).GetOrThrow(); if (!IsEnvInited()) { initEnv().GetOrThrow(); } of::SetShuttingDown(false); } @@ -141,11 +139,7 @@ void release() { of::Global::Get()->TryClose().GetOrThrow(); of::Global::Delete(); // destory env - if (of::IsMultiClient().GetOrThrow()) { - OF_ENV_BARRIER(); - } else { - of::ClusterInstruction::MasterSendHalt(); - } + OF_ENV_BARRIER(); of::Global::Delete(); } of::SetShuttingDown(); diff --git a/oneflow/api/python/eager/single_client.cpp b/oneflow/api/python/eager/single_client.cpp deleted file mode 100644 index da34eaa18d3..00000000000 --- a/oneflow/api/python/eager/single_client.cpp +++ /dev/null @@ -1,25 +0,0 @@ -/* -Copyright 2020 The OneFlow Authors. All rights reserved. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ -#include -#include "oneflow/api/python/of_api_registry.h" -#include "oneflow/core/vm/vm_util.h" - -ONEFLOW_API_PYBIND11_MODULE("eager.single_client", m) { - using namespace oneflow; - namespace py = pybind11; - m.def( - "Sync", []() { vm::ClusterSync().GetOrThrow(); }, py::call_guard()); -} diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 9a0cadd541e..d352751ad3c 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -34,8 +34,6 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("GetWorldSize", &GetWorldSize); m.def("GetNodeSize", &GetNodeSize); m.def("GetLocalRank", &GetLocalRank); - m.def("IsMultiClient", &IsMultiClient); - m.def("SetIsMultiClient", &SetIsMultiClient); m.def("CudaGetDeviceCount", &CudaGetDeviceCount); m.def("SetFLAGS_alsologtostderr", &SetFLAGS_alsologtostderr); m.def("GetFLAGS_alsologtostderr", &GetFLAGS_alsologtostderr); diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index d78a8e7c45c..d31329d2c01 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -18,7 +18,6 @@ limitations under the License. #include #include -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/job/cluster.h" #include "oneflow/core/job/cluster_instruction.h" @@ -50,16 +49,12 @@ inline Maybe IsEnvInited() { return Global::Get() ! inline Maybe DestroyEnv() { if (Global::Get() == nullptr) { return Maybe::Ok(); } - if (JUST(IsMultiClient())) { - OF_ENV_BARRIER(); - } else { - if (GlobalProcessCtx::IsThisProcessMaster()) { ClusterInstruction::MasterSendHalt(); } - } + OF_ENV_BARRIER(); Global::Delete(); return Maybe::Ok(); } -inline Maybe InitEnv(const std::string& env_proto_str, bool is_multi_client) { +inline Maybe InitEnv(const std::string& env_proto_str) { EnvProto env_proto; CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) << "failed to parse env_proto" << env_proto_str; @@ -68,7 +63,6 @@ inline Maybe InitEnv(const std::string& env_proto_str, bool is_multi_clien // because glog is not constructed yet and LOG(INFO) has bad bahavior Global::SetAllocated(new EnvGlobalObjectsScope()); JUST(Global::Get()->Init(env_proto)); - if (!GlobalProcessCtx::IsThisProcessMaster() && !is_multi_client) { JUST(Cluster::WorkerLoop()); } return Maybe::Ok(); } diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 9611d24a19a..13702b74bbd 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -28,8 +28,8 @@ inline void EnableEagerEnvironment(bool enable_eager_execution) { inline bool IsEnvInited() { return oneflow::IsEnvInited().GetOrThrow(); } -inline void InitEnv(const std::string& env_proto_str, bool is_multi_client) { - return oneflow::InitEnv(env_proto_str, is_multi_client).GetOrThrow(); +inline void InitEnv(const std::string& env_proto_str) { + return oneflow::InitEnv(env_proto_str).GetOrThrow(); } inline void DestroyEnv() { return oneflow::DestroyEnv().GetOrThrow(); } @@ -44,12 +44,6 @@ inline size_t GetNodeSize() { return oneflow::GetNodeSize().GetOrThrow(); } inline size_t GetLocalRank() { return oneflow::GetLocalRank().GetOrThrow(); } -inline bool IsMultiClient() { return oneflow::IsMultiClient().GetOrThrow(); } - -inline void SetIsMultiClient(bool is_multi_client) { - return oneflow::SetIsMultiClient(is_multi_client).GetOrThrow(); -} - inline size_t CudaGetDeviceCount() { return oneflow::CudaGetDeviceCount().GetOrThrow(); } inline void SetFLAGS_alsologtostderr(bool flag) { diff --git a/oneflow/core/common/multi_client.h b/oneflow/core/common/multi_client.h deleted file mode 100644 index 8912b396e17..00000000000 --- a/oneflow/core/common/multi_client.h +++ /dev/null @@ -1,36 +0,0 @@ -/* -Copyright 2020 The OneFlow 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. -*/ -#ifndef ONEFLOW_CORE_COMMON_MULTICLIENT_H_ -#define ONEFLOW_CORE_COMMON_MULTICLIENT_H_ - -#include "oneflow/core/common/maybe.h" -#include "oneflow/core/common/optional.h" -#include "oneflow/core/job/global_for.h" - -namespace oneflow { - -inline Optional* IsMultiClientPtr() { return Global, MultiClient>::Get(); } - -inline Maybe IsMultiClient() { return JUST(*Global, MultiClient>::Get()); } - -inline Maybe SetIsMultiClient(bool is_multi_client) { - CHECK_NOTNULL_OR_RETURN(IsMultiClientPtr()); - *IsMultiClientPtr() = is_multi_client; - return Maybe::Ok(); -} -} // namespace oneflow - -#endif diff --git a/oneflow/core/device/cuda_util.cpp b/oneflow/core/device/cuda_util.cpp index 81a1d44f11e..908fb02bb9a 100644 --- a/oneflow/core/device/cuda_util.cpp +++ b/oneflow/core/device/cuda_util.cpp @@ -16,7 +16,6 @@ limitations under the License. #include #include "oneflow/core/device/cuda_util.h" #include "oneflow/core/common/global.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/hardware/node_device_descriptor_manager.h" #include "oneflow/core/hardware/cuda_device_descriptor.h" #include "oneflow/core/rpc/include/global_process_ctx.h" @@ -162,13 +161,7 @@ void CublasMathModeGuard::SetMathMode(cublasMath_t new_mode) { } int GetCudaDeviceIndex() { - int cuda_device_index = 0; - if (CHECK_JUST(IsMultiClient())) { - cuda_device_index = GlobalProcessCtx::LocalRank(); - } else { - OF_CUDA_CHECK(cudaGetDevice(&cuda_device_index)); - } - return cuda_device_index; + return GlobalProcessCtx::LocalRank(); } int GetCudaDeviceCount() { diff --git a/oneflow/core/framework/instructions_builder.cpp b/oneflow/core/framework/instructions_builder.cpp index f84239d047e..512e144c347 100644 --- a/oneflow/core/framework/instructions_builder.cpp +++ b/oneflow/core/framework/instructions_builder.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/framework/instructions_builder.h" #include "oneflow/core/framework/symbol_storage_util.h" #include "oneflow/core/device/event_record.h" diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 5ef471d0da3..9c66b37c8cf 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -16,7 +16,6 @@ limitations under the License. #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/common/maybe.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/framework/load_library.h" #include "oneflow/core/job/resource.pb.h" @@ -61,7 +60,6 @@ int32_t GetCpuDeviceNum() { return std::thread::hardware_concurrency(); } Maybe MultiClientSessionContext::TryInit(const ConfigProto& config_proto) { if (!is_inited_) { - CHECK_OR_RETURN(JUST(IsMultiClient())); DumpVersionInfo(); Resource resource = config_proto.resource(); diff --git a/oneflow/core/framework/sbp_infer_util.cpp b/oneflow/core/framework/sbp_infer_util.cpp index 14cd4a92e8d..46943bcf466 100644 --- a/oneflow/core/framework/sbp_infer_util.cpp +++ b/oneflow/core/framework/sbp_infer_util.cpp @@ -18,7 +18,6 @@ limitations under the License. #include "oneflow/core/auto_parallel/boxing_collector.h" #include "oneflow/core/graph/boxing/hierarchical_sub_task_graph_builder_impl.h" #include "oneflow/core/boxing/eager_boxing_interpreter_mgr.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/util.h" #include "oneflow/core/job/lazy_mode.h" #include "oneflow/core/job/parallel_desc.h" @@ -54,7 +53,7 @@ Maybe ComputCopyCostBetweenTwoSbpParallel(const SbpParallel& producer_sb } // Not supporting S->P for lazy boxing now. - if (LazyMode::is_enabled() || /*single_client=*/(!JUST(IsMultiClient()))) { + if (LazyMode::is_enabled()) { if (consumer_sbp_parallel.has_partial_sum_parallel() && producer_sbp_parallel.has_split_parallel()) { return kUnsupportedBoxing; @@ -303,7 +302,7 @@ Maybe ComputeEagerCopyCostBetweenNdSbp(const NdSbp& producer_sbp_paralle using CopyCostFunc = Maybe(const NdSbp&, const NdSbp&, const BlobDesc&, const ParallelDesc&, const ParallelDesc&, bool); Maybe GetComputeCopyCostFunc() { - if (LazyMode::is_enabled() || /*single_client=*/(!JUST(IsMultiClient()))) { + if (LazyMode::is_enabled()) { return &ComputeCopyCostWithMiddleNodes; } else { return &ComputeEagerCopyCostBetweenNdSbp; diff --git a/oneflow/core/functional/impl/random_functor.cpp b/oneflow/core/functional/impl/random_functor.cpp index cb12529d593..1ebd62988c6 100644 --- a/oneflow/core/functional/impl/random_functor.cpp +++ b/oneflow/core/functional/impl/random_functor.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "oneflow/core/common/global.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/optional.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/framework/attr_map.h" diff --git a/oneflow/core/graph/task_graph.cpp b/oneflow/core/graph/task_graph.cpp index f4d13f0ddbe..e21d5f9cd4c 100644 --- a/oneflow/core/graph/task_graph.cpp +++ b/oneflow/core/graph/task_graph.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "oneflow/core/graph/task_graph.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/util.h" #include "oneflow/core/graph/inplace_lbi_graph.h" #include "oneflow/core/register/blob_desc.h" diff --git a/oneflow/core/job/job_build_and_infer_ctx.cpp b/oneflow/core/job/job_build_and_infer_ctx.cpp index 64701212f84..02695544def 100644 --- a/oneflow/core/job/job_build_and_infer_ctx.cpp +++ b/oneflow/core/job/job_build_and_infer_ctx.cpp @@ -1020,10 +1020,8 @@ Maybe LazyJobBuildAndInferCtx::Complete() { if (GlobalJobDesc().Bool("__is_user_function__")) { JUST(DoPass("ModelUpdateConfCompatiblePass")); - JUST(DoPass("SetDefaultVariableConf")); JUST(DoPass("AddInputOutputOpsPass")); JUST(DoPass("NormalizationExponentialAverageAutoTickPass")); - JUST(DoPass("GradientAccumulationRewritePass")); #ifdef WITH_CUDA JUST(DoPass("AutoMixedPrecision")); JUST(DoPass("PruneAmpWhiteIdentityOpPass")); diff --git a/oneflow/core/job/job_build_and_infer_ctx_mgr.cpp b/oneflow/core/job/job_build_and_infer_ctx_mgr.cpp index 26abf3402a3..486f6fc31c8 100644 --- a/oneflow/core/job/job_build_and_infer_ctx_mgr.cpp +++ b/oneflow/core/job/job_build_and_infer_ctx_mgr.cpp @@ -15,7 +15,6 @@ limitations under the License. */ #include "oneflow/core/job/job_build_and_infer_ctx_mgr.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/util.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/job/lazy_mode.h" @@ -106,16 +105,7 @@ Maybe EagerJobBuildAndInferCtxMgr::VirtualCloseJob() { bool EagerExecutionEnabled() { return *Global::Get(); } Maybe GlobalJobBuildAndInferCtxMgr() { - if (JUST(IsMultiClient())) { - return JUST(GlobalMaybe()); - } else { - // single-client - if (EagerExecutionEnabled()) { - return JUST(GlobalMaybe()); - } else { - return JUST(GlobalMaybe()); - } - } + return JUST(GlobalMaybe()); } Maybe GetJobBuildAndInferCtx(const std::string& job_name) { diff --git a/oneflow/core/job/parallel_desc.cpp b/oneflow/core/job/parallel_desc.cpp index 67d1b24de3b..19e05ba252a 100644 --- a/oneflow/core/job/parallel_desc.cpp +++ b/oneflow/core/job/parallel_desc.cpp @@ -18,7 +18,6 @@ limitations under the License. #include "oneflow/core/job/placement.cfg.h" #include "oneflow/core/common/decorator.h" #include "oneflow/core/common/util.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/cpp_attribute.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/job/id_manager.h" @@ -319,34 +318,33 @@ Maybe ParallelDesc::CheckWithResourceDesc(const ResourceDesc& resource_des } Maybe ParallelDesc::CheckDeviceIdsIsValid() const { - if (likely(JUST(IsMultiClient()))) { - const auto& sorted_dev_phy_ids_iter = - machine_id2sorted_dev_phy_ids_->find(GlobalProcessCtx::Rank()); - for (int64_t machine_id : sorted_machine_ids_) { - CHECK_LT_OR_RETURN(machine_id, GlobalProcessCtx::WorldSize()) - << "Placment is invalid because rank must be less than world size!"; - } - if (sorted_dev_phy_ids_iter != machine_id2sorted_dev_phy_ids_->end()) { - for (int64_t dev_phy_id : *sorted_dev_phy_ids_iter->second) { - if (device_type_ == DeviceType::kCUDA) { - const int64_t gpu_device_num = GetGpuDeviceNum(); - CHECK_NE_OR_RETURN(gpu_device_num, 0) - << "Placment with \"cuda\" type is invalid because there is no CUDA device!"; - int64_t device_num = std::min(GlobalProcessCtx::NumOfProcessPerNode(), gpu_device_num); - CHECK_LT_OR_RETURN(dev_phy_id, device_num) - << "Placment is invalid because device id must be less than " - << (gpu_device_num < GlobalProcessCtx::NumOfProcessPerNode() - ? "num of CUDA devices on node" - : "num of process per node"); - } else if (device_type_ == DeviceType::kCPU) { - CHECK_LT_OR_RETURN(dev_phy_id, GlobalProcessCtx::NumOfProcessPerNode()) - << "Placment is invalid because device id must be less than num of process per node"; - } else { - OF_UNIMPLEMENTED(); - } + const auto& sorted_dev_phy_ids_iter = + machine_id2sorted_dev_phy_ids_->find(GlobalProcessCtx::Rank()); + for (int64_t machine_id : sorted_machine_ids_) { + CHECK_LT_OR_RETURN(machine_id, GlobalProcessCtx::WorldSize()) + << "Placment is invalid because rank must be less than world size!"; + } + if (sorted_dev_phy_ids_iter != machine_id2sorted_dev_phy_ids_->end()) { + for (int64_t dev_phy_id : *sorted_dev_phy_ids_iter->second) { + if (device_type_ == DeviceType::kCUDA) { + const int64_t gpu_device_num = GetGpuDeviceNum(); + CHECK_NE_OR_RETURN(gpu_device_num, 0) + << "Placment with \"cuda\" type is invalid because there is no CUDA device!"; + int64_t device_num = std::min(GlobalProcessCtx::NumOfProcessPerNode(), gpu_device_num); + CHECK_LT_OR_RETURN(dev_phy_id, device_num) + << "Placment is invalid because device id must be less than " + << (gpu_device_num < GlobalProcessCtx::NumOfProcessPerNode() + ? "num of CUDA devices on node" + : "num of process per node"); + } else if (device_type_ == DeviceType::kCPU) { + CHECK_LT_OR_RETURN(dev_phy_id, GlobalProcessCtx::NumOfProcessPerNode()) + << "Placment is invalid because device id must be less than num of process per node"; + } else { + OF_UNIMPLEMENTED(); } } } + return Maybe::Ok(); } diff --git a/oneflow/core/job/plan_util.cpp b/oneflow/core/job/plan_util.cpp index 20e7f5f28a4..7b36d20897c 100644 --- a/oneflow/core/job/plan_util.cpp +++ b/oneflow/core/job/plan_util.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "oneflow/core/common/constant.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/str_util.h" #include "oneflow/core/job/plan_util.h" #include "oneflow/core/job/global_for.h" @@ -70,44 +69,6 @@ void PlanUtil::GenMemBlockAndChunk4Plan(Plan* plan) { namespace { -void GenChunkInSingleClient( - Plan* plan, HashMap>* mem_block_id2mem_block) { - // mzuid = memory zone unique id - HashMap mzuid2chunk; - auto GenChunk4ReusedMemBlockIfNeed = [&](MemBlockProto* mem_block) { - int64_t mzuid = - MemoryCaseUtil::GenMemZoneUniqueId(mem_block->machine_id(), mem_block->mem_case()); - if (mzuid2chunk.find(mzuid) == mzuid2chunk.end()) { - ChunkProto chunk; - chunk.set_chunk_id(Global::Get()->NewChunkId()); - chunk.add_job_id(mem_block->job_id(0)); - chunk.set_machine_id(mem_block->machine_id()); - *(chunk.mutable_mem_case()) = mem_block->mem_case(); - chunk.set_mem_size(mem_block->mem_size()); - CHECK(mzuid2chunk.emplace(mzuid, chunk).second); - mem_block->set_chunk_id(chunk.chunk_id()); - mem_block->set_chunk_offset(0); - } else { - ChunkProto* chunk = &(mzuid2chunk.at(mzuid)); - CHECK_EQ(chunk->job_id(0), mem_block->job_id(0)); - mem_block->set_chunk_id(chunk->chunk_id()); - mem_block->set_chunk_offset(chunk->mem_size()); - chunk->set_mem_size(chunk->mem_size() + mem_block->mem_size()); - } - }; - - for (auto& pair : *mem_block_id2mem_block) { - MemBlockProto* mem_block = pair.second.get(); - CHECK(mem_block->has_chunk_id() == false); - CHECK(mem_block->has_chunk_offset() == false); - if (mem_block->enable_reuse_mem()) { GenChunk4ReusedMemBlockIfNeed(mem_block); } - } - - for (const auto& pair : mzuid2chunk) { - *(plan->mutable_block_chunk_list()->add_chunk()) = pair.second; - } -} - void GenChunkForMultiNNGraphMemoryReuseInMultiClient( Plan* plan, HashMap>* mem_block_id2mem_block) { HashMap> mzuid2mem_blocks; @@ -325,12 +286,7 @@ void PlanUtil::GenMemBlockAndChunkWithVariableOpNames4Plan( } } - if (CHECK_JUST(IsMultiClient())) { - GenChunkForMultiNNGraphMemoryReuseInMultiClient(plan, &mem_block_id2mem_block); - } else { - CHECK(variable_op_names.empty()); - GenChunkInSingleClient(plan, &mem_block_id2mem_block); - } + GenChunkForMultiNNGraphMemoryReuseInMultiClient(plan, &mem_block_id2mem_block); for (const auto& pair : mem_block_id2mem_block) { *(plan->mutable_block_chunk_list()->add_mem_block()) = *(pair.second); diff --git a/oneflow/core/job_rewriter/autotick.cpp b/oneflow/core/job_rewriter/autotick.cpp index 60809b085f2..15aba01f66f 100644 --- a/oneflow/core/job_rewriter/autotick.cpp +++ b/oneflow/core/job_rewriter/autotick.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "oneflow/core/common/container_util.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/job_rewriter/autotick.h" #include "oneflow/core/job/job_builder.h" @@ -23,7 +22,6 @@ limitations under the License. #include "oneflow/core/common/container_util.h" #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/job/global_for.h" -#include "oneflow/core/common/multi_client.h" namespace oneflow { @@ -571,27 +569,7 @@ Maybe AutoSourceAndSinkTick( return Maybe::Ok(); } -Maybe SingleClientAutoSourceAndSinkTick(const OpGraph& op_graph, JobBuilder* job_builder) { - if (JUST(IsMultiClient())) { return Maybe::Ok(); } - auto* critical_section = - Global::Get()->AddCriticalSection(GlobalJobDesc().job_id()); - critical_section->mutable_total_job_critical_section(); - auto* src_map = critical_section->mutable_machine_id2source_tick_op_name(); - const auto& DoEachSrc = [&](int64_t machine_id, const std::string& op_name) -> Maybe { - (*src_map)[machine_id] = op_name; - return Maybe::Ok(); - }; - auto* sink_map = critical_section->mutable_machine_id2sink_tick_op_name(); - const auto& DoEachSink = [&](int64_t machine_id, const std::string& op_name) -> Maybe { - (*sink_map)[machine_id] = op_name; - return Maybe::Ok(); - }; - JUST(AutoSourceAndSinkTick(op_graph, job_builder, DoEachSrc, DoEachSink)); - return Maybe::Ok(); -} - Maybe MultiClientAutoSourceAndSinkTick(const OpGraph& op_graph, Job* job) { - if (!JUST(IsMultiClient())) { return Maybe::Ok(); } HashMap machine_id2src_op_name; HashMap machine_id2sink_op_name; { @@ -781,7 +759,6 @@ Maybe ConnectCriticalSectionCallbackToJobSoleDstSubsetTick( } // namespace Maybe MultiClientAutoInterfaceCriticalSectionTick(const OpGraph& op_graph, Job* job) { - if (!JUST(IsMultiClient())) { return Maybe::Ok(); } JobBuilder job_builder(job); std::vector> critical_section_callback_lbis; { @@ -811,30 +788,4 @@ Maybe MultiClientAutoInterfaceCriticalSectionTick(const OpGraph& op_graph, return Maybe::Ok(); } -Maybe SingleClientAddGlobalInputCriticalSections(const OpGraph& op_graph, - JobBuilder* job_builder) { - if (JUST(IsMultiClient())) { return Maybe::Ok(); } - JUST(ForEachInputCriticalSectionOpNodes( - op_graph, - [&](const HashSet& op_nodes, - const std::vector& lbi_producer_op_names) -> Maybe { - JUST(AddGlobalInputOutputCriticalSection(op_nodes, lbi_producer_op_names, job_builder)); - return Maybe::Ok(); - })); - return Maybe::Ok(); -} - -Maybe SingleClientAddGlobalOutputCriticalSections(const OpGraph& op_graph, - JobBuilder* job_builder) { - if (JUST(IsMultiClient())) { return Maybe::Ok(); } - JUST(ForEachOutputCriticalSectionOpNodes( - op_graph, - [&](const HashSet& op_nodes, - const std::vector& lbi_producer_op_names) -> Maybe { - JUST(AddGlobalInputOutputCriticalSection(op_nodes, lbi_producer_op_names, job_builder)); - return Maybe::Ok(); - })); - return Maybe::Ok(); -} - } // namespace oneflow diff --git a/oneflow/core/job_rewriter/autotick.h b/oneflow/core/job_rewriter/autotick.h index 53a81b90408..6926cfe0ed6 100644 --- a/oneflow/core/job_rewriter/autotick.h +++ b/oneflow/core/job_rewriter/autotick.h @@ -24,11 +24,6 @@ namespace oneflow { Maybe AutoPrependTick(const OpGraph& op_graph, JobBuilder* job_builder); Maybe AddTickForTimeShape(const OpGraph& op_graph, JobBuilder* job_builder); -Maybe SingleClientAutoSourceAndSinkTick(const OpGraph& op_graph, JobBuilder* job_builder); -Maybe SingleClientAddGlobalInputCriticalSections(const OpGraph& op_graph, - JobBuilder* job_builder); -Maybe SingleClientAddGlobalOutputCriticalSections(const OpGraph& op_graph, - JobBuilder* job_builder); Maybe MultiClientAutoSourceAndSinkTick(const OpGraph& op_graph, Job* job); Maybe MultiClientAutoInterfaceCriticalSectionTick(const OpGraph& op_graph, Job* job); diff --git a/oneflow/core/job_rewriter/gradient_accumulation_rewrite_pass.cpp b/oneflow/core/job_rewriter/gradient_accumulation_rewrite_pass.cpp deleted file mode 100644 index 72f0cb61709..00000000000 --- a/oneflow/core/job_rewriter/gradient_accumulation_rewrite_pass.cpp +++ /dev/null @@ -1,197 +0,0 @@ -/* -Copyright 2020 The OneFlow 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 "oneflow/core/common/multi_client.h" -#include "oneflow/core/job_rewriter/job_pass.h" -#include "oneflow/core/framework/framework.h" - -namespace oneflow { - -class GradientAccumulationRewritePass final : public JobPass { - public: - GradientAccumulationRewritePass() = default; - ~GradientAccumulationRewritePass() override = default; - - Maybe Apply(Job* job, JobPassCtx* ctx) const override; -}; - -Maybe GradientAccumulationRewritePass::Apply(Job* job, JobPassCtx* ctx) const { - const JobConfigProto& job_conf = ctx->job_desc().job_conf(); - if (!job_conf.has_train_conf()) { return Maybe::Ok(); } - if ((!job_conf.has_num_gradient_accumulation_steps()) - || job_conf.num_gradient_accumulation_steps() <= 1) { - return Maybe::Ok(); - } - const bool is_multi_client = CHECK_JUST(IsMultiClient()); - if (is_multi_client) { - // GradientAccumulationRewritePass has been re-implemented in op interpreter in multi client. - return Maybe::Ok(); - } - const OpGraph op_graph(*job); - JobBuilder job_builder(job); - HashMap name2op_conf; - auto GetOperatorConf4Modify = [&name2op_conf](const OperatorConf& op_conf) { - const auto& it = name2op_conf.find(op_conf.name()); - if (it != name2op_conf.end()) { - return &it->second; - } else { - name2op_conf[op_conf.name()] = op_conf; - return &name2op_conf.at(op_conf.name()); - } - }; - const int64_t repeat_num = GlobalJobDesc().job_conf().num_gradient_accumulation_steps(); - JUST(op_graph.TopoForEachNodeWithErrorCaptured([&](const OpNode* node) -> Maybe { - const OperatorConf& op_conf = node->op().op_conf(); - if (node->in_edges().empty()) { // sources - if (op_conf.has_input_conf()) { // input - // NOTE(chengcheng): - // We assume that the input data is one mini-batch which containing multi micro-batches. - // So we need unpack input data for each micro-batch. - const LogicalBlobId input_lbi = node->op().BnInOp2Lbi("out"); - const std::string input_lbn = GenLogicalBlobName(input_lbi); - - user_op::UserOpConfWrapperBuilder unpack_builder("System-GradientAccumulation-InputUnpack-" - + op_conf.name() + "-" + NewUniqueId()); - const auto unpack_op = unpack_builder.OpTypeName("unpack") - .Input("in", input_lbn) - .Output("out") - .Attr("unpack_num", repeat_num) - .ScopeSymbolId(op_conf.scope_symbol_id()) - .Build(); - job_builder.AddOps(node->parallel_desc().parallel_conf(), {unpack_op.op_conf()}); - const std::string unpack_lbn = unpack_op.output("out", 0); - node->ForEachNodeOnOutEdge([&](const OpNode* dst) { - const auto& dst_op = dst->op(); - OperatorConf* new_dst_op_conf = GetOperatorConf4Modify(dst_op.op_conf()); - for (const auto& ibn : dst_op.input_bns()) { - if (dst_op.BnInOp2Lbi(ibn) == input_lbi) { - const auto& old_val = - ReplaceInputLbnInOpCustomizedConf(new_dst_op_conf, ibn, unpack_lbn); - CHECK_EQ(input_lbn, old_val); - } - } - }); - - return Maybe::Ok(); - } else if (op_conf.has_variable_conf()) { // repeat variable - const LogicalBlobId variable_lbi = node->op().BnInOp2Lbi("out"); - const std::string variable_lbn = GenLogicalBlobName(variable_lbi); - HashMap parallel_conf2repeat_lbn; - node->ForEachNodeOnOutEdge([&](const OpNode* dst) { - const auto& dst_op = dst->op(); - const ParallelConf& parallel_conf = dst->parallel_desc().parallel_conf(); - std::string repeat_lbn; - const auto& it = parallel_conf2repeat_lbn.find(parallel_conf); - if (it == parallel_conf2repeat_lbn.end()) { - user_op::UserOpConfWrapperBuilder repeat_builder( - "System-GradientAccumulation-Repeat-" + op_conf.name() + "-" + NewUniqueId()); - const auto repeat_op = repeat_builder.OpTypeName("repeat") - .Input("in", variable_lbn) - .Output("out") - .Attr("repeat_num", repeat_num) - .ScopeSymbolId(dst_op.op_conf().scope_symbol_id()) - .Build(); - job_builder.AddOps(parallel_conf, {repeat_op.op_conf()}); - repeat_lbn = repeat_op.output("out", 0); - parallel_conf2repeat_lbn.emplace(parallel_conf, repeat_lbn); - } else { - repeat_lbn = it->second; - } - OperatorConf* new_dst_op_conf = GetOperatorConf4Modify(dst_op.op_conf()); - for (const auto& ibn : dst_op.input_bns()) { - if (dst_op.BnInOp2Lbi(ibn) == variable_lbi) { - const auto& old_val = - ReplaceInputLbnInOpCustomizedConf(new_dst_op_conf, ibn, repeat_lbn); - CHECK_EQ(variable_lbn, old_val); - } - } - }); - return Maybe::Ok(); - } else if (op_conf.has_user_conf()) { // repeat tick - OperatorConf* new_op_conf = GetOperatorConf4Modify(op_conf); - OperatorConf tick_conf{}; - tick_conf.set_name("System-GradientAccumulation-RepeatTick-DeviceTick-" + op_conf.name()); - tick_conf.mutable_device_tick_conf()->set_out("out"); - tick_conf.set_scope_symbol_id(op_conf.scope_symbol_id()); - auto tick_lbn = GenLogicalBlobName(tick_conf.name(), tick_conf.device_tick_conf().out()); - user_op::UserOpConfWrapperBuilder repeat_builder( - "System-GradientAccumulation-RepeatTick-Repeat-" + op_conf.name()); - const auto repeat_op = repeat_builder.OpTypeName("repeat") - .Input("in", tick_lbn) - .Output("out") - .Attr("repeat_num", repeat_num) - .ScopeSymbolId(op_conf.scope_symbol_id()) - .Build(); - job_builder.AddOps(node->parallel_desc().parallel_conf(), {tick_conf, repeat_op.op_conf()}); - (*new_op_conf->mutable_user_conf()->mutable_input())[user_op::kUserSourceOpTickInputArgName] - .add_s(repeat_op.output("out", 0)); - return Maybe::Ok(); - } else { - LOG(ERROR) << "Gradient accumulation unsupported op : " << op_conf.DebugString(); - return Error::UnimplementedError(); - } - } else if ((is_multi_client && op_conf.has_output_conf()) - || (!is_multi_client && op_conf.has_return_conf())) { - // NOTE(chengcheng): - // in Single-Client GlobalFunction return op is output - // in Multi-Client nn.Graph output op is output. - const LogicalBlobId return_in_lbi = node->op().BnInOp2Lbi("in"); - const std::string return_in_lbn = GenLogicalBlobName(return_in_lbi); - user_op::UserOpConfWrapperBuilder pack_builder("System-GradientAccumulation-ReturnPack-" - + op_conf.name()); - const auto return_pack_op = pack_builder.OpTypeName("pack") - .Input("in", return_in_lbn) - .Output("out") - .Attr("pack_num", repeat_num) - .ScopeSymbolId(op_conf.scope_symbol_id()) - .Build(); - job_builder.AddOps(node->parallel_desc().parallel_conf(), {return_pack_op.op_conf()}); - OperatorConf* new_return_op_conf = GetOperatorConf4Modify(op_conf); - const auto& old_val = ReplaceInputLbnInOpCustomizedConf(new_return_op_conf, "in", - return_pack_op.output("out", 0)); - CHECK_EQ(return_in_lbn, old_val); - return Maybe::Ok(); - } else if (is_multi_client && op_conf.has_user_conf() - && op_conf.user_conf().op_type_name() == "reshape") { - const LogicalBlobId in_lbi = node->op().BnInOp2Lbi(node->op().SoleIbn()); - const LogicalBlobId out_lbi = node->op().BnInOp2Lbi(node->op().SoleObn()); - const Shape& in_shape = node->LogicalBlobDesc4Lbi(in_lbi).shape(); - const Shape& out_shape = node->LogicalBlobDesc4Lbi(out_lbi).shape(); - if (in_shape.NumAxes() > 0 && out_shape.NumAxes() > 0 && in_shape.At(0) == out_shape.At(0)) { - // NOTE(chengcheng): - // in nn.Graph GradientAccumulation, the reshape conf in JobBuild and after insert - // acc/unpack maybe NOT equal because of dim 0 scaled, so need set dim 0 as -1 for - // dynamic infer. - OperatorConf* new_reshape_op_conf = GetOperatorConf4Modify(op_conf); - AttrValue* attr_val = &(*new_reshape_op_conf->mutable_user_conf()->mutable_attr())["shape"]; - CHECK(attr_val->has_at_shape()); - ShapeProto* shape_conf = attr_val->mutable_at_shape(); - CHECK_GT(shape_conf->dim_size(), 0); - shape_conf->set_dim(0, -1); - LOG(INFO) << " Replace ReshapeOpConf from: " << op_conf.DebugString() << " to " - << new_reshape_op_conf->DebugString() << " for dynamic infer by insert unpack."; - } - return Maybe::Ok(); - } else { - return Maybe::Ok(); - } - })); - for (const auto& pair : name2op_conf) { job_builder.MutOpsOnlyOnce({pair.second}); } - return Maybe::Ok(); -} - -REGISTER_JOB_PASS("GradientAccumulationRewritePass", GradientAccumulationRewritePass); - -} // namespace oneflow diff --git a/oneflow/core/job_rewriter/job_completer.cpp b/oneflow/core/job_rewriter/job_completer.cpp index 0238cf382de..3adda093383 100644 --- a/oneflow/core/job_rewriter/job_completer.cpp +++ b/oneflow/core/job_rewriter/job_completer.cpp @@ -13,7 +13,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job_rewriter/job_completer.h" #include "oneflow/core/job_rewriter/job_pass.h" #include "oneflow/core/job_rewriter/autograd.h" @@ -34,28 +33,14 @@ Maybe CheckOpGraph(const OpGraph& op_graph) { size_t in_cnt = 0; op_graph.ForEachDataAndCtrlInNode(op_node, [&](OpNode*) { ++in_cnt; }); if (in_cnt == 0) { - // NOTE(chengcheng): - // in single-client source op is SourceTickOpConf, - // in multi-client source op is WaitAndSendIdsOpConf_ - if (JUST(IsMultiClient())) { - CHECK_OR_RETURN(op_node->op().op_conf().has_wait_and_send_ids_conf()); - } else { - CHECK_OR_RETURN(op_node->op().op_conf().has_source_tick_conf()); - } + CHECK_OR_RETURN(op_node->op().op_conf().has_wait_and_send_ids_conf()); } size_t out_cnt = 0; op_graph.ForEachDataAndCtrlOutNode(op_node, [&](OpNode*) { ++out_cnt; }); if (out_cnt == 0) { - // NOTE(chengcheng): - // in single-client source op is SinkTickOpConf, - // in multi-client source op is CallbackNotifyOpConf. - if (JUST(IsMultiClient())) { - CHECK_OR_RETURN(op_node->op().op_conf().has_callback_notify_conf()); - } else { - CHECK_OR_RETURN(op_node->op().op_conf().has_sink_tick_conf()); - } + CHECK_OR_RETURN(op_node->op().op_conf().has_callback_notify_conf()); } return Maybe::Ok(); })); @@ -135,9 +120,6 @@ Maybe JobCompleter::Complete(Job* job) const { // complete tick ops JUST(WithOpGraphAndMutJobBuilder(job, &AutoPrependTick)); JUST(WithOpGraphAndMutJobBuilder(job, &AddTickForTimeShape)); - JUST(WithOpGraphAndMutJobBuilder(job, &SingleClientAutoSourceAndSinkTick)); - JUST(WithOpGraphAndMutJobBuilder(job, &SingleClientAddGlobalInputCriticalSections)); - JUST(WithOpGraphAndMutJobBuilder(job, &SingleClientAddGlobalOutputCriticalSections)); JUST(WithOpGraphAndMutJob(job, &MultiClientAutoSourceAndSinkTick)); JUST(WithOpGraphAndMutJob(job, &MultiClientAutoInterfaceCriticalSectionTick)); JUST(JobPass4Name("SystemOpFillJobNamePass")(job, &job_pass_ctx)); diff --git a/oneflow/core/job_rewriter/set_default_variable_conf.cpp b/oneflow/core/job_rewriter/set_default_variable_conf.cpp deleted file mode 100644 index d5e2b329f4e..00000000000 --- a/oneflow/core/job_rewriter/set_default_variable_conf.cpp +++ /dev/null @@ -1,85 +0,0 @@ -/* -Copyright 2020 The OneFlow 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 "oneflow/core/common/multi_client.h" -#include "oneflow/core/job_rewriter/job_pass.h" -#include "oneflow/core/job/global_for.h" -#include "oneflow/core/job/job_builder.h" -#include "oneflow/core/job/job_set_compile_ctx.h" - -namespace oneflow { - -namespace { - -class SetDefaultVariableConf final : public JobPass { - public: - Maybe Apply(Job* job, JobPassCtx* ctx) const override { - const OpGraph op_graph(*job); - JobBuilder job_builder(job); - return Apply(op_graph, &job_builder); - } - - Maybe Apply(const OpGraph& op_graph, JobBuilder* job_builder) const { - if (JUST(IsMultiClient())) { - // NOTE(chengcheng): Multi-Client Variable is inited by Eager. - return Maybe::Ok(); - } - op_graph.ForEachNode([&](OpNode* op_node) { - if (op_node->op().op_conf().has_variable_conf()) { - OperatorConf variable_op_conf(op_node->op().op_conf()); - VariableOpConf* variable_conf = variable_op_conf.mutable_variable_conf(); - if (!variable_conf->has_data_type()) { - variable_conf->set_data_type(job_builder->job().job_conf().default_data_type()); - } - if (!variable_conf->has_initializer() && !variable_conf->has_initialize_with_snapshot()) { - if (job_builder->job().job_conf().has_default_initializer_conf()) { - *variable_conf->mutable_initializer() = - job_builder->job().job_conf().default_initializer_conf(); - } else if (job_builder->job().job_conf().has_default_initialize_with_snapshot_path()) { - variable_conf->mutable_initialize_with_snapshot()->set_path( - job_builder->job().job_conf().default_initialize_with_snapshot_path()); - variable_conf->mutable_initialize_with_snapshot()->set_key( - GenLogicalBlobName(op_node->op().BnInOp2Lbi("out"))); - } else { - UNIMPLEMENTED(); - } - } - int64_t random_seed; - auto* var_op_name2random = Global::Get()->GetVarOpName2randomSeed(); - const std::string& var_op_name = variable_op_conf.name(); - if (variable_conf->has_random_seed()) { - random_seed = variable_conf->random_seed(); - } else { - random_seed = NewRandomSeed(); - } - const auto& pair = var_op_name2random->insert({var_op_name, random_seed}); - if (variable_conf->has_random_seed()) { - CHECK_EQ(variable_conf->random_seed(), pair.first->second); - } else { - variable_conf->set_random_seed(pair.first->second); - } - job_builder->AddOrMutOpsOnlyOnce(op_node->parallel_desc().parallel_conf(), - {variable_op_conf}); - } - }); - return Maybe::Ok(); - } -}; - -REGISTER_JOB_PASS("SetDefaultVariableConf", SetDefaultVariableConf); - -} // namespace - -} // namespace oneflow diff --git a/oneflow/core/kernel/callback_notify_kernel.cpp b/oneflow/core/kernel/callback_notify_kernel.cpp index f64bfb3ba62..45ff84be7e4 100644 --- a/oneflow/core/kernel/callback_notify_kernel.cpp +++ b/oneflow/core/kernel/callback_notify_kernel.cpp @@ -15,7 +15,6 @@ limitations under the License. */ #include "oneflow/core/kernel/kernel.h" #include "oneflow/core/common/buffer_manager.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job/job_instance.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/common/buffer_manager.h" @@ -38,13 +37,8 @@ template void CallbackNotifyKernel::ForwardDataContent(KernelContext* ctx) const { auto* buffer_mgr = Global>>::Get(); std::string buffer_name; - if (CHECK_JUST(IsMultiClient())) { - CHECK(this->op_conf().callback_notify_conf().has_job_name()); - buffer_name = GetCallbackNotifierBufferName(this->op_conf().callback_notify_conf().job_name()); - } else { - T job_id = *ctx->BnInOp2Blob("in")->dptr(); - buffer_name = this->op_conf().callback_notify_conf().callback_buffer_name(job_id); - } + CHECK(this->op_conf().callback_notify_conf().has_job_name()); + buffer_name = GetCallbackNotifierBufferName(this->op_conf().callback_notify_conf().job_name()); std::shared_ptr foreign_job_instance; BufferStatus buffer_status = buffer_mgr->Get(buffer_name)->TryReceive(&foreign_job_instance); CHECK_NE(buffer_status, kBufferStatusEmpty); diff --git a/oneflow/core/kernel/critical_section_callback_tick_kernel.cpp b/oneflow/core/kernel/critical_section_callback_tick_kernel.cpp index 27650328777..bd53c99de4c 100644 --- a/oneflow/core/kernel/critical_section_callback_tick_kernel.cpp +++ b/oneflow/core/kernel/critical_section_callback_tick_kernel.cpp @@ -18,7 +18,6 @@ limitations under the License. #include "oneflow/core/job/critical_section_instance.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/common/buffer_manager.h" -#include "oneflow/core/common/multi_client.h" namespace oneflow { @@ -35,8 +34,6 @@ class CriticalSectionCallbackTickKernel final : public Kernel { void CriticalSectionCallbackTickKernel::ForwardDataContent(KernelContext* ctx) const { auto* buffer_mgr = Global>>::Get(); - bool is_multi_client = CHECK_JUST(IsMultiClient()); - CHECK(is_multi_client); CHECK(op_conf().has_critical_section_callback_tick_conf()); const std::string& buffer_name = op_conf().critical_section_callback_tick_conf().buffer_name(); std::shared_ptr foreign_critical_section_instance; diff --git a/oneflow/core/kernel/critical_section_wait_tick_kernel.cpp b/oneflow/core/kernel/critical_section_wait_tick_kernel.cpp index 89198e542fb..64c30b467b6 100644 --- a/oneflow/core/kernel/critical_section_wait_tick_kernel.cpp +++ b/oneflow/core/kernel/critical_section_wait_tick_kernel.cpp @@ -18,7 +18,6 @@ limitations under the License. #include "oneflow/core/job/critical_section_instance.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/common/buffer_manager.h" -#include "oneflow/core/common/multi_client.h" namespace oneflow { @@ -35,8 +34,6 @@ class CriticalSectionWaitTickKernel final : public Kernel { void CriticalSectionWaitTickKernel::ForwardDataContent(KernelContext* ctx) const { auto* buffer_mgr = Global>>::Get(); - bool is_multi_client = CHECK_JUST(IsMultiClient()); - CHECK(is_multi_client); CHECK(this->op_conf().has_critical_section_wait_tick_conf()); const std::string& buffer_name = this->op_conf().critical_section_wait_tick_conf().buffer_name(); std::shared_ptr foreign_critical_section_instance; diff --git a/oneflow/core/kernel/input_kernel.cpp b/oneflow/core/kernel/input_kernel.cpp index 0c948ac50e7..596a2256e11 100644 --- a/oneflow/core/kernel/input_kernel.cpp +++ b/oneflow/core/kernel/input_kernel.cpp @@ -17,7 +17,6 @@ limitations under the License. #include "oneflow/core/kernel/kernel.h" #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/job/critical_section_instance.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job/global_for.h" namespace oneflow { @@ -32,7 +31,6 @@ class InputKernel final : public Kernel { private: void ForwardDataContent(KernelContext* ctx) const override { - if (CHECK_JUST(IsMultiClient())) { CHECK(this->op_conf().input_conf().has_job_name()); const auto& job_name = this->op_conf().input_conf().job_name(); const auto& op_name = this->op_conf().name(); @@ -45,7 +43,6 @@ class InputKernel final : public Kernel { OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("out")); critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); } - } } void ForwardHeader(KernelContext* ctx) const override {} }; diff --git a/oneflow/core/kernel/output_kernel.cpp b/oneflow/core/kernel/output_kernel.cpp index fcd82e6893c..6d426c1613c 100644 --- a/oneflow/core/kernel/output_kernel.cpp +++ b/oneflow/core/kernel/output_kernel.cpp @@ -16,7 +16,6 @@ limitations under the License. #include "oneflow/core/kernel/kernel.h" #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/job/critical_section_instance.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job/global_for.h" namespace oneflow { @@ -33,7 +32,6 @@ class OutputKernel final : public Kernel { }; void OutputKernel::ForwardDataContent(KernelContext* ctx) const { - if (CHECK_JUST(IsMultiClient())) { CHECK(this->op_conf().output_conf().has_job_name()); const auto& job_name = this->op_conf().output_conf().job_name(); const auto& op_name = this->op_conf().name(); @@ -46,17 +44,10 @@ void OutputKernel::ForwardDataContent(KernelContext* ctx) const { OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); } - } else { - AutoMemcpy(ctx->stream(), ctx->BnInOp2Blob("out"), ctx->BnInOp2Blob("in")); - } } void OutputKernel::ForwardHeader(KernelContext* ctx) const { - if (CHECK_JUST(IsMultiClient())) { - // Do nothing. - } else { - ctx->BnInOp2Blob("out")->CopyHeaderFrom(ctx->BnInOp2Blob("in")); - } + // Do nothing. } REGISTER_KERNEL(OperatorConf::kOutputConf, OutputKernel); diff --git a/oneflow/core/kernel/return_kernel.cpp b/oneflow/core/kernel/return_kernel.cpp index 1cc7d30ad50..e719b41aa55 100644 --- a/oneflow/core/kernel/return_kernel.cpp +++ b/oneflow/core/kernel/return_kernel.cpp @@ -16,7 +16,6 @@ limitations under the License. #include "oneflow/core/kernel/kernel.h" #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/job/critical_section_instance.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job/global_for.h" namespace oneflow { @@ -33,7 +32,6 @@ class ReturnKernel final : public Kernel { }; void ReturnKernel::ForwardDataContent(KernelContext* ctx) const { - if (CHECK_JUST(IsMultiClient())) { CHECK(this->op_conf().return_conf().has_job_name()); const auto& job_name = this->op_conf().return_conf().job_name(); const auto& op_name = this->op_conf().name(); @@ -46,18 +44,10 @@ void ReturnKernel::ForwardDataContent(KernelContext* ctx) const { OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); } - } else { - AutoMemcpy(ctx->stream(), ctx->BnInOp2Blob("out"), ctx->BnInOp2Blob("in")); - CHECK_JUST(ctx->stream()->Sync()); - } } void ReturnKernel::ForwardHeader(KernelContext* ctx) const { - if (CHECK_JUST(IsMultiClient())) { - // Do nothing. - } else { - ctx->BnInOp2Blob("out")->CopyHeaderFrom(ctx->BnInOp2Blob("in")); - } + // Do nothing. } REGISTER_KERNEL(OperatorConf::kReturnConf, ReturnKernel); diff --git a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp index a4b0d9a31b8..3829d2dc2d0 100644 --- a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp +++ b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp @@ -16,7 +16,6 @@ limitations under the License. #include "oneflow/core/kernel/wait_and_send_ids_kernel.h" #include "oneflow/core/common/buffer_manager.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/job/job_instance.h" #include "oneflow/core/job/global_for.h" @@ -32,7 +31,6 @@ void WaitAndSendIdsKernel::ForwardDataContent(KernelContext* ctx) const { auto* status = CHECK_NOTNULL(dynamic_cast(ctx->state().get())); const auto& conf = this->op_conf().wait_and_send_ids_conf(); if (status->out_idx_ >= status->out_num_) { - if (CHECK_JUST(IsMultiClient())) { CHECK(this->op_conf().wait_and_send_ids_conf().has_job_name()); const auto& job_name = this->op_conf().wait_and_send_ids_conf().job_name(); auto* buffer_mgr = Global>>::Get(); @@ -45,20 +43,9 @@ void WaitAndSendIdsKernel::ForwardDataContent(KernelContext* ctx) const { if (status->buffer_status_ == kBufferStatusErrorClosed) { return; } status->out_idx_ = 0; status->out_num_ = 1; - } else { - auto* buffer_mgr = Global>::Get(); - status->buffer_status_ = buffer_mgr->Get(conf.wait_buffer_name())->Pull(&status->in_id_); - if (status->buffer_status_ == kBufferStatusErrorClosed) { return; } - status->out_idx_ = 0; - status->out_num_ = conf.id_list(status->in_id_).value_size(); - } } - if (CHECK_JUST(IsMultiClient())) { - *ctx->BnInOp2Blob("out")->mut_dptr() = 0; - } else { - *ctx->BnInOp2Blob("out")->mut_dptr() = conf.id_list(status->in_id_).value(status->out_idx_); - } + *ctx->BnInOp2Blob("out")->mut_dptr() = 0; ++status->out_idx_; } diff --git a/oneflow/core/operator/output_op.cpp b/oneflow/core/operator/output_op.cpp index 8766235d568..f46ec4858d1 100644 --- a/oneflow/core/operator/output_op.cpp +++ b/oneflow/core/operator/output_op.cpp @@ -13,7 +13,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/operator/interface_op_util.h" #include "oneflow/core/operator/output_op.h" #include "oneflow/core/job/sbp_signature_builder.h" @@ -32,12 +31,7 @@ Maybe OutputOp::InferLogicalOutBlobDescs( const std::function& BlobDesc4BnInOp, const ParallelDesc& parallel_desc) const { BlobDesc* out_blob_desc = BlobDesc4BnInOp("out"); - if (CHECK_JUST(IsMultiClient())) { - *out_blob_desc = *BlobDesc4BnInOp("in"); - } else { - JUST(InterfaceOpUtil::InferLogicalOutBlobDesc(op_conf().output_conf().blob_conf(), - out_blob_desc, parallel_desc)); - } + *out_blob_desc = *BlobDesc4BnInOp("in"); return Maybe::Ok(); } @@ -46,22 +40,10 @@ Maybe OutputOp::InferOutBlobDescs( const ParallelContext* parallel_ctx) const { const BlobDesc* in_blob_desc = GetBlobDesc4BnInOp("in"); BlobDesc* out_blob_desc = GetBlobDesc4BnInOp("out"); - if (CHECK_JUST(IsMultiClient())) { - // NOTE(chengcheng): - // In multi-client, in blob shape maybe changed and NOT equal with output_conf.blob_conf, - // and the output op actually is return op (used in single-client) with NO blob conf. - *out_blob_desc = *in_blob_desc; - } else { - if (in_blob_desc->is_dynamic()) { - *out_blob_desc = *in_blob_desc; - } else { - JUST(InterfaceOpUtil::InferOutBlobDesc(op_conf().output_conf().blob_conf(), out_blob_desc, - parallel_ctx, *JUST(GetOpParallelDesc()))); - CHECK_OR_RETURN(out_blob_desc->shape() == in_blob_desc->shape()); - CHECK_OR_RETURN(out_blob_desc->data_type() == in_blob_desc->data_type()); - CHECK_OR_RETURN(*out_blob_desc == *in_blob_desc); - } - } + // NOTE(chengcheng): + // In multi-client, in blob shape maybe changed and NOT equal with output_conf.blob_conf, + // and the output op actually is return op (used in single-client) with NO blob conf. + *out_blob_desc = *in_blob_desc; return Maybe::Ok(); } diff --git a/oneflow/core/rpc/lib/grpc.cpp b/oneflow/core/rpc/lib/grpc.cpp index 9ed0edab226..a0a5a637068 100644 --- a/oneflow/core/rpc/lib/grpc.cpp +++ b/oneflow/core/rpc/lib/grpc.cpp @@ -15,7 +15,6 @@ limitations under the License. */ #ifdef RPC_BACKEND_GRPC -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/control/ctrl_bootstrap.h" #include "oneflow/core/control/ctrl_server.h" #include "oneflow/core/rpc/include/grpc.h" diff --git a/oneflow/core/vm/id_generator.cpp b/oneflow/core/vm/id_generator.cpp index e302f95e302..61232a5b082 100644 --- a/oneflow/core/vm/id_generator.cpp +++ b/oneflow/core/vm/id_generator.cpp @@ -13,7 +13,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/control/global_process_ctx.h" #include "oneflow/core/vm/id_generator.h" #include "oneflow/core/vm/id_util.h" @@ -22,24 +21,15 @@ namespace oneflow { namespace vm { Maybe LogicalIdGenerator::NewSymbolId() { - if (JUST(IsMultiClient())) { - // NOTE(chengcheng): in Multi-Client LogicalIdGenerator will degenerate directly to - // PhysicalIdGenerator, because each rank will generate id ONLY from itself, NOT the master. - return IdUtil::NewPhysicalSymbolId(GlobalProcessCtx::Rank()); - } - CHECK_OR_RETURN(GlobalProcessCtx::IsThisProcessMaster()); - return IdUtil::NewLogicalSymbolId(); + // NOTE(chengcheng): in Multi-Client LogicalIdGenerator will degenerate directly to + // PhysicalIdGenerator, because each rank will generate id ONLY from itself, NOT the master. + return IdUtil::NewPhysicalSymbolId(GlobalProcessCtx::Rank()); } Maybe LogicalIdGenerator::NewObjectId() { - if (JUST(IsMultiClient())) { - // NOTE(chengcheng): in Multi-Client LogicalIdGenerator will degenerate directly to - // PhysicalIdGenerator, because each rank will generate id ONLY from itself, NOT the master. - return IdUtil::NewPhysicalObjectId(GlobalProcessCtx::Rank()); - } - - CHECK_OR_RETURN(GlobalProcessCtx::IsThisProcessMaster()); - return IdUtil::NewLogicalObjectId(); + // NOTE(chengcheng): in Multi-Client LogicalIdGenerator will degenerate directly to + // PhysicalIdGenerator, because each rank will generate id ONLY from itself, NOT the master. + return IdUtil::NewPhysicalObjectId(GlobalProcessCtx::Rank()); } Maybe PhysicalIdGenerator::NewSymbolId() { diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index 14abd81e3a8..935b3c9f5df 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -19,7 +19,6 @@ limitations under the License. #include "oneflow/core/vm/no_arg_cb_phy_instr_operand.h" #include "oneflow/core/vm/vm_util.h" #include "oneflow/core/common/blocking_counter.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/cpp_attribute.h" #include "oneflow/core/control/global_process_ctx.h" #include "oneflow/core/job/global_for.h" @@ -54,7 +53,6 @@ Maybe ForEachThreadCtx(vm::VirtualMachineEngine* vm, void GetSchedulerThreadInitializer(std::function* Initializer) { *Initializer = [&]() { - if (!CHECK_JUST(IsMultiClient())) { return; } CHECK_JUST(InitThisThreadUniqueConsistentId(kThreadConsistentIdScheduler, "scheduler")); OF_PROFILER_NAME_THIS_HOST_THREAD("_VM::Scheduler"); }; @@ -62,7 +60,6 @@ void GetSchedulerThreadInitializer(std::function* Initializer) { void GetCallbackThreadInitializer(std::function* Initializer) { *Initializer = [&]() { - if (!CHECK_JUST(IsMultiClient())) { return; } OF_PROFILER_NAME_THIS_HOST_THREAD("_VM::Callback"); }; } @@ -97,7 +94,6 @@ void GetWorkerThreadInitializer(intrusive::shared_ptr stream_type_index2consistent_id[stream_type_index] = thread_consistent_id++; } *Initializer = [stream_type_index2consistent_id](vm::ThreadCtx* thread_ctx) { - if (!CHECK_JUST(IsMultiClient())) { return; } const auto& stream_type_index = GetStreamTypeIndex(thread_ctx); const auto& iter = stream_type_index2consistent_id.find(stream_type_index); if (iter != stream_type_index2consistent_id.end()) { @@ -196,7 +192,6 @@ std::string VirtualMachine::GetBlockingDebugString() { Maybe VirtualMachine::Receive(vm::InstructionMsgList* instr_list) { if (unlikely(pthread_fork::IsForkedSubProcess())) { - CHECK_OR_RETURN(JUST(IsMultiClient())); INTRUSIVE_FOR_EACH_PTR(instr_msg, instr_list) { const auto& parallel_desc = instr_msg->phy_instr_parallel_desc(); CHECK(!parallel_desc || parallel_desc->device_type() == DeviceType::kCPU) diff --git a/oneflow/core/vm/vm_util.cpp b/oneflow/core/vm/vm_util.cpp index 0345c4c6ca6..3a39a93256c 100644 --- a/oneflow/core/vm/vm_util.cpp +++ b/oneflow/core/vm/vm_util.cpp @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include "oneflow/core/common/blocking_counter.h" -#include "oneflow/core/common/multi_client.h" + #include "oneflow/core/common/util.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/job/cluster_instruction.h" diff --git a/oneflow/user/data/distributed_util.h b/oneflow/user/data/distributed_util.h index 8463f018b45..2dee5b5f9e6 100644 --- a/oneflow/user/data/distributed_util.h +++ b/oneflow/user/data/distributed_util.h @@ -17,7 +17,6 @@ limitations under the License. #define ONEFLOW_USER_DATA_DISTRIBUTED_UTIL_H_ #include "oneflow/core/framework/framework.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/nd_index_offset_helper.h" #include "oneflow/core/job/sbp_parallel.h" #include "oneflow/core/rpc/include/global_process_ctx.h" @@ -29,7 +28,7 @@ namespace data { inline Maybe InitDataSourceDistributedInfo(user_op::KernelInitContext* ctx, size_t& world_size, int64_t& rank) { auto nd_sbp_str_vec = ctx->Attr>("nd_sbp"); - if (nd_sbp_str_vec.empty() && JUST(IsMultiClient())) { + if (nd_sbp_str_vec.empty()) { world_size = GlobalProcessCtx::WorldSize(); rank = GlobalProcessCtx::Rank(); } else { diff --git a/oneflow/user/data/ofrecord_dataset.h b/oneflow/user/data/ofrecord_dataset.h index 0331c2a9008..4bb0ff09ee3 100644 --- a/oneflow/user/data/ofrecord_dataset.h +++ b/oneflow/user/data/ofrecord_dataset.h @@ -17,7 +17,6 @@ limitations under the License. #define ONEFLOW_USER_DATA_OFRECORD_DATASET_H_ #include "oneflow/core/common/balanced_splitter.h" -#include "oneflow/core/common/multi_client.h" #include "oneflow/core/common/str_util.h" #include "oneflow/core/framework/op_kernel.h" #include "oneflow/core/persistence/persistent_in_stream.h" @@ -65,7 +64,7 @@ class OFRecordDataset final : public Dataset { auto nd_sbp_str_vec = ctx->Attr>("nd_sbp"); // NOTE(zwx): OFRecordDataset is not consistent since attr nd_sbp is empty, // we assume that it works in DDP - if (nd_sbp_str_vec.empty() && CHECK_JUST(IsMultiClient())) { is_local = true; } + if (nd_sbp_str_vec.empty()) { is_local = true; } } if (is_local) { parallel_id_ = GlobalProcessCtx::Rank(); diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index 0d2b487264b..84c4ec6b7f2 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -211,7 +211,6 @@ def is_deprecated(func_or_class): if not env_util.HasAllMultiClientEnvVars(): env_util.SetDefaultMultiClientEnvVars() -oneflow._oneflow_internal.SetIsMultiClient(True) env_util.api_env_init() oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() @@ -262,10 +261,7 @@ def is_normal_exit(self): def atexit_hook(hook): if hook.is_normal_exit(): if oneflow._oneflow_internal.IsEnvInited(): - if oneflow.env.is_multi_client(): - oneflow._oneflow_internal.eager.multi_client.Sync() - elif oneflow.env.get_rank() == 0: - oneflow._oneflow_internal.eager.single_client.Sync() + oneflow._oneflow_internal.eager.multi_client.Sync() oneflow.framework.session_context.TryCloseDefaultSession() if hook.is_normal_exit(): oneflow._oneflow_internal.DestroyEnv() diff --git a/python/oneflow/__main__.py b/python/oneflow/__main__.py index c53d68972df..1c4d0f60510 100644 --- a/python/oneflow/__main__.py +++ b/python/oneflow/__main__.py @@ -17,29 +17,11 @@ import os parser = argparse.ArgumentParser() -parser.add_argument( - "--start_worker", default=False, action="store_true", required=False -) -parser.add_argument("--env_proto", type=str, required=False) parser.add_argument("--doctor", default=False, action="store_true", required=False) args = parser.parse_args() -def StartWorker(env_proto): - import oneflow._oneflow_internal - - oneflow._oneflow_internal.InitEnv(env_proto, False) - - def main(): - start_worker = args.start_worker - if start_worker: - env_proto = args.env_proto - assert os.path.isfile( - env_proto - ), "env_proto not found, please check your env_proto path: {}".format(env_proto) - with open(env_proto, "rb") as f: - StartWorker(f.read()) if args.doctor: import oneflow import oneflow.sysconfig diff --git a/python/oneflow/env.py b/python/oneflow/env.py index 6a7763a84e1..6eb148acb4c 100644 --- a/python/oneflow/env.py +++ b/python/oneflow/env.py @@ -68,12 +68,3 @@ def get_world_size(): """ return oneflow._oneflow_internal.GetWorldSize() - -def is_multi_client(): - """Returns whether it is currently in multi client mode. - - Returns: - True if currently in multi client mode, otherwise returns Flase. - - """ - return oneflow._oneflow_internal.IsMultiClient() diff --git a/python/oneflow/framework/c_api_util.py b/python/oneflow/framework/c_api_util.py index 82f0a993a7b..2382a3e2b4a 100644 --- a/python/oneflow/framework/c_api_util.py +++ b/python/oneflow/framework/c_api_util.py @@ -42,10 +42,10 @@ def EnvResource(): return text_format.Parse(resource, resource_util.Resource()) -def InitEnv(env_proto, is_multi_client): +def InitEnv(env_proto): assert type(env_proto) is env_pb2.EnvProto env_proto_str = text_format.MessageToString(env_proto) - oneflow._oneflow_internal.InitEnv(env_proto_str, is_multi_client) + oneflow._oneflow_internal.InitEnv(env_proto_str) def InitLazyGlobalSession(config_proto): diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 552f76c1669..209db0059f1 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -93,17 +93,11 @@ def check_non_localhost_proxy_and_print_warning(): @enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) def env_init(): global default_env_proto - is_multi_client = oneflow._oneflow_internal.IsMultiClient() assert len(default_env_proto.machine) > 0 - CompleteEnvProto(default_env_proto, is_multi_client) + CompleteEnvProto(default_env_proto) if default_env_proto.ctrl_bootstrap_conf.world_size > 1: check_non_localhost_proxy_and_print_warning() - c_api_util.InitEnv(default_env_proto, is_multi_client) - if not is_multi_client: - if oneflow._oneflow_internal.CurrentMachineId() == 0: - scope_util.InitScopeStack() - else: - exit(0) + c_api_util.InitEnv(default_env_proto) return True @@ -229,9 +223,8 @@ def do_nothing(*args, **kwargs): return False -def CompleteEnvProto(env_proto, is_multi_client): - if is_multi_client: - _UpdateDefaultEnvProtoByMultiClientEnvVars(env_proto) +def CompleteEnvProto(env_proto): + _UpdateDefaultEnvProtoByMultiClientEnvVars(env_proto) if env_proto.HasField("ctrl_port") == False: if len(env_proto.machine) == 1: env_proto.ctrl_port = _FindFreePort() diff --git a/python/oneflow/framework/unittest.py b/python/oneflow/framework/unittest.py index d04b077301c..6ec5e9c2703 100644 --- a/python/oneflow/framework/unittest.py +++ b/python/oneflow/framework/unittest.py @@ -114,21 +114,11 @@ def node_size(): def has_world_size(): - if oneflow.env.is_multi_client(): - return True - if os.getenv("ONEFLOW_TEST_WORLD_SIZE"): - assert os.getenv( - "ONEFLOW_TEST_WORLD_SIZE" - ).isdigit(), "env var ONEFLOW_TEST_WORLD_SIZE must be num" - return True - else: - return False + return True def world_size(): - if oneflow.env.is_multi_client(): - return oneflow.env.get_world_size() - return int(os.getenv("ONEFLOW_TEST_WORLD_SIZE")) + return oneflow.env.get_world_size() def device_num(): diff --git a/python/oneflow/test/graph/test_graph.py b/python/oneflow/test/graph/test_graph.py index d66756c03f7..b0048da0e2b 100644 --- a/python/oneflow/test/graph/test_graph.py +++ b/python/oneflow/test/graph/test_graph.py @@ -152,7 +152,6 @@ def build(self, x): ) return x - test_case.assertTrue(oneflow._oneflow_internal.IsMultiClient()) g = CustomGraphGraphBuildCtx() test_case.assertEqual(graph_build_util.lazy_mode.is_enabled(), False) data = np.array([2.0, 1.0, 0.0, -1.0, -2.0]) diff --git a/python/oneflow/test/graph/test_input_op_expr.py b/python/oneflow/test/graph/test_input_op_expr.py index 57c12b2ea10..34c85d17615 100644 --- a/python/oneflow/test/graph/test_input_op_expr.py +++ b/python/oneflow/test/graph/test_input_op_expr.py @@ -32,7 +32,6 @@ @flow.unittest.skip_unless_1n1d() class TestFeedInputTensor(unittest.TestCase): def test_feed_input_tensor(test_case): - test_case.assertTrue(oneflow.env.is_multi_client()) test_case.assertTrue(oneflow.framework.env_util.HasAllMultiClientEnvVars()) x = flow.Tensor(1, 1, 10, 10) flow.nn.init.uniform_(x, a=-1.0, b=1.0) diff --git a/python/oneflow/test/graph/test_multi_client_session.py b/python/oneflow/test/graph/test_multi_client_session.py index d3bca08b97e..c385fc370ab 100644 --- a/python/oneflow/test/graph/test_multi_client_session.py +++ b/python/oneflow/test/graph/test_multi_client_session.py @@ -27,7 +27,6 @@ @flow.unittest.skip_unless_1n1d() class TestMultiClientSession(unittest.TestCase): def test_case1(self): - self.assertTrue(flow.env.is_multi_client()) sess = session_ctx.GetDefaultSession() self.assertTrue(isinstance(sess, MultiClientSession)) sess.TryInit() @@ -35,7 +34,6 @@ def test_case1(self): def test_case2(self): print("test_case2") - self.assertTrue(flow.env.is_multi_client()) sess = session_ctx.GetDefaultSession() self.assertTrue(isinstance(sess, MultiClientSession)) sess.TryInit() diff --git a/python/oneflow/test/graph/test_output_op_expr.py b/python/oneflow/test/graph/test_output_op_expr.py index d6901bed6a6..a1cafa179e3 100644 --- a/python/oneflow/test/graph/test_output_op_expr.py +++ b/python/oneflow/test/graph/test_output_op_expr.py @@ -32,7 +32,6 @@ @flow.unittest.skip_unless_1n1d() class TestFetchOutputTensor(unittest.TestCase): def test_fetch_output_tensor(test_case): - test_case.assertTrue(oneflow.env.is_multi_client()) test_case.assertTrue(oneflow.framework.env_util.HasAllMultiClientEnvVars()) x = flow.Tensor(1, 1, 10, 10) flow.nn.init.uniform_(x, a=-1.0, b=1.0) diff --git a/python/oneflow/test/graph/test_user_op_expr.py b/python/oneflow/test/graph/test_user_op_expr.py index 0576c718b4c..7a0e2811e20 100644 --- a/python/oneflow/test/graph/test_user_op_expr.py +++ b/python/oneflow/test/graph/test_user_op_expr.py @@ -36,7 +36,6 @@ def _get_c_tensor(t): def _test_user_op_graph(test_case, is_cuda): - test_case.assertTrue(oneflow.env.is_multi_client()) test_case.assertTrue(oneflow.framework.env_util.HasAllMultiClientEnvVars()) x0 = flow.tensor(np.random.rand(20, 30), dtype=flow.float32) diff --git a/python/oneflow/test/graph/test_variable_op_expr.py b/python/oneflow/test/graph/test_variable_op_expr.py index f7cfb258e40..821eb1e89f5 100644 --- a/python/oneflow/test/graph/test_variable_op_expr.py +++ b/python/oneflow/test/graph/test_variable_op_expr.py @@ -32,7 +32,6 @@ @flow.unittest.skip_unless_1n1d() class TestFeedVariableTensor(unittest.TestCase): def test_feed_var_tensor(test_case): - test_case.assertTrue(oneflow.env.is_multi_client()) test_case.assertTrue(oneflow.framework.env_util.HasAllMultiClientEnvVars()) x = flow.Tensor(1, 1, 10, 10) flow.nn.init.uniform_(x, a=-1.0, b=1.0) diff --git a/python/oneflow/utils/data/distributed.py b/python/oneflow/utils/data/distributed.py index 67d82dbb67a..80466117abe 100644 --- a/python/oneflow/utils/data/distributed.py +++ b/python/oneflow/utils/data/distributed.py @@ -81,9 +81,6 @@ def __init__( seed: int = 0, drop_last: bool = False, ) -> None: - if not flow.env.is_multi_client(): - raise RuntimeError("Requires multi-client env to be available") - if num_replicas is None: num_replicas = flow.env.get_world_size() if rank is None: From 959346b7b068b5e49733d321aea4a682df430aa7 Mon Sep 17 00:00:00 2001 From: daquexian Date: Thu, 3 Mar 2022 19:33:05 +0800 Subject: [PATCH 02/54] rename eager.multi_client to eager Signed-off-by: daquexian --- oneflow/api/python/eager/{multi_client.cpp => eager.cpp} | 2 +- oneflow/core/kernel/wait_and_send_ids_kernel.cpp | 1 - python/oneflow/__init__.py | 2 +- python/oneflow/comm/comm_ops.py | 2 +- python/oneflow/framework/graph_build_util.py | 2 +- python/oneflow/nn/graph/graph.py | 6 +++--- python/oneflow/nn/optimizer/optimizer.py | 2 +- python/oneflow/test/modules/test_fork_sub_process.py | 2 +- .../test/modules/test_sparse_softmax_cross_entropy.py | 2 +- 9 files changed, 10 insertions(+), 11 deletions(-) rename oneflow/api/python/eager/{multi_client.cpp => eager.cpp} (95%) diff --git a/oneflow/api/python/eager/multi_client.cpp b/oneflow/api/python/eager/eager.cpp similarity index 95% rename from oneflow/api/python/eager/multi_client.cpp rename to oneflow/api/python/eager/eager.cpp index f700a8dee72..3200c4e3b4a 100644 --- a/oneflow/api/python/eager/multi_client.cpp +++ b/oneflow/api/python/eager/eager.cpp @@ -18,7 +18,7 @@ limitations under the License. #include "oneflow/core/vm/vm_util.h" #include "oneflow/core/eager/dev_vm_dep_object_consume_mode.h" -ONEFLOW_API_PYBIND11_MODULE("eager.multi_client", m) { +ONEFLOW_API_PYBIND11_MODULE("eager", m) { using namespace oneflow; namespace py = pybind11; m.def( diff --git a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp index 3829d2dc2d0..d050346bd3b 100644 --- a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp +++ b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp @@ -29,7 +29,6 @@ void WaitAndSendIdsKernel::VirtualKernelInit(KernelContext* ctx) { template void WaitAndSendIdsKernel::ForwardDataContent(KernelContext* ctx) const { auto* status = CHECK_NOTNULL(dynamic_cast(ctx->state().get())); - const auto& conf = this->op_conf().wait_and_send_ids_conf(); if (status->out_idx_ >= status->out_num_) { CHECK(this->op_conf().wait_and_send_ids_conf().has_job_name()); const auto& job_name = this->op_conf().wait_and_send_ids_conf().job_name(); diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index 84c4ec6b7f2..f4156f35e54 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -261,7 +261,7 @@ def is_normal_exit(self): def atexit_hook(hook): if hook.is_normal_exit(): if oneflow._oneflow_internal.IsEnvInited(): - oneflow._oneflow_internal.eager.multi_client.Sync() + oneflow._oneflow_internal.eager.Sync() oneflow.framework.session_context.TryCloseDefaultSession() if hook.is_normal_exit(): oneflow._oneflow_internal.DestroyEnv() diff --git a/python/oneflow/comm/comm_ops.py b/python/oneflow/comm/comm_ops.py index a442e579e9f..93f47931d78 100644 --- a/python/oneflow/comm/comm_ops.py +++ b/python/oneflow/comm/comm_ops.py @@ -257,7 +257,7 @@ def barrier(): Synchronizes all processes. """ - flow._oneflow_internal.eager.multi_client.Sync() + flow._oneflow_internal.eager.Sync() def reduce_scatter(output, input_list): diff --git a/python/oneflow/framework/graph_build_util.py b/python/oneflow/framework/graph_build_util.py index 71fbdbef798..4e18898655c 100644 --- a/python/oneflow/framework/graph_build_util.py +++ b/python/oneflow/framework/graph_build_util.py @@ -141,7 +141,7 @@ def build_scope(builder): assert new_scope is not None oneflow._oneflow_internal.deprecated.PhysicalRun(build_scope) - oneflow._oneflow_internal.eager.multi_client.Sync() + oneflow._oneflow_internal.eager.Sync() return new_scope diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 7017e59c86d..0618b1132bd 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -309,7 +309,7 @@ def state_dict( """ # Sync to make sure states has been updated. - oneflow._oneflow_internal.eager.multi_client.Sync() + oneflow._oneflow_internal.eager.Sync() if destination is None: destination = OrderedDict() destination._metadata = OrderedDict() @@ -383,7 +383,7 @@ def load_state_dict( additional_var_names, convert_to_tensor_tuple(additional_var_tensors) ) # Sync to make sure states has been loaded. - oneflow._oneflow_internal.eager.multi_client.Sync() + oneflow._oneflow_internal.eager.Sync() @property def name(self): @@ -884,7 +884,7 @@ def __run(self, *args, **kwargs): ] eager_outputs = self._eager_outputs_buffer[self._cur_index_of_ouputs_buffer] - # oneflow._oneflow_internal.eager.multi_client.Sync() NOTE(chengcheng): Need Sync? + # oneflow._oneflow_internal.eager.Sync() NOTE(chengcheng): Need Sync? oneflow._oneflow_internal.nn.graph.RunLazyNNGraph( convert_to_tensor_tuple(flattened_eager_args), outputs_tensor_tuple, diff --git a/python/oneflow/nn/optimizer/optimizer.py b/python/oneflow/nn/optimizer/optimizer.py index 4c75ec4e175..ebb55a9ed71 100644 --- a/python/oneflow/nn/optimizer/optimizer.py +++ b/python/oneflow/nn/optimizer/optimizer.py @@ -87,7 +87,7 @@ def __init__(self): def __enter__(self): self.guard = ( - flow._oneflow_internal.eager.multi_client.SourceOpOnlyResourceDependenceModeGuard() + flow._oneflow_internal.eager.SourceOpOnlyResourceDependenceModeGuard() ) def __exit__(self, *args, **kwargs): diff --git a/python/oneflow/test/modules/test_fork_sub_process.py b/python/oneflow/test/modules/test_fork_sub_process.py index 43d6fc0b212..b0e7012d3cd 100644 --- a/python/oneflow/test/modules/test_fork_sub_process.py +++ b/python/oneflow/test/modules/test_fork_sub_process.py @@ -38,7 +38,7 @@ def _test_fork_sub_process(id): @flow.unittest.skip_unless_1n1d() class TestForkSubProcess(flow.unittest.TestCase): def test_fork_sub_process(test_case): - flow._oneflow_internal.eager.multi_client.Sync() + flow._oneflow_internal.eager.Sync() print("=============main process start=============") # process pool num_process = 4 diff --git a/python/oneflow/test/modules/test_sparse_softmax_cross_entropy.py b/python/oneflow/test/modules/test_sparse_softmax_cross_entropy.py index 4a63221fb0a..c3d81e6ca84 100644 --- a/python/oneflow/test/modules/test_sparse_softmax_cross_entropy.py +++ b/python/oneflow/test/modules/test_sparse_softmax_cross_entropy.py @@ -220,7 +220,7 @@ def build(self, logits, labels): of_output = of_output.to_global(placement=placement, sbp=[flow.sbp.broadcast]) of_output = of_output.to_local() - flow._oneflow_internal.eager.multi_client.Sync() + flow._oneflow_internal.eager.Sync() if rank == 0: assert np.allclose( From 9a03edbee8985fc73e8e90bc4b8172b7496af0fd Mon Sep 17 00:00:00 2001 From: oneflow-ci-bot Date: Thu, 3 Mar 2022 11:39:22 +0000 Subject: [PATCH 03/54] auto format by CI --- ci/test/distributed_run.py | 1 - oneflow/core/device/cuda_util.cpp | 4 +--- oneflow/core/job/parallel_desc.cpp | 16 ++++++------- oneflow/core/job_rewriter/job_completer.cpp | 8 ++----- oneflow/core/kernel/input_kernel.cpp | 24 +++++++++---------- oneflow/core/kernel/output_kernel.cpp | 24 +++++++++---------- oneflow/core/kernel/return_kernel.cpp | 24 +++++++++---------- .../core/kernel/wait_and_send_ids_kernel.cpp | 24 +++++++++---------- oneflow/core/vm/virtual_machine.cpp | 4 +--- python/oneflow/env.py | 1 - 10 files changed, 60 insertions(+), 70 deletions(-) diff --git a/ci/test/distributed_run.py b/ci/test/distributed_run.py index 5b95fc644f5..5e87ad94ac0 100644 --- a/ci/test/distributed_run.py +++ b/ci/test/distributed_run.py @@ -646,4 +646,3 @@ def exit_handler(): ], ) ) - diff --git a/oneflow/core/device/cuda_util.cpp b/oneflow/core/device/cuda_util.cpp index 908fb02bb9a..1a3f24183c7 100644 --- a/oneflow/core/device/cuda_util.cpp +++ b/oneflow/core/device/cuda_util.cpp @@ -160,9 +160,7 @@ void CublasMathModeGuard::SetMathMode(cublasMath_t new_mode) { if (new_mode_ != saved_mode_) { OF_CUBLAS_CHECK(cublasSetMathMode(handle_, saved_mode_)); } } -int GetCudaDeviceIndex() { - return GlobalProcessCtx::LocalRank(); -} +int GetCudaDeviceIndex() { return GlobalProcessCtx::LocalRank(); } int GetCudaDeviceCount() { /* static */ int cuda_device_count = 0; diff --git a/oneflow/core/job/parallel_desc.cpp b/oneflow/core/job/parallel_desc.cpp index 19e05ba252a..800060284dd 100644 --- a/oneflow/core/job/parallel_desc.cpp +++ b/oneflow/core/job/parallel_desc.cpp @@ -319,26 +319,26 @@ Maybe ParallelDesc::CheckWithResourceDesc(const ResourceDesc& resource_des Maybe ParallelDesc::CheckDeviceIdsIsValid() const { const auto& sorted_dev_phy_ids_iter = - machine_id2sorted_dev_phy_ids_->find(GlobalProcessCtx::Rank()); + machine_id2sorted_dev_phy_ids_->find(GlobalProcessCtx::Rank()); for (int64_t machine_id : sorted_machine_ids_) { CHECK_LT_OR_RETURN(machine_id, GlobalProcessCtx::WorldSize()) - << "Placment is invalid because rank must be less than world size!"; + << "Placment is invalid because rank must be less than world size!"; } if (sorted_dev_phy_ids_iter != machine_id2sorted_dev_phy_ids_->end()) { for (int64_t dev_phy_id : *sorted_dev_phy_ids_iter->second) { if (device_type_ == DeviceType::kCUDA) { const int64_t gpu_device_num = GetGpuDeviceNum(); CHECK_NE_OR_RETURN(gpu_device_num, 0) - << "Placment with \"cuda\" type is invalid because there is no CUDA device!"; + << "Placment with \"cuda\" type is invalid because there is no CUDA device!"; int64_t device_num = std::min(GlobalProcessCtx::NumOfProcessPerNode(), gpu_device_num); CHECK_LT_OR_RETURN(dev_phy_id, device_num) - << "Placment is invalid because device id must be less than " - << (gpu_device_num < GlobalProcessCtx::NumOfProcessPerNode() - ? "num of CUDA devices on node" - : "num of process per node"); + << "Placment is invalid because device id must be less than " + << (gpu_device_num < GlobalProcessCtx::NumOfProcessPerNode() + ? "num of CUDA devices on node" + : "num of process per node"); } else if (device_type_ == DeviceType::kCPU) { CHECK_LT_OR_RETURN(dev_phy_id, GlobalProcessCtx::NumOfProcessPerNode()) - << "Placment is invalid because device id must be less than num of process per node"; + << "Placment is invalid because device id must be less than num of process per node"; } else { OF_UNIMPLEMENTED(); } diff --git a/oneflow/core/job_rewriter/job_completer.cpp b/oneflow/core/job_rewriter/job_completer.cpp index 3adda093383..a44033808e9 100644 --- a/oneflow/core/job_rewriter/job_completer.cpp +++ b/oneflow/core/job_rewriter/job_completer.cpp @@ -32,16 +32,12 @@ Maybe CheckOpGraph(const OpGraph& op_graph) { JUST(op_graph.MaybeForEachNode([&](OpNode* op_node) -> Maybe { size_t in_cnt = 0; op_graph.ForEachDataAndCtrlInNode(op_node, [&](OpNode*) { ++in_cnt; }); - if (in_cnt == 0) { - CHECK_OR_RETURN(op_node->op().op_conf().has_wait_and_send_ids_conf()); - } + if (in_cnt == 0) { CHECK_OR_RETURN(op_node->op().op_conf().has_wait_and_send_ids_conf()); } size_t out_cnt = 0; op_graph.ForEachDataAndCtrlOutNode(op_node, [&](OpNode*) { ++out_cnt; }); - if (out_cnt == 0) { - CHECK_OR_RETURN(op_node->op().op_conf().has_callback_notify_conf()); - } + if (out_cnt == 0) { CHECK_OR_RETURN(op_node->op().op_conf().has_callback_notify_conf()); } return Maybe::Ok(); })); return Maybe::Ok(); diff --git a/oneflow/core/kernel/input_kernel.cpp b/oneflow/core/kernel/input_kernel.cpp index 596a2256e11..50d5f502366 100644 --- a/oneflow/core/kernel/input_kernel.cpp +++ b/oneflow/core/kernel/input_kernel.cpp @@ -31,18 +31,18 @@ class InputKernel final : public Kernel { private: void ForwardDataContent(KernelContext* ctx) const override { - CHECK(this->op_conf().input_conf().has_job_name()); - const auto& job_name = this->op_conf().input_conf().job_name(); - const auto& op_name = this->op_conf().name(); - auto* buffer_mgr = Global>>::Get(); - auto* buffer = buffer_mgr->Get(GetInputBufferName(job_name, op_name)); - std::shared_ptr critical_section_instance; - BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); - CHECK_NE(buffer_status, kBufferStatusEmpty); - if (buffer_status == kBufferStatusSuccess) { - OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("out")); - critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); - } + CHECK(this->op_conf().input_conf().has_job_name()); + const auto& job_name = this->op_conf().input_conf().job_name(); + const auto& op_name = this->op_conf().name(); + auto* buffer_mgr = Global>>::Get(); + auto* buffer = buffer_mgr->Get(GetInputBufferName(job_name, op_name)); + std::shared_ptr critical_section_instance; + BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); + CHECK_NE(buffer_status, kBufferStatusEmpty); + if (buffer_status == kBufferStatusSuccess) { + OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("out")); + critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); + } } void ForwardHeader(KernelContext* ctx) const override {} }; diff --git a/oneflow/core/kernel/output_kernel.cpp b/oneflow/core/kernel/output_kernel.cpp index 6d426c1613c..ce18d08ff8b 100644 --- a/oneflow/core/kernel/output_kernel.cpp +++ b/oneflow/core/kernel/output_kernel.cpp @@ -32,18 +32,18 @@ class OutputKernel final : public Kernel { }; void OutputKernel::ForwardDataContent(KernelContext* ctx) const { - CHECK(this->op_conf().output_conf().has_job_name()); - const auto& job_name = this->op_conf().output_conf().job_name(); - const auto& op_name = this->op_conf().name(); - auto* buffer_mgr = Global>>::Get(); - auto* buffer = buffer_mgr->Get(GetOutputBufferName(job_name, op_name)); - std::shared_ptr critical_section_instance; - BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); - CHECK_NE(buffer_status, kBufferStatusEmpty); - if (buffer_status == kBufferStatusSuccess) { - OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); - critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); - } + CHECK(this->op_conf().output_conf().has_job_name()); + const auto& job_name = this->op_conf().output_conf().job_name(); + const auto& op_name = this->op_conf().name(); + auto* buffer_mgr = Global>>::Get(); + auto* buffer = buffer_mgr->Get(GetOutputBufferName(job_name, op_name)); + std::shared_ptr critical_section_instance; + BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); + CHECK_NE(buffer_status, kBufferStatusEmpty); + if (buffer_status == kBufferStatusSuccess) { + OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); + critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); + } } void OutputKernel::ForwardHeader(KernelContext* ctx) const { diff --git a/oneflow/core/kernel/return_kernel.cpp b/oneflow/core/kernel/return_kernel.cpp index e719b41aa55..6535b9a158a 100644 --- a/oneflow/core/kernel/return_kernel.cpp +++ b/oneflow/core/kernel/return_kernel.cpp @@ -32,18 +32,18 @@ class ReturnKernel final : public Kernel { }; void ReturnKernel::ForwardDataContent(KernelContext* ctx) const { - CHECK(this->op_conf().return_conf().has_job_name()); - const auto& job_name = this->op_conf().return_conf().job_name(); - const auto& op_name = this->op_conf().name(); - auto* buffer_mgr = Global>>::Get(); - auto* buffer = buffer_mgr->Get(GetOutputBufferName(job_name, op_name)); - std::shared_ptr critical_section_instance; - BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); - CHECK_NE(buffer_status, kBufferStatusEmpty); - if (buffer_status == kBufferStatusSuccess) { - OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); - critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); - } + CHECK(this->op_conf().return_conf().has_job_name()); + const auto& job_name = this->op_conf().return_conf().job_name(); + const auto& op_name = this->op_conf().name(); + auto* buffer_mgr = Global>>::Get(); + auto* buffer = buffer_mgr->Get(GetOutputBufferName(job_name, op_name)); + std::shared_ptr critical_section_instance; + BufferStatus buffer_status = buffer->TryReceive(&critical_section_instance); + CHECK_NE(buffer_status, kBufferStatusEmpty); + if (buffer_status == kBufferStatusSuccess) { + OfBlob ofblob(ctx->stream(), ctx->BnInOp2Blob("in")); + critical_section_instance->AccessBlobByOpName(reinterpret_cast(&ofblob), op_name); + } } void ReturnKernel::ForwardHeader(KernelContext* ctx) const { diff --git a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp index d050346bd3b..4bfe2aa7323 100644 --- a/oneflow/core/kernel/wait_and_send_ids_kernel.cpp +++ b/oneflow/core/kernel/wait_and_send_ids_kernel.cpp @@ -30,18 +30,18 @@ template void WaitAndSendIdsKernel::ForwardDataContent(KernelContext* ctx) const { auto* status = CHECK_NOTNULL(dynamic_cast(ctx->state().get())); if (status->out_idx_ >= status->out_num_) { - CHECK(this->op_conf().wait_and_send_ids_conf().has_job_name()); - const auto& job_name = this->op_conf().wait_and_send_ids_conf().job_name(); - auto* buffer_mgr = Global>>::Get(); - auto* buffer = buffer_mgr->Get(GetSourceTickBufferName(job_name)); - status->in_id_ = 0; - { - std::shared_ptr job_instance; - status->buffer_status_ = buffer->Pull(&job_instance); - } - if (status->buffer_status_ == kBufferStatusErrorClosed) { return; } - status->out_idx_ = 0; - status->out_num_ = 1; + CHECK(this->op_conf().wait_and_send_ids_conf().has_job_name()); + const auto& job_name = this->op_conf().wait_and_send_ids_conf().job_name(); + auto* buffer_mgr = Global>>::Get(); + auto* buffer = buffer_mgr->Get(GetSourceTickBufferName(job_name)); + status->in_id_ = 0; + { + std::shared_ptr job_instance; + status->buffer_status_ = buffer->Pull(&job_instance); + } + if (status->buffer_status_ == kBufferStatusErrorClosed) { return; } + status->out_idx_ = 0; + status->out_num_ = 1; } *ctx->BnInOp2Blob("out")->mut_dptr() = 0; diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index 935b3c9f5df..c4ad3b061a4 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -59,9 +59,7 @@ void GetSchedulerThreadInitializer(std::function* Initializer) { } void GetCallbackThreadInitializer(std::function* Initializer) { - *Initializer = [&]() { - OF_PROFILER_NAME_THIS_HOST_THREAD("_VM::Callback"); - }; + *Initializer = [&]() { OF_PROFILER_NAME_THIS_HOST_THREAD("_VM::Callback"); }; } std::type_index GetStreamTypeIndex(const vm::ThreadCtx* thread_ctx) { diff --git a/python/oneflow/env.py b/python/oneflow/env.py index 6eb148acb4c..c8fed89968f 100644 --- a/python/oneflow/env.py +++ b/python/oneflow/env.py @@ -67,4 +67,3 @@ def get_world_size(): """ return oneflow._oneflow_internal.GetWorldSize() - From c59716f6386a53b27a11ffd05867b293a35349f1 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 5 Mar 2022 20:21:16 +0800 Subject: [PATCH 04/54] add py ref --- python/oneflow/__init__.py | 14 +++----- python/oneflow/framework/env_util.py | 35 +++++++++++++++++++ .../oneflow/framework/multi_client_session.py | 13 ++++--- python/oneflow/framework/session_context.py | 3 +- python/oneflow/nn/graph/graph.py | 20 +++++------ 5 files changed, 58 insertions(+), 27 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index 0d2b487264b..f4020b9f4a2 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -209,18 +209,16 @@ def is_deprecated(func_or_class): from oneflow.framework.multi_client_session import MultiClientSession from oneflow.framework.tensor_str import set_printoptions -if not env_util.HasAllMultiClientEnvVars(): - env_util.SetDefaultMultiClientEnvVars() -oneflow._oneflow_internal.SetIsMultiClient(True) -env_util.api_env_init() +env_util.GetEnvHolder() oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() session_ctx.OpenDefaultSession( - MultiClientSession(oneflow._oneflow_internal.NewSessionId()) + MultiClientSession( + oneflow._oneflow_internal.NewSessionId(), env_util.GetEnvHolder() + ) ) scope_util.InitScopeStack() oneflow._oneflow_internal.EnableEagerEnvironment(True) -del env_util from oneflow.framework import python_callback, register_python_callback oneflow._oneflow_internal.RegisterGlobalForeignCallback( @@ -267,9 +265,7 @@ def atexit_hook(hook): elif oneflow.env.get_rank() == 0: oneflow._oneflow_internal.eager.single_client.Sync() oneflow.framework.session_context.TryCloseDefaultSession() - if hook.is_normal_exit(): - oneflow._oneflow_internal.DestroyEnv() - oneflow._oneflow_internal.SetShuttingDown() + env_util.DelEnvHolder(hook.is_normal_exit()) atexit.register(atexit_hook, hook) diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 552f76c1669..61d86b2c6e4 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -394,6 +394,41 @@ def str2int(env_config): cpp_logging_conf.logbuflevel = os.getenv("GLOG_logbuflevel") env_proto.cpp_logging_conf.CopyFrom(cpp_logging_conf) +class EnvHolder(object): + def __init__(self): + self._is_normal_exit = True + if not HasAllMultiClientEnvVars(): + SetDefaultMultiClientEnvVars() + oneflow._oneflow_internal.SetIsMultiClient(True) + api_env_init() + + def set_is_normal_exit(self, is_normal_exit): + self._is_normal_exit = is_normal_exit + + def __del__(self): + if self._is_normal_exit: + oneflow._oneflow_internal.DestroyEnv() + oneflow._oneflow_internal.SetShuttingDown() + print("oneflow env del") + + +def GetEnvHolder(): + global _env_holder + if _env_holder is not None: + return _env_holder + else: + _env_holder = EnvHolder() + return _env_holder + + +def DelEnvHolder(is_normal_exit): + global _env_holder + assert _env_holder is not None + _env_holder.set_is_normal_exit(is_normal_exit) + del _env_holder + + +_env_holder = None device_tag2default_parallel_conf = {} default_env_proto = _DefaultEnvProto() diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index ae39abf5b97..cddc5436d49 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -29,7 +29,8 @@ class Status(enum.Enum): INITED = 2 CLOSED = 3 - def __init__(self, sess_id): + def __init__(self, sess_id, env_holder): + self._env_holder = env_holder self.sess_ = oneflow._oneflow_internal.RegsiterSession(sess_id) oneflow._oneflow_internal.CreateMultiClientSessionContext() self.config_proto_ = self._make_config_proto() @@ -39,9 +40,6 @@ def __init__(self, sess_id): self._update_scope_attr_name2defaultVal() self.status_ = self.Status.CREATED - def __del__(self): - self.TryClose() - def TryInit(self): self._check_status(self.Status.CREATED, self.Status.INITED) if self.status_ == self.Status.CREATED: @@ -49,7 +47,7 @@ def TryInit(self): oneflow._oneflow_internal.InitMultiClientSessionContext(config_proto_str) self.status_ = self.Status.INITED - def TryClose(self): + def _TryClose(self): if self.status_ != self.Status.CLOSED: oneflow._oneflow_internal.TryDestroyMultiClientSessionContext() oneflow._oneflow_internal.ClearSessionById(self.id) @@ -126,3 +124,8 @@ def update_resource_eagerly(self, resource_config): oneflow._oneflow_internal.MultiClientSessionContextUpdateResource( config_proto_str ) + + def __del__(self): + self._TryClose() + print("oneflow session del") + diff --git a/python/oneflow/framework/session_context.py b/python/oneflow/framework/session_context.py index 66198047abe..948a07ae8a1 100644 --- a/python/oneflow/framework/session_context.py +++ b/python/oneflow/framework/session_context.py @@ -43,8 +43,7 @@ def TryCloseDefaultSession(): default_sess_id = oneflow._oneflow_internal.GetDefaultSessionId() assert default_sess_id in _sess_id2sess if default_sess_id in _sess_id2sess: - _sess_id2sess[default_sess_id].TryClose() - del _sess_id2sess[default_sess_id] + del _sess_id2sess[default_sess_id] def try_init_default_session(func): diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 1d41658e2cf..ee46bbb2ee5 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -131,10 +131,10 @@ def __init__(self): self._cur_index_of_ouputs_buffer = 0 self._c_nn_graph = oneflow._oneflow_internal.nn.graph.CNNGraph(self._name) - session = session_ctx.GetDefaultSession() - assert type(session) is MultiClientSession - session.TryInit() - session.AddCGraph(self._c_nn_graph) + self._session = session_ctx.GetDefaultSession() + assert type(self._session) is MultiClientSession + self._session.TryInit() + self._session.AddCGraph(self._c_nn_graph) def build(self, *args, **kwargs): r"""The ``build()`` method must be overridden to define neural network @@ -522,9 +522,7 @@ def _config_proto(self): @property def _optimization_conf_proto(self): - session = session_ctx.GetDefaultSession() - assert type(session) is MultiClientSession - return session.resource + return self._session.resource @property def _graph_proto(self): @@ -697,9 +695,6 @@ def _compile(self, *args, **kwargs): return eager_outputs def __build_graph(self, *args, **kwargs): - session = session_ctx.GetDefaultSession() - assert type(session) is MultiClientSession - # Filter to get unique states in graph state_op_names = self._filter_states() @@ -720,7 +715,7 @@ def __build_graph(self, *args, **kwargs): + " end building graph builders of parameters and buffers.", ) - with graph_build_util.graph_build_context(self.config.proto, session): + with graph_build_util.graph_build_context(self.config.proto, self._session): # Deal with inputs self.__print(0, 1, self._shallow_repr() + " start building graph inputs.") arg_op_names, lazy_args, lazy_kwargs, self._args_repr, _ = self.__build_io( @@ -1167,6 +1162,9 @@ def __getattr__(self, name: str): "'{}' object has no attribute '{}'".format(type(self).__name__, name) ) + def __del__(self): + print(f"oneflow graph {self.name} del") + if __name__ == "__main__": import doctest From d531d432593eedad8f97f492f06d4201b170221a Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 5 Mar 2022 20:42:22 +0800 Subject: [PATCH 05/54] refine new session --- python/oneflow/__init__.py | 8 ++------ python/oneflow/framework/multi_client_session.py | 6 ++++-- python/oneflow/framework/session_context.py | 8 +++++--- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index f4020b9f4a2..30dbfeacaa4 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -206,17 +206,13 @@ def is_deprecated(func_or_class): import oneflow.framework.env_util as env_util import oneflow.framework.scope_util as scope_util import oneflow.framework.session_context as session_ctx -from oneflow.framework.multi_client_session import MultiClientSession from oneflow.framework.tensor_str import set_printoptions env_util.GetEnvHolder() oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() -session_ctx.OpenDefaultSession( - MultiClientSession( - oneflow._oneflow_internal.NewSessionId(), env_util.GetEnvHolder() - ) -) +session_ctx.NewDefaultSession() + scope_util.InitScopeStack() oneflow._oneflow_internal.EnableEagerEnvironment(True) from oneflow.framework import python_callback, register_python_callback diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index cddc5436d49..b598113fddf 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -21,6 +21,7 @@ import oneflow._oneflow_internal import oneflow.core.job.job_set_pb2 as job_set_util import oneflow.framework.c_api_util as c_api_util +import oneflow.framework.env_util as env_util class MultiClientSession(object): @@ -29,8 +30,9 @@ class Status(enum.Enum): INITED = 2 CLOSED = 3 - def __init__(self, sess_id, env_holder): - self._env_holder = env_holder + def __init__(self, sess_id): + self._env_holder = env_util.GetEnvHolder() + assert self._env_holder is not None self.sess_ = oneflow._oneflow_internal.RegsiterSession(sess_id) oneflow._oneflow_internal.CreateMultiClientSessionContext() self.config_proto_ = self._make_config_proto() diff --git a/python/oneflow/framework/session_context.py b/python/oneflow/framework/session_context.py index 948a07ae8a1..0c52eb34b96 100644 --- a/python/oneflow/framework/session_context.py +++ b/python/oneflow/framework/session_context.py @@ -17,6 +17,7 @@ import oneflow import oneflow._oneflow_internal +from oneflow.framework.multi_client_session import MultiClientSession class SessionStatus: @@ -32,10 +33,11 @@ def GetDefaultSession(): return _sess_id2sess[default_sess_id] -def OpenDefaultSession(sess): +def NewDefaultSession(): + new_default_sess = MultiClientSession(oneflow._oneflow_internal.NewSessionId()) global _sess_id2sess - assert sess.id not in _sess_id2sess - _sess_id2sess[sess.id] = sess + assert new_default_sess.id not in _sess_id2sess + _sess_id2sess[new_default_sess.id] = new_default_sess def TryCloseDefaultSession(): From ade9d9cb51efd11f735af1742382ac8d428e87ed Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 5 Mar 2022 22:36:49 +0800 Subject: [PATCH 06/54] clean code --- python/oneflow/__init__.py | 5 +--- python/oneflow/env.py | 1 - python/oneflow/framework/env_util.py | 25 +++---------------- .../oneflow/framework/multi_client_session.py | 3 ++- 4 files changed, 7 insertions(+), 27 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index 30dbfeacaa4..171ba79363f 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -209,9 +209,9 @@ def is_deprecated(func_or_class): from oneflow.framework.tensor_str import set_printoptions env_util.GetEnvHolder() +session_ctx.NewDefaultSession() oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() -session_ctx.NewDefaultSession() scope_util.InitScopeStack() oneflow._oneflow_internal.EnableEagerEnvironment(True) @@ -281,9 +281,6 @@ def atexit_hook(hook): from oneflow.framework.check_point_v2 import load from oneflow.framework.check_point_v2 import save from oneflow.framework.dtype import convert_oneflow_dtype_to_numpy_dtype, dtypes -from oneflow.framework.env_util import ( - api_enable_eager_execution as enable_eager_execution, -) from oneflow.framework.function_util import FunctionConfig from oneflow.framework.function_util import FunctionConfig as function_config from oneflow.framework.generator import create_generator as Generator diff --git a/python/oneflow/env.py b/python/oneflow/env.py index 6a7763a84e1..ca42086f654 100644 --- a/python/oneflow/env.py +++ b/python/oneflow/env.py @@ -16,7 +16,6 @@ from oneflow.framework.env_util import api_all_device_placement as all_device_placement from oneflow.framework.env_util import api_ctrl_port as ctrl_port from oneflow.framework.env_util import api_data_port as data_port -from oneflow.framework.env_util import api_env_init as init from oneflow.framework.env_util import api_grpc_use_no_signal as grpc_use_no_signal from oneflow.framework.env_util import api_init_bootstrap_confs as init_bootstrap_confs from oneflow.framework.env_util import api_log_dir as log_dir diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 61d86b2c6e4..b74ee7eecaa 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -51,22 +51,8 @@ def api_all_device_placement(device_type: str) -> oneflow._oneflow_internal.plac return oneflow._oneflow_internal.AllDevicePlacement(device_type) -def api_enable_eager_execution(val: bool = True) -> None: - """If True, job will execute in eager mode, else use lazy mode(static graph). - - Args: - val (bool, optional): Whether eager execution or not. Defaults to True. - """ - return enable_if.unique([enable_eager_environment])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.any_global_function_defined) -def enable_eager_environment(val=True): - return oneflow._oneflow_internal.EnableEagerEnvironment(val) - - def api_env_init() -> bool: - """Init environment for job + """Init environment Returns: bool: [description] @@ -90,20 +76,17 @@ def check_non_localhost_proxy_and_print_warning(): break -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) +@enable_if.condition(~hob.env_initialized) def env_init(): global default_env_proto is_multi_client = oneflow._oneflow_internal.IsMultiClient() + if not is_multi_client: + exit(0) assert len(default_env_proto.machine) > 0 CompleteEnvProto(default_env_proto, is_multi_client) if default_env_proto.ctrl_bootstrap_conf.world_size > 1: check_non_localhost_proxy_and_print_warning() c_api_util.InitEnv(default_env_proto, is_multi_client) - if not is_multi_client: - if oneflow._oneflow_internal.CurrentMachineId() == 0: - scope_util.InitScopeStack() - else: - exit(0) return True diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index b598113fddf..a63b4304e74 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -34,6 +34,7 @@ def __init__(self, sess_id): self._env_holder = env_util.GetEnvHolder() assert self._env_holder is not None self.sess_ = oneflow._oneflow_internal.RegsiterSession(sess_id) + self._id = sess_id oneflow._oneflow_internal.CreateMultiClientSessionContext() self.config_proto_ = self._make_config_proto() self.function_flag_name2default_val_ = {} @@ -65,7 +66,7 @@ def status(self): @property def id(self): - return self.sess_.id + return self._id @property def config_proto(self): From e0d7dbb6aa8608cb5d7e1909ce530d7f513ce915 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 5 Mar 2022 22:52:46 +0800 Subject: [PATCH 07/54] make scope api inner use --- python/oneflow/__init__.py | 1 - python/oneflow/framework/scope_util.py | 15 +-------------- python/oneflow/nn/modules/dataset.py | 5 +++-- python/oneflow/serving/inference_session.py | 2 +- python/oneflow/test/graph/test_graph.py | 7 ++++--- .../graph/test_graph_activation_checkpoint.py | 5 +++-- python/oneflow/test/graph/test_graph_block.py | 7 ++++--- 7 files changed, 16 insertions(+), 26 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index 171ba79363f..ee119322fa7 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -293,7 +293,6 @@ def atexit_hook(hook): # NOTE(chengcheng) oneflow.Model is unavailable now. # from oneflow.framework.model import Model -from oneflow.framework.scope_util import api_current_scope as current_scope from oneflow.framework.tensor import Tensor from oneflow.framework.tensor import is_nonzero from oneflow.framework.type_tensor import * diff --git a/python/oneflow/framework/scope_util.py b/python/oneflow/framework/scope_util.py index aca6200faad..aa300032c0a 100644 --- a/python/oneflow/framework/scope_util.py +++ b/python/oneflow/framework/scope_util.py @@ -47,7 +47,7 @@ def SetScopeProto(scope_proto): return ScopeContext(scope) -def api_current_scope(): +def current_scope(): """ Return current scope """ return oneflow._oneflow_internal.GetCurrentScope() @@ -56,19 +56,6 @@ def api_current_scope(): from oneflow import oneflow_deprecate -@oneflow_deprecate() -def deprecated_current_scope(*args, **kwargs): - print( - "WARNING:", - "oneflow.scope.current_scope", - "will be removed in the future, use {} instead.".format( - "oneflow.current_scope" - ), - ) - print(traceback.format_stack()[-2]) - return api_current_scope(*args, **kwargs) - - def MakeScope(build_func): scope = None old_scope = oneflow._oneflow_internal.GetCurrentScope() diff --git a/python/oneflow/nn/modules/dataset.py b/python/oneflow/nn/modules/dataset.py index 5bf99d2e54d..b23a78ab1d0 100644 --- a/python/oneflow/nn/modules/dataset.py +++ b/python/oneflow/nn/modules/dataset.py @@ -22,6 +22,7 @@ import oneflow as flow import oneflow._oneflow_internal._C as _C from oneflow.framework.tensor import Tensor +from oneflow.framework.scope_util import current_scope from oneflow.nn.common_types import _size_1_t, _size_2_t, _size_3_t, _size_any_t from oneflow.nn.module import Module from oneflow.nn.modules.utils import _pair, _reverse_repeat_tuple, _single, _triple @@ -836,7 +837,7 @@ def forward(self): # local apply outputs = _C.dispatch_coco_reader( self._op, - session_id=flow.current_scope().session_id, + session_id=current_scope().session_id, annotation_file=self.annotation_file, image_dir=self.image_dir, batch_size=self.batch_size, @@ -851,7 +852,7 @@ def forward(self): # consistent apply outputs = _C.dispatch_coco_reader( self._op, - session_id=flow.current_scope().session_id, + session_id=current_scope().session_id, annotation_file=self.annotation_file, image_dir=self.image_dir, batch_size=self.batch_size, diff --git a/python/oneflow/serving/inference_session.py b/python/oneflow/serving/inference_session.py index 2f13fc42782..2e8cc93b9d8 100644 --- a/python/oneflow/serving/inference_session.py +++ b/python/oneflow/serving/inference_session.py @@ -263,7 +263,7 @@ def open(self, job_name, signature=None, batch_size=None): def compile(self, op_list): self._check_status(self.SessionStatus.OPEN) - scope = flow.current_scope() + scope = scope_util.current_scope() device_tag = scope.device_parallel_desc_symbol.device_tag for op_conf in op_list: if _need_check_device_tag(op_conf) and op_conf.device_tag != device_tag: diff --git a/python/oneflow/test/graph/test_graph.py b/python/oneflow/test/graph/test_graph.py index d66756c03f7..cb6638e102f 100644 --- a/python/oneflow/test/graph/test_graph.py +++ b/python/oneflow/test/graph/test_graph.py @@ -22,6 +22,7 @@ import oneflow import oneflow as flow import oneflow.framework.graph_build_util as graph_build_util +import oneflow.framework.scope_util as scope_util import oneflow.unittest @@ -143,7 +144,7 @@ def build(self, x): test_case.assertEqual(type(session), MultiClientSession) import oneflow.framework.scope_util as scope_util - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) test_case.assertEqual(session.id, scope_proto.session_id) test_case.assertEqual( @@ -168,7 +169,7 @@ def __init__(self): self.conv1 = flow.nn.Conv2d(1, 1, 5) def forward(self, x): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) @@ -188,7 +189,7 @@ def __init__(self): self.register_buffer("dummy_buff", flow.Tensor(1, 4)) def forward(self, x): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) test_case.assertEqual( scope_proto.parent_scope_symbol_id, self.prev_scope.symbol_id diff --git a/python/oneflow/test/graph/test_graph_activation_checkpoint.py b/python/oneflow/test/graph/test_graph_activation_checkpoint.py index f6f6208715f..cee58c93dcc 100644 --- a/python/oneflow/test/graph/test_graph_activation_checkpoint.py +++ b/python/oneflow/test/graph/test_graph_activation_checkpoint.py @@ -23,6 +23,7 @@ import oneflow import oneflow as flow import oneflow.framework.graph_build_util as graph_build_util +import oneflow.framework.scope_util as scope_util import oneflow.unittest @@ -40,7 +41,7 @@ def __init__(self): self.model = model def forward(self, x): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) @@ -53,7 +54,7 @@ def __init__(self): self.model = model1 def forward(self, x): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) diff --git a/python/oneflow/test/graph/test_graph_block.py b/python/oneflow/test/graph/test_graph_block.py index f82c93dea5d..8a316fd7987 100644 --- a/python/oneflow/test/graph/test_graph_block.py +++ b/python/oneflow/test/graph/test_graph_block.py @@ -21,6 +21,7 @@ import oneflow as flow import oneflow.unittest import oneflow.framework.graph_build_util as graph_build_util +import oneflow.framework.scope_util as scope_util @unittest.skipIf(os.getenv("ONEFLOW_TEST_CPU_ONLY"), "only test cpu cases") @@ -114,7 +115,7 @@ def __init__(self): def forward(self, x): if graph_build_util.lazy_mode.is_enabled(): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) @@ -160,7 +161,7 @@ def __init__(self): def forward(self, x): if graph_build_util.lazy_mode.is_enabled(): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) @@ -213,7 +214,7 @@ def __init__(self, out): def forward(self, x): if graph_build_util.lazy_mode.is_enabled(): - scope = oneflow.current_scope() + scope = scope_util.current_scope() scope_proto = graph_build_util.scope_to_proto(scope) ck_bool = scope_proto.attr_name2attr_value["checkpointing"].at_bool test_case.assertEqual(ck_bool, True) From ba9e31de8c5afec78bd6fb9466a8908c0853fb49 Mon Sep 17 00:00:00 2001 From: strint Date: Sun, 6 Mar 2022 22:05:15 +0800 Subject: [PATCH 08/54] use session with ref cnt --- oneflow/api/cpp/framework/graph.cpp | 3 ++- oneflow/api/python/framework/nn_graph.cpp | 3 ++- oneflow/api/python/session/session.cpp | 15 ++++++++++-- oneflow/api/python/session/session.h | 7 ------ oneflow/api/python/session/session_api.h | 5 ---- .../multi_client_session_context.cpp | 24 ++++++++++++------- .../framework/multi_client_session_context.h | 17 +++++++++---- oneflow/core/framework/nn_graph.cpp | 9 +++---- oneflow/core/framework/nn_graph.h | 8 +++++++ python/oneflow/framework/env_util.py | 3 ++- .../oneflow/framework/multi_client_session.py | 24 +++++++++---------- python/oneflow/nn/graph/graph.py | 5 ++-- 12 files changed, 71 insertions(+), 52 deletions(-) diff --git a/oneflow/api/cpp/framework/graph.cpp b/oneflow/api/cpp/framework/graph.cpp index e1626c20a5a..f7e08cd1977 100644 --- a/oneflow/api/cpp/framework/graph.cpp +++ b/oneflow/api/cpp/framework/graph.cpp @@ -214,7 +214,8 @@ Graph::GraphImpl::GraphImpl(const std::string& model_path, const Device& device) job_.mutable_job_conf()->mutable_predict_conf(); job_.mutable_job_conf()->set_job_name(job_.mutable_job_conf()->job_name() + of::NewUniqueId()); graph_ = std::make_shared(job_.job_conf().job_name()); - of::Global::Get()->AddCGraph(graph_).GetOrThrow(); + // TODO(strint): rm + // of::Global::Get()->AddCGraph(graph_).GetOrThrow(); } Graph::GraphImpl::GraphImpl(GraphImpl&& graph) noexcept diff --git a/oneflow/api/python/framework/nn_graph.cpp b/oneflow/api/python/framework/nn_graph.cpp index 8d5ed3cfa03..49b63532712 100644 --- a/oneflow/api/python/framework/nn_graph.cpp +++ b/oneflow/api/python/framework/nn_graph.cpp @@ -19,6 +19,7 @@ limitations under the License. #include #include "oneflow/api/python/job_build/job_build_and_infer.h" #include "oneflow/api/python/of_api_registry.h" +#include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/framework/tensor.h" #include "oneflow/core/framework/nn_graph.h" #include "oneflow/core/job/runtime.h" @@ -45,7 +46,7 @@ py::object APINNGraphAdditionalVarTensors(const std::shared_ptr& graph) ONEFLOW_API_PYBIND11_MODULE("nn.graph.", m) { using namespace oneflow; py::class_>(m, "CNNGraph") - .def(py::init()) + .def(py::init&>()) .def_property_readonly("name", &NNGraph::job_name) .def( "register_input_op_names_and_tensors", diff --git a/oneflow/api/python/session/session.cpp b/oneflow/api/python/session/session.cpp index d53c4a10045..bc96facc748 100644 --- a/oneflow/api/python/session/session.cpp +++ b/oneflow/api/python/session/session.cpp @@ -17,6 +17,7 @@ limitations under the License. #include #include "oneflow/api/python/of_api_registry.h" #include "oneflow/core/job/session.h" +#include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/api/python/session/session_api.h" namespace py = pybind11; @@ -34,12 +35,22 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("CreateMultiClientSessionContext", &CreateMultiClientSessionContext); m.def("InitMultiClientSessionContext", &InitMultiClientSessionContext); m.def("MultiClientSessionContextUpdateResource", &MultiClientSessionContextUpdateResource); - m.def("MultiClientSessionContextAddCGraph", &MultiClientSessionContextAddCGraph); m.def("TryDestroyMultiClientSessionContext", &TryDestroyMultiClientSessionContext); using namespace oneflow; + py::class_>( + m, "SessionContext") + .def(py::init()) + .def("try_init", + [](MultiClientSessionContext& session, const std::string& config_proto_str) { + return session.TryInit(config_proto_str).GetOrThrow(); + }) + .def("update_resource", + [](MultiClientSessionContext& session, const std::string& reso_proto_str) { + return session.UpdateResource(reso_proto_str).GetOrThrow(); + }); + m.def("NewSessionId", &NewSessionId); py::class_(m, "LogicalConfigProtoContext") .def(py::init()); - ; } diff --git a/oneflow/api/python/session/session.h b/oneflow/api/python/session/session.h index fe960a499cb..f9794947321 100644 --- a/oneflow/api/python/session/session.h +++ b/oneflow/api/python/session/session.h @@ -138,13 +138,6 @@ inline Maybe MultiClientSessionContextUpdateResource(const std::string& re return Maybe::Ok(); } -inline Maybe MultiClientSessionContextAddCGraph( - const std::shared_ptr& c_graph_ptr) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - JUST(Global::Get()->AddCGraph(c_graph_ptr)); - return Maybe::Ok(); -} - inline Maybe TryDestroyMultiClientSessionContext() { // Global::Delete is not allowed to be called here // because glog is not constructed yet and LOG(INFO) has bad bahavior diff --git a/oneflow/api/python/session/session_api.h b/oneflow/api/python/session/session_api.h index dd289da8a33..9f201b1869f 100644 --- a/oneflow/api/python/session/session_api.h +++ b/oneflow/api/python/session/session_api.h @@ -47,11 +47,6 @@ inline void MultiClientSessionContextUpdateResource(const std::string& resource_ return oneflow::MultiClientSessionContextUpdateResource(resource_proto_str).GetOrThrow(); } -inline void MultiClientSessionContextAddCGraph( - const std::shared_ptr& c_graph_ptr) { - return oneflow::MultiClientSessionContextAddCGraph(c_graph_ptr).GetOrThrow(); -} - inline void TryDestroyMultiClientSessionContext() { return oneflow::TryDestroyMultiClientSessionContext().GetOrThrow(); } diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 5ef471d0da3..18882c1b663 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -14,9 +14,11 @@ See the License for the specific language governing permissions and limitations under the License. */ +#include #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/common/maybe.h" #include "oneflow/core/common/multi_client.h" +#include "oneflow/core/common/protobuf.h" #include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/framework/load_library.h" #include "oneflow/core/job/resource.pb.h" @@ -119,16 +121,24 @@ Maybe MultiClientSessionContext::TryInit(const ConfigProto& config_proto) return Maybe::Ok(); } +Maybe MultiClientSessionContext::TryInit(const std::string& config_proto_str) { + ConfigProto config_proto; + CHECK_OR_RETURN(TxtString2PbMessage(config_proto_str, &config_proto)) + << "failed to parse config_proto: " << config_proto_str; + return TryInit(config_proto); +} + Maybe MultiClientSessionContext::UpdateResource(const Resource& reso_proto) { CHECK_NOTNULL_OR_RETURN((Global::Get())); Global::Get()->Update(reso_proto); return Maybe::Ok(); } -Maybe MultiClientSessionContext::AddCGraph( - const std::shared_ptr& c_graph_ptr) { - graphs_.emplace_back(c_graph_ptr); - return Maybe::Ok(); +Maybe MultiClientSessionContext::UpdateResource(const std::string& reso_proto_str) { + Resource reso_proto; + CHECK_OR_RETURN(TxtString2PbMessage(reso_proto_str, &reso_proto)) + << "failed to parse config_proto: " << reso_proto_str; + return UpdateResource(reso_proto); } Maybe MultiClientSessionContext::TryClose() { @@ -137,11 +147,6 @@ Maybe MultiClientSessionContext::TryClose() { // sync before NNGraph release to ensure LaunchLazyJob instruction was completed and released JUST(vm::ClusterSync()); - for (const auto& graph : graphs_) { - VLOG(2) << "Try to close graph: " << graph->job_name() << std::endl; - JUST(graph->Close()); - } - graphs_.clear(); { // NOTE(chengcheng): delete runtime global objects Global::Delete(); @@ -166,6 +171,7 @@ Maybe MultiClientSessionContext::TryClose() { // NOTE(chengcheng): New after delete because in EnvGlobalObjectScope once created ResourceDesc. Global::New(Global::Get()->resource(), GlobalProcessCtx::NumOfProcessPerNode()); + is_inited_ = false; } VLOG(2) << "Finish delete multi client session context." << std::endl; return Maybe::Ok(); diff --git a/oneflow/core/framework/multi_client_session_context.h b/oneflow/core/framework/multi_client_session_context.h index a9722be1163..9352c2caede 100644 --- a/oneflow/core/framework/multi_client_session_context.h +++ b/oneflow/core/framework/multi_client_session_context.h @@ -19,7 +19,6 @@ limitations under the License. #include "oneflow/core/common/util.h" #include "oneflow/core/job/job_set.pb.h" #include "oneflow/core/common/maybe.h" -#include "oneflow/core/framework/nn_graph.h" #include "oneflow/core/framework/tensor.h" namespace oneflow { @@ -27,12 +26,21 @@ namespace oneflow { class MultiClientSessionContext { public: OF_DISALLOW_COPY_AND_MOVE(MultiClientSessionContext); - MultiClientSessionContext() : is_inited_(false) {} - ~MultiClientSessionContext() {} + MultiClientSessionContext() : is_inited_(false) { + CHECK(Global::Get() == nullptr); + Global::SetAllocated(this); + } + ~MultiClientSessionContext() { + CHECK_JUST(TryClose()); + if (Global::Get() != nullptr) { + Global::SetAllocated(nullptr); + } + } Maybe TryInit(const ConfigProto& config_proto); + Maybe TryInit(const std::string& config_proto_str); Maybe UpdateResource(const Resource& reso_proto); - Maybe AddCGraph(const std::shared_ptr& c_graph_ptr); + Maybe UpdateResource(const std::string& reso_proto_str); Maybe TryClose(); // NOTE(chengcheng): for nn.Graph catch free EagerTensor in Graph.build(). @@ -51,7 +59,6 @@ class MultiClientSessionContext { bool is_inited_; HashMap>>> graph_name2free_eager_tensors_; - std::vector> graphs_; }; } // namespace oneflow diff --git a/oneflow/core/framework/nn_graph.cpp b/oneflow/core/framework/nn_graph.cpp index d4ee6cc0a84..f87b66a2bc7 100644 --- a/oneflow/core/framework/nn_graph.cpp +++ b/oneflow/core/framework/nn_graph.cpp @@ -23,7 +23,6 @@ limitations under the License. #include "oneflow/core/control/global_process_ctx.h" #include "oneflow/core/eager/eager_blob_object.h" #include "oneflow/core/framework/instructions_builder.h" -#include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/framework/nd_sbp.h" #include "oneflow/core/framework/tensor_name_scope.h" #include "oneflow/core/functional/functional.h" @@ -79,7 +78,7 @@ Maybe NNGraph::Close() { VLOG(2) << "Try to close c nn graph name " << name_ << "." << std::endl; CloseRuntimeBuffers(); runtime_.reset(); - Global::Get()->RemoveGraphFreeEagerTensors(name_); + ctx_->RemoveGraphFreeEagerTensors(name_); is_closed_ = true; VLOG(2) << "Finish close c nn graph name " << name_ << "." << std::endl; } @@ -176,8 +175,7 @@ Maybe NNGraph::RegisterVariableOpNamesAndTensors( Maybe NNGraph::RegisterFreeEagerTensorsToVariableOpNames() { JUST(vm::CurrentRankSync()); - const auto& free_eager_tensors = - Global::Get()->GetFreeEagerTensorNamePairByGraphName(name_); + const auto& free_eager_tensors = ctx_->GetFreeEagerTensorNamePairByGraphName(name_); for (const auto& pair : free_eager_tensors) { const std::string& var_name = pair.first; const std::shared_ptr& var = pair.second; @@ -358,8 +356,7 @@ Maybe NNGraph::GetVariableRealBlobAfterSyncPlan() { *JUST(MapAt(&variable_op_name2tensor_, var_name)) = tensor; // NOTE(chengcheng): Just for tensor lifetime hold by session context in graph lifetime // valid. - Global::Get()->StoreFreeEagerTensorWithNameByGraphName( - name_, tensor, var_name); + ctx_->StoreFreeEagerTensorWithNameByGraphName(name_, tensor, var_name); const std::shared_ptr local_var = JUST(tensor->cur_rank_phy_tensor()); var_blob = JUST(local_var->eager_blob_object())->mut_blob(); diff --git a/oneflow/core/framework/nn_graph.h b/oneflow/core/framework/nn_graph.h index c745b7212e0..27c7ad2ce3c 100644 --- a/oneflow/core/framework/nn_graph.h +++ b/oneflow/core/framework/nn_graph.h @@ -16,9 +16,12 @@ limitations under the License. #ifndef ONEFLOW_CORE_FRAMEWORK_NN_GRAPH_H_ #define ONEFLOW_CORE_FRAMEWORK_NN_GRAPH_H_ +#include +#include "oneflow/core/common/util.h" #include "oneflow/core/framework/nn_graph_if.h" #include "oneflow/core/framework/tensor.h" #include "oneflow/core/framework/tensor_tuple.h" +#include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/job/job.pb.h" #include "oneflow/core/job/plan.pb.h" #include "oneflow/core/job/runtime.h" @@ -31,6 +34,10 @@ class NNGraph final : public NNGraphIf { public: explicit NNGraph(const std::string& name) : name_(name), runtime_inited_(false), is_closed_(false) {} + explicit NNGraph(const std::string& name, + const std::shared_ptr& sessioin_ctx) + : name_(name), ctx_(sessioin_ctx), runtime_inited_(false), is_closed_(false) {} + OF_DISALLOW_COPY_AND_MOVE(NNGraph); ~NNGraph(); const std::string& job_name() const override { return name_; } @@ -68,6 +75,7 @@ class NNGraph final : public NNGraphIf { void CloseRuntimeBuffers(); std::string name_; + std::shared_ptr ctx_; std::vector inputs_op_names_; std::vector outputs_op_names_; std::vector input_tensors_valid_; diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index b74ee7eecaa..aee5043e7f0 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -377,6 +377,7 @@ def str2int(env_config): cpp_logging_conf.logbuflevel = os.getenv("GLOG_logbuflevel") env_proto.cpp_logging_conf.CopyFrom(cpp_logging_conf) + class EnvHolder(object): def __init__(self): self._is_normal_exit = True @@ -384,7 +385,7 @@ def __init__(self): SetDefaultMultiClientEnvVars() oneflow._oneflow_internal.SetIsMultiClient(True) api_env_init() - + def set_is_normal_exit(self, is_normal_exit): self._is_normal_exit = is_normal_exit diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index a63b4304e74..dc76a0e5dce 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -31,11 +31,12 @@ class Status(enum.Enum): CLOSED = 3 def __init__(self, sess_id): + self._id = sess_id self._env_holder = env_util.GetEnvHolder() assert self._env_holder is not None - self.sess_ = oneflow._oneflow_internal.RegsiterSession(sess_id) - self._id = sess_id - oneflow._oneflow_internal.CreateMultiClientSessionContext() + self._internal_sess = oneflow._oneflow_internal.RegsiterSession(sess_id) + # New a MultiClientSessionContext + self._graph_ctx = oneflow._oneflow_internal.SessionContext() self.config_proto_ = self._make_config_proto() self.function_flag_name2default_val_ = {} self._update_function_flag_name2defaultVal() @@ -47,19 +48,16 @@ def TryInit(self): self._check_status(self.Status.CREATED, self.Status.INITED) if self.status_ == self.Status.CREATED: config_proto_str = text_format.MessageToString(self.config_proto) - oneflow._oneflow_internal.InitMultiClientSessionContext(config_proto_str) + # oneflow._oneflow_internal.InitMultiClientSessionContext(config_proto_str) + self._graph_ctx.try_init(config_proto_str) self.status_ = self.Status.INITED def _TryClose(self): if self.status_ != self.Status.CLOSED: - oneflow._oneflow_internal.TryDestroyMultiClientSessionContext() + # oneflow._oneflow_internal.TryDestroyMultiClientSessionContext() oneflow._oneflow_internal.ClearSessionById(self.id) self.status_ = self.Status.CLOSED - def AddCGraph(self, graph): - self._check_status(self.Status.INITED) - oneflow._oneflow_internal.MultiClientSessionContextAddCGraph(graph) - @property def status(self): return self.status_ @@ -124,11 +122,11 @@ def _update_scope_attr_name2defaultVal(self): def update_resource_eagerly(self, resource_config): self._check_status(self.Status.INITED) config_proto_str = text_format.MessageToString(resource_config) - oneflow._oneflow_internal.MultiClientSessionContextUpdateResource( - config_proto_str - ) + # oneflow._oneflow_internal.MultiClientSessionContextUpdateResource( + # config_proto_str + # ) + self._graph_ctx.update_resource(config_proto_str) def __del__(self): self._TryClose() print("oneflow session del") - diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index ee46bbb2ee5..282de9ab9bd 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -130,11 +130,12 @@ def __init__(self): self._outputs_buffer_size = 2 self._cur_index_of_ouputs_buffer = 0 - self._c_nn_graph = oneflow._oneflow_internal.nn.graph.CNNGraph(self._name) self._session = session_ctx.GetDefaultSession() assert type(self._session) is MultiClientSession self._session.TryInit() - self._session.AddCGraph(self._c_nn_graph) + self._c_nn_graph = oneflow._oneflow_internal.nn.graph.CNNGraph( + self._name, self._session._graph_ctx + ) def build(self, *args, **kwargs): r"""The ``build()`` method must be overridden to define neural network From b8f7f89afcec3723b3980ed8d16e34d0e3a4b5e5 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Sun, 6 Mar 2022 23:32:59 +0800 Subject: [PATCH 09/54] run barrier callback in BarrierPhyInstrOperand::~BarrierPhyInstrOperand --- .../core/framework/instructions_builder.cpp | 6 ++--- ..._operand.h => barrier_phy_instr_operand.h} | 18 +++++++------ oneflow/core/vm/instruction.h | 4 ++- .../core/vm/sequential_instruction_type.cpp | 15 +++-------- oneflow/core/vm/stream.cpp | 2 +- oneflow/core/vm/virtual_machine.cpp | 4 +-- oneflow/core/vm/virtual_machine_engine.cpp | 25 +++++++++++++++---- oneflow/core/vm/virtual_machine_engine.h | 3 ++- 8 files changed, 45 insertions(+), 32 deletions(-) rename oneflow/core/vm/{no_arg_cb_phy_instr_operand.h => barrier_phy_instr_operand.h} (72%) diff --git a/oneflow/core/framework/instructions_builder.cpp b/oneflow/core/framework/instructions_builder.cpp index f84239d047e..c9d15624d97 100644 --- a/oneflow/core/framework/instructions_builder.cpp +++ b/oneflow/core/framework/instructions_builder.cpp @@ -33,7 +33,7 @@ limitations under the License. #include "oneflow/core/common/decorator.h" #include "oneflow/core/common/blocking_counter.h" #include "oneflow/core/rpc/include/global_process_ctx.h" -#include "oneflow/core/vm/no_arg_cb_phy_instr_operand.h" +#include "oneflow/core/vm/barrier_phy_instr_operand.h" #include "oneflow/core/vm/access_blob_arg_cb_phy_instr_operand.h" #include "oneflow/core/vm/consume_local_dep_object_phy_instr_operand.h" #include "oneflow/core/eager/release_tensor_arg_phy_instr_operand.h" @@ -576,7 +576,7 @@ template Maybe InstructionsBuilder::AccessBlobByCallback( Maybe InstructionsBuilder::ComputeRankFrontSeqCallback( const std::function& callback) { - const auto& phy_instr_operand = std::make_shared(callback); + const auto& phy_instr_operand = std::make_shared(callback); auto instruction = intrusive::make_shared( Global::Get()->mut_vm(), "ComputeRankFrontSeqCallback", std::shared_ptr(), phy_instr_operand); @@ -585,7 +585,7 @@ Maybe InstructionsBuilder::ComputeRankFrontSeqCallback( } Maybe InstructionsBuilder::ComputeGlobalFrontSeqBarrier() { - const auto& phy_instr_operand = std::make_shared([] {}); + const auto& phy_instr_operand = std::make_shared([] {}); auto instruction = intrusive::make_shared( Global::Get()->mut_vm(), "ComputeGlobalFrontSeqBarrier", std::shared_ptr(), phy_instr_operand); diff --git a/oneflow/core/vm/no_arg_cb_phy_instr_operand.h b/oneflow/core/vm/barrier_phy_instr_operand.h similarity index 72% rename from oneflow/core/vm/no_arg_cb_phy_instr_operand.h rename to oneflow/core/vm/barrier_phy_instr_operand.h index 43428eb7527..cef0fb24beb 100644 --- a/oneflow/core/vm/no_arg_cb_phy_instr_operand.h +++ b/oneflow/core/vm/barrier_phy_instr_operand.h @@ -13,8 +13,8 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -#ifndef ONEFLOW_CORE_VM_NO_ARG_CB_PHY_INSTR_OPERAND_H_ -#define ONEFLOW_CORE_VM_NO_ARG_CB_PHY_INSTR_OPERAND_H_ +#ifndef ONEFLOW_CORE_VM_BARRIER_PHY_INSTR_OPERAND_H_ +#define ONEFLOW_CORE_VM_BARRIER_PHY_INSTR_OPERAND_H_ #include #include "oneflow/core/vm/phy_instr_operand.h" @@ -23,14 +23,16 @@ namespace oneflow { namespace vm { // no arg callback physical instruction operand -class NoArgCbPhyInstrOperand : public PhyInstrOperand { +class BarrierPhyInstrOperand : public PhyInstrOperand { public: - NoArgCbPhyInstrOperand(const std::function& callback) : callback_(callback) { + BarrierPhyInstrOperand(const std::function& callback) : callback_(callback) { stream_sequential_dependence_ = nullptr; } - ~NoArgCbPhyInstrOperand() = default; - - const std::function& callback() const { return callback_; } + ~BarrierPhyInstrOperand() { + // Make sure barrier callbacks run after all objects of previous instructions are destructed in + // Callback thread. + callback_(); + } const DependenceVector& input_dependences() const override { static DependenceVector dependences{}; @@ -48,4 +50,4 @@ class NoArgCbPhyInstrOperand : public PhyInstrOperand { } // namespace vm } // namespace oneflow -#endif // ONEFLOW_CORE_VM_NO_ARG_CB_PHY_INSTR_OPERAND_H_ +#endif // ONEFLOW_CORE_VM_BARRIER_PHY_INSTR_OPERAND_H_ diff --git a/oneflow/core/vm/instruction.h b/oneflow/core/vm/instruction.h index d385a4814d3..303dc6c5ad9 100644 --- a/oneflow/core/vm/instruction.h +++ b/oneflow/core/vm/instruction.h @@ -62,6 +62,8 @@ class InstructionMsg final : public intrusive::Base { intrusive::shared_ptr Clone() const; + intrusive::Ref::RefCntType ref_cnt() const { return intrusive_ref_.ref_cnt(); } + private: friend class intrusive::Ref; intrusive::Ref* mut_intrusive_ref() { return &intrusive_ref_; } @@ -184,7 +186,7 @@ class Instruction final : public intrusive::Base { Stream* mut_stream() { return stream_; } InstructionMsg* mut_instr_msg() { if (unlikely(!instr_msg_)) { instr_msg_ = intrusive::make_shared(); } - return instr_msg_.Mutable(); + return CHECK_NOTNULL(instr_msg_.Mutable()); } void reset_instr_msg(InstructionMsg* instr_msg) { instr_msg_.Reset(instr_msg); } void clear_instr_msg() { instr_msg_.Reset(); } diff --git a/oneflow/core/vm/sequential_instruction_type.cpp b/oneflow/core/vm/sequential_instruction_type.cpp index 9df14e44f46..3f5216355b8 100644 --- a/oneflow/core/vm/sequential_instruction_type.cpp +++ b/oneflow/core/vm/sequential_instruction_type.cpp @@ -20,7 +20,7 @@ limitations under the License. #include "oneflow/core/vm/instruction_type.h" #include "oneflow/core/vm/instruction.h" #include "oneflow/core/vm/virtual_machine_engine.h" -#include "oneflow/core/vm/no_arg_cb_phy_instr_operand.h" +#include "oneflow/core/vm/barrier_phy_instr_operand.h" #include "oneflow/core/control/global_process_ctx.h" namespace oneflow { @@ -34,13 +34,6 @@ class RankFrontSeqCallbackInstructionType : public InstructionType { bool IsFrontSequential() const override { return true; } protected: - void Run(const InstructionMsg& instr_msg) const { - const auto& phy_instr_operand = instr_msg.phy_instr_operand(); - CHECK(static_cast(phy_instr_operand)); - const auto* ptr = dynamic_cast(phy_instr_operand.get()); - CHECK_NOTNULL(ptr); - ptr->callback()(); - } }; class ComputeRankFrontSeqCallbackInstructionType final @@ -51,8 +44,8 @@ class ComputeRankFrontSeqCallbackInstructionType final using stream_type = ControlStreamType; - void Compute(Instruction* instruction) const override { Run(instruction->instr_msg()); } - void ComputeInFuseMode(InstructionMsg* instr_msg) const override { Run(*instr_msg); } + void Compute(Instruction* instruction) const override {} + void ComputeInFuseMode(InstructionMsg* instr_msg) const override {} }; COMMAND(RegisterInstructionType( "ComputeRankFrontSeqCallback")); @@ -65,7 +58,7 @@ class CtrlComputeRankFrontSeqCallbackInstructionType final using stream_type = ControlStreamType; - void Compute(Instruction* instruction) const override { Run(instruction->instr_msg()); } + void Compute(Instruction* instruction) const override {} }; COMMAND(RegisterInstructionType( "CtrlComputeRankFrontSeqCallback")); diff --git a/oneflow/core/vm/stream.cpp b/oneflow/core/vm/stream.cpp index ec8d493e57d..50f3ea09262 100644 --- a/oneflow/core/vm/stream.cpp +++ b/oneflow/core/vm/stream.cpp @@ -58,7 +58,6 @@ void Stream::MoveToFreeList(intrusive::shared_ptr&& instruction) { CHECK_EQ(instruction->ref_cnt(), 1); auto* instruction_ptr = instruction.Mutable(); mut_free_instruction_list()->EmplaceBack(std::move(instruction)); - instruction_ptr->Delete(); } void Stream::MoveFromZombieListToFreeList() { @@ -85,6 +84,7 @@ void Stream::DeleteInstruction(intrusive::shared_ptr&& instruction) CHECK(instruction->instruction_hook().empty()); CHECK(instruction->pending_instruction_hook().empty()); CHECK(instruction->dispatched_instruction_hook().empty()); + instruction->Delete(); // the value of instruction->ref_cnt() may be updated by a worker thread size_t ref_cnt = instruction->ref_cnt(); if (ref_cnt == 1) { diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index 75d1e1fc989..69d8fe5b5a1 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -17,7 +17,7 @@ limitations under the License. #include "oneflow/core/vm/virtual_machine.h" #include "oneflow/core/vm/instruction.h" #include "oneflow/core/vm/instruction_type.h" -#include "oneflow/core/vm/no_arg_cb_phy_instr_operand.h" +#include "oneflow/core/vm/barrier_phy_instr_operand.h" #include "oneflow/core/vm/vm_util.h" #include "oneflow/core/common/blocking_counter.h" #include "oneflow/core/common/multi_client.h" @@ -142,7 +142,7 @@ namespace { void MakeCtrlSeqInstructions(vm::VirtualMachineEngine* vm, vm::InstructionMsgList* list, const std::function& ComputeCallback) { - const auto& phy_instr_operand = std::make_shared(ComputeCallback); + const auto& phy_instr_operand = std::make_shared(ComputeCallback); auto instruction = intrusive::make_shared( vm, "CtrlComputeRankFrontSeqCallback", std::shared_ptr(), phy_instr_operand); diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 0444d815900..d2d5b2a3b47 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -26,6 +26,7 @@ limitations under the License. #include "oneflow/core/profiler/profiler.h" #include "oneflow/core/common/cpp_attribute.h" #include "oneflow/core/common/global.h" +#include namespace oneflow { namespace vm { @@ -189,19 +190,21 @@ void VirtualMachineEngine::ReleaseFinishedInstructions() { // in stream->DeleteInstruction(...) intrusive::shared_ptr instr_msg(instruction_ptr->mut_instr_msg()); stream->DeleteInstruction(LivelyInstructionListErase(instruction_ptr)); - MoveInstructionMsgToGarbageMsgList(std::move(instr_msg)); + static constexpr int kFlushWindowSize = 32; + MoveInstructionMsgToGarbageMsgList(kFlushWindowSize, std::move(instr_msg)); } if (stream->running_instruction_list().empty()) { mut_active_stream_list()->Erase(stream); } } } void VirtualMachineEngine::MoveInstructionMsgToGarbageMsgList( - intrusive::shared_ptr&& instr_msg) { + int flush_window_size, intrusive::shared_ptr&& instr_msg) { local_garbage_msg_list_.EmplaceBack(std::move(instr_msg)); - static constexpr int kWindowSize = 32; // local_garbage_msg_list_ is the cache of garbage_msg_list_. // `kWindowSize` controls the frequency of the usage of mutexed list. - if (unlikely(local_garbage_msg_list_.size() > kWindowSize)) { MoveToGarbageMsgListAndNotifyGC(); } + if (unlikely(local_garbage_msg_list_.size() > flush_window_size)) { + MoveToGarbageMsgListAndNotifyGC(); + } } void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC() { @@ -495,7 +498,11 @@ void VirtualMachineEngine::TryRunBarrierInstruction() { CHECK(OnSchedulerThread(stream_type)); stream_type.Run(sequnential_instruction); mut_barrier_instruction_list()->Erase(sequnential_instruction); + intrusive::shared_ptr instr_msg(sequnential_instruction->mut_instr_msg()); LivelyInstructionListErase(sequnential_instruction); + sequnential_instruction->clear_instr_msg(); + constexpr int kZeroWindowSize = 0; // flush immediately. + MoveInstructionMsgToGarbageMsgList(kZeroWindowSize, std::move(instr_msg)); OF_PROFILER_RANGE_POP(); } @@ -534,7 +541,15 @@ void VirtualMachineEngine::Schedule() { void VirtualMachineEngine::Callback() { InstructionMsgList garbage_msg_list; mut_garbage_msg_list()->MoveTo(&garbage_msg_list); - // destruct garbage_msg_list. + INTRUSIVE_FOR_EACH(garbage, &garbage_msg_list) { + garbage_msg_list.Erase(garbage.Mutable()); + while (garbage->ref_cnt() > 1) { + // Do nothing. wait until all other threads ref_cnts released. + } + CHECK_NOTNULL(garbage->phy_instr_operand()); + CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); + // Destruct garbage. + } } void VirtualMachineEngine::NotifyCallback() { MoveToGarbageMsgListAndNotifyGC(); } diff --git a/oneflow/core/vm/virtual_machine_engine.h b/oneflow/core/vm/virtual_machine_engine.h index 41c0f343c16..8a1aefbf01e 100644 --- a/oneflow/core/vm/virtual_machine_engine.h +++ b/oneflow/core/vm/virtual_machine_engine.h @@ -129,7 +129,8 @@ class VirtualMachineEngine final : public intrusive::Base { ReadyInstructionList* mut_ready_instruction_list() { return &ready_instruction_list_; } void ReleaseFinishedInstructions(); - void MoveInstructionMsgToGarbageMsgList(intrusive::shared_ptr&& instr_msg); + void MoveInstructionMsgToGarbageMsgList(int flush_window_size, + intrusive::shared_ptr&& instr_msg); void MoveToGarbageMsgListAndNotifyGC(); void HandleLocalPending(); void GetRewritedPendingInstructionsByWindowSize(size_t window_size, From 5f1324fed6e498de1605f37e20d5c12cfada8eb0 Mon Sep 17 00:00:00 2001 From: strint Date: Mon, 7 Mar 2022 00:09:06 +0800 Subject: [PATCH 10/54] test pass --- oneflow/api/python/env/env.cpp | 12 ++++-- oneflow/api/python/env/env.h | 2 + oneflow/api/python/env/env_api.h | 2 - oneflow/api/python/session/session.cpp | 9 +---- oneflow/api/python/session/session.h | 38 ------------------- oneflow/api/python/session/session_api.h | 16 -------- .../multi_client_session_context.cpp | 13 +++++++ .../framework/multi_client_session_context.h | 17 +++------ oneflow/core/framework/nn_graph.cpp | 6 +-- oneflow/core/framework/nn_graph.h | 4 +- oneflow/core/job/env_global_objects_scope.cpp | 23 ++++++++--- oneflow/core/job/env_global_objects_scope.h | 3 +- python/oneflow/__init__.py | 7 ++-- python/oneflow/framework/c_api_util.py | 6 ++- python/oneflow/framework/env_util.py | 16 ++++---- .../oneflow/framework/multi_client_session.py | 12 +++--- python/oneflow/framework/session_context.py | 4 +- python/oneflow/nn/graph/graph.py | 2 +- 18 files changed, 82 insertions(+), 110 deletions(-) diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 9a0cadd541e..d1aff9b806c 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -16,6 +16,7 @@ limitations under the License. #include #include "oneflow/api/python/of_api_registry.h" #include "oneflow/api/python/env/env_api.h" +#include "oneflow/core/job/env_global_objects_scope.h" namespace py = pybind11; @@ -24,10 +25,15 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("EnvResource", &EnvResource); m.def("EnableEagerEnvironment", &EnableEagerEnvironment); - m.def("IsEnvInited", &IsEnvInited); - m.def("InitEnv", &InitEnv); - m.def("DestroyEnv", &DestroyEnv, py::call_guard()); + py::class_>( + m, "EnvContext") + .def(py::init<>()) + .def("init", + [](oneflow::EnvGlobalObjectsScope& env, const std::string& env_proto_str) { + return env.Init(env_proto_str).GetOrThrow(); + }); + m.def("IsEnvInited", &IsEnvInited); m.def("CurrentMachineId", &CurrentMachineId); m.def("GetRank", &GetRank); diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index d78a8e7c45c..8ca23952af2 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -48,6 +48,7 @@ inline Maybe EnableEagerEnvironment(bool enable_eager_execution) { inline Maybe IsEnvInited() { return Global::Get() != nullptr; } +// TODO(strint): rm inline Maybe DestroyEnv() { if (Global::Get() == nullptr) { return Maybe::Ok(); } if (JUST(IsMultiClient())) { @@ -59,6 +60,7 @@ inline Maybe DestroyEnv() { return Maybe::Ok(); } +// TODO(strint): rm inline Maybe InitEnv(const std::string& env_proto_str, bool is_multi_client) { EnvProto env_proto; CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 9611d24a19a..b6e1dc7fe01 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -32,8 +32,6 @@ inline void InitEnv(const std::string& env_proto_str, bool is_multi_client) { return oneflow::InitEnv(env_proto_str, is_multi_client).GetOrThrow(); } -inline void DestroyEnv() { return oneflow::DestroyEnv().GetOrThrow(); } - inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } diff --git a/oneflow/api/python/session/session.cpp b/oneflow/api/python/session/session.cpp index bc96facc748..42ac68244c8 100644 --- a/oneflow/api/python/session/session.cpp +++ b/oneflow/api/python/session/session.cpp @@ -17,6 +17,7 @@ limitations under the License. #include #include "oneflow/api/python/of_api_registry.h" #include "oneflow/core/job/session.h" +#include "oneflow/core/job/env_global_objects_scope.h" #include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/api/python/session/session_api.h" @@ -31,16 +32,10 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("StartLazyGlobalSession", &StartLazyGlobalSession); m.def("StopLazyGlobalSession", &StopLazyGlobalSession); - // multi-client lazy global session context - m.def("CreateMultiClientSessionContext", &CreateMultiClientSessionContext); - m.def("InitMultiClientSessionContext", &InitMultiClientSessionContext); - m.def("MultiClientSessionContextUpdateResource", &MultiClientSessionContextUpdateResource); - m.def("TryDestroyMultiClientSessionContext", &TryDestroyMultiClientSessionContext); - using namespace oneflow; py::class_>( m, "SessionContext") - .def(py::init()) + .def(py::init&>()) .def("try_init", [](MultiClientSessionContext& session, const std::string& config_proto_str) { return session.TryInit(config_proto_str).GetOrThrow(); diff --git a/oneflow/api/python/session/session.h b/oneflow/api/python/session/session.h index f9794947321..1d6a502cceb 100644 --- a/oneflow/api/python/session/session.h +++ b/oneflow/api/python/session/session.h @@ -111,44 +111,6 @@ inline Maybe StopLazyGlobalSession() { return Maybe::Ok(); } -inline Maybe CreateMultiClientSessionContext() { - CHECK_ISNULL_OR_RETURN(Global::Get()); - Global::New(); - return Maybe::Ok(); -} - -inline Maybe InitMultiClientSessionContext(const std::string& config_proto_str) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - CHECK_NOTNULL_OR_RETURN(Global::Get()); - CHECK_NOTNULL_OR_RETURN(Global::Get()) << "env not found"; - - ConfigProto config_proto; - CHECK_OR_RETURN(TxtString2PbMessage(config_proto_str, &config_proto)) - << "failed to parse config_proto: " << config_proto_str; - JUST(Global::Get()->TryInit(config_proto)); - return Maybe::Ok(); -} - -inline Maybe MultiClientSessionContextUpdateResource(const std::string& resource_proto_str) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - Resource reso_proto; - CHECK_OR_RETURN(TxtString2PbMessage(resource_proto_str, &reso_proto)) - << "failed to parse config_proto: " << resource_proto_str; - JUST(Global::Get()->UpdateResource(reso_proto)); - return Maybe::Ok(); -} - -inline Maybe TryDestroyMultiClientSessionContext() { - // Global::Delete is not allowed to be called here - // because glog is not constructed yet and LOG(INFO) has bad bahavior - if (Global::Get() != nullptr) { - JUST(Global::Get()->TryClose()); - delete Global::Get(); - Global::SetAllocated(nullptr); - } - return Maybe::Ok(); -} - } // namespace oneflow #endif // ONEFLOW_API_PYTHON_SESSION_SESSION_H_ diff --git a/oneflow/api/python/session/session_api.h b/oneflow/api/python/session/session_api.h index 9f201b1869f..9abdf8e74f6 100644 --- a/oneflow/api/python/session/session_api.h +++ b/oneflow/api/python/session/session_api.h @@ -35,20 +35,4 @@ inline void StartLazyGlobalSession() { return oneflow::StartLazyGlobalSession(). inline void StopLazyGlobalSession() { return oneflow::StopLazyGlobalSession().GetOrThrow(); } -inline void CreateMultiClientSessionContext() { - return oneflow::CreateMultiClientSessionContext().GetOrThrow(); -} - -inline void InitMultiClientSessionContext(const std::string& config_proto_str) { - return oneflow::InitMultiClientSessionContext(config_proto_str).GetOrThrow(); -} - -inline void MultiClientSessionContextUpdateResource(const std::string& resource_proto_str) { - return oneflow::MultiClientSessionContextUpdateResource(resource_proto_str).GetOrThrow(); -} - -inline void TryDestroyMultiClientSessionContext() { - return oneflow::TryDestroyMultiClientSessionContext().GetOrThrow(); -} - #endif // ONEFLOW_API_PYTHON_SESSION_SESSION_API_H_ diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 18882c1b663..3e326729b51 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -61,6 +61,19 @@ int32_t GetCpuDeviceNum() { return std::thread::hardware_concurrency(); } } // namespace +MultiClientSessionContext::MultiClientSessionContext(const std::shared_ptr& env_ctx) { + env_ctx_ = env_ctx; + CHECK(Global::Get() == nullptr); + Global::SetAllocated(this); +} + +MultiClientSessionContext::~MultiClientSessionContext() { + CHECK_JUST(TryClose()); + if (Global::Get() != nullptr) { + Global::SetAllocated(nullptr); + } +} + Maybe MultiClientSessionContext::TryInit(const ConfigProto& config_proto) { if (!is_inited_) { CHECK_OR_RETURN(JUST(IsMultiClient())); diff --git a/oneflow/core/framework/multi_client_session_context.h b/oneflow/core/framework/multi_client_session_context.h index 9352c2caede..a8c25a9e116 100644 --- a/oneflow/core/framework/multi_client_session_context.h +++ b/oneflow/core/framework/multi_client_session_context.h @@ -16,26 +16,20 @@ limitations under the License. #ifndef ONEFLOW_CORE_FRAMEWORK_MULTI_CLIENT_SESSION_CONTEXT_H_ #define ONEFLOW_CORE_FRAMEWORK_MULTI_CLIENT_SESSION_CONTEXT_H_ +#include #include "oneflow/core/common/util.h" #include "oneflow/core/job/job_set.pb.h" #include "oneflow/core/common/maybe.h" #include "oneflow/core/framework/tensor.h" +#include "oneflow/core/job/env_global_objects_scope.h" namespace oneflow { class MultiClientSessionContext { public: OF_DISALLOW_COPY_AND_MOVE(MultiClientSessionContext); - MultiClientSessionContext() : is_inited_(false) { - CHECK(Global::Get() == nullptr); - Global::SetAllocated(this); - } - ~MultiClientSessionContext() { - CHECK_JUST(TryClose()); - if (Global::Get() != nullptr) { - Global::SetAllocated(nullptr); - } - } + explicit MultiClientSessionContext(const std::shared_ptr&); + ~MultiClientSessionContext(); Maybe TryInit(const ConfigProto& config_proto); Maybe TryInit(const std::string& config_proto_str); @@ -56,7 +50,8 @@ class MultiClientSessionContext { void RemoveGraphFreeEagerTensors(const std::string& graph_name); private: - bool is_inited_; + bool is_inited_ = false; + std::shared_ptr env_ctx_; HashMap>>> graph_name2free_eager_tensors_; }; diff --git a/oneflow/core/framework/nn_graph.cpp b/oneflow/core/framework/nn_graph.cpp index f87b66a2bc7..ba4dff32492 100644 --- a/oneflow/core/framework/nn_graph.cpp +++ b/oneflow/core/framework/nn_graph.cpp @@ -78,7 +78,7 @@ Maybe NNGraph::Close() { VLOG(2) << "Try to close c nn graph name " << name_ << "." << std::endl; CloseRuntimeBuffers(); runtime_.reset(); - ctx_->RemoveGraphFreeEagerTensors(name_); + session_ctx_->RemoveGraphFreeEagerTensors(name_); is_closed_ = true; VLOG(2) << "Finish close c nn graph name " << name_ << "." << std::endl; } @@ -175,7 +175,7 @@ Maybe NNGraph::RegisterVariableOpNamesAndTensors( Maybe NNGraph::RegisterFreeEagerTensorsToVariableOpNames() { JUST(vm::CurrentRankSync()); - const auto& free_eager_tensors = ctx_->GetFreeEagerTensorNamePairByGraphName(name_); + const auto& free_eager_tensors = session_ctx_->GetFreeEagerTensorNamePairByGraphName(name_); for (const auto& pair : free_eager_tensors) { const std::string& var_name = pair.first; const std::shared_ptr& var = pair.second; @@ -356,7 +356,7 @@ Maybe NNGraph::GetVariableRealBlobAfterSyncPlan() { *JUST(MapAt(&variable_op_name2tensor_, var_name)) = tensor; // NOTE(chengcheng): Just for tensor lifetime hold by session context in graph lifetime // valid. - ctx_->StoreFreeEagerTensorWithNameByGraphName(name_, tensor, var_name); + session_ctx_->StoreFreeEagerTensorWithNameByGraphName(name_, tensor, var_name); const std::shared_ptr local_var = JUST(tensor->cur_rank_phy_tensor()); var_blob = JUST(local_var->eager_blob_object())->mut_blob(); diff --git a/oneflow/core/framework/nn_graph.h b/oneflow/core/framework/nn_graph.h index 27c7ad2ce3c..2fed93a1a24 100644 --- a/oneflow/core/framework/nn_graph.h +++ b/oneflow/core/framework/nn_graph.h @@ -36,7 +36,7 @@ class NNGraph final : public NNGraphIf { : name_(name), runtime_inited_(false), is_closed_(false) {} explicit NNGraph(const std::string& name, const std::shared_ptr& sessioin_ctx) - : name_(name), ctx_(sessioin_ctx), runtime_inited_(false), is_closed_(false) {} + : name_(name), session_ctx_(sessioin_ctx), runtime_inited_(false), is_closed_(false) {} OF_DISALLOW_COPY_AND_MOVE(NNGraph); ~NNGraph(); @@ -75,7 +75,7 @@ class NNGraph final : public NNGraphIf { void CloseRuntimeBuffers(); std::string name_; - std::shared_ptr ctx_; + std::shared_ptr session_ctx_; std::vector inputs_op_names_; std::vector outputs_op_names_; std::vector input_tensors_valid_; diff --git a/oneflow/core/job/env_global_objects_scope.cpp b/oneflow/core/job/env_global_objects_scope.cpp index 8c680eaee4b..c708285efaf 100644 --- a/oneflow/core/job/env_global_objects_scope.cpp +++ b/oneflow/core/job/env_global_objects_scope.cpp @@ -138,6 +138,18 @@ bool CommNetIBEnabled() { } // namespace +EnvGlobalObjectsScope::EnvGlobalObjectsScope() { + CHECK(Global::Get() == nullptr); + Global::SetAllocated(this); +} + +Maybe EnvGlobalObjectsScope::Init(const std::string& env_proto_str) { + EnvProto env_proto; + CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) + << "failed to parse env_proto" << env_proto_str; + return Init(env_proto); +} + Maybe EnvGlobalObjectsScope::Init(const EnvProto& env_proto) { InitLogging(env_proto.cpp_logging_conf()); Global::New(env_proto); @@ -227,11 +239,8 @@ Maybe EnvGlobalObjectsScope::Init(const EnvProto& env_proto) { } EnvGlobalObjectsScope::~EnvGlobalObjectsScope() { - auto session_ctx = Global::Get(); - if (session_ctx != nullptr) { - VLOG(2) << "Multi client session has not closed , env close it at env scope destruction."; - CHECK_JUST(session_ctx->TryClose()); - } + VLOG(2) << "Try to close env global objects scope." << std::endl; + OF_ENV_BARRIER(); TensorBufferPool::Delete(); Global::Delete(); if (!Global::Get()->enable_dry_run()) { @@ -265,6 +274,10 @@ EnvGlobalObjectsScope::~EnvGlobalObjectsScope() { Global::Delete(); ClearAllSymbolAndIdCache(); google::ShutdownGoogleLogging(); + if (Global::Get() != nullptr) { + Global::SetAllocated(nullptr); + } + VLOG(2) << "Finish closing env global objects scope." << std::endl; } } // namespace oneflow diff --git a/oneflow/core/job/env_global_objects_scope.h b/oneflow/core/job/env_global_objects_scope.h index ceef4c52092..759c5320355 100644 --- a/oneflow/core/job/env_global_objects_scope.h +++ b/oneflow/core/job/env_global_objects_scope.h @@ -27,10 +27,11 @@ class ParallelDesc; class EnvGlobalObjectsScope final { public: OF_DISALLOW_COPY_AND_MOVE(EnvGlobalObjectsScope); - EnvGlobalObjectsScope() = default; + EnvGlobalObjectsScope(); ~EnvGlobalObjectsScope(); Maybe Init(const EnvProto& env_proto); + Maybe Init(const std::string& env_proto_str); }; } // namespace oneflow diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index ee119322fa7..fe566616b42 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -208,8 +208,9 @@ def is_deprecated(func_or_class): import oneflow.framework.session_context as session_ctx from oneflow.framework.tensor_str import set_printoptions -env_util.GetEnvHolder() -session_ctx.NewDefaultSession() +_env = env_util.GetEnv() +session_ctx.NewDefaultSession(_env) +del _env oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() @@ -261,7 +262,7 @@ def atexit_hook(hook): elif oneflow.env.get_rank() == 0: oneflow._oneflow_internal.eager.single_client.Sync() oneflow.framework.session_context.TryCloseDefaultSession() - env_util.DelEnvHolder(hook.is_normal_exit()) + env_util.DelEnv(hook.is_normal_exit()) atexit.register(atexit_hook, hook) diff --git a/python/oneflow/framework/c_api_util.py b/python/oneflow/framework/c_api_util.py index 82f0a993a7b..71ed9fff259 100644 --- a/python/oneflow/framework/c_api_util.py +++ b/python/oneflow/framework/c_api_util.py @@ -42,10 +42,12 @@ def EnvResource(): return text_format.Parse(resource, resource_util.Resource()) -def InitEnv(env_proto, is_multi_client): +def GetEnvContext(env_proto): assert type(env_proto) is env_pb2.EnvProto env_proto_str = text_format.MessageToString(env_proto) - oneflow._oneflow_internal.InitEnv(env_proto_str, is_multi_client) + env_ctx = oneflow._oneflow_internal.EnvContext() + env_ctx.init(env_proto_str) + return env_ctx def InitLazyGlobalSession(config_proto): diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index aee5043e7f0..e04edf3a1f3 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -51,7 +51,7 @@ def api_all_device_placement(device_type: str) -> oneflow._oneflow_internal.plac return oneflow._oneflow_internal.AllDevicePlacement(device_type) -def api_env_init() -> bool: +def api_env_init(): """Init environment Returns: @@ -86,8 +86,7 @@ def env_init(): CompleteEnvProto(default_env_proto, is_multi_client) if default_env_proto.ctrl_bootstrap_conf.world_size > 1: check_non_localhost_proxy_and_print_warning() - c_api_util.InitEnv(default_env_proto, is_multi_client) - return True + return c_api_util.GetEnvContext(default_env_proto) def api_machine(*val: list) -> None: @@ -384,19 +383,20 @@ def __init__(self): if not HasAllMultiClientEnvVars(): SetDefaultMultiClientEnvVars() oneflow._oneflow_internal.SetIsMultiClient(True) - api_env_init() + self._env_cxt = api_env_init() def set_is_normal_exit(self, is_normal_exit): self._is_normal_exit = is_normal_exit def __del__(self): - if self._is_normal_exit: - oneflow._oneflow_internal.DestroyEnv() + # TODO(strint): deal with abnormal exit + # if self._is_normal_exit: + # del self._env_cxt oneflow._oneflow_internal.SetShuttingDown() print("oneflow env del") -def GetEnvHolder(): +def GetEnv(): global _env_holder if _env_holder is not None: return _env_holder @@ -405,7 +405,7 @@ def GetEnvHolder(): return _env_holder -def DelEnvHolder(is_normal_exit): +def DelEnv(is_normal_exit): global _env_holder assert _env_holder is not None _env_holder.set_is_normal_exit(is_normal_exit) diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index dc76a0e5dce..dc2bf23c57d 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -30,13 +30,13 @@ class Status(enum.Enum): INITED = 2 CLOSED = 3 - def __init__(self, sess_id): + def __init__(self, env, sess_id): self._id = sess_id - self._env_holder = env_util.GetEnvHolder() - assert self._env_holder is not None + self._env= env + assert self._env is not None self._internal_sess = oneflow._oneflow_internal.RegsiterSession(sess_id) # New a MultiClientSessionContext - self._graph_ctx = oneflow._oneflow_internal.SessionContext() + self._session_ctx = oneflow._oneflow_internal.SessionContext(self._env._env_cxt) self.config_proto_ = self._make_config_proto() self.function_flag_name2default_val_ = {} self._update_function_flag_name2defaultVal() @@ -49,7 +49,7 @@ def TryInit(self): if self.status_ == self.Status.CREATED: config_proto_str = text_format.MessageToString(self.config_proto) # oneflow._oneflow_internal.InitMultiClientSessionContext(config_proto_str) - self._graph_ctx.try_init(config_proto_str) + self._session_ctx.try_init(config_proto_str) self.status_ = self.Status.INITED def _TryClose(self): @@ -125,7 +125,7 @@ def update_resource_eagerly(self, resource_config): # oneflow._oneflow_internal.MultiClientSessionContextUpdateResource( # config_proto_str # ) - self._graph_ctx.update_resource(config_proto_str) + self._session_ctx.update_resource(config_proto_str) def __del__(self): self._TryClose() diff --git a/python/oneflow/framework/session_context.py b/python/oneflow/framework/session_context.py index 0c52eb34b96..0d67cd29ea8 100644 --- a/python/oneflow/framework/session_context.py +++ b/python/oneflow/framework/session_context.py @@ -33,8 +33,8 @@ def GetDefaultSession(): return _sess_id2sess[default_sess_id] -def NewDefaultSession(): - new_default_sess = MultiClientSession(oneflow._oneflow_internal.NewSessionId()) +def NewDefaultSession(env): + new_default_sess = MultiClientSession(env, oneflow._oneflow_internal.NewSessionId()) global _sess_id2sess assert new_default_sess.id not in _sess_id2sess _sess_id2sess[new_default_sess.id] = new_default_sess diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 282de9ab9bd..644457d79bb 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -134,7 +134,7 @@ def __init__(self): assert type(self._session) is MultiClientSession self._session.TryInit() self._c_nn_graph = oneflow._oneflow_internal.nn.graph.CNNGraph( - self._name, self._session._graph_ctx + self._name, self._session._session_ctx ) def build(self, *args, **kwargs): From 9095397a09ca1b983ee456c572ebcdd45f474a53 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Mon, 7 Mar 2022 13:37:54 +0800 Subject: [PATCH 11/54] lock gil in vm Callback thread --- oneflow/core/vm/virtual_machine_engine.cpp | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index d2d5b2a3b47..655c80dacf1 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -26,6 +26,7 @@ limitations under the License. #include "oneflow/core/profiler/profiler.h" #include "oneflow/core/common/cpp_attribute.h" #include "oneflow/core/common/global.h" +#include "oneflow/core/common/foreign_lock_helper.h" #include namespace oneflow { @@ -542,13 +543,16 @@ void VirtualMachineEngine::Callback() { InstructionMsgList garbage_msg_list; mut_garbage_msg_list()->MoveTo(&garbage_msg_list); INTRUSIVE_FOR_EACH(garbage, &garbage_msg_list) { - garbage_msg_list.Erase(garbage.Mutable()); - while (garbage->ref_cnt() > 1) { - // Do nothing. wait until all other threads ref_cnts released. - } - CHECK_NOTNULL(garbage->phy_instr_operand()); - CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); - // Destruct garbage. + CHECK_JUST(Global::Get()->WithScopedAcquire([&, this]() -> Maybe { + garbage_msg_list.Erase(garbage.Mutable()); + while (garbage->ref_cnt() > 1) { + // Do nothing. wait until all other threads ref_cnts released. + } + CHECK_NOTNULL(garbage->phy_instr_operand()); + CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); + // Destruct garbage. + return Maybe::Ok(); + })); } } From 969ba8c2248d8a4a3bfa4208fb0f3c703afa7ec3 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Tue, 8 Mar 2022 10:41:12 +0800 Subject: [PATCH 12/54] more comments for VirtualMachineEngine::Callback() --- oneflow/core/vm/virtual_machine_engine.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 655c80dacf1..87e2de6697e 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -543,10 +543,24 @@ void VirtualMachineEngine::Callback() { InstructionMsgList garbage_msg_list; mut_garbage_msg_list()->MoveTo(&garbage_msg_list); INTRUSIVE_FOR_EACH(garbage, &garbage_msg_list) { - CHECK_JUST(Global::Get()->WithScopedAcquire([&, this]() -> Maybe { + CHECK_JUST(Global::Get()->WithScopedAcquire([&]() -> Maybe { garbage_msg_list.Erase(garbage.Mutable()); + // There may be a tiny gap between appending `garbage` to garbage_list and dereferencing + // `garbage` in scheduler thread or work thread. + // e.g. + // + // void Foo() { + // auto garbage = GetGarbage(); + // AppendToGarbageList(garbage); + // + // // **Callback thread maybe handle garbage in the same time**. From it's point view, + // ref_cnt > 1. + // + // garbage.reset(); // explicitly dereference garbage for better understood. + // } + // while (garbage->ref_cnt() > 1) { - // Do nothing. wait until all other threads ref_cnts released. + // Do nothing. Wait until all other threads ref_cnts released. } CHECK_NOTNULL(garbage->phy_instr_operand()); CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); From 3b852392dbbae0db574759159d215623bc8f9098 Mon Sep 17 00:00:00 2001 From: strint Date: Tue, 8 Mar 2022 13:01:59 +0800 Subject: [PATCH 13/54] merge --- oneflow/api/python/session/session.h | 48 ---------------------------- 1 file changed, 48 deletions(-) diff --git a/oneflow/api/python/session/session.h b/oneflow/api/python/session/session.h index 5ddbeabc7e5..952ff5a15c5 100644 --- a/oneflow/api/python/session/session.h +++ b/oneflow/api/python/session/session.h @@ -111,54 +111,6 @@ inline Maybe StopLazyGlobalSession() { return Maybe::Ok(); } -<<<<<<< HEAD -======= -inline Maybe CreateMultiClientSessionContext() { - CHECK_ISNULL_OR_RETURN(Global::Get()); - Global::New(); - return Maybe::Ok(); -} - -inline Maybe InitMultiClientSessionContext(const std::string& config_proto_str) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - CHECK_NOTNULL_OR_RETURN(Global::Get()); - CHECK_NOTNULL_OR_RETURN(Global::Get()) << "env not found"; - - ConfigProto config_proto; - CHECK_OR_RETURN(TxtString2PbMessage(config_proto_str, &config_proto)) - << "failed to parse config_proto: " << config_proto_str; - JUST(Global::Get()->TryInit(config_proto)); - return Maybe::Ok(); -} - -inline Maybe MultiClientSessionContextUpdateResource(const std::string& resource_proto_str) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - Resource reso_proto; - CHECK_OR_RETURN(TxtString2PbMessage(resource_proto_str, &reso_proto)) - << "failed to parse config_proto: " << resource_proto_str; - JUST(Global::Get()->UpdateResource(reso_proto)); - return Maybe::Ok(); -} - -inline Maybe MultiClientSessionContextAddCGraph( - const std::shared_ptr& c_graph_ptr) { - CHECK_NOTNULL_OR_RETURN(Global::Get()); - JUST(Global::Get()->AddCGraph(c_graph_ptr)); - return Maybe::Ok(); -} - -inline Maybe TryDestroyMultiClientSessionContext() { - // Global::Delete is not allowed to be called here - // because glog is not constructed yet and has bad bahavior - if (Global::Get() != nullptr) { - JUST(Global::Get()->TryClose()); - delete Global::Get(); - Global::SetAllocated(nullptr); - } - return Maybe::Ok(); -} - ->>>>>>> 17a18b90a7391c84b12e31901cf2dd845dba3f57 } // namespace oneflow #endif // ONEFLOW_API_PYTHON_SESSION_SESSION_H_ From 21f49bae85e1429fbc7a3c81bb5f7686699c61cd Mon Sep 17 00:00:00 2001 From: strint Date: Tue, 8 Mar 2022 18:23:50 +0800 Subject: [PATCH 14/54] merge rm single client --- python/oneflow/framework/env_util.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 8cd3f171300..f407fc02620 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -378,7 +378,6 @@ def __init__(self): self._is_normal_exit = True if not HasAllMultiClientEnvVars(): SetDefaultMultiClientEnvVars() - oneflow._oneflow_internal.SetIsMultiClient(True) self._env_cxt = api_env_init() def set_is_normal_exit(self, is_normal_exit): From d0e4c8fc212af566bd4b87397aca0ed34e817755 Mon Sep 17 00:00:00 2001 From: strint Date: Wed, 9 Mar 2022 17:08:54 +0800 Subject: [PATCH 15/54] rm initenv --- oneflow/api/python/env/env.h | 13 ------------- oneflow/api/python/env/env_api.h | 4 ---- 2 files changed, 17 deletions(-) diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index 6332ce3a3f0..d792ba0a6db 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -47,19 +47,6 @@ inline Maybe EnableEagerEnvironment(bool enable_eager_execution) { inline Maybe IsEnvInited() { return Global::Get() != nullptr; } -// TODO(strint): rm -inline Maybe InitEnv(const std::string& env_proto_str) { - EnvProto env_proto; - CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) - << "failed to parse env_proto" << env_proto_str; - CHECK_ISNULL_OR_RETURN(Global::Get()); - // Global::New is not allowed to be called here - // because glog is not constructed yet and has bad bahavior. - Global::SetAllocated(new EnvGlobalObjectsScope()); - JUST(Global::Get()->Init(env_proto)); - return Maybe::Ok(); -} - inline Maybe CurrentMachineId() { return GlobalProcessCtx::Rank(); } inline Maybe GetRank() { return GlobalProcessCtx::Rank(); } diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 1bab6e0f631..3aff5b84afc 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -28,10 +28,6 @@ inline void EnableEagerEnvironment(bool enable_eager_execution) { inline bool IsEnvInited() { return oneflow::IsEnvInited().GetOrThrow(); } -inline void InitEnv(const std::string& env_proto_str) { - return oneflow::InitEnv(env_proto_str).GetOrThrow(); -} - inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } From c1e664bdda45116836e0a02c4f72d1ef7ba4d9ef Mon Sep 17 00:00:00 2001 From: strint Date: Wed, 9 Mar 2022 21:25:28 +0800 Subject: [PATCH 16/54] merge and fix master --- python/oneflow/__init__.py | 1 - python/oneflow/framework/env_util.py | 5 ----- python/oneflow/framework/graph_build_util.py | 2 +- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index a905936b3dd..1e275f3b621 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -213,7 +213,6 @@ def is_deprecated(func_or_class): oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() -scope_util.InitScopeStack() oneflow._oneflow_internal.EnableEagerEnvironment(True) from oneflow.framework import python_callback, register_python_callback diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index fd7ae8d6bf8..f407fc02620 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -83,12 +83,7 @@ def env_init(): CompleteEnvProto(default_env_proto) if default_env_proto.ctrl_bootstrap_conf.world_size > 1: check_non_localhost_proxy_and_print_warning() -<<<<<<< HEAD return c_api_util.GetEnvContext(default_env_proto) -======= - c_api_util.InitEnv(default_env_proto) - return True ->>>>>>> a81140cecf79f240e55fdf95b6e21132d1257600 def api_machine(*val: list) -> None: diff --git a/python/oneflow/framework/graph_build_util.py b/python/oneflow/framework/graph_build_util.py index 4e18898655c..089fd993ed4 100644 --- a/python/oneflow/framework/graph_build_util.py +++ b/python/oneflow/framework/graph_build_util.py @@ -94,7 +94,7 @@ class GLogScopeContext(object): def __init__(self, s_level, v_level=0): self._prev_v = oneflow._oneflow_internal.GetFLAGS_v() self._prev_logtostderr = oneflow._oneflow_internal.GetFLAGS_alsologtostderr() - self._v = v_level + self._v = max(v_level, self._prev_v) self._s = s_level def __enter__(self): From 760d1310900f45bd38110a2c168ba39c2eca7575 Mon Sep 17 00:00:00 2001 From: strint Date: Thu, 10 Mar 2022 01:26:42 +0800 Subject: [PATCH 17/54] refactor env c api --- oneflow/api/cpp/env.cpp | 126 ++------------------------- oneflow/api/cpp/env_impl.cpp | 115 ++++++++++++++++++++++++ oneflow/api/cpp/env_impl.h | 24 +++++ oneflow/api/cpp/framework/graph.cpp | 6 +- oneflow/api/cpp/tests/graph_test.cpp | 8 +- oneflow/core/framework/nn_graph.h | 2 - 6 files changed, 153 insertions(+), 128 deletions(-) create mode 100644 oneflow/api/cpp/env_impl.cpp create mode 100644 oneflow/api/cpp/env_impl.h diff --git a/oneflow/api/cpp/env.cpp b/oneflow/api/cpp/env.cpp index 1c4d2f0513a..b7c52043117 100644 --- a/oneflow/api/cpp/env.cpp +++ b/oneflow/api/cpp/env.cpp @@ -14,136 +14,22 @@ See the License for the specific language governing permissions and limitations under the License. */ -#include -#include -#include -#include -#include -#include -#include -#include -#include #include "oneflow/api/cpp/env.h" -#include "oneflow/core/common/global.h" -#include "oneflow/core/common/just.h" -#include "oneflow/core/common/optional.h" -#include "oneflow/core/framework/multi_client_session_context.h" -#include "oneflow/core/framework/session_util.h" +#include "oneflow/api/cpp/env_impl.h" #include "oneflow/core/framework/shut_down_util.h" -#include "oneflow/core/job/cluster_instruction.h" -#include "oneflow/core/job/env.pb.h" -#include "oneflow/core/job/env_global_objects_scope.h" -#include "oneflow/core/control/ctrl_bootstrap.h" -#include "oneflow/core/job/session.h" -#include "oneflow/core/rpc/include/base.h" -#include "oneflow/core/vm/vm_util.h" #include "oneflow/core/thread/thread_consistent_id.h" namespace oneflow_api { - -namespace of = oneflow; - -namespace { // for inltialize - -inline bool IsEnvInited() { return of::Global::Get() != nullptr; } - -bool HasEnvVar(const std::string& key) { - const char* value = getenv(key.c_str()); - return value != nullptr; -} - -std::string GetEnvVar(const std::string& key, const std::string& default_value) { - const char* value = getenv(key.c_str()); - if (value == nullptr) { return default_value; } - return std::string(value); -} - -int64_t GetEnvVar(const std::string& key, int64_t default_value) { - const char* value = getenv(key.c_str()); - if (value == nullptr) { return default_value; } - return std::atoll(value); -} - -int32_t FindFreePort(const std::string& addr) { -#ifdef __linux__ - int sock = socket(AF_INET, SOCK_STREAM, IPPROTO_TCP); - CHECK_GE(sock, 0) << "fail to find a free port."; - int optval = 1; - setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, &optval, sizeof(optval)); - - std::mt19937 rng; - rng.seed(std::random_device()()); - std::uniform_int_distribution dist(1, 1000); - - int count = 0; - int num_attempts = 200; - do { - int port = 5000 + dist(rng); - struct sockaddr_in sockaddr {}; - memset(&sockaddr, 0, sizeof(sockaddr)); - sockaddr.sin_family = AF_INET; - sockaddr.sin_port = htons(port); - sockaddr.sin_addr.s_addr = inet_addr(addr.c_str()); - int error = bind(sock, (struct sockaddr*)&sockaddr, sizeof(sockaddr)); - if (error == 0) { return port; } - ++count; - } while (count < num_attempts); - CHECK_NE(count, num_attempts) << "fail to find a free port."; -#endif // __linux__ - return -1; -} - -void CompleteEnvProto(of::EnvProto& env_proto) { - auto bootstrap_conf = env_proto.mutable_ctrl_bootstrap_conf(); - auto master_addr = bootstrap_conf->mutable_master_addr(); - const std::string addr = GetEnvVar("MASTER_ADDR", "127.0.0.1"); - master_addr->set_host(addr); - master_addr->set_port(GetEnvVar("MASTER_PORT", FindFreePort(addr))); - bootstrap_conf->set_world_size(GetEnvVar("WORLD_SIZE", 1)); - bootstrap_conf->set_rank(GetEnvVar("RANK", 0)); - - auto cpp_logging_conf = env_proto.mutable_cpp_logging_conf(); - if (HasEnvVar("GLOG_log_dir")) { cpp_logging_conf->set_log_dir(GetEnvVar("GLOG_log_dir", "")); } - if (HasEnvVar("GLOG_logtostderr")) { - cpp_logging_conf->set_logtostderr(GetEnvVar("GLOG_logtostderr", -1)); - } - if (HasEnvVar("GLOG_logbuflevel")) { - cpp_logging_conf->set_logbuflevel(GetEnvVar("GLOG_logbuflevel", -1)); - } -} - -of::Maybe initEnv() { - of::EnvProto env_proto; - CompleteEnvProto(env_proto); - of::Global::SetAllocated(new of::EnvGlobalObjectsScope()); - JUST(of::Global::Get()->Init(env_proto)); - - of::ConfigProto config_proto; - config_proto.mutable_resource()->set_cpu_device_num(1); // useless, will be set in TryInit - const int64_t session_id = of::NewSessionId(); - JUST(of::RegsiterSession(session_id)); - config_proto.set_session_id(session_id); - of::Global::New(); - of::Global::Get()->TryInit(config_proto).GetOrThrow(); - return of::Maybe::Ok(); -} - -} // namespace - void initialize() { - if (!IsEnvInited()) { initEnv().GetOrThrow(); } + if (of::Global::Get() == nullptr) { + of::Global::New(); + } of::SetShuttingDown(false); } void release() { - if (IsEnvInited()) { - // sync multi_client - of::vm::ClusterSync().GetOrThrow(); - of::Global::Get()->TryClose().GetOrThrow(); - of::Global::Delete(); - // destory env - OF_ENV_BARRIER(); - of::Global::Delete(); + if (of::Global::Get() != nullptr) { + of::Global::Delete(); } of::SetShuttingDown(); of::ResetThisThreadUniqueConsistentId().GetOrThrow(); diff --git a/oneflow/api/cpp/env_impl.cpp b/oneflow/api/cpp/env_impl.cpp new file mode 100644 index 00000000000..7a1f23a4d00 --- /dev/null +++ b/oneflow/api/cpp/env_impl.cpp @@ -0,0 +1,115 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "oneflow/api/cpp/env_impl.h" +#include "oneflow/core/common/global.h" +#include "oneflow/core/common/just.h" +#include "oneflow/core/common/optional.h" +#include "oneflow/core/common/util.h" +#include "oneflow/core/framework/session_util.h" +#include "oneflow/core/job/env.pb.h" +#include "oneflow/core/job/cluster_instruction.h" +#include "oneflow/core/control/ctrl_bootstrap.h" +#include "oneflow/core/job/session.h" +#include "oneflow/core/rpc/include/base.h" +#include "oneflow/core/vm/vm_util.h" + +namespace oneflow_api { + +namespace of = oneflow; + +namespace { // for inltialize + +inline bool IsEnvInited() { return of::Global::Get() != nullptr; } + +bool HasEnvVar(const std::string& key) { + const char* value = getenv(key.c_str()); + return value != nullptr; +} + +std::string GetEnvVar(const std::string& key, const std::string& default_value) { + const char* value = getenv(key.c_str()); + if (value == nullptr) { return default_value; } + return std::string(value); +} + +int64_t GetEnvVar(const std::string& key, int64_t default_value) { + const char* value = getenv(key.c_str()); + if (value == nullptr) { return default_value; } + return std::atoll(value); +} + +int32_t FindFreePort(const std::string& addr) { +#ifdef __linux__ + int sock = socket(AF_INET, SOCK_STREAM, IPPROTO_TCP); + CHECK_GE(sock, 0) << "fail to find a free port."; + int optval = 1; + setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, &optval, sizeof(optval)); + + std::mt19937 rng; + rng.seed(std::random_device()()); + std::uniform_int_distribution dist(1, 1000); + + int count = 0; + int num_attempts = 200; + do { + int port = 5000 + dist(rng); + struct sockaddr_in sockaddr {}; + memset(&sockaddr, 0, sizeof(sockaddr)); + sockaddr.sin_family = AF_INET; + sockaddr.sin_port = htons(port); + sockaddr.sin_addr.s_addr = inet_addr(addr.c_str()); + int error = bind(sock, (struct sockaddr*)&sockaddr, sizeof(sockaddr)); + if (error == 0) { return port; } + ++count; + } while (count < num_attempts); + CHECK_NE(count, num_attempts) << "fail to find a free port."; +#endif // __linux__ + return -1; +} + +void CompleteEnvProto(of::EnvProto& env_proto) { + auto bootstrap_conf = env_proto.mutable_ctrl_bootstrap_conf(); + auto master_addr = bootstrap_conf->mutable_master_addr(); + const std::string addr = GetEnvVar("MASTER_ADDR", "127.0.0.1"); + master_addr->set_host(addr); + master_addr->set_port(GetEnvVar("MASTER_PORT", FindFreePort(addr))); + bootstrap_conf->set_world_size(GetEnvVar("WORLD_SIZE", 1)); + bootstrap_conf->set_rank(GetEnvVar("RANK", 0)); + + auto cpp_logging_conf = env_proto.mutable_cpp_logging_conf(); + if (HasEnvVar("GLOG_log_dir")) { cpp_logging_conf->set_log_dir(GetEnvVar("GLOG_log_dir", "")); } + if (HasEnvVar("GLOG_logtostderr")) { + cpp_logging_conf->set_logtostderr(GetEnvVar("GLOG_logtostderr", -1)); + } + if (HasEnvVar("GLOG_logbuflevel")) { + cpp_logging_conf->set_logbuflevel(GetEnvVar("GLOG_logbuflevel", -1)); + } +} +} // namespace + +OneFlowEnv::OneFlowEnv() { + of::EnvProto env_proto; + CompleteEnvProto(env_proto); + + env_ctx_ = std::make_shared(); + env_ctx_->Init(env_proto); + + of::ConfigProto config_proto; + config_proto.mutable_resource()->set_cpu_device_num(1); // useless, will be set in TryInit + const int64_t session_id = of::NewSessionId(); + CHECK_JUST(of::RegsiterSession(session_id)); + config_proto.set_session_id(session_id); + + session_ctx_ = std::make_shared(env_ctx_); + session_ctx_->TryInit(config_proto); +} + +} // namespace oneflow_api diff --git a/oneflow/api/cpp/env_impl.h b/oneflow/api/cpp/env_impl.h new file mode 100644 index 00000000000..493ef9b89ee --- /dev/null +++ b/oneflow/api/cpp/env_impl.h @@ -0,0 +1,24 @@ +#include +#include "oneflow/core/framework/multi_client_session_context.h" +#include "oneflow/core/job/env_global_objects_scope.h" + +#ifndef ONEFLOW_API_CPP_ENV_IMPL_H_ +#define ONEFLOW_API_CPP_ENV_IMPL_H_ + +namespace oneflow_api { +namespace of = oneflow; +class OneFlowEnv { + public: + OF_DISALLOW_COPY(OneFlowEnv); + OneFlowEnv(); + ~OneFlowEnv() = default; + std::shared_ptr GetSessionCtx() { + return session_ctx_; + } +private: + std::shared_ptr env_ctx_; + std::shared_ptr session_ctx_; +}; +} // namespace oneflow_api + +#endif // ONEFLOW_API_CPP_ENV_IMPL_H_ \ No newline at end of file diff --git a/oneflow/api/cpp/framework/graph.cpp b/oneflow/api/cpp/framework/graph.cpp index 014e7ca8d22..f2a65678596 100644 --- a/oneflow/api/cpp/framework/graph.cpp +++ b/oneflow/api/cpp/framework/graph.cpp @@ -15,6 +15,7 @@ limitations under the License. */ #include "oneflow/api/common/ofblob.h" +#include "oneflow/api/cpp/env_impl.h" #include "oneflow/api/cpp/framework/device.h" #include "oneflow/api/cpp/framework/graph.h" #include "oneflow/api/cpp/framework/ivalue.h" @@ -212,9 +213,8 @@ Graph::GraphImpl::GraphImpl(const std::string& model_path, const Device& device) } job_.mutable_job_conf()->mutable_predict_conf(); job_.mutable_job_conf()->set_job_name(job_.mutable_job_conf()->job_name() + of::NewUniqueId()); - graph_ = std::make_shared(job_.job_conf().job_name()); - // TODO(strint): rm - // of::Global::Get()->AddCGraph(graph_).GetOrThrow(); + CHECK(of::Global::Get() != nullptr); + graph_ = std::make_shared(job_.job_conf().job_name(), of::Global::Get()->GetSessionCtx()); } Graph::GraphImpl::GraphImpl(GraphImpl&& graph) noexcept diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index 497da6b1bbb..17ff2e653fe 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -65,9 +65,11 @@ TEST(Api, graph_cpu_test) { #ifdef WITH_CUDA TEST(Api, graph_gpu_test) { EnvScope scope; - Device device("cuda", 0); - Graph graph = LoadGraph(device); - Forward(graph, device); + { + Device device("cuda", 0); + Graph graph = LoadGraph(device); + Forward(graph, device); + } } TEST(Api, graph_multi_gpu_test) { diff --git a/oneflow/core/framework/nn_graph.h b/oneflow/core/framework/nn_graph.h index 2fed93a1a24..e49ef2dc7a5 100644 --- a/oneflow/core/framework/nn_graph.h +++ b/oneflow/core/framework/nn_graph.h @@ -32,8 +32,6 @@ class Blob; class NNGraph final : public NNGraphIf { public: - explicit NNGraph(const std::string& name) - : name_(name), runtime_inited_(false), is_closed_(false) {} explicit NNGraph(const std::string& name, const std::shared_ptr& sessioin_ctx) : name_(name), session_ctx_(sessioin_ctx), runtime_inited_(false), is_closed_(false) {} From 30d4cc0dd98e83d0318b4fcbdaddc34560e704d8 Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 11 Mar 2022 20:06:58 +0800 Subject: [PATCH 18/54] add debug code --- cmake/oneflow.cmake | 2 +- oneflow/api/cpp/env.cpp | 1 + oneflow/api/cpp/env_impl.cpp | 5 +++++ oneflow/api/cpp/env_impl.h | 2 +- oneflow/api/cpp/tests/graph_test.cpp | 7 ++++++- oneflow/core/framework/multi_client_session_context.cpp | 4 ++-- oneflow/core/framework/nn_graph.cpp | 4 +++- oneflow/core/job/env_global_objects_scope.cpp | 2 +- 8 files changed, 20 insertions(+), 7 deletions(-) diff --git a/cmake/oneflow.cmake b/cmake/oneflow.cmake index 274f2023c94..f5bb6f1b811 100644 --- a/cmake/oneflow.cmake +++ b/cmake/oneflow.cmake @@ -440,7 +440,7 @@ if(BUILD_TESTING) WORKING_DIRECTORY ${PROJECT_SOURCE_DIR}) find_package(Threads REQUIRED) - target_link_libraries(oneflow_cpp_api_testexe oneflow_cpp ${oneflow_test_libs} Threads::Threads) + target_link_libraries(oneflow_cpp_api_testexe oneflow_cpp ${oneflow_test_libs} glog::glog Threads::Threads) endif() endif() diff --git a/oneflow/api/cpp/env.cpp b/oneflow/api/cpp/env.cpp index b7c52043117..fa01bb4ceed 100644 --- a/oneflow/api/cpp/env.cpp +++ b/oneflow/api/cpp/env.cpp @@ -14,6 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. */ +#include #include "oneflow/api/cpp/env.h" #include "oneflow/api/cpp/env_impl.h" #include "oneflow/core/framework/shut_down_util.h" diff --git a/oneflow/api/cpp/env_impl.cpp b/oneflow/api/cpp/env_impl.cpp index 7a1f23a4d00..be098cabec4 100644 --- a/oneflow/api/cpp/env_impl.cpp +++ b/oneflow/api/cpp/env_impl.cpp @@ -112,4 +112,9 @@ OneFlowEnv::OneFlowEnv() { session_ctx_->TryInit(config_proto); } +OneFlowEnv::~OneFlowEnv() { + session_ctx_.reset(); + env_ctx_.reset(); +} + } // namespace oneflow_api diff --git a/oneflow/api/cpp/env_impl.h b/oneflow/api/cpp/env_impl.h index 493ef9b89ee..8a0016ff92e 100644 --- a/oneflow/api/cpp/env_impl.h +++ b/oneflow/api/cpp/env_impl.h @@ -11,7 +11,7 @@ class OneFlowEnv { public: OF_DISALLOW_COPY(OneFlowEnv); OneFlowEnv(); - ~OneFlowEnv() = default; + ~OneFlowEnv(); std::shared_ptr GetSessionCtx() { return session_ctx_; } diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index 17ff2e653fe..aebbc823b59 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -25,6 +25,8 @@ limitations under the License. #include #include "oneflow/api/cpp/framework.h" #include "oneflow/api/cpp/tests/api_test.h" +#include "oneflow/core/vm/vm_util.h" + namespace oneflow_api { @@ -32,7 +34,7 @@ namespace { inline Graph LoadGraph(const Device& device) { Graph graph = - Graph::Load("./oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); + Graph::Load("/home/xuxiaoyu/dev/oneflow/oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); return graph; } @@ -70,6 +72,9 @@ TEST(Api, graph_gpu_test) { Graph graph = LoadGraph(device); Forward(graph, device); } + std::cerr << "after graph" << std::endl; + oneflow::vm::ClusterSync(); + std::cerr << "after sync" << std::endl; } TEST(Api, graph_multi_gpu_test) { diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index a81a184798a..3a14c6dd5e9 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -155,7 +155,6 @@ Maybe MultiClientSessionContext::UpdateResource(const std::string& reso_pr Maybe MultiClientSessionContext::TryClose() { if (is_inited_) { VLOG(1) << "Try to delete multi client session context." << std::endl; - // sync before NNGraph release to ensure LaunchLazyJob instruction was completed and released JUST(vm::ClusterSync()); { @@ -182,9 +181,10 @@ Maybe MultiClientSessionContext::TryClose() { // NOTE(chengcheng): New after delete because in EnvGlobalObjectScope once created ResourceDesc. Global::New(Global::Get()->resource(), GlobalProcessCtx::NumOfProcessPerNode()); + VLOG(1) << "Finish delete multi client session context." << std::endl; + env_ctx_.reset(); is_inited_ = false; } - VLOG(1) << "Finish delete multi client session context." << std::endl; return Maybe::Ok(); } diff --git a/oneflow/core/framework/nn_graph.cpp b/oneflow/core/framework/nn_graph.cpp index fcdb3068f4c..a64286c344d 100644 --- a/oneflow/core/framework/nn_graph.cpp +++ b/oneflow/core/framework/nn_graph.cpp @@ -79,8 +79,10 @@ Maybe NNGraph::Close() { CloseRuntimeBuffers(); runtime_.reset(); session_ctx_->RemoveGraphFreeEagerTensors(name_); - is_closed_ = true; VLOG(1) << "Finish close c nn graph name " << name_ << "." << std::endl; + + session_ctx_.reset(); + is_closed_ = true; } return Maybe::Ok(); } diff --git a/oneflow/core/job/env_global_objects_scope.cpp b/oneflow/core/job/env_global_objects_scope.cpp index 9170b88c25e..93893c46497 100644 --- a/oneflow/core/job/env_global_objects_scope.cpp +++ b/oneflow/core/job/env_global_objects_scope.cpp @@ -273,11 +273,11 @@ EnvGlobalObjectsScope::~EnvGlobalObjectsScope() { Global::Delete(); Global::Delete(); ClearAllSymbolAndIdCache(); - google::ShutdownGoogleLogging(); if (Global::Get() != nullptr) { Global::SetAllocated(nullptr); } VLOG(2) << "Finish closing env global objects scope." << std::endl; + google::ShutdownGoogleLogging(); } } // namespace oneflow From 24f02eadd0d246ea1bfda9adb9aee5f14d3791e5 Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 11 Mar 2022 23:19:12 +0800 Subject: [PATCH 19/54] fix and serving test pass --- oneflow/api/cpp/framework/graph.cpp | 7 ++++++- oneflow/api/cpp/tests/graph_test.cpp | 3 --- oneflow/core/ep/cuda/cuda_stream.cpp | 5 ++++- .../oneflow/test/graph/test_graph_session_env_destruct.py | 2 +- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/oneflow/api/cpp/framework/graph.cpp b/oneflow/api/cpp/framework/graph.cpp index f2a65678596..4185d6b3487 100644 --- a/oneflow/api/cpp/framework/graph.cpp +++ b/oneflow/api/cpp/framework/graph.cpp @@ -54,6 +54,7 @@ limitations under the License. #include "oneflow/core/operator/interface_blob_conf.pb.h" #include "oneflow/core/operator/op_conf.pb.h" #include "oneflow/core/register/logical_blob_id.pb.h" +#include "oneflow/core/vm/vm_util.h" namespace oneflow_api { @@ -129,7 +130,7 @@ class Graph::GraphImpl final { GraphImpl(const GraphImpl& graph) = delete; GraphImpl(GraphImpl&& graph) noexcept; - ~GraphImpl() = default; + ~GraphImpl(); GraphImpl& operator=(const GraphImpl& graph) = delete; GraphImpl& operator=(GraphImpl&& graph) noexcept; @@ -394,4 +395,8 @@ of::Maybe Graph::GraphImpl::RegisterTensors(const std::vector& inp return of::Maybe::Ok(); } +Graph::GraphImpl::~GraphImpl() { + of::vm::ClusterSync().GetOrThrow(); +} + } // namespace oneflow_api diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index aebbc823b59..ed04e911bb6 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -72,9 +72,6 @@ TEST(Api, graph_gpu_test) { Graph graph = LoadGraph(device); Forward(graph, device); } - std::cerr << "after graph" << std::endl; - oneflow::vm::ClusterSync(); - std::cerr << "after sync" << std::endl; } TEST(Api, graph_multi_gpu_test) { diff --git a/oneflow/core/ep/cuda/cuda_stream.cpp b/oneflow/core/ep/cuda/cuda_stream.cpp index 2acc0a4af7b..61ae1322b44 100644 --- a/oneflow/core/ep/cuda/cuda_stream.cpp +++ b/oneflow/core/ep/cuda/cuda_stream.cpp @@ -149,7 +149,10 @@ Maybe CudaStream::Sync() { void CudaStream::RecordEvent(Event* event) { auto* cuda_event = static_cast(event); // NOLINT - OF_CUDA_CHECK(cudaEventRecord(cuda_event->cuda_event(), cuda_stream_)); + // OF_CUDA_CHECK(cudaEventRecord(cuda_event->cuda_event(), cuda_stream_)); + if (cudaEventRecord(cuda_event->cuda_event(), cuda_stream_) != cudaSuccess) { + std::cerr << "cuda even record error" << std::endl; + } } cudaStream_t CudaStream::cuda_stream() const { return cuda_stream_; } diff --git a/python/oneflow/test/graph/test_graph_session_env_destruct.py b/python/oneflow/test/graph/test_graph_session_env_destruct.py index fdc25d2e181..4fd90d927f9 100644 --- a/python/oneflow/test/graph/test_graph_session_env_destruct.py +++ b/python/oneflow/test/graph/test_graph_session_env_destruct.py @@ -51,4 +51,4 @@ def test_linear_graph_destruct(test_case): if __name__ == "__main__": - unittest.main() + unittest.main() \ No newline at end of file From 9d7617b3bbcd6785bc9c7290bc599e04f40516ac Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 19:25:10 +0800 Subject: [PATCH 20/54] test passed --- oneflow/api/cpp/env.cpp | 8 +- oneflow/api/cpp/env_impl.cpp | 15 + oneflow/api/cpp/env_impl.h | 24 +- oneflow/api/cpp/framework/graph.cpp | 7 +- oneflow/api/cpp/tests/graph_test.cpp | 14 +- oneflow/api/python/env/env.cpp | 7 +- .../multi_client_session_context.cpp | 9 +- oneflow/core/vm/virtual_machine.cpp | 7 +- python/oneflow/framework/env_util.py | 1 + .../oneflow/framework/multi_client_session.py | 2 +- python/oneflow/nn/graph/graph.py | 2 + .../graph/log_stuck/oneflow-23/oneflow.INFO | 1 + ....xuxiaoyu.log.INFO.20220312-181836.1174409 | 987 ++++++++++++++++++ .../graph/test_graph_session_env_destruct.py | 2 +- 14 files changed, 1052 insertions(+), 34 deletions(-) create mode 120000 python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO create mode 100644 python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 diff --git a/oneflow/api/cpp/env.cpp b/oneflow/api/cpp/env.cpp index fa01bb4ceed..0d3c673d09e 100644 --- a/oneflow/api/cpp/env.cpp +++ b/oneflow/api/cpp/env.cpp @@ -22,16 +22,12 @@ limitations under the License. namespace oneflow_api { void initialize() { - if (of::Global::Get() == nullptr) { - of::Global::New(); - } + if (of::Global::Get() == nullptr) { of::Global::New(); } of::SetShuttingDown(false); } void release() { - if (of::Global::Get() != nullptr) { - of::Global::Delete(); - } + if (of::Global::Get() != nullptr) { of::Global::Delete(); } of::SetShuttingDown(); of::ResetThisThreadUniqueConsistentId().GetOrThrow(); } diff --git a/oneflow/api/cpp/env_impl.cpp b/oneflow/api/cpp/env_impl.cpp index be098cabec4..60118e72fd9 100644 --- a/oneflow/api/cpp/env_impl.cpp +++ b/oneflow/api/cpp/env_impl.cpp @@ -1,3 +1,18 @@ +/* +Copyright 2020 The OneFlow Authors. All rights reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ #include #include #include diff --git a/oneflow/api/cpp/env_impl.h b/oneflow/api/cpp/env_impl.h index 8a0016ff92e..9e645f29276 100644 --- a/oneflow/api/cpp/env_impl.h +++ b/oneflow/api/cpp/env_impl.h @@ -1,3 +1,18 @@ +/* +Copyright 2020 The OneFlow Authors. All rights reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ #include #include "oneflow/core/framework/multi_client_session_context.h" #include "oneflow/core/job/env_global_objects_scope.h" @@ -8,14 +23,13 @@ namespace oneflow_api { namespace of = oneflow; class OneFlowEnv { - public: + public: OF_DISALLOW_COPY(OneFlowEnv); OneFlowEnv(); ~OneFlowEnv(); - std::shared_ptr GetSessionCtx() { - return session_ctx_; - } -private: + std::shared_ptr GetSessionCtx() { return session_ctx_; } + + private: std::shared_ptr env_ctx_; std::shared_ptr session_ctx_; }; diff --git a/oneflow/api/cpp/framework/graph.cpp b/oneflow/api/cpp/framework/graph.cpp index 4185d6b3487..a5f18404cf2 100644 --- a/oneflow/api/cpp/framework/graph.cpp +++ b/oneflow/api/cpp/framework/graph.cpp @@ -215,7 +215,8 @@ Graph::GraphImpl::GraphImpl(const std::string& model_path, const Device& device) job_.mutable_job_conf()->mutable_predict_conf(); job_.mutable_job_conf()->set_job_name(job_.mutable_job_conf()->job_name() + of::NewUniqueId()); CHECK(of::Global::Get() != nullptr); - graph_ = std::make_shared(job_.job_conf().job_name(), of::Global::Get()->GetSessionCtx()); + graph_ = std::make_shared(job_.job_conf().job_name(), + of::Global::Get()->GetSessionCtx()); } Graph::GraphImpl::GraphImpl(GraphImpl&& graph) noexcept @@ -395,8 +396,6 @@ of::Maybe Graph::GraphImpl::RegisterTensors(const std::vector& inp return of::Maybe::Ok(); } -Graph::GraphImpl::~GraphImpl() { - of::vm::ClusterSync().GetOrThrow(); -} +Graph::GraphImpl::~GraphImpl() { of::vm::ClusterSync().GetOrThrow(); } } // namespace oneflow_api diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index ed04e911bb6..8825707feca 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -27,14 +27,14 @@ limitations under the License. #include "oneflow/api/cpp/tests/api_test.h" #include "oneflow/core/vm/vm_util.h" - namespace oneflow_api { namespace { inline Graph LoadGraph(const Device& device) { - Graph graph = - Graph::Load("/home/xuxiaoyu/dev/oneflow/oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); + Graph graph = Graph::Load( + "/home/xuxiaoyu/dev/oneflow/oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", + device); return graph; } @@ -67,11 +67,9 @@ TEST(Api, graph_cpu_test) { #ifdef WITH_CUDA TEST(Api, graph_gpu_test) { EnvScope scope; - { - Device device("cuda", 0); - Graph graph = LoadGraph(device); - Forward(graph, device); - } + Device device("cuda", 0); + Graph graph = LoadGraph(device); + Forward(graph, device); } TEST(Api, graph_multi_gpu_test) { diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index e980443e3dc..9fc64cb0039 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -28,10 +28,9 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { py::class_>( m, "EnvContext") .def(py::init<>()) - .def("init", - [](oneflow::EnvGlobalObjectsScope& env, const std::string& env_proto_str) { - return env.Init(env_proto_str).GetOrThrow(); - }); + .def("init", [](oneflow::EnvGlobalObjectsScope& env, const std::string& env_proto_str) { + return env.Init(env_proto_str).GetOrThrow(); + }); m.def("IsEnvInited", &IsEnvInited); m.def("CurrentMachineId", &CurrentMachineId); diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 3a14c6dd5e9..cab52431256 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -60,14 +60,15 @@ int32_t GetCpuDeviceNum() { return std::thread::hardware_concurrency(); } } // namespace -MultiClientSessionContext::MultiClientSessionContext(const std::shared_ptr& env_ctx) { +MultiClientSessionContext::MultiClientSessionContext( + const std::shared_ptr& env_ctx) { env_ctx_ = env_ctx; CHECK(Global::Get() == nullptr); Global::SetAllocated(this); } -MultiClientSessionContext::~MultiClientSessionContext() { - CHECK_JUST(TryClose()); +MultiClientSessionContext::~MultiClientSessionContext() { + CHECK_JUST(TryClose()); if (Global::Get() != nullptr) { Global::SetAllocated(nullptr); } @@ -156,7 +157,7 @@ Maybe MultiClientSessionContext::TryClose() { if (is_inited_) { VLOG(1) << "Try to delete multi client session context." << std::endl; // sync before NNGraph release to ensure LaunchLazyJob instruction was completed and released - JUST(vm::ClusterSync()); + // JUST(vm::ClusterSync()); { // NOTE(chengcheng): delete runtime global objects Global::Delete(); diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index e7801a17e6b..dcc669a1238 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -154,7 +154,12 @@ void VirtualMachine::ControlSync() { } VirtualMachine::~VirtualMachine() { - ControlSync(); + // NOTE(xuxiaoyu): In some case ControlSync at env destructoin cause infinite waiting. + // So it's removed at the moment. + // Test case to reproduce the bug: + // oneflow/python/oneflow/test/graph/test_graph_session_env_destruct.py In this case, a graph + // destructed after oneflow init called atexit hook. + // ControlSync(); pending_notifier_.Close(); schedule_thread_.join(); CHECK(!vm_); diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index f407fc02620..38e8bb3ad71 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -387,6 +387,7 @@ def __del__(self): # TODO(strint): deal with abnormal exit # if self._is_normal_exit: # del self._env_cxt + oneflow._oneflow_internal.eager.Sync() oneflow._oneflow_internal.SetShuttingDown() print("oneflow env del") diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index dc2bf23c57d..f1a5c6b8729 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -32,7 +32,7 @@ class Status(enum.Enum): def __init__(self, env, sess_id): self._id = sess_id - self._env= env + self._env = env assert self._env is not None self._internal_sess = oneflow._oneflow_internal.RegsiterSession(sess_id) # New a MultiClientSessionContext diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 3f72279c88f..06828d4de7d 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -13,6 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. """ +import atexit import os import time from collections import OrderedDict @@ -1166,6 +1167,7 @@ def __getattr__(self, name: str): ) def __del__(self): + oneflow._oneflow_internal.eager.Sync() print(f"oneflow graph {self.name} del") diff --git a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO new file mode 120000 index 00000000000..d630a829c21 --- /dev/null +++ b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO @@ -0,0 +1 @@ +oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 \ No newline at end of file diff --git a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 new file mode 100644 index 00000000000..874488b52a0 --- /dev/null +++ b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 @@ -0,0 +1,987 @@ +Log file created at: 2022/03/12 18:18:36 +Running on machine: oneflow-23 +Running duration (h:mm:ss): 0:00:00 +Log line format: [IWEF]yyyymmdd hh:mm:ss.uuuuuu threadid file:line] msg +I20220312 18:18:36.996003 1174409 global.h:36] NewGlobal N7oneflow7EnvDescE +I20220312 18:18:36.996137 1174409 global.h:36] NewGlobal N7oneflow10ProcessCtxE +I20220312 18:18:36.996150 1174409 env_global_objects_scope.cpp:170] Using rpc backend: local +I20220312 18:18:37.024075 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE +I20220312 18:18:37.024150 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE +I20220312 18:18:37.075817 1174409 global.h:36] NewGlobal N7oneflow2ep21DeviceManagerRegistryE +I20220312 18:18:37.075935 1174409 global.h:36] NewGlobal N7oneflow10ThreadPoolE +I20220312 18:18:37.079267 1174409 global.h:36] NewGlobal N7oneflow16EagerNcclCommMgrE +I20220312 18:18:37.079327 1174409 global.h:36] NewGlobal N7oneflow18CudnnConvAlgoCacheE +I20220312 18:18:37.079352 1174409 global.h:36] NewGlobal N7oneflow2vm19VirtualMachineScopeE +I20220312 18:18:37.079371 1174409 global.h:36] NewGlobal N7oneflow14VirtualMachineE +I20220312 18:18:37.080256 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm13CpuStreamTypeE +I20220312 18:18:37.080309 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm14CudaStreamTypeE +I20220312 18:18:37.080322 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm19AsyncCudaStreamTypeE +I20220312 18:18:37.080749 1174409 global.h:36] NewGlobal N7oneflow27EagerJobBuildAndInferCtxMgrE +I20220312 18:18:37.080799 1174409 global.h:36] NewGlobal N7oneflow12EpollCommNetE +I20220312 18:18:37.081071 1174409 epoll_comm_network.cpp:63] CommNet:Epoll listening on 0.0.0.0:44791 +I20220312 18:18:37.081171 1174409 epoll_comm_network.cpp:197] machine 0 sockfd -1 +I20220312 18:18:37.081408 1174409 global.h:36] NewGlobal N7oneflow9TransportE +I20220312 18:18:37.087878 1174409 global.h:43] DeleteGlobal N7oneflow17ForeignLockHelperE +I20220312 18:18:37.174685 1174409 version.cpp:22] OneFlow git version: v0.6.0-388-g24f02eadd0-snapshot +I20220312 18:18:37.174739 1174409 cuda_device_manager_factory.cpp:63] CUDA runtime version: 11.2 +I20220312 18:18:37.174753 1174409 cuda_device_manager_factory.cpp:72] cuDNN version: 8.1.1 +I20220312 18:18:37.174758 1174409 cuda_device_manager_factory.cpp:85] NCCL version: 2.11.4 +I20220312 18:18:37.174772 1174409 global.h:43] DeleteGlobal N7oneflow12ResourceDescE +I20220312 18:18:37.174782 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE +I20220312 18:18:37.174798 1174409 global.h:36] NewGlobal N7oneflow5IDMgrE +I20220312 18:18:37.174808 1174409 global.h:36] NewGlobal N7oneflow22TaskStreamIndexManagerE +I20220312 18:18:37.174813 1174409 global.h:36] NewGlobal N7oneflow26LazyJobBuildAndInferCtxMgrE +I20220312 18:18:37.174821 1174409 global.h:36] NewGlobal N7oneflow9BufferMgrISt10shared_ptrINS_11JobInstanceEEEE +I20220312 18:18:37.174829 1174409 global.h:36] NewGlobal N7oneflow9BufferMgrISt10shared_ptrINS_23CriticalSectionInstanceEEEE +I20220312 18:18:37.174834 1174409 global.h:36] NewGlobal N7oneflow10RuntimeCtxE +I20220312 18:18:37.174839 1174409 global.h:36] NewGlobal N7oneflow15MemoryAllocatorE +I20220312 18:18:37.174844 1174409 global.h:36] NewGlobal N7oneflow8ChunkMgrE +I20220312 18:18:37.174850 1174409 global.h:36] NewGlobal N7oneflow8RegstMgrE +I20220312 18:18:37.174856 1174409 global.h:36] NewGlobal N7oneflow11ActorMsgBusE +I20220312 18:18:37.174861 1174409 global.h:36] NewGlobal N7oneflow9ThreadMgrE +I20220312 18:18:37.174865 1174409 global.h:36] NewGlobal N7oneflow15RuntimeJobDescsE +I20220312 18:18:37.174870 1174409 global.h:36] NewGlobal N7oneflow7summary12EventsWriterE +I20220312 18:18:37.174877 1174409 global.h:36] NewGlobal N7oneflow6boxing10collective9SchedulerE +I20220312 18:18:37.179025 1174409 global.h:36] NewGlobal N7oneflow7JobDescE +I20220312 18:18:37.180269 1174409 lazy_op_interpreter.cpp:508] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: +: name: "_LinearGraphDestruct_0_input.0.0_2" +device_tag: "cpu" +scope_symbol_id: 4611686018427416575 +input_conf { + out: "out" + blob_conf { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + nd_sbp { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.184612 1174409 lazy_op_interpreter.cpp:511] Lazy nn.Graph name LinearGraphDestruct_0 add op : +_LinearGraphDestruct_0_input.0.0_2 +I20220312 18:18:37.184638 1174409 lazy_op_interpreter.cpp:513] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : +output_bns: "out" +op_conf { + name: "_LinearGraphDestruct_0_input.0.0_2" + device_tag: "cpu" + scope_symbol_id: 4611686018427416575 + input_conf { + out: "out" + blob_conf { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + nd_sbp { + sbp_parallel { + broadcast_parallel { + } + } + } + } + } +} +arg_signature { + bn_in_op2lbi { + key: "out" + value { + op_name: "_LinearGraphDestruct_0_input.0.0_2" + blob_name: "out" + } + } +} +arg_modifier_signature { + obn2output_blob_modifier { + key: "out" + value { + is_mutable: true + requires_grad: false + header_infered_before_compute: false + } + } +} +blob_backward_used_signature { + bn_in_op2blob_backward_used { + key: "out" + value: false + } +} +sbp_signature { + bn_in_op2sbp_parallel { + key: "out" + value { + broadcast_parallel { + } + } + } +} +mirrored_signature { + bn_in_op2opt_mirrored_parallel { + key: "out" + value { + } + } +} +logical_blob_desc_signature { + bn_in_op2blob_desc { + key: "out" + value { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + } + } +} +parallel_signature { + op_parallel_desc_symbol_id: 4611686018427412479 + bn_in_op2parallel_desc_symbol_id { + key: "out" + value: 4611686018427412479 + } +} +parallel_conf_signature { + op_parallel_conf { + device_name: "@0:0" + device_tag: "cpu" + hierarchy { + dim: 1 + } + } +} +nd_sbp_signature { + bn_in_op2nd_sbp { + key: "out" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.188503 1174409 lazy_op_interpreter.cpp:566] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: +: name: "my_linear.weight" +device_tag: "cpu" +scope_symbol_id: 4611686018427428863 +variable_conf { + out: "out" + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + initializer { + empty_conf { + } + } + nd_sbp: "B" +} + +I20220312 18:18:37.189946 1174409 lazy_op_interpreter.cpp:569] Lazy nn.Graph name LinearGraphDestruct_0 add op : +my_linear.weight +I20220312 18:18:37.189968 1174409 lazy_op_interpreter.cpp:571] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : +output_bns: "out" +op_conf { + name: "my_linear.weight" + device_tag: "cpu" + scope_symbol_id: 4611686018427428863 + variable_conf { + out: "out" + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + initializer { + empty_conf { + } + } + nd_sbp: "B" + } +} +arg_signature { + bn_in_op2lbi { + key: "out" + value { + op_name: "my_linear.weight" + blob_name: "out" + } + } +} +arg_modifier_signature { + obn2output_blob_modifier { + key: "out" + value { + is_mutable: true + requires_grad: true + } + } +} +blob_backward_used_signature { + bn_in_op2blob_backward_used { + key: "out" + value: false + } +} +sbp_signature { + bn_in_op2sbp_parallel { + key: "out" + value { + broadcast_parallel { + } + } + } +} +mirrored_signature { + bn_in_op2opt_mirrored_parallel { + key: "out" + value { + } + } +} +logical_blob_desc_signature { + bn_in_op2blob_desc { + key: "out" + value { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + } + } +} +parallel_signature { + op_parallel_desc_symbol_id: 4611686018427412479 + bn_in_op2parallel_desc_symbol_id { + key: "out" + value: 4611686018427412479 + } +} +parallel_conf_signature { + op_parallel_conf { + device_name: "@0:0" + device_tag: "cpu" + hierarchy { + dim: 1 + } + } +} +nd_sbp_signature { + bn_in_op2nd_sbp { + key: "out" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.191457 1174409 lazy_op_interpreter.cpp:930] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: +name: "my_linear-matmul-0" +device_tag: "cpu" +scope_symbol_id: 4611686018427432959 +loc: "Python stack[-2]: ; Python stack[-1]: ; C API: " +user_conf { + op_type_name: "matmul" + input { + key: "a" + value { + s: "_LinearGraphDestruct_0_input.0.0_2/out" + } + } + input { + key: "b" + value { + s: "my_linear.weight/out" + } + } + output { + key: "out" + value { + s: "my_linear-matmul-0/out_0" + } + } + attr { + key: "alpha" + value { + at_double: 1 + } + } + attr { + key: "transpose_a" + value { + at_bool: false + } + } + attr { + key: "transpose_b" + value { + at_bool: true + } + } + input_order: "a" + input_order: "b" + output_order: "out" +} + +I20220312 18:18:37.192956 1174409 lazy_op_interpreter.cpp:933] Lazy nn.Graph name LinearGraphDestruct_0 add op : +my_linear-matmul-0 +I20220312 18:18:37.192975 1174409 lazy_op_interpreter.cpp:934] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : +input_bns: "a_0" +input_bns: "b_0" +output_bns: "out_0" +tmp_bns: "tmp_buffer_0" +op_conf { + name: "undefined-op-name" + device_tag: "cpu" + scope_symbol_id: 4611686018427432959 + loc: "Python stack[-2]: ; Python stack[-1]: ; C API: " + user_conf { + op_type_name: "matmul" + input { + key: "a" + value { + s: "undefined-op-name/undefined-ibn" + } + } + input { + key: "b" + value { + s: "undefined-op-name/undefined-ibn" + } + } + output { + key: "out" + value { + s: "undefined-op-name/out_0" + } + } + attr { + key: "alpha" + value { + at_double: 1 + } + } + attr { + key: "transpose_a" + value { + at_bool: false + } + } + attr { + key: "transpose_b" + value { + at_bool: true + } + } + input_order: "a" + input_order: "b" + output_order: "out" + } +} +arg_signature { + bn_in_op2lbi { + key: "a_0" + value { + op_name: "_LinearGraphDestruct_0_input.0.0_2" + blob_name: "out" + } + } + bn_in_op2lbi { + key: "b_0" + value { + op_name: "my_linear.weight" + blob_name: "out" + } + } + bn_in_op2lbi { + key: "out_0" + value { + op_name: "my_linear-matmul-0" + blob_name: "out_0" + } + } + bn_in_op2lbi { + key: "tmp_buffer_0" + value { + op_name: "my_linear-matmul-0" + blob_name: "tmp_buffer_0" + } + } +} +arg_modifier_signature { + ibn2input_blob_modifier { + key: "a_0" + value { + requires_grad: true + } + } + ibn2input_blob_modifier { + key: "b_0" + value { + requires_grad: true + } + } + obn2output_blob_modifier { + key: "out_0" + value { + requires_grad: true + } + } +} +blob_backward_used_signature { + bn_in_op2blob_backward_used { + key: "a_0" + value: false + } + bn_in_op2blob_backward_used { + key: "b_0" + value: false + } + bn_in_op2blob_backward_used { + key: "out_0" + value: false + } +} +sbp_signature { + bn_in_op2sbp_parallel { + key: "a_0" + value { + broadcast_parallel { + } + } + } + bn_in_op2sbp_parallel { + key: "b_0" + value { + broadcast_parallel { + } + } + } + bn_in_op2sbp_parallel { + key: "out_0" + value { + broadcast_parallel { + } + } + } +} +mirrored_signature { + bn_in_op2opt_mirrored_parallel { + key: "a_0" + value { + } + } + bn_in_op2opt_mirrored_parallel { + key: "b_0" + value { + } + } + bn_in_op2opt_mirrored_parallel { + key: "out_0" + value { + } + } +} +logical_blob_desc_signature { + bn_in_op2blob_desc { + key: "a_0" + value { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + } + } + bn_in_op2blob_desc { + key: "b_0" + value { + shape { + dim: 8 + dim: 3 + } + data_type: kFloat + is_dynamic: false + } + } + bn_in_op2blob_desc { + key: "out_0" + value { + shape { + dim: 8 + dim: 8 + } + data_type: kFloat + is_dynamic: false + } + } +} +parallel_signature { + op_parallel_desc_symbol_id: 4611686018427412479 + bn_in_op2parallel_desc_symbol_id { + key: "a_0" + value: 4611686018427412479 + } + bn_in_op2parallel_desc_symbol_id { + key: "b_0" + value: 4611686018427412479 + } + bn_in_op2parallel_desc_symbol_id { + key: "out_0" + value: 4611686018427412479 + } + bn_in_op2parallel_desc_symbol_id { + key: "tmp_buffer_0" + value: 4611686018427412479 + } +} +parallel_conf_signature { + op_parallel_conf { + device_name: "@0:0" + device_tag: "cpu" + hierarchy { + dim: 1 + } + } +} +nd_sbp_signature { + bn_in_op2nd_sbp { + key: "a_0" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } + bn_in_op2nd_sbp { + key: "b_0" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } + bn_in_op2nd_sbp { + key: "out_0" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.194921 1174409 lazy_op_interpreter.cpp:641] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: +name: "_LinearGraphDestruct_0_output.0.0_2" +device_tag: "cpu" +scope_symbol_id: 4611686018427416575 +output_conf { + in: "my_linear-matmul-0/out_0" + out: "out" + blob_conf { + shape { + dim: 8 + dim: 8 + } + data_type: kFloat + is_dynamic: false + nd_sbp { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.195819 1174409 lazy_op_interpreter.cpp:644] Lazy nn.Graph name LinearGraphDestruct_0 add op : +_LinearGraphDestruct_0_output.0.0_2 +I20220312 18:18:37.195837 1174409 lazy_op_interpreter.cpp:646] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : +input_bns: "in" +output_bns: "out" +op_conf { + name: "_LinearGraphDestruct_0_output.0.0_2" + device_tag: "cpu" + scope_symbol_id: 4611686018427416575 + output_conf { + in: "my_linear-matmul-0/out_0" + out: "out" + blob_conf { + shape { + dim: 8 + dim: 8 + } + data_type: kFloat + is_dynamic: false + nd_sbp { + sbp_parallel { + broadcast_parallel { + } + } + } + } + } +} +arg_signature { + bn_in_op2lbi { + key: "in" + value { + op_name: "my_linear-matmul-0" + blob_name: "out_0" + } + } + bn_in_op2lbi { + key: "out" + value { + op_name: "_LinearGraphDestruct_0_output.0.0_2" + blob_name: "out" + } + } +} +arg_modifier_signature { + ibn2input_blob_modifier { + key: "in" + value { + requires_grad: true + } + } + obn2output_blob_modifier { + key: "out" + value { + is_mutable: true + requires_grad: true + } + } +} +blob_backward_used_signature { + bn_in_op2blob_backward_used { + key: "in" + value: false + } + bn_in_op2blob_backward_used { + key: "out" + value: false + } +} +sbp_signature { + bn_in_op2sbp_parallel { + key: "in" + value { + broadcast_parallel { + } + } + } + bn_in_op2sbp_parallel { + key: "out" + value { + broadcast_parallel { + } + } + } +} +mirrored_signature { + bn_in_op2opt_mirrored_parallel { + key: "in" + value { + } + } + bn_in_op2opt_mirrored_parallel { + key: "out" + value { + } + } +} +logical_blob_desc_signature { + bn_in_op2blob_desc { + key: "in" + value { + shape { + dim: 8 + dim: 8 + } + data_type: kFloat + is_dynamic: false + } + } + bn_in_op2blob_desc { + key: "out" + value { + shape { + dim: 8 + dim: 8 + } + data_type: kFloat + is_dynamic: false + } + } +} +parallel_signature { + op_parallel_desc_symbol_id: 4611686018427412479 + bn_in_op2parallel_desc_symbol_id { + key: "in" + value: 4611686018427412479 + } + bn_in_op2parallel_desc_symbol_id { + key: "out" + value: 4611686018427412479 + } +} +parallel_conf_signature { + op_parallel_conf { + device_name: "@0:0" + device_tag: "cpu" + hierarchy { + dim: 1 + } + } +} +nd_sbp_signature { + bn_in_op2nd_sbp { + key: "in" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } + bn_in_op2nd_sbp { + key: "out" + value { + sbp_parallel { + broadcast_parallel { + } + } + } + } +} + +I20220312 18:18:37.196897 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE +I20220312 18:18:37.196943 1174409 global.h:36] NewGlobal N7oneflow7JobDescE +I20220312 18:18:37.208326 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE +I20220312 18:18:37.213130 1174409 global.h:36] NewGlobal N7oneflow7JobDescE +I20220312 18:18:38.376291 1174409 global.h:36] NewGlobal N7oneflow7OpGraphE +I20220312 18:18:38.393191 1174409 global.h:43] DeleteGlobal N7oneflow7OpGraphE +I20220312 18:18:38.396402 1174409 nn_graph.cpp:265] Graph name: LinearGraphDestruct_0 compile time: 1.18321 seconds. +I20220312 18:18:38.399426 1174409 runtime_context.cpp:21] NewCounter constructing_actor_cnt 26 +I20220312 18:18:38.402170 1174805 naive_actor.cpp:25] actor 1099532599296 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.402868 1174809 naive_actor.cpp:25] actor 1099522113536 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.403072 1174808 naive_actor.cpp:25] actor 1099526307840 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.403460 1174802 wait_and_send_ids_actor.cpp:53] actor 1099647942656 switch to &WaitAndSendIdsActor::HandlerWaitToStart +I20220312 18:18:38.403594 1174804 sink_actor.cpp:21] actor 1099650039808 switch to &SinkActor::HandlerNormal +I20220312 18:18:38.403934 1174807 naive_actor.cpp:25] actor 1099520016384 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.404458 1174811 naive_actor.cpp:25] actor 1099524210688 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.402642 1174801 thread.cpp:101] Thread 524352 construct LightActor kTick 1099645845509 +I20220312 18:18:38.404510 1174811 thread.cpp:98] Thread 524294 construct Actor kDeviceTick 1099524210688 +I20220312 18:18:38.403481 1174802 thread.cpp:98] Thread 524353 construct Actor kWaitAndSendIds 1099647942656 +I20220312 18:18:38.402925 1174809 thread.cpp:98] Thread 524293 construct Actor kDeviceTick 1099522113536 +I20220312 18:18:38.403652 1174804 thread.cpp:98] Thread 524354 construct Actor kCallbackNotify 1099650039808 +I20220312 18:18:38.403141 1174808 thread.cpp:98] Thread 524295 construct Actor kDeviceTick 1099526307840 +I20220312 18:18:38.403987 1174807 thread.cpp:98] Thread 524292 construct Actor kDeviceTick 1099520016384 +I20220312 18:18:38.404947 1174802 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 25 +I20220312 18:18:38.404937 1174816 thread.cpp:101] Thread 524288 construct LightActor kNormalForward 1099511627776 +I20220312 18:18:38.404438 1174813 thread.cpp:101] Thread 524291 construct LightActor kNormalForward 1099517919232 +I20220312 18:18:38.402662 1174803 thread.cpp:101] Thread 524289 construct LightActor kNormalForward 1099513724928 +I20220312 18:18:38.405063 1174809 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 22 +I20220312 18:18:38.405038 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 24 +I20220312 18:18:38.405054 1174811 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 23 +I20220312 18:18:38.405068 1174804 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 21 +I20220312 18:18:38.405153 1174807 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 20 +I20220312 18:18:38.405160 1174808 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 19 +I20220312 18:18:38.405215 1174816 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 18 +I20220312 18:18:38.405194 1174815 naive_actor.cpp:25] actor 1099530502144 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.402251 1174805 thread.cpp:98] Thread 524298 construct Actor kDeviceTick 1099532599296 +I20220312 18:18:38.405298 1174815 thread.cpp:98] Thread 524297 construct Actor kDeviceTick 1099530502144 +I20220312 18:18:38.405289 1174813 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 16 +I20220312 18:18:38.405282 1174803 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 17 +I20220312 18:18:38.405474 1174805 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 15 +I20220312 18:18:38.405505 1174815 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 14 +I20220312 18:18:38.406764 1174801 naive_actor.cpp:25] actor 1099645845512 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.406805 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845512 +I20220312 18:18:38.407002 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 13 +I20220312 18:18:38.407407 1174810 naive_actor.cpp:25] actor 1099652136960 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.407409 1174806 naive_actor.cpp:25] actor 1099534696448 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.407465 1174810 thread.cpp:98] Thread 524355 construct Actor kCriticalSectionWaitTick 1099652136960 +I20220312 18:18:38.407501 1174806 thread.cpp:98] Thread 524299 construct Actor kNormalForward 1099534696448 +I20220312 18:18:38.407704 1174810 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 12 +I20220312 18:18:38.407716 1174806 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 11 +I20220312 18:18:38.407800 1174814 thread.cpp:101] Thread 524290 construct LightActor kNormalForward 1099515822080 +I20220312 18:18:38.408062 1174814 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 10 +I20220312 18:18:38.408448 1174800 naive_actor.cpp:25] actor 1099528404992 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.408502 1174800 thread.cpp:98] Thread 524296 construct Actor kNormalForward 1099528404992 +I20220312 18:18:38.408633 1174801 naive_actor.cpp:25] actor 1099645845505 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.408670 1174801 thread.cpp:98] Thread 524352 construct Actor kTick 1099645845505 +I20220312 18:18:38.408730 1174800 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 9 +I20220312 18:18:38.408787 1174812 naive_actor.cpp:25] actor 1099654234112 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.408847 1174812 thread.cpp:98] Thread 524356 construct Actor kCriticalSectionWaitTick 1099654234112 +I20220312 18:18:38.408864 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 8 +I20220312 18:18:38.409101 1174812 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 7 +I20220312 18:18:38.410570 1174801 naive_actor.cpp:25] actor 1099645845513 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.410610 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845513 +I20220312 18:18:38.410810 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 6 +I20220312 18:18:38.412608 1174801 naive_actor.cpp:25] actor 1099645845511 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.412645 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845511 +I20220312 18:18:38.412847 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 5 +I20220312 18:18:38.414455 1174801 naive_actor.cpp:25] actor 1099645845507 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.414489 1174801 thread.cpp:98] Thread 524352 construct Actor kTick 1099645845507 +I20220312 18:18:38.414666 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 4 +I20220312 18:18:38.415032 1174801 thread.cpp:101] Thread 524352 construct LightActor kTick 1099645845508 +I20220312 18:18:38.415199 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 3 +I20220312 18:18:38.416571 1174801 naive_actor.cpp:25] actor 1099645845504 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.416602 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845504 +I20220312 18:18:38.416765 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 2 +I20220312 18:18:38.418165 1174801 naive_actor.cpp:25] actor 1099645845510 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.418198 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845510 +I20220312 18:18:38.418359 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 1 +I20220312 18:18:38.419966 1174801 naive_actor.cpp:25] actor 1099645845506 switch to &NaiveActor::HandlerNormal +I20220312 18:18:38.419997 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845506 +I20220312 18:18:38.420188 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 0 +I20220312 18:18:38.420208 1174409 runtime.cpp:95] Actors on this machine constructed +I20220312 18:18:38.420279 1174409 runtime.cpp:97] Actors on every machine constructed +I20220312 18:18:38.420292 1174409 runtime_context.cpp:21] NewCounter job_0_running_actor_count 26 +I20220312 18:18:38.420327 1174802 wait_and_send_ids_actor.cpp:73] actor 1099647942656 switch to &WaitAndSendIdsActor::HandlerNormal +I20220312 18:18:38.420346 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE +I20220312 18:18:38.480592 1174409 nn_graph.cpp:72] Graph destructor Try to close c nn graph name LinearGraphDestruct_0. +I20220312 18:18:38.480641 1174409 nn_graph.cpp:78] Try to close c nn graph name LinearGraphDestruct_0. +I20220312 18:18:38.480790 1174802 actor.cpp:399] actor 1099647942656 switch to &Actor::HandlerZombie +I20220312 18:18:38.480824 1174802 thread.cpp:76] thread 524353 deconstruct actor 1099647942656 +I20220312 18:18:38.480865 1174801 actor.cpp:396] actor 1099645845505 switch to nullptr +I20220312 18:18:38.480876 1174810 actor.cpp:396] actor 1099652136960 switch to nullptr +I20220312 18:18:38.480865 1174812 actor.cpp:396] actor 1099654234112 switch to nullptr +I20220312 18:18:38.480914 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845505 +I20220312 18:18:38.480942 1174810 thread.cpp:76] thread 524355 deconstruct actor 1099652136960 +I20220312 18:18:38.480975 1174812 thread.cpp:76] thread 524356 deconstruct actor 1099654234112 +I20220312 18:18:38.481328 1174802 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 25 +I20220312 18:18:38.481544 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 24 +I20220312 18:18:38.481626 1174801 actor.cpp:396] actor 1099645845504 switch to nullptr +I20220312 18:18:38.481642 1174810 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 23 +I20220312 18:18:38.481642 1174812 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 22 +I20220312 18:18:38.481643 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845504 +I20220312 18:18:38.481714 1174807 actor.cpp:396] actor 1099520016384 switch to nullptr +I20220312 18:18:38.481765 1174807 thread.cpp:76] thread 524292 deconstruct actor 1099520016384 +I20220312 18:18:38.481760 1174803 thread.cpp:76] thread 524289 deconstruct actor 1099513724928 +I20220312 18:18:38.482008 1174803 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 21 +I20220312 18:18:38.482120 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 20 +I20220312 18:18:38.482188 1174801 actor.cpp:399] actor 1099645845506 switch to &Actor::HandlerZombie +I20220312 18:18:38.482219 1174801 actor.cpp:396] actor 1099645845507 switch to nullptr +I20220312 18:18:38.482262 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845507 +I20220312 18:18:38.482215 1174807 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 19 +I20220312 18:18:38.482662 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 18 +I20220312 18:18:38.482718 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845508 +I20220312 18:18:38.482865 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 17 +I20220312 18:18:38.482918 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845509 +I20220312 18:18:38.482961 1174804 actor.cpp:396] actor 1099650039808 switch to nullptr +I20220312 18:18:38.482992 1174804 thread.cpp:76] thread 524354 deconstruct actor 1099650039808 +I20220312 18:18:38.483058 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 16 +I20220312 18:18:38.483129 1174801 actor.cpp:396] actor 1099645845510 switch to nullptr +I20220312 18:18:38.483144 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845510 +I20220312 18:18:38.483191 1174811 actor.cpp:396] actor 1099524210688 switch to nullptr +I20220312 18:18:38.483223 1174811 thread.cpp:76] thread 524294 deconstruct actor 1099524210688 +I20220312 18:18:38.483242 1174816 thread.cpp:76] thread 524288 deconstruct actor 1099511627776 +I20220312 18:18:38.483260 1174814 thread.cpp:76] thread 524290 deconstruct actor 1099515822080 +I20220312 18:18:38.483296 1174808 actor.cpp:396] actor 1099526307840 switch to nullptr +I20220312 18:18:38.483330 1174808 thread.cpp:76] thread 524295 deconstruct actor 1099526307840 +I20220312 18:18:38.483377 1174804 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 15 +I20220312 18:18:38.483532 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 14 +I20220312 18:18:38.483613 1174801 actor.cpp:396] actor 1099645845512 switch to nullptr +I20220312 18:18:38.483629 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845512 +I20220312 18:18:38.483675 1174811 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 13 +I20220312 18:18:38.483678 1174815 actor.cpp:396] actor 1099530502144 switch to nullptr +I20220312 18:18:38.483729 1174813 thread.cpp:76] thread 524291 deconstruct actor 1099517919232 +I20220312 18:18:38.483748 1174815 thread.cpp:76] thread 524297 deconstruct actor 1099530502144 +I20220312 18:18:38.483714 1174816 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 12 +I20220312 18:18:38.483773 1174809 actor.cpp:396] actor 1099522113536 switch to nullptr +I20220312 18:18:38.483811 1174808 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 11 +I20220312 18:18:38.483794 1174805 actor.cpp:396] actor 1099532599296 switch to nullptr +I20220312 18:18:38.483832 1174809 thread.cpp:76] thread 524293 deconstruct actor 1099522113536 +I20220312 18:18:38.483884 1174805 thread.cpp:76] thread 524298 deconstruct actor 1099532599296 +I20220312 18:18:38.484004 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 10 +I20220312 18:18:38.484081 1174801 actor.cpp:399] actor 1099645845511 switch to &Actor::HandlerZombie +I20220312 18:18:38.484122 1174801 actor.cpp:399] actor 1099645845513 switch to &Actor::HandlerZombie +I20220312 18:18:38.484148 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845511 +I20220312 18:18:38.484148 1174800 actor.cpp:396] actor 1099528404992 switch to nullptr +I20220312 18:18:38.484184 1174800 thread.cpp:76] thread 524296 deconstruct actor 1099528404992 +I20220312 18:18:38.484184 1174806 actor.cpp:396] actor 1099534696448 switch to nullptr +I20220312 18:18:38.484220 1174806 thread.cpp:76] thread 524299 deconstruct actor 1099534696448 +I20220312 18:18:38.484237 1174814 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 9 +I20220312 18:18:38.484272 1174815 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 7 +I20220312 18:18:38.484288 1174805 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 6 +I20220312 18:18:38.484295 1174809 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 5 +I20220312 18:18:38.484272 1174813 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 8 +I20220312 18:18:38.484560 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 4 +I20220312 18:18:38.484597 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845513 +I20220312 18:18:38.484836 1174806 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 3 +I20220312 18:18:38.484954 1174800 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 2 +I20220312 18:18:38.484987 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 1 +I20220312 18:18:38.485024 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845506 +I20220312 18:18:38.485464 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 0 +I20220312 18:18:38.485555 1174409 nn_graph.cpp:82] Finish close c nn graph name LinearGraphDestruct_0. +I20220312 18:18:38.485572 1174409 multi_client_session_context.cpp:157] Try to delete multi client session context. +I20220312 18:18:38.485576 1174409 global.h:43] DeleteGlobal N7oneflow6boxing10collective9SchedulerE +I20220312 18:18:38.485613 1174409 global.h:43] DeleteGlobal N7oneflow7summary12EventsWriterE +I20220312 18:18:38.485626 1174409 global.h:43] DeleteGlobal N7oneflow15RuntimeJobDescsE +I20220312 18:18:38.485689 1174409 global.h:43] DeleteGlobal N7oneflow9ThreadMgrE +I20220312 18:18:38.485827 1174409 thread_manager.cpp:29] actor thread 524288 finish +I20220312 18:18:38.485935 1174409 thread_manager.cpp:29] actor thread 524290 finish +I20220312 18:18:38.486022 1174409 thread_manager.cpp:29] actor thread 524291 finish +I20220312 18:18:38.486085 1174409 thread_manager.cpp:29] actor thread 524289 finish +I20220312 18:18:38.486219 1174409 thread_manager.cpp:29] actor thread 524296 finish +I20220312 18:18:38.486351 1174409 thread_manager.cpp:29] actor thread 524354 finish +I20220312 18:18:38.486469 1174409 thread_manager.cpp:29] actor thread 524299 finish +I20220312 18:18:38.486572 1174409 thread_manager.cpp:29] actor thread 524292 finish +I20220312 18:18:38.486670 1174409 thread_manager.cpp:29] actor thread 524353 finish +I20220312 18:18:38.486773 1174409 thread_manager.cpp:29] actor thread 524295 finish +I20220312 18:18:38.486891 1174409 thread_manager.cpp:29] actor thread 524293 finish +I20220312 18:18:38.486994 1174409 thread_manager.cpp:29] actor thread 524297 finish +I20220312 18:18:38.487115 1174409 thread_manager.cpp:29] actor thread 524355 finish +I20220312 18:18:38.487239 1174409 thread_manager.cpp:29] actor thread 524352 finish +I20220312 18:18:38.487354 1174409 thread_manager.cpp:29] actor thread 524294 finish +I20220312 18:18:38.487460 1174409 thread_manager.cpp:29] actor thread 524298 finish +I20220312 18:18:38.487589 1174409 thread_manager.cpp:29] actor thread 524356 finish +I20220312 18:18:38.487601 1174409 global.h:43] DeleteGlobal N7oneflow11ActorMsgBusE +I20220312 18:18:38.487610 1174409 global.h:43] DeleteGlobal N7oneflow8RegstMgrE +I20220312 18:18:38.487810 1174409 global.h:43] DeleteGlobal N7oneflow8ChunkMgrE +I20220312 18:18:38.487820 1174409 global.h:43] DeleteGlobal N7oneflow15MemoryAllocatorE +I20220312 18:18:38.487974 1174409 global.h:43] DeleteGlobal N7oneflow10RuntimeCtxE +I20220312 18:18:38.487984 1174409 global.h:43] DeleteGlobal N7oneflow9BufferMgrISt10shared_ptrINS_23CriticalSectionInstanceEEEE +I20220312 18:18:38.488000 1174409 global.h:43] DeleteGlobal N7oneflow9BufferMgrISt10shared_ptrINS_11JobInstanceEEEE +I20220312 18:18:38.488014 1174409 global.h:43] DeleteGlobal N7oneflow26LazyJobBuildAndInferCtxMgrE +I20220312 18:18:38.488508 1174409 global.h:43] DeleteGlobal N7oneflow22TaskStreamIndexManagerE +I20220312 18:18:38.488531 1174409 global.h:43] DeleteGlobal N7oneflow5IDMgrE +I20220312 18:18:38.488552 1174409 global.h:43] DeleteGlobal N7oneflow12ResourceDescE +I20220312 18:18:38.488564 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE +I20220312 18:18:38.488585 1174409 multi_client_session_context.cpp:184] Finish delete multi client session context. +I20220312 18:18:38.488593 1174409 env_global_objects_scope.cpp:242] Try to close env global objects scope. +I20220312 18:18:38.488628 1174409 global.h:43] DeleteGlobal N7oneflow14KernelObserverE +I20220312 18:18:38.488641 1174409 global.h:43] DeleteGlobal N7oneflow9TransportE +I20220312 18:18:38.488854 1174409 global.h:43] DeleteGlobal N7oneflow12EpollCommNetE +I20220312 18:18:38.488862 1174409 epoll_comm_network.cpp:87] CommNet Thread 0 finish +I20220312 18:18:38.488978 1174409 epoll_comm_network.cpp:87] CommNet Thread 1 finish +I20220312 18:18:38.489063 1174409 epoll_comm_network.cpp:87] CommNet Thread 2 finish +I20220312 18:18:38.489166 1174409 epoll_comm_network.cpp:87] CommNet Thread 3 finish +I20220312 18:18:38.489331 1174409 global.h:43] DeleteGlobal N7oneflow27EagerJobBuildAndInferCtxMgrE +I20220312 18:18:38.489342 1174409 global.h:43] DeleteGlobal N7oneflow2vm19VirtualMachineScopeE +I20220312 18:18:38.489351 1174409 global.h:43] DeleteGlobal N7oneflow14VirtualMachineE diff --git a/python/oneflow/test/graph/test_graph_session_env_destruct.py b/python/oneflow/test/graph/test_graph_session_env_destruct.py index 4fd90d927f9..fdc25d2e181 100644 --- a/python/oneflow/test/graph/test_graph_session_env_destruct.py +++ b/python/oneflow/test/graph/test_graph_session_env_destruct.py @@ -51,4 +51,4 @@ def test_linear_graph_destruct(test_case): if __name__ == "__main__": - unittest.main() \ No newline at end of file + unittest.main() From 26abf824d08e1b9a54084ec5c5e6902ad9d92259 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 19:26:11 +0800 Subject: [PATCH 21/54] rm useless --- .../graph/log_stuck/oneflow-23/oneflow.INFO | 1 - ....xuxiaoyu.log.INFO.20220312-181836.1174409 | 987 ------------------ 2 files changed, 988 deletions(-) delete mode 120000 python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO delete mode 100644 python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 diff --git a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO deleted file mode 120000 index d630a829c21..00000000000 --- a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.INFO +++ /dev/null @@ -1 +0,0 @@ -oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 \ No newline at end of file diff --git a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 b/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 deleted file mode 100644 index 874488b52a0..00000000000 --- a/python/oneflow/test/graph/log_stuck/oneflow-23/oneflow.oneflow-23.xuxiaoyu.log.INFO.20220312-181836.1174409 +++ /dev/null @@ -1,987 +0,0 @@ -Log file created at: 2022/03/12 18:18:36 -Running on machine: oneflow-23 -Running duration (h:mm:ss): 0:00:00 -Log line format: [IWEF]yyyymmdd hh:mm:ss.uuuuuu threadid file:line] msg -I20220312 18:18:36.996003 1174409 global.h:36] NewGlobal N7oneflow7EnvDescE -I20220312 18:18:36.996137 1174409 global.h:36] NewGlobal N7oneflow10ProcessCtxE -I20220312 18:18:36.996150 1174409 env_global_objects_scope.cpp:170] Using rpc backend: local -I20220312 18:18:37.024075 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE -I20220312 18:18:37.024150 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE -I20220312 18:18:37.075817 1174409 global.h:36] NewGlobal N7oneflow2ep21DeviceManagerRegistryE -I20220312 18:18:37.075935 1174409 global.h:36] NewGlobal N7oneflow10ThreadPoolE -I20220312 18:18:37.079267 1174409 global.h:36] NewGlobal N7oneflow16EagerNcclCommMgrE -I20220312 18:18:37.079327 1174409 global.h:36] NewGlobal N7oneflow18CudnnConvAlgoCacheE -I20220312 18:18:37.079352 1174409 global.h:36] NewGlobal N7oneflow2vm19VirtualMachineScopeE -I20220312 18:18:37.079371 1174409 global.h:36] NewGlobal N7oneflow14VirtualMachineE -I20220312 18:18:37.080256 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm13CpuStreamTypeE -I20220312 18:18:37.080309 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm14CudaStreamTypeE -I20220312 18:18:37.080322 1174409 virtual_machine.cpp:92] transport stream type: N7oneflow2vm19AsyncCudaStreamTypeE -I20220312 18:18:37.080749 1174409 global.h:36] NewGlobal N7oneflow27EagerJobBuildAndInferCtxMgrE -I20220312 18:18:37.080799 1174409 global.h:36] NewGlobal N7oneflow12EpollCommNetE -I20220312 18:18:37.081071 1174409 epoll_comm_network.cpp:63] CommNet:Epoll listening on 0.0.0.0:44791 -I20220312 18:18:37.081171 1174409 epoll_comm_network.cpp:197] machine 0 sockfd -1 -I20220312 18:18:37.081408 1174409 global.h:36] NewGlobal N7oneflow9TransportE -I20220312 18:18:37.087878 1174409 global.h:43] DeleteGlobal N7oneflow17ForeignLockHelperE -I20220312 18:18:37.174685 1174409 version.cpp:22] OneFlow git version: v0.6.0-388-g24f02eadd0-snapshot -I20220312 18:18:37.174739 1174409 cuda_device_manager_factory.cpp:63] CUDA runtime version: 11.2 -I20220312 18:18:37.174753 1174409 cuda_device_manager_factory.cpp:72] cuDNN version: 8.1.1 -I20220312 18:18:37.174758 1174409 cuda_device_manager_factory.cpp:85] NCCL version: 2.11.4 -I20220312 18:18:37.174772 1174409 global.h:43] DeleteGlobal N7oneflow12ResourceDescE -I20220312 18:18:37.174782 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE -I20220312 18:18:37.174798 1174409 global.h:36] NewGlobal N7oneflow5IDMgrE -I20220312 18:18:37.174808 1174409 global.h:36] NewGlobal N7oneflow22TaskStreamIndexManagerE -I20220312 18:18:37.174813 1174409 global.h:36] NewGlobal N7oneflow26LazyJobBuildAndInferCtxMgrE -I20220312 18:18:37.174821 1174409 global.h:36] NewGlobal N7oneflow9BufferMgrISt10shared_ptrINS_11JobInstanceEEEE -I20220312 18:18:37.174829 1174409 global.h:36] NewGlobal N7oneflow9BufferMgrISt10shared_ptrINS_23CriticalSectionInstanceEEEE -I20220312 18:18:37.174834 1174409 global.h:36] NewGlobal N7oneflow10RuntimeCtxE -I20220312 18:18:37.174839 1174409 global.h:36] NewGlobal N7oneflow15MemoryAllocatorE -I20220312 18:18:37.174844 1174409 global.h:36] NewGlobal N7oneflow8ChunkMgrE -I20220312 18:18:37.174850 1174409 global.h:36] NewGlobal N7oneflow8RegstMgrE -I20220312 18:18:37.174856 1174409 global.h:36] NewGlobal N7oneflow11ActorMsgBusE -I20220312 18:18:37.174861 1174409 global.h:36] NewGlobal N7oneflow9ThreadMgrE -I20220312 18:18:37.174865 1174409 global.h:36] NewGlobal N7oneflow15RuntimeJobDescsE -I20220312 18:18:37.174870 1174409 global.h:36] NewGlobal N7oneflow7summary12EventsWriterE -I20220312 18:18:37.174877 1174409 global.h:36] NewGlobal N7oneflow6boxing10collective9SchedulerE -I20220312 18:18:37.179025 1174409 global.h:36] NewGlobal N7oneflow7JobDescE -I20220312 18:18:37.180269 1174409 lazy_op_interpreter.cpp:508] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: -: name: "_LinearGraphDestruct_0_input.0.0_2" -device_tag: "cpu" -scope_symbol_id: 4611686018427416575 -input_conf { - out: "out" - blob_conf { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - nd_sbp { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.184612 1174409 lazy_op_interpreter.cpp:511] Lazy nn.Graph name LinearGraphDestruct_0 add op : -_LinearGraphDestruct_0_input.0.0_2 -I20220312 18:18:37.184638 1174409 lazy_op_interpreter.cpp:513] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : -output_bns: "out" -op_conf { - name: "_LinearGraphDestruct_0_input.0.0_2" - device_tag: "cpu" - scope_symbol_id: 4611686018427416575 - input_conf { - out: "out" - blob_conf { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - nd_sbp { - sbp_parallel { - broadcast_parallel { - } - } - } - } - } -} -arg_signature { - bn_in_op2lbi { - key: "out" - value { - op_name: "_LinearGraphDestruct_0_input.0.0_2" - blob_name: "out" - } - } -} -arg_modifier_signature { - obn2output_blob_modifier { - key: "out" - value { - is_mutable: true - requires_grad: false - header_infered_before_compute: false - } - } -} -blob_backward_used_signature { - bn_in_op2blob_backward_used { - key: "out" - value: false - } -} -sbp_signature { - bn_in_op2sbp_parallel { - key: "out" - value { - broadcast_parallel { - } - } - } -} -mirrored_signature { - bn_in_op2opt_mirrored_parallel { - key: "out" - value { - } - } -} -logical_blob_desc_signature { - bn_in_op2blob_desc { - key: "out" - value { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - } - } -} -parallel_signature { - op_parallel_desc_symbol_id: 4611686018427412479 - bn_in_op2parallel_desc_symbol_id { - key: "out" - value: 4611686018427412479 - } -} -parallel_conf_signature { - op_parallel_conf { - device_name: "@0:0" - device_tag: "cpu" - hierarchy { - dim: 1 - } - } -} -nd_sbp_signature { - bn_in_op2nd_sbp { - key: "out" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.188503 1174409 lazy_op_interpreter.cpp:566] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: -: name: "my_linear.weight" -device_tag: "cpu" -scope_symbol_id: 4611686018427428863 -variable_conf { - out: "out" - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - initializer { - empty_conf { - } - } - nd_sbp: "B" -} - -I20220312 18:18:37.189946 1174409 lazy_op_interpreter.cpp:569] Lazy nn.Graph name LinearGraphDestruct_0 add op : -my_linear.weight -I20220312 18:18:37.189968 1174409 lazy_op_interpreter.cpp:571] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : -output_bns: "out" -op_conf { - name: "my_linear.weight" - device_tag: "cpu" - scope_symbol_id: 4611686018427428863 - variable_conf { - out: "out" - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - initializer { - empty_conf { - } - } - nd_sbp: "B" - } -} -arg_signature { - bn_in_op2lbi { - key: "out" - value { - op_name: "my_linear.weight" - blob_name: "out" - } - } -} -arg_modifier_signature { - obn2output_blob_modifier { - key: "out" - value { - is_mutable: true - requires_grad: true - } - } -} -blob_backward_used_signature { - bn_in_op2blob_backward_used { - key: "out" - value: false - } -} -sbp_signature { - bn_in_op2sbp_parallel { - key: "out" - value { - broadcast_parallel { - } - } - } -} -mirrored_signature { - bn_in_op2opt_mirrored_parallel { - key: "out" - value { - } - } -} -logical_blob_desc_signature { - bn_in_op2blob_desc { - key: "out" - value { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - } - } -} -parallel_signature { - op_parallel_desc_symbol_id: 4611686018427412479 - bn_in_op2parallel_desc_symbol_id { - key: "out" - value: 4611686018427412479 - } -} -parallel_conf_signature { - op_parallel_conf { - device_name: "@0:0" - device_tag: "cpu" - hierarchy { - dim: 1 - } - } -} -nd_sbp_signature { - bn_in_op2nd_sbp { - key: "out" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.191457 1174409 lazy_op_interpreter.cpp:930] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: -name: "my_linear-matmul-0" -device_tag: "cpu" -scope_symbol_id: 4611686018427432959 -loc: "Python stack[-2]: ; Python stack[-1]: ; C API: " -user_conf { - op_type_name: "matmul" - input { - key: "a" - value { - s: "_LinearGraphDestruct_0_input.0.0_2/out" - } - } - input { - key: "b" - value { - s: "my_linear.weight/out" - } - } - output { - key: "out" - value { - s: "my_linear-matmul-0/out_0" - } - } - attr { - key: "alpha" - value { - at_double: 1 - } - } - attr { - key: "transpose_a" - value { - at_bool: false - } - } - attr { - key: "transpose_b" - value { - at_bool: true - } - } - input_order: "a" - input_order: "b" - output_order: "out" -} - -I20220312 18:18:37.192956 1174409 lazy_op_interpreter.cpp:933] Lazy nn.Graph name LinearGraphDestruct_0 add op : -my_linear-matmul-0 -I20220312 18:18:37.192975 1174409 lazy_op_interpreter.cpp:934] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : -input_bns: "a_0" -input_bns: "b_0" -output_bns: "out_0" -tmp_bns: "tmp_buffer_0" -op_conf { - name: "undefined-op-name" - device_tag: "cpu" - scope_symbol_id: 4611686018427432959 - loc: "Python stack[-2]: ; Python stack[-1]: ; C API: " - user_conf { - op_type_name: "matmul" - input { - key: "a" - value { - s: "undefined-op-name/undefined-ibn" - } - } - input { - key: "b" - value { - s: "undefined-op-name/undefined-ibn" - } - } - output { - key: "out" - value { - s: "undefined-op-name/out_0" - } - } - attr { - key: "alpha" - value { - at_double: 1 - } - } - attr { - key: "transpose_a" - value { - at_bool: false - } - } - attr { - key: "transpose_b" - value { - at_bool: true - } - } - input_order: "a" - input_order: "b" - output_order: "out" - } -} -arg_signature { - bn_in_op2lbi { - key: "a_0" - value { - op_name: "_LinearGraphDestruct_0_input.0.0_2" - blob_name: "out" - } - } - bn_in_op2lbi { - key: "b_0" - value { - op_name: "my_linear.weight" - blob_name: "out" - } - } - bn_in_op2lbi { - key: "out_0" - value { - op_name: "my_linear-matmul-0" - blob_name: "out_0" - } - } - bn_in_op2lbi { - key: "tmp_buffer_0" - value { - op_name: "my_linear-matmul-0" - blob_name: "tmp_buffer_0" - } - } -} -arg_modifier_signature { - ibn2input_blob_modifier { - key: "a_0" - value { - requires_grad: true - } - } - ibn2input_blob_modifier { - key: "b_0" - value { - requires_grad: true - } - } - obn2output_blob_modifier { - key: "out_0" - value { - requires_grad: true - } - } -} -blob_backward_used_signature { - bn_in_op2blob_backward_used { - key: "a_0" - value: false - } - bn_in_op2blob_backward_used { - key: "b_0" - value: false - } - bn_in_op2blob_backward_used { - key: "out_0" - value: false - } -} -sbp_signature { - bn_in_op2sbp_parallel { - key: "a_0" - value { - broadcast_parallel { - } - } - } - bn_in_op2sbp_parallel { - key: "b_0" - value { - broadcast_parallel { - } - } - } - bn_in_op2sbp_parallel { - key: "out_0" - value { - broadcast_parallel { - } - } - } -} -mirrored_signature { - bn_in_op2opt_mirrored_parallel { - key: "a_0" - value { - } - } - bn_in_op2opt_mirrored_parallel { - key: "b_0" - value { - } - } - bn_in_op2opt_mirrored_parallel { - key: "out_0" - value { - } - } -} -logical_blob_desc_signature { - bn_in_op2blob_desc { - key: "a_0" - value { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - } - } - bn_in_op2blob_desc { - key: "b_0" - value { - shape { - dim: 8 - dim: 3 - } - data_type: kFloat - is_dynamic: false - } - } - bn_in_op2blob_desc { - key: "out_0" - value { - shape { - dim: 8 - dim: 8 - } - data_type: kFloat - is_dynamic: false - } - } -} -parallel_signature { - op_parallel_desc_symbol_id: 4611686018427412479 - bn_in_op2parallel_desc_symbol_id { - key: "a_0" - value: 4611686018427412479 - } - bn_in_op2parallel_desc_symbol_id { - key: "b_0" - value: 4611686018427412479 - } - bn_in_op2parallel_desc_symbol_id { - key: "out_0" - value: 4611686018427412479 - } - bn_in_op2parallel_desc_symbol_id { - key: "tmp_buffer_0" - value: 4611686018427412479 - } -} -parallel_conf_signature { - op_parallel_conf { - device_name: "@0:0" - device_tag: "cpu" - hierarchy { - dim: 1 - } - } -} -nd_sbp_signature { - bn_in_op2nd_sbp { - key: "a_0" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } - bn_in_op2nd_sbp { - key: "b_0" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } - bn_in_op2nd_sbp { - key: "out_0" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.194921 1174409 lazy_op_interpreter.cpp:641] Lazy nn.Graph name LinearGraphDestruct_0 try to add op: -name: "_LinearGraphDestruct_0_output.0.0_2" -device_tag: "cpu" -scope_symbol_id: 4611686018427416575 -output_conf { - in: "my_linear-matmul-0/out_0" - out: "out" - blob_conf { - shape { - dim: 8 - dim: 8 - } - data_type: kFloat - is_dynamic: false - nd_sbp { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.195819 1174409 lazy_op_interpreter.cpp:644] Lazy nn.Graph name LinearGraphDestruct_0 add op : -_LinearGraphDestruct_0_output.0.0_2 -I20220312 18:18:37.195837 1174409 lazy_op_interpreter.cpp:646] Lazy nn.Graph name LinearGraphDestruct_0 infer and and op attr : -input_bns: "in" -output_bns: "out" -op_conf { - name: "_LinearGraphDestruct_0_output.0.0_2" - device_tag: "cpu" - scope_symbol_id: 4611686018427416575 - output_conf { - in: "my_linear-matmul-0/out_0" - out: "out" - blob_conf { - shape { - dim: 8 - dim: 8 - } - data_type: kFloat - is_dynamic: false - nd_sbp { - sbp_parallel { - broadcast_parallel { - } - } - } - } - } -} -arg_signature { - bn_in_op2lbi { - key: "in" - value { - op_name: "my_linear-matmul-0" - blob_name: "out_0" - } - } - bn_in_op2lbi { - key: "out" - value { - op_name: "_LinearGraphDestruct_0_output.0.0_2" - blob_name: "out" - } - } -} -arg_modifier_signature { - ibn2input_blob_modifier { - key: "in" - value { - requires_grad: true - } - } - obn2output_blob_modifier { - key: "out" - value { - is_mutable: true - requires_grad: true - } - } -} -blob_backward_used_signature { - bn_in_op2blob_backward_used { - key: "in" - value: false - } - bn_in_op2blob_backward_used { - key: "out" - value: false - } -} -sbp_signature { - bn_in_op2sbp_parallel { - key: "in" - value { - broadcast_parallel { - } - } - } - bn_in_op2sbp_parallel { - key: "out" - value { - broadcast_parallel { - } - } - } -} -mirrored_signature { - bn_in_op2opt_mirrored_parallel { - key: "in" - value { - } - } - bn_in_op2opt_mirrored_parallel { - key: "out" - value { - } - } -} -logical_blob_desc_signature { - bn_in_op2blob_desc { - key: "in" - value { - shape { - dim: 8 - dim: 8 - } - data_type: kFloat - is_dynamic: false - } - } - bn_in_op2blob_desc { - key: "out" - value { - shape { - dim: 8 - dim: 8 - } - data_type: kFloat - is_dynamic: false - } - } -} -parallel_signature { - op_parallel_desc_symbol_id: 4611686018427412479 - bn_in_op2parallel_desc_symbol_id { - key: "in" - value: 4611686018427412479 - } - bn_in_op2parallel_desc_symbol_id { - key: "out" - value: 4611686018427412479 - } -} -parallel_conf_signature { - op_parallel_conf { - device_name: "@0:0" - device_tag: "cpu" - hierarchy { - dim: 1 - } - } -} -nd_sbp_signature { - bn_in_op2nd_sbp { - key: "in" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } - bn_in_op2nd_sbp { - key: "out" - value { - sbp_parallel { - broadcast_parallel { - } - } - } - } -} - -I20220312 18:18:37.196897 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE -I20220312 18:18:37.196943 1174409 global.h:36] NewGlobal N7oneflow7JobDescE -I20220312 18:18:37.208326 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE -I20220312 18:18:37.213130 1174409 global.h:36] NewGlobal N7oneflow7JobDescE -I20220312 18:18:38.376291 1174409 global.h:36] NewGlobal N7oneflow7OpGraphE -I20220312 18:18:38.393191 1174409 global.h:43] DeleteGlobal N7oneflow7OpGraphE -I20220312 18:18:38.396402 1174409 nn_graph.cpp:265] Graph name: LinearGraphDestruct_0 compile time: 1.18321 seconds. -I20220312 18:18:38.399426 1174409 runtime_context.cpp:21] NewCounter constructing_actor_cnt 26 -I20220312 18:18:38.402170 1174805 naive_actor.cpp:25] actor 1099532599296 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.402868 1174809 naive_actor.cpp:25] actor 1099522113536 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.403072 1174808 naive_actor.cpp:25] actor 1099526307840 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.403460 1174802 wait_and_send_ids_actor.cpp:53] actor 1099647942656 switch to &WaitAndSendIdsActor::HandlerWaitToStart -I20220312 18:18:38.403594 1174804 sink_actor.cpp:21] actor 1099650039808 switch to &SinkActor::HandlerNormal -I20220312 18:18:38.403934 1174807 naive_actor.cpp:25] actor 1099520016384 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.404458 1174811 naive_actor.cpp:25] actor 1099524210688 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.402642 1174801 thread.cpp:101] Thread 524352 construct LightActor kTick 1099645845509 -I20220312 18:18:38.404510 1174811 thread.cpp:98] Thread 524294 construct Actor kDeviceTick 1099524210688 -I20220312 18:18:38.403481 1174802 thread.cpp:98] Thread 524353 construct Actor kWaitAndSendIds 1099647942656 -I20220312 18:18:38.402925 1174809 thread.cpp:98] Thread 524293 construct Actor kDeviceTick 1099522113536 -I20220312 18:18:38.403652 1174804 thread.cpp:98] Thread 524354 construct Actor kCallbackNotify 1099650039808 -I20220312 18:18:38.403141 1174808 thread.cpp:98] Thread 524295 construct Actor kDeviceTick 1099526307840 -I20220312 18:18:38.403987 1174807 thread.cpp:98] Thread 524292 construct Actor kDeviceTick 1099520016384 -I20220312 18:18:38.404947 1174802 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 25 -I20220312 18:18:38.404937 1174816 thread.cpp:101] Thread 524288 construct LightActor kNormalForward 1099511627776 -I20220312 18:18:38.404438 1174813 thread.cpp:101] Thread 524291 construct LightActor kNormalForward 1099517919232 -I20220312 18:18:38.402662 1174803 thread.cpp:101] Thread 524289 construct LightActor kNormalForward 1099513724928 -I20220312 18:18:38.405063 1174809 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 22 -I20220312 18:18:38.405038 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 24 -I20220312 18:18:38.405054 1174811 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 23 -I20220312 18:18:38.405068 1174804 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 21 -I20220312 18:18:38.405153 1174807 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 20 -I20220312 18:18:38.405160 1174808 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 19 -I20220312 18:18:38.405215 1174816 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 18 -I20220312 18:18:38.405194 1174815 naive_actor.cpp:25] actor 1099530502144 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.402251 1174805 thread.cpp:98] Thread 524298 construct Actor kDeviceTick 1099532599296 -I20220312 18:18:38.405298 1174815 thread.cpp:98] Thread 524297 construct Actor kDeviceTick 1099530502144 -I20220312 18:18:38.405289 1174813 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 16 -I20220312 18:18:38.405282 1174803 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 17 -I20220312 18:18:38.405474 1174805 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 15 -I20220312 18:18:38.405505 1174815 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 14 -I20220312 18:18:38.406764 1174801 naive_actor.cpp:25] actor 1099645845512 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.406805 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845512 -I20220312 18:18:38.407002 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 13 -I20220312 18:18:38.407407 1174810 naive_actor.cpp:25] actor 1099652136960 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.407409 1174806 naive_actor.cpp:25] actor 1099534696448 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.407465 1174810 thread.cpp:98] Thread 524355 construct Actor kCriticalSectionWaitTick 1099652136960 -I20220312 18:18:38.407501 1174806 thread.cpp:98] Thread 524299 construct Actor kNormalForward 1099534696448 -I20220312 18:18:38.407704 1174810 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 12 -I20220312 18:18:38.407716 1174806 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 11 -I20220312 18:18:38.407800 1174814 thread.cpp:101] Thread 524290 construct LightActor kNormalForward 1099515822080 -I20220312 18:18:38.408062 1174814 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 10 -I20220312 18:18:38.408448 1174800 naive_actor.cpp:25] actor 1099528404992 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.408502 1174800 thread.cpp:98] Thread 524296 construct Actor kNormalForward 1099528404992 -I20220312 18:18:38.408633 1174801 naive_actor.cpp:25] actor 1099645845505 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.408670 1174801 thread.cpp:98] Thread 524352 construct Actor kTick 1099645845505 -I20220312 18:18:38.408730 1174800 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 9 -I20220312 18:18:38.408787 1174812 naive_actor.cpp:25] actor 1099654234112 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.408847 1174812 thread.cpp:98] Thread 524356 construct Actor kCriticalSectionWaitTick 1099654234112 -I20220312 18:18:38.408864 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 8 -I20220312 18:18:38.409101 1174812 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 7 -I20220312 18:18:38.410570 1174801 naive_actor.cpp:25] actor 1099645845513 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.410610 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845513 -I20220312 18:18:38.410810 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 6 -I20220312 18:18:38.412608 1174801 naive_actor.cpp:25] actor 1099645845511 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.412645 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845511 -I20220312 18:18:38.412847 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 5 -I20220312 18:18:38.414455 1174801 naive_actor.cpp:25] actor 1099645845507 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.414489 1174801 thread.cpp:98] Thread 524352 construct Actor kTick 1099645845507 -I20220312 18:18:38.414666 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 4 -I20220312 18:18:38.415032 1174801 thread.cpp:101] Thread 524352 construct LightActor kTick 1099645845508 -I20220312 18:18:38.415199 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 3 -I20220312 18:18:38.416571 1174801 naive_actor.cpp:25] actor 1099645845504 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.416602 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845504 -I20220312 18:18:38.416765 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 2 -I20220312 18:18:38.418165 1174801 naive_actor.cpp:25] actor 1099645845510 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.418198 1174801 thread.cpp:98] Thread 524352 construct Actor kSrcSubsetTick 1099645845510 -I20220312 18:18:38.418359 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 1 -I20220312 18:18:38.419966 1174801 naive_actor.cpp:25] actor 1099645845506 switch to &NaiveActor::HandlerNormal -I20220312 18:18:38.419997 1174801 thread.cpp:98] Thread 524352 construct Actor kDstSubsetTick 1099645845506 -I20220312 18:18:38.420188 1174801 runtime_context.cpp:29] DecreaseCounter constructing_actor_cnt, current val is 0 -I20220312 18:18:38.420208 1174409 runtime.cpp:95] Actors on this machine constructed -I20220312 18:18:38.420279 1174409 runtime.cpp:97] Actors on every machine constructed -I20220312 18:18:38.420292 1174409 runtime_context.cpp:21] NewCounter job_0_running_actor_count 26 -I20220312 18:18:38.420327 1174802 wait_and_send_ids_actor.cpp:73] actor 1099647942656 switch to &WaitAndSendIdsActor::HandlerNormal -I20220312 18:18:38.420346 1174409 global.h:43] DeleteGlobal N7oneflow7JobDescE -I20220312 18:18:38.480592 1174409 nn_graph.cpp:72] Graph destructor Try to close c nn graph name LinearGraphDestruct_0. -I20220312 18:18:38.480641 1174409 nn_graph.cpp:78] Try to close c nn graph name LinearGraphDestruct_0. -I20220312 18:18:38.480790 1174802 actor.cpp:399] actor 1099647942656 switch to &Actor::HandlerZombie -I20220312 18:18:38.480824 1174802 thread.cpp:76] thread 524353 deconstruct actor 1099647942656 -I20220312 18:18:38.480865 1174801 actor.cpp:396] actor 1099645845505 switch to nullptr -I20220312 18:18:38.480876 1174810 actor.cpp:396] actor 1099652136960 switch to nullptr -I20220312 18:18:38.480865 1174812 actor.cpp:396] actor 1099654234112 switch to nullptr -I20220312 18:18:38.480914 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845505 -I20220312 18:18:38.480942 1174810 thread.cpp:76] thread 524355 deconstruct actor 1099652136960 -I20220312 18:18:38.480975 1174812 thread.cpp:76] thread 524356 deconstruct actor 1099654234112 -I20220312 18:18:38.481328 1174802 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 25 -I20220312 18:18:38.481544 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 24 -I20220312 18:18:38.481626 1174801 actor.cpp:396] actor 1099645845504 switch to nullptr -I20220312 18:18:38.481642 1174810 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 23 -I20220312 18:18:38.481642 1174812 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 22 -I20220312 18:18:38.481643 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845504 -I20220312 18:18:38.481714 1174807 actor.cpp:396] actor 1099520016384 switch to nullptr -I20220312 18:18:38.481765 1174807 thread.cpp:76] thread 524292 deconstruct actor 1099520016384 -I20220312 18:18:38.481760 1174803 thread.cpp:76] thread 524289 deconstruct actor 1099513724928 -I20220312 18:18:38.482008 1174803 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 21 -I20220312 18:18:38.482120 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 20 -I20220312 18:18:38.482188 1174801 actor.cpp:399] actor 1099645845506 switch to &Actor::HandlerZombie -I20220312 18:18:38.482219 1174801 actor.cpp:396] actor 1099645845507 switch to nullptr -I20220312 18:18:38.482262 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845507 -I20220312 18:18:38.482215 1174807 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 19 -I20220312 18:18:38.482662 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 18 -I20220312 18:18:38.482718 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845508 -I20220312 18:18:38.482865 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 17 -I20220312 18:18:38.482918 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845509 -I20220312 18:18:38.482961 1174804 actor.cpp:396] actor 1099650039808 switch to nullptr -I20220312 18:18:38.482992 1174804 thread.cpp:76] thread 524354 deconstruct actor 1099650039808 -I20220312 18:18:38.483058 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 16 -I20220312 18:18:38.483129 1174801 actor.cpp:396] actor 1099645845510 switch to nullptr -I20220312 18:18:38.483144 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845510 -I20220312 18:18:38.483191 1174811 actor.cpp:396] actor 1099524210688 switch to nullptr -I20220312 18:18:38.483223 1174811 thread.cpp:76] thread 524294 deconstruct actor 1099524210688 -I20220312 18:18:38.483242 1174816 thread.cpp:76] thread 524288 deconstruct actor 1099511627776 -I20220312 18:18:38.483260 1174814 thread.cpp:76] thread 524290 deconstruct actor 1099515822080 -I20220312 18:18:38.483296 1174808 actor.cpp:396] actor 1099526307840 switch to nullptr -I20220312 18:18:38.483330 1174808 thread.cpp:76] thread 524295 deconstruct actor 1099526307840 -I20220312 18:18:38.483377 1174804 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 15 -I20220312 18:18:38.483532 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 14 -I20220312 18:18:38.483613 1174801 actor.cpp:396] actor 1099645845512 switch to nullptr -I20220312 18:18:38.483629 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845512 -I20220312 18:18:38.483675 1174811 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 13 -I20220312 18:18:38.483678 1174815 actor.cpp:396] actor 1099530502144 switch to nullptr -I20220312 18:18:38.483729 1174813 thread.cpp:76] thread 524291 deconstruct actor 1099517919232 -I20220312 18:18:38.483748 1174815 thread.cpp:76] thread 524297 deconstruct actor 1099530502144 -I20220312 18:18:38.483714 1174816 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 12 -I20220312 18:18:38.483773 1174809 actor.cpp:396] actor 1099522113536 switch to nullptr -I20220312 18:18:38.483811 1174808 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 11 -I20220312 18:18:38.483794 1174805 actor.cpp:396] actor 1099532599296 switch to nullptr -I20220312 18:18:38.483832 1174809 thread.cpp:76] thread 524293 deconstruct actor 1099522113536 -I20220312 18:18:38.483884 1174805 thread.cpp:76] thread 524298 deconstruct actor 1099532599296 -I20220312 18:18:38.484004 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 10 -I20220312 18:18:38.484081 1174801 actor.cpp:399] actor 1099645845511 switch to &Actor::HandlerZombie -I20220312 18:18:38.484122 1174801 actor.cpp:399] actor 1099645845513 switch to &Actor::HandlerZombie -I20220312 18:18:38.484148 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845511 -I20220312 18:18:38.484148 1174800 actor.cpp:396] actor 1099528404992 switch to nullptr -I20220312 18:18:38.484184 1174800 thread.cpp:76] thread 524296 deconstruct actor 1099528404992 -I20220312 18:18:38.484184 1174806 actor.cpp:396] actor 1099534696448 switch to nullptr -I20220312 18:18:38.484220 1174806 thread.cpp:76] thread 524299 deconstruct actor 1099534696448 -I20220312 18:18:38.484237 1174814 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 9 -I20220312 18:18:38.484272 1174815 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 7 -I20220312 18:18:38.484288 1174805 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 6 -I20220312 18:18:38.484295 1174809 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 5 -I20220312 18:18:38.484272 1174813 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 8 -I20220312 18:18:38.484560 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 4 -I20220312 18:18:38.484597 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845513 -I20220312 18:18:38.484836 1174806 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 3 -I20220312 18:18:38.484954 1174800 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 2 -I20220312 18:18:38.484987 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 1 -I20220312 18:18:38.485024 1174801 thread.cpp:76] thread 524352 deconstruct actor 1099645845506 -I20220312 18:18:38.485464 1174801 runtime_context.cpp:29] DecreaseCounter job_0_running_actor_count, current val is 0 -I20220312 18:18:38.485555 1174409 nn_graph.cpp:82] Finish close c nn graph name LinearGraphDestruct_0. -I20220312 18:18:38.485572 1174409 multi_client_session_context.cpp:157] Try to delete multi client session context. -I20220312 18:18:38.485576 1174409 global.h:43] DeleteGlobal N7oneflow6boxing10collective9SchedulerE -I20220312 18:18:38.485613 1174409 global.h:43] DeleteGlobal N7oneflow7summary12EventsWriterE -I20220312 18:18:38.485626 1174409 global.h:43] DeleteGlobal N7oneflow15RuntimeJobDescsE -I20220312 18:18:38.485689 1174409 global.h:43] DeleteGlobal N7oneflow9ThreadMgrE -I20220312 18:18:38.485827 1174409 thread_manager.cpp:29] actor thread 524288 finish -I20220312 18:18:38.485935 1174409 thread_manager.cpp:29] actor thread 524290 finish -I20220312 18:18:38.486022 1174409 thread_manager.cpp:29] actor thread 524291 finish -I20220312 18:18:38.486085 1174409 thread_manager.cpp:29] actor thread 524289 finish -I20220312 18:18:38.486219 1174409 thread_manager.cpp:29] actor thread 524296 finish -I20220312 18:18:38.486351 1174409 thread_manager.cpp:29] actor thread 524354 finish -I20220312 18:18:38.486469 1174409 thread_manager.cpp:29] actor thread 524299 finish -I20220312 18:18:38.486572 1174409 thread_manager.cpp:29] actor thread 524292 finish -I20220312 18:18:38.486670 1174409 thread_manager.cpp:29] actor thread 524353 finish -I20220312 18:18:38.486773 1174409 thread_manager.cpp:29] actor thread 524295 finish -I20220312 18:18:38.486891 1174409 thread_manager.cpp:29] actor thread 524293 finish -I20220312 18:18:38.486994 1174409 thread_manager.cpp:29] actor thread 524297 finish -I20220312 18:18:38.487115 1174409 thread_manager.cpp:29] actor thread 524355 finish -I20220312 18:18:38.487239 1174409 thread_manager.cpp:29] actor thread 524352 finish -I20220312 18:18:38.487354 1174409 thread_manager.cpp:29] actor thread 524294 finish -I20220312 18:18:38.487460 1174409 thread_manager.cpp:29] actor thread 524298 finish -I20220312 18:18:38.487589 1174409 thread_manager.cpp:29] actor thread 524356 finish -I20220312 18:18:38.487601 1174409 global.h:43] DeleteGlobal N7oneflow11ActorMsgBusE -I20220312 18:18:38.487610 1174409 global.h:43] DeleteGlobal N7oneflow8RegstMgrE -I20220312 18:18:38.487810 1174409 global.h:43] DeleteGlobal N7oneflow8ChunkMgrE -I20220312 18:18:38.487820 1174409 global.h:43] DeleteGlobal N7oneflow15MemoryAllocatorE -I20220312 18:18:38.487974 1174409 global.h:43] DeleteGlobal N7oneflow10RuntimeCtxE -I20220312 18:18:38.487984 1174409 global.h:43] DeleteGlobal N7oneflow9BufferMgrISt10shared_ptrINS_23CriticalSectionInstanceEEEE -I20220312 18:18:38.488000 1174409 global.h:43] DeleteGlobal N7oneflow9BufferMgrISt10shared_ptrINS_11JobInstanceEEEE -I20220312 18:18:38.488014 1174409 global.h:43] DeleteGlobal N7oneflow26LazyJobBuildAndInferCtxMgrE -I20220312 18:18:38.488508 1174409 global.h:43] DeleteGlobal N7oneflow22TaskStreamIndexManagerE -I20220312 18:18:38.488531 1174409 global.h:43] DeleteGlobal N7oneflow5IDMgrE -I20220312 18:18:38.488552 1174409 global.h:43] DeleteGlobal N7oneflow12ResourceDescE -I20220312 18:18:38.488564 1174409 global.h:36] NewGlobal N7oneflow12ResourceDescE -I20220312 18:18:38.488585 1174409 multi_client_session_context.cpp:184] Finish delete multi client session context. -I20220312 18:18:38.488593 1174409 env_global_objects_scope.cpp:242] Try to close env global objects scope. -I20220312 18:18:38.488628 1174409 global.h:43] DeleteGlobal N7oneflow14KernelObserverE -I20220312 18:18:38.488641 1174409 global.h:43] DeleteGlobal N7oneflow9TransportE -I20220312 18:18:38.488854 1174409 global.h:43] DeleteGlobal N7oneflow12EpollCommNetE -I20220312 18:18:38.488862 1174409 epoll_comm_network.cpp:87] CommNet Thread 0 finish -I20220312 18:18:38.488978 1174409 epoll_comm_network.cpp:87] CommNet Thread 1 finish -I20220312 18:18:38.489063 1174409 epoll_comm_network.cpp:87] CommNet Thread 2 finish -I20220312 18:18:38.489166 1174409 epoll_comm_network.cpp:87] CommNet Thread 3 finish -I20220312 18:18:38.489331 1174409 global.h:43] DeleteGlobal N7oneflow27EagerJobBuildAndInferCtxMgrE -I20220312 18:18:38.489342 1174409 global.h:43] DeleteGlobal N7oneflow2vm19VirtualMachineScopeE -I20220312 18:18:38.489351 1174409 global.h:43] DeleteGlobal N7oneflow14VirtualMachineE From 19be1099ab485771736d6c41f4495e7b5919b774 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 19:38:58 +0800 Subject: [PATCH 22/54] rm useless code --- cmake/oneflow.cmake | 2 +- oneflow/api/cpp/env_impl.h | 2 +- oneflow/api/cpp/tests/graph_test.cpp | 4 +--- oneflow/core/ep/cuda/cuda_stream.cpp | 5 +---- oneflow/core/framework/multi_client_session_context.cpp | 2 -- python/oneflow/framework/multi_client_session.py | 6 ------ python/oneflow/nn/graph/graph.py | 3 +-- 7 files changed, 5 insertions(+), 19 deletions(-) diff --git a/cmake/oneflow.cmake b/cmake/oneflow.cmake index f5bb6f1b811..274f2023c94 100644 --- a/cmake/oneflow.cmake +++ b/cmake/oneflow.cmake @@ -440,7 +440,7 @@ if(BUILD_TESTING) WORKING_DIRECTORY ${PROJECT_SOURCE_DIR}) find_package(Threads REQUIRED) - target_link_libraries(oneflow_cpp_api_testexe oneflow_cpp ${oneflow_test_libs} glog::glog Threads::Threads) + target_link_libraries(oneflow_cpp_api_testexe oneflow_cpp ${oneflow_test_libs} Threads::Threads) endif() endif() diff --git a/oneflow/api/cpp/env_impl.h b/oneflow/api/cpp/env_impl.h index 9e645f29276..8524db790d9 100644 --- a/oneflow/api/cpp/env_impl.h +++ b/oneflow/api/cpp/env_impl.h @@ -35,4 +35,4 @@ class OneFlowEnv { }; } // namespace oneflow_api -#endif // ONEFLOW_API_CPP_ENV_IMPL_H_ \ No newline at end of file +#endif // ONEFLOW_API_CPP_ENV_IMPL_H_ diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index 8825707feca..0072a3494fa 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -32,9 +32,7 @@ namespace oneflow_api { namespace { inline Graph LoadGraph(const Device& device) { - Graph graph = Graph::Load( - "/home/xuxiaoyu/dev/oneflow/oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", - device); + Graph graph = Graph::Load("./oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); return graph; } diff --git a/oneflow/core/ep/cuda/cuda_stream.cpp b/oneflow/core/ep/cuda/cuda_stream.cpp index 61ae1322b44..2acc0a4af7b 100644 --- a/oneflow/core/ep/cuda/cuda_stream.cpp +++ b/oneflow/core/ep/cuda/cuda_stream.cpp @@ -149,10 +149,7 @@ Maybe CudaStream::Sync() { void CudaStream::RecordEvent(Event* event) { auto* cuda_event = static_cast(event); // NOLINT - // OF_CUDA_CHECK(cudaEventRecord(cuda_event->cuda_event(), cuda_stream_)); - if (cudaEventRecord(cuda_event->cuda_event(), cuda_stream_) != cudaSuccess) { - std::cerr << "cuda even record error" << std::endl; - } + OF_CUDA_CHECK(cudaEventRecord(cuda_event->cuda_event(), cuda_stream_)); } cudaStream_t CudaStream::cuda_stream() const { return cuda_stream_; } diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index cab52431256..15c368959b1 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -156,8 +156,6 @@ Maybe MultiClientSessionContext::UpdateResource(const std::string& reso_pr Maybe MultiClientSessionContext::TryClose() { if (is_inited_) { VLOG(1) << "Try to delete multi client session context." << std::endl; - // sync before NNGraph release to ensure LaunchLazyJob instruction was completed and released - // JUST(vm::ClusterSync()); { // NOTE(chengcheng): delete runtime global objects Global::Delete(); diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index f1a5c6b8729..a6fe62116a4 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -48,13 +48,11 @@ def TryInit(self): self._check_status(self.Status.CREATED, self.Status.INITED) if self.status_ == self.Status.CREATED: config_proto_str = text_format.MessageToString(self.config_proto) - # oneflow._oneflow_internal.InitMultiClientSessionContext(config_proto_str) self._session_ctx.try_init(config_proto_str) self.status_ = self.Status.INITED def _TryClose(self): if self.status_ != self.Status.CLOSED: - # oneflow._oneflow_internal.TryDestroyMultiClientSessionContext() oneflow._oneflow_internal.ClearSessionById(self.id) self.status_ = self.Status.CLOSED @@ -122,11 +120,7 @@ def _update_scope_attr_name2defaultVal(self): def update_resource_eagerly(self, resource_config): self._check_status(self.Status.INITED) config_proto_str = text_format.MessageToString(resource_config) - # oneflow._oneflow_internal.MultiClientSessionContextUpdateResource( - # config_proto_str - # ) self._session_ctx.update_resource(config_proto_str) def __del__(self): self._TryClose() - print("oneflow session del") diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 06828d4de7d..3e4729eaf1a 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -13,7 +13,6 @@ See the License for the specific language governing permissions and limitations under the License. """ -import atexit import os import time from collections import OrderedDict @@ -1167,8 +1166,8 @@ def __getattr__(self, name: str): ) def __del__(self): + # Ensure vm has finished running this graph. oneflow._oneflow_internal.eager.Sync() - print(f"oneflow graph {self.name} del") if __name__ == "__main__": From 669dc9c931b2b57a8673efd439055999a8051d08 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 19:42:06 +0800 Subject: [PATCH 23/54] format --- oneflow/api/cpp/tests/graph_test.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index 0072a3494fa..70519a23ffb 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -32,7 +32,8 @@ namespace oneflow_api { namespace { inline Graph LoadGraph(const Device& device) { - Graph graph = Graph::Load("./oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); + Graph graph = + Graph::Load("./oneflow/api/cpp/tests/graph_test_model/affine_with_parameter", device); return graph; } From b0dbcdb341c68ea8f2daf285c64d5ba22bc2eb9d Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 20:32:05 +0800 Subject: [PATCH 24/54] rm useless include --- oneflow/api/cpp/tests/graph_test.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/oneflow/api/cpp/tests/graph_test.cpp b/oneflow/api/cpp/tests/graph_test.cpp index 70519a23ffb..497da6b1bbb 100644 --- a/oneflow/api/cpp/tests/graph_test.cpp +++ b/oneflow/api/cpp/tests/graph_test.cpp @@ -25,7 +25,6 @@ limitations under the License. #include #include "oneflow/api/cpp/framework.h" #include "oneflow/api/cpp/tests/api_test.h" -#include "oneflow/core/vm/vm_util.h" namespace oneflow_api { From 3281975416aa487811a3a342f5718b56a2a79431 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 12 Mar 2022 20:46:06 +0800 Subject: [PATCH 25/54] rm sync in py --- python/oneflow/framework/env_util.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 38e8bb3ad71..f6d7391bb28 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -387,9 +387,7 @@ def __del__(self): # TODO(strint): deal with abnormal exit # if self._is_normal_exit: # del self._env_cxt - oneflow._oneflow_internal.eager.Sync() oneflow._oneflow_internal.SetShuttingDown() - print("oneflow env del") def GetEnv(): From 29ccd30e889318ab39d7e52bab10b17acd665a62 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Mon, 14 Mar 2022 00:11:41 +0800 Subject: [PATCH 26/54] the Env is never destroyed. --- oneflow/api/python/env/env.cpp | 1 - oneflow/api/python/env/env.h | 7 ------ oneflow/api/python/env/env_api.h | 2 -- oneflow/core/vm/virtual_machine.cpp | 14 ++++++++++- oneflow/core/vm/virtual_machine_engine.cpp | 6 ++++- oneflow/core/vm/virtual_machine_engine.h | 1 + python/oneflow/__init__.py | 2 -- .../test/modules/test_sync_after_atexist.py | 24 +++++++++++++++++++ 8 files changed, 43 insertions(+), 14 deletions(-) create mode 100644 python/oneflow/test/modules/test_sync_after_atexist.py diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index d352751ad3c..db44fa16e02 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -26,7 +26,6 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("IsEnvInited", &IsEnvInited); m.def("InitEnv", &InitEnv); - m.def("DestroyEnv", &DestroyEnv, py::call_guard()); m.def("CurrentMachineId", &CurrentMachineId); diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index b65adee28d6..1f21b61f392 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -47,13 +47,6 @@ inline Maybe EnableEagerEnvironment(bool enable_eager_execution) { inline Maybe IsEnvInited() { return Global::Get() != nullptr; } -inline Maybe DestroyEnv() { - if (Global::Get() == nullptr) { return Maybe::Ok(); } - OF_ENV_BARRIER(); - Global::Delete(); - return Maybe::Ok(); -} - inline Maybe InitEnv(const std::string& env_proto_str) { EnvProto env_proto; CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 13702b74bbd..1bab6e0f631 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -32,8 +32,6 @@ inline void InitEnv(const std::string& env_proto_str) { return oneflow::InitEnv(env_proto_str).GetOrThrow(); } -inline void DestroyEnv() { return oneflow::DestroyEnv().GetOrThrow(); } - inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index 45051fc43a4..e4c009f6f49 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -29,6 +29,7 @@ limitations under the License. #include "oneflow/core/profiler/profiler.h" #include "oneflow/core/platform/include/pthread_fork.h" #include "oneflow/core/common/env_var.h" +#include "oneflow/core/framework/shut_down_util.h" #include "oneflow/core/framework/device.h" namespace oneflow { @@ -115,7 +116,9 @@ VirtualMachine::VirtualMachine(const Resource& resource, int64_t this_machine_id // In order to notify threads in VirtualMachineEngine, a notify callback lambda should be take as // an argument for VirtualMachineEngine's constructor. vm_ = intrusive::make_shared( - vm::MakeVmDesc(resource, this_machine_id).Get(), [this]() { callback_notifier_.Notify(); }); + vm::MakeVmDesc(resource, this_machine_id).Get(), [this]() { + if (!IsShuttingDown()) { callback_notifier_.Notify(); } + }); OF_PROFILER_NAME_THIS_HOST_THREAD("_Main"); std::function WorkerInitializer; GetWorkerThreadInitializer(vm_, &WorkerInitializer); @@ -199,6 +202,15 @@ Maybe VirtualMachine::Receive(vm::InstructionMsgList* instr_list) { // `ComputeInFuseMode` will be replaced by `Compute` soon. instr_msg->mut_instr_type_id()->instruction_type().ComputeInFuseMode(instr_msg); } + } else if (IsShuttingDown()) { + CHECK_OR_RETURN(vm_->Empty()); + CHECK_OR_RETURN(vm_->CallbackEmpty()); + JUST(vm_->Receive(instr_list)); + while (!(vm_->Empty() && vm_->CallbackEmpty())) { + vm_->Schedule(); + vm_->FlushGarbageMsgList(); + vm_->Callback(); + } } else { const int64_t kHighWaterMark = GetInstructionHighWaterMark(); if (vm_->flying_instruction_cnt() > kHighWaterMark) { diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 0444d815900..618a5301245 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -204,8 +204,12 @@ void VirtualMachineEngine::MoveInstructionMsgToGarbageMsgList( if (unlikely(local_garbage_msg_list_.size() > kWindowSize)) { MoveToGarbageMsgListAndNotifyGC(); } } -void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC() { +void VirtualMachineEngine::FlushGarbageMsgList() { garbage_msg_list_.MoveFrom(&local_garbage_msg_list_); +} + +void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC() { + FlushGarbageMsgList(); notify_callback_thread_(); } diff --git a/oneflow/core/vm/virtual_machine_engine.h b/oneflow/core/vm/virtual_machine_engine.h index 41c0f343c16..ecb0e52a697 100644 --- a/oneflow/core/vm/virtual_machine_engine.h +++ b/oneflow/core/vm/virtual_machine_engine.h @@ -104,6 +104,7 @@ class VirtualMachineEngine final : public intrusive::Base { // Returns true if old pending_instruction_list is empty Maybe Receive(intrusive::shared_ptr&& instruction_msg); void Schedule(); + void FlushGarbageMsgList(); void Callback(); void NotifyCallback(); bool ThreadUnsafeEmpty() const; diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index dd1b260b582..eed248a2750 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -261,8 +261,6 @@ def atexit_hook(hook): if oneflow._oneflow_internal.IsEnvInited(): oneflow._oneflow_internal.eager.Sync() oneflow.framework.session_context.TryCloseDefaultSession() - if hook.is_normal_exit(): - oneflow._oneflow_internal.DestroyEnv() oneflow._oneflow_internal.SetShuttingDown() diff --git a/python/oneflow/test/modules/test_sync_after_atexist.py b/python/oneflow/test/modules/test_sync_after_atexist.py new file mode 100644 index 00000000000..0aaf7f2639d --- /dev/null +++ b/python/oneflow/test/modules/test_sync_after_atexist.py @@ -0,0 +1,24 @@ +""" +Copyright 2020 The OneFlow 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 oneflow + + +class TestSyncLate: + def __del__(self): + oneflow._oneflow_internal.eager.Sync() + + +test_case = TestSyncLate() From 8509484ea27698ba1aaaada1461be2e4e005df38 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Mon, 14 Mar 2022 17:40:43 +0800 Subject: [PATCH 27/54] export Env into python --- oneflow/api/python/env/env.cpp | 7 +- oneflow/api/python/env/env.h | 13 +- oneflow/api/python/env/env_api.h | 6 - oneflow/api/python/init.cpp | 6 - oneflow/api/python/session/session.h | 1 - .../core/framework/random_generator_impl.cpp | 4 +- python/oneflow/__init__.py | 10 +- python/oneflow/env.py | 11 - python/oneflow/framework/c_api_util.py | 4 +- python/oneflow/framework/env_util.py | 208 +----------------- python/oneflow/framework/hob.py | 6 - python/oneflow/framework/tensor.py | 1 + python/oneflow/framework/unittest.py | 129 ----------- python/oneflow/serving/inference_session.py | 3 +- 14 files changed, 25 insertions(+), 384 deletions(-) diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index db44fa16e02..879c837635a 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -24,8 +24,11 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("EnvResource", &EnvResource); m.def("EnableEagerEnvironment", &EnableEagerEnvironment); - m.def("IsEnvInited", &IsEnvInited); - m.def("InitEnv", &InitEnv); + using Env = oneflow::EnvGlobalObjectsScope; + py::class_>(m, "Env").def( + py::init([](const std::string& env_proto_str) { + return oneflow::CreateEnv(env_proto_str).GetPtrOrThrow(); + })); m.def("CurrentMachineId", &CurrentMachineId); diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index 1f21b61f392..f0e1b200283 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -45,18 +45,13 @@ inline Maybe EnableEagerEnvironment(bool enable_eager_execution) { return Maybe::Ok(); } -inline Maybe IsEnvInited() { return Global::Get() != nullptr; } - -inline Maybe InitEnv(const std::string& env_proto_str) { +inline Maybe CreateEnv(const std::string& env_proto_str) { EnvProto env_proto; CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) << "failed to parse env_proto" << env_proto_str; - CHECK_ISNULL_OR_RETURN(Global::Get()); - // Global::New is not allowed to be called here - // because glog is not constructed yet and has bad bahavior. - Global::SetAllocated(new EnvGlobalObjectsScope()); - JUST(Global::Get()->Init(env_proto)); - return Maybe::Ok(); + auto env = std::make_shared(); + JUST(env->Init(env_proto)); + return env; } inline Maybe CurrentMachineId() { return GlobalProcessCtx::Rank(); } diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 1bab6e0f631..2946924bd70 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -26,12 +26,6 @@ inline void EnableEagerEnvironment(bool enable_eager_execution) { return oneflow::EnableEagerEnvironment(enable_eager_execution).GetOrThrow(); } -inline bool IsEnvInited() { return oneflow::IsEnvInited().GetOrThrow(); } - -inline void InitEnv(const std::string& env_proto_str) { - return oneflow::InitEnv(env_proto_str).GetOrThrow(); -} - inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } diff --git a/oneflow/api/python/init.cpp b/oneflow/api/python/init.cpp index 48dc3abbec0..a85309eaedf 100644 --- a/oneflow/api/python/init.cpp +++ b/oneflow/api/python/init.cpp @@ -51,12 +51,6 @@ bool Int2IntListMapContaining(const Int2IntListMap& bigger, const Int2IntListMap } // namespace PYBIND11_MODULE(_oneflow_internal, m) { - m.def("MasterSendAbort", []() { - if (Global::Get() != nullptr) { - return ClusterInstruction::MasterSendAbort(); - } - }); - using IntList = std::vector; using Int2IntListMap = std::unordered_map>; diff --git a/oneflow/api/python/session/session.h b/oneflow/api/python/session/session.h index e1c25f1a76b..2b4b9e10643 100644 --- a/oneflow/api/python/session/session.h +++ b/oneflow/api/python/session/session.h @@ -119,7 +119,6 @@ inline Maybe CreateMultiClientSessionContext() { inline Maybe InitMultiClientSessionContext(const std::string& config_proto_str) { CHECK_NOTNULL_OR_RETURN(Global::Get()); - CHECK_NOTNULL_OR_RETURN(Global::Get()); CHECK_NOTNULL_OR_RETURN(Global::Get()) << "env not found"; ConfigProto config_proto; diff --git a/oneflow/core/framework/random_generator_impl.cpp b/oneflow/core/framework/random_generator_impl.cpp index 117eba9ecd3..81bfc86e90a 100644 --- a/oneflow/core/framework/random_generator_impl.cpp +++ b/oneflow/core/framework/random_generator_impl.cpp @@ -20,7 +20,7 @@ limitations under the License. #include "oneflow/core/framework/instructions_builder.h" #include "oneflow/core/framework/tensor_util.h" #include "oneflow/core/functional/functional.h" -#include "oneflow/core/job/env_global_objects_scope.h" +#include "oneflow/core/vm/virtual_machine.h" #include "oneflow/core/register/ofblob.h" #include "oneflow/core/vm/vm_util.h" #ifdef WITH_CUDA @@ -35,7 +35,7 @@ namespace one { namespace { Maybe CPUSynchronize() { - if (Global::Get() != nullptr) { return vm::CurrentRankSync(); } + if (Global::Get() != nullptr) { return vm::CurrentRankSync(); } return Maybe::Ok(); } diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index eed248a2750..f0a754c2b19 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -210,7 +210,7 @@ def is_deprecated(func_or_class): if not env_util.HasAllMultiClientEnvVars(): env_util.SetDefaultMultiClientEnvVars() -env_util.api_env_init() +_oneflow_global_unique_env_ = env_util.create_env() oneflow._oneflow_internal.RegisterGILForeignLockHelper() oneflow._oneflow_internal.InitDefaultConsistentTransportTokenScope() session_ctx.OpenDefaultSession( @@ -258,8 +258,7 @@ def is_normal_exit(self): def atexit_hook(hook): if hook.is_normal_exit(): - if oneflow._oneflow_internal.IsEnvInited(): - oneflow._oneflow_internal.eager.Sync() + oneflow._oneflow_internal.eager.Sync() oneflow.framework.session_context.TryCloseDefaultSession() oneflow._oneflow_internal.SetShuttingDown() @@ -281,9 +280,6 @@ def atexit_hook(hook): from oneflow.framework.check_point_v2 import load from oneflow.framework.check_point_v2 import save from oneflow.framework.dtype import convert_oneflow_dtype_to_numpy_dtype, dtypes -from oneflow.framework.env_util import ( - api_enable_eager_execution as enable_eager_execution, -) from oneflow.framework.function_util import FunctionConfig from oneflow.framework.function_util import FunctionConfig as function_config from oneflow.framework.generator import create_generator as Generator @@ -394,7 +390,7 @@ def atexit_hook(hook): boxing, backends, amp, -) # , saved_model NOTE(chengcheng): unavailable now +) import oneflow.utils.data import oneflow.comm import oneflow.framework.docstr as docstr diff --git a/python/oneflow/env.py b/python/oneflow/env.py index c8fed89968f..4e67fc7a0cd 100644 --- a/python/oneflow/env.py +++ b/python/oneflow/env.py @@ -13,17 +13,6 @@ See the License for the specific language governing permissions and limitations under the License. """ -from oneflow.framework.env_util import api_all_device_placement as all_device_placement -from oneflow.framework.env_util import api_ctrl_port as ctrl_port -from oneflow.framework.env_util import api_data_port as data_port -from oneflow.framework.env_util import api_env_init as init -from oneflow.framework.env_util import api_grpc_use_no_signal as grpc_use_no_signal -from oneflow.framework.env_util import api_init_bootstrap_confs as init_bootstrap_confs -from oneflow.framework.env_util import api_log_dir as log_dir -from oneflow.framework.env_util import api_logbuflevel as logbuflevel -from oneflow.framework.env_util import api_logtostderr as logtostderr -from oneflow.framework.env_util import api_machine as machine - import oneflow._oneflow_internal diff --git a/python/oneflow/framework/c_api_util.py b/python/oneflow/framework/c_api_util.py index 2382a3e2b4a..cf13851b7d5 100644 --- a/python/oneflow/framework/c_api_util.py +++ b/python/oneflow/framework/c_api_util.py @@ -42,10 +42,10 @@ def EnvResource(): return text_format.Parse(resource, resource_util.Resource()) -def InitEnv(env_proto): +def CreateEnv(env_proto): assert type(env_proto) is env_pb2.EnvProto env_proto_str = text_format.MessageToString(env_proto) - oneflow._oneflow_internal.InitEnv(env_proto_str) + return oneflow._oneflow_internal.Env(env_proto_str) def InitLazyGlobalSession(config_proto): diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index 209db0059f1..daefabd483f 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -23,11 +23,8 @@ import oneflow.core.job.env_pb2 as env_pb import oneflow.core.job.resource_pb2 as resource_util import oneflow.framework.c_api_util as c_api_util -import oneflow.framework.hob as hob import oneflow.framework.scope_util as scope_util import oneflow.framework.session_context as session_ctx -import oneflow.support.enable_if as enable_if -from oneflow import oneflow_deprecate def api_all_device_placement(device_type: str) -> oneflow._oneflow_internal.placement: @@ -51,29 +48,6 @@ def api_all_device_placement(device_type: str) -> oneflow._oneflow_internal.plac return oneflow._oneflow_internal.AllDevicePlacement(device_type) -def api_enable_eager_execution(val: bool = True) -> None: - """If True, job will execute in eager mode, else use lazy mode(static graph). - - Args: - val (bool, optional): Whether eager execution or not. Defaults to True. - """ - return enable_if.unique([enable_eager_environment])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.any_global_function_defined) -def enable_eager_environment(val=True): - return oneflow._oneflow_internal.EnableEagerEnvironment(val) - - -def api_env_init() -> bool: - """Init environment for job - - Returns: - bool: [description] - """ - return enable_if.unique([env_init, do_nothing])() - - def check_non_localhost_proxy_and_print_warning(): for env_var_name in ["http_proxy", "HTTP_PROXY", "https_proxy", "HTTPS_PROXY"]: env_var_value = os.getenv(env_var_name) @@ -90,137 +64,18 @@ def check_non_localhost_proxy_and_print_warning(): break -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def env_init(): +def create_env(): + """create environment + + Returns: + Env: [description] + """ global default_env_proto assert len(default_env_proto.machine) > 0 CompleteEnvProto(default_env_proto) if default_env_proto.ctrl_bootstrap_conf.world_size > 1: check_non_localhost_proxy_and_print_warning() - c_api_util.InitEnv(default_env_proto) - return True - - -def api_machine(*val: list) -> None: - """Set machines' hostnames. - - For instance: - - oneflow.env.machine([{"addr": "192.168.1.1"}, {"addr": "192.168.1.2"}]) - - Args: - val: `list`, `tuple` or multiple arguments of `dict`. First in the list is the master machine. - """ - return enable_if.unique([machine, do_nothing])(*val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def machine(*val): - del default_env_proto.machine[:] - if len(val) == 1 and isinstance(val[0], (list, tuple)): - val = val[0] - default_env_proto.ClearField("machine") - default_env_proto.machine.extend(_MakeMachine(val)) - - -def api_ctrl_port(val: int) -> None: - """Set port number used to control the execution across multiple machines. Same on every machine. - - Args: - val: a port number accessible to peer machines - """ - return enable_if.unique([ctrl_port, do_nothing])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def ctrl_port(val): - assert type(val) is int - default_env_proto.ctrl_port = val - - -def api_data_port(val: int) -> None: - """Set port number used to data transfer among multiple machines. Same on every machine. - - Args: - val: a port number accessible to peer machines - """ - return enable_if.unique([data_port, do_nothing])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def data_port(val): - assert type(val) is int - default_env_proto.data_port = val - - -from oneflow import oneflow_deprecate - - -@oneflow_deprecate() -def api_grpc_use_no_signal(val: bool = True) -> None: - """Set rpc use signal or not (deprecate) - - Args: - val (bool, optional): True or False. Defaults to True. - """ - print( - "WARNING:", - "oneflow.env.grpc_use_no_signal is deprecated, users no longer need to set rpc use signal or not. \n", - traceback.format_stack()[-2], - ) - return None - - -def api_log_dir(val: str) -> None: - """Specify a dir to store OneFlow's logging files. If not specified, it is `./log` by default. - - Args: - val (str): string , log file path - """ - return enable_if.unique([log_dir, do_nothing])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def log_dir(val): - assert type(val) is str - default_env_proto.cpp_logging_conf.log_dir = val - - -def api_logtostderr(val: int) -> None: - """Set whether log messages go to stderr instead of logfiles - - Args: - val (int): [description] - """ - return enable_if.unique([logtostderr, do_nothing])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def logtostderr(val): - assert type(val) is int - default_env_proto.cpp_logging_conf.logtostderr = val - - -def api_logbuflevel(val: int) -> None: - """Log messages at a level <= this flag are buffered. - Log messages at a higher level are flushed immediately. - - Args: - val (int): int, number of level - """ - return enable_if.unique([logbuflevel, do_nothing])(val) - - -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def logbuflevel(val): - assert type(val) is int - default_env_proto.cpp_logging_conf.logbuflevel = val - - -@enable_if.condition(hob.in_normal_mode & hob.env_initialized) -def do_nothing(*args, **kwargs): - print("Environment has been initialized, this env init will do nothing.") - return False + return c_api_util.CreateEnv(default_env_proto) def CompleteEnvProto(env_proto): @@ -261,10 +116,6 @@ def _MakeMachine(machines): return rp_machine -def api_init_bootstrap_confs(*val: list, **kargs) -> None: - return enable_if.unique([MakeBootstrapConfs, do_nothing])(*val, **kargs) - - def _MakeBootstrapConf(bootstrap_info: dict): global config_master_addr assert config_master_addr.HasField("host"), "must config master host first" @@ -286,51 +137,6 @@ def _MakeBootstrapConf(bootstrap_info: dict): return bootstrap_conf -@enable_if.condition(hob.in_normal_mode & ~hob.env_initialized) -def MakeBootstrapConfs( - node_list, master_port, world_size=0, ctrl_port=-1, node_size=-1 -): - """Set ctrl_bootstrap_conf' info. - - For instance: - - ONEFLOW_TEST_NODE_LIST=192.168.1.16,192.168.1.15 ONEFLOW_TEST_MASTER_PORT=43256 - ONEFLOW_TEST_WORLD_SIZE=2 ONEFLOW_TEST_RANK_CTRL_PORT=34527 - - Args: - val: `list`, First in the list is the master machine. - """ - if isinstance(node_list, str): - node_list = [node_list] - global global_ctrl_bootstrap_confs - assert len(global_ctrl_bootstrap_confs) == 0, "ctrl_bootstrap_conf has been inited" - global config_master_addr - config_master_addr.host = node_list[0] - config_master_addr.port = master_port - global config_world_size - if world_size == 0: - config_world_size = len(node_list) - else: - assert world_size % len(node_list) == 0 - config_world_size = world_size - global config_bootstrap_ctrl_port - if ctrl_port != -1: - config_bootstrap_ctrl_port = ctrl_port - global config_node_size - if node_size != -1: - config_node_size = node_size - rank = 0 - for rank_host in node_list: - assert isinstance(rank_host, str) - bootstrap_conf = _MakeBootstrapConf({"rank": rank, "host": rank_host}) - if rank == 0: - global default_env_proto - default_env_proto.ctrl_bootstrap_conf.CopyFrom(bootstrap_conf) - global_ctrl_bootstrap_confs.append(bootstrap_conf) - rank += 1 - return global_ctrl_bootstrap_confs - - def _DefaultEnvProto(): env_proto = env_pb.EnvProto() machine = env_proto.machine.add() diff --git a/python/oneflow/framework/hob.py b/python/oneflow/framework/hob.py index 2520e080b05..ad0c9a2a0a2 100644 --- a/python/oneflow/framework/hob.py +++ b/python/oneflow/framework/hob.py @@ -35,12 +35,6 @@ def in_device_mode(ctx): return rt_mode.CurrentMode() == rt_mode.DEVICE_MODE -@bool_functor("Environment initialized") -def env_initialized(ctx): - assert in_normal_mode(ctx) - return oneflow._oneflow_internal.IsEnvInited() - - @bool_functor("Any global function defined") def any_global_function_defined(ctx): assert in_normal_mode(ctx) diff --git a/python/oneflow/framework/tensor.py b/python/oneflow/framework/tensor.py index be71dc83c12..7c9f85d5823 100644 --- a/python/oneflow/framework/tensor.py +++ b/python/oneflow/framework/tensor.py @@ -23,6 +23,7 @@ import numpy as np from typing import Union +import oneflow._oneflow_internal Tensor = flow._oneflow_internal.Tensor TensorTuple = flow._oneflow_internal.TensorTuple diff --git a/python/oneflow/framework/unittest.py b/python/oneflow/framework/unittest.py index 6ec5e9c2703..5d173624dee 100644 --- a/python/oneflow/framework/unittest.py +++ b/python/oneflow/framework/unittest.py @@ -30,7 +30,6 @@ import oneflow import oneflow.env -import oneflow.framework.env_util as env_util import oneflow.sysconfig from oneflow.core.job.env_pb2 import EnvProto @@ -144,7 +143,6 @@ def find_free_port(): return s.getsockname()[1] -_unittest_env_initilized = False _unittest_worker_initilized = False @@ -197,133 +195,6 @@ def launch_worker_via_agent(host=None, env_proto=None): conn.close() -class TestCase(unittest.TestCase): - def setUp(self): - global _unittest_env_initilized - global _unittest_worker_initilized - if has_node_list(): - assert node_size() > 1 - if _unittest_worker_initilized == False: - master_port = os.getenv("ONEFLOW_TEST_MASTER_PORT") - assert master_port, "env var ONEFLOW_TEST_MASTER_PORT not set" - oneflow.env.ctrl_port(int(master_port)) - data_port = os.getenv("ONEFLOW_TEST_DATA_PORT") - if data_port: - oneflow.env.data_port(int(data_port)) - if enable_init_by_host_list(): - oneflow.env.machine(node_list()) - data_port = os.getenv("ONEFLOW_TEST_DATA_PORT") - print("initializing worker...") - for machine in env_util.default_env_proto.machine: - if machine.id == 0: - pass - else: - launch_worker_via_agent( - host=machine.addr, env_proto=env_util.default_env_proto - ) - else: - ctrl_port = os.getenv("ONEFLOW_TEST_CTRL_PORT") - config_rank_ctrl_port = -1 - if ctrl_port: - config_rank_ctrl_port = int(ctrl_port) - if has_world_size(): - config_world_size = world_size() - else: - config_world_size = 0 - config_node_size = -1 - env_node_size = os.getenv("ONEFLOW_TEST_NODE_SIZE") - if env_node_size: - config_node_size = int(env_node_size) - bootstrap_conf_list = oneflow.env.init_bootstrap_confs( - node_list(), - int(master_port), - config_world_size, - config_rank_ctrl_port, - config_node_size, - ) - worker_env_proto = EnvProto() - worker_env_proto.CopyFrom(env_util.default_env_proto) - worker_env_proto.ClearField("ctrl_bootstrap_conf") - for bootstrap_conf in bootstrap_conf_list: - if bootstrap_conf.rank == 0: - continue - assert bootstrap_conf.HasField("host") - worker_env_proto.ctrl_bootstrap_conf.CopyFrom(bootstrap_conf) - launch_worker_via_agent( - host=bootstrap_conf.host, env_proto=worker_env_proto - ) - _unittest_worker_initilized = True - elif device_num() > 1 and enable_multi_process(): - master_port = find_free_port() - oneflow.env.ctrl_port(master_port) - config_world_size = device_num() - bootstrap_conf_list = oneflow.env.init_bootstrap_confs( - ["127.0.0.1"], master_port, config_world_size - ) - env_proto = env_util.default_env_proto - assert ( - len(env_proto.machine) == 1 - and env_proto.HasField("ctrl_bootstrap_conf") == 1 - ) - run_dir = os.getenv("HOME") + "/oneflow_temp/" + str(uuid.uuid1()) - run_dir = os.path.abspath(os.path.expanduser(run_dir)) - if not os.path.exists(run_dir): - os.makedirs(run_dir) - for rank in range(1, config_world_size): - worker_env_proto = EnvProto() - worker_env_proto.CopyFrom(env_proto) - worker_env_proto.ctrl_bootstrap_conf.rank = rank - worker_env_proto.cpp_logging_conf.log_dir = ( - run_dir + "/log_" + str(rank) - ) - env_file = NamedTemporaryFile(delete=False) - if sys.version_info >= (3, 0): - env_file.write(pbtxt.MessageToString(worker_env_proto).encode()) - else: - env_file.write(pbtxt.MessageToString(worker_env_proto)) - env_file.close() - if not os.path.exists(run_dir + "/log_" + str(rank)): - os.mkdir(run_dir + "/log_" + str(rank)) - os.system( - "cp " - + env_file.name - + " " - + run_dir - + "/log_" - + str(rank) - + "/env_proto_" - + str(rank) - + ".proto" - ) - oneflow_cmd = ( - "python3 -m oneflow --start_worker" - + " --env_proto=" - + run_dir - + "/log_" - + str(rank) - + "/" - + "env_proto_" - + str(rank) - + ".proto" - ) - subprocess.Popen( - oneflow_cmd, - stdout=subprocess.DEVNULL, - stderr=subprocess.DEVNULL, - shell=True, - ) - os.remove(env_file.name) - atexit.register( - oneflow.deprecated.delete_worker_of_multi_process, run_dir=run_dir - ) - log_dir = os.getenv("ONEFLOW_TEST_LOG_DIR") - if log_dir: - oneflow.env.log_dir(log_dir) - if _unittest_env_initilized == False: - env_util.api_env_init() - _unittest_env_initilized = True - - def skip_unless(n, d): if (n > 1 or d > 1) and oneflow.sysconfig.has_rpc_backend_grpc() == False: return unittest.skip( diff --git a/python/oneflow/serving/inference_session.py b/python/oneflow/serving/inference_session.py index 2f13fc42782..b8edbded52b 100644 --- a/python/oneflow/serving/inference_session.py +++ b/python/oneflow/serving/inference_session.py @@ -158,8 +158,7 @@ def _init_event_loop(self): self.event_loop_ = asyncio.get_event_loop() def init(self): - if not oneflow._oneflow_internal.IsEnvInited(): - flow.env.init() + raise NotImplementedError("InferenceSession is deprecated.") if not oneflow._oneflow_internal.IsSessionInited(): self._make_config_proto() # session_util._TryCompleteConfigProto(self.config_proto_) From 434b7af9f8a4e1415d4dca664a477c377926b644 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Mon, 14 Mar 2022 17:49:13 +0800 Subject: [PATCH 28/54] more unittests --- python/oneflow/framework/tensor.py | 2 -- ...sync_after_atexist.py => test_shutting_down.py} | 14 ++++++++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) rename python/oneflow/test/modules/{test_sync_after_atexist.py => test_shutting_down.py} (66%) diff --git a/python/oneflow/framework/tensor.py b/python/oneflow/framework/tensor.py index 7c9f85d5823..e0860476e90 100644 --- a/python/oneflow/framework/tensor.py +++ b/python/oneflow/framework/tensor.py @@ -23,8 +23,6 @@ import numpy as np from typing import Union -import oneflow._oneflow_internal - Tensor = flow._oneflow_internal.Tensor TensorTuple = flow._oneflow_internal.TensorTuple diff --git a/python/oneflow/test/modules/test_sync_after_atexist.py b/python/oneflow/test/modules/test_shutting_down.py similarity index 66% rename from python/oneflow/test/modules/test_sync_after_atexist.py rename to python/oneflow/test/modules/test_shutting_down.py index 0aaf7f2639d..bd990ad6634 100644 --- a/python/oneflow/test/modules/test_sync_after_atexist.py +++ b/python/oneflow/test/modules/test_shutting_down.py @@ -15,10 +15,20 @@ """ import oneflow +class TestCallWhenShuttingDown: + def __init__(self): + tensor = oneflow.ones((2, 2)) + print(tensor) -class TestSyncLate: + def __del__(self): + tensor = oneflow.ones((2, 2)) + print(tensor) + +test_call_when_shutting_down = TestCallWhenShuttingDown() + +class TestSyncWhenShuttingDown: def __del__(self): oneflow._oneflow_internal.eager.Sync() -test_case = TestSyncLate() +test_sync_when_shutting_down = TestSyncWhenShuttingDown() From a64a55214c3c010b2cbc148d1ec6f5e4906b4ed1 Mon Sep 17 00:00:00 2001 From: strint Date: Tue, 15 Mar 2022 00:35:41 +0800 Subject: [PATCH 29/54] fix and pass tests --- oneflow/core/vm/virtual_machine_engine.cpp | 3 +++ python/oneflow/framework/unittest.py | 5 +++++ python/oneflow/test/modules/test_shutting_down.py | 3 +++ 3 files changed, 11 insertions(+) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index f5a7a9aeeff..8e440a40838 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -567,6 +567,9 @@ void VirtualMachineEngine::Callback() { // Do nothing. Wait until all other threads ref_cnts released. } CHECK_NOTNULL(garbage->phy_instr_operand()); + while (garbage->phy_instr_operand().use_count() > 1) { + // Do nothing. Wait until all other threads ref_cnts released. + } CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); // Destruct garbage. return Maybe::Ok(); diff --git a/python/oneflow/framework/unittest.py b/python/oneflow/framework/unittest.py index 5d173624dee..6f85f0f014c 100644 --- a/python/oneflow/framework/unittest.py +++ b/python/oneflow/framework/unittest.py @@ -195,6 +195,11 @@ def launch_worker_via_agent(host=None, env_proto=None): conn.close() +class TestCase(unittest.TestCase): + def setUp(self): + return + + def skip_unless(n, d): if (n > 1 or d > 1) and oneflow.sysconfig.has_rpc_backend_grpc() == False: return unittest.skip( diff --git a/python/oneflow/test/modules/test_shutting_down.py b/python/oneflow/test/modules/test_shutting_down.py index bd990ad6634..48273ae611b 100644 --- a/python/oneflow/test/modules/test_shutting_down.py +++ b/python/oneflow/test/modules/test_shutting_down.py @@ -15,6 +15,7 @@ """ import oneflow + class TestCallWhenShuttingDown: def __init__(self): tensor = oneflow.ones((2, 2)) @@ -24,8 +25,10 @@ def __del__(self): tensor = oneflow.ones((2, 2)) print(tensor) + test_call_when_shutting_down = TestCallWhenShuttingDown() + class TestSyncWhenShuttingDown: def __del__(self): oneflow._oneflow_internal.eager.Sync() From 60b59e5d32220e91b0e0c2d3315b5d3458885938 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Tue, 15 Mar 2022 12:01:06 +0800 Subject: [PATCH 30/54] revert virtual_machine.cpp --- oneflow/core/vm/virtual_machine.cpp | 14 +------------- python/oneflow/test/modules/test_shutting_down.py | 3 +++ 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index e4c009f6f49..45051fc43a4 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -29,7 +29,6 @@ limitations under the License. #include "oneflow/core/profiler/profiler.h" #include "oneflow/core/platform/include/pthread_fork.h" #include "oneflow/core/common/env_var.h" -#include "oneflow/core/framework/shut_down_util.h" #include "oneflow/core/framework/device.h" namespace oneflow { @@ -116,9 +115,7 @@ VirtualMachine::VirtualMachine(const Resource& resource, int64_t this_machine_id // In order to notify threads in VirtualMachineEngine, a notify callback lambda should be take as // an argument for VirtualMachineEngine's constructor. vm_ = intrusive::make_shared( - vm::MakeVmDesc(resource, this_machine_id).Get(), [this]() { - if (!IsShuttingDown()) { callback_notifier_.Notify(); } - }); + vm::MakeVmDesc(resource, this_machine_id).Get(), [this]() { callback_notifier_.Notify(); }); OF_PROFILER_NAME_THIS_HOST_THREAD("_Main"); std::function WorkerInitializer; GetWorkerThreadInitializer(vm_, &WorkerInitializer); @@ -202,15 +199,6 @@ Maybe VirtualMachine::Receive(vm::InstructionMsgList* instr_list) { // `ComputeInFuseMode` will be replaced by `Compute` soon. instr_msg->mut_instr_type_id()->instruction_type().ComputeInFuseMode(instr_msg); } - } else if (IsShuttingDown()) { - CHECK_OR_RETURN(vm_->Empty()); - CHECK_OR_RETURN(vm_->CallbackEmpty()); - JUST(vm_->Receive(instr_list)); - while (!(vm_->Empty() && vm_->CallbackEmpty())) { - vm_->Schedule(); - vm_->FlushGarbageMsgList(); - vm_->Callback(); - } } else { const int64_t kHighWaterMark = GetInstructionHighWaterMark(); if (vm_->flying_instruction_cnt() > kHighWaterMark) { diff --git a/python/oneflow/test/modules/test_shutting_down.py b/python/oneflow/test/modules/test_shutting_down.py index bd990ad6634..48273ae611b 100644 --- a/python/oneflow/test/modules/test_shutting_down.py +++ b/python/oneflow/test/modules/test_shutting_down.py @@ -15,6 +15,7 @@ """ import oneflow + class TestCallWhenShuttingDown: def __init__(self): tensor = oneflow.ones((2, 2)) @@ -24,8 +25,10 @@ def __del__(self): tensor = oneflow.ones((2, 2)) print(tensor) + test_call_when_shutting_down = TestCallWhenShuttingDown() + class TestSyncWhenShuttingDown: def __del__(self): oneflow._oneflow_internal.eager.Sync() From d9609c6c177f24ee18ac394c2d476b6f6d2ec0f5 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Tue, 15 Mar 2022 12:03:36 +0800 Subject: [PATCH 31/54] revert core/vm --- oneflow/core/vm/virtual_machine_engine.cpp | 6 +----- oneflow/core/vm/virtual_machine_engine.h | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 618a5301245..0444d815900 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -204,12 +204,8 @@ void VirtualMachineEngine::MoveInstructionMsgToGarbageMsgList( if (unlikely(local_garbage_msg_list_.size() > kWindowSize)) { MoveToGarbageMsgListAndNotifyGC(); } } -void VirtualMachineEngine::FlushGarbageMsgList() { - garbage_msg_list_.MoveFrom(&local_garbage_msg_list_); -} - void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC() { - FlushGarbageMsgList(); + garbage_msg_list_.MoveFrom(&local_garbage_msg_list_); notify_callback_thread_(); } diff --git a/oneflow/core/vm/virtual_machine_engine.h b/oneflow/core/vm/virtual_machine_engine.h index ecb0e52a697..41c0f343c16 100644 --- a/oneflow/core/vm/virtual_machine_engine.h +++ b/oneflow/core/vm/virtual_machine_engine.h @@ -104,7 +104,6 @@ class VirtualMachineEngine final : public intrusive::Base { // Returns true if old pending_instruction_list is empty Maybe Receive(intrusive::shared_ptr&& instruction_msg); void Schedule(); - void FlushGarbageMsgList(); void Callback(); void NotifyCallback(); bool ThreadUnsafeEmpty() const; From 62540a46e6a3851393877878c6d11c51668b423e Mon Sep 17 00:00:00 2001 From: lixinqi Date: Tue, 15 Mar 2022 12:13:01 +0800 Subject: [PATCH 32/54] remove outdated python class oneflow.unittest.TestCase --- python/oneflow/framework/unittest.py | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/python/oneflow/framework/unittest.py b/python/oneflow/framework/unittest.py index 5d173624dee..de4878aa1a1 100644 --- a/python/oneflow/framework/unittest.py +++ b/python/oneflow/framework/unittest.py @@ -182,18 +182,7 @@ def call(conn=None, cmd=None, msg=None): return conn.recv().decode() -def launch_worker_via_agent(host=None, env_proto=None): - print("[unittest]", "launching worker via agent at", host) - from multiprocessing.connection import Client - - address = ("localhost", worker_agent_port()) - conn = Client(address, authkey=worker_agent_authkey().encode()) - cast(conn=conn, cmd="host", msg=host) - cast(conn=conn, cmd="env_proto", msg=pbtxt.MessageToString(env_proto)) - assert call(conn=conn, cmd="start_worker") == "ok" - print("[unittest]", "worker launched via agent at", host) - conn.close() - +TestCase = unittest.TestCase def skip_unless(n, d): if (n > 1 or d > 1) and oneflow.sysconfig.has_rpc_backend_grpc() == False: From ad0be012e544f7979fd75117a20073e9b55b1f20 Mon Sep 17 00:00:00 2001 From: strint Date: Wed, 16 Mar 2022 01:23:26 +0800 Subject: [PATCH 33/54] graph test passed --- python/oneflow/framework/env_util.py | 11 +---------- python/oneflow/test/graph/test_graph_io_check.py | 4 ++-- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index cdc72e2bafb..f9c1a796ed7 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -196,22 +196,13 @@ def str2int(env_config): class EnvHolder(object): def __init__(self): - self._is_normal_exit = True if not HasAllMultiClientEnvVars(): SetDefaultMultiClientEnvVars() self._env_cxt = create_env() def GetEnv(): - global _env_holder - if _env_holder is not None: - return _env_holder - else: - _env_holder = EnvHolder() - return _env_holder - - -_env_holder = None + return EnvHolder() device_tag2default_parallel_conf = {} default_env_proto = _DefaultEnvProto() diff --git a/python/oneflow/test/graph/test_graph_io_check.py b/python/oneflow/test/graph/test_graph_io_check.py index b90b4704f08..b90d33156f8 100644 --- a/python/oneflow/test/graph/test_graph_io_check.py +++ b/python/oneflow/test/graph/test_graph_io_check.py @@ -140,7 +140,7 @@ def __init__(self): def forward(self, t): return t[0] - class CustomGraph(flow.nn.Graph): + class CustomGraphCheck1Ret(flow.nn.Graph): def __init__(self): super().__init__() self.m = CustomModule() @@ -150,7 +150,7 @@ def build(self, t): return rt model = CustomModule() - graph = CustomGraph() + graph = CustomGraphCheck1Ret() model_out = model(input) graph_out = graph(input) From 3925eb4a7f9a890234c3a42e7b063103738d2b26 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Wed, 16 Mar 2022 22:07:41 +0800 Subject: [PATCH 34/54] wait shared_ptr.use_count() == 0 --- oneflow/core/vm/virtual_machine_engine.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 87e2de6697e..62f6919f481 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -563,7 +563,9 @@ void VirtualMachineEngine::Callback() { // Do nothing. Wait until all other threads ref_cnts released. } CHECK_NOTNULL(garbage->phy_instr_operand()); - CHECK_EQ(garbage->phy_instr_operand().use_count(), 1) << garbage->DebugName(); + while (garbage->phy_instr_operand().use_count() > 1) { + // Do nothing. Wait until all other threads ref_cnts released. + } // Destruct garbage. return Maybe::Ok(); })); From 86296cb2d72a3e0aa71873d3fb73ad83a5c4440d Mon Sep 17 00:00:00 2001 From: lixinqi Date: Wed, 16 Mar 2022 22:21:32 +0800 Subject: [PATCH 35/54] export unittest.TestCase in framework/unittest.py --- python/oneflow/framework/unittest.py | 12 +----------- python/oneflow/test/modules/test_shutting_down.py | 3 +++ 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/python/oneflow/framework/unittest.py b/python/oneflow/framework/unittest.py index 5d173624dee..ab5aa20ded6 100644 --- a/python/oneflow/framework/unittest.py +++ b/python/oneflow/framework/unittest.py @@ -182,17 +182,7 @@ def call(conn=None, cmd=None, msg=None): return conn.recv().decode() -def launch_worker_via_agent(host=None, env_proto=None): - print("[unittest]", "launching worker via agent at", host) - from multiprocessing.connection import Client - - address = ("localhost", worker_agent_port()) - conn = Client(address, authkey=worker_agent_authkey().encode()) - cast(conn=conn, cmd="host", msg=host) - cast(conn=conn, cmd="env_proto", msg=pbtxt.MessageToString(env_proto)) - assert call(conn=conn, cmd="start_worker") == "ok" - print("[unittest]", "worker launched via agent at", host) - conn.close() +TestCase = unittest.TestCase def skip_unless(n, d): diff --git a/python/oneflow/test/modules/test_shutting_down.py b/python/oneflow/test/modules/test_shutting_down.py index bd990ad6634..48273ae611b 100644 --- a/python/oneflow/test/modules/test_shutting_down.py +++ b/python/oneflow/test/modules/test_shutting_down.py @@ -15,6 +15,7 @@ """ import oneflow + class TestCallWhenShuttingDown: def __init__(self): tensor = oneflow.ones((2, 2)) @@ -24,8 +25,10 @@ def __del__(self): tensor = oneflow.ones((2, 2)) print(tensor) + test_call_when_shutting_down = TestCallWhenShuttingDown() + class TestSyncWhenShuttingDown: def __del__(self): oneflow._oneflow_internal.eager.Sync() From 454f5e714d70af0b2be0fc7ba0f6a83dee521f97 Mon Sep 17 00:00:00 2001 From: lixinqi Date: Wed, 16 Mar 2022 23:58:38 +0800 Subject: [PATCH 36/54] SwitchToShuttingDownPhase --- oneflow/api/python/env/env.cpp | 25 ++++++++++++++++--- oneflow/core/job/env_global_objects_scope.cpp | 1 + oneflow/core/job/env_global_objects_scope.h | 10 ++++++++ python/oneflow/__init__.py | 4 +-- 4 files changed, 34 insertions(+), 6 deletions(-) diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 879c837635a..330451e17db 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -16,19 +16,38 @@ limitations under the License. #include #include "oneflow/api/python/of_api_registry.h" #include "oneflow/api/python/env/env_api.h" +#include "oneflow/core/vm/vm_util.h" +#include "oneflow/core/framework/shut_down_util.h" namespace py = pybind11; +namespace oneflow { + +Maybe SwitchToShuttingDownPhase(EnvGlobalObjectsScope* env, bool is_normal_exit) { + if (is_normal_exit) { JUST(vm::ClusterSync()); } + JUST(env->init_is_normal_exit(is_normal_exit)); + SetShuttingDown(true); + return Maybe::Ok(); +} + +} // namespace oneflow + ONEFLOW_API_PYBIND11_MODULE("", m) { m.def("CurrentResource", &CurrentResource); m.def("EnvResource", &EnvResource); m.def("EnableEagerEnvironment", &EnableEagerEnvironment); using Env = oneflow::EnvGlobalObjectsScope; - py::class_>(m, "Env").def( - py::init([](const std::string& env_proto_str) { + py::class_>(m, "Env") + .def(py::init([](const std::string& env_proto_str) { return oneflow::CreateEnv(env_proto_str).GetPtrOrThrow(); - })); + })) + .def( + "SwitchToShuttingDownPhase", + [](Env* env, bool is_normal_exit) { + oneflow::SwitchToShuttingDownPhase(env, is_normal_exit).GetOrThrow(); + }, + py::call_guard()); m.def("CurrentMachineId", &CurrentMachineId); diff --git a/oneflow/core/job/env_global_objects_scope.cpp b/oneflow/core/job/env_global_objects_scope.cpp index cd8a31fc152..c3bcd1e4299 100644 --- a/oneflow/core/job/env_global_objects_scope.cpp +++ b/oneflow/core/job/env_global_objects_scope.cpp @@ -227,6 +227,7 @@ Maybe EnvGlobalObjectsScope::Init(const EnvProto& env_proto) { } EnvGlobalObjectsScope::~EnvGlobalObjectsScope() { + if (!CHECK_JUST(is_normal_exit_)) { return; } auto session_ctx = Global::Get(); if (session_ctx != nullptr) { VLOG(1) << "Multi client session has not closed , env close it at env scope destruction."; diff --git a/oneflow/core/job/env_global_objects_scope.h b/oneflow/core/job/env_global_objects_scope.h index ceef4c52092..37196b64d65 100644 --- a/oneflow/core/job/env_global_objects_scope.h +++ b/oneflow/core/job/env_global_objects_scope.h @@ -17,6 +17,7 @@ limitations under the License. #define ONEFLOW_CORE_JOB_CLUSTER_OBJECTS_SCOPE_H_ #include "oneflow/core/common/util.h" #include "oneflow/core/common/maybe.h" +#include "oneflow/core/common/optional.h" #include "oneflow/core/job/env_desc.h" #include "oneflow/core/framework/device.h" @@ -31,6 +32,15 @@ class EnvGlobalObjectsScope final { ~EnvGlobalObjectsScope(); Maybe Init(const EnvProto& env_proto); + + Maybe init_is_normal_exit(bool is_normal_exit) { + CHECK_OR_RETURN(!is_normal_exit_.has_value()); + is_normal_exit_ = is_normal_exit; + return Maybe::Ok(); + } + + private: + Optional is_normal_exit_; }; } // namespace oneflow diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index f0a754c2b19..23791d1aa4c 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -257,10 +257,8 @@ def is_normal_exit(self): def atexit_hook(hook): - if hook.is_normal_exit(): - oneflow._oneflow_internal.eager.Sync() oneflow.framework.session_context.TryCloseDefaultSession() - oneflow._oneflow_internal.SetShuttingDown() + _oneflow_global_unique_env_.SwitchToShuttingDownPhase(hook.is_normal_exit()) atexit.register(atexit_hook, hook) From d1d9ad76d8599336c3b9596452d3d7b3ca1d981d Mon Sep 17 00:00:00 2001 From: lixinqi Date: Thu, 17 Mar 2022 00:07:13 +0800 Subject: [PATCH 37/54] optional is_normal_exit --- oneflow/core/job/env_global_objects_scope.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/oneflow/core/job/env_global_objects_scope.cpp b/oneflow/core/job/env_global_objects_scope.cpp index c3bcd1e4299..acfe01c88ba 100644 --- a/oneflow/core/job/env_global_objects_scope.cpp +++ b/oneflow/core/job/env_global_objects_scope.cpp @@ -227,7 +227,7 @@ Maybe EnvGlobalObjectsScope::Init(const EnvProto& env_proto) { } EnvGlobalObjectsScope::~EnvGlobalObjectsScope() { - if (!CHECK_JUST(is_normal_exit_)) { return; } + if (is_normal_exit_.has_value() && !CHECK_JUST(is_normal_exit_)) { return; } auto session_ctx = Global::Get(); if (session_ctx != nullptr) { VLOG(1) << "Multi client session has not closed , env close it at env scope destruction."; From a58348d8feb8239d7b69f2021ea939bc97bb6aee Mon Sep 17 00:00:00 2001 From: lixinqi Date: Thu, 17 Mar 2022 16:48:26 +0800 Subject: [PATCH 38/54] VirtualMachine::CloseVMThreads --- oneflow/api/python/env/env.cpp | 8 +- oneflow/core/intrusive/channel.h | 108 ------------------- oneflow/core/intrusive/channel_test.cpp | 116 --------------------- oneflow/core/vm/thread_ctx.cpp | 21 +--- oneflow/core/vm/thread_ctx.h | 24 +++-- oneflow/core/vm/virtual_machine.cpp | 97 +++++++++++++---- oneflow/core/vm/virtual_machine.h | 5 + oneflow/core/vm/virtual_machine_engine.cpp | 40 +++---- oneflow/core/vm/virtual_machine_engine.h | 34 +++--- 9 files changed, 147 insertions(+), 306 deletions(-) delete mode 100644 oneflow/core/intrusive/channel.h delete mode 100644 oneflow/core/intrusive/channel_test.cpp diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 330451e17db..d2d5d98bced 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -16,7 +16,9 @@ limitations under the License. #include #include "oneflow/api/python/of_api_registry.h" #include "oneflow/api/python/env/env_api.h" +#include "oneflow/core/common/global.h" #include "oneflow/core/vm/vm_util.h" +#include "oneflow/core/vm/virtual_machine.h" #include "oneflow/core/framework/shut_down_util.h" namespace py = pybind11; @@ -24,7 +26,11 @@ namespace py = pybind11; namespace oneflow { Maybe SwitchToShuttingDownPhase(EnvGlobalObjectsScope* env, bool is_normal_exit) { - if (is_normal_exit) { JUST(vm::ClusterSync()); } + if (is_normal_exit) { + JUST(vm::ClusterSync()); + auto* vm = JUST(GlobalMaybe()); + JUST(vm->CloseVMThreads()); + } JUST(env->init_is_normal_exit(is_normal_exit)); SetShuttingDown(true); return Maybe::Ok(); diff --git a/oneflow/core/intrusive/channel.h b/oneflow/core/intrusive/channel.h deleted file mode 100644 index 3041aa6af04..00000000000 --- a/oneflow/core/intrusive/channel.h +++ /dev/null @@ -1,108 +0,0 @@ -/* -Copyright 2020 The OneFlow 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. -*/ -#ifndef ONEFLOW_CORE_INTRUSIVE_CHANNEL_H_ -#define ONEFLOW_CORE_INTRUSIVE_CHANNEL_H_ - -#include -#include -#include "oneflow/core/intrusive/list.h" - -namespace oneflow { - -namespace intrusive { - -enum ChannelStatus { - kChannelStatusSuccess = 0, - kChannelStatusErrorClosed, -}; - -template -class Channel { - public: - using value_type = typename HookField::struct_type; - - Channel(const Channel&) = delete; - Channel(Channel&&) = delete; - Channel() : list_head_(), mutex_(), cond_(), is_closed_(false) {} - ~Channel() = default; - - bool Empty() { - std::unique_lock lock(*mut_mutex()); - return list_head_.empty(); - } - - ChannelStatus EmplaceBack(intrusive::shared_ptr&& ptr) { - std::unique_lock lock(*mut_mutex()); - if (is_closed_) { return kChannelStatusErrorClosed; } - list_head_.EmplaceBack(std::move(ptr)); - mut_cond()->notify_one(); - return kChannelStatusSuccess; - } - ChannelStatus PushBack(value_type* ptr) { - return EmplaceBack(intrusive::shared_ptr(ptr)); - } - ChannelStatus PopFront(intrusive::shared_ptr* ptr) { - std::unique_lock lock(*mut_mutex()); - mut_cond()->wait(lock, [this]() { return (!list_head_.empty()) || is_closed_; }); - if (list_head_.empty()) { return kChannelStatusErrorClosed; } - *ptr = list_head_.PopFront(); - return kChannelStatusSuccess; - } - - ChannelStatus MoveFrom(List* src) { - std::unique_lock lock(*mut_mutex()); - if (is_closed_) { return kChannelStatusErrorClosed; } - src->MoveToDstBack(&list_head_); - mut_cond()->notify_one(); - return kChannelStatusSuccess; - } - - ChannelStatus MoveTo(List* dst) { - std::unique_lock lock(*mut_mutex()); - mut_cond()->wait(lock, [this]() { return (!list_head_.empty()) || is_closed_; }); - if (list_head_.empty()) { return kChannelStatusErrorClosed; } - list_head_.MoveToDstBack(dst); - return kChannelStatusSuccess; - } - - ChannelStatus TryMoveTo(List* dst) { - if (list_head_.size() == 0) { return kChannelStatusSuccess; } - std::unique_lock lock(*mut_mutex()); - list_head_.MoveToDstBack(dst); - return kChannelStatusSuccess; - } - - void Close() { - std::unique_lock lock(*mut_mutex()); - is_closed_ = true; - mut_cond()->notify_all(); - } - - private: - std::mutex* mut_mutex() { return &mutex_; } - std::condition_variable* mut_cond() { return &cond_; } - - List list_head_; - std::mutex mutex_; - std::condition_variable cond_; - bool is_closed_; -}; - -} // namespace intrusive - -} // namespace oneflow - -#endif // ONEFLOW_CORE_INTRUSIVE_CHANNEL_H_ diff --git a/oneflow/core/intrusive/channel_test.cpp b/oneflow/core/intrusive/channel_test.cpp deleted file mode 100644 index d1a4202c9c0..00000000000 --- a/oneflow/core/intrusive/channel_test.cpp +++ /dev/null @@ -1,116 +0,0 @@ -/* -Copyright 2020 The OneFlow 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 "gtest/gtest.h" -#include "oneflow/core/intrusive/intrusive.h" -#include "oneflow/core/intrusive/channel.h" -#include "oneflow/core/common/util.h" -#include "oneflow/core/common/range.h" - -namespace oneflow { - -namespace intrusive { - -namespace test { - -namespace { - -class Foo final : public intrusive::Base { - public: - int x() const { return x_; } - void set_x(int val) { x_ = val; } - - private: - Foo() : intrusive_ref_(), x_(), hook_() {} - friend class intrusive::Ref; - intrusive::Ref* mut_intrusive_ref() { return &intrusive_ref_; } - intrusive::Ref intrusive_ref_; - // fields - int x_; - - public: - // list hooks - intrusive::ListHook hook_; -}; - -using ChannelFoo = intrusive::Channel; - -void CallFromSenderThread(ChannelFoo* condition_list, Range range) { - for (int i = range.begin(); i < range.end(); ++i) { - auto foo = intrusive::make_shared(); - foo->set_x(i); - if (condition_list->EmplaceBack(std::move(foo)) != intrusive::kChannelStatusSuccess) { break; } - } -} - -void CallFromReceiverThreadByPopFront(std::vector* visit, ChannelFoo* condition_list) { - intrusive::shared_ptr foo; - while (condition_list->PopFront(&foo) == intrusive::kChannelStatusSuccess) { - ++visit->at(foo->x()); - } -} - -void CallFromReceiverThreadByMoveTo(std::vector* visit, ChannelFoo* condition_list) { - intrusive::List tmp_list; - while (condition_list->MoveTo(&tmp_list) == intrusive::kChannelStatusSuccess) { - INTRUSIVE_FOR_EACH_PTR(foo, &tmp_list) { - ++visit->at(foo->x()); - tmp_list.Erase(foo); - } - } -} - -typedef void (*ThreadHandlerType)(std::vector* visit, ChannelFoo* condition_list); - -void TestChannel(ThreadHandlerType ThreadHandler) { - ChannelFoo condition_list; - std::vector senders; - std::vector receivers; - int sender_num = 30; - int receiver_num = 40; - int range_num = 200; - std::vector> visits; - for (int i = 0; i < receiver_num; ++i) { - std::vector visit_i; - for (int j = 0; j < range_num; j++) { visit_i.emplace_back(0); } - visits.emplace_back(visit_i); - } - for (int i = 0; i < sender_num; ++i) { - senders.emplace_back(std::thread(CallFromSenderThread, &condition_list, Range(0, range_num))); - } - for (int i = 0; i < receiver_num; ++i) { - receivers.emplace_back(std::thread(ThreadHandler, &visits[i], &condition_list)); - } - for (std::thread& this_thread : senders) { this_thread.join(); } - condition_list.Close(); - for (std::thread& this_thread : receivers) { this_thread.join(); } - for (int i = 0; i < range_num; ++i) { - int visit_count = 0; - for (int j = 0; j < receiver_num; j++) { visit_count += visits[j][i]; } - ASSERT_EQ(visit_count, sender_num); - } -} - -TEST(Channel, 30sender40receiver_pop_front) { TestChannel(&CallFromReceiverThreadByPopFront); } - -TEST(Channel, 30sender40receiver_move_to) { TestChannel(&CallFromReceiverThreadByMoveTo); } - -} // namespace - -} // namespace test - -} // namespace intrusive - -} // namespace oneflow diff --git a/oneflow/core/vm/thread_ctx.cpp b/oneflow/core/vm/thread_ctx.cpp index cf83526488a..c347fa1d9ed 100644 --- a/oneflow/core/vm/thread_ctx.cpp +++ b/oneflow/core/vm/thread_ctx.cpp @@ -19,29 +19,16 @@ limitations under the License. namespace oneflow { namespace vm { -template -intrusive::ChannelStatus ThreadCtx::MoveAndRun(size_t* cnt) { +size_t ThreadCtx::TryReceiveAndRun() { const StreamType& stream_type = stream_rt_desc().stream_type(); intrusive::List tmp_list; - intrusive::ChannelStatus status = (mut_pending_instruction_list()->*Move)(&tmp_list); - *cnt = tmp_list.size(); - if (*cnt == 0) { return status; } + mut_pending_instruction_list()->MoveTo(&tmp_list); + size_t size = tmp_list.size(); INTRUSIVE_FOR_EACH(instruction, &tmp_list) { tmp_list.Erase(instruction.Mutable()); stream_type.Run(instruction.Mutable()); } - return status; -} - -intrusive::ChannelStatus ThreadCtx::ReceiveAndRun() { - size_t cnt = 0; - return MoveAndRun<&PendingInstructionChannel::MoveTo>(&cnt); -} - -size_t ThreadCtx::TryReceiveAndRun() { - size_t cnt = 0; - MoveAndRun<&PendingInstructionChannel::TryMoveTo>(&cnt); - return cnt; + return size; } } // namespace vm diff --git a/oneflow/core/vm/thread_ctx.h b/oneflow/core/vm/thread_ctx.h index d36d57f3012..150b09f29fc 100644 --- a/oneflow/core/vm/thread_ctx.h +++ b/oneflow/core/vm/thread_ctx.h @@ -18,15 +18,16 @@ limitations under the License. #include #include "oneflow/core/intrusive/intrusive.h" -#include "oneflow/core/intrusive/channel.h" +#include "oneflow/core/intrusive/mutexed_list.h" +#include "oneflow/core/common/notifier.h" #include "oneflow/core/vm/stream.h" #include "oneflow/core/vm/stream_runtime_desc.h" namespace oneflow { namespace vm { -using PendingInstructionChannel = - intrusive::Channel; +using PendingInstructionMutexedList = + intrusive::MutexedList; using PendingInstructionList = intrusive::List; @@ -46,7 +47,9 @@ class ThreadCtx final : public intrusive::Base { void set_stream_rt_desc(const StreamRtDesc* val) { stream_rt_desc_ = val; } void clear_stream_rt_desc() { stream_rt_desc_ = nullptr; } StreamList* mut_stream_list() { return &stream_list_; } - PendingInstructionChannel* mut_pending_instruction_list() { return &pending_instruction_list_; } + PendingInstructionMutexedList* mut_pending_instruction_list() { + return &pending_instruction_list_; + } // methods void __Init__(const StreamRtDesc& stream_rt_desc) { @@ -54,12 +57,10 @@ class ThreadCtx final : public intrusive::Base { set_stream_rt_desc(&stream_rt_desc); } size_t TryReceiveAndRun(); - intrusive::ChannelStatus ReceiveAndRun(); - private: - template - intrusive::ChannelStatus MoveAndRun(size_t* cnt); + Notifier* mut_notifier() { return ¬ifier_; } + private: friend class intrusive::Ref; intrusive::Ref* mut_intrusive_ref() { return &intrusive_ref_; } @@ -67,14 +68,17 @@ class ThreadCtx final : public intrusive::Base { : intrusive_ref_(), stream_rt_desc_(), stream_list_(), - pending_instruction_list_(), + pending_instruction_mutex_(), + pending_instruction_list_(&pending_instruction_mutex_), thread_ctx_hook_() {} intrusive::Ref intrusive_ref_; // fields const StreamRtDesc* stream_rt_desc_; // lists StreamList stream_list_; - PendingInstructionChannel pending_instruction_list_; + std::mutex pending_instruction_mutex_; + PendingInstructionMutexedList pending_instruction_list_; + Notifier notifier_; public: // list hooks diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index e4c009f6f49..c02fb589919 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -105,7 +105,9 @@ void GetWorkerThreadInitializer(intrusive::shared_ptr void WorkerLoop(vm::ThreadCtx* thread_ctx, const std::function& Initializer) { Initializer(thread_ctx); - while (thread_ctx->ReceiveAndRun() == intrusive::kChannelStatusSuccess) {} + while (thread_ctx->mut_notifier()->WaitAndClearNotifiedCnt() == kNotifierStatusSuccess) { + while (thread_ctx->TryReceiveAndRun()) {} + } } } // namespace @@ -116,9 +118,7 @@ VirtualMachine::VirtualMachine(const Resource& resource, int64_t this_machine_id // In order to notify threads in VirtualMachineEngine, a notify callback lambda should be take as // an argument for VirtualMachineEngine's constructor. vm_ = intrusive::make_shared( - vm::MakeVmDesc(resource, this_machine_id).Get(), [this]() { - if (!IsShuttingDown()) { callback_notifier_.Notify(); } - }); + vm::MakeVmDesc(resource, this_machine_id).Get()); OF_PROFILER_NAME_THIS_HOST_THREAD("_Main"); std::function WorkerInitializer; GetWorkerThreadInitializer(vm_, &WorkerInitializer); @@ -133,6 +133,7 @@ VirtualMachine::VirtualMachine(const Resource& resource, int64_t this_machine_id std::function SchedulerInitializer; GetSchedulerThreadInitializer(&SchedulerInitializer); schedule_thread_ = std::thread(&VirtualMachine::ScheduleLoop, this, SchedulerInitializer); + vm_threads_closed_ = false; } namespace { @@ -156,11 +157,20 @@ void VirtualMachine::ControlSync() { CHECK_JUST(bc->WaitUntilCntEqualZero(VirtualMachine::GetPredicatorNoMoreInstructionsFinished())); } -VirtualMachine::~VirtualMachine() { +Maybe VirtualMachine::CloseVMThreads() { + CHECK_OR_RETURN(!vm_threads_closed_); ControlSync(); pending_notifier_.Close(); schedule_thread_.join(); - CHECK(!vm_); + vm_threads_closed_ = true; + return Maybe::Ok(); +} + +VirtualMachine::~VirtualMachine() { + if (!vm_threads_closed_) { CHECK_JUST(CloseVMThreads()); } + CHECK(vm_->Empty()); + CHECK(vm_->CallbackEmpty()); + vm_.Reset(); callback_notifier_.Close(); callback_thread_.join(); } @@ -202,15 +212,8 @@ Maybe VirtualMachine::Receive(vm::InstructionMsgList* instr_list) { // `ComputeInFuseMode` will be replaced by `Compute` soon. instr_msg->mut_instr_type_id()->instruction_type().ComputeInFuseMode(instr_msg); } - } else if (IsShuttingDown()) { - CHECK_OR_RETURN(vm_->Empty()); - CHECK_OR_RETURN(vm_->CallbackEmpty()); - JUST(vm_->Receive(instr_list)); - while (!(vm_->Empty() && vm_->CallbackEmpty())) { - vm_->Schedule(); - vm_->FlushGarbageMsgList(); - vm_->Callback(); - } + } else if (unlikely(vm_threads_closed_)) { + JUST(RunInCurrentThread(instr_list)); } else { const int64_t kHighWaterMark = GetInstructionHighWaterMark(); if (vm_->flying_instruction_cnt() > kHighWaterMark) { @@ -235,8 +238,60 @@ Maybe VirtualMachine::Receive(vm::InstructionMsgList* instr_list) { return Maybe::Ok(); } +namespace { + +class SingleThreadScheduleCtx : public vm::ScheduleCtx { + public: + explicit SingleThreadScheduleCtx(vm::VirtualMachineEngine* vm) : vm_(vm) {} + ~SingleThreadScheduleCtx() = default; + + void OnGarbageMsgPending() const override { vm_->Callback(); } + void OnWorkerLoadPending(vm::ThreadCtx* thread_ctx) const override { + while (thread_ctx->TryReceiveAndRun() > 0) {} + } + + private: + vm::VirtualMachineEngine* vm_; +}; + +void ScheduleUntilVMEmpty(vm::VirtualMachineEngine* vm, const vm::ScheduleCtx& schedule_ctx) { + while (!(vm->Empty() && vm->CallbackEmpty())) { + vm->Schedule(schedule_ctx); + vm->MoveToGarbageMsgListAndNotifyGC(schedule_ctx); + } +} + +} // namespace + +Maybe VirtualMachine::RunInCurrentThread(vm::InstructionMsgList* instr_list) { + CHECK_OR_RETURN(vm_->Empty()); + CHECK_OR_RETURN(vm_->CallbackEmpty()); + JUST(vm_->Receive(instr_list)); + ScheduleUntilVMEmpty(vm_.Mutable(), SingleThreadScheduleCtx(vm_.Mutable())); + return Maybe::Ok(); +} + +namespace { + +class MultiThreadScheduleCtx : public vm::ScheduleCtx { + public: + explicit MultiThreadScheduleCtx(Notifier* cb_notifier) : cb_notifier_(cb_notifier) {} + MultiThreadScheduleCtx() = default; + + void OnGarbageMsgPending() const override { cb_notifier_->Notify(); } + void OnWorkerLoadPending(vm::ThreadCtx* thread_ctx) const override { + thread_ctx->mut_notifier()->Notify(); + } + + private: + Notifier* cb_notifier_; +}; + +} // namespace + void VirtualMachine::ScheduleLoop(const std::function& Initializer) { Initializer(); + MultiThreadScheduleCtx schedule_ctx(&callback_notifier_); auto* vm = mut_vm(); while (pending_notifier_.WaitAndClearNotifiedCnt() == kNotifierStatusSuccess) { OF_PROFILER_RANGE_PUSH("VirtualMachine::ScheduleLoop"); @@ -263,22 +318,18 @@ void VirtualMachine::ScheduleLoop(const std::function& Initializer) { // VirtualMachine::Receive may be less effiencient if the thread safe version `vm->Empty()` // used // here, because VirtualMachine::ScheduleLoop is more likely to get the mutex lock. - do { vm->Schedule(); } while (!vm->ThreadUnsafeEmpty()); - vm->NotifyCallback(); + do { vm->Schedule(schedule_ctx); } while (!vm->ThreadUnsafeEmpty()); + vm->MoveToGarbageMsgListAndNotifyGC(schedule_ctx); } while (++i < kNumSchedulingPerTimoutTest); } while (MicrosecondsFrom(start) < kWorkingMicroseconds); OF_PROFILER_RANGE_POP(); } - while (!(vm->Empty() && vm->CallbackEmpty())) { - vm->Schedule(); - vm->NotifyCallback(); - } + ScheduleUntilVMEmpty(vm, schedule_ctx); CHECK_JUST(ForEachThreadCtx(vm_.Mutable(), [&](vm::ThreadCtx* thread_ctx) -> Maybe { - thread_ctx->mut_pending_instruction_list()->Close(); + thread_ctx->mut_notifier()->Close(); return Maybe::Ok(); })); for (const auto& worker_thread : worker_threads_) { worker_thread->join(); } - vm_.Reset(); } void VirtualMachine::CallbackLoop(const std::function& Initializer) { diff --git a/oneflow/core/vm/virtual_machine.h b/oneflow/core/vm/virtual_machine.h index bdbb7b49bfd..99153082e8a 100644 --- a/oneflow/core/vm/virtual_machine.h +++ b/oneflow/core/vm/virtual_machine.h @@ -41,6 +41,8 @@ class VirtualMachine final { const vm::VirtualMachineEngine& vm() const { return *vm_; } + Maybe CloseVMThreads(); + private: friend class InstructionsBuilder; @@ -50,6 +52,8 @@ class VirtualMachine final { vm::VirtualMachineEngine* mut_vm() { return vm_.Mutable(); } void ControlSync(); + Maybe RunInCurrentThread(vm::InstructionMsgList* instr_list); + intrusive::shared_ptr vm_; // for asynchronized execution std::list> worker_threads_; @@ -57,6 +61,7 @@ class VirtualMachine final { Notifier pending_notifier_; std::thread callback_thread_; Notifier callback_notifier_; + bool vm_threads_closed_; }; } // namespace oneflow diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 618a5301245..2de18142715 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -178,7 +178,7 @@ intrusive::shared_ptr VirtualMachineEngine::LivelyInstructionListEr } // Collect ready instructions onto ready_instruction_list_ -void VirtualMachineEngine::ReleaseFinishedInstructions() { +void VirtualMachineEngine::ReleaseFinishedInstructions(const ScheduleCtx& schedule_ctx) { INTRUSIVE_FOR_EACH_PTR(stream, mut_active_stream_list()) { while (true) { auto* instruction_ptr = stream->mut_running_instruction_list()->Begin(); @@ -189,28 +189,30 @@ void VirtualMachineEngine::ReleaseFinishedInstructions() { // in stream->DeleteInstruction(...) intrusive::shared_ptr instr_msg(instruction_ptr->mut_instr_msg()); stream->DeleteInstruction(LivelyInstructionListErase(instruction_ptr)); - MoveInstructionMsgToGarbageMsgList(std::move(instr_msg)); + MoveInstructionMsgToGarbageMsgList(std::move(instr_msg), schedule_ctx); } if (stream->running_instruction_list().empty()) { mut_active_stream_list()->Erase(stream); } } } void VirtualMachineEngine::MoveInstructionMsgToGarbageMsgList( - intrusive::shared_ptr&& instr_msg) { + intrusive::shared_ptr&& instr_msg, const ScheduleCtx& schedule_ctx) { local_garbage_msg_list_.EmplaceBack(std::move(instr_msg)); static constexpr int kWindowSize = 32; // local_garbage_msg_list_ is the cache of garbage_msg_list_. // `kWindowSize` controls the frequency of the usage of mutexed list. - if (unlikely(local_garbage_msg_list_.size() > kWindowSize)) { MoveToGarbageMsgListAndNotifyGC(); } + if (unlikely(local_garbage_msg_list_.size() > kWindowSize)) { + MoveToGarbageMsgListAndNotifyGC(schedule_ctx); + } } void VirtualMachineEngine::FlushGarbageMsgList() { garbage_msg_list_.MoveFrom(&local_garbage_msg_list_); } -void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC() { +void VirtualMachineEngine::MoveToGarbageMsgListAndNotifyGC(const ScheduleCtx& schedule_ctx) { FlushGarbageMsgList(); - notify_callback_thread_(); + schedule_ctx.OnGarbageMsgPending(); } int64_t VirtualMachineEngine::this_machine_id() const { @@ -312,7 +314,7 @@ bool VirtualMachineEngine::Dispatchable(Instruction* instruction) const { } // Dispatch ready instructions and put prescheduled instructions onto ready_instruction_list_. -void VirtualMachineEngine::DispatchAndPrescheduleInstructions() { +void VirtualMachineEngine::DispatchAndPrescheduleInstructions(const ScheduleCtx& schedule_ctx) { ReadyInstructionList tmp_ready_instruction_list; mut_ready_instruction_list()->MoveTo(&tmp_ready_instruction_list); OF_PROFILER_RANGE_PUSH("DispatchAndPrescheduleInstructions"); @@ -321,7 +323,7 @@ void VirtualMachineEngine::DispatchAndPrescheduleInstructions() { // `instruction.dispatched_instruction_hook_` are used in DispatchInstruction. tmp_ready_instruction_list.Erase(instruction.Mutable()); OF_PROFILER_RANGE_PUSH("D:" + instruction->instr_msg().DebugName()); - DispatchInstruction(instruction.Mutable()); + DispatchInstruction(instruction.Mutable(), schedule_ctx); // preschedule instructions INTRUSIVE_UNSAFE_FOR_EACH_PTR(edge, instruction->mut_out_edges()) { auto* out_instruction = edge->mut_dst_instruction(); @@ -336,7 +338,8 @@ void VirtualMachineEngine::DispatchAndPrescheduleInstructions() { OF_PROFILER_RANGE_POP(); } -void VirtualMachineEngine::DispatchInstruction(Instruction* instruction) { +void VirtualMachineEngine::DispatchInstruction(Instruction* instruction, + const ScheduleCtx& schedule_ctx) { auto* stream = instruction->mut_stream(); stream->mut_running_instruction_list()->PushBack(instruction); if (stream->active_stream_hook().empty()) { mut_active_stream_list()->PushBack(stream); } @@ -345,12 +348,11 @@ void VirtualMachineEngine::DispatchInstruction(Instruction* instruction) { stream_type.Run(instruction); } else { stream->mut_thread_ctx()->mut_pending_instruction_list()->PushBack(instruction); + schedule_ctx.OnWorkerLoadPending(stream->mut_thread_ctx()); } } -void VirtualMachineEngine::__Init__(const VmDesc& vm_desc, - const std::function& notify_callback_thread) { - notify_callback_thread_ = notify_callback_thread; +void VirtualMachineEngine::__Init__(const VmDesc& vm_desc) { mut_vm_resource_desc()->CopyFrom(vm_desc.vm_resource_desc()); CHECK_GT(vm_desc.machine_id_range().size(), 0); *mut_machine_id_range() = vm_desc.machine_id_range(); @@ -503,9 +505,9 @@ void VirtualMachineEngine::TryRunBarrierInstruction() { OF_PROFILER_RANGE_POP(); } -void VirtualMachineEngine::Schedule() { +void VirtualMachineEngine::Schedule(const ScheduleCtx& schedule_ctx) { // Release finished instructions and try to schedule out instructions in DAG onto ready list. - if (unlikely(mut_active_stream_list()->size())) { ReleaseFinishedInstructions(); } + if (unlikely(mut_active_stream_list()->size())) { ReleaseFinishedInstructions(schedule_ctx); } // Try run the first barrier instruction. if (unlikely(mut_barrier_instruction_list()->size())) { TryRunBarrierInstruction(); } // Handle pending instructions, and try schedule them to ready list. @@ -525,7 +527,9 @@ void VirtualMachineEngine::Schedule() { HandleLocalPending(); } // dispatch ready instructions and try to schedule out instructions in DAG onto ready list. - if (unlikely(mut_ready_instruction_list()->size())) { DispatchAndPrescheduleInstructions(); } + if (unlikely(mut_ready_instruction_list()->size())) { + DispatchAndPrescheduleInstructions(schedule_ctx); + } // handle probes if (unlikely(local_probe_list_.size())) { HandleLocalProbe(); @@ -541,8 +545,6 @@ void VirtualMachineEngine::Callback() { // destruct garbage_msg_list. } -void VirtualMachineEngine::NotifyCallback() { MoveToGarbageMsgListAndNotifyGC(); } - bool VirtualMachineEngine::ThreadUnsafeEmpty() const { return local_pending_msg_list().empty() && active_stream_list().empty() && flying_instruction_cnt() == 0; @@ -553,7 +555,9 @@ bool VirtualMachineEngine::Empty() const { return pending_msg_list().empty() && ThreadUnsafeEmpty(); } -bool VirtualMachineEngine::CallbackEmpty() const { return garbage_msg_list_.empty(); } +bool VirtualMachineEngine::CallbackEmpty() const { + return garbage_msg_list_.empty() && local_garbage_msg_list_.empty(); +} } // namespace vm } // namespace oneflow diff --git a/oneflow/core/vm/virtual_machine_engine.h b/oneflow/core/vm/virtual_machine_engine.h index ecb0e52a697..c4a5fabca23 100644 --- a/oneflow/core/vm/virtual_machine_engine.h +++ b/oneflow/core/vm/virtual_machine_engine.h @@ -35,6 +35,19 @@ namespace oneflow { namespace vm { +class ThreadCtx; + +class ScheduleCtx { + public: + ScheduleCtx() = default; + virtual ~ScheduleCtx() = default; + + virtual void OnGarbageMsgPending() const = 0; + virtual void OnWorkerLoadPending(vm::ThreadCtx* thread_ctx) const = 0; + + private: +}; + struct VmDesc; class VirtualMachineEngine final : public intrusive::Base { public: @@ -95,21 +108,18 @@ class VirtualMachineEngine final : public intrusive::Base { StreamType2StreamRtDesc* mut_stream_type2stream_rt_desc() { return &stream_type2stream_rt_desc_; } // methods - void __Init__(const VmDesc& vm_desc) { - __Init__(vm_desc, []() {}); - } - void __Init__(const VmDesc& vm_desc, const std::function& notify_callback_thread); + void __Init__(const VmDesc& vm_desc); // Returns true if old pending_instruction_list is empty Maybe Receive(InstructionMsgList* instr_list); // Returns true if old pending_instruction_list is empty Maybe Receive(intrusive::shared_ptr&& instruction_msg); - void Schedule(); + void Schedule(const ScheduleCtx& schedule_ctx); void FlushGarbageMsgList(); void Callback(); - void NotifyCallback(); bool ThreadUnsafeEmpty() const; bool Empty() const; bool CallbackEmpty() const; + void MoveToGarbageMsgListAndNotifyGC(const ScheduleCtx& schedule_ctx); std::string GetLivelyInstructionListDebugString(int64_t debug_cnt); int64_t this_machine_id() const; @@ -129,16 +139,16 @@ class VirtualMachineEngine final : public intrusive::Base { ReadyInstructionList* mut_ready_instruction_list() { return &ready_instruction_list_; } - void ReleaseFinishedInstructions(); - void MoveInstructionMsgToGarbageMsgList(intrusive::shared_ptr&& instr_msg); - void MoveToGarbageMsgListAndNotifyGC(); + void ReleaseFinishedInstructions(const ScheduleCtx& schedule_ctx); + void MoveInstructionMsgToGarbageMsgList(intrusive::shared_ptr&& instr_msg, + const ScheduleCtx& schedule_ctx); void HandleLocalPending(); void GetRewritedPendingInstructionsByWindowSize(size_t window_size, InstructionMsgList* /*out*/ pending_instr_msgs); void MakeAndAppendFusedInstruction(InstructionMsgList&& fused_instr_msg_list, InstructionMsgList* /*out*/ pending_instr_msgs); void TryRunBarrierInstruction(); - void DispatchAndPrescheduleInstructions(); + void DispatchAndPrescheduleInstructions(const ScheduleCtx& schedule_ctx); bool OnSchedulerThread(const StreamType& stream_type); void ReleaseInstruction(Instruction* instruction); @@ -150,7 +160,7 @@ class VirtualMachineEngine final : public intrusive::Base { DependenceAccess* AccessMirroredObject(OperandAccessType access_type, MirroredObject* mirrored_object, Instruction* instrution); void ConsumeMirroredObjects(Instruction* instruction); - void DispatchInstruction(Instruction* instruction); + void DispatchInstruction(Instruction* instruction, const ScheduleCtx& schedule_ctx); bool EdgeDispatchable(const Instruction* src, const Instruction* dst) const; bool Dispatchable(Instruction* instruction) const; @@ -177,7 +187,6 @@ class VirtualMachineEngine final : public intrusive::Base { callback_msg_mutex_(), garbage_msg_list_(&callback_msg_mutex_), local_garbage_msg_list_(), - notify_callback_thread_([]() {}), ready_instruction_list_(), lively_instruction_list_(), total_inserted_lively_instruction_cnt_(0), @@ -204,7 +213,6 @@ class VirtualMachineEngine final : public intrusive::Base { InstructionMsgMutexedList garbage_msg_list_; // local_garbage_msg_list_ should be consider as the cache of garbage_msg_list_. InstructionMsgList local_garbage_msg_list_; - std::function notify_callback_thread_; ReadyInstructionList ready_instruction_list_; LivelyInstructionList lively_instruction_list_; size_t total_inserted_lively_instruction_cnt_; From fe643794a64202d3f981be79de9637d596507359 Mon Sep 17 00:00:00 2001 From: Li Xinqi Date: Fri, 18 Mar 2022 16:07:54 +0800 Subject: [PATCH 39/54] Delete env_api.h env_api.h is deleted by master --- oneflow/api/python/env/env_api.h | 51 -------------------------------- 1 file changed, 51 deletions(-) delete mode 100644 oneflow/api/python/env/env_api.h diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h deleted file mode 100644 index 2946924bd70..00000000000 --- a/oneflow/api/python/env/env_api.h +++ /dev/null @@ -1,51 +0,0 @@ -/* -Copyright 2020 The OneFlow 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. -*/ -#ifndef ONEFLOW_API_PYTHON_ENV_ENV_API_H_ -#define ONEFLOW_API_PYTHON_ENV_ENV_API_H_ - -#include "oneflow/api/python/env/env.h" - -inline std::string CurrentResource() { return oneflow::CurrentResource().GetOrThrow(); } - -inline std::string EnvResource() { return oneflow::EnvResource().GetOrThrow(); } - -inline void EnableEagerEnvironment(bool enable_eager_execution) { - return oneflow::EnableEagerEnvironment(enable_eager_execution).GetOrThrow(); -} - -inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } - -inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } - -inline size_t GetWorldSize() { return oneflow::GetWorldSize().GetOrThrow(); } - -inline size_t GetNodeSize() { return oneflow::GetNodeSize().GetOrThrow(); } - -inline size_t GetLocalRank() { return oneflow::GetLocalRank().GetOrThrow(); } - -inline size_t CudaGetDeviceCount() { return oneflow::CudaGetDeviceCount().GetOrThrow(); } - -inline void SetFLAGS_alsologtostderr(bool flag) { - return oneflow::SetFLAGS_alsologtostderr(flag).GetOrThrow(); -} - -inline bool GetFLAGS_alsologtostderr() { return oneflow::GetFLAGS_alsologtostderr().GetOrThrow(); } - -inline void SetFLAGS_v(int32_t v_level) { return oneflow::SetFLAGS_v(v_level).GetOrThrow(); } - -inline int32_t GetFLAGS_v() { return oneflow::GetFLAGS_v().GetOrThrow(); } - -#endif // ONEFLOW_API_PYTHON_ENV_ENV_API_H_ From 72426fcbb3c7e4c283cf04e2d53a8fe5c45ac4fe Mon Sep 17 00:00:00 2001 From: lixinqi Date: Tue, 22 Mar 2022 12:59:33 +0800 Subject: [PATCH 40/54] address pr comments --- oneflow/api/python/env/env.cpp | 8 ++------ oneflow/core/vm/virtual_machine.cpp | 6 +++--- oneflow/core/vm/virtual_machine.h | 2 +- oneflow/core/vm/virtual_machine_engine.h | 2 -- 4 files changed, 6 insertions(+), 12 deletions(-) diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 9cdbe712a1b..5b404805857 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -45,12 +45,8 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { .def(py::init([](const std::string& env_proto_str) { return CreateEnv(env_proto_str).GetPtrOrThrow(); })) - .def( - "SwitchToShuttingDownPhase", - [](EnvGlobalObjectsScope* env, bool is_normal_exit) { - SwitchToShuttingDownPhase(env, is_normal_exit).GetOrThrow(); - }, - py::call_guard()); + .def("SwitchToShuttingDownPhase", &SwitchToShuttingDownPhase, + py::call_guard()); m.def("CurrentMachineId", &CurrentMachineId); diff --git a/oneflow/core/vm/virtual_machine.cpp b/oneflow/core/vm/virtual_machine.cpp index c02fb589919..1ccfd6a6713 100644 --- a/oneflow/core/vm/virtual_machine.cpp +++ b/oneflow/core/vm/virtual_machine.cpp @@ -112,7 +112,8 @@ void WorkerLoop(vm::ThreadCtx* thread_ctx, const std::function SchedulerInitializer; GetSchedulerThreadInitializer(&SchedulerInitializer); schedule_thread_ = std::thread(&VirtualMachine::ScheduleLoop, this, SchedulerInitializer); - vm_threads_closed_ = false; } namespace { @@ -276,7 +276,7 @@ namespace { class MultiThreadScheduleCtx : public vm::ScheduleCtx { public: explicit MultiThreadScheduleCtx(Notifier* cb_notifier) : cb_notifier_(cb_notifier) {} - MultiThreadScheduleCtx() = default; + ~MultiThreadScheduleCtx() = default; void OnGarbageMsgPending() const override { cb_notifier_->Notify(); } void OnWorkerLoadPending(vm::ThreadCtx* thread_ctx) const override { diff --git a/oneflow/core/vm/virtual_machine.h b/oneflow/core/vm/virtual_machine.h index 99153082e8a..3ab012f666b 100644 --- a/oneflow/core/vm/virtual_machine.h +++ b/oneflow/core/vm/virtual_machine.h @@ -54,6 +54,7 @@ class VirtualMachine final { Maybe RunInCurrentThread(vm::InstructionMsgList* instr_list); + bool vm_threads_closed_; intrusive::shared_ptr vm_; // for asynchronized execution std::list> worker_threads_; @@ -61,7 +62,6 @@ class VirtualMachine final { Notifier pending_notifier_; std::thread callback_thread_; Notifier callback_notifier_; - bool vm_threads_closed_; }; } // namespace oneflow diff --git a/oneflow/core/vm/virtual_machine_engine.h b/oneflow/core/vm/virtual_machine_engine.h index c4a5fabca23..4df30028533 100644 --- a/oneflow/core/vm/virtual_machine_engine.h +++ b/oneflow/core/vm/virtual_machine_engine.h @@ -44,8 +44,6 @@ class ScheduleCtx { virtual void OnGarbageMsgPending() const = 0; virtual void OnWorkerLoadPending(vm::ThreadCtx* thread_ctx) const = 0; - - private: }; struct VmDesc; From 174bc3df32901c2de1c95059ba49fecc9862eab5 Mon Sep 17 00:00:00 2001 From: strint Date: Tue, 22 Mar 2022 14:18:13 +0800 Subject: [PATCH 41/54] rm is env init --- oneflow/api/python/env/env.cpp | 1 - oneflow/api/python/env/env.h | 1 - oneflow/api/python/env/env_api.h | 1 - 3 files changed, 3 deletions(-) diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 080baf08ec7..ab9e470b189 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -56,7 +56,6 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { }, py::call_guard()); - m.def("IsEnvInited", &IsEnvInited); m.def("CurrentMachineId", &CurrentMachineId); m.def("GetRank", &GetRank); diff --git a/oneflow/api/python/env/env.h b/oneflow/api/python/env/env.h index 40fb872a194..1d5772071a7 100644 --- a/oneflow/api/python/env/env.h +++ b/oneflow/api/python/env/env.h @@ -46,7 +46,6 @@ inline Maybe EnableEagerEnvironment(bool enable_eager_execution) { return Maybe::Ok(); } -inline Maybe IsEnvInited() { return Global::Get() != nullptr; } inline Maybe CurrentMachineId() { return GlobalProcessCtx::Rank(); } diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h index 3aff5b84afc..60b96993bc4 100644 --- a/oneflow/api/python/env/env_api.h +++ b/oneflow/api/python/env/env_api.h @@ -26,7 +26,6 @@ inline void EnableEagerEnvironment(bool enable_eager_execution) { return oneflow::EnableEagerEnvironment(enable_eager_execution).GetOrThrow(); } -inline bool IsEnvInited() { return oneflow::IsEnvInited().GetOrThrow(); } inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } From 3e8585e5fa20b97229d6b0be46a7ff814dc8cd83 Mon Sep 17 00:00:00 2001 From: cheng cheng <472491134@qq.com> Date: Tue, 22 Mar 2022 15:02:12 +0800 Subject: [PATCH 42/54] Clear empty thread when graph destroy (#7633) --- .github/workflows/test.yml | 11 ++++- .../multi_client_session_context.cpp | 1 + oneflow/core/framework/nn_graph.cpp | 39 +++++++-------- oneflow/core/job/runtime.cpp | 15 +++++- oneflow/core/job/runtime.h | 3 ++ oneflow/core/thread/thread.cpp | 2 + oneflow/core/thread/thread.h | 2 + oneflow/core/thread/thread_manager.cpp | 47 ++++++++++++++----- oneflow/core/thread/thread_manager.h | 5 +- 9 files changed, 90 insertions(+), 35 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 859664c29e8..4c15e89f3b8 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -529,7 +529,7 @@ jobs: working-directory: ${{ env.ONEFLOW_SRC }} run: | docker run -d --rm --privileged --shm-size=8g \ - --pids-limit -1 \ + --pids-limit 1000 \ --cap-add=SYS_PTRACE --security-opt seccomp=unconfined \ --runtime=nvidia \ -v /dataset:/dataset:ro -v /model_zoo:/model_zoo:ro \ @@ -691,6 +691,14 @@ jobs: EXTRA_DOCKER_ARGS+=" --env ONEFLOW_THREAD_ENABLE_LOCAL_MESSAGE_QUEUE=1" EXTRA_DOCKER_ARGS+=" --env ONEFLOW_KERNEL_DISABLE_BLOB_ACCESS_CHECKER=1" echo "EXTRA_DOCKER_ARGS=${EXTRA_DOCKER_ARGS}" >> $GITHUB_ENV + - name: Set Thread Limit (CPU) + if: ${{ !fromJson(matrix.cache-hit) && matrix.device == 'cpu' }} + run: | + echo "THREAD_LIMIT=8000" >> $GITHUB_ENV + - name: Set Thread Limit (CUDA) + if: ${{ !fromJson(matrix.cache-hit) && matrix.device == 'cuda' }} + run: | + echo "THREAD_LIMIT=3000" >> $GITHUB_ENV - name: Enable ONEFLOW_TEST_VERBOSE if: ${{ contains(github.event.pull_request.labels.*.name, 'need-test-verbose') }} run: | @@ -710,6 +718,7 @@ jobs: working-directory: ${{ env.ONEFLOW_SRC }} run: | docker run -d --rm --privileged --shm-size=8g \ + --pids-limit ${{ env.THREAD_LIMIT }} \ --cap-add=SYS_PTRACE --security-opt seccomp=unconfined \ --runtime=nvidia \ -v /dataset:/dataset:ro -v /model_zoo:/model_zoo:ro \ diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 15c368959b1..f106482894a 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -141,6 +141,7 @@ Maybe MultiClientSessionContext::TryInit(const std::string& config_proto_s } Maybe MultiClientSessionContext::UpdateResource(const Resource& reso_proto) { + CHECK_OR_RETURN(is_inited_); CHECK_NOTNULL_OR_RETURN((Global::Get())); Global::Get()->Update(reso_proto); return Maybe::Ok(); diff --git a/oneflow/core/framework/nn_graph.cpp b/oneflow/core/framework/nn_graph.cpp index a64286c344d..9ccd7e488ea 100644 --- a/oneflow/core/framework/nn_graph.cpp +++ b/oneflow/core/framework/nn_graph.cpp @@ -76,9 +76,12 @@ NNGraph::~NNGraph() { Maybe NNGraph::Close() { if (!is_closed_) { VLOG(1) << "Try to close c nn graph name " << name_ << "." << std::endl; - CloseRuntimeBuffers(); - runtime_.reset(); + if (runtime_inited_) { + CloseRuntimeBuffers(); + runtime_.reset(); + } session_ctx_->RemoveGraphFreeEagerTensors(name_); + is_closed_ = true; VLOG(1) << "Finish close c nn graph name " << name_ << "." << std::endl; session_ctx_.reset(); @@ -428,25 +431,23 @@ void NNGraph::NewRuntimeBuffers() { } void NNGraph::CloseRuntimeBuffers() { - if (runtime_inited_) { - { - auto* buffer_mgr = Global>>::Get(); - for (const std::string& output_op_name : outputs_op_names_) { - buffer_mgr->Get(GetOutputBufferName(name_, output_op_name))->Close(); - } - for (const std::string& input_op_name : inputs_op_names_) { - buffer_mgr->Get(GetInputBufferName(name_, input_op_name))->Close(); - } - buffer_mgr->Get(GetOutputCriticalSectionCallbackBufferName(name_))->Close(); - buffer_mgr->Get(GetOutputCriticalSectionWaitBufferName(name_))->Close(); - buffer_mgr->Get(GetInputCriticalSectionCallbackBufferName(name_))->Close(); - buffer_mgr->Get(GetInputCriticalSectionWaitBufferName(name_))->Close(); + { + auto* buffer_mgr = Global>>::Get(); + for (const std::string& output_op_name : outputs_op_names_) { + buffer_mgr->Get(GetOutputBufferName(name_, output_op_name))->Close(); } - { - auto* buffer_mgr = Global>>::Get(); - buffer_mgr->Get(GetCallbackNotifierBufferName(name_))->Close(); - buffer_mgr->Get(GetSourceTickBufferName(name_))->Close(); + for (const std::string& input_op_name : inputs_op_names_) { + buffer_mgr->Get(GetInputBufferName(name_, input_op_name))->Close(); } + buffer_mgr->Get(GetOutputCriticalSectionCallbackBufferName(name_))->Close(); + buffer_mgr->Get(GetOutputCriticalSectionWaitBufferName(name_))->Close(); + buffer_mgr->Get(GetInputCriticalSectionCallbackBufferName(name_))->Close(); + buffer_mgr->Get(GetInputCriticalSectionWaitBufferName(name_))->Close(); + } + { + auto* buffer_mgr = Global>>::Get(); + buffer_mgr->Get(GetCallbackNotifierBufferName(name_))->Close(); + buffer_mgr->Get(GetSourceTickBufferName(name_))->Close(); } } diff --git a/oneflow/core/job/runtime.cpp b/oneflow/core/job/runtime.cpp index f15357abc47..4607a7faa2f 100644 --- a/oneflow/core/job/runtime.cpp +++ b/oneflow/core/job/runtime.cpp @@ -58,10 +58,11 @@ bool HasNonCtrlConsumedRegstDescId(const TaskProto& task) { } // namespace Runtime::Runtime(const Plan& plan, const HashMap& variable_op_name2eager_blob) { + DumpThreadIdsFromPlan(plan); { // NOTE(chengcheng): All runtime Global objects AddPlan Global::Get()->AddPlan(plan, variable_op_name2eager_blob); - Global::Get()->AddPlan(plan); + Global::Get()->AddThreads(thread_ids_); Global::Get()->AddPlan(plan); collective_boxing_scheduler_plan_token_ = Global::Get()->AddPlan(plan); @@ -106,7 +107,19 @@ Runtime::~Runtime() { Global::Get()->WaitUntilCntEqualZero(GetRunningActorCountKeyByJobId(pair.first)); } OF_SESSION_BARRIER(); + Global::Get()->TryDeleteThreads(thread_ids_); Global::Get()->DeletePlan(collective_boxing_scheduler_plan_token_); } +void Runtime::DumpThreadIdsFromPlan(const Plan& plan) { + const int64_t this_rank = GlobalProcessCtx::Rank(); + for (const TaskProto& task : plan.task()) { + TaskId task_id = DecodeTaskIdFromInt64(task.task_id()); + StreamId stream_id = task_id.stream_id(); + if (stream_id.rank() != this_rank) { continue; } + int64_t thrd_id = EncodeStreamIdToInt64(stream_id); + thread_ids_.insert(thrd_id); + } +} + } // namespace oneflow diff --git a/oneflow/core/job/runtime.h b/oneflow/core/job/runtime.h index d784de07fb1..984142e8617 100644 --- a/oneflow/core/job/runtime.h +++ b/oneflow/core/job/runtime.h @@ -33,7 +33,10 @@ class Runtime final { Runtime(const Plan& plan, const HashMap& variable_op_name2eager_blob); private: + void DumpThreadIdsFromPlan(const Plan& plan); + HashMap job_id2actor_size_; + HashSet thread_ids_; boxing::collective::SchedulerPlanToken* collective_boxing_scheduler_plan_token_; }; diff --git a/oneflow/core/thread/thread.cpp b/oneflow/core/thread/thread.cpp index 5361bf1c694..6feac565cca 100644 --- a/oneflow/core/thread/thread.cpp +++ b/oneflow/core/thread/thread.cpp @@ -50,6 +50,8 @@ void Thread::AddTask(const TaskProto& task) { CHECK(id2task_.emplace(task.task_id(), task).second); } +bool Thread::Empty() const { return id2actor_ptr_.empty(); } + void Thread::PollMsgChannel() { while (true) { if (local_msg_queue_.empty()) { diff --git a/oneflow/core/thread/thread.h b/oneflow/core/thread/thread.h index e51f1eb0fda..7c7c2bacca8 100644 --- a/oneflow/core/thread/thread.h +++ b/oneflow/core/thread/thread.h @@ -34,6 +34,8 @@ class Thread { virtual ~Thread(); void AddTask(const TaskProto&); + // NOTE(chengcheng): Indicates whether all actors in the thread have been destructed. + bool Empty() const; Channel* GetMsgChannelPtr() { return &msg_channel_; } diff --git a/oneflow/core/thread/thread_manager.cpp b/oneflow/core/thread/thread_manager.cpp index 16db1d8f151..4e65efc0160 100644 --- a/oneflow/core/thread/thread_manager.cpp +++ b/oneflow/core/thread/thread_manager.cpp @@ -17,17 +17,12 @@ limitations under the License. #include "oneflow/core/job/resource_desc.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/control/global_process_ctx.h" -#include "oneflow/core/job/global_for.h" namespace oneflow { ThreadMgr::~ThreadMgr() { - for (auto& thread_pair : threads_) { - ActorMsg msg = ActorMsg::BuildCommandMsg(-1, ActorCmd::kStopThread); - thread_pair.second->GetMsgChannelPtr()->Send(msg); - thread_pair.second.reset(); - VLOG(3) << "actor thread " << thread_pair.first << " finish"; - } + CHECK(threads_.empty()) << " Runtime Error! num = " << threads_.size() + << " threads did not destroy with graph."; } Thread* ThreadMgr::GetThrd(int64_t thrd_id) { @@ -36,20 +31,46 @@ Thread* ThreadMgr::GetThrd(int64_t thrd_id) { return iter->second.get(); } -void ThreadMgr::AddPlan(const Plan& plan) { +void ThreadMgr::AddThreads(const HashSet& thread_ids) { const int64_t this_rank = GlobalProcessCtx::Rank(); - for (const TaskProto& task : plan.task()) { - TaskId task_id = DecodeTaskIdFromInt64(task.task_id()); - StreamId stream_id = task_id.stream_id(); + for (int64_t thrd_id : thread_ids) { + const auto& it = threads_.find(thrd_id); + if (it != threads_.end()) { + // NOTE(chengcheng): check thread is not null. + CHECK(it->second) << " Runtime Error! Thread: " << thrd_id << " in manager must be NOT null."; + continue; + } + StreamId stream_id = DecodeStreamIdFromInt64(thrd_id); if (stream_id.rank() != this_rank) { continue; } - int64_t thrd_id = EncodeStreamIdToInt64(stream_id); - if (threads_.find(thrd_id) != threads_.end()) { continue; } Thread* thread = new Thread(stream_id); CHECK_NOTNULL(thread); threads_[thrd_id].reset(thread); } } +void ThreadMgr::TryDeleteThreads(const HashSet& thread_ids) { + std::unique_lock lock(mutex4del_threads_); + for (int64_t thrd_id : thread_ids) { + const auto& it = threads_.find(thrd_id); + if (it == threads_.end()) { continue; } + auto& thread = it->second; + CHECK(thread) << " actor thread " << thrd_id << " non-existent but want to delete"; + if (thread->Empty()) { + // NOTE(chengcheng): Only delete thread when it is empty. + // We need send Stop msg to exit the main loop of the thread. Here we can safely call reset + // directly, because the thread destructor will specify actor_thread_.join() to blocking + // wait for the end of the thread real execution. + ActorMsg msg = ActorMsg::BuildCommandMsg(-1, ActorCmd::kStopThread); + thread->GetMsgChannelPtr()->Send(msg); + thread.reset(); + VLOG(2) << " actor thread " << thrd_id << " finish."; + threads_.erase(it); + } else { + LOG(INFO) << " actor thread " << thrd_id << " not delete because it is not empty."; + } + } +} + void SingleThreadLoop(size_t num, std::function Callback) { FOR_RANGE(size_t, i, 0, num) { Callback(i); } } diff --git a/oneflow/core/thread/thread_manager.h b/oneflow/core/thread/thread_manager.h index 39602dd8681..b8be8b6f11b 100644 --- a/oneflow/core/thread/thread_manager.h +++ b/oneflow/core/thread/thread_manager.h @@ -16,6 +16,7 @@ limitations under the License. #ifndef ONEFLOW_CORE_THREAD_THREAD_MANAGER_H_ #define ONEFLOW_CORE_THREAD_THREAD_MANAGER_H_ +#include #include "oneflow/core/common/channel.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/common/auto_registration_factory.h" @@ -36,13 +37,15 @@ class ThreadMgr final { ThreadMgr() = default; ~ThreadMgr(); - void AddPlan(const Plan& plan); + void AddThreads(const HashSet& thread_ids); + void TryDeleteThreads(const HashSet& thread_ids); Thread* GetThrd(int64_t thrd_id); private: friend class Global; HashMap> threads_; + std::mutex mutex4del_threads_; }; void SingleThreadLoop(size_t num, std::function Callback); From 46c9d9727a8c5863f65652137bf17350e8d92bd3 Mon Sep 17 00:00:00 2001 From: cheng cheng <472491134@qq.com> Date: Tue, 22 Mar 2022 15:04:51 +0800 Subject: [PATCH 43/54] Revert "Clear empty thread when graph destroy (#7633)" (#7860) This reverts commit 3e8585e5fa20b97229d6b0be46a7ff814dc8cd83. --- .github/workflows/test.yml | 11 +---- .../multi_client_session_context.cpp | 1 - oneflow/core/framework/nn_graph.cpp | 39 ++++++++------- oneflow/core/job/runtime.cpp | 15 +----- oneflow/core/job/runtime.h | 3 -- oneflow/core/thread/thread.cpp | 2 - oneflow/core/thread/thread.h | 2 - oneflow/core/thread/thread_manager.cpp | 47 +++++-------------- oneflow/core/thread/thread_manager.h | 5 +- 9 files changed, 35 insertions(+), 90 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 4c15e89f3b8..859664c29e8 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -529,7 +529,7 @@ jobs: working-directory: ${{ env.ONEFLOW_SRC }} run: | docker run -d --rm --privileged --shm-size=8g \ - --pids-limit 1000 \ + --pids-limit -1 \ --cap-add=SYS_PTRACE --security-opt seccomp=unconfined \ --runtime=nvidia \ -v /dataset:/dataset:ro -v /model_zoo:/model_zoo:ro \ @@ -691,14 +691,6 @@ jobs: EXTRA_DOCKER_ARGS+=" --env ONEFLOW_THREAD_ENABLE_LOCAL_MESSAGE_QUEUE=1" EXTRA_DOCKER_ARGS+=" --env ONEFLOW_KERNEL_DISABLE_BLOB_ACCESS_CHECKER=1" echo "EXTRA_DOCKER_ARGS=${EXTRA_DOCKER_ARGS}" >> $GITHUB_ENV - - name: Set Thread Limit (CPU) - if: ${{ !fromJson(matrix.cache-hit) && matrix.device == 'cpu' }} - run: | - echo "THREAD_LIMIT=8000" >> $GITHUB_ENV - - name: Set Thread Limit (CUDA) - if: ${{ !fromJson(matrix.cache-hit) && matrix.device == 'cuda' }} - run: | - echo "THREAD_LIMIT=3000" >> $GITHUB_ENV - name: Enable ONEFLOW_TEST_VERBOSE if: ${{ contains(github.event.pull_request.labels.*.name, 'need-test-verbose') }} run: | @@ -718,7 +710,6 @@ jobs: working-directory: ${{ env.ONEFLOW_SRC }} run: | docker run -d --rm --privileged --shm-size=8g \ - --pids-limit ${{ env.THREAD_LIMIT }} \ --cap-add=SYS_PTRACE --security-opt seccomp=unconfined \ --runtime=nvidia \ -v /dataset:/dataset:ro -v /model_zoo:/model_zoo:ro \ diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index f106482894a..15c368959b1 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -141,7 +141,6 @@ Maybe MultiClientSessionContext::TryInit(const std::string& config_proto_s } Maybe MultiClientSessionContext::UpdateResource(const Resource& reso_proto) { - CHECK_OR_RETURN(is_inited_); CHECK_NOTNULL_OR_RETURN((Global::Get())); Global::Get()->Update(reso_proto); return Maybe::Ok(); diff --git a/oneflow/core/framework/nn_graph.cpp b/oneflow/core/framework/nn_graph.cpp index 9ccd7e488ea..a64286c344d 100644 --- a/oneflow/core/framework/nn_graph.cpp +++ b/oneflow/core/framework/nn_graph.cpp @@ -76,12 +76,9 @@ NNGraph::~NNGraph() { Maybe NNGraph::Close() { if (!is_closed_) { VLOG(1) << "Try to close c nn graph name " << name_ << "." << std::endl; - if (runtime_inited_) { - CloseRuntimeBuffers(); - runtime_.reset(); - } + CloseRuntimeBuffers(); + runtime_.reset(); session_ctx_->RemoveGraphFreeEagerTensors(name_); - is_closed_ = true; VLOG(1) << "Finish close c nn graph name " << name_ << "." << std::endl; session_ctx_.reset(); @@ -431,23 +428,25 @@ void NNGraph::NewRuntimeBuffers() { } void NNGraph::CloseRuntimeBuffers() { - { - auto* buffer_mgr = Global>>::Get(); - for (const std::string& output_op_name : outputs_op_names_) { - buffer_mgr->Get(GetOutputBufferName(name_, output_op_name))->Close(); + if (runtime_inited_) { + { + auto* buffer_mgr = Global>>::Get(); + for (const std::string& output_op_name : outputs_op_names_) { + buffer_mgr->Get(GetOutputBufferName(name_, output_op_name))->Close(); + } + for (const std::string& input_op_name : inputs_op_names_) { + buffer_mgr->Get(GetInputBufferName(name_, input_op_name))->Close(); + } + buffer_mgr->Get(GetOutputCriticalSectionCallbackBufferName(name_))->Close(); + buffer_mgr->Get(GetOutputCriticalSectionWaitBufferName(name_))->Close(); + buffer_mgr->Get(GetInputCriticalSectionCallbackBufferName(name_))->Close(); + buffer_mgr->Get(GetInputCriticalSectionWaitBufferName(name_))->Close(); } - for (const std::string& input_op_name : inputs_op_names_) { - buffer_mgr->Get(GetInputBufferName(name_, input_op_name))->Close(); + { + auto* buffer_mgr = Global>>::Get(); + buffer_mgr->Get(GetCallbackNotifierBufferName(name_))->Close(); + buffer_mgr->Get(GetSourceTickBufferName(name_))->Close(); } - buffer_mgr->Get(GetOutputCriticalSectionCallbackBufferName(name_))->Close(); - buffer_mgr->Get(GetOutputCriticalSectionWaitBufferName(name_))->Close(); - buffer_mgr->Get(GetInputCriticalSectionCallbackBufferName(name_))->Close(); - buffer_mgr->Get(GetInputCriticalSectionWaitBufferName(name_))->Close(); - } - { - auto* buffer_mgr = Global>>::Get(); - buffer_mgr->Get(GetCallbackNotifierBufferName(name_))->Close(); - buffer_mgr->Get(GetSourceTickBufferName(name_))->Close(); } } diff --git a/oneflow/core/job/runtime.cpp b/oneflow/core/job/runtime.cpp index 4607a7faa2f..f15357abc47 100644 --- a/oneflow/core/job/runtime.cpp +++ b/oneflow/core/job/runtime.cpp @@ -58,11 +58,10 @@ bool HasNonCtrlConsumedRegstDescId(const TaskProto& task) { } // namespace Runtime::Runtime(const Plan& plan, const HashMap& variable_op_name2eager_blob) { - DumpThreadIdsFromPlan(plan); { // NOTE(chengcheng): All runtime Global objects AddPlan Global::Get()->AddPlan(plan, variable_op_name2eager_blob); - Global::Get()->AddThreads(thread_ids_); + Global::Get()->AddPlan(plan); Global::Get()->AddPlan(plan); collective_boxing_scheduler_plan_token_ = Global::Get()->AddPlan(plan); @@ -107,19 +106,7 @@ Runtime::~Runtime() { Global::Get()->WaitUntilCntEqualZero(GetRunningActorCountKeyByJobId(pair.first)); } OF_SESSION_BARRIER(); - Global::Get()->TryDeleteThreads(thread_ids_); Global::Get()->DeletePlan(collective_boxing_scheduler_plan_token_); } -void Runtime::DumpThreadIdsFromPlan(const Plan& plan) { - const int64_t this_rank = GlobalProcessCtx::Rank(); - for (const TaskProto& task : plan.task()) { - TaskId task_id = DecodeTaskIdFromInt64(task.task_id()); - StreamId stream_id = task_id.stream_id(); - if (stream_id.rank() != this_rank) { continue; } - int64_t thrd_id = EncodeStreamIdToInt64(stream_id); - thread_ids_.insert(thrd_id); - } -} - } // namespace oneflow diff --git a/oneflow/core/job/runtime.h b/oneflow/core/job/runtime.h index 984142e8617..d784de07fb1 100644 --- a/oneflow/core/job/runtime.h +++ b/oneflow/core/job/runtime.h @@ -33,10 +33,7 @@ class Runtime final { Runtime(const Plan& plan, const HashMap& variable_op_name2eager_blob); private: - void DumpThreadIdsFromPlan(const Plan& plan); - HashMap job_id2actor_size_; - HashSet thread_ids_; boxing::collective::SchedulerPlanToken* collective_boxing_scheduler_plan_token_; }; diff --git a/oneflow/core/thread/thread.cpp b/oneflow/core/thread/thread.cpp index 6feac565cca..5361bf1c694 100644 --- a/oneflow/core/thread/thread.cpp +++ b/oneflow/core/thread/thread.cpp @@ -50,8 +50,6 @@ void Thread::AddTask(const TaskProto& task) { CHECK(id2task_.emplace(task.task_id(), task).second); } -bool Thread::Empty() const { return id2actor_ptr_.empty(); } - void Thread::PollMsgChannel() { while (true) { if (local_msg_queue_.empty()) { diff --git a/oneflow/core/thread/thread.h b/oneflow/core/thread/thread.h index 7c7c2bacca8..e51f1eb0fda 100644 --- a/oneflow/core/thread/thread.h +++ b/oneflow/core/thread/thread.h @@ -34,8 +34,6 @@ class Thread { virtual ~Thread(); void AddTask(const TaskProto&); - // NOTE(chengcheng): Indicates whether all actors in the thread have been destructed. - bool Empty() const; Channel* GetMsgChannelPtr() { return &msg_channel_; } diff --git a/oneflow/core/thread/thread_manager.cpp b/oneflow/core/thread/thread_manager.cpp index 4e65efc0160..16db1d8f151 100644 --- a/oneflow/core/thread/thread_manager.cpp +++ b/oneflow/core/thread/thread_manager.cpp @@ -17,12 +17,17 @@ limitations under the License. #include "oneflow/core/job/resource_desc.h" #include "oneflow/core/job/global_for.h" #include "oneflow/core/control/global_process_ctx.h" +#include "oneflow/core/job/global_for.h" namespace oneflow { ThreadMgr::~ThreadMgr() { - CHECK(threads_.empty()) << " Runtime Error! num = " << threads_.size() - << " threads did not destroy with graph."; + for (auto& thread_pair : threads_) { + ActorMsg msg = ActorMsg::BuildCommandMsg(-1, ActorCmd::kStopThread); + thread_pair.second->GetMsgChannelPtr()->Send(msg); + thread_pair.second.reset(); + VLOG(3) << "actor thread " << thread_pair.first << " finish"; + } } Thread* ThreadMgr::GetThrd(int64_t thrd_id) { @@ -31,46 +36,20 @@ Thread* ThreadMgr::GetThrd(int64_t thrd_id) { return iter->second.get(); } -void ThreadMgr::AddThreads(const HashSet& thread_ids) { +void ThreadMgr::AddPlan(const Plan& plan) { const int64_t this_rank = GlobalProcessCtx::Rank(); - for (int64_t thrd_id : thread_ids) { - const auto& it = threads_.find(thrd_id); - if (it != threads_.end()) { - // NOTE(chengcheng): check thread is not null. - CHECK(it->second) << " Runtime Error! Thread: " << thrd_id << " in manager must be NOT null."; - continue; - } - StreamId stream_id = DecodeStreamIdFromInt64(thrd_id); + for (const TaskProto& task : plan.task()) { + TaskId task_id = DecodeTaskIdFromInt64(task.task_id()); + StreamId stream_id = task_id.stream_id(); if (stream_id.rank() != this_rank) { continue; } + int64_t thrd_id = EncodeStreamIdToInt64(stream_id); + if (threads_.find(thrd_id) != threads_.end()) { continue; } Thread* thread = new Thread(stream_id); CHECK_NOTNULL(thread); threads_[thrd_id].reset(thread); } } -void ThreadMgr::TryDeleteThreads(const HashSet& thread_ids) { - std::unique_lock lock(mutex4del_threads_); - for (int64_t thrd_id : thread_ids) { - const auto& it = threads_.find(thrd_id); - if (it == threads_.end()) { continue; } - auto& thread = it->second; - CHECK(thread) << " actor thread " << thrd_id << " non-existent but want to delete"; - if (thread->Empty()) { - // NOTE(chengcheng): Only delete thread when it is empty. - // We need send Stop msg to exit the main loop of the thread. Here we can safely call reset - // directly, because the thread destructor will specify actor_thread_.join() to blocking - // wait for the end of the thread real execution. - ActorMsg msg = ActorMsg::BuildCommandMsg(-1, ActorCmd::kStopThread); - thread->GetMsgChannelPtr()->Send(msg); - thread.reset(); - VLOG(2) << " actor thread " << thrd_id << " finish."; - threads_.erase(it); - } else { - LOG(INFO) << " actor thread " << thrd_id << " not delete because it is not empty."; - } - } -} - void SingleThreadLoop(size_t num, std::function Callback) { FOR_RANGE(size_t, i, 0, num) { Callback(i); } } diff --git a/oneflow/core/thread/thread_manager.h b/oneflow/core/thread/thread_manager.h index b8be8b6f11b..39602dd8681 100644 --- a/oneflow/core/thread/thread_manager.h +++ b/oneflow/core/thread/thread_manager.h @@ -16,7 +16,6 @@ limitations under the License. #ifndef ONEFLOW_CORE_THREAD_THREAD_MANAGER_H_ #define ONEFLOW_CORE_THREAD_THREAD_MANAGER_H_ -#include #include "oneflow/core/common/channel.h" #include "oneflow/core/common/protobuf.h" #include "oneflow/core/common/auto_registration_factory.h" @@ -37,15 +36,13 @@ class ThreadMgr final { ThreadMgr() = default; ~ThreadMgr(); - void AddThreads(const HashSet& thread_ids); - void TryDeleteThreads(const HashSet& thread_ids); + void AddPlan(const Plan& plan); Thread* GetThrd(int64_t thrd_id); private: friend class Global; HashMap> threads_; - std::mutex mutex4del_threads_; }; void SingleThreadLoop(size_t num, std::function Callback); From 3be6f810e4b4cbc51d136ee07aacd8f4bc1bd268 Mon Sep 17 00:00:00 2001 From: chengtbf <472491134@qq.com> Date: Wed, 23 Mar 2022 12:08:15 +0800 Subject: [PATCH 44/54] fix a ref-cnt bug in TryRunBarrierInstruction. --- oneflow/core/vm/virtual_machine_engine.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/oneflow/core/vm/virtual_machine_engine.cpp b/oneflow/core/vm/virtual_machine_engine.cpp index 561dab60a93..1c69b00a081 100644 --- a/oneflow/core/vm/virtual_machine_engine.cpp +++ b/oneflow/core/vm/virtual_machine_engine.cpp @@ -506,7 +506,6 @@ void VirtualMachineEngine::TryRunBarrierInstruction(const ScheduleCtx& schedule_ mut_barrier_instruction_list()->Erase(sequnential_instruction); intrusive::shared_ptr instr_msg(sequnential_instruction->mut_instr_msg()); LivelyInstructionListErase(sequnential_instruction); - sequnential_instruction->clear_instr_msg(); constexpr int kZeroWindowSize = 0; // flush immediately. MoveInstructionMsgToGarbageMsgList(kZeroWindowSize, std::move(instr_msg), schedule_ctx); OF_PROFILER_RANGE_POP(); From a80aea637e2bb469b51b4f641cc495402e64f7b1 Mon Sep 17 00:00:00 2001 From: chengtbf <472491134@qq.com> Date: Wed, 23 Mar 2022 14:26:06 +0800 Subject: [PATCH 45/54] rm env_api --- oneflow/api/python/env/env.cpp | 1 - oneflow/api/python/env/env_api.h | 51 -------------------------------- 2 files changed, 52 deletions(-) delete mode 100644 oneflow/api/python/env/env_api.h diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 57e92712b79..099b080c706 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -16,7 +16,6 @@ limitations under the License. #include #include "oneflow/api/python/env/env.h" #include "oneflow/api/python/of_api_registry.h" -#include "oneflow/api/python/env/env_api.h" #include "oneflow/core/job/env_global_objects_scope.h" #include "oneflow/core/common/global.h" #include "oneflow/core/vm/vm_util.h" diff --git a/oneflow/api/python/env/env_api.h b/oneflow/api/python/env/env_api.h deleted file mode 100644 index 2946924bd70..00000000000 --- a/oneflow/api/python/env/env_api.h +++ /dev/null @@ -1,51 +0,0 @@ -/* -Copyright 2020 The OneFlow 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. -*/ -#ifndef ONEFLOW_API_PYTHON_ENV_ENV_API_H_ -#define ONEFLOW_API_PYTHON_ENV_ENV_API_H_ - -#include "oneflow/api/python/env/env.h" - -inline std::string CurrentResource() { return oneflow::CurrentResource().GetOrThrow(); } - -inline std::string EnvResource() { return oneflow::EnvResource().GetOrThrow(); } - -inline void EnableEagerEnvironment(bool enable_eager_execution) { - return oneflow::EnableEagerEnvironment(enable_eager_execution).GetOrThrow(); -} - -inline long long CurrentMachineId() { return oneflow::CurrentMachineId().GetOrThrow(); } - -inline int64_t GetRank() { return oneflow::GetRank().GetOrThrow(); } - -inline size_t GetWorldSize() { return oneflow::GetWorldSize().GetOrThrow(); } - -inline size_t GetNodeSize() { return oneflow::GetNodeSize().GetOrThrow(); } - -inline size_t GetLocalRank() { return oneflow::GetLocalRank().GetOrThrow(); } - -inline size_t CudaGetDeviceCount() { return oneflow::CudaGetDeviceCount().GetOrThrow(); } - -inline void SetFLAGS_alsologtostderr(bool flag) { - return oneflow::SetFLAGS_alsologtostderr(flag).GetOrThrow(); -} - -inline bool GetFLAGS_alsologtostderr() { return oneflow::GetFLAGS_alsologtostderr().GetOrThrow(); } - -inline void SetFLAGS_v(int32_t v_level) { return oneflow::SetFLAGS_v(v_level).GetOrThrow(); } - -inline int32_t GetFLAGS_v() { return oneflow::GetFLAGS_v().GetOrThrow(); } - -#endif // ONEFLOW_API_PYTHON_ENV_ENV_API_H_ From ec04de3c80f45e0afeb9c2027a103bbdae02dc55 Mon Sep 17 00:00:00 2001 From: chengtbf <472491134@qq.com> Date: Wed, 23 Mar 2022 16:58:14 +0800 Subject: [PATCH 46/54] fix clang-tidy error --- oneflow/core/framework/multi_client_session_context.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index 15c368959b1..ee4ea4a71e7 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -61,8 +61,8 @@ int32_t GetCpuDeviceNum() { return std::thread::hardware_concurrency(); } } // namespace MultiClientSessionContext::MultiClientSessionContext( - const std::shared_ptr& env_ctx) { - env_ctx_ = env_ctx; + const std::shared_ptr& env_ctx) + : env_ctx_(env_ctx) { CHECK(Global::Get() == nullptr); Global::SetAllocated(this); } From fde02a62e967b5fdab4677f3baa9dde35f115743 Mon Sep 17 00:00:00 2001 From: chengtbf <472491134@qq.com> Date: Wed, 23 Mar 2022 17:15:19 +0800 Subject: [PATCH 47/54] fix clang-tidy in env_imp --- oneflow/api/cpp/env_impl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/oneflow/api/cpp/env_impl.cpp b/oneflow/api/cpp/env_impl.cpp index 60118e72fd9..c6b199ad95b 100644 --- a/oneflow/api/cpp/env_impl.cpp +++ b/oneflow/api/cpp/env_impl.cpp @@ -115,7 +115,7 @@ OneFlowEnv::OneFlowEnv() { CompleteEnvProto(env_proto); env_ctx_ = std::make_shared(); - env_ctx_->Init(env_proto); + CHECK_JUST(env_ctx_->Init(env_proto)); of::ConfigProto config_proto; config_proto.mutable_resource()->set_cpu_device_num(1); // useless, will be set in TryInit @@ -124,7 +124,7 @@ OneFlowEnv::OneFlowEnv() { config_proto.set_session_id(session_id); session_ctx_ = std::make_shared(env_ctx_); - session_ctx_->TryInit(config_proto); + CHECK_JUST(session_ctx_->TryInit(config_proto)); } OneFlowEnv::~OneFlowEnv() { From b7774c1c540fb05d843f1894c2197031df48404c Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 8 Apr 2022 19:01:58 +0800 Subject: [PATCH 48/54] refine env api --- oneflow/api/cpp/env_impl.cpp | 3 +-- oneflow/api/python/env/env.cpp | 6 +----- .../framework/multi_client_session_context.h | 3 +++ oneflow/core/job/env_global_objects_scope.cpp | 18 ++++++++++-------- oneflow/core/job/env_global_objects_scope.h | 10 ++++++---- python/oneflow/framework/c_api_util.py | 3 +-- .../oneflow/framework/multi_client_session.py | 1 + 7 files changed, 23 insertions(+), 21 deletions(-) diff --git a/oneflow/api/cpp/env_impl.cpp b/oneflow/api/cpp/env_impl.cpp index c6b199ad95b..04b3fce0d9f 100644 --- a/oneflow/api/cpp/env_impl.cpp +++ b/oneflow/api/cpp/env_impl.cpp @@ -114,8 +114,7 @@ OneFlowEnv::OneFlowEnv() { of::EnvProto env_proto; CompleteEnvProto(env_proto); - env_ctx_ = std::make_shared(); - CHECK_JUST(env_ctx_->Init(env_proto)); + env_ctx_ = std::make_shared(env_proto); of::ConfigProto config_proto; config_proto.mutable_resource()->set_cpu_device_num(1); // useless, will be set in TryInit diff --git a/oneflow/api/python/env/env.cpp b/oneflow/api/python/env/env.cpp index 477326aa28e..80e4ab4d6d9 100644 --- a/oneflow/api/python/env/env.cpp +++ b/oneflow/api/python/env/env.cpp @@ -45,11 +45,7 @@ ONEFLOW_API_PYBIND11_MODULE("", m) { py::class_>( m, "EnvContext") - .def(py::init<>()) - .def("init", - [](oneflow::EnvGlobalObjectsScope& env, const std::string& env_proto_str) { - return env.Init(env_proto_str).GetOrThrow(); - }) + .def(py::init()) .def("SwitchToShuttingDownPhase", &SwitchToShuttingDownPhase, py::call_guard()); diff --git a/oneflow/core/framework/multi_client_session_context.h b/oneflow/core/framework/multi_client_session_context.h index a8c25a9e116..b18e1c87c4d 100644 --- a/oneflow/core/framework/multi_client_session_context.h +++ b/oneflow/core/framework/multi_client_session_context.h @@ -49,6 +49,9 @@ class MultiClientSessionContext { GetFreeEagerTensorNamePairByGraphName(const std::string& graph_name); void RemoveGraphFreeEagerTensors(const std::string& graph_name); + private: + MultiClientSessionContext() {} + private: bool is_inited_ = false; std::shared_ptr env_ctx_; diff --git a/oneflow/core/job/env_global_objects_scope.cpp b/oneflow/core/job/env_global_objects_scope.cpp index f692c5a31aa..164f3c208fd 100644 --- a/oneflow/core/job/env_global_objects_scope.cpp +++ b/oneflow/core/job/env_global_objects_scope.cpp @@ -139,19 +139,21 @@ bool CommNetIBEnabled() { } // namespace -EnvGlobalObjectsScope::EnvGlobalObjectsScope() { - CHECK(Global::Get() == nullptr); - Global::SetAllocated(this); -} - -Maybe EnvGlobalObjectsScope::Init(const std::string& env_proto_str) { +EnvGlobalObjectsScope::EnvGlobalObjectsScope(const std::string& env_proto_str) { EnvProto env_proto; - CHECK_OR_RETURN(TxtString2PbMessage(env_proto_str, &env_proto)) + CHECK(TxtString2PbMessage(env_proto_str, &env_proto)) << "failed to parse env_proto" << env_proto_str; - return Init(env_proto); + CHECK_JUST(Init(env_proto)); +} + +EnvGlobalObjectsScope::EnvGlobalObjectsScope(const EnvProto& env_proto) { + CHECK_JUST(Init(env_proto)); } Maybe EnvGlobalObjectsScope::Init(const EnvProto& env_proto) { + CHECK(Global::Get() == nullptr); + Global::SetAllocated(this); + InitLogging(env_proto.cpp_logging_conf()); Global::New(env_proto); Global::New(); diff --git a/oneflow/core/job/env_global_objects_scope.h b/oneflow/core/job/env_global_objects_scope.h index 40151cc6559..9efb4ec5be7 100644 --- a/oneflow/core/job/env_global_objects_scope.h +++ b/oneflow/core/job/env_global_objects_scope.h @@ -28,18 +28,20 @@ class ParallelDesc; class EnvGlobalObjectsScope final { public: OF_DISALLOW_COPY_AND_MOVE(EnvGlobalObjectsScope); - EnvGlobalObjectsScope(); + explicit EnvGlobalObjectsScope(const std::string& env_proto_str); + explicit EnvGlobalObjectsScope(const EnvProto& env_proto); ~EnvGlobalObjectsScope(); - Maybe Init(const EnvProto& env_proto); - Maybe Init(const std::string& env_proto_str); - Maybe init_is_normal_exit(bool is_normal_exit) { CHECK_OR_RETURN(!is_normal_exit_.has_value()); is_normal_exit_ = is_normal_exit; return Maybe::Ok(); } + private: + EnvGlobalObjectsScope() {} + Maybe Init(const EnvProto& env_proto); + private: Optional is_normal_exit_; }; diff --git a/python/oneflow/framework/c_api_util.py b/python/oneflow/framework/c_api_util.py index 71ed9fff259..ce046ced063 100644 --- a/python/oneflow/framework/c_api_util.py +++ b/python/oneflow/framework/c_api_util.py @@ -45,8 +45,7 @@ def EnvResource(): def GetEnvContext(env_proto): assert type(env_proto) is env_pb2.EnvProto env_proto_str = text_format.MessageToString(env_proto) - env_ctx = oneflow._oneflow_internal.EnvContext() - env_ctx.init(env_proto_str) + env_ctx = oneflow._oneflow_internal.EnvContext(env_proto_str) return env_ctx diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index a6fe62116a4..4a3eaf27860 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -34,6 +34,7 @@ def __init__(self, env, sess_id): self._id = sess_id self._env = env assert self._env is not None + # TODO(strint): Remove old session. self._internal_sess = oneflow._oneflow_internal.RegsiterSession(sess_id) # New a MultiClientSessionContext self._session_ctx = oneflow._oneflow_internal.SessionContext(self._env._env_cxt) From 8d4513469821cc2dc7d914c29db59e73c2061739 Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 8 Apr 2022 19:14:34 +0800 Subject: [PATCH 49/54] format --- python/oneflow/framework/multi_client_session.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/oneflow/framework/multi_client_session.py b/python/oneflow/framework/multi_client_session.py index 4a3eaf27860..72c6e093779 100644 --- a/python/oneflow/framework/multi_client_session.py +++ b/python/oneflow/framework/multi_client_session.py @@ -34,7 +34,7 @@ def __init__(self, env, sess_id): self._id = sess_id self._env = env assert self._env is not None - # TODO(strint): Remove old session. + # TODO(strint): Remove old session. self._internal_sess = oneflow._oneflow_internal.RegsiterSession(sess_id) # New a MultiClientSessionContext self._session_ctx = oneflow._oneflow_internal.SessionContext(self._env._env_cxt) From 2104348eafa57eed04b79052c000e49ccb8c6468 Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 8 Apr 2022 19:35:53 +0800 Subject: [PATCH 50/54] refine graph del and sync at shuttingdown --- python/oneflow/__init__.py | 2 +- python/oneflow/framework/env_util.py | 18 ++++++++++++++++-- python/oneflow/nn/graph/graph.py | 4 ++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/python/oneflow/__init__.py b/python/oneflow/__init__.py index e5b2585c4b9..2be840572e0 100755 --- a/python/oneflow/__init__.py +++ b/python/oneflow/__init__.py @@ -257,7 +257,7 @@ def is_normal_exit(self): def atexit_hook(hook): oneflow.framework.session_context.TryCloseDefaultSession() - __oneflow_global_unique_env.SwitchToShuttingDownPhase(hook.is_normal_exit()) + __oneflow_global_unique_env.switch_to_shutting_down(hook.is_normal_exit()) atexit.register(atexit_hook, hook) diff --git a/python/oneflow/framework/env_util.py b/python/oneflow/framework/env_util.py index afc63e07934..ef6b80ca327 100644 --- a/python/oneflow/framework/env_util.py +++ b/python/oneflow/framework/env_util.py @@ -201,8 +201,22 @@ def __init__(self): if not HasAllMultiClientEnvVars(): SetDefaultMultiClientEnvVars() self._env_cxt = create_env() - - def SwitchToShuttingDownPhase(self, is_normal_exit=True): + self._shutting_down = [False] + + def is_shutting_down(self): + """ + Whether the interpreter is currently shutting down. + For use in finalizers, __del__ methods, and similar; it is advised + to early bind this function rather than look it up when calling it, + since at shutdown module globals may be cleared. + + Please refer to: https://github.com/Oneflow-Inc/OneTeam/issues/1219#issuecomment-1092370402 + This solution is obtained from cupy code: https://github.com/cupy/cupy/pull/2809 + """ + return self._shutting_down[0] + + def switch_to_shutting_down(self, is_normal_exit=True): + self._shutting_down[0] = True self._env_cxt.SwitchToShuttingDownPhase(is_normal_exit) diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index 348aedc3fc8..ae8d62258af 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -1176,6 +1176,10 @@ def __getattr__(self, name: str): def __del__(self): # Ensure vm has finished running this graph. + if self._session._env.is_shutting_down(): + # Env shutting down will do sync in SwitchToShuttingDownPhase. + # So it's save to skip sync here. + return oneflow._oneflow_internal.eager.Sync() From 809b483b1e2db6731b6188ca3722062395b579fd Mon Sep 17 00:00:00 2001 From: strint Date: Fri, 8 Apr 2022 19:42:13 +0800 Subject: [PATCH 51/54] fix typo --- python/oneflow/nn/graph/graph.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index ae8d62258af..bb2f575b52f 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -1178,7 +1178,7 @@ def __del__(self): # Ensure vm has finished running this graph. if self._session._env.is_shutting_down(): # Env shutting down will do sync in SwitchToShuttingDownPhase. - # So it's save to skip sync here. + # So it's safe to skip sync here. return oneflow._oneflow_internal.eager.Sync() From 6199229e5cb31b80c382610093b7dcda4e68a0b9 Mon Sep 17 00:00:00 2001 From: strint Date: Sat, 9 Apr 2022 14:34:55 +0800 Subject: [PATCH 52/54] add comment --- python/oneflow/nn/graph/graph.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/oneflow/nn/graph/graph.py b/python/oneflow/nn/graph/graph.py index bb2f575b52f..d760907f629 100644 --- a/python/oneflow/nn/graph/graph.py +++ b/python/oneflow/nn/graph/graph.py @@ -1177,7 +1177,8 @@ def __getattr__(self, name: str): def __del__(self): # Ensure vm has finished running this graph. if self._session._env.is_shutting_down(): - # Env shutting down will do sync in SwitchToShuttingDownPhase. + # After python shutting down, it's not safe to call oneflow._oneflow_internal.eager. + # But shutting down will do sync in SwitchToShuttingDownPhase. # So it's safe to skip sync here. return oneflow._oneflow_internal.eager.Sync() From da54f3603ea1146f1c1778b332073fe513b7dba3 Mon Sep 17 00:00:00 2001 From: strint Date: Sun, 10 Apr 2022 15:25:33 +0800 Subject: [PATCH 53/54] rm useless --- oneflow/core/framework/multi_client_session_context.cpp | 1 - oneflow/core/framework/multi_client_session_context.h | 1 - 2 files changed, 2 deletions(-) diff --git a/oneflow/core/framework/multi_client_session_context.cpp b/oneflow/core/framework/multi_client_session_context.cpp index ee4ea4a71e7..e26f32c2ed1 100644 --- a/oneflow/core/framework/multi_client_session_context.cpp +++ b/oneflow/core/framework/multi_client_session_context.cpp @@ -14,7 +14,6 @@ See the License for the specific language governing permissions and limitations under the License. */ -#include #include "oneflow/core/common/buffer_manager.h" #include "oneflow/core/common/maybe.h" #include "oneflow/core/common/protobuf.h" diff --git a/oneflow/core/framework/multi_client_session_context.h b/oneflow/core/framework/multi_client_session_context.h index b18e1c87c4d..2f2d420e8f3 100644 --- a/oneflow/core/framework/multi_client_session_context.h +++ b/oneflow/core/framework/multi_client_session_context.h @@ -16,7 +16,6 @@ limitations under the License. #ifndef ONEFLOW_CORE_FRAMEWORK_MULTI_CLIENT_SESSION_CONTEXT_H_ #define ONEFLOW_CORE_FRAMEWORK_MULTI_CLIENT_SESSION_CONTEXT_H_ -#include #include "oneflow/core/common/util.h" #include "oneflow/core/job/job_set.pb.h" #include "oneflow/core/common/maybe.h" From 5166fe78a5ee75ff7f4907f89087dc8b88b78d9b Mon Sep 17 00:00:00 2001 From: strint Date: Sun, 10 Apr 2022 15:33:35 +0800 Subject: [PATCH 54/54] rm useless --- oneflow/core/framework/multi_client_session_context.h | 3 --- oneflow/core/job/env_global_objects_scope.h | 1 - 2 files changed, 4 deletions(-) diff --git a/oneflow/core/framework/multi_client_session_context.h b/oneflow/core/framework/multi_client_session_context.h index 2f2d420e8f3..b99656c7169 100644 --- a/oneflow/core/framework/multi_client_session_context.h +++ b/oneflow/core/framework/multi_client_session_context.h @@ -48,9 +48,6 @@ class MultiClientSessionContext { GetFreeEagerTensorNamePairByGraphName(const std::string& graph_name); void RemoveGraphFreeEagerTensors(const std::string& graph_name); - private: - MultiClientSessionContext() {} - private: bool is_inited_ = false; std::shared_ptr env_ctx_; diff --git a/oneflow/core/job/env_global_objects_scope.h b/oneflow/core/job/env_global_objects_scope.h index 9efb4ec5be7..845aff0cb04 100644 --- a/oneflow/core/job/env_global_objects_scope.h +++ b/oneflow/core/job/env_global_objects_scope.h @@ -39,7 +39,6 @@ class EnvGlobalObjectsScope final { } private: - EnvGlobalObjectsScope() {} Maybe Init(const EnvProto& env_proto); private: