minor fixes

This commit is contained in:
Anton Popov 2021-05-16 21:59:43 +03:00
parent 5c7371333e
commit 86d6673d61
20 changed files with 36 additions and 44 deletions

View File

@ -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:

View File

@ -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;

View File

@ -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();
} }

View File

@ -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
) )

View File

@ -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", []

View File

@ -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;
} }

View File

@ -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;
} }

View File

@ -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();

View File

@ -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

View File

@ -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_)

View File

@ -1,3 +1,5 @@
#pragma once
#include <DataTypes/Serializations/SimpleTextSerialization.h> #include <DataTypes/Serializations/SimpleTextSerialization.h>
#include <Common/Exception.h> #include <Common/Exception.h>

View File

@ -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;
} }

View File

@ -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;

View File

@ -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

View File

@ -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);

View File

@ -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();

View File

@ -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();

View File

@ -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):

View File

@ -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):

View File

@ -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):