auto_parallel_grad_clip.py 19.1 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16
# Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
#     http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

from functools import reduce

17 18
import numpy as np

19
import paddle
20
from paddle.distributed.fleet.meta_optimizers.common import OP_ROLE_KEY, OpRole
21
from paddle.fluid.executor import _is_enable_standalone_executor
22

23
from ..auto_parallel.dist_attribute import OperatorDistAttr, TensorDistAttr
24 25 26 27 28 29 30 31
from ..auto_parallel.operators.common import (
    SyncMode,
    is_data_parallel_reduce_op,
)
from ..auto_parallel.process_group import (
    get_all_process_groups,
    get_world_process_group,
)
32
from ..auto_parallel.process_mesh import ProcessMesh
33
from ..auto_parallel.reshard import Resharder
34 35
from ..auto_parallel.utils import (
    _get_comm_group,
36
    insert_dependencies_for_vars,
37 38
    is_gradient_clip_op,
    is_optimize_op,
39
)
40
from .auto_parallel_sharding import ShardingPass
41
from .pass_base import PassBase, register_pass
42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71


def _get_params_grads(block):
    params_grads = []
    for op in reversed(block.ops):
        if not is_optimize_op(op):
            break
        if "Param" in op.input_names and "Grad" in op.input_names:
            param_name = op.input("Param")[0]
            grad_name = op.input("Grad")[0]
            param = block.var(param_name)
            grad = block.var(grad_name)
            params_grads.append((param, grad))
    return params_grads


def _get_dpmp_topology(origin_topology, sharding_group):
    """
    Get dpmp topology from origin_topology

    Example:
        the parallel strategy: dp4-mp2-sharding2
        the complete process_mesh:
            topology: [4, 2]
            processes: [0, 1, 2, 3, 4, 5, 6, 7]
        the dpmp topology: [2, 2]
        the sharding axis: 1
    """
    sharding_axis = 1
    dp_sharding_topology = [
72 73
        origin_topology[0] // sharding_group.nranks,
        sharding_group.nranks,
74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128
    ]
    if dp_sharding_topology[0] == 1:
        sharding_axis = 0
        dp_sharding_topology = dp_sharding_topology[1:]

    product_dp_sharding = reduce(lambda x, y: x * y, dp_sharding_topology)
    product_topology = reduce(lambda x, y: x * y, origin_topology)

    if product_topology == product_dp_sharding:
        dpmp_topology = dp_sharding_topology
    else:
        assert product_topology % product_dp_sharding == 0
        mp_degree = product_topology // product_dp_sharding
        dpmp_topology = dp_sharding_topology + [mp_degree]

    return dpmp_topology, sharding_axis


def _get_dpmp_process_mesh(rank_id, topology, processes, sharding_group):
    """
    Get dpmp process_mesh from the complete process_mesh which apply sharding.

    Example:
        the parallel strategy: dp4-mp2-sharding2
        the complete process_mesh:
            topology: [4, 2]
            processes: [0, 1, 2, 3, 4, 5, 6, 7]
        the dpmp process_mesh is:
            1) topology: [2, 2], processes: [0, 1, 4, 5]
            2) topology: [2, 2], processes: [2, 3, 6, 7]
    """
    if sharding_group is None:
        return topology, processes

    # get dpmp_topology
    dpmp_topology, sharding_axis = _get_dpmp_topology(topology, sharding_group)

    # get all sharding_groups of ranks
    sharding_groups = []
    for rank in processes:
        group = _get_comm_group(processes, dpmp_topology, sharding_axis, rank)
        if group not in sharding_groups:
            sharding_groups.append(group)

    # get dpmp_processes
    sharding_groups = np.array(sharding_groups)
    dpmp_processes_in_sharding = None
    for i in range(sharding_groups.shape[-1]):
        if rank_id in sharding_groups[:, i]:
            dpmp_processes_in_sharding = sharding_groups[:, i]

    assert dpmp_processes_in_sharding is not None
    return dpmp_topology, list(dpmp_processes_in_sharding)


129 130 131
def _is_about_global_norm(
    rank_id, tensor_shape, topology, processes, dims_mapping, sharding_group
):
132 133
    # get current process_mesh where the parameter exist.
    dpmp_topology, dpmp_processes = _get_dpmp_process_mesh(
134 135
        rank_id, topology, processes, sharding_group
    )
136

137 138 139
    complete_shape = Resharder.compute_complete_shape(
        tensor_shape, dpmp_topology, dims_mapping
    )
