Merge branch 'master' into fix-new-setting

This commit is contained in:
Antonio Andelic 2024-02-19 14:30:12 +01:00 committed by GitHub
commit 1512116679
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
33 changed files with 112 additions and 51 deletions

View File

@ -2,7 +2,7 @@
#if USE_AZURE_BLOB_STORAGE #if USE_AZURE_BLOB_STORAGE
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <IO/HTTPHeaderEntries.h> #include <IO/HTTPHeaderEntries.h>

View File

@ -2,7 +2,7 @@
#if USE_AWS_S3 #if USE_AWS_S3
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>

View File

@ -91,6 +91,7 @@ enum class MagicNumber : uint8_t
Decimal32 = 19, Decimal32 = 19,
Decimal64 = 20, Decimal64 = 20,
IPv4 = 21, IPv4 = 21,
Date32 = 22,
}; };
MagicNumber serializeTypeId(std::optional<TypeIndex> type_id) MagicNumber serializeTypeId(std::optional<TypeIndex> type_id)
@ -109,6 +110,7 @@ MagicNumber serializeTypeId(std::optional<TypeIndex> type_id)
case TypeIndex::Int32: return MagicNumber::Int32; case TypeIndex::Int32: return MagicNumber::Int32;
case TypeIndex::Int64: return MagicNumber::Int64; case TypeIndex::Int64: return MagicNumber::Int64;
case TypeIndex::Date: return MagicNumber::Date; case TypeIndex::Date: return MagicNumber::Date;
case TypeIndex::Date32: return MagicNumber::Date32;
case TypeIndex::DateTime: return MagicNumber::DateTime; case TypeIndex::DateTime: return MagicNumber::DateTime;
case TypeIndex::DateTime64: return MagicNumber::DateTime64; case TypeIndex::DateTime64: return MagicNumber::DateTime64;
case TypeIndex::Enum8: return MagicNumber::Enum8; case TypeIndex::Enum8: return MagicNumber::Enum8;
@ -137,6 +139,7 @@ TypeIndex deserializeTypeId(uint8_t serialized_type_id)
case MagicNumber::Int32: return TypeIndex::Int32; case MagicNumber::Int32: return TypeIndex::Int32;
case MagicNumber::Int64: return TypeIndex::Int64; case MagicNumber::Int64: return TypeIndex::Int64;
case MagicNumber::Date: return TypeIndex::Date; case MagicNumber::Date: return TypeIndex::Date;
case MagicNumber::Date32: return TypeIndex::Date32;
case MagicNumber::DateTime: return TypeIndex::DateTime; case MagicNumber::DateTime: return TypeIndex::DateTime;
case MagicNumber::DateTime64: return TypeIndex::DateTime64; case MagicNumber::DateTime64: return TypeIndex::DateTime64;
case MagicNumber::Enum8: return TypeIndex::Enum8; case MagicNumber::Enum8: return TypeIndex::Enum8;
@ -165,6 +168,7 @@ TypeIndex baseType(TypeIndex type_idx)
return TypeIndex::Int16; return TypeIndex::Int16;
case TypeIndex::Int32: case TypeIndex::Int32:
case TypeIndex::Decimal32: case TypeIndex::Decimal32:
case TypeIndex::Date32:
return TypeIndex::Int32; return TypeIndex::Int32;
case TypeIndex::Int64: case TypeIndex::Int64:
case TypeIndex::Decimal64: case TypeIndex::Decimal64:
@ -205,6 +209,7 @@ TypeIndex typeIdx(const IDataType * data_type)
case TypeIndex::UInt16: case TypeIndex::UInt16:
case TypeIndex::Enum16: case TypeIndex::Enum16:
case TypeIndex::Date: case TypeIndex::Date:
case TypeIndex::Date32:
case TypeIndex::Int32: case TypeIndex::Int32:
case TypeIndex::UInt32: case TypeIndex::UInt32:
case TypeIndex::IPv4: case TypeIndex::IPv4:

