persist cluster version for ddl.
This commit is contained in:
		@ -67,6 +67,7 @@ int ObColumnRedefinitionTask::init(const uint64_t tenant_id, const int64_t task_
 | 
			
		||||
    task_id_ = task_id;
 | 
			
		||||
    parallelism_ = parallelism;
 | 
			
		||||
    execution_id_ = 1L;
 | 
			
		||||
    cluster_version_ = GET_MIN_CLUSTER_VERSION();
 | 
			
		||||
    is_inited_ = true;
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
@ -152,6 +153,7 @@ int ObColumnRedefinitionTask::send_build_single_replica_request()
 | 
			
		||||
    param.task_id_ = task_id_;
 | 
			
		||||
    param.parallelism_ = alter_table_arg_.parallelism_;
 | 
			
		||||
    param.execution_id_ = execution_id_;
 | 
			
		||||
    param.cluster_version_ = cluster_version_;
 | 
			
		||||
    if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, object_id_, param.source_tablet_ids_))) {
 | 
			
		||||
      LOG_WARN("fail to get tablets", K(ret), K(tenant_id_), K(object_id_));
 | 
			
		||||
    } else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id_, target_object_id_, param.dest_tablet_ids_))) {
 | 
			
		||||
 | 
			
		||||
@ -48,8 +48,8 @@ ObDDLRedefinitionSSTableBuildTask::ObDDLRedefinitionSSTableBuildTask(
 | 
			
		||||
    const common::ObAddr &inner_sql_exec_addr)
 | 
			
		||||
  : is_inited_(false), tenant_id_(tenant_id), task_id_(task_id), data_table_id_(data_table_id),
 | 
			
		||||
    dest_table_id_(dest_table_id), schema_version_(schema_version), snapshot_version_(snapshot_version),
 | 
			
		||||
    execution_id_(execution_id), sql_mode_(sql_mode), trace_id_(trace_id), parallelism_(parallelism),
 | 
			
		||||
    use_heap_table_ddl_plan_(use_heap_table_ddl_plan), root_service_(root_service),
 | 
			
		||||
    execution_id_(execution_id), sql_mode_(sql_mode), trace_id_(trace_id),
 | 
			
		||||
    parallelism_(parallelism), use_heap_table_ddl_plan_(use_heap_table_ddl_plan), root_service_(root_service),
 | 
			
		||||
    inner_sql_exec_addr_(inner_sql_exec_addr)
 | 
			
		||||
{
 | 
			
		||||
  set_retry_times(0); // do not retry
 | 
			
		||||
@ -1310,7 +1310,7 @@ int ObDDLRedefinitionTask::serialize_params_to_message(char *buf, const int64_t
 | 
			
		||||
  } else if (OB_FAIL(alter_table_arg_.serialize(buf, buf_len, pos))) {
 | 
			
		||||
    LOG_WARN("serialize table arg failed", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_ENCODE, parallelism_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_ENCODE, parallelism_, cluster_version_);
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -1329,7 +1329,7 @@ int ObDDLRedefinitionTask::deserlize_params_from_message(const char *buf, const
 | 
			
		||||
  } else if (OB_FAIL(deep_copy_table_arg(allocator_, tmp_arg, alter_table_arg_))) {
 | 
			
		||||
    LOG_WARN("deep copy table arg failed", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_DECODE, parallelism_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_DECODE, parallelism_, cluster_version_);
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -1337,7 +1337,7 @@ int ObDDLRedefinitionTask::deserlize_params_from_message(const char *buf, const
 | 
			
		||||
int64_t ObDDLRedefinitionTask::get_serialize_param_size() const
 | 
			
		||||
{
 | 
			
		||||
  return alter_table_arg_.get_serialize_size() + serialization::encoded_length_i64(task_version_)
 | 
			
		||||
         + serialization::encoded_length_i64(parallelism_);
 | 
			
		||||
         + serialization::encoded_length_i64(parallelism_) + serialization::encoded_length_i64(cluster_version_);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObDDLRedefinitionTask::check_health()
 | 
			
		||||
 | 
			
		||||
@ -44,6 +44,7 @@ int ObDDLSingleReplicaExecutor::build(const ObDDLSingleReplicaExecutorParam &par
 | 
			
		||||
    task_id_ = param.task_id_;
 | 
			
		||||
    execution_id_ = param.execution_id_;
 | 
			
		||||
    parallelism_ = param.parallelism_;
 | 
			
		||||
    cluster_version_ = param.cluster_version_;
 | 
			
		||||
 | 
			
		||||
    common::ObIArray<ObPartitionBuildInfo> &build_infos = partition_build_stat_;
 | 
			
		||||
    common::ObIArray<ObTabletID> &tablet_ids = source_tablet_ids_;
 | 
			
		||||
@ -117,6 +118,7 @@ int ObDDLSingleReplicaExecutor::schedule_task()
 | 
			
		||||
          arg.task_id_ = task_id_;
 | 
			
		||||
          arg.parallelism_ = parallelism_;
 | 
			
		||||
          arg.execution_id_ = execution_id_;
 | 
			
		||||
          arg.cluster_version_ = cluster_version_;
 | 
			
		||||
          arg.tablet_task_id_ = tablet_task_ids_.at(i);
 | 
			
		||||
          if (OB_FAIL(location_service->get(tenant_id_, arg.source_tablet_id_,
 | 
			
		||||
                  expire_renew_time, is_cache_hit, ls_id))) {
 | 
			
		||||
 | 
			
		||||
@ -36,18 +36,20 @@ public:
 | 
			
		||||
      snapshot_version_(0),
 | 
			
		||||
      task_id_(0),
 | 
			
		||||
      parallelism_(0),
 | 
			
		||||
      execution_id_(-1)
 | 
			
		||||
      execution_id_(-1),
 | 
			
		||||
      cluster_version_(0)
 | 
			
		||||
  {}
 | 
			
		||||
  ~ObDDLSingleReplicaExecutorParam() = default;
 | 
			
		||||
  bool is_valid() const {
 | 
			
		||||
    return common::OB_INVALID_TENANT_ID != tenant_id_ && share::DDL_INVALID != type_
 | 
			
		||||
           && source_tablet_ids_.count() > 0 && dest_tablet_ids_.count() > 0
 | 
			
		||||
           && common::OB_INVALID_ID != source_table_id_ && common::OB_INVALID_ID != dest_table_id_
 | 
			
		||||
           && schema_version_ > 0 && snapshot_version_ > 0 && task_id_ > 0 && execution_id_ >= 0;
 | 
			
		||||
           && schema_version_ > 0 && snapshot_version_ > 0 && task_id_ > 0 && execution_id_ >= 0
 | 
			
		||||
           && cluster_version_ > 0;
 | 
			
		||||
  }
 | 
			
		||||
  TO_STRING_KV(K_(tenant_id), K_(type), K_(source_tablet_ids), K_(dest_tablet_ids),
 | 
			
		||||
               K_(source_table_id), K_(dest_table_id), K_(schema_version),
 | 
			
		||||
               K_(snapshot_version), K_(task_id), K_(parallelism), K_(execution_id));
 | 
			
		||||
               K_(snapshot_version), K_(task_id), K_(parallelism), K_(execution_id), K_(cluster_version));
 | 
			
		||||
public:
 | 
			
		||||
  uint64_t tenant_id_;
 | 
			
		||||
  share::ObDDLType type_;
 | 
			
		||||
@ -60,6 +62,7 @@ public:
 | 
			
		||||
  int64_t task_id_;
 | 
			
		||||
  int64_t parallelism_;
 | 
			
		||||
  int64_t execution_id_;
 | 
			
		||||
  int64_t cluster_version_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
class ObDDLSingleReplicaExecutor
 | 
			
		||||
@ -111,6 +114,7 @@ private:
 | 
			
		||||
  int64_t task_id_;
 | 
			
		||||
  int64_t parallelism_;
 | 
			
		||||
  int64_t execution_id_;
 | 
			
		||||
  int64_t cluster_version_;
 | 
			
		||||
  common::ObArray<ObPartitionBuildInfo> partition_build_stat_;
 | 
			
		||||
  common::ObSpinLock lock_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
@ -274,7 +274,7 @@ public:
 | 
			
		||||
      target_object_id_(0), task_status_(share::ObDDLTaskStatus::PREPARE), snapshot_version_(0), ret_code_(OB_SUCCESS), task_id_(0),
 | 
			
		||||
      parent_task_id_(0), parent_task_key_(), task_version_(0), parallelism_(0),
 | 
			
		||||
      allocator_(lib::ObLabel("DdlTask")), compat_mode_(lib::Worker::CompatMode::INVALID), err_code_occurence_cnt_(0),
 | 
			
		||||
      delay_schedule_time_(0), next_schedule_ts_(0), execution_id_(-1), sql_exec_addr_()
 | 
			
		||||
      delay_schedule_time_(0), next_schedule_ts_(0), execution_id_(-1), sql_exec_addr_(), cluster_version_(0)
 | 
			
		||||
  {}
 | 
			
		||||
  virtual ~ObDDLTask() {}
 | 
			
		||||
  virtual int process() = 0;
 | 
			
		||||
@ -300,6 +300,7 @@ public:
 | 
			
		||||
  int64_t get_task_version() const { return task_version_; }
 | 
			
		||||
  int64_t get_execution_id() const { return execution_id_; }
 | 
			
		||||
  int64_t get_parallelism() const { return parallelism_; }
 | 
			
		||||
  int64_t get_cluster_version() const { return cluster_version_; }
 | 
			
		||||
  static int deep_copy_table_arg(common::ObIAllocator &allocator, const obrpc::ObDDLArg &source_arg, obrpc::ObDDLArg &dest_arg);
 | 
			
		||||
  static int fetch_new_task_id(ObMySQLProxy &sql_proxy, int64_t &new_task_id);
 | 
			
		||||
  virtual int serialize_params_to_message(char *buf, const int64_t buf_size, int64_t &pos) const = 0;
 | 
			
		||||
@ -336,7 +337,8 @@ public:
 | 
			
		||||
      K(target_object_id_), K(task_status_), K(snapshot_version_),
 | 
			
		||||
      K_(ret_code), K_(task_id), K_(parent_task_id), K_(parent_task_key),
 | 
			
		||||
      K_(task_version), K_(parallelism), K_(ddl_stmt_str), K_(compat_mode),
 | 
			
		||||
      K_(sys_task_id), K_(err_code_occurence_cnt), K_(next_schedule_ts), K_(delay_schedule_time), K(execution_id_), K(sql_exec_addr_));
 | 
			
		||||
      K_(sys_task_id), K_(err_code_occurence_cnt), K_(next_schedule_ts), K_(delay_schedule_time),
 | 
			
		||||
      K(execution_id_), K(sql_exec_addr_), K_(cluster_version));
 | 
			
		||||
protected:
 | 
			
		||||
  virtual bool is_error_need_retry(const int ret_code)
 | 
			
		||||
  {
 | 
			
		||||
@ -371,6 +373,7 @@ protected:
 | 
			
		||||
  int64_t next_schedule_ts_;
 | 
			
		||||
  int64_t execution_id_;
 | 
			
		||||
  common::ObAddr sql_exec_addr_;
 | 
			
		||||
  int64_t cluster_version_;
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
enum ColChecksumStat
 | 
			
		||||
 | 
			
		||||
@ -321,6 +321,7 @@ int ObIndexBuildTask::init(
 | 
			
		||||
    task_id_ = task_id;
 | 
			
		||||
    parent_task_id_ = parent_task_id;
 | 
			
		||||
    task_version_ = OB_INDEX_BUILD_TASK_VERSION;
 | 
			
		||||
    cluster_version_ = GET_MIN_CLUSTER_VERSION();
 | 
			
		||||
    if (OB_SUCC(ret)) {
 | 
			
		||||
      task_status_ = static_cast<ObDDLTaskStatus>(task_status);
 | 
			
		||||
      is_inited_ = true;
 | 
			
		||||
@ -1238,7 +1239,7 @@ int ObIndexBuildTask::serialize_params_to_message(char *buf, const int64_t buf_l
 | 
			
		||||
    LOG_WARN("serialize create index arg failed", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_ENCODE, check_unique_snapshot_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_ENCODE, parallelism_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_ENCODE, parallelism_, cluster_version_);
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -1258,7 +1259,7 @@ int ObIndexBuildTask::deserlize_params_from_message(const char *buf, const int64
 | 
			
		||||
    LOG_WARN("deep copy create index arg failed", K(ret));
 | 
			
		||||
  } else {
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_DECODE, check_unique_snapshot_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_DECODE, parallelism_);
 | 
			
		||||
    LST_DO_CODE(OB_UNIS_DECODE, parallelism_, cluster_version_);
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -1266,5 +1267,6 @@ int ObIndexBuildTask::deserlize_params_from_message(const char *buf, const int64
 | 
			
		||||
int64_t ObIndexBuildTask::get_serialize_param_size() const
 | 
			
		||||
{
 | 
			
		||||
  return create_index_arg_.get_serialize_size() + serialization::encoded_length_i64(check_unique_snapshot_)
 | 
			
		||||
         + serialization::encoded_length_i64(task_version_) + serialization::encoded_length_i64(parallelism_);
 | 
			
		||||
         + serialization::encoded_length_i64(task_version_) + serialization::encoded_length_i64(parallelism_)
 | 
			
		||||
         + serialization::encoded_length_i64(cluster_version_);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -38,8 +38,7 @@ public:
 | 
			
		||||
      : task_id_(task_id), tenant_id_(tenant_id), data_table_id_(data_table_id), dest_table_id_(dest_table_id),
 | 
			
		||||
        schema_version_(schema_version), snapshot_version_(snapshot_version), execution_id_(execution_id),
 | 
			
		||||
        trace_id_(trace_id), parallelism_(parallelism), allocator_("IdxSSTBuildTask"),
 | 
			
		||||
        root_service_(root_service),
 | 
			
		||||
        inner_sql_exec_addr_(inner_sql_exec_addr)
 | 
			
		||||
        root_service_(root_service), inner_sql_exec_addr_(inner_sql_exec_addr)
 | 
			
		||||
  {
 | 
			
		||||
    set_retry_times(0);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
@ -70,6 +70,7 @@ int ObTableRedefinitionTask::init(const uint64_t tenant_id, const int64_t task_i
 | 
			
		||||
    task_version_ = OB_TABLE_REDEFINITION_TASK_VERSION;
 | 
			
		||||
    task_id_ = task_id;
 | 
			
		||||
    parallelism_ = parallelism;
 | 
			
		||||
    cluster_version_ = GET_MIN_CLUSTER_VERSION();
 | 
			
		||||
    alter_table_arg_.exec_tenant_id_ = tenant_id_;
 | 
			
		||||
    is_inited_ = true;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
		Reference in New Issue
	
	Block a user