[refactor](remove unused code) Remove unused mempool declare or function params (#16222)

* Remove unused mempool declare or function params

---------

Co-authored-by: yiguolei <yiguolei@gmail.com>
This commit is contained in:
yiguolei
2023-01-30 13:03:18 +08:00
committed by GitHub
parent f87f80a9ef
commit 4b6a4b3cf7
50 changed files with 32 additions and 517 deletions

View File

@ -45,7 +45,6 @@ class TBrokerRangeDesc;
class TNetworkAddress;
class RuntimeState;
class SlotDescriptor;
class MemPool;
class FileReader;
struct Statistics {

View File

@ -52,7 +52,6 @@ class TBrokerRangeDesc;
class TNetworkAddress;
class RuntimeState;
class SlotDescriptor;
class MemPool;
class FileReader;
class RowGroupReader;

View File

@ -304,7 +304,7 @@ const std::string& ScrollParser::get_scroll_id() {
Status ScrollParser::fill_columns(const TupleDescriptor* tuple_desc,
std::vector<vectorized::MutableColumnPtr>& columns,
MemPool* tuple_pool, bool* line_eof,
bool* line_eof,
const std::map<std::string, std::string>& docvalue_context) {
*line_eof = true;

View File

@ -35,8 +35,8 @@ public:
Status parse(const std::string& scroll_result, bool exactly_once = false);
Status fill_columns(const TupleDescriptor* _tuple_desc,
std::vector<vectorized::MutableColumnPtr>& columns, MemPool* mem_pool,
bool* line_eof, const std::map<std::string, std::string>& docvalue_context);
std::vector<vectorized::MutableColumnPtr>& columns, bool* line_eof,
const std::map<std::string, std::string>& docvalue_context);
const std::string& get_scroll_id();
int get_size() const;

View File

@ -34,8 +34,6 @@
namespace doris {
class MemPool;
struct OlapTableIndexSchema {
int64_t index_id;
std::vector<SlotDescriptor*> slots;

View File

@ -188,35 +188,6 @@ bool TextConverter::write_vec_column(const SlotDescriptor* slot_desc,
return true;
}
void TextConverter::unescape_string(StringRef* value, MemPool* pool) {
char* new_data = reinterpret_cast<char*>(pool->allocate(value->size));
unescape_string(value->data, new_data, &value->size);
value->data = new_data;
}
void TextConverter::unescape_string(const char* src, char* dest, size_t* len) {
char* dest_ptr = dest;
const char* end = src + *len;
bool escape_next_char = false;
while (src < end) {
if (*src == _escape_char) {
escape_next_char = !escape_next_char;
} else {
escape_next_char = false;
}
if (escape_next_char) {
++src;
} else {
*dest_ptr++ = *src++;
}
}
char* dest_start = reinterpret_cast<char*>(dest);
*len = dest_ptr - dest_start;
}
void TextConverter::unescape_string_on_spot(const char* src, size_t* len) {
char* dest_ptr = const_cast<char*>(src);
const char* end = src + *len;

View File

@ -20,7 +20,6 @@
#include "vec/columns/column.h"
namespace doris {
class MemPool;
class SlotDescriptor;
class Status;
struct StringRef;
@ -46,15 +45,7 @@ public:
bool write_vec_column(const SlotDescriptor* slot_desc, vectorized::IColumn* nullable_col_ptr,
const char* data, size_t len, bool copy_string, bool need_escape,
size_t rows);
// Removes escape characters from len characters of the null-terminated string src,
// and copies the unescaped string into dest, changing *len to the unescaped length.
// No null-terminator is added to dest.
void unescape_string(const char* src, char* dest, size_t* len);
void unescape_string_on_spot(const char* src, size_t* len);
// Removes escape characters from 'str', allocating a new string from pool.
// 'str' is updated with the new ptr and length.
void unescape_string(StringRef* str, MemPool* pool);
private:
char _escape_char;

View File

@ -58,7 +58,6 @@ public:
int64_t replica_id() const;
int32_t schema_hash() const;
int16_t shard_id() const;
bool equal(int64_t tablet_id, int32_t schema_hash) const;
const std::string& storage_policy() const { return _tablet_meta->storage_policy(); }
@ -142,10 +141,6 @@ inline int16_t BaseTablet::shard_id() const {
return _tablet_meta->shard_id();
}
inline bool BaseTablet::equal(int64_t id, int32_t hash) const {
return (tablet_id() == id) && (schema_hash() == hash);
}
inline TabletSchemaSPtr BaseTablet::tablet_schema() const {
return _schema;
}

View File

@ -86,21 +86,6 @@ public:
return local;
}
// Test if these two cell is equal with each other
template <typename LhsCellType, typename RhsCellType>
bool equal(const LhsCellType& lhs, const RhsCellType& rhs) const {
bool l_null = lhs.is_null();
bool r_null = rhs.is_null();
if (l_null != r_null) {
return false;
} else if (l_null) {
return true;
} else {
return _type_info->equal(lhs.cell_ptr(), rhs.cell_ptr());
}
}
// Only compare column content, without considering nullptr condition.
// RETURNS:
// 0 means equal,

View File

@ -62,7 +62,6 @@ MemTable::MemTable(TabletSharedPtr tablet, Schema* schema, const TabletSchema* t
_insert_mem_tracker_use_hook = std::make_unique<MemTracker>(
fmt::format("MemTableHookInsert:TabletId={}", std::to_string(tablet_id())));
#endif
_buffer_mem_pool = std::make_unique<MemPool>(_insert_mem_tracker.get());
_table_mem_pool = std::make_unique<MemPool>(_insert_mem_tracker.get());
_vec_row_comparator = std::make_shared<RowInBlockComparator>(_schema);
// TODO: Support ZOrderComparator in the future
@ -134,7 +133,6 @@ MemTable::~MemTable() {
}
std::for_each(_row_in_blocks.begin(), _row_in_blocks.end(), std::default_delete<RowInBlock>());
_insert_mem_tracker->release(_mem_usage);
_buffer_mem_pool->free_all();
_table_mem_pool->free_all();
_flush_mem_tracker->set_consumption(0);
DCHECK_EQ(_insert_mem_tracker->consumption(), 0)

View File

@ -128,9 +128,6 @@ private:
// The memory value automatically tracked by the mem hook is 20% less than the manually recorded
// value in the memtable, because some freed memory is not allocated in the DeltaWriter.
std::unique_ptr<MemTracker> _insert_mem_tracker_use_hook;
// This is a buffer, to hold the memory referenced by the rows that have not
// been inserted into the SkipList
std::unique_ptr<MemPool> _buffer_mem_pool;
// Only the rows will be inserted into SkipList can allocate memory from _table_mem_pool.
// In this way, we can make MemTable::memory_usage() to be more accurate, and eventually
// reduce the number of segment files that are generated by current load

View File

@ -87,7 +87,7 @@ Status Merger::vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
while (!eof && !StorageEngine::instance()->stopped()) {
// Read one block from block reader
RETURN_NOT_OK_LOG(
reader.next_block_with_aggregation(&block, nullptr, nullptr, &eof),
reader.next_block_with_aggregation(&block, &eof),
"failed to read next block when merging rowsets of tablet " + tablet->full_name());
RETURN_NOT_OK_LOG(
dst_rowset_writer->add_block(&block),
@ -214,7 +214,7 @@ Status Merger::vertical_compact_one_group(
while (!eof && !StorageEngine::instance()->stopped()) {
// Read one block from block reader
RETURN_NOT_OK_LOG(
reader.next_block_with_aggregation(&block, nullptr, nullptr, &eof),
reader.next_block_with_aggregation(&block, &eof),
"failed to read next block when merging rowsets of tablet " + tablet->full_name());
RETURN_NOT_OK_LOG(
dst_rowset_writer->add_columns(&block, column_group, is_key, max_rows_per_segment),

View File

@ -102,18 +102,6 @@ struct TabletSize {
size_t tablet_size;
};
enum RangeCondition {
GT = 0, // greater than
GE = 1, // greater or equal
LT = 2, // less than
LE = 3, // less or equal
};
enum DelCondSatisfied {
DEL_SATISFIED = 0, //satisfy delete condition
DEL_NOT_SATISFIED = 1, //not satisfy delete condition
DEL_PARTIAL_SATISFIED = 2, //partially satisfy delete condition
};
// Define all data types supported by Field.
// If new filed_type is defined, not only new TypeInfo may need be defined,
// but also some functions like get_type_info in types.cpp need to be changed.
@ -173,16 +161,6 @@ enum FieldAggregationMethod {
OLAP_FIELD_AGGREGATION_QUANTILE_UNION = 9
};
// Compression algorithm type
enum OLAPCompressionType {
// Compression algorithm used for network transmission, low compression rate, low cpu overhead
OLAP_COMP_TRANSPORT = 1,
// Compression algorithm used for hard disk data, with high compression rate and high CPU overhead
OLAP_COMP_STORAGE = 2,
// The compression algorithm used for storage, the compression rate is low, and the cpu overhead is low
OLAP_COMP_LZ4 = 3,
};
enum PushType {
PUSH_NORMAL = 1, // for broker/hadoop load, not used any more
PUSH_FOR_DELETE = 2, // for delete

View File

@ -320,7 +320,6 @@ Status PushBrokerReader::init(const Schema* schema, const TBrokerScanRange& t_sc
}
_runtime_profile = _runtime_state->runtime_profile();
_runtime_profile->set_name("PushBrokerReader");
_tuple_buffer_pool.reset(new MemPool());
_counter.reset(new ScannerCounter());

View File

@ -95,7 +95,6 @@ private:
bool _eof;
std::unique_ptr<RuntimeState> _runtime_state;
RuntimeProfile* _runtime_profile;
std::unique_ptr<MemPool> _tuple_buffer_pool;
std::unique_ptr<ScannerCounter> _counter;
std::unique_ptr<BaseScanner> _scanner;
// Not used, just for placeholding

View File

@ -144,8 +144,7 @@ public:
// Return OK and set `*eof` to true when no more rows can be read.
// Return others when unexpected error happens.
// TODO: Rethink here we still need mem_pool and agg_pool?
virtual Status next_block_with_aggregation(vectorized::Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) {
virtual Status next_block_with_aggregation(vectorized::Block* block, bool* eof) {
return Status::Error<ErrorCode::READER_INITIALIZE_ERROR>();
}

View File

@ -207,41 +207,6 @@ Status BinaryPrefixPageDecoder::seek_at_or_after_value(const void* value, bool*
}
}
Status BinaryPrefixPageDecoder::_read_next_value_to_output(Slice prev, MemPool* mem_pool,
Slice* output) {
if (_cur_pos >= _num_values) {
return Status::NotFound("no more value to read");
}
uint32_t shared_len;
uint32_t non_shared_len;
auto data_ptr = _decode_value_lengths(_next_ptr, &shared_len, &non_shared_len);
if (data_ptr == nullptr) {
return Status::Corruption("Failed to decode value at position {}", _cur_pos);
}
output->size = shared_len + non_shared_len;
if (output->size > 0) {
output->data = (char*)mem_pool->allocate(output->size);
memcpy(output->data, prev.data, shared_len);
memcpy(output->data + shared_len, data_ptr, non_shared_len);
}
_next_ptr = data_ptr + non_shared_len;
return Status::OK();
}
Status BinaryPrefixPageDecoder::_copy_current_to_output(MemPool* mem_pool, Slice* output) {
output->size = _current_value.size();
if (output->size > 0) {
output->data = (char*)mem_pool->allocate(output->size);
if (output->data == nullptr) {
return Status::MemoryAllocFailed("failed to allocate {} bytes", output->size);
}
memcpy(output->data, _current_value.data(), output->size);
}
return Status::OK();
}
Status BinaryPrefixPageDecoder::next_batch(size_t* n, vectorized::MutableColumnPtr& dst) {
DCHECK(_parsed);
if (PREDICT_FALSE(*n == 0 || _cur_pos >= _num_values)) {

View File

@ -145,12 +145,6 @@ private:
// seek to the first value at the given restart point
Status _seek_to_restart_point(size_t restart_point_index);
// like _read_next_value, but directly copy next value to output, not _current_value
Status _read_next_value_to_output(Slice prev, MemPool* mem_pool, Slice* output);
// copy `_current_value` into `output`.
Status _copy_current_to_output(MemPool* mem_pool, Slice* output);
Slice _data;
bool _parsed = false;
size_t _num_values = 0;

View File

@ -60,7 +60,6 @@ Status BitmapIndexIterator::read_bitmap(rowid_t ordinal, roaring::Roaring* resul
DCHECK(num_to_read == num_read);
*result = roaring::Roaring::read(column->get_data_at(0).data, false);
_pool->clear();
return Status::OK();
}

View File

@ -24,7 +24,6 @@
#include "io/fs/file_reader.h"
#include "olap/rowset/segment_v2/common.h"
#include "olap/rowset/segment_v2/indexed_column_reader.h"
#include "runtime/mem_pool.h"
namespace doris {
@ -70,8 +69,7 @@ public:
: _reader(reader),
_dict_column_iter(reader->_dict_column_reader.get()),
_bitmap_column_iter(reader->_bitmap_column_reader.get()),
_current_rowid(0),
_pool(new MemPool()) {}
_current_rowid(0) {}
bool has_null_bitmap() const { return _reader->_has_null; }
@ -108,7 +106,6 @@ private:
IndexedColumnIterator _dict_column_iter;
IndexedColumnIterator _bitmap_column_iter;
rowid_t _current_rowid;
std::unique_ptr<MemPool> _pool;
};
} // namespace segment_v2

View File

@ -78,7 +78,7 @@ Status EngineChecksumTask::_compute_checksum() {
SipHash block_hash;
uint64_t rows = 0;
while (!eof) {
RETURN_IF_ERROR(reader.next_block_with_aggregation(&block, nullptr, nullptr, &eof));
RETURN_IF_ERROR(reader.next_block_with_aggregation(&block, &eof));
rows += block.rows();
block.update_hash(block_hash);

View File

@ -62,7 +62,6 @@ TypeInfoPtr create_dynamic_type_info_ptr(const TypeInfo* type_info);
class TypeInfo {
public:
virtual ~TypeInfo() = default;
virtual bool equal(const void* left, const void* right) const = 0;
virtual int cmp(const void* left, const void* right) const = 0;
virtual void deep_copy(void* dest, const void* src, MemPool* mem_pool) const = 0;
@ -87,8 +86,6 @@ public:
class ScalarTypeInfo : public TypeInfo {
public:
bool equal(const void* left, const void* right) const override { return _equal(left, right); }
int cmp(const void* left, const void* right) const override { return _cmp(left, right); }
void deep_copy(void* dest, const void* src, MemPool* mem_pool) const override {
@ -116,8 +113,7 @@ public:
template <typename TypeTraitsClass>
ScalarTypeInfo(TypeTraitsClass t)
: _equal(TypeTraitsClass::equal),
_cmp(TypeTraitsClass::cmp),
: _cmp(TypeTraitsClass::cmp),
_deep_copy(TypeTraitsClass::deep_copy),
_direct_copy(TypeTraitsClass::direct_copy),
_direct_copy_may_cut(TypeTraitsClass::direct_copy_may_cut),
@ -129,7 +125,6 @@ public:
_field_type(TypeTraitsClass::type) {}
private:
bool (*_equal)(const void* left, const void* right);
int (*_cmp)(const void* left, const void* right);
void (*_shallow_copy)(void* dest, const void* src);
@ -156,41 +151,6 @@ public:
: _item_type_info(std::move(item_type_info)), _item_size(_item_type_info->size()) {}
~ArrayTypeInfo() override = default;
inline bool equal(const void* left, const void* right) const override {
auto l_value = reinterpret_cast<const CollectionValue*>(left);
auto r_value = reinterpret_cast<const CollectionValue*>(right);
if (l_value->length() != r_value->length()) {
return false;
}
size_t len = l_value->length();
if (!l_value->has_null() && !r_value->has_null()) {
for (size_t i = 0; i < len; ++i) {
if (!_item_type_info->equal((uint8_t*)(l_value->data()) + i * _item_size,
(uint8_t*)(r_value->data()) + i * _item_size)) {
return false;
}
}
} else {
for (size_t i = 0; i < len; ++i) {
if (l_value->is_null_at(i)) {
if (r_value->is_null_at(i)) { // both are null
continue;
} else { // left is null & right is not null
return false;
}
} else if (r_value->is_null_at(i)) { // left is not null & right is null
return false;
}
if (!_item_type_info->equal((uint8_t*)(l_value->data()) + i * _item_size,
(uint8_t*)(r_value->data()) + i * _item_size)) {
return false;
}
}
}
return true;
}
int cmp(const void* left, const void* right) const override {
auto l_value = reinterpret_cast<const CollectionValue*>(left);
auto r_value = reinterpret_cast<const CollectionValue*>(right);
@ -522,10 +482,6 @@ struct BaseFieldtypeTraits : public CppTypeTraits<field_type> {
memcpy(address, &value, sizeof(CppType));
}
static inline bool equal(const void* left, const void* right) {
return get_cpp_type_value(left) == get_cpp_type_value(right);
}
static inline int cmp(const void* left, const void* right) {
CppType left_value = get_cpp_type_value(left);
CppType right_value = get_cpp_type_value(right);
@ -1014,11 +970,6 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME>
template <>
struct FieldTypeTraits<OLAP_FIELD_TYPE_CHAR> : public BaseFieldtypeTraits<OLAP_FIELD_TYPE_CHAR> {
static bool equal(const void* left, const void* right) {
auto l_slice = reinterpret_cast<const Slice*>(left);
auto r_slice = reinterpret_cast<const Slice*>(right);
return *l_slice == *r_slice;
}
static int cmp(const void* left, const void* right) {
auto l_slice = reinterpret_cast<const Slice*>(left);
auto r_slice = reinterpret_cast<const Slice*>(right);

View File

@ -59,143 +59,6 @@ using std::vector;
namespace doris {
using namespace ErrorCode;
Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len,
size_t* written_len, OLAPCompressionType compression_type) {
if (nullptr == src_buf || nullptr == dest_buf || nullptr == written_len) {
LOG(WARNING) << "input param with nullptr pointer. src_buf is nullptr: "
<< (src_buf == nullptr ? "true" : "false") << " src_buf=["
<< (src_buf == nullptr ? "nullptr" : src_buf)
<< "], dest_buf is nullptr: " << (dest_buf == nullptr ? "true" : "false")
<< " dest_buf=[" << (dest_buf == nullptr ? "nullptr" : dest_buf)
<< "], written_len is nullptr: "
<< (written_len == nullptr ? "true" : " false") << " written_len=["
<< (dest_buf == nullptr ? -1 : *dest_buf) << "]";
return Status::Error<INVALID_ARGUMENT>();
}
*written_len = dest_len;
switch (compression_type) {
#ifdef DORIS_WITH_LZO
case OLAP_COMP_TRANSPORT: {
// A small buffer(hundreds of bytes) for LZO1X
unsigned char mem[LZO1X_1_MEM_COMPRESS];
int lzo_res = 0;
if (LZO_E_OK != (lzo_res = lzo1x_1_compress(
reinterpret_cast<const lzo_byte*>(src_buf), src_len,
reinterpret_cast<unsigned char*>(dest_buf), written_len, mem))) {
LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
<< "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
return Status::Error<COMPRESS_ERROR>();
} else if (*written_len > dest_len) {
VLOG_NOTICE << "buffer overflow when compressing. "
<< "dest_len=" << dest_len << ", written_len=" << *written_len;
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
case OLAP_COMP_STORAGE: {
// data for LZO1C_99
unsigned char mem[LZO1C_99_MEM_COMPRESS];
int lzo_res = 0;
if (LZO_E_OK != (lzo_res = lzo1c_99_compress(
reinterpret_cast<const lzo_byte*>(src_buf), src_len,
reinterpret_cast<unsigned char*>(dest_buf), written_len, mem))) {
LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
<< "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
return Status::Error<COMPRESS_ERROR>();
} else if (*written_len > dest_len) {
VLOG_NOTICE << "buffer overflow when compressing. "
<< ", dest_len=" << dest_len << ", written_len=" << *written_len;
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
#endif
case OLAP_COMP_LZ4: {
// int lz4_res = LZ4_compress_limitedOutput(src_buf, dest_buf, src_len, dest_len);
int lz4_res = LZ4_compress_default(src_buf, dest_buf, src_len, dest_len);
*written_len = lz4_res;
if (0 == lz4_res) {
VLOG_TRACE << "compress failed. src_len=" << src_len << ", dest_len=" << dest_len
<< ", written_len=" << *written_len << ", lz4_res=" << lz4_res;
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
default:
LOG(WARNING) << "unknown compression type. [type=" << compression_type << "]";
break;
}
return Status::OK();
}
Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len,
size_t* written_len, OLAPCompressionType compression_type) {
if (nullptr == src_buf || nullptr == dest_buf || nullptr == written_len) {
LOG(WARNING) << "input param with nullptr pointer. [src_buf=" << src_buf
<< " dest_buf=" << dest_buf << " written_len=" << written_len << "]";
return Status::Error<INVALID_ARGUMENT>();
}
*written_len = dest_len;
switch (compression_type) {
#ifdef DORIS_WITH_LZO
case OLAP_COMP_TRANSPORT: {
int lzo_res = lzo1x_decompress_safe(reinterpret_cast<const lzo_byte*>(src_buf), src_len,
reinterpret_cast<unsigned char*>(dest_buf), written_len,
nullptr);
if (LZO_E_OK != lzo_res) {
LOG(WARNING) << "decompress failed. src_len=" << src_len << "; dest_len= " << dest_len
<< "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
return Status::Error<DECOMPRESS_ERROR>();
} else if (*written_len > dest_len) {
LOG(WARNING) << "buffer overflow when decompressing. [dest_len=" << dest_len
<< " written_len=" << *written_len << "]";
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
case OLAP_COMP_STORAGE: {
int lzo_res = lzo1c_decompress_safe(reinterpret_cast<const lzo_byte*>(src_buf), src_len,
reinterpret_cast<unsigned char*>(dest_buf), written_len,
nullptr);
if (LZO_E_OK != lzo_res) {
LOG(WARNING) << "compress failed. src_len=" << src_len << "; dest_len= " << dest_len
<< "; written_len=" << *written_len << "; lzo_res=" << lzo_res;
return Status::Error<DECOMPRESS_ERROR>();
} else if (*written_len > dest_len) {
LOG(WARNING) << "buffer overflow when decompressing. [dest_len=" << dest_len
<< " written_len=" << *written_len << "]";
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
#endif
case OLAP_COMP_LZ4: {
int lz4_res = LZ4_decompress_safe(src_buf, dest_buf, src_len, dest_len);
*written_len = lz4_res;
if (lz4_res < 0) {
LOG(WARNING) << "decompress failed. src_len=" << src_len << "; dest_len= " << dest_len
<< "; written_len=" << *written_len << "; lzo_res=" << lz4_res;
return Status::Error<BUFFER_OVERFLOW>();
}
break;
}
default:
LOG(FATAL) << "unknown compress kind. kind=" << compression_type;
break;
}
return Status::OK();
}
uint32_t olap_adler32(uint32_t adler, const char* buf, size_t len) {
return adler32(adler, reinterpret_cast<const Bytef*>(buf), len);
}

View File

@ -105,19 +105,6 @@ void _destruct_object(const void* obj, void*) {
delete ((const T*)obj);
}
template <typename T>
void _destruct_array(const void* array, void*) {
delete[] ((const T*)array);
}
// 根据压缩类型的不同,执行压缩。dest_buf_len是dest_buf的最大长度,
// 通过指针返回的written_len是实际写入的长度。
Status olap_compress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len,
size_t* written_len, OLAPCompressionType compression_type);
Status olap_decompress(const char* src_buf, size_t src_len, char* dest_buf, size_t dest_len,
size_t* written_len, OLAPCompressionType compression_type);
// 计算adler32的包装函数
// 第一次使用的时候第一个参数传宏ADLER32_INIT, 之后的调用传上次计算的结果
#define ADLER32_INIT adler32(0L, Z_NULL, 0)

View File

@ -36,7 +36,6 @@ struct ArrayIteratorFunctionsBase;
class ArrayIterator;
class Status;
class ObjectPool;
class MemPool;
struct TypeDescriptor;
template <PrimitiveType type>

View File

@ -125,7 +125,6 @@ Status FoldConstantExecutor::_init(const TQueryGlobals& query_globals) {
_runtime_profile = _runtime_state->runtime_profile();
_runtime_profile->set_name("FoldConstantExpr");
_mem_tracker = std::make_unique<MemTracker>("FoldConstantExpr");
_mem_pool.reset(new MemPool(_mem_tracker.get()));
return Status::OK();
}

View File

@ -51,7 +51,6 @@ private:
std::unique_ptr<RuntimeState> _runtime_state;
std::unique_ptr<MemTracker> _mem_tracker;
RuntimeProfile* _runtime_profile = nullptr;
std::unique_ptr<MemPool> _mem_pool;
ObjectPool _pool;
static TUniqueId _dummy_id;
};

View File

@ -31,7 +31,6 @@
namespace doris {
class MemPool;
class SlotDescriptor;
class Tuple;

View File

@ -147,7 +147,6 @@ RuntimeState::RuntimeState()
}
RuntimeState::~RuntimeState() {
_block_mgr2.reset();
// close error log file
if (_error_log_file != nullptr && _error_log_file->is_open()) {
_error_log_file->close();
@ -219,16 +218,6 @@ Status RuntimeState::init_mem_trackers(const TUniqueId& query_id) {
return Status::OK();
}
bool RuntimeState::error_log_is_empty() {
std::lock_guard<std::mutex> l(_error_log_lock);
return (_error_log.size() > 0);
}
std::string RuntimeState::error_log() {
std::lock_guard<std::mutex> l(_error_log_lock);
return boost::algorithm::join(_error_log, "\n");
}
bool RuntimeState::log_error(const std::string& error) {
std::lock_guard<std::mutex> l(_error_log_lock);

View File

@ -43,7 +43,6 @@ class DataStreamRecvr;
class ResultBufferMgr;
class TmpFileMgr;
class BufferedBlockMgr;
class BufferedBlockMgr2;
class LoadErrorHub;
class RowDescriptor;
class RuntimeFilterMgr;
@ -99,7 +98,6 @@ public:
const std::string& timezone() const { return _timezone; }
const cctz::time_zone& timezone_obj() const { return _timezone_obj; }
const std::string& user() const { return _user; }
const std::vector<std::string>& error_log() const { return _error_log; }
const TUniqueId& query_id() const { return _query_id; }
const TUniqueId& fragment_instance_id() const { return _fragment_instance_id; }
ExecEnv* exec_env() { return _exec_env; }
@ -136,11 +134,6 @@ public:
// on first use.
Status create_codegen();
BufferedBlockMgr2* block_mgr2() {
DCHECK(_block_mgr2.get() != nullptr);
return _block_mgr2.get();
}
Status query_status() {
std::lock_guard<std::mutex> l(_process_status_lock);
return _process_status;
@ -155,12 +148,6 @@ public:
return _error_log.size() < _query_options.max_errors;
}
// Return true if error log is empty.
bool error_log_is_empty();
// Returns the error log lines as a string joined with '\n'.
std::string error_log();
// Append all _error_log[_unreported_error_idx+] to new_errors and set
// _unreported_error_idx to _errors_log.size()
void get_unreported_errors(std::vector<std::string>* new_errors);
@ -365,9 +352,6 @@ public:
std::vector<TErrorTabletInfo>& error_tablet_infos() { return _error_tablet_infos; }
/// Helper to call QueryState::StartSpilling().
Status StartSpilling(MemTracker* mem_tracker);
// get mem limit for load channel
// if load mem limit is not set, or is zero, using query mem limit instead.
int64_t get_load_mem_limit();
@ -412,11 +396,6 @@ public:
}
private:
// Use a custom block manager for the query for testing purposes.
void set_block_mgr2(const std::shared_ptr<BufferedBlockMgr2>& block_mgr) {
_block_mgr2 = block_mgr;
}
Status create_error_log_file();
static const int DEFAULT_BATCH_SIZE = 2048;
@ -483,12 +462,6 @@ private:
// will not necessarily be set in all error cases.
std::mutex _process_status_lock;
Status _process_status;
//std::unique_ptr<MemPool> _udf_pool;
// BufferedBlockMgr object used to allocate and manage blocks of input data in memory
// with a fixed memory budget.
// The block mgr is shared by all fragments for this query.
std::shared_ptr<BufferedBlockMgr2> _block_mgr2;
// This is the node id of the root node for this plan fragment. This is used as the
// hash seed and has two useful properties:

View File

@ -205,14 +205,6 @@ FunctionContext* FunctionContextImpl::clone(MemPool* pool) {
namespace doris_udf {
static const int MAX_WARNINGS = 1000;
FunctionContext* FunctionContext::create_test_context(doris::MemPool* mem_pool = nullptr) {
FunctionContext* context = new FunctionContext();
context->impl()->_debug = true;
context->impl()->_state = nullptr;
context->impl()->_pool = new doris::FreePool(mem_pool);
return context;
}
FunctionContext::FunctionContext() : _impl(new doris::FunctionContextImpl(this)) {}
FunctionContext::~FunctionContext() {

View File

@ -38,7 +38,6 @@ class BitmapValue;
class DecimalV2Value;
class DateTimeValue;
class CollectionValue;
class MemPool;
} // namespace doris
namespace doris_udf {
@ -247,11 +246,6 @@ public:
doris::ColumnPtrWrapper* get_constant_col(int arg_idx) const;
// Create a test FunctionContext object. The caller is responsible for calling delete
// on it. This context has additional debugging validation enabled.
// And the default value of mem_pool is nullprt.
static FunctionContext* create_test_context(doris::MemPool* mem_pool);
~FunctionContext();
private:

View File

@ -1,37 +0,0 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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.
// This file is copied from
// https://github.com/apache/impala/blob/branch-2.9.0/be/src/util/dummy-runtime-profile.h
// and modified by Doris
#pragma once
#include "common/object_pool.h"
#include "util/runtime_profile.h"
namespace doris {
class DummyProfile {
public:
DummyProfile() : _pool(), _profile(new RuntimeProfile("dummy", false)) {}
RuntimeProfile* profile() { return _profile; }
virtual ~DummyProfile() { delete _profile; }
private:
ObjectPool _pool;
RuntimeProfile* const _profile;
};
} // namespace doris

View File

@ -39,7 +39,6 @@
namespace doris {
class MemPool;
class RowDescriptor;
class Status;
class TupleDescriptor;

View File

@ -27,7 +27,6 @@ namespace doris {
class TextConverter;
class TupleDescriptor;
class RuntimeState;
class MemPool;
class Status;
namespace vectorized {

View File

@ -35,7 +35,6 @@ NewEsScanner::NewEsScanner(RuntimeState* state, NewEsScanNode* parent, int64_t l
_batch_eof(false),
_tuple_id(tuple_id),
_tuple_desc(nullptr),
_mem_pool(nullptr),
_es_reader(nullptr),
_es_scroll_parser(nullptr),
_docvalue_context(docvalue_context),
@ -86,7 +85,6 @@ Status NewEsScanner::open(RuntimeState* state) {
RETURN_IF_ERROR(VScanner::open(state));
RETURN_IF_ERROR(_es_reader->open());
_mem_pool.reset(new MemPool());
return Status::OK();
}
@ -175,8 +173,8 @@ Status NewEsScanner::_get_next(std::vector<vectorized::MutableColumnPtr>& column
COUNTER_UPDATE(new_es_scan_node->_rows_read_counter, 1);
SCOPED_TIMER(new_es_scan_node->_materialize_timer);
RETURN_IF_ERROR(_es_scroll_parser->fill_columns(_tuple_desc, columns, _mem_pool.get(),
&_line_eof, _docvalue_context));
RETURN_IF_ERROR(_es_scroll_parser->fill_columns(_tuple_desc, columns, &_line_eof,
_docvalue_context));
if (!_line_eof) {
break;
}

View File

@ -57,8 +57,6 @@ private:
TupleId _tuple_id;
const TupleDescriptor* _tuple_desc;
std::unique_ptr<MemPool> _mem_pool;
std::unique_ptr<ESScanReader> _es_reader;
std::unique_ptr<ScrollParser> _es_scroll_parser;

View File

@ -383,7 +383,7 @@ Status NewOlapScanner::_get_block_impl(RuntimeState* state, Block* block, bool*
// Read one block from block reader
// ATTN: Here we need to let the _get_block_impl method guarantee the semantics of the interface,
// that is, eof can be set to true only when the returned block is empty.
RETURN_IF_ERROR(_tablet_reader->next_block_with_aggregation(block, nullptr, nullptr, eof));
RETURN_IF_ERROR(_tablet_reader->next_block_with_aggregation(block, eof));
if (!_profile_updated) {
_profile_updated = _tablet_reader->update_profile(_profile);
}

View File

@ -50,7 +50,6 @@ VFileScanner::VFileScanner(RuntimeState* state, NewFileScanNode* parent, int64_t
_next_range(0),
_cur_reader(nullptr),
_cur_reader_eof(false),
_mem_pool(std::make_unique<MemPool>()),
_kv_cache(kv_cache),
_strict_mode(false) {
if (scan_range.params.__isset.strict_mode) {

View File

@ -104,9 +104,6 @@ protected:
// row desc for default exprs
std::unique_ptr<RowDescriptor> _default_val_row_desc;
// Mem pool used to allocate _src_tuple and _src_tuple_row
std::unique_ptr<MemPool> _mem_pool;
KVCache<std::string>& _kv_cache;
bool _scanner_eof = false;

View File

@ -28,7 +28,6 @@ namespace doris {
class TextConverter;
class TupleDescriptor;
class RuntimeState;
class MemPool;
class Status;
namespace vectorized {

View File

@ -77,12 +77,6 @@ Status VMysqlScanNode::prepare(RuntimeState* state) {
return Status::InternalError("new a mysql scanner failed.");
}
_tuple_pool.reset(new (std::nothrow) MemPool());
if (_tuple_pool.get() == nullptr) {
return Status::InternalError("new a mem pool failed.");
}
_text_converter.reset(new (std::nothrow) TextConverter('\\'));
if (_text_converter.get() == nullptr) {
@ -227,8 +221,6 @@ Status VMysqlScanNode::close(RuntimeState* state) {
START_AND_SCOPE_SPAN(state->get_tracer(), span, "VMysqlScanNode::close");
SCOPED_TIMER(_runtime_profile->total_time_counter());
_tuple_pool.reset();
return ExecNode::close(state);
}

View File

@ -76,8 +76,6 @@ private:
const TupleDescriptor* _tuple_desc;
// Tuple index in tuple row.
int _slot_num;
// Pool for allocating tuple data, including all varying-length slots.
std::unique_ptr<MemPool> _tuple_pool;
// Jni helper for scanning an HBase table.
std::unique_ptr<MysqlScanner> _mysql_scanner;
// Helper class for converting text to other types;

View File

@ -43,7 +43,6 @@ VSchemaScanNode::VSchemaScanNode(ObjectPool* pool, const TPlanNode& tnode,
_dest_tuple_desc(nullptr),
_tuple_idx(0),
_slot_num(0),
_tuple_pool(nullptr),
_schema_scanner(nullptr) {}
VSchemaScanNode::~VSchemaScanNode() {}
@ -138,13 +137,6 @@ Status VSchemaScanNode::prepare(RuntimeState* state) {
START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSchemaScanNode::prepare");
RETURN_IF_ERROR(ScanNode::prepare(state));
// new one mem pool
_tuple_pool.reset(new (std::nothrow) MemPool());
if (nullptr == _tuple_pool) {
return Status::InternalError("Allocate MemPool failed.");
}
// get dest tuple desc
_dest_tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
@ -284,8 +276,6 @@ Status VSchemaScanNode::close(RuntimeState* state) {
}
START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSchemaScanNode::close");
SCOPED_TIMER(_runtime_profile->total_time_counter());
_tuple_pool.reset();
return ExecNode::close(state);
}

View File

@ -66,8 +66,6 @@ private:
int _tuple_idx;
// slot num need to fill in and return
int _slot_num;
// Pool for allocating tuple data, including all varying-length slots.
std::unique_ptr<MemPool> _tuple_pool;
// Jni helper for scanning an schema table.
std::unique_ptr<SchemaScanner> _schema_scanner;
};

View File

@ -20,7 +20,6 @@
#include "common/status.h"
#include "olap/like_column_predicate.h"
#include "olap/olap_common.h"
#include "runtime/mem_pool.h"
#include "vec/aggregate_functions/aggregate_function_reader.h"
#include "vec/olap/vcollect_iterator.h"
@ -189,8 +188,7 @@ Status BlockReader::init(const ReaderParams& read_params) {
return Status::OK();
}
Status BlockReader::_direct_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) {
Status BlockReader::_direct_next_block(Block* block, bool* eof) {
auto res = _vcollect_iter.next(block);
if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
return res;
@ -207,13 +205,11 @@ Status BlockReader::_direct_next_block(Block* block, MemPool* mem_pool, ObjectPo
return Status::OK();
}
Status BlockReader::_direct_agg_key_next_block(Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) {
Status BlockReader::_direct_agg_key_next_block(Block* block, bool* eof) {
return Status::OK();
}
Status BlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) {
Status BlockReader::_agg_key_next_block(Block* block, bool* eof) {
if (UNLIKELY(_eof)) {
*eof = true;
return Status::OK();
@ -263,8 +259,7 @@ Status BlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool, ObjectP
return Status::OK();
}
Status BlockReader::_unique_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) {
Status BlockReader::_unique_key_next_block(Block* block, bool* eof) {
if (UNLIKELY(_eof)) {
*eof = true;
return Status::OK();

View File

@ -35,9 +35,8 @@ public:
// Initialize BlockReader with tablet, data version and fetch range.
Status init(const ReaderParams& read_params) override;
Status next_block_with_aggregation(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) override {
return (this->*_next_block_func)(block, mem_pool, agg_pool, eof);
Status next_block_with_aggregation(Block* block, bool* eof) override {
return (this->*_next_block_func)(block, eof);
}
std::vector<RowLocation> current_block_row_locations() { return _block_row_locations; }
@ -51,18 +50,17 @@ public:
private:
// Directly read row from rowset and pass to upper caller. No need to do aggregation.
// This is usually used for DUPLICATE KEY tables
Status _direct_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _direct_next_block(Block* block, bool* eof);
// Just same as _direct_next_block, but this is only for AGGREGATE KEY tables.
// And this is an optimization for AGGR tables.
// When there is only one rowset and is not overlapping, we can read it directly without aggregation.
Status _direct_agg_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof);
Status _direct_agg_key_next_block(Block* block, bool* eof);
// For normal AGGREGATE KEY tables, read data by a merge heap.
Status _agg_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _agg_key_next_block(Block* block, bool* eof);
// For UNIQUE KEY tables, read data by a merge heap.
// The difference from _agg_key_next_block is that it will read the data from high version to low version,
// to minimize the comparison time in merge heap.
Status _unique_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _unique_key_next_block(Block* block, bool* eof);
Status _init_collect_iter(const ReaderParams& read_params,
std::vector<RowsetReaderSharedPtr>* valid_rs_readers);
@ -106,8 +104,7 @@ private:
bool _eof = false;
Status (BlockReader::*_next_block_func)(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) = nullptr;
Status (BlockReader::*_next_block_func)(Block* block, bool* eof) = nullptr;
std::vector<RowLocation> _block_row_locations;

View File

@ -20,7 +20,6 @@
#include "common/status.h"
#include "olap/like_column_predicate.h"
#include "olap/olap_common.h"
#include "runtime/mem_pool.h"
#include "vec/aggregate_functions/aggregate_function_reader.h"
#include "vec/olap/block_reader.h"
#include "vec/olap/vcollect_iterator.h"
@ -187,8 +186,7 @@ Status VerticalBlockReader::init(const ReaderParams& read_params) {
return Status::OK();
}
Status VerticalBlockReader::_direct_next_block(Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) {
Status VerticalBlockReader::_direct_next_block(Block* block, bool* eof) {
auto res = _vcollect_iter->next_batch(block);
if (UNLIKELY(!res.ok() && !res.is<END_OF_FILE>())) {
return res;
@ -298,8 +296,7 @@ size_t VerticalBlockReader::_copy_agg_data() {
return copy_size;
}
Status VerticalBlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) {
Status VerticalBlockReader::_agg_key_next_block(Block* block, bool* eof) {
if (_reader_context.is_key_column_group) {
// collect_iter will filter agg keys
auto res = _vcollect_iter->next_batch(block);
@ -352,8 +349,7 @@ Status VerticalBlockReader::_agg_key_next_block(Block* block, MemPool* mem_pool,
return Status::OK();
}
Status VerticalBlockReader::_unique_key_next_block(Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) {
Status VerticalBlockReader::_unique_key_next_block(Block* block, bool* eof) {
if (_reader_context.is_key_column_group) {
// Record row_source_buffer current size for key column agg flag
// _vcollect_iter->next_batch(block) will fill row_source_buffer but delete sign is ignored

View File

@ -41,9 +41,8 @@ public:
// Initialize VerticalBlockReader with tablet, data version and fetch range.
Status init(const ReaderParams& read_params) override;
Status next_block_with_aggregation(Block* block, MemPool* mem_pool, ObjectPool* agg_pool,
bool* eof) override {
return (this->*_next_block_func)(block, mem_pool, agg_pool, eof);
Status next_block_with_aggregation(Block* block, bool* eof) override {
return (this->*_next_block_func)(block, eof);
}
uint64_t merged_rows() const override {
@ -55,13 +54,13 @@ public:
private:
// Directly read row from rowset and pass to upper caller. No need to do aggregation.
// This is usually used for DUPLICATE KEY tables
Status _direct_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _direct_next_block(Block* block, bool* eof);
// For normal AGGREGATE KEY tables, read data by a merge heap.
Status _agg_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _agg_key_next_block(Block* block, bool* eof);
// For UNIQUE KEY tables, read data by a merge heap.
// The difference from _agg_key_next_block is that it will read the data from high version to low version,
// to minimize the comparison time in merge heap.
Status _unique_key_next_block(Block* block, MemPool* mem_pool, ObjectPool* agg_pool, bool* eof);
Status _unique_key_next_block(Block* block, bool* eof);
Status _init_collect_iter(const ReaderParams& read_params);
@ -82,8 +81,7 @@ private:
bool _eof = false;
Status (VerticalBlockReader::*_next_block_func)(Block* block, MemPool* mem_pool,
ObjectPool* agg_pool, bool* eof) = nullptr;
Status (VerticalBlockReader::*_next_block_func)(Block* block, bool* eof) = nullptr;
RowSourcesBuffer* _row_sources_buffer;
ColumnPtr _delete_filter_column;

View File

@ -40,13 +40,11 @@ void common_test(typename TypeTraits<field_type>::CppType src_val) {
typename TypeTraits<field_type>::CppType dst_val;
MemPool pool;
type->deep_copy((char*)&dst_val, (char*)&src_val, &pool);
EXPECT_TRUE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, type->cmp((char*)&src_val, (char*)&dst_val));
}
{
typename TypeTraits<field_type>::CppType dst_val;
type->direct_copy((char*)&dst_val, (char*)&src_val);
EXPECT_TRUE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, type->cmp((char*)&src_val, (char*)&dst_val));
}
// test min
@ -54,7 +52,6 @@ void common_test(typename TypeTraits<field_type>::CppType src_val) {
typename TypeTraits<field_type>::CppType dst_val;
type->set_to_min((char*)&dst_val);
EXPECT_FALSE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_TRUE(type->cmp((char*)&src_val, (char*)&dst_val) > 0);
}
// test max
@ -79,14 +76,12 @@ void test_char(Slice src_val) {
Slice dst_val(buf, sizeof(buf));
MemPool pool;
type->deep_copy((char*)&dst_val, (char*)&src_val, &pool);
EXPECT_TRUE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, type->cmp((char*)&src_val, (char*)&dst_val));
}
{
char buf[64];
Slice dst_val(buf, sizeof(buf));
type->direct_copy((char*)&dst_val, (char*)&src_val);
EXPECT_TRUE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, type->cmp((char*)&src_val, (char*)&dst_val));
}
// test min
@ -95,7 +90,6 @@ void test_char(Slice src_val) {
Slice dst_val(buf, sizeof(buf));
field->set_to_min((char*)&dst_val);
EXPECT_FALSE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_TRUE(type->cmp((char*)&src_val, (char*)&dst_val) > 0);
}
// test max
@ -104,7 +98,6 @@ void test_char(Slice src_val) {
Slice dst_val(buf, sizeof(buf));
field->set_to_max((char*)&dst_val);
EXPECT_FALSE(type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_TRUE(type->cmp((char*)&src_val, (char*)&dst_val) < 0);
}
delete field;
@ -162,7 +155,6 @@ void common_test_array(CollectionValue src_val) {
CollectionValue dst_val;
MemPool pool;
array_type->deep_copy((char*)&dst_val, (char*)&src_val, &pool);
EXPECT_TRUE(array_type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, array_type->cmp((char*)&src_val, (char*)&dst_val));
}
{ // test direct copy
@ -170,7 +162,6 @@ void common_test_array(CollectionValue src_val) {
uint8_t data[50];
CollectionValue dst_val(data, sizeof(null_signs), null_signs);
array_type->direct_copy((char*)&dst_val, (char*)&src_val);
EXPECT_TRUE(array_type->equal((char*)&src_val, (char*)&dst_val));
EXPECT_EQ(0, array_type->cmp((char*)&src_val, (char*)&dst_val));
}
}