View File

@ -1019,6 +1019,7 @@ class IColumn;
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \ M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \
M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats", 0) \
M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \ M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \
M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \ M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \
M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \ M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \

View File

@ -86,6 +86,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{ {
{"24.2", { {"24.2", {
{"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"},
{"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"},
{"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."},
{"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"},
{"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"}, {"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"},

View File

@ -3,7 +3,7 @@
#include <memory> #include <memory>
#include <IO/AsynchronousReader.h> #include <IO/AsynchronousReader.h>
#include <Common/ThreadPool_fwd.h> #include <Common/ThreadPool_fwd.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
namespace DB namespace DB

View File

@ -3,7 +3,7 @@
#include <IO/AsynchronousReader.h> #include <IO/AsynchronousReader.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <Common/ThreadPool_fwd.h> #include <Common/ThreadPool_fwd.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
namespace DB namespace DB
{ {

View File

@ -22,7 +22,7 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <Disks/DirectoryIterator.h> #include <Disks/DirectoryIterator.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include "config.h" #include "config.h"

View File

@ -2,7 +2,7 @@
#include <Disks/ObjectStorages/ObjectStorageIterator.h> #include <Disks/ObjectStorages/ObjectStorageIterator.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <mutex> #include <mutex>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>

View File

@ -13,7 +13,7 @@
#include <IO/S3/getObjectInfo.h> #include <IO/S3/getObjectInfo.h>
#include <IO/S3/copyS3File.h> #include <IO/S3/copyS3File.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <IO/S3/BlobStorageLogWriter.h> #include <IO/S3/BlobStorageLogWriter.h>
#include <Disks/ObjectStorages/S3/diskSettings.h> #include <Disks/ObjectStorages/S3/diskSettings.h>

View File

@ -229,6 +229,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.try_infer_integers = settings.input_format_try_infer_integers; format_settings.try_infer_integers = settings.input_format_try_infer_integers;
format_settings.try_infer_dates = settings.input_format_try_infer_dates; format_settings.try_infer_dates = settings.input_format_try_infer_dates;
format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes; format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes;
format_settings.try_infer_exponent_floats = settings.input_format_try_infer_exponent_floats;
format_settings.markdown.escape_special_characters = settings.output_format_markdown_escape_special_characters; format_settings.markdown.escape_special_characters = settings.output_format_markdown_escape_special_characters;
format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string;
format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference;

View File

@ -46,6 +46,7 @@ struct FormatSettings
bool try_infer_integers = false; bool try_infer_integers = false;
bool try_infer_dates = false; bool try_infer_dates = false;
bool try_infer_datetimes = false; bool try_infer_datetimes = false;
bool try_infer_exponent_floats = false;
enum class DateTimeInputFormat enum class DateTimeInputFormat
{ {

View File

@ -17,6 +17,7 @@
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/parseDateTimeBestEffort.h> #include <IO/parseDateTimeBestEffort.h>
#include <IO/PeekableReadBuffer.h> #include <IO/PeekableReadBuffer.h>
#include <IO/readFloatText.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
@ -865,6 +866,13 @@ namespace
return std::make_shared<DataTypeTuple>(nested_types); return std::make_shared<DataTypeTuple>(nested_types);
} }
bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings)
{
if (settings.try_infer_exponent_floats)
return tryReadFloatText(value, buf);
return tryReadFloatTextNoExponent(value, buf);
}
DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings) DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings)
{ {
if (buf.eof()) if (buf.eof())
@ -903,7 +911,7 @@ namespace
buf.position() = number_start; buf.position() = number_start;
} }
if (tryReadFloatText(tmp_float, buf)) if (tryReadFloat(tmp_float, buf, settings))
{ {
if (read_int && buf.position() == int_end) if (read_int && buf.position() == int_end)
return std::make_shared<DataTypeInt64>(); return std::make_shared<DataTypeInt64>();
@ -937,7 +945,7 @@ namespace
peekable_buf.rollbackToCheckpoint(true); peekable_buf.rollbackToCheckpoint(true);
} }
if (tryReadFloatText(tmp_float, peekable_buf)) if (tryReadFloat(tmp_float, peekable_buf, settings))
{ {
/// Float parsing reads no fewer bytes than integer parsing, /// Float parsing reads no fewer bytes than integer parsing,
/// so position of the buffer is either the same, or further. /// so position of the buffer is either the same, or further.
@ -949,7 +957,7 @@ namespace
return std::make_shared<DataTypeFloat64>(); return std::make_shared<DataTypeFloat64>();
} }
} }
else if (tryReadFloatText(tmp_float, buf)) else if (tryReadFloat(tmp_float, buf, settings))
{ {
return std::make_shared<DataTypeFloat64>(); return std::make_shared<DataTypeFloat64>();
} }
@ -1390,7 +1398,7 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting
buf.position() = buf.buffer().begin(); buf.position() = buf.buffer().begin();
Float64 tmp; Float64 tmp;
if (tryReadFloatText(tmp, buf) && buf.eof()) if (tryReadFloat(tmp, buf, settings) && buf.eof())
return std::make_shared<DataTypeFloat64>(); return std::make_shared<DataTypeFloat64>();
return nullptr; return nullptr;

View File

@ -6,7 +6,7 @@
#include <Storages/StorageAzureBlobCluster.h> #include <Storages/StorageAzureBlobCluster.h>
#include <Storages/StorageAzureBlob.h> #include <Storages/StorageAzureBlob.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <base/types.h> #include <base/types.h>
#include <functional> #include <functional>
#include <memory> #include <memory>

View File

@ -3,7 +3,7 @@
#include <IO/BufferWithOwnMemory.h> #include <IO/BufferWithOwnMemory.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Common/ArenaWithFreeLists.h> #include <Common/ArenaWithFreeLists.h>
namespace DB namespace DB

View File

@ -5,7 +5,7 @@
#if USE_AWS_S3 #if USE_AWS_S3
#include <Storages/StorageS3Settings.h> #include <Storages/StorageS3Settings.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <IO/S3/BlobStorageLogWriter.h> #include <IO/S3/BlobStorageLogWriter.h>
#include <base/types.h> #include <base/types.h>
#include <functional> #include <functional>

View File

@ -10,7 +10,7 @@
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteSettings.h> #include <IO/WriteSettings.h>
#include <Storages/StorageS3Settings.h> #include <Storages/StorageS3Settings.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <IO/S3/BlobStorageLogWriter.h> #include <IO/S3/BlobStorageLogWriter.h>
#include <memory> #include <memory>

View File

@ -67,4 +67,7 @@ template void readFloatText<Float64>(Float64 &, ReadBuffer &);
template bool tryReadFloatText<Float32>(Float32 &, ReadBuffer &); template bool tryReadFloatText<Float32>(Float32 &, ReadBuffer &);
template bool tryReadFloatText<Float64>(Float64 &, ReadBuffer &); template bool tryReadFloatText<Float64>(Float64 &, ReadBuffer &);
template bool tryReadFloatTextNoExponent<Float32>(Float32 &, ReadBuffer &);
template bool tryReadFloatTextNoExponent<Float64>(Float64 &, ReadBuffer &);
} }

View File

@ -324,7 +324,7 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf)
} }
template <typename T, typename ReturnType> template <typename T, typename ReturnType, bool allow_exponent = true>
ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in) ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
{ {
static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for readFloatTextImpl must be float or double"); static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for readFloatTextImpl must be float or double");
@ -395,6 +395,8 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
after_point_exponent = (read_digits > significant_digits ? -significant_digits : static_cast<int>(-read_digits)) - after_point_num_leading_zeros; after_point_exponent = (read_digits > significant_digits ? -significant_digits : static_cast<int>(-read_digits)) - after_point_num_leading_zeros;
} }
if constexpr (allow_exponent)
{
if (checkChar('e', in) || checkChar('E', in)) if (checkChar('e', in) || checkChar('E', in))
{ {
if (in.eof()) if (in.eof())
@ -420,6 +422,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (exponent_negative) if (exponent_negative)
exponent = -exponent; exponent = -exponent;
} }
}
if (after_point) if (after_point)
x += static_cast<T>(shift10(after_point, after_point_exponent)); x += static_cast<T>(shift10(after_point, after_point_exponent));
@ -604,4 +607,7 @@ template <typename T> bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { retu
template <typename T> void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); } template <typename T> void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); }
template <typename T> bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); } template <typename T> bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); }
/// Don't read exponent part of the number.
template <typename T> bool tryReadFloatTextNoExponent(T & x, ReadBuffer & in) { return readFloatTextFastImpl<T, bool, false>(x, in); }
} }

