adjust ObDMLStmt::tables_hash_ from ObRowDesc to ObHashMap
This commit is contained in:
		@ -240,18 +240,6 @@ public:
 | 
			
		||||
  int pre_process_json_object_contain_star(ParseNode *node, common::ObIAllocator &allocator);
 | 
			
		||||
  int transfer_to_inner_joined(const ParseNode &parse_node, JoinedTable *&joined_table);
 | 
			
		||||
  virtual int check_special_join_table(const TableItem &join_table, bool is_left_child, ObItemType join_type);
 | 
			
		||||
  virtual int init_stmt()
 | 
			
		||||
  {
 | 
			
		||||
    int ret = common::OB_SUCCESS;
 | 
			
		||||
    ObDMLStmt *stmt = get_stmt();
 | 
			
		||||
    if (OB_ISNULL(stmt)) {
 | 
			
		||||
      ret = common::OB_NOT_INIT;
 | 
			
		||||
      SQL_RESV_LOG(ERROR, "stmt_ is null");
 | 
			
		||||
    } else if (OB_FAIL(stmt->set_table_bit_index(common::OB_INVALID_ID))) {
 | 
			
		||||
      SQL_RESV_LOG(ERROR, "add invalid id to table index desc failed", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
  }
 | 
			
		||||
  /**
 | 
			
		||||
   * 为一个 `JoinedTable` 分配内存
 | 
			
		||||
   * @param joined_table 新的`JoinedTable`
 | 
			
		||||
 | 
			
		||||
@ -450,7 +450,7 @@ int ObDMLStmt::assign(const ObDMLStmt &other)
 | 
			
		||||
    LOG_WARN("failed to copy stmt", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(table_items_.assign(other.table_items_))) {
 | 
			
		||||
    LOG_WARN("assign table items failed", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(tables_hash_.assign(other.tables_hash_))) {
 | 
			
		||||
  } else if (OB_FAIL(assign_tables_hash(other.tables_hash_))) {
 | 
			
		||||
    LOG_WARN("assign table hash desc failed", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(column_items_.assign(other.column_items_))) {
 | 
			
		||||
    LOG_WARN("assign column items failed", K(ret));
 | 
			
		||||
@ -596,7 +596,7 @@ int ObDMLStmt::deep_copy_stmt_struct(ObIAllocator &allocator,
 | 
			
		||||
                                              other.table_items_,
 | 
			
		||||
                                              table_items_))) {
 | 
			
		||||
    LOG_WARN("failed to deep copy table items", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(tables_hash_.assign(other.tables_hash_))) {
 | 
			
		||||
  } else if (OB_FAIL(assign_tables_hash(other.tables_hash_))) {
 | 
			
		||||
    LOG_WARN("assign table hash desc failed", K(ret));
 | 
			
		||||
  } else if (OB_FAIL(deep_copy_join_tables(allocator, expr_copier, other))) {
 | 
			
		||||
    LOG_WARN("failed to copy joined tables", K(ret));
 | 
			
		||||
@ -1089,10 +1089,7 @@ int ObDMLStmt::update_stmt_table_id(const ObDMLStmt &other)
 | 
			
		||||
  }
 | 
			
		||||
  // reset tables hash
 | 
			
		||||
  if (OB_SUCC(ret)) {
 | 
			
		||||
    tables_hash_.reset();
 | 
			
		||||
    if (OB_FAIL(set_table_bit_index(common::OB_INVALID_ID))) {
 | 
			
		||||
      LOG_WARN("failed to set table bit index", K(ret));
 | 
			
		||||
    } else { /*do nothing*/ }
 | 
			
		||||
    tables_hash_.reuse();
 | 
			
		||||
  }
 | 
			
		||||
  // reset table id from column items
 | 
			
		||||
  if (OB_SUCC(ret) && &other != this) {
 | 
			
		||||
@ -2314,7 +2311,7 @@ int ObDMLStmt::add_table_item(const ObSQLSessionInfo *session_info, TableItem *t
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  LOG_DEBUG("finish to add table item", K(*table_item), K(tables_hash_), KPC(table_item->ref_query_),
 | 
			
		||||
  LOG_DEBUG("finish to add table item", K(*table_item), KPC(table_item->ref_query_),
 | 
			
		||||
                                        K(common::lbt()));
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
@ -2352,7 +2349,7 @@ int ObDMLStmt::add_table_item(const ObSQLSessionInfo *session_info, TableItem *t
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  LOG_DEBUG("finish to add table item", K(*table_item), K(tables_hash_), K(common::lbt()));
 | 
			
		||||
  LOG_DEBUG("finish to add table item", K(*table_item), K(common::lbt()));
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -2684,13 +2681,45 @@ int ObDMLStmt::get_ddl_view_output(const TableItem &table,
 | 
			
		||||
 | 
			
		||||
int32_t ObDMLStmt::get_table_bit_index(uint64_t table_id) const
 | 
			
		||||
{
 | 
			
		||||
  int64_t idx = tables_hash_.get_idx(table_id, OB_INVALID_ID);
 | 
			
		||||
  int64_t idx = OB_INVALID_INDEX;
 | 
			
		||||
  if (OB_UNLIKELY(OB_SUCCESS != tables_hash_.get_refactored(table_id, idx))) {
 | 
			
		||||
    idx = OB_INVALID_INDEX;
 | 
			
		||||
  }
 | 
			
		||||
  return static_cast<int32_t>(idx);
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObDMLStmt::set_table_bit_index(uint64_t table_id)
 | 
			
		||||
{
 | 
			
		||||
  return tables_hash_.add_column_desc(table_id, OB_INVALID_ID);
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  int64_t idx = tables_hash_.size() + 1;  // bit index is start from 1
 | 
			
		||||
  if (OB_FAIL(tables_hash_.set_refactored(table_id, idx))) {
 | 
			
		||||
    LOG_WARN("failed to set refactored", K(ret));
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObDMLStmt::assign_tables_hash(const ObDMLStmtTableHash &tables_hash)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  tables_hash_.reuse();
 | 
			
		||||
  ObDMLStmtTableHash::const_iterator it = tables_hash.begin();
 | 
			
		||||
  ObDMLStmtTableHash::const_iterator it_end = tables_hash.end();
 | 
			
		||||
  for (; OB_SUCC(ret) && it != it_end; ++it) {
 | 
			
		||||
    if (OB_FAIL(tables_hash_.set_refactored(it->first, it->second))) {
 | 
			
		||||
      LOG_WARN("failed to set refactored", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObDMLStmt::init_stmt(TableHashAllocator &table_hash_alloc, ObWrapperAllocator &wrapper_alloc)
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  if (!tables_hash_.created() &&
 | 
			
		||||
      OB_FAIL(tables_hash_.create(64, &table_hash_alloc, &wrapper_alloc))) {
 | 
			
		||||
    LOG_WARN("failed to create qb name map", K(ret));
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
int ObDMLStmt::relids_to_table_ids(const ObSqlBitSet<> &table_set,
 | 
			
		||||
@ -3422,22 +3451,7 @@ int ObDMLStmt::rebuild_tables_hash()
 | 
			
		||||
{
 | 
			
		||||
  int ret = OB_SUCCESS;
 | 
			
		||||
  TableItem *ti = NULL;
 | 
			
		||||
  ObSEArray<uint64_t, 4> table_id_list;
 | 
			
		||||
  ObSEArray<int64_t, 4> bit_index_map;
 | 
			
		||||
  // dump old table id - rel id map
 | 
			
		||||
  for (int64_t i = 0; OB_SUCC(ret) && i < tables_hash_.get_column_num(); ++i) {
 | 
			
		||||
    uint64_t tid = OB_INVALID_ID;
 | 
			
		||||
    uint64_t cid = OB_INVALID_ID;
 | 
			
		||||
    if (OB_FAIL(tables_hash_.get_tid_cid(i, tid, cid))) {
 | 
			
		||||
      LOG_WARN("failed to get tid cid", K(ret));
 | 
			
		||||
    } else if (OB_FAIL(table_id_list.push_back(tid))) {
 | 
			
		||||
      LOG_WARN("failed to push back table id", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  tables_hash_.reset();
 | 
			
		||||
  if (OB_FAIL(set_table_bit_index(OB_INVALID_ID))) {
 | 
			
		||||
    LOG_WARN("fail to add table_id to hash table", K(ret));
 | 
			
		||||
  }
 | 
			
		||||
  tables_hash_.reuse();
 | 
			
		||||
  for (int64_t i = 0; OB_SUCC(ret) && i < table_items_.count(); i++) {
 | 
			
		||||
    if (OB_ISNULL(ti = table_items_.at(i))) {
 | 
			
		||||
      ret = OB_INVALID_ARGUMENT;
 | 
			
		||||
@ -3446,12 +3460,6 @@ int ObDMLStmt::rebuild_tables_hash()
 | 
			
		||||
      LOG_WARN("fail to add table_id to hash table", K(ret), K(ti), K(table_items_));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  // create old rel id - new rel id map
 | 
			
		||||
  for (int64_t i = 0; OB_SUCC(ret) && i < table_id_list.count(); ++i) {
 | 
			
		||||
    if (OB_FAIL(bit_index_map.push_back(get_table_bit_index(table_id_list.at(i))))) {
 | 
			
		||||
      LOG_WARN("failed to push back new bit index", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -643,8 +643,12 @@ public:
 | 
			
		||||
    ObSEArray<int64_t, 4,common::ModulePageAllocator, true> check_flags_;
 | 
			
		||||
  };
 | 
			
		||||
 | 
			
		||||
  typedef common::ObSEArray<uint64_t, 8, common::ModulePageAllocator, true> ObViewTableIds;
 | 
			
		||||
 | 
			
		||||
  typedef common::hash::ObHashMap<uint64_t, int64_t, common::hash::NoPthreadDefendMode,
 | 
			
		||||
                                  common::hash::hash_func<uint64_t>,
 | 
			
		||||
                                  common::hash::equal_to<uint64_t>,
 | 
			
		||||
                                  TableHashAllocator,
 | 
			
		||||
                                  common::hash::NormalPointer,
 | 
			
		||||
                                  common::ObWrapperAllocator> ObDMLStmtTableHash;
 | 
			
		||||
public:
 | 
			
		||||
 | 
			
		||||
  explicit ObDMLStmt(stmt::StmtType type);
 | 
			
		||||
@ -672,6 +676,8 @@ public:
 | 
			
		||||
    return OB_SUCCESS;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  virtual int init_stmt(TableHashAllocator &table_hash_alloc, ObWrapperAllocator &wrapper_alloc) override;
 | 
			
		||||
 | 
			
		||||
  bool is_hierarchical_query() const;
 | 
			
		||||
 | 
			
		||||
  int replace_relation_exprs(const common::ObIArray<ObRawExpr *> &other_exprs,
 | 
			
		||||
@ -903,6 +909,7 @@ public:
 | 
			
		||||
                             bool is_anonymous = false);
 | 
			
		||||
  int32_t get_table_bit_index(uint64_t table_id) const;
 | 
			
		||||
  int set_table_bit_index(uint64_t table_id);
 | 
			
		||||
  int assign_tables_hash(const ObDMLStmtTableHash &tables_hash);
 | 
			
		||||
  ColumnItem *get_column_item(uint64_t table_id, const common::ObString &col_name);
 | 
			
		||||
  ColumnItem *get_column_item(uint64_t table_id, uint64_t column_id);
 | 
			
		||||
  int add_column_item(ColumnItem &column_item);
 | 
			
		||||
@ -920,7 +927,6 @@ public:
 | 
			
		||||
  { return pseudo_column_like_exprs_; }
 | 
			
		||||
  const common::ObIArray<ObRawExpr *> &get_pseudo_column_like_exprs() const
 | 
			
		||||
  { return pseudo_column_like_exprs_; }
 | 
			
		||||
  common::ObRowDesc &get_table_hash() { return tables_hash_; }
 | 
			
		||||
  int rebuild_tables_hash();
 | 
			
		||||
  int update_rel_ids(ObRelIds &rel_ids, const ObIArray<int64_t> &bit_index_map);
 | 
			
		||||
  int update_column_item_rel_id();
 | 
			
		||||
@ -1204,9 +1210,7 @@ protected:
 | 
			
		||||
  common::ObSEArray<ObRawExpr *, 16, common::ModulePageAllocator, true> condition_exprs_;
 | 
			
		||||
  // 存放共享的类伪列表达式, 我们认为除了一般的伪列表达式ObPseudoColumnRawExpr, rownum和sequence也属于伪列
 | 
			
		||||
  common::ObSEArray<ObRawExpr *, 8, common::ModulePageAllocator, true> pseudo_column_like_exprs_;
 | 
			
		||||
  // it is only used to record the table_id--bit_index map
 | 
			
		||||
  // although it is a little weird, but it is high-performance than ObHashMap
 | 
			
		||||
  common::ObRowDesc tables_hash_;
 | 
			
		||||
  ObDMLStmtTableHash tables_hash_;
 | 
			
		||||
  common::ObSEArray<ObQueryRefRawExpr*, 4, common::ModulePageAllocator, true> subquery_exprs_;
 | 
			
		||||
  const TransposeItem *transpose_item_;
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
@ -279,6 +279,8 @@ int ObStmtFactory::create_stmt<ObSelectStmt>(ObSelectStmt *&stmt)
 | 
			
		||||
      SQL_RESV_LOG(WARN, "store stmt failed", K(ret));
 | 
			
		||||
      stmt->~ObSelectStmt();
 | 
			
		||||
      stmt = NULL;
 | 
			
		||||
    } else if (OB_FAIL(stmt->init_stmt(table_hash_allocator_, wrapper_allocator_))) {
 | 
			
		||||
      LOG_WARN("failed to init tables hash", K(ret));
 | 
			
		||||
    }
 | 
			
		||||
  }
 | 
			
		||||
  return ret;
 | 
			
		||||
 | 
			
		||||
@ -22,6 +22,7 @@
 | 
			
		||||
#include "sql/resolver/expr/ob_raw_expr.h"
 | 
			
		||||
#include "sql/resolver/ob_stmt_type.h"
 | 
			
		||||
#include "share/schema/ob_dependency_info.h"      // ObReferenceObjTable
 | 
			
		||||
#include "lib/allocator/ob_pooled_allocator.h"
 | 
			
		||||
namespace oceanbase
 | 
			
		||||
{
 | 
			
		||||
namespace sql
 | 
			
		||||
@ -68,6 +69,9 @@ struct expr_equal_to
 | 
			
		||||
  }
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
typedef common::ObPooledAllocator<common::hash::HashMapTypes<uint64_t, int64_t>::AllocType,
 | 
			
		||||
                                    common::ObWrapperAllocator> TableHashAllocator;
 | 
			
		||||
 | 
			
		||||
/// the base class of all statements
 | 
			
		||||
class ObStmt
 | 
			
		||||
{
 | 
			
		||||
@ -533,6 +537,7 @@ public:
 | 
			
		||||
                          common::ObIAllocator &allocator);
 | 
			
		||||
  const share::schema::ObReferenceObjTable *get_ref_obj_table() const;
 | 
			
		||||
  share::schema::ObReferenceObjTable *get_ref_obj_table();
 | 
			
		||||
  virtual int init_stmt(TableHashAllocator &table_hash_alloc, ObWrapperAllocator &wrapper_alloc) { return common::OB_SUCCESS; }
 | 
			
		||||
protected:
 | 
			
		||||
  void print_indentation(FILE *fp, int32_t level) const;
 | 
			
		||||
 | 
			
		||||
@ -591,6 +596,8 @@ class ObStmtFactory
 | 
			
		||||
public:
 | 
			
		||||
  explicit ObStmtFactory(common::ObIAllocator &alloc)
 | 
			
		||||
    : allocator_(alloc),
 | 
			
		||||
      wrapper_allocator_(&alloc),
 | 
			
		||||
      table_hash_allocator_(OB_MALLOC_NORMAL_BLOCK_SIZE, wrapper_allocator_),
 | 
			
		||||
      stmt_store_(alloc),
 | 
			
		||||
      free_list_(alloc),
 | 
			
		||||
      query_ctx_(NULL)
 | 
			
		||||
@ -614,6 +621,8 @@ public:
 | 
			
		||||
        SQL_RESV_LOG(WARN, "store stmt failed", K(ret));
 | 
			
		||||
        stmt->~StmtType();
 | 
			
		||||
        stmt = NULL;
 | 
			
		||||
      } else if (OB_FAIL(stmt->init_stmt(table_hash_allocator_, wrapper_allocator_))) {
 | 
			
		||||
        SQL_RESV_LOG(WARN, "failed to init tables hash", K(ret));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    return ret;
 | 
			
		||||
@ -631,6 +640,8 @@ public:
 | 
			
		||||
  inline common::ObIAllocator &get_allocator() { return allocator_; }
 | 
			
		||||
private:
 | 
			
		||||
  common::ObIAllocator &allocator_;
 | 
			
		||||
  common::ObWrapperAllocator wrapper_allocator_;
 | 
			
		||||
  TableHashAllocator table_hash_allocator_;
 | 
			
		||||
  common::ObObjStore<ObStmt*, common::ObIAllocator&, true> stmt_store_;
 | 
			
		||||
  common::ObObjStore<ObSelectStmt*, common::ObIAllocator&, true> free_list_;
 | 
			
		||||
  ObQueryCtx *query_ctx_;
 | 
			
		||||
 | 
			
		||||
@ -130,9 +130,6 @@ public:
 | 
			
		||||
      stmt_->get_query_ctx()->set_sql_stmt_coll_type(get_obj_print_params(params_.session_info_).cs_type_);
 | 
			
		||||
      if (OB_FAIL(stmt_->set_stmt_id())) {
 | 
			
		||||
        SQL_RESV_LOG(WARN, "fail to set stmt id", K(ret));
 | 
			
		||||
      } else if (OB_FAIL(init_stmt())) {
 | 
			
		||||
        stmt = NULL;
 | 
			
		||||
        SQL_RESV_LOG(ERROR, "init stmt failed", K(ret));
 | 
			
		||||
      }
 | 
			
		||||
    }
 | 
			
		||||
    return stmt;
 | 
			
		||||
@ -155,7 +152,6 @@ public:
 | 
			
		||||
 | 
			
		||||
protected:
 | 
			
		||||
  int normalize_table_or_database_names(common::ObString &name);
 | 
			
		||||
  virtual int init_stmt() { return common::OB_SUCCESS; }
 | 
			
		||||
 | 
			
		||||
private:
 | 
			
		||||
  // disallow copy
 | 
			
		||||
 | 
			
		||||
		Reference in New Issue
	
	Block a user