140 141 142 143 144

    complete_partitions = []
    complete_param_ranks = []
    for process in dpmp_processes:
        partition_index = Resharder.compute_partition_index(
145 146
            process, complete_shape, dims_mapping, dpmp_topology, dpmp_processes
        )
147 148 149 150 151 152 153
        if partition_index not in complete_partitions:
            complete_partitions.append(partition_index)
            complete_param_ranks.append(process)

    return rank_id in complete_param_ranks


154
class ClipHelper:
155 156 157
    def __init__(
        self, params_grads, rank_id, block, dist_context, pass_context
    ):
158 159 160 161 162 163
        params, _ = zip(*params_grads)
        self.params = list(params)
        self.params_name = [p.name for p in self.params]
        self.rank_id = rank_id
        self.block = block
        self.dist_context = dist_context
164
        self.pass_context = pass_context
165 166 167 168 169
        self.sharding_group = None
        self.world_ranks = get_world_process_group().ranks
        if hasattr(dist_context, '_sharding_group'):
            self.sharding_group = dist_context._sharding_group

170 171 172
        self.world_nranks = len(self.world_ranks)
        self.pure_data_parallel = self._is_pure_data_parallel()
        self.rank_to_params = self._partition_parameters(params)
173

174 175 176 177 178 179
    def is_calcuate_norm(self, name):
        """
        whether the param_name@GRAD paticipate in the calculation of global_norm
        """
        if not self.is_local_param(name):
            return False
180

181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196
        param = self.params[self.params_name.index(name)]
        if not self.pure_data_parallel:
            dist_attr = self._get_dist_attr(name)
            topology = dist_attr.process_mesh.shape
            processes = dist_attr.process_mesh.process_ids
            dims_mapping = dist_attr.dims_mapping
            return _is_about_global_norm(
                self.rank_id,
                param.shape,
                topology,
                processes,
                dims_mapping,
                self.sharding_group,
            )
        else:
            return param.name in self.rank_to_params[self.rank_id]
197

198 199 200 201
    def is_local_param(self, name):
        """
        whether the param_name is updated with opt in cur_rank
        """
202 203 204 205
        if name not in self.params_name:
            return False
        return True

206 207 208 209 210 211 212 213
    def _get_dist_attr(self, name):
        var = self.block.vars[name]
        return self.dist_context.get_tensor_dist_attr_for_program(var)

    def is_local_var_with_dist_attr(self, name):
        """
        whether the var_name is belong to cur_rank
        """
214 215
        dist_attr = self._get_dist_attr(name)
        assert dist_attr is not None
216
        return self.rank_id in dist_attr.process_mesh.process_ids
217 218

    def _init_dist_attr(self, op):
219 220
        op_dist_attr = OperatorDistAttr()
        op_dist_attr.process_mesh = ProcessMesh(self.world_ranks)
221 222
        for in_name in op.input_arg_names:
            in_var = self.block.vars[in_name]
223 224
            in_dist_attr = TensorDistAttr()
            in_dist_attr.process_mesh = ProcessMesh(self.world_ranks)
225 226
            in_dist_attr.dims_mapping = [-1]
            self.dist_context.set_tensor_dist_attr_for_program(
227 228
                in_var, in_dist_attr
            )
229 230 231
            op_dist_attr.set_input_dist_attr(in_name, in_dist_attr)
        for out_name in op.output_arg_names:
            out_var = self.block.vars[out_name]
232 233
            out_dist_attr = TensorDistAttr()
            out_dist_attr.process_mesh = ProcessMesh(self.world_ranks)
234 235
            out_dist_attr.dims_mapping = [-1]
            self.dist_context.set_tensor_dist_attr_for_program(
236 237
                out_var, out_dist_attr
            )
238 239 240
            op_dist_attr.set_output_dist_attr(out_name, out_dist_attr)
        self.dist_context.set_op_dist_attr_for_program(op, op_dist_attr)

241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284
    def _is_pure_data_parallel(self):
        for applied_pass in self.pass_context.passes:
            if isinstance(applied_pass, ShardingPass):
                return False

        groups = get_all_process_groups()
        for g in groups:
            if g.nranks != self.world_nranks:
                return False

        for op in self.block.ops:
            if op.type in [
                "c_reduce_sum",
                "c_allreduce_sum",
            ] and not is_data_parallel_reduce_op(op):
                return False

        return True

    def _partition_parameters(self, params):
        """
        build rank_id_to_params by the param's numel
        to guarantee params in every rank of dp_group as even as possible.
        """
        mapping = {}
        if not self.pure_data_parallel:
            for rank_ in range(self.world_nranks):
                mapping[rank_] = [p.name for p in params]
        else:
            for rank_ in range(self.world_nranks):
                mapping[rank_] = []
            sizes = [0] * self.world_nranks
            for param in params:
                rank = sizes.index(min(sizes))
                mapping[rank].append(param.name)
                numel = reduce(lambda x, y: x * y, param.shape)
                assert (
                    numel > 0
                ), "param [{}] should larger than 0, but it is [{}]".format(
                    param.name, numel
                )
                sizes[rank] += numel
        return mapping