View File

@ -35,7 +35,7 @@
#include <Storages/extractKeyExpressionList.h> #include <Storages/extractKeyExpressionList.h>
#include <Storages/PartitionCommands.h> #include <Storages/PartitionCommands.h>
#include <Interpreters/PartLog.h> #include <Interpreters/PartLog.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <boost/multi_index_container.hpp> #include <boost/multi_index_container.hpp>

View File

@ -1,6 +1,6 @@
#include <Compression/CompressedReadBufferFromFile.h> #include <Compression/CompressedReadBufferFromFile.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeMarksLoader.h> #include <Storages/MergeTree/MergeTreeMarksLoader.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>

View File

@ -1,6 +1,6 @@
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Storages/MergeTree/AlterConversions.h> #include <Storages/MergeTree/AlterConversions.h>
#include <Storages/MergeTree/IMergeTreeReader.h> #include <Storages/MergeTree/IMergeTreeReader.h>
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h> #include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>

View File

@ -1,6 +1,6 @@
#include <Storages/MergeTree/MergeTreeSource.h> #include <Storages/MergeTree/MergeTreeSource.h>
#include <Storages/MergeTree/MergeTreeSelectProcessor.h> #include <Storages/MergeTree/MergeTreeSelectProcessor.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <Common/EventFD.h> #include <Common/EventFD.h>

