Fix sort impl mem leak
This commit is contained in:
		
				
					committed by
					
						
						wangzelin.wzl
					
				
			
			
				
	
			
			
			
						parent
						
							35313f0152
						
					
				
				
					commit
					d0cb764457
				
			@ -1,168 +0,0 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright (c) 2021 OceanBase
 | 
			
		||||
 * OceanBase CE is licensed under Mulan PubL v2.
 | 
			
		||||
 * You can use this software according to the terms and conditions of the Mulan PubL v2.
 | 
			
		||||
 * You may obtain a copy of Mulan PubL v2 at:
 | 
			
		||||
 *          http://license.coscl.org.cn/MulanPubL-2.0
 | 
			
		||||
 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
 | 
			
		||||
 * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
 | 
			
		||||
 * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
 | 
			
		||||
 * See the Mulan PubL v2 for more details.
 | 
			
		||||
 */
 | 
			
		||||
 | 
			
		||||
#define USING_LOG_PREFIX SQL_ENG
 | 
			
		||||
#include "sql/engine/px/datahub/components/ob_dh_barrier.h"
 | 
			
		||||
#include "sql/engine/px/datahub/ob_dh_msg_ctx.h"
 | 
			
		||||
#include "sql/engine/px/ob_dfo.h"
 | 
			
		||||
#include "sql/engine/px/ob_px_util.h"
 | 
			
		||||
#include "sql/engine/px/datahub/ob_dh_msg.h"
 | 
			
		||||
#include "sql/engine/px/datahub/components/ob_dh_rollup_key.h"
 | 
			
		||||
 | 
			
		||||
using namespace oceanbase::sql;
 | 
			
		||||
using namespace oceanbase::common;
 | 
			
		||||
 | 
			
		||||
OB_SERIALIZE_MEMBER(ObRollupNDVInfo, ndv_, n_keys_, dop_, max_keys_);
 | 
			
		||||
OB_SERIALIZE_MEMBER((ObRollupKeyPieceMsg, ObDatahubPieceMsg), rollup_ndv_);
 | 
			
		||||