285 286 287 288 289 290 291 292 293 294

@register_pass("auto_parallel_grad_clip")
class ClipGradByGloblNormPass(PassBase):
    """
    1. Remove norm-compute op and grad-scale op when the grad is not in current rank
       or is independent of the calculation of norm.
    2. Each rank computes its own norm value, then gets global_norm by allreduce_sum only once.
    """

    def __init__(self):
295
        super().__init__()
296 297
        self.set_attr("rank_id", None)
        self.set_attr("dist_context", None)
298
        self.set_attr("params_grads", None)
299 300 301 302 303

    def _check_self(self):
        if self.get_attr("dist_context") is None:
            return False
        dist_context = self.get_attr("dist_context")
Z
zhaoyingli 已提交
304
        if dist_context._serial_optimizer._grad_clip is None:
305
            return False
306 307
        if self.get_attr("params_grads") is None:
            return False
308 309 310 311 312 313 314 315 316
        return True

    def _check_conflict(self, other_pass):
        return True

    def _apply_single_impl(self, main_program, startup_program, context):
        dist_context = self.get_attr("dist_context", None)
        rank_id = self.get_attr("rank_id", None)
        block = main_program.global_block()
317 318
        dist_params_grads = self.get_attr("params_grads", None)
        # dist_params_grads = _get_params_grads(block)
319

320
        self.clip_helper = ClipHelper(
321
            dist_params_grads, rank_id, block, dist_context, context
322
        )
323 324 325 326 327
        self._remove_no_need_ops_vars(block)

    def _remove_no_need_ops_vars(self, block):

        removed_op_out_type = [
328 329 330
            'squared_l2_norm',
            'square',
            'reduce_sum',
331 332 333 334 335 336 337 338
        ]

        removed_op_idx = set()
        removed_tmp_var = set()
        for idx, op in enumerate(block.ops):
            if not is_gradient_clip_op(op):
                continue

339 340
            if op.type == 'clip_by_norm':
                # remove 'clip_by_norm' op if the param is not updated with opt in current rank
341 342
                input_name = op.input("X")[0]
                if input_name.find("@GRAD") != -1:
343
                    param_name = input_name[: input_name.find("@GRAD")]
344 345 346 347 348 349 350 351 352 353 354 355 356 357
                    is_local = self.clip_helper.is_local_param(param_name)
                    if not is_local:
                        removed_op_idx.add(idx)
                        removed_tmp_var.update(set(op.output_arg_names))

            elif op.type in removed_op_out_type:
                input_name = op.input("X")[0]
                if input_name.find("@GRAD") != -1:
                    # remove 'squared_l2_norm' and 'square' ops,
                    # if the param@GRAD in cur_rank does not participate in the calculation of global_norm
                    param_name = input_name[: input_name.find("@GRAD")]
                    is_local = self.clip_helper.is_local_param(param_name)
                    is_calculate = self.clip_helper.is_calcuate_norm(param_name)
                    if not is_local or not is_calculate:
358 359 360
                        removed_op_idx.add(idx)
                        removed_tmp_var.update(set(op.output_arg_names))
                else:
361
                    # 'reduce_sum' must be behind 'square'
362 363 364 365 366
                    if idx - 1 in removed_op_idx:
                        removed_op_idx.add(idx)
                        removed_tmp_var.update(set(op.output_arg_names))

            elif op.type == 'elementwise_mul':
367 368
                # 'elementwise_mul' scale the param@GRAD with global_norm
                # remove 'elementwise_mul' op if the param is not updated with opt in current rank
369 370
                input_name = op.input("X")[0]
                if input_name.find("@GRAD") != -1:
371
                    param_name = input_name[: input_name.find("@GRAD")]
372
                    is_local = self.clip_helper.is_local_param(param_name)
373 374 375 376 377
                    if not is_local:
                        removed_op_idx.add(idx)
                        if block.ops[idx - 1].type == 'cast':
                            removed_op_idx.add(idx - 1)
                            removed_tmp_var.update(
378 379
                                set(block.ops[idx - 1].output_arg_names)
                            )