View File

@ -19,7 +19,7 @@
#include <IO/CompressionMethod.h> #include <IO/CompressionMethod.h>
#include <IO/SeekableReadBuffer.h> #include <IO/SeekableReadBuffer.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Storages/Cache/SchemaCache.h> #include <Storages/Cache/SchemaCache.h>
#include <Storages/SelectQueryInfo.h> #include <Storages/SelectQueryInfo.h>
#include <Storages/StorageConfiguration.h> #include <Storages/StorageConfiguration.h>

View File

@ -5,7 +5,7 @@
#include <Storages/VirtualColumnUtils.h> #include <Storages/VirtualColumnUtils.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTInsertQuery.h> #include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>

View File

@ -11,7 +11,7 @@
#include <Processors/Sources/SourceFromSingleChunk.h> #include <Processors/Sources/SourceFromSingleChunk.h>
#include <QueryPipeline/Pipe.h> #include <QueryPipeline/Pipe.h>
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <Interpreters/threadPoolCallbackRunner.h> #include <Common/threadPoolCallbackRunner.h>
#include <mutex> #include <mutex>

View File

@ -0,0 +1,4 @@
1970-01-01 1970-01-01 1950-01-01 1950-01-01
1970-01-01 1970-01-01 1970-01-01 1970-01-01
2149-06-06 2149-06-06 2149-06-06 2149-06-06
2149-06-06 2149-06-06 2149-06-08 2149-06-08

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS t64;
CREATE TABLE t64
(
date16 Date,
t_date16 Date Codec(T64, ZSTD),
date_32 Date32,
t_date32 Date32 Codec(T64, ZSTD)
) ENGINE MergeTree() ORDER BY tuple();
INSERT INTO t64 values ('1970-01-01', '1970-01-01', '1970-01-01', '1970-01-01');
INSERT INTO t64 values ('2149-06-06', '2149-06-06', '2149-06-06', '2149-06-06');
INSERT INTO t64 values ('2149-06-08', '2149-06-08', '2149-06-08', '2149-06-08');
INSERT INTO t64 values ('1950-01-01', '1950-01-01', '1950-01-01', '1950-01-01');
SELECT * FROM t64 ORDER BY date_32;
SELECT * FROM t64 WHERE date16 != t_date16;
SELECT * FROM t64 WHERE date_32 != t_date32;
OPTIMIZE TABLE t64 FINAL;
SELECT * FROM t64 WHERE date16 != t_date16;
SELECT * FROM t64 WHERE date_32 != t_date32;
DROP TABLE t64;

