提交 985e449b 编写于 作者: N Nikolai Kochetov

added IDataType::updateAvgValueSizeHints; renamed PODArray::allocated_size to...

added IDataType::updateAvgValueSizeHints; renamed PODArray::allocated_size to PODArray::allocated_bytes; renamed renamed IColumn::allocatedSize to IColumn::allocatedBytes; removed IColumn.h from IDataType.h
上级 7c450aa5
......@@ -6,6 +6,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Common/StringUtils.h>
#include <Poco/String.h>
#include <Common/typeid_cast.h>
namespace DB
......
......@@ -7,6 +7,7 @@
#include <Parsers/ASTFunction.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
......
......@@ -4,6 +4,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
......
......@@ -185,9 +185,9 @@ size_t ColumnAggregateFunction::byteSize() const
/// Like byteSize(), highly overestimates size
size_t ColumnAggregateFunction::allocatedSize() const
size_t ColumnAggregateFunction::allocatedBytes() const
{
size_t res = getData().allocated_size() * sizeof(getData()[0]);
size_t res = getData().allocated_bytes();
for (const auto & arena : arenas)
res += arena.get()->size();
......
......@@ -137,7 +137,7 @@ public:
size_t byteSize() const override;
size_t allocatedSize() const override;
size_t allocatedBytes() const override;
void insertRangeFrom(const IColumn & from, size_t start, size_t length) override;
......
......@@ -298,9 +298,9 @@ size_t ColumnArray::byteSize() const
}
size_t ColumnArray::allocatedSize() const
size_t ColumnArray::allocatedBytes() const
{
return getData().allocatedSize() + getOffsets().allocated_size() * sizeof(getOffsets()[0]);
return getData().allocatedBytes() + getOffsets().allocated_bytes() * sizeof(getOffsets()[0]);
}
......
......@@ -48,7 +48,7 @@ public:
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
void reserve(size_t n) override;
size_t byteSize() const override;
size_t allocatedSize() const override;
size_t allocatedBytes() const override;
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
ColumnPtr convertToFullColumnIfConst() const override;
void getExtremes(Field & min, Field & max) const override;
......
......@@ -175,7 +175,7 @@ public:
}
size_t byteSize() const override { return sizeof(data) + sizeof(s); }
size_t allocatedSize() const override { return byteSize(); }
size_t allocatedBytes() const override { return byteSize(); }
ColumnPtr permute(const Permutation & perm, size_t limit) const override
{
......
......@@ -116,7 +116,7 @@ size_t ColumnConstAggregateFunction::byteSize() const
return sizeof(value) + sizeof(s);
}
size_t ColumnConstAggregateFunction::allocatedSize() const
size_t ColumnConstAggregateFunction::allocatedBytes() const
{
return byteSize();
}
......
......@@ -91,7 +91,7 @@ public:
size_t byteSize() const override;
size_t allocatedSize() const override;
size_t allocatedBytes() const override;
private:
DataTypePtr data_type;
......
......@@ -57,9 +57,9 @@ public:
return chars.size() + sizeof(n);
}
size_t allocatedSize() const override
size_t allocatedBytes() const override
{
return chars.allocated_size() + sizeof(n);
return chars.allocated_bytes() + sizeof(n);
}
Field operator[](size_t index) const override
......
......@@ -315,9 +315,9 @@ size_t ColumnNullable::byteSize() const
return nested_column->byteSize() + getNullMapConcreteColumn().byteSize();
}
size_t ColumnNullable::allocatedSize() const
size_t ColumnNullable::allocatedBytes() const
{
return nested_column->allocatedSize() + getNullMapConcreteColumn().allocatedSize();
return nested_column->allocatedBytes() + getNullMapConcreteColumn().allocatedBytes();
}
......
......@@ -56,7 +56,7 @@ public:
void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override;
void reserve(size_t n) override;
size_t byteSize() const override;
size_t allocatedSize() const override;
size_t allocatedBytes() const override;
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
ColumnPtr convertToFullColumnIfConst() const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
......
......@@ -54,9 +54,9 @@ public:
return chars.size() + offsets.size() * sizeof(offsets[0]);
}
size_t allocatedSize() const override
size_t allocatedBytes() const override
{
return chars.allocated_size() + offsets.allocated_size();
return chars.allocated_bytes() + offsets.allocated_bytes();
}
ColumnPtr cloneResized(size_t to_size) const override;
......
......@@ -251,11 +251,11 @@ size_t ColumnTuple::byteSize() const
return res;
}
size_t ColumnTuple::allocatedSize() const
size_t ColumnTuple::allocatedBytes() const
{
size_t res = 0;
for (const auto & column : columns)
res += column->allocatedSize();
res += column->allocatedBytes();
return res;
}
......
......@@ -57,7 +57,7 @@ public:
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
void reserve(size_t n) override;
size_t byteSize() const override;
size_t allocatedSize() const override;
size_t allocatedBytes() const override;
ColumnPtr convertToFullColumnIfConst() const override;
const Block & getData() const { return data; }
......
......@@ -182,9 +182,9 @@ public:
return data.size() * sizeof(data[0]);
}
size_t allocatedSize() const override
size_t allocatedBytes() const override
{
return data.allocated_size();
return data.allocated_bytes();
}
void insert(const T value)
......
......@@ -249,7 +249,7 @@ public:
/// Size of memory, allocated for column.
/// This is greater or equals to byteSize due to memory reservation in containers.
/// Zero, if could be determined.
virtual size_t allocatedSize() const = 0;
virtual size_t allocatedBytes() const = 0;
virtual ~IColumn() {}
......
......@@ -30,7 +30,7 @@ public:
void insertDefault() override { ++s; }
void popBack(size_t n) override { s -= n; }
size_t byteSize() const override { return 0; }
size_t allocatedSize() const override { return 0; }
size_t allocatedBytes() const override { return 0; }
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { return 0; }
Field operator[](size_t n) const override { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
......
......@@ -77,7 +77,7 @@ private:
if (c_start == nullptr)
return;
TAllocator::free(c_start, allocated_size());
TAllocator::free(c_start, allocated_bytes());
}
void realloc(size_t bytes)
......@@ -90,7 +90,7 @@ private:
ptrdiff_t end_diff = c_end - c_start;
c_start = reinterpret_cast<char *>(TAllocator::realloc(c_start, allocated_size(), bytes));
c_start = reinterpret_cast<char *>(TAllocator::realloc(c_start, allocated_bytes(), bytes));
c_end = c_start + end_diff;
c_end_of_storage = c_start + bytes - pad_right;
......@@ -104,13 +104,13 @@ private:
bool isAllocatedFromStack() const
{
constexpr size_t stack_threshold = TAllocator::getStackThreshold();
return (stack_threshold > 0) && (allocated_size() <= stack_threshold);
return (stack_threshold > 0) && (allocated_bytes() <= stack_threshold);
}
public:
using value_type = T;
size_t allocated_size() const { return c_end_of_storage - c_start + pad_right; }
size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right; }
/// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions.
struct iterator : public boost::iterator_adaptor<iterator, T*>
......@@ -195,7 +195,7 @@ public:
if (size() == 0)
realloc(std::max(INITIAL_SIZE, minimum_memory_for_elements(1)));
else
realloc(allocated_size() * 2);
realloc(allocated_bytes() * 2);
}
void resize(size_t n)
......@@ -305,10 +305,10 @@ public:
auto swap_stack_heap = [](PODArray & arr1, PODArray & arr2)
{
size_t stack_size = arr1.size();
size_t stack_allocated = arr1.allocated_size();
size_t stack_allocated = arr1.allocated_bytes();
size_t heap_size = arr2.size();
size_t heap_allocated = arr2.allocated_size();
size_t heap_allocated = arr2.allocated_bytes();
/// Keep track of the stack content we have to copy.
char * stack_c_start = arr1.c_start;
......@@ -330,7 +330,7 @@ public:
if (src.isAllocatedFromStack())
{
dest.dealloc();
dest.alloc(src.allocated_size());
dest.alloc(src.allocated_bytes());
memcpy(dest.c_start, src.c_start, byte_size(src.size()));
dest.c_end = dest.c_start + (src.c_end - src.c_start);
......@@ -379,10 +379,10 @@ public:
}
size_t lhs_size = size();
size_t lhs_allocated = allocated_size();
size_t lhs_allocated = allocated_bytes();
size_t rhs_size = rhs.size();
size_t rhs_allocated = rhs.allocated_size();
size_t rhs_allocated = rhs.allocated_bytes();
c_end_of_storage = c_start + rhs_allocated - pad_right;
rhs.c_end_of_storage = rhs.c_start + lhs_allocated - pad_right;
......
......@@ -309,11 +309,11 @@ size_t Block::bytes() const
return res;
}
size_t Block::allocatedSize() const
size_t Block::allocatedBytes() const
{
size_t res = 0;
for (const auto & elem : data)
res += elem.column->allocatedSize();
res += elem.column->allocatedBytes();
return res;
}
......
......@@ -88,7 +88,7 @@ public:
size_t bytes() const;
/// Approximate number of allocated bytes in memory - for profiling and limits.
size_t allocatedSize() const;
size_t allocatedBytes() const;
operator bool() const { return !data.empty(); }
bool operator!() const { return data.empty(); }
......
......@@ -182,23 +182,12 @@ void NativeBlockInputStream::updateAvgValueSizeHints(const Block & block)
if (rows < 10)
return;
bool has_prev_hints = !avg_value_size_hints.empty();
avg_value_size_hints.resize(block.columns());
avg_value_size_hints.resize_fill(block.columns(), 0);
for (auto idx : ext::range(0, block.columns()))
{
double current_avg_value_size = block.getByPosition(idx).column->byteSize() / (rows + .0);
auto & avg_value_size_hint = avg_value_size_hints[idx];
if (!has_prev_hints)
avg_value_size_hint = current_avg_value_size;
else
{
/// Heuristic is chosen so that avg_value_size_hint increases rapidly but decreases slowly.
if (current_avg_value_size > avg_value_size_hint)
avg_value_size_hint = std::min(1024., current_avg_value_size); /// avoid overestimation
else if (current_avg_value_size * 2 < avg_value_size_hint)
avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4;
}
IDataType::updateAvgValueSizeHints(*block.getByPosition(idx).column, avg_value_size_hint);
}
}
......
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNested.h>
......
......@@ -127,7 +127,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, Column
{
#if __SSE2__
/// An optimistic branch in which more efficient copying is possible.
if (offset + 16 * UNROLL_TIMES <= data.allocated_size() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
if (offset + 16 * UNROLL_TIMES <= data.allocated_bytes() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
{
const __m128i * sse_src_pos = reinterpret_cast<const __m128i *>(istr.position());
const __m128i * sse_src_end = sse_src_pos + (size + (16 * UNROLL_TIMES - 1)) / 16 / UNROLL_TIMES * UNROLL_TIMES;
......
#include <Columns/IColumn.h>
#include <DataTypes/IDataType.h>
namespace DB
{
void IDataType::updateAvgValueSizeHints(const IColumn & column, double & avg_value_size_hint)
{
/// Update the average value size hint if amount of read rows isn't too small
size_t column_size = column.size();
if (column_size > 10)
{
double current_avg_value_size = static_cast<double>(column.byteSize()) / column_size;
/// Heuristic is chosen so that avg_value_size_hint increases rapidly but decreases slowly.
if (current_avg_value_size > avg_value_size_hint)
avg_value_size_hint = std::min(1024., current_avg_value_size); /// avoid overestimation
else if (current_avg_value_size * 2 < avg_value_size_hint)
avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4;
}
}
}
......@@ -3,7 +3,6 @@
#include <memory>
#include <Core/Field.h>
#include <Columns/IColumn.h>
namespace DB
......@@ -15,6 +14,9 @@ class WriteBuffer;
class IDataType;
struct FormatSettingsJSON;
class IColumn;
using ColumnPtr = std::shared_ptr<IColumn>;
using DataTypePtr = std::shared_ptr<IDataType>;
using DataTypes = std::vector<DataTypePtr>;
......@@ -152,6 +154,9 @@ public:
}
virtual ~IDataType() {}
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
static void updateAvgValueSizeHints(const IColumn & column, double & avg_value_size_hint);
};
......
#include <Dictionaries/DictionaryStructure.h>
#include <Common/StringUtils.h>
#include <Columns/IColumn.h>
#include <unordered_set>
......
......@@ -317,7 +317,7 @@ template <typename T>
void FlatDictionary::addAttributeSize(const Attribute & attribute)
{
const auto & array_ref = std::get<ContainerPtrType<T>>(attribute.arrays);
bytes_allocated += sizeof(PaddedPODArray<T>) + array_ref->allocated_size();
bytes_allocated += sizeof(PaddedPODArray<T>) + array_ref->allocated_bytes();
bucket_count = array_ref->capacity();
}
......
......@@ -8,6 +8,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
......
......@@ -4,6 +4,7 @@
#include <DataTypes/DataTypeTraits.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
......
......@@ -713,7 +713,7 @@ size_t MergeTreeDataPart::getIndexSizeInAllocatedBytes() const
{
size_t res = 0;
for (const ColumnPtr & column : index)
res += column->allocatedSize();
res += column->allocatedBytes();
return res;
}
......
......@@ -4,6 +4,7 @@
#include <Core/NamesAndTypes.h>
#include <Storages/MergeTree/ActiveDataPartSet.h>
#include <Poco/RWLock.h>
#include <Columns/IColumn.h>
class SipHash;
......
......@@ -513,18 +513,7 @@ void MergeTreeReader::readData(
stream.seekToMark(from_mark);
type.deserializeBinaryBulk(column, *stream.data_buffer, max_rows_to_read, avg_value_size_hint);
/// Update the average value size hint if amount of read rows isn't too small
size_t column_size = column.size();
if (column_size > 10)
{
double current_avg_value_size = static_cast<double>(column.byteSize()) / column_size;
/// Heuristic is chosen so that avg_value_size_hint increases rapidly but decreases slowly.
if (current_avg_value_size > avg_value_size_hint)
avg_value_size_hint = std::min(1024., current_avg_value_size); /// avoid overestimation
else if (current_avg_value_size * 2 < avg_value_size_hint)
avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4;
}
IDataType::updateAvgValueSizeHints(column, avg_value_size_hint);
}
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册