380 381

            elif op.type == 'sum':
382
                # 'sum' op is used to calculate global_norm, and need to filter inputs which is not in cur_rank
383 384
                reserved_vars = []
                for input_name in op.input_arg_names:
385 386
                    if (
                        input_name not in removed_tmp_var
387 388 389
                        and self.clip_helper.is_local_var_with_dist_attr(
                            input_name
                        )
390
                    ):
391 392 393 394 395 396 397
                        reserved_vars.append(input_name)
                if not reserved_vars:
                    removed_op_idx.add(idx)
                    removed_tmp_var.update(set(op.output_arg_names))
                    if block.ops[idx + 1].type == 'cast':
                        removed_op_idx.add(idx + 1)
                        removed_tmp_var.update(
398 399
                            set(block.ops[idx + 1].output_arg_names)
                        )
400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418
                else:
                    op.desc.set_input("X", reserved_vars)

        for idx, op in reversed(list(enumerate(block.ops))):
            if not is_optimize_op(op):
                break
            if not is_gradient_clip_op(op):
                continue
            if idx in removed_op_idx:
                block._remove_op(idx, sync=False)

        for idx, op in reversed(list(enumerate(block.ops))):
            if not is_optimize_op(op):
                break
            if not is_gradient_clip_op(op):
                continue
            if op.type == 'sqrt':
                input_name = op.input("X")[0]
                input_var = block.vars[input_name]
419
                insert_leaf_fill_constant_node = False
420 421 422 423 424 425 426 427 428 429 430 431 432 433
                if paddle.distributed.get_world_size() > 1:
                    offset = 0
                    if input_name in removed_tmp_var:
                        removed_tmp_var.remove(input_name)
                        fill_constant_op = block._insert_op(
                            idx,
                            type='fill_constant',
                            inputs={},
                            outputs={'Out': [input_var]},
                            attrs={
                                'shape': [1],
                                'dtype': input_var.dtype,
                                'value': 0,
                                'force_cpu': False,
434 435 436 437 438 439
                                OP_ROLE_KEY: OpRole.Optimize,
                            },
                        )
                        fill_constant_op._set_attr(
                            'op_namescope', "/gradient_clip_pass"
                        )
440 441
                        offset += 1
                        self.clip_helper._init_dist_attr(fill_constant_op)
442
                        insert_leaf_fill_constant_node = True
443 444 445 446 447 448 449 450 451 452

                    allreduce_op = block._insert_op(
                        idx + offset,
                        type='c_allreduce_sum',
                        inputs={'X': [input_var]},
                        outputs={'Out': [input_var]},
                        attrs={
                            'ring_id': 0,
                            'use_calc_stream': True,
                            OP_ROLE_KEY: OpRole.Optimize,
453 454
                        },
                    )
455
                    # TODO better regular the usage of op namescope
456
                    allreduce_op._set_attr(
457
                        'op_namescope', str('/') + SyncMode.GlobalNormSync
458
                    )
459 460
                    self.clip_helper._init_dist_attr(allreduce_op)

461
                    if (
462
                        _is_enable_standalone_executor()
463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479
                        and insert_leaf_fill_constant_node
                    ):

                        # NOTE add naive deps for global norm sync in graph exe
                        j = idx - 1
                        prior_op = None
                        while j > 0:
                            op_type = block.ops[j].type
                            if op_type in [
                                'update_loss_scaling',
                                'check_finite_and_unscale',
                            ] or op_type.endswith("_grad"):
                                prior_op = block.ops[j]
                                break
                            j -= 1
                        assert (
                            prior_op is not None
480
                        ), "Unexpected: ClipByGlobalNorm could not find priory depend op"
481 482 483
                        prior_var = block.vars[prior_op.output_arg_names[0]]
                        assert (
                            prior_var is not None
484 485
                        ), "Unexpected: ClipByGlobalNorm could not find priory depend var"
                        insert_dependencies_for_vars(
486 487 488 489 490 491 492 493 494 495 496
                            block,
                            idx,
                            prior_var,
                            input_var,
                            self.clip_helper.dist_context,
                            OpRole.Optimize,
                            process_mesh=[
                                -1
                            ],  # hack to avoid initialize the dist attr for coalesc var
                            is_recompute=False,
                            sync=False,
497
                            op_namescope="grad_clip_fill_constant_dep",
498 499
                        )

500 501 502 503
        for varname in removed_tmp_var:
            block._remove_var(varname, sync=False)

        block._sync_with_cpp()