OB_SERIALIZE_MEMBER((ObRollupKeyWholeMsg, ObDatahubWholeMsg), rollup_ndv_);
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyPieceMsgListener::on_message(
 | 
			
		||||
    ObRollupKeyPieceMsgCtx &ctx,
 | 
			
		||||
    common::ObIArray<ObPxSqcMeta *> &sqcs,
 | 
			
		||||
    const ObRollupKeyPieceMsg &pkt)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (pkt.op_id_ != ctx.op_id_) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("unexpected piece msg", K(pkt), K(ctx));
 | 
			
		||||
  } else if (ctx.received_ >= ctx.task_cnt_) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("should not receive any more pkt. already get all pkt expected",
 | 
			
		||||
             K(pkt), K(ctx));
 | 
			
		||||
  } else if (OB_FAIL(ctx.received_msgs_.push_back(pkt))) {
 | 
			
		||||
    LOG_WARN("failed to push back pkt", K(pkt), K(ret));
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_SUCC(ret)) {
 | 
			
		||||
     ctx.received_++;
 | 
			
		||||
    LOG_TRACE("got a win buf picece msg", "all_got", ctx.received_, "expected", ctx.task_cnt_);
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_SUCC(ret) && ctx.received_ == ctx.task_cnt_) {
 | 
			
		||||
    // all piece msg has been received
 | 
			
		||||
    ctx.whole_msg_.op_id_ = ctx.op_id_;
 | 
			
		||||
    if (OB_FAIL(ctx.process_ndv())) {
 | 
			
		||||
      LOG_WARN("failed to process ndv", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
    ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) {
 | 
			
		||||
      dtl::ObDtlChannel *ch = sqcs.at(idx)->get_qc_channel();
 | 
			
		||||
      if (OB_ISNULL(ch)) {
 | 
			
		||||
        ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
        LOG_WARN("null expected", K(ret));
 | 
			
		||||
      } else if (OB_FAIL(ch->send(ctx.whole_msg_, ctx.timeout_ts_))) {
 | 
			
		||||
        LOG_WARN("fail push data to channel", K(ret));
 | 
			
		||||
      } else if (OB_FAIL(ch->flush(true, false))) {
 | 
			
		||||
        LOG_WARN("fail flush dtl data", K(ret));
 | 
			
		||||
      } else {
 | 
			
		||||
        LOG_DEBUG("dispatched winbuf whole msg",
 | 
			
		||||
                  K(idx), K(cnt), K(ctx.whole_msg_), K(*ch));
 | 
			
		||||
      }
 | 
			
		||||
      if (OB_SUCC(ret) && OB_FAIL(ObPxChannelUtil::sqcs_channles_asyn_wait(sqcs))) {
 | 
			
		||||
        LOG_WARN("failed to wait response", K(ret));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    ctx.destroy();
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// find keys that ndv >> dop
 | 
			
		||||
int ObRollupKeyPieceMsgCtx::process_ndv()
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  // analyze all rollup keys and get optimal keys that make the data evenly distributed
 | 
			
		||||
  int64_t dop = 0;
 | 
			
		||||
  ObRollupNDVInfo optimal_rollup_ndv;
 | 
			
		||||
  ObRollupNDVInfo max_rollup_ndv;
 | 
			
		||||
  optimal_rollup_ndv.n_keys_ = INT64_MAX;
 | 
			
		||||
  max_rollup_ndv.n_keys_ = 0;
 | 
			
		||||
  for (int64_t i = 0; OB_SUCC(ret) && i < received_msgs_.count(); ++i) {
 | 
			
		||||
    ObRollupNDVInfo &rollup_ndv = received_msgs_.at(i).rollup_ndv_;
 | 
			
		||||
    if (0 == dop) {
 | 
			
		||||
      dop = rollup_ndv.dop_;
 | 
			
		||||
    } else if (dop != rollup_ndv.dop_) {
 | 
			
		||||
      LOG_WARN("unexpected status: dop is not match", K(dop), K(rollup_ndv.dop_));
 | 
			
		||||
    }
 | 
			
		||||
    if (rollup_ndv.ndv_ >= rollup_ndv.dop_ * FAR_GREATER_THAN_RATIO &&
 | 
			
		||||
        optimal_rollup_ndv.n_keys_ > rollup_ndv.n_keys_) {
 | 
			
		||||
      optimal_rollup_ndv.n_keys_ = rollup_ndv.n_keys_;
 | 
			
		||||
      optimal_rollup_ndv.ndv_ = rollup_ndv.ndv_;
 | 
			
		||||
      optimal_rollup_ndv.dop_ = rollup_ndv.dop_;
 | 
			
		||||
      optimal_rollup_ndv.max_keys_ = rollup_ndv.max_keys_;
 | 
			
		||||
    }
 | 
			
		||||
    // set max
 | 
			
		||||
    if (max_rollup_ndv.n_keys_ < rollup_ndv.n_keys_) {
 | 
			
		||||
      max_rollup_ndv.n_keys_ = rollup_ndv.n_keys_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.ndv_ < rollup_ndv.ndv_) {
 | 
			
		||||
      max_rollup_ndv.ndv_ = rollup_ndv.ndv_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.dop_ < rollup_ndv.dop_) {
 | 
			
		||||
      max_rollup_ndv.dop_ = rollup_ndv.dop_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.max_keys_ < rollup_ndv.max_keys_) {
 | 
			
		||||
      max_rollup_ndv.max_keys_ = rollup_ndv.max_keys_;
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  if (INT64_MAX == optimal_rollup_ndv.n_keys_) {
 | 
			
		||||
    // can't found ndv that ndv >> dop
 | 
			
		||||
    optimal_rollup_ndv = max_rollup_ndv;
 | 
			
		||||
  }
 | 
			
		||||
  if (0 == optimal_rollup_ndv.n_keys_) {
 | 
			
		||||
    // it may has no data
 | 
			
		||||
    optimal_rollup_ndv = max_rollup_ndv;
 | 
			
		||||
  }
 | 
			
		||||
  whole_msg_.rollup_ndv_ = optimal_rollup_ndv;
 | 
			
		||||
  if (OB_SUCC(ret)) {
 | 
			
		||||
    // set partial rollup keys
 | 
			
		||||
    ret = E(EventTable::EN_ROLLUP_ADAPTIVE_KEY_NUM) ret;
 | 
			
		||||
    if (OB_FAIL(ret)) {
 | 
			
		||||
      whole_msg_.rollup_ndv_.n_keys_ = (-ret);
 | 
			
		||||
    }
 | 
			
		||||
    ret = OB_SUCCESS;
 | 
			
		||||
  }
 | 
			
		||||
  // FIXME: now use max_keys
 | 
			
		||||
  // three stage only use max_keys
 | 
			
		||||
  if (0 < max_rollup_ndv.max_keys_) {
 | 
			
		||||
    whole_msg_.rollup_ndv_.n_keys_ = max_rollup_ndv.max_keys_;
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyPieceMsgCtx::alloc_piece_msg_ctx(const ObRollupKeyPieceMsg &pkt,
 | 
			
		||||
                                                ObPxCoordInfo &,
 | 
			
		||||
                                                ObExecContext &ctx,
 | 
			
		||||
                                                int64_t task_cnt,
 | 
			
		||||
                                                ObPieceMsgCtx *&msg_ctx)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (OB_ISNULL(ctx.get_my_session()) ||
 | 
			
		||||
      OB_ISNULL(ctx.get_physical_plan_ctx())) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("session is null or physical plan ctx is null", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    void *buf = ctx.get_allocator().alloc(sizeof(ObRollupKeyPieceMsgCtx));
 | 
			
		||||
    if (OB_ISNULL(buf)) {
 | 
			
		||||
      ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
    } else {
 | 
			
		||||
      msg_ctx = new (buf) ObRollupKeyPieceMsgCtx(pkt.op_id_, task_cnt,
 | 
			
		||||
          ctx.get_physical_plan_ctx()->get_timeout_timestamp(),
 | 
			
		||||
          ctx.get_my_session()->get_effective_tenant_id());
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyWholeMsg::assign(const ObRollupKeyWholeMsg &other)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  rollup_ndv_ = other.rollup_ndv_;
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										167
									
								
								src/sql/engine/px/datahub/components/ob_dh_rollup_key.cpp
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										167
									
								
								src/sql/engine/px/datahub/components/ob_dh_rollup_key.cpp
									
									
									
									
									
										Normal file
									
								
							@ -0,0 +1,167 @@
 | 
			
		||||
/**
 | 
			
		||||
 * Copyright (c) 2021 OceanBase
 | 
			
		||||
 * OceanBase CE is licensed under Mulan PubL v2.
 | 
			
		||||
 * You can use this software according to the terms and conditions of the Mulan PubL v2.
 | 
			
		||||
 * You may obtain a copy of Mulan PubL v2 at:
 | 
			
		||||
 *          http://license.coscl.org.cn/MulanPubL-2.0
 | 
			
		||||
 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
 | 
			
		||||
 * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
 | 
			
		||||
 * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
 | 
			
		||||
 * See the Mulan PubL v2 for more details.
 | 
			
		||||
 */
 | 
			
		||||
 | 
			
		||||
#define USING_LOG_PREFIX SQL_ENG
 | 
			
		||||
#include "sql/engine/px/datahub/components/ob_dh_barrier.h"
 | 
			
		||||
#include "sql/engine/px/datahub/ob_dh_msg_ctx.h"
 | 
			
		||||
#include "sql/engine/px/ob_dfo.h"
 | 
			
		||||
#include "sql/engine/px/ob_px_util.h"
 | 
			
		||||
#include "sql/engine/px/datahub/ob_dh_msg.h"
 | 
			
		||||
#include "sql/engine/px/datahub/components/ob_dh_rollup_key.h"
 | 
			
		||||
 | 
			
		||||
using namespace oceanbase::sql;
 | 
			
		||||
using namespace oceanbase::common;
 | 
			
		||||
 | 
			
		||||
OB_SERIALIZE_MEMBER(ObRollupNDVInfo, ndv_, n_keys_, dop_, max_keys_);
 | 
			
		||||
OB_SERIALIZE_MEMBER((ObRollupKeyPieceMsg, ObDatahubPieceMsg), rollup_ndv_);
 | 
			
		||||
OB_SERIALIZE_MEMBER((ObRollupKeyWholeMsg, ObDatahubWholeMsg), rollup_ndv_);
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyPieceMsgListener::on_message(
 | 
			
		||||
    ObRollupKeyPieceMsgCtx &ctx,
 | 
			
		||||
    common::ObIArray<ObPxSqcMeta *> &sqcs,
 | 
			
		||||
    const ObRollupKeyPieceMsg &pkt)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (pkt.op_id_ != ctx.op_id_) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("unexpected piece msg", K(pkt), K(ctx));
 | 
			
		||||
  } else if (ctx.received_ >= ctx.task_cnt_) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("should not receive any more pkt. already get all pkt expected",
 | 
			
		||||
             K(pkt), K(ctx));
 | 
			
		||||
  } else if (OB_FAIL(ctx.received_msgs_.push_back(pkt))) {
 | 
			
		||||
    LOG_WARN("failed to push back pkt", K(pkt), K(ret));
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_SUCC(ret)) {
 | 
			
		||||
     ctx.received_++;
 | 
			
		||||
    LOG_TRACE("got a win buf picece msg", "all_got", ctx.received_, "expected", ctx.task_cnt_);
 | 
			
		||||
  }
 | 
			
		||||
  if (OB_SUCC(ret) && ctx.received_ == ctx.task_cnt_) {
 | 
			
		||||
    // all piece msg has been received
 | 
			
		||||
    ctx.whole_msg_.op_id_ = ctx.op_id_;
 | 
			
		||||
    if (OB_FAIL(ctx.process_ndv())) {
 | 
			
		||||
      LOG_WARN("failed to process ndv", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
    ARRAY_FOREACH_X(sqcs, idx, cnt, OB_SUCC(ret)) {
 | 
			
		||||
      dtl::ObDtlChannel *ch = sqcs.at(idx)->get_qc_channel();
 | 
			
		||||
      if (OB_ISNULL(ch)) {
 | 
			
		||||
        ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
        LOG_WARN("null expected", K(ret));
 | 
			
		||||
      } else if (OB_FAIL(ch->send(ctx.whole_msg_, ctx.timeout_ts_))) {
 | 
			
		||||
        LOG_WARN("fail push data to channel", K(ret));
 | 
			
		||||
      } else if (OB_FAIL(ch->flush(true, false))) {
 | 
			
		||||
        LOG_WARN("fail flush dtl data", K(ret));
 | 
			
		||||
      } else {
 | 
			
		||||
        LOG_DEBUG("dispatched winbuf whole msg",
 | 
			
		||||
                  K(idx), K(cnt), K(ctx.whole_msg_), K(*ch));
 | 
			
		||||
      }
 | 
			
		||||
      if (OB_SUCC(ret) && OB_FAIL(ObPxChannelUtil::sqcs_channles_asyn_wait(sqcs))) {
 | 
			
		||||
        LOG_WARN("failed to wait response", K(ret));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// find keys that ndv >> dop
 | 
			
		||||
int ObRollupKeyPieceMsgCtx::process_ndv()
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  // analyze all rollup keys and get optimal keys that make the data evenly distributed
 | 
			
		||||
  int64_t dop = 0;
 | 
			
		||||
  ObRollupNDVInfo optimal_rollup_ndv;
 | 
			
		||||
  ObRollupNDVInfo max_rollup_ndv;
 | 
			
		||||
  optimal_rollup_ndv.n_keys_ = INT64_MAX;
 | 
			
		||||
  max_rollup_ndv.n_keys_ = 0;
 | 
			
		||||
  for (int64_t i = 0; OB_SUCC(ret) && i < received_msgs_.count(); ++i) {
 | 
			
		||||
    ObRollupNDVInfo &rollup_ndv = received_msgs_.at(i).rollup_ndv_;
 | 
			
		||||
    if (0 == dop) {
 | 
			
		||||
      dop = rollup_ndv.dop_;
 | 
			
		||||
    } else if (dop != rollup_ndv.dop_) {
 | 
			
		||||
      LOG_WARN("unexpected status: dop is not match", K(dop), K(rollup_ndv.dop_));
 | 
			
		||||
    }
 | 
			
		||||
    if (rollup_ndv.ndv_ >= rollup_ndv.dop_ * FAR_GREATER_THAN_RATIO &&
 | 
			
		||||
        optimal_rollup_ndv.n_keys_ > rollup_ndv.n_keys_) {
 | 
			
		||||
      optimal_rollup_ndv.n_keys_ = rollup_ndv.n_keys_;
 | 
			
		||||
      optimal_rollup_ndv.ndv_ = rollup_ndv.ndv_;
 | 
			
		||||
      optimal_rollup_ndv.dop_ = rollup_ndv.dop_;
 | 
			
		||||
      optimal_rollup_ndv.max_keys_ = rollup_ndv.max_keys_;
 | 
			
		||||
    }
 | 
			
		||||
    // set max
 | 
			
		||||
    if (max_rollup_ndv.n_keys_ < rollup_ndv.n_keys_) {
 | 
			
		||||
      max_rollup_ndv.n_keys_ = rollup_ndv.n_keys_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.ndv_ < rollup_ndv.ndv_) {
 | 
			
		||||
      max_rollup_ndv.ndv_ = rollup_ndv.ndv_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.dop_ < rollup_ndv.dop_) {
 | 
			
		||||
      max_rollup_ndv.dop_ = rollup_ndv.dop_;
 | 
			
		||||
    }
 | 
			
		||||
    if (max_rollup_ndv.max_keys_ < rollup_ndv.max_keys_) {
 | 
			
		||||
      max_rollup_ndv.max_keys_ = rollup_ndv.max_keys_;
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  if (INT64_MAX == optimal_rollup_ndv.n_keys_) {
 | 
			
		||||
    // can't found ndv that ndv >> dop
 | 
			
		||||
    optimal_rollup_ndv = max_rollup_ndv;
 | 
			
		||||
  }
 | 
			
		||||
  if (0 == optimal_rollup_ndv.n_keys_) {
 | 
			
		||||
    // it may has no data
 | 
			
		||||
    optimal_rollup_ndv = max_rollup_ndv;
 | 
			
		||||
  }
 | 
			
		||||
  whole_msg_.rollup_ndv_ = optimal_rollup_ndv;
 | 
			
		||||
  if (OB_SUCC(ret)) {
 | 
			
		||||
    // set partial rollup keys
 | 
			
		||||
    ret = E(EventTable::EN_ROLLUP_ADAPTIVE_KEY_NUM) ret;
 | 
			
		||||
    if (OB_FAIL(ret)) {
 | 
			
		||||
      whole_msg_.rollup_ndv_.n_keys_ = (-ret);
 | 
			
		||||
    }
 | 
			
		||||
    ret = OB_SUCCESS;
 | 
			
		||||
  }
 | 
			
		||||
  // FIXME: now use max_keys
 | 
			
		||||
  // three stage only use max_keys
 | 
			
		||||
  if (0 < max_rollup_ndv.max_keys_) {
 | 
			
		||||
    whole_msg_.rollup_ndv_.n_keys_ = max_rollup_ndv.max_keys_;
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyPieceMsgCtx::alloc_piece_msg_ctx(const ObRollupKeyPieceMsg &pkt,
 | 
			
		||||
                                                ObPxCoordInfo &,
 | 
			
		||||
                                                ObExecContext &ctx,
 | 
			
		||||
                                                int64_t task_cnt,
 | 
			
		||||
                                                ObPieceMsgCtx *&msg_ctx)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (OB_ISNULL(ctx.get_my_session()) ||
 | 
			
		||||
      OB_ISNULL(ctx.get_physical_plan_ctx())) {
 | 
			
		||||
    ret = OB_ERR_UNEXPECTED;
 | 
			
		||||
    LOG_WARN("session is null or physical plan ctx is null", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    void *buf = ctx.get_allocator().alloc(sizeof(ObRollupKeyPieceMsgCtx));
 | 
			
		||||
    if (OB_ISNULL(buf)) {
 | 
			
		||||
      ret = OB_ALLOCATE_MEMORY_FAILED;
 | 
			
		||||
    } else {
 | 
			
		||||
      msg_ctx = new (buf) ObRollupKeyPieceMsgCtx(pkt.op_id_, task_cnt,
 | 
			
		||||
          ctx.get_physical_plan_ctx()->get_timeout_timestamp(),
 | 
			
		||||
          ctx.get_my_session()->get_effective_tenant_id());
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObRollupKeyWholeMsg::assign(const ObRollupKeyWholeMsg &other)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  rollup_ndv_ = other.rollup_ndv_;
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -94,7 +94,7 @@ public:
 | 
			
		||||
    : ObPieceMsgCtx(op_id, task_cnt, timeout_ts), received_(0),
 | 
			
		||||
                    tenant_id_(tenant_id), whole_msg_(), received_msgs_() {}
 | 
			
		||||
  ~ObRollupKeyPieceMsgCtx() = default;
 | 
			
		||||
  void destroy()
 | 
			
		||||
  virtual void destroy()
 | 
			
		||||
  {
 | 
			
		||||
    received_msgs_.reset();
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
@ -595,7 +595,6 @@ int ObDynamicSamplePieceMsgCtx::on_message(
 | 
			
		||||
        LOG_WARN("failed to wait response", K(ret));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    IGNORE_RETURN destroy();
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -111,7 +111,7 @@ public:
 | 
			
		||||
      const SortDef &sort_def);
 | 
			
		||||
  virtual ~ObDynamicSamplePieceMsgCtx() = default;
 | 
			
		||||
  int init(const ObIArray<uint64_t> &tablet_ids);
 | 
			
		||||
  void destroy();
 | 
			
		||||
  virtual void destroy();
 | 
			
		||||
  int process_piece(const ObDynamicSamplePieceMsg &piece);
 | 
			
		||||
  int split_range(
 | 
			
		||||
      const ObChunkDatumStore *sample_store,
 | 
			
		||||
 | 
			
		||||
@ -26,6 +26,7 @@ public:
 | 
			
		||||
  ObPieceMsgCtx(uint64_t op_id, int64_t task_cnt, int64_t timeout_ts)
 | 
			
		||||
      : op_id_(op_id), task_cnt_(task_cnt), timeout_ts_(timeout_ts) {}
 | 
			
		||||
  VIRTUAL_TO_STRING_KV(K_(op_id), K_(task_cnt));
 | 
			
		||||
  virtual void destroy() {}
 | 
			
		||||
  uint64_t op_id_;    // 哪个算子使用 datahub 服务
 | 
			
		||||
  int64_t task_cnt_;  // 这个 dfo 下实际执行的 task 数, 即:期望收到的 piece 数量
 | 
			
		||||
  int64_t timeout_ts_; //超时时间, DTL发送消息时会使用
 | 
			
		||||
@ -38,6 +39,11 @@ public:
 | 
			
		||||
  ~ObPieceMsgCtxMgr() = default;
 | 
			
		||||
  void reset()
 | 
			
		||||
  {
 | 
			
		||||
    for (int i = 0; i < ctxs_.count(); ++i) {
 | 
			
		||||
      if (OB_NOT_NULL(ctxs_[i])) {
 | 
			
		||||
        ctxs_[i]->destroy();
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    ctxs_.reset();
 | 
			
		||||
  }
 | 
			
		||||
  int find_piece_ctx(uint64_t op_id, ObPieceMsgCtx *&ctx)
 | 
			
		||||
 | 
			
		||||
		Reference in New Issue
	
	Block a user