mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
minor fixes
This commit is contained in:
parent
5c7371333e
commit
86d6673d61
@ -11,11 +11,6 @@ class Collator;
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
}
|
|
||||||
|
|
||||||
class ColumnSparse final : public COWHelper<IColumn, ColumnSparse>
|
class ColumnSparse final : public COWHelper<IColumn, ColumnSparse>
|
||||||
{
|
{
|
||||||
private:
|
private:
|
||||||
|
@ -56,7 +56,7 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows)
|
|||||||
}
|
}
|
||||||
|
|
||||||
ssize_t offsets_diff = static_cast<ssize_t>(total_rows) - current_offset;
|
ssize_t offsets_diff = static_cast<ssize_t>(total_rows) - current_offset;
|
||||||
if(offsets_diff > 1)
|
if (offsets_diff > 1)
|
||||||
res->insertManyFrom(*this, 0, offsets_diff - 1);
|
res->insertManyFrom(*this, 0, offsets_diff - 1);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
|
@ -54,6 +54,9 @@ bool checkEquals(const IColumn & lhs, const IColumn & rhs)
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Can't use ErrorCodes, because of 'using namespace DB'.
|
||||||
|
constexpr int error_code = 12345;
|
||||||
|
|
||||||
constexpr size_t T = 5000;
|
constexpr size_t T = 5000;
|
||||||
constexpr size_t MAX_ROWS = 10000;
|
constexpr size_t MAX_ROWS = 10000;
|
||||||
constexpr size_t sparse_ratios[] = {1, 2, 5, 10, 32, 50, 64, 100, 256, 500, 1000, 5000, 10000};
|
constexpr size_t sparse_ratios[] = {1, 2, 5, 10, 32, 50, 64, 100, 256, 500, 1000, 5000, 10000};
|
||||||
@ -79,7 +82,7 @@ TEST(ColumnSparse, InsertRangeFrom)
|
|||||||
DUMP_COLUMN(sparse_dst);
|
DUMP_COLUMN(sparse_dst);
|
||||||
DUMP_COLUMN(full_dst);
|
DUMP_COLUMN(full_dst);
|
||||||
DUMP_NON_DEFAULTS(full_dst);
|
DUMP_NON_DEFAULTS(full_dst);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal");
|
throw Exception(error_code, "Columns are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -102,7 +105,7 @@ TEST(ColumnSparse, InsertRangeFrom)
|
|||||||
test_case(n1, k1, n2, k2, from, to - from);
|
test_case(n1, k1, n2, k2, from, to - from);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
@ -122,7 +125,7 @@ TEST(ColumnSparse, PopBack)
|
|||||||
DUMP_COLUMN(sparse_dst);
|
DUMP_COLUMN(sparse_dst);
|
||||||
DUMP_COLUMN(full_dst);
|
DUMP_COLUMN(full_dst);
|
||||||
DUMP_NON_DEFAULTS(full_dst);
|
DUMP_NON_DEFAULTS(full_dst);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal");
|
throw Exception(error_code, "Columns are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -137,7 +140,7 @@ TEST(ColumnSparse, PopBack)
|
|||||||
test_case(n, k, m);
|
test_case(n, k, m);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
@ -163,7 +166,7 @@ TEST(ColumnSparse, Filter)
|
|||||||
DUMP_COLUMN(sparse_dst);
|
DUMP_COLUMN(sparse_dst);
|
||||||
DUMP_COLUMN(full_dst);
|
DUMP_COLUMN(full_dst);
|
||||||
DUMP_NON_DEFAULTS(full_dst);
|
DUMP_NON_DEFAULTS(full_dst);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal");
|
throw Exception(error_code, "Columns are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -178,7 +181,7 @@ TEST(ColumnSparse, Filter)
|
|||||||
test_case(n, k, m);
|
test_case(n, k, m);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
@ -190,7 +193,7 @@ TEST(ColumnSparse, Permute)
|
|||||||
{
|
{
|
||||||
auto [sparse_src, full_src] = createColumns(n, k);
|
auto [sparse_src, full_src] = createColumns(n, k);
|
||||||
|
|
||||||
PaddedPODArray<UInt64> perm(n);
|
IColumn::Permutation perm;
|
||||||
std::iota(perm.begin(), perm.end(), 0);
|
std::iota(perm.begin(), perm.end(), 0);
|
||||||
std::shuffle(perm.begin(), perm.end(), rng);
|
std::shuffle(perm.begin(), perm.end(), rng);
|
||||||
|
|
||||||
@ -210,7 +213,7 @@ TEST(ColumnSparse, Permute)
|
|||||||
DUMP_COLUMN(sparse_dst);
|
DUMP_COLUMN(sparse_dst);
|
||||||
DUMP_COLUMN(full_dst);
|
DUMP_COLUMN(full_dst);
|
||||||
DUMP_NON_DEFAULTS(full_dst);
|
DUMP_NON_DEFAULTS(full_dst);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal");
|
throw Exception(error_code, "Columns are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -225,7 +228,7 @@ TEST(ColumnSparse, Permute)
|
|||||||
test_case(n, k, limit);
|
test_case(n, k, limit);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
@ -250,7 +253,7 @@ TEST(ColumnSparse, CompareColumn)
|
|||||||
DUMP_COLUMN(full_src1);
|
DUMP_COLUMN(full_src1);
|
||||||
DUMP_COLUMN(sparse_src2);
|
DUMP_COLUMN(sparse_src2);
|
||||||
DUMP_COLUMN(full_src2);
|
DUMP_COLUMN(full_src2);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Compare results are unequal");
|
throw Exception(error_code, "Compare results are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -269,7 +272,7 @@ TEST(ColumnSparse, CompareColumn)
|
|||||||
test_case(n1, k1, n2, k2, row_num);
|
test_case(n1, k1, n2, k2, row_num);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
@ -281,8 +284,8 @@ TEST(ColumnSparse, GetPermutation)
|
|||||||
{
|
{
|
||||||
auto [sparse_src, full_src] = createColumns(n, k);
|
auto [sparse_src, full_src] = createColumns(n, k);
|
||||||
|
|
||||||
PaddedPODArray<UInt64> perm_sparse;
|
IColumn::Permutation perm_sparse;
|
||||||
PaddedPODArray<UInt64> perm_full;
|
IColumn::Permutation perm_full;
|
||||||
|
|
||||||
sparse_src->getPermutation(reverse, limit, 1, perm_sparse);
|
sparse_src->getPermutation(reverse, limit, 1, perm_sparse);
|
||||||
full_src->getPermutation(reverse, limit, 1, perm_full);
|
full_src->getPermutation(reverse, limit, 1, perm_full);
|
||||||
@ -303,7 +306,7 @@ TEST(ColumnSparse, GetPermutation)
|
|||||||
DUMP_COLUMN(sparse_sorted);
|
DUMP_COLUMN(sparse_sorted);
|
||||||
DUMP_COLUMN(full_sorted);
|
DUMP_COLUMN(full_sorted);
|
||||||
DUMP_NON_DEFAULTS(full_sorted);
|
DUMP_NON_DEFAULTS(full_sorted);
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sorted columns are unequal");
|
throw Exception(error_code, "Sorted columns are unequal");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -320,7 +323,7 @@ TEST(ColumnSparse, GetPermutation)
|
|||||||
test_case(n, k, limit, reverse);
|
test_case(n, k, limit, reverse);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
FAIL() << e.displayText();
|
FAIL() << e.displayText();
|
||||||
}
|
}
|
||||||
|
@ -18,7 +18,6 @@ PEERDIR(
|
|||||||
contrib/libs/openssl
|
contrib/libs/openssl
|
||||||
contrib/libs/poco/NetSSL_OpenSSL
|
contrib/libs/poco/NetSSL_OpenSSL
|
||||||
contrib/libs/re2
|
contrib/libs/re2
|
||||||
contrib/libs/cxxsupp/libcxxabi-parts
|
|
||||||
contrib/restricted/dragonbox
|
contrib/restricted/dragonbox
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -5,12 +5,6 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int ILLEGAL_COLUMN;
|
|
||||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
|
||||||
}
|
|
||||||
|
|
||||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerSimpleDataTypeCustom("IPv4", []
|
factory.registerSimpleDataTypeCustom("IPv4", []
|
||||||
|
@ -25,7 +25,6 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||||
extern const int TOO_LARGE_STRING_SIZE;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -36,6 +36,7 @@ namespace ErrorCodes
|
|||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -192,7 +192,7 @@ SerializationPtr IDataType::getSerialization(const ISerialization::Kinds & kinds
|
|||||||
{
|
{
|
||||||
if (!kinds.subcolumns.empty())
|
if (!kinds.subcolumns.empty())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR,"Data type {} doesn't support "
|
throw Exception(ErrorCodes::LOGICAL_ERROR,"Data type {} doesn't support "
|
||||||
"custom kinds of serialization for subcolumns ot doesn't have subcolumns at all.", getName());
|
"custom kinds of serialization for subcolumns or doesn't have subcolumns at all.", getName());
|
||||||
|
|
||||||
if (kinds.main == ISerialization::Kind::SPARSE)
|
if (kinds.main == ISerialization::Kind::SPARSE)
|
||||||
return getSparseSerialization();
|
return getSparseSerialization();
|
||||||
|
@ -24,6 +24,8 @@ String ISerialization::kindToString(Kind kind)
|
|||||||
case Kind::SPARSE:
|
case Kind::SPARSE:
|
||||||
return "Sparse";
|
return "Sparse";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
__builtin_unreachable();
|
||||||
}
|
}
|
||||||
|
|
||||||
void ISerialization::Kinds::writeBinary(WriteBuffer & ostr) const
|
void ISerialization::Kinds::writeBinary(WriteBuffer & ostr) const
|
||||||
|
@ -10,6 +10,7 @@ namespace DB
|
|||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
}
|
}
|
||||||
|
|
||||||
SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_)
|
SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_)
|
||||||
|
@ -1,3 +1,5 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
|
||||||
|
@ -19,7 +19,6 @@ namespace ErrorCodes
|
|||||||
{
|
{
|
||||||
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
||||||
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -467,7 +467,7 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu
|
|||||||
{
|
{
|
||||||
const auto * column_sparse = checkAndGetColumn<ColumnSparse>(arguments[i].column.get());
|
const auto * column_sparse = checkAndGetColumn<ColumnSparse>(arguments[i].column.get());
|
||||||
/// In rare case, when sparse column doesn't have default values,
|
/// In rare case, when sparse column doesn't have default values,
|
||||||
/// it's more convinient to convert it to full before execution of function.
|
/// it's more convenient to convert it to full before execution of function.
|
||||||
if (column_sparse && column_sparse->getNumberOfDefaults())
|
if (column_sparse && column_sparse->getNumberOfDefaults())
|
||||||
{
|
{
|
||||||
sparse_column_position = i;
|
sparse_column_position = i;
|
||||||
|
@ -35,7 +35,7 @@ private:
|
|||||||
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
|
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
|
||||||
|
|
||||||
ColumnPtr executeWithoutSparseColumns(
|
ColumnPtr executeWithoutSparseColumns(
|
||||||
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
|
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
class FunctionBaseAdaptor final : public IFunctionBase
|
class FunctionBaseAdaptor final : public IFunctionBase
|
||||||
|
@ -613,7 +613,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
|
|||||||
if (inst->offsets)
|
if (inst->offsets)
|
||||||
inst->batch_that->addBatchSinglePlace(
|
inst->batch_that->addBatchSinglePlace(
|
||||||
inst->offsets[static_cast<ssize_t>(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena);
|
inst->offsets[static_cast<ssize_t>(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena);
|
||||||
else if(inst->has_sparse_arguments)
|
else if (inst->has_sparse_arguments)
|
||||||
inst->batch_that->addBatchSparseSinglePlace(res + inst->state_offset, inst->batch_arguments, arena);
|
inst->batch_that->addBatchSparseSinglePlace(res + inst->state_offset, inst->batch_arguments, arena);
|
||||||
else
|
else
|
||||||
inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena);
|
inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena);
|
||||||
|
@ -403,7 +403,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name,
|
|||||||
{
|
{
|
||||||
const auto & serialization = serializations[name];
|
const auto & serialization = serializations[name];
|
||||||
|
|
||||||
if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::SPARSE)
|
if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT)
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName());
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName());
|
||||||
|
|
||||||
auto disk = data_part->volume->getDisk();
|
auto disk = data_part->volume->getDisk();
|
||||||
|
@ -564,9 +564,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart(
|
|||||||
const IMergeTreeDataPart * parent_part,
|
const IMergeTreeDataPart * parent_part,
|
||||||
size_t block_num)
|
size_t block_num)
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/// Size of part would not be greater than block.bytes() + epsilon
|
/// Size of part would not be greater than block.bytes() + epsilon
|
||||||
size_t expected_size = block.bytes();
|
size_t expected_size = block.bytes();
|
||||||
|
|
||||||
|
@ -64,8 +64,8 @@ def cluster():
|
|||||||
|
|
||||||
FILES_OVERHEAD = 1
|
FILES_OVERHEAD = 1
|
||||||
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
||||||
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
|
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1
|
||||||
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
|
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1
|
||||||
|
|
||||||
|
|
||||||
def random_string(length):
|
def random_string(length):
|
||||||
|
@ -32,8 +32,8 @@ def cluster():
|
|||||||
|
|
||||||
FILES_OVERHEAD = 1
|
FILES_OVERHEAD = 1
|
||||||
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
||||||
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
|
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1
|
||||||
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
|
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1
|
||||||
|
|
||||||
|
|
||||||
def random_string(length):
|
def random_string(length):
|
||||||
|
@ -32,8 +32,8 @@ def cluster():
|
|||||||
|
|
||||||
FILES_OVERHEAD = 1
|
FILES_OVERHEAD = 1
|
||||||
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
||||||
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
|
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1
|
||||||
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
|
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1
|
||||||
|
|
||||||
|
|
||||||
def random_string(length):
|
def random_string(length):
|
||||||
|
Loading…
Reference in New Issue
Block a user