View File

@ -1,6 +1,7 @@
-- Tags: no-fasttest -- Tags: no-fasttest
set input_format_try_infer_integers=1; set input_format_try_infer_integers=1;
set input_format_try_infer_exponent_floats=1;
select 'JSONEachRow'; select 'JSONEachRow';
desc format(JSONEachRow, '{"x" : 123}'); desc format(JSONEachRow, '{"x" : 123}');

View File

@ -8,10 +8,10 @@ $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1.2}')";
echo '{"x" : 1.2}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; echo '{"x" : 1.2}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test";
$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1}')"; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1}')";
echo '{"x" : 1}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; echo '{"x" : 1}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test";
$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1e10}')"; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : 1e10}')" --input_format_try_infer_exponent_floats=1;
echo '{"x" : 1e10}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; echo '{"x" : 1e10}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1;
$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, 1, 1e10]}')"; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, 1, 1e10]}')" --input_format_try_infer_exponent_floats=1;
echo '{"x" : [1, 42.42, 1, 1e10]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; echo '{"x" : [1, 42.42, 1, 1e10]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1;
$CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, false]}')"; $CLICKHOUSE_LOCAL -q "desc format(JSONEachRow, '{\"x\" : [1, 42.42, false]}')";
echo '{"x" : [1, 42.42, false]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test"; echo '{"x" : [1, 42.42, false]}' | $CLICKHOUSE_LOCAL --input-format='JSONEachRow' --table='test' -q "desc test";
@ -19,10 +19,10 @@ $CLICKHOUSE_LOCAL -q "desc format(TSV, '1.2')";
echo '1.2' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; echo '1.2' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test";
$CLICKHOUSE_LOCAL -q "desc format(TSV, '1')"; $CLICKHOUSE_LOCAL -q "desc format(TSV, '1')";
echo '1' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; echo '1' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test";
$CLICKHOUSE_LOCAL -q "desc format(TSV, '1e10')"; $CLICKHOUSE_LOCAL -q "desc format(TSV, '1e10')" --input_format_try_infer_exponent_floats=1;
echo '1e10' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; echo '1e10' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1;
$CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, 1, 1e10]')"; $CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, 1, 1e10]')" --input_format_try_infer_exponent_floats=1;
echo '[1, 42.42, 1, 1e10]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; echo '[1, 42.42, 1, 1e10]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test" --input_format_try_infer_exponent_floats=1;
$CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, false]')"; $CLICKHOUSE_LOCAL -q "desc format(TSV, '[1, 42.42, false]')";
echo '[1, 42.42, false]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test"; echo '[1, 42.42, false]' | $CLICKHOUSE_LOCAL --input-format='TSV' --table='test' -q "desc test";

View File

@ -0,0 +1,2 @@
c1 Nullable(String)
c1 Nullable(Float64)

View File

@ -0,0 +1,2 @@
DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 0;
DESC format(CSV, '1E20\n1.1E20') settings input_format_try_infer_exponent_floats = 1;