Skip to content

Commit

Permalink
Support tensor.to()/to_local() (#5271)
Browse files Browse the repository at this point in the history
* support_tensor_to/to_local

* export consistent_tensor.to_local()

* refine code

* export tensor.to()...

* refine code

* refine code

* optimize code

* refine code

* refine

* back up

* add tensor.to func

* make of_format

* remove to in pyTensor

* sync gpu data

* refine

* refine

* refine

* refine

* refine

* refine

* refine

* refine

* refine

* backup

* refine

* rebase

* check in gen py

* merge master and fix bugs

* address pr comments

* address pr comments

* auto format by CI

* remove boxing

* refine

* Fix optional

* remove to in tensor.cpp

* update

* Support symbol placement type in functional.

* add sbp and sbp list arg

* refine

* use functional

* refactor CastConsistentOpExpr

* to_consistent(flow.B) backward

* Cache op expr

* add EagerNcclOpKernelState

* refine

* refine

* refine

* refine

* refine

* refine

* minor fix

* capture OpInterpContext

* unimplemented apply

* add GetNdSbp

* add mutex

* refine

* merge EagerConsistentTensorImpl::NewWithPhyTensor and EagerConsistentTensorImpl::NewWithoutPhyTensor into EagerConsistentTensorImpl::New

* rename functiona SyncData to SyncMetaAndData

* of_format

* add to_local to pybind

* add placement_sbp_util

* minor fix

* sync shape and data when tensor_to_local

* fix rpc_token bugs

* refactor AsyncRpcCtx

* set logical_shape correctly

* simplify implementation of consistent_tensor.to_local

* initialize rpc_token with zero

* refactor grad functions of to_consistent/to_local

* reformat and address pr comment

* reformat

* refactor eager_nccl_reduce lernel

Co-authored-by: tsai <jackalcooper@gmail.com>
Co-authored-by: Xinqi Li <lixinqi0703106@163.com>
Co-authored-by: Li Xinqi <lixinqi2010@gmail.com>
Co-authored-by: oneflow-ci-bot <ci-bot@oneflow.org>
Co-authored-by: hjchen2 <chenhoujiangcug@gmail.com>
Co-authored-by: oneflow-ci-bot <69100618+oneflow-ci-bot@users.noreply.github.com>
  • Loading branch information
7 people committed Aug 3, 2021
1 parent c8b6d39 commit a72c21d
Show file tree
Hide file tree
Showing 45 changed files with 1,573 additions and 237 deletions.
3 changes: 2 additions & 1 deletion oneflow/api/python/framework/device.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ limitations under the License.
#include <pybind11/pybind11.h>
#include <pybind11/operators.h>
#include "oneflow/api/python/common.h"
#include "oneflow/core/control/global_process_ctx.h"
#include "oneflow/api/python/framework/device.h"
#include "oneflow/api/python/of_api_registry.h"
#include "oneflow/core/framework/device.h"
Expand All @@ -28,7 +29,7 @@ namespace oneflow {
std::string type;
int device_id = -1;
ParsingDeviceTag(type_and_id, &type, &device_id).GetOrThrow();
if (device_id == -1) { device_id = 0; }
if (device_id == -1) { device_id = GlobalProcessCtx::LocalRank(); }
return MakeDevice(type, device_id);
}

Expand Down
104 changes: 104 additions & 0 deletions oneflow/core/autograd/gradient_funcs/consistent_cast.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
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/framework/op_expr_grad_function.h"
#include "oneflow/core/framework/op_interpreter/op_interpreter_util.h"
#include "oneflow/core/framework/op_expr.h"
#include "oneflow/core/framework/op_expr_helper.h"
#include "oneflow/core/framework/nd_sbp.h"

namespace oneflow {
namespace one {

struct CastConsistentOpExprInterpState : public OpExprInterpState {
Symbol<ParallelDesc> parallel_desc;
Symbol<cfg::ParallelDistribution> parallel_distribution;
std::shared_ptr<const Shape> shape;
};

class CastToConsistent : public OpExprGradFunction<CastConsistentOpExprInterpState> {
public:
Maybe<void> Init(const OpExpr& op) override {
const auto* fw_op_expr = dynamic_cast<const CastToConsistentOpExpr*>(&op);
CHECK_NOTNULL_OR_RETURN(fw_op_expr);
const std::string& op_name = fw_op_expr->op_name();
grad_op_ = JUST(one::CastFromConsistentOpExpr::New(GradientOpName(op_name)));
return Maybe<void>::Ok();
}

Maybe<void> Capture(CastConsistentOpExprInterpState* ctx, const TensorTuple& inputs,
const TensorTuple& outputs,
const OpExprInterpContext& interp_ctx) const override {
ctx->parallel_desc = JUST(interp_ctx.parallel_desc.value());
ctx->parallel_distribution = JUST(interp_ctx.parallel_distribution.value());
return Maybe<void>::Ok();
}

Maybe<void> Apply(const CastConsistentOpExprInterpState* ctx, const TensorTuple& out_grads,
TensorTuple* in_grads) const override {
const auto& out_grad = out_grads.at(0);
CHECK_OR_RETURN(out_grad->is_consistent());
const auto& bw_parallel_distribution = JUST(out_grad->parallel_distribution());
const auto& dual_parallel_distribution = JUST(GetDualNdSbp(ctx->parallel_distribution));
CHECK_OR_RETURN(bw_parallel_distribution == dual_parallel_distribution);
in_grads->at(0) = JUST(OpInterpUtil::Dispatch<Tensor>(*grad_op_, {out_grads.at(0)}));
return Maybe<void>::Ok();
}

private:
std::shared_ptr<OpExpr> grad_op_;
};

REGISTER_OP_EXPR_GRAD_FUNCTION("cast_to_consistent", CastToConsistent);

class CastFromConsistent : public OpExprGradFunction<CastConsistentOpExprInterpState> {
public:
Maybe<void> Init(const OpExpr& op) override {
const auto* fw_op_expr = dynamic_cast<const CastFromConsistentOpExpr*>(&op);
CHECK_NOTNULL_OR_RETURN(fw_op_expr);
const std::string& op_name = fw_op_expr->op_name();
grad_op_ = JUST(one::CastToConsistentOpExpr::New(GradientOpName(op_name)));
return Maybe<void>::Ok();
}

Maybe<void> Capture(CastConsistentOpExprInterpState* ctx, const TensorTuple& inputs,
const TensorTuple& outputs, const AttrMap& attrs) const override {
const auto& input = inputs.at(0);
CHECK_OR_RETURN(input->is_consistent());
ctx->parallel_desc = JUST(input->parallel_desc());
ctx->parallel_distribution = JUST(input->parallel_distribution());
ctx->shape = input->shape();
return Maybe<void>::Ok();
}

Maybe<void> Apply(const CastConsistentOpExprInterpState* ctx, const TensorTuple& out_grads,
TensorTuple* in_grads) const override {
const auto& dual_parallel_distribution = JUST(GetDualNdSbp(ctx->parallel_distribution));
MutableAttrMap attrs;
JUST(attrs.SetAttr<Shape>("shape", *ctx->shape));
in_grads->at(0) = JUST(OpInterpUtil::Dispatch<Tensor>(
*grad_op_, {out_grads.at(0)},
OpExprInterpContext(attrs, ctx->parallel_desc, dual_parallel_distribution)));
return Maybe<void>::Ok();
}

private:
std::shared_ptr<OpExpr> grad_op_;
};

REGISTER_OP_EXPR_GRAD_FUNCTION("cast_from_consistent", CastFromConsistent);

} // namespace one
} // namespace oneflow
88 changes: 88 additions & 0 deletions oneflow/core/autograd/gradient_funcs/eager_nccl_broadcast.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
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/framework/id_util.h"
#include "oneflow/core/framework/op_builder.h"
#include "oneflow/core/framework/op_expr_grad_function.h"
#include "oneflow/core/framework/device.h"
#include "oneflow/core/framework/op_builder.h"
#include "oneflow/core/framework/op_interpreter/op_interpreter_util.h"
#include "oneflow/core/framework/op_expr.h"
#include "oneflow/core/framework/op_expr_helper.h"

namespace oneflow {

namespace one {

namespace {

Maybe<one::UserOpExpr> EagerNcclReduce(Symbol<ParallelDesc> parallel_desc, int64_t root) {
return one::OpBuilder("eager_nccl_reduce", *JUST(UniqueStr("eager_nccl_reduce")))
.Input("in")
.Output("out")
.Attr<std::string>("parallel_conf", PbMessage2TxtString(parallel_desc->parallel_conf()))
.Attr<int64_t>("root", root)
.Build();
}

Maybe<one::UserOpExpr> FindOrCreatEagerNcclReduceOpExpr(Symbol<ParallelDesc> parallel_desc,
int64_t root) {
thread_local HashMap<std::pair<Symbol<ParallelDesc>, int64_t>, std::shared_ptr<one::UserOpExpr>>
parallel_desc_and_root_device2eager_nccl_reduce;
const auto& key = std::make_pair(parallel_desc, root);
auto iter = parallel_desc_and_root_device2eager_nccl_reduce.find(key);
if (iter == parallel_desc_and_root_device2eager_nccl_reduce.end()) {
std::shared_ptr<UserOpExpr> op_expr = JUST(EagerNcclReduce(parallel_desc, root));
iter = parallel_desc_and_root_device2eager_nccl_reduce.emplace(key, op_expr).first;
}
return iter->second;
}

} // namespace

struct EagerNcclBroadcastOpExprInterpState : public OpExprInterpState {
Symbol<ParallelDesc> parallel_desc;
int64_t root;
};

class EagerNcclBroadcast : public OpExprGradFunction<EagerNcclBroadcastOpExprInterpState> {
public:
Maybe<void> Init(const OpExpr& op) override {
const auto* fw_op_expr = dynamic_cast<const UserOpExpr*>(&op);
CHECK_NOTNULL_OR_RETURN(fw_op_expr);
return Maybe<void>::Ok();
}

Maybe<void> Capture(EagerNcclBroadcastOpExprInterpState* ctx, const TensorTuple& inputs,
const TensorTuple& outputs,
const OpExprInterpContext& interp_ctx) const override {
ctx->root = JUST(interp_ctx.attrs.GetAttr<int64_t>("root"));
ctx->parallel_desc = JUST(interp_ctx.parallel_desc.value());
return Maybe<void>::Ok();
}

Maybe<void> Apply(const EagerNcclBroadcastOpExprInterpState* ctx, const TensorTuple& out_grads,
TensorTuple* in_grads) const override {
const auto& grad_op = JUST(FindOrCreatEagerNcclReduceOpExpr(ctx->parallel_desc, ctx->root));
in_grads->resize(1);
in_grads->at(0) = JUST(OpInterpUtil::Dispatch<Tensor>(*grad_op, {out_grads.at(0)}));
return Maybe<void>::Ok();
}
};

REGISTER_OP_EXPR_GRAD_FUNCTION("eager_nccl_broadcast", EagerNcclBroadcast);

} // namespace one
} // namespace oneflow
27 changes: 18 additions & 9 deletions oneflow/core/common/flat_shape.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -18,19 +18,28 @@ limitations under the License.

namespace oneflow {

Maybe<void> FlatShape::Init(const std::shared_ptr<const Shape>& shape) {
CHECK_LE_OR_RETURN(shape->NumAxes(), SHAPE_MAX_AXIS_SIZE);
this->set_num_axes(shape->NumAxes());
for (int i = 0; i < this->num_axes(); ++i) { *this->mutable_dim()->Mutable(i) = shape->At(i); }
/*static*/ Maybe<FlatShape> FlatShape::New(const Shape& shape) {
const auto& flat_shape = std::make_shared<FlatShape>();
JUST(flat_shape->Init(shape));
return flat_shape;
}

Maybe<void> FlatShape::Init(const Shape& shape) {
CHECK_LE_OR_RETURN(shape.NumAxes(), SHAPE_MAX_AXIS_SIZE);
for (int i = 0; i < shape.NumAxes(); ++i) { *this->mutable_dim()->Add() = shape.At(i); }
return Maybe<void>::Ok();
}

Maybe<void> FlatShape::Check(const std::shared_ptr<const Shape>& shape) const {
CHECK_EQ_OR_RETURN(this->num_axes(), shape->NumAxes());
for (int i = 0; i < this->num_axes(); ++i) {
CHECK_EQ_OR_RETURN(this->dim().Get(i), shape->At(i));
}
Maybe<void> FlatShape::Check(const Shape& shape) const {
CHECK_EQ_OR_RETURN(this->dim_size(), shape.NumAxes());
for (int i = 0; i < this->dim_size(); ++i) { CHECK_EQ_OR_RETURN(this->dim(i), shape.At(i)); }
return Maybe<void>::Ok();
}

Maybe<Shape> FlatShape::ToShape() const {
DimVector dim_vec;
for (int i = 0; i < this->dim_size(); ++i) { dim_vec.push_back(this->dim(i)); }
return std::make_shared<Shape>(dim_vec);
}

} // namespace oneflow
11 changes: 7 additions & 4 deletions oneflow/core/common/flat_shape.h
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,15 @@ class Shape;

FLAT_MSG_BEGIN(FlatShape);
// Methods
OF_PUBLIC Maybe<void> Init(const std::shared_ptr<const Shape>& shape);
OF_PUBLIC Maybe<void> Check(const std::shared_ptr<const Shape>& shape) const;
OF_PUBLIC static Maybe<FlatShape> New(const Shape& shape);
OF_PUBLIC Maybe<void> Init(const Shape& shape);
OF_PUBLIC Maybe<void> Check(const Shape& shape) const;
OF_PUBLIC Maybe<Shape> ToShape() const;
OF_PUBLIC int64_t At(int i) const { return dim(i); }
OF_PUBLIC int64_t NumAxes() const { return dim_size(); }

// Fields
FLAT_MSG_DEFINE_OPTIONAL(int64_t, num_axes);
FLAT_MSG_DEFINE_REPEATED(int64_t, dim, SHAPE_MAX_AXIS_SIZE);
FLAT_MSG_DEFINE_REPEATED(int64_t, dim, SHAPE_MAX_AXIS_SIZE);
FLAT_MSG_END(FlatShape);

// clang-format on
Expand Down
41 changes: 41 additions & 0 deletions oneflow/core/common/optional_test.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
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/optional.h"
#include "oneflow/core/common/util.h"

namespace oneflow {
namespace test {

TEST(Optional, copy_constructor) {
Optional<int64_t> a(0);
std::vector<Optional<int64_t>> vec;
vec.push_back(a);
ASSERT_TRUE(vec[0].has_value());
int64_t val = CHECK_JUST(vec[0].value());
ASSERT_EQ(val, 0);
}

TEST(Optional, move_constructor) {
Optional<int64_t> a(0);
std::map<int64_t, Optional<int64_t>> map;
map.emplace(0, a);
ASSERT_TRUE(map.at(0).has_value());
int64_t val = CHECK_JUST(map.at(0).value());
ASSERT_EQ(val, 0);
}

} // namespace test
} // namespace oneflow
25 changes: 20 additions & 5 deletions oneflow/core/device/device_context.h
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,26 @@ class DeviceCtx {
virtual ~DeviceCtx() = default;

#ifdef WITH_CUDA
virtual const cudaStream_t& cuda_stream() const { UNIMPLEMENTED(); }
virtual const cublasHandle_t& cublas_pmh_handle() const { UNIMPLEMENTED(); }
virtual const cublasHandle_t& cublas_pmd_handle() const { UNIMPLEMENTED(); }
virtual const cublasHandle_t& cublas_tensor_op_math_handle() const { UNIMPLEMENTED(); }
virtual const cudnnHandle_t& cudnn_handle() const { UNIMPLEMENTED(); }
virtual const cudaStream_t& cuda_stream() const {
UNIMPLEMENTED();
return *(const cudaStream_t*)nullptr;
}
virtual const cublasHandle_t& cublas_pmh_handle() const {
UNIMPLEMENTED();
return *(const cublasHandle_t*)nullptr;
}
virtual const cublasHandle_t& cublas_pmd_handle() const {
UNIMPLEMENTED();
return *(const cublasHandle_t*)nullptr;
}
virtual const cublasHandle_t& cublas_tensor_op_math_handle() const {
UNIMPLEMENTED();
return *(const cublasHandle_t*)nullptr;
}
virtual const cudnnHandle_t& cudnn_handle() const {
UNIMPLEMENTED();
return *(const cudnnHandle_t*)nullptr;
}
#endif

virtual void SyncDevice() { UNIMPLEMENTED(); }
Expand Down
2 changes: 2 additions & 0 deletions oneflow/core/eager/foreign_boxing_util.h
Original file line number Diff line number Diff line change
Expand Up @@ -31,12 +31,14 @@ class ForeignBoxingUtil {
InstructionsBuilder* builder, const std::shared_ptr<compatible_py::BlobObject>& blob_object,
const std::shared_ptr<compatible_py::OpArgParallelAttribute>& op_arg_parallel_attr) const {
UNIMPLEMENTED();
return std::shared_ptr<compatible_py::BlobObject>();
}

virtual std::shared_ptr<ParallelDesc> TryReplaceDeviceTag(
InstructionsBuilder* builder, const std::shared_ptr<ParallelDesc>& parallel_desc_symbol,
const std::string& device_tag) const {
UNIMPLEMENTED();
return std::shared_ptr<ParallelDesc>();
}

virtual void Assign(InstructionsBuilder* builder,
Expand Down
2 changes: 1 addition & 1 deletion oneflow/core/framework/device.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ Maybe<void> Device::Init() {
}

/* static */ Maybe<Symbol<Device>> Device::New(const std::string& type) {
return New(type, GlobalProcessCtx::Rank() % GlobalProcessCtx::NumOfProcessPerNode());
return New(type, GlobalProcessCtx::LocalRank());
}

const std::shared_ptr<const ParallelDesc>& Device::parallel_desc_ptr() const {
Expand Down

0 comments on commit a72c21d

Please sign in to comment.