mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Move unit tests to gtest_convertFieldToType.cpp
This commit is contained in:
parent
1c883b802d
commit
9d7bb2a430
@ -147,7 +147,7 @@ INSTANTIATE_TEST_SUITE_P(
|
||||
DecimalField(DateTime64(123 * Day * 1'000'000), 6)
|
||||
}
|
||||
})
|
||||
);
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
DateTimeToDateTime64,
|
||||
@ -179,3 +179,84 @@ INSTANTIATE_TEST_SUITE_P(
|
||||
},
|
||||
})
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
StringToNumber,
|
||||
ConvertFieldToTypeTest,
|
||||
::testing::ValuesIn(std::initializer_list<ConvertFieldToTypeTestParams>{
|
||||
{
|
||||
"String",
|
||||
Field("1"),
|
||||
"Int8",
|
||||
Field(1)
|
||||
},
|
||||
{
|
||||
"String",
|
||||
Field("256"),
|
||||
"Int8",
|
||||
Field()
|
||||
},
|
||||
{
|
||||
"String",
|
||||
Field("not a number"),
|
||||
"Int8",
|
||||
{}
|
||||
},
|
||||
{
|
||||
"String",
|
||||
Field("1.1"),
|
||||
"Int8",
|
||||
{} /// we can not convert '1.1' to Int8
|
||||
},
|
||||
{
|
||||
"String",
|
||||
Field("1.1"),
|
||||
"Float64",
|
||||
Field(1.1)
|
||||
},
|
||||
})
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
NumberToString,
|
||||
ConvertFieldToTypeTest,
|
||||
::testing::ValuesIn(std::initializer_list<ConvertFieldToTypeTestParams>{
|
||||
{
|
||||
"Int8",
|
||||
Field(1),
|
||||
"String",
|
||||
Field("1")
|
||||
},
|
||||
{
|
||||
"Int8",
|
||||
Field(-1),
|
||||
"String",
|
||||
Field("-1")
|
||||
},
|
||||
{
|
||||
"Float64",
|
||||
Field(1.1),
|
||||
"String",
|
||||
Field("1.1")
|
||||
},
|
||||
})
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
StringToDate,
|
||||
ConvertFieldToTypeTest,
|
||||
::testing::ValuesIn(std::initializer_list<ConvertFieldToTypeTestParams>{
|
||||
{
|
||||
"String",
|
||||
Field("2024-07-12"),
|
||||
"Date",
|
||||
Field(static_cast<UInt16>(19916))
|
||||
},
|
||||
{
|
||||
"String",
|
||||
Field("not a date"),
|
||||
"Date",
|
||||
{}
|
||||
},
|
||||
})
|
||||
);
|
||||
|
@ -204,15 +204,15 @@ void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Va
|
||||
|
||||
MergeTreeStatisticsFactory::MergeTreeStatisticsFactory()
|
||||
{
|
||||
registerValidator(StatisticsType::TDigest, TDigestValidator);
|
||||
registerCreator(StatisticsType::TDigest, TDigestCreator);
|
||||
registerValidator(StatisticsType::TDigest, tdigestValidator);
|
||||
registerCreator(StatisticsType::TDigest, tdigestCreator);
|
||||
|
||||
registerValidator(StatisticsType::Uniq, UniqValidator);
|
||||
registerCreator(StatisticsType::Uniq, UniqCreator);
|
||||
registerValidator(StatisticsType::Uniq, uniqValidator);
|
||||
registerCreator(StatisticsType::Uniq, uniqCreator);
|
||||
|
||||
#if USE_DATASKETCHES
|
||||
registerValidator(StatisticsType::CountMinSketch, CountMinSketchValidator);
|
||||
registerCreator(StatisticsType::CountMinSketch, CountMinSketchCreator);
|
||||
registerValidator(StatisticsType::CountMinSketch, countMinSketchValidator);
|
||||
registerCreator(StatisticsType::CountMinSketch, countMinSketchCreator);
|
||||
#endif
|
||||
}
|
||||
|
||||
|
@ -17,13 +17,18 @@ extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_STATISTICS;
|
||||
}
|
||||
|
||||
/// Constants chosen based on rolling dices, which provides an error tolerance of 0.1% (ε = 0.001) and a confidence level of 99.9% (δ = 0.001).
|
||||
/// Constants chosen based on rolling dices.
|
||||
/// The values provides:
|
||||
/// 1. an error tolerance of 0.1% (ε = 0.001)
|
||||
/// 2. a confidence level of 99.9% (δ = 0.001).
|
||||
/// And sketch the size is 152kb.
|
||||
static constexpr auto num_hashes = 7uz;
|
||||
static constexpr auto num_buckets = 2718uz;
|
||||
|
||||
StatisticsCountMinSketch::StatisticsCountMinSketch(const SingleStatisticsDescription & stat_, DataTypePtr data_type_)
|
||||
: IStatistics(stat_), sketch(num_hashes, num_buckets), data_type(data_type_)
|
||||
: IStatistics(stat_)
|
||||
, sketch(num_hashes, num_buckets)
|
||||
, data_type(data_type_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -50,8 +55,7 @@ Float64 StatisticsCountMinSketch::estimateEqual(const Field & val) const
|
||||
|
||||
void StatisticsCountMinSketch::update(const ColumnPtr & column)
|
||||
{
|
||||
size_t size = column->size();
|
||||
for (size_t row = 0; row < size; ++row)
|
||||
for (size_t row = 0; row < column->size(); ++row)
|
||||
{
|
||||
if (column->isNullAt(row))
|
||||
continue;
|
||||
@ -80,16 +84,15 @@ void StatisticsCountMinSketch::deserialize(ReadBuffer & buf)
|
||||
}
|
||||
|
||||
|
||||
void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
{
|
||||
data_type = removeNullable(data_type);
|
||||
data_type = removeLowCardinalityAndNullable(data_type);
|
||||
/// Data types of Numeric, String family, IPv4, IPv6, Date family, Enum family are supported.
|
||||
if (!data_type->isValueRepresentedByNumber() && !isStringOrFixedString(data_type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'count_min' does not support type {}", data_type->getName());
|
||||
}
|
||||
|
||||
StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
|
||||
StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
|
||||
{
|
||||
return std::make_shared<StatisticsCountMinSketch>(stat, data_type);
|
||||
}
|
||||
|
@ -31,8 +31,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void CountMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr CountMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr);
|
||||
void countMinSketchValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr countMinSketchCreator(const SingleStatisticsDescription & stat, DataTypePtr);
|
||||
|
||||
}
|
||||
|
||||
|
@ -46,7 +46,7 @@ Float64 StatisticsTDigest::estimateLess(const Field & val) const
|
||||
auto val_as_float = StatisticsUtils::tryConvertToFloat64(val);
|
||||
if (val_as_float)
|
||||
return t_digest.getCountLessThan(*val_as_float);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimate value of type {}", val.getTypeName());
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName());
|
||||
}
|
||||
|
||||
Float64 StatisticsTDigest::estimateEqual(const Field & val) const
|
||||
@ -57,7 +57,7 @@ Float64 StatisticsTDigest::estimateEqual(const Field & val) const
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Statistics 'tdigest' does not support estimating value of type {}", val.getTypeName());
|
||||
}
|
||||
|
||||
void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
{
|
||||
data_type = removeNullable(data_type);
|
||||
data_type = removeLowCardinalityAndNullable(data_type);
|
||||
@ -65,7 +65,7 @@ void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type
|
||||
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' do not support type {}", data_type->getName());
|
||||
}
|
||||
|
||||
StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr)
|
||||
StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr)
|
||||
{
|
||||
return std::make_shared<StatisticsTDigest>(stat);
|
||||
}
|
||||
|
@ -23,7 +23,7 @@ private:
|
||||
QuantileTDigest<Float64> t_digest;
|
||||
};
|
||||
|
||||
void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr TDigestCreator(const SingleStatisticsDescription & stat, DataTypePtr);
|
||||
void tdigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr tdigestCreator(const SingleStatisticsDescription & stat, DataTypePtr);
|
||||
|
||||
}
|
||||
|
@ -52,7 +52,7 @@ UInt64 StatisticsUniq::estimateCardinality() const
|
||||
return column->getUInt(0);
|
||||
}
|
||||
|
||||
void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
{
|
||||
data_type = removeNullable(data_type);
|
||||
data_type = removeLowCardinalityAndNullable(data_type);
|
||||
@ -60,7 +60,7 @@ void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' do not support type {}", data_type->getName());
|
||||
}
|
||||
|
||||
StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
|
||||
StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type)
|
||||
{
|
||||
return std::make_shared<StatisticsUniq>(stat, data_type);
|
||||
}
|
||||
|
@ -27,7 +27,7 @@ private:
|
||||
|
||||
};
|
||||
|
||||
void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type);
|
||||
void uniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type);
|
||||
StatisticsPtr uniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type);
|
||||
|
||||
}
|
||||
|
@ -44,50 +44,3 @@ TEST(Statistics, TDigestLessThan)
|
||||
std::reverse(data.begin(), data.end());
|
||||
test_less_than(data, {-1, 1e9, 50000.0, 3000.0, 30.0}, {0, 100000, 50000, 3000, 30}, {0, 0, 0.001, 0.001, 0.001});
|
||||
}
|
||||
|
||||
using Fields = std::vector<Field>;
|
||||
|
||||
template <typename T>
|
||||
void testConvertFieldToDataType(const DataTypePtr & data_type, const Fields & fields, const T & expected_value, bool convert_failed = false)
|
||||
{
|
||||
for (const auto & field : fields)
|
||||
{
|
||||
Field converted_value;
|
||||
try
|
||||
{
|
||||
converted_value = convertFieldToType(field, *data_type);
|
||||
}
|
||||
catch(...)
|
||||
{
|
||||
ASSERT_TRUE(convert_failed);
|
||||
}
|
||||
if (convert_failed)
|
||||
ASSERT_TRUE(converted_value.isNull());
|
||||
else
|
||||
ASSERT_EQ(converted_value.template get<T>(), expected_value);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(Statistics, convertFieldToType)
|
||||
{
|
||||
Fields fields;
|
||||
|
||||
auto data_type_int8 = DataTypeFactory::instance().get("Int8");
|
||||
fields = {1, 1.0, "1"};
|
||||
testConvertFieldToDataType(data_type_int8, fields, static_cast<Int8>(1));
|
||||
|
||||
fields = {256, 1.1, "not a number"};
|
||||
testConvertFieldToDataType(data_type_int8, fields, static_cast<Int8>(1), true);
|
||||
|
||||
auto data_type_float64 = DataTypeFactory::instance().get("Float64");
|
||||
fields = {1, 1.0, "1.0"};
|
||||
testConvertFieldToDataType(data_type_float64, fields, static_cast<Float64>(1.0));
|
||||
|
||||
auto data_type_string = DataTypeFactory::instance().get("String");
|
||||
fields = {1, "1"};
|
||||
testConvertFieldToDataType(data_type_string, fields, static_cast<String>("1"));
|
||||
|
||||
auto data_type_date = DataTypeFactory::instance().get("Date");
|
||||
fields = {"2024-07-12", 19916};
|
||||
testConvertFieldToDataType(data_type_date, fields, static_cast<UInt64>(19916));
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user