2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Defines.h>
|
2013-02-03 18:39:09 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
2010-05-13 16:13:38 +00:00
|
|
|
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
|
2019-10-04 17:46:36 +00:00
|
|
|
#include <Core/Field.h>
|
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2019-02-19 20:01:31 +00:00
|
|
|
#include <Formats/ProtobufReader.h>
|
2019-01-23 19:41:18 +00:00
|
|
|
#include <Formats/ProtobufWriter.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
2017-07-16 03:05:40 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
2010-05-13 16:13:38 +00:00
|
|
|
|
2020-05-18 23:53:41 +00:00
|
|
|
#include <Parsers/IAST.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <IO/VarInt.h>
|
2010-06-04 18:25:25 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
#ifdef __SSE2__
|
2017-04-01 07:20:54 +00:00
|
|
|
#include <emmintrin.h>
|
2016-01-13 21:05:11 +00:00
|
|
|
#endif
|
2015-02-15 14:25:43 +00:00
|
|
|
|
2010-05-13 16:13:38 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-05-18 23:53:41 +00:00
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int UNEXPECTED_AST_STRUCTURE;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2010-06-04 18:25:25 +00:00
|
|
|
void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & s = get<const String &>(field);
|
|
|
|
writeVarUInt(s.size(), ostr);
|
|
|
|
writeString(s, ostr);
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2010-06-04 18:25:25 +00:00
|
|
|
void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt64 size;
|
|
|
|
readVarUInt(size, istr);
|
|
|
|
field = String();
|
|
|
|
String & s = get<String &>(field);
|
|
|
|
s.resize(size);
|
2018-09-02 03:00:04 +00:00
|
|
|
istr.readStrict(s.data(), size);
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
const StringRef & s = assert_cast<const ColumnString &>(column).getDataAt(row_num);
|
2017-04-01 07:20:54 +00:00
|
|
|
writeVarUInt(s.size, ostr);
|
|
|
|
writeString(s, ostr);
|
2016-02-16 16:39:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) const
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
ColumnString & column_string = assert_cast<ColumnString &>(column);
|
2018-11-25 00:08:50 +00:00
|
|
|
ColumnString::Chars & data = column_string.getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets = column_string.getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
UInt64 size;
|
|
|
|
readVarUInt(size, istr);
|
|
|
|
|
|
|
|
size_t old_chars_size = data.size();
|
|
|
|
size_t offset = old_chars_size + size + 1;
|
|
|
|
offsets.push_back(offset);
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
data.resize(offset);
|
|
|
|
istr.readStrict(reinterpret_cast<char*>(&data[offset - size - 1]), size);
|
|
|
|
data.back() = 0;
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
offsets.pop_back();
|
|
|
|
data.resize_assume_reserved(old_chars_size);
|
|
|
|
throw;
|
|
|
|
}
|
2016-02-16 16:39:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-19 00:41:24 +00:00
|
|
|
void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
const ColumnString & column_string = typeid_cast<const ColumnString &>(column);
|
2018-11-25 00:08:50 +00:00
|
|
|
const ColumnString::Chars & data = column_string.getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
const ColumnString::Offsets & offsets = column_string.getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
size_t size = column.size();
|
|
|
|
if (!size)
|
|
|
|
return;
|
|
|
|
|
|
|
|
size_t end = limit && offset + limit < size
|
|
|
|
? offset + limit
|
|
|
|
: size;
|
|
|
|
|
|
|
|
if (offset == 0)
|
|
|
|
{
|
|
|
|
UInt64 str_size = offsets[0] - 1;
|
|
|
|
writeVarUInt(str_size, ostr);
|
2018-09-02 03:00:04 +00:00
|
|
|
ostr.write(reinterpret_cast<const char *>(data.data()), str_size);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
++offset;
|
|
|
|
}
|
|
|
|
|
|
|
|
for (size_t i = offset; i < end; ++i)
|
|
|
|
{
|
|
|
|
UInt64 str_size = offsets[i] - offsets[i - 1] - 1;
|
|
|
|
writeVarUInt(str_size, ostr);
|
|
|
|
ostr.write(reinterpret_cast<const char *>(&data[offsets[i - 1]]), str_size);
|
|
|
|
}
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-02-15 15:18:21 +00:00
|
|
|
template <int UNROLL_TIMES>
|
2019-02-19 00:41:24 +00:00
|
|
|
static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit)
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t offset = data.size();
|
|
|
|
for (size_t i = 0; i < limit; ++i)
|
|
|
|
{
|
|
|
|
if (istr.eof())
|
|
|
|
break;
|
2010-06-04 18:38:56 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt64 size;
|
|
|
|
readVarUInt(size, istr);
|
2010-05-13 16:13:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
offset += size + 1;
|
|
|
|
offsets.push_back(offset);
|
2010-05-13 16:13:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
data.resize(offset);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (size)
|
|
|
|
{
|
2019-03-08 03:26:12 +00:00
|
|
|
#ifdef __SSE2__
|
2017-04-01 07:20:54 +00:00
|
|
|
/// An optimistic branch in which more efficient copying is possible.
|
2019-03-08 03:32:43 +00:00
|
|
|
if (offset + 16 * UNROLL_TIMES <= data.capacity() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
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;
|
|
|
|
__m128i * sse_dst_pos = reinterpret_cast<__m128i *>(&data[offset - size - 1]);
|
|
|
|
|
|
|
|
while (sse_src_pos < sse_src_end)
|
|
|
|
{
|
2019-03-08 03:26:12 +00:00
|
|
|
for (size_t j = 0; j < UNROLL_TIMES; ++j)
|
|
|
|
_mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
sse_src_pos += UNROLL_TIMES;
|
|
|
|
sse_dst_pos += UNROLL_TIMES;
|
|
|
|
}
|
|
|
|
|
|
|
|
istr.position() += size;
|
|
|
|
}
|
|
|
|
else
|
2016-01-13 21:05:11 +00:00
|
|
|
#endif
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
istr.readStrict(reinterpret_cast<char*>(&data[offset - size - 1]), size);
|
|
|
|
}
|
|
|
|
}
|
2015-02-15 14:25:43 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
data[offset - 1] = 0;
|
|
|
|
}
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-19 00:41:24 +00:00
|
|
|
void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
|
2015-02-15 15:18:21 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnString & column_string = typeid_cast<ColumnString &>(column);
|
2018-11-25 00:08:50 +00:00
|
|
|
ColumnString::Chars & data = column_string.getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets = column_string.getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-19 12:31:26 +00:00
|
|
|
double avg_chars_size = 1; /// By default reserve only for empty strings.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0]))
|
|
|
|
{
|
|
|
|
/// Randomly selected.
|
|
|
|
constexpr auto avg_value_size_hint_reserve_multiplier = 1.2;
|
|
|
|
|
|
|
|
avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier;
|
|
|
|
}
|
2018-11-19 12:31:26 +00:00
|
|
|
|
2018-11-21 03:19:29 +00:00
|
|
|
size_t size_to_reserve = data.size() + std::ceil(limit * avg_chars_size);
|
|
|
|
|
|
|
|
/// Never reserve for too big size.
|
|
|
|
if (size_to_reserve < 256 * 1024 * 1024)
|
2018-11-19 12:31:26 +00:00
|
|
|
{
|
2018-11-21 03:19:29 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
data.reserve(size_to_reserve);
|
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
e.addMessage(
|
|
|
|
"(avg_value_size_hint = " + toString(avg_value_size_hint)
|
|
|
|
+ ", avg_chars_size = " + toString(avg_chars_size)
|
|
|
|
+ ", limit = " + toString(limit) + ")");
|
|
|
|
throw;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
offsets.reserve(offsets.size() + limit);
|
|
|
|
|
|
|
|
if (avg_chars_size >= 64)
|
|
|
|
deserializeBinarySSE2<4>(data, offsets, istr, limit);
|
|
|
|
else if (avg_chars_size >= 48)
|
|
|
|
deserializeBinarySSE2<3>(data, offsets, istr, limit);
|
|
|
|
else if (avg_chars_size >= 32)
|
|
|
|
deserializeBinarySSE2<2>(data, offsets, istr, limit);
|
|
|
|
else
|
|
|
|
deserializeBinarySSE2<1>(data, offsets, istr, limit);
|
2015-02-15 15:18:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeEscapedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-02-16 16:39:39 +00:00
|
|
|
template <typename Reader>
|
2017-12-01 20:21:35 +00:00
|
|
|
static inline void read(IColumn & column, Reader && reader)
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
ColumnString & column_string = assert_cast<ColumnString &>(column);
|
2018-11-25 00:08:50 +00:00
|
|
|
ColumnString::Chars & data = column_string.getChars();
|
2017-12-15 21:32:25 +00:00
|
|
|
ColumnString::Offsets & offsets = column_string.getOffsets();
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t old_chars_size = data.size();
|
|
|
|
size_t old_offsets_size = offsets.size();
|
|
|
|
try
|
|
|
|
{
|
|
|
|
reader(data);
|
|
|
|
data.push_back(0);
|
|
|
|
offsets.push_back(data.size());
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
offsets.resize_assume_reserved(old_offsets_size);
|
|
|
|
data.resize_assume_reserved(old_chars_size);
|
|
|
|
throw;
|
|
|
|
}
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-05-18 21:07:23 +00:00
|
|
|
void DataTypeString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
|
|
|
{
|
|
|
|
read(column, [&](ColumnString::Chars & data) { readStringInto(data, istr); });
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2018-11-25 00:08:50 +00:00
|
|
|
read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); });
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeQuotedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2018-11-25 00:08:50 +00:00
|
|
|
read(column, [&](ColumnString::Chars & data) { readQuotedStringInto<true>(data, istr); });
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-08-06 10:02:26 +00:00
|
|
|
void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
2013-05-16 12:52:09 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeJSONString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr, settings);
|
2013-05-16 12:52:09 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
2016-02-18 11:44:50 +00:00
|
|
|
{
|
2018-11-25 00:08:50 +00:00
|
|
|
read(column, [&](ColumnString::Chars & data) { readJSONStringInto(data, istr); });
|
2016-02-18 11:44:50 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-08 01:51:55 +00:00
|
|
|
void DataTypeString::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
2016-02-14 02:37:42 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeXMLString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
2016-02-14 02:37:42 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
2016-02-07 08:42:21 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
writeCSVString<>(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
2016-02-07 08:42:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-06-05 21:39:01 +00:00
|
|
|
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
2016-02-07 08:42:21 +00:00
|
|
|
{
|
2018-11-25 00:08:50 +00:00
|
|
|
read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); });
|
2016-02-07 08:42:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-26 14:06:05 +00:00
|
|
|
void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
|
2019-01-23 19:41:18 +00:00
|
|
|
{
|
2019-02-26 14:06:05 +00:00
|
|
|
if (value_index)
|
|
|
|
return;
|
2019-08-21 02:28:04 +00:00
|
|
|
value_index = static_cast<bool>(protobuf.writeString(assert_cast<const ColumnString &>(column).getDataAt(row_num)));
|
2019-01-23 19:41:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-19 20:01:31 +00:00
|
|
|
void DataTypeString::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const
|
|
|
|
{
|
|
|
|
row_added = false;
|
2019-08-21 02:28:04 +00:00
|
|
|
auto & column_string = assert_cast<ColumnString &>(column);
|
2019-02-19 20:01:31 +00:00
|
|
|
ColumnString::Chars & data = column_string.getChars();
|
|
|
|
ColumnString::Offsets & offsets = column_string.getOffsets();
|
|
|
|
size_t old_size = offsets.size();
|
|
|
|
try
|
|
|
|
{
|
|
|
|
if (allow_add_row)
|
|
|
|
{
|
|
|
|
if (protobuf.readStringInto(data))
|
|
|
|
{
|
|
|
|
data.emplace_back(0);
|
|
|
|
offsets.emplace_back(data.size());
|
|
|
|
row_added = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
data.resize_assume_reserved(offsets.back());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ColumnString::Chars temp_data;
|
|
|
|
if (protobuf.readStringInto(temp_data))
|
|
|
|
{
|
|
|
|
temp_data.emplace_back(0);
|
|
|
|
column_string.popBack(1);
|
|
|
|
old_size = offsets.size();
|
|
|
|
data.insertSmallAllowReadWriteOverflow15(temp_data.begin(), temp_data.end());
|
|
|
|
offsets.emplace_back(data.size());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
offsets.resize_assume_reserved(old_size);
|
|
|
|
data.resize_assume_reserved(offsets.back());
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-04 17:46:36 +00:00
|
|
|
Field DataTypeString::getDefault() const
|
|
|
|
{
|
|
|
|
return String();
|
|
|
|
}
|
2019-02-19 20:01:31 +00:00
|
|
|
|
2017-12-14 03:56:56 +00:00
|
|
|
MutableColumnPtr DataTypeString::createColumn() const
|
2010-05-13 16:13:38 +00:00
|
|
|
{
|
2017-12-14 01:43:19 +00:00
|
|
|
return ColumnString::create();
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|
|
|
|
|
2011-08-12 18:27:39 +00:00
|
|
|
|
2017-12-23 01:05:29 +00:00
|
|
|
bool DataTypeString::equals(const IDataType & rhs) const
|
|
|
|
{
|
|
|
|
return typeid(rhs) == typeid(*this);
|
|
|
|
}
|
|
|
|
|
2020-05-18 23:53:41 +00:00
|
|
|
static DataTypePtr create(const ASTPtr & arguments)
|
|
|
|
{
|
2020-06-18 12:52:05 +00:00
|
|
|
if (arguments && !arguments->children.empty())
|
2020-05-27 12:13:35 +00:00
|
|
|
{
|
2020-05-18 23:53:41 +00:00
|
|
|
if (arguments->children.size() > 1)
|
2020-06-18 12:52:05 +00:00
|
|
|
throw Exception("String data type family mustn't have more than one argument - size in characters", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2020-05-18 23:53:41 +00:00
|
|
|
|
|
|
|
const auto * argument = arguments->children[0]->as<ASTLiteral>();
|
|
|
|
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
|
2020-06-18 12:52:05 +00:00
|
|
|
throw Exception("String data type family may have only a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
2020-05-18 23:53:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeString>();
|
|
|
|
}
|
|
|
|
|
2017-12-23 01:05:29 +00:00
|
|
|
|
2017-07-16 03:05:40 +00:00
|
|
|
void registerDataTypeString(DataTypeFactory & factory)
|
|
|
|
{
|
2020-05-18 23:53:41 +00:00
|
|
|
factory.registerDataType("String", create);
|
2017-07-16 05:09:47 +00:00
|
|
|
|
2020-06-27 19:05:00 +00:00
|
|
|
/// These synonims are added for compatibility.
|
2017-07-16 05:09:47 +00:00
|
|
|
|
2018-07-23 16:10:57 +00:00
|
|
|
factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-18 19:13:13 +00:00
|
|
|
factory.registerAlias("NCHAR", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-26 20:58:51 +00:00
|
|
|
factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive);
|
2018-07-23 16:10:57 +00:00
|
|
|
factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-18 19:13:13 +00:00
|
|
|
factory.registerAlias("NVARCHAR", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-17 08:35:08 +00:00
|
|
|
factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle
|
2018-07-23 16:10:57 +00:00
|
|
|
factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-17 08:35:08 +00:00
|
|
|
factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive);
|
2018-07-23 16:10:57 +00:00
|
|
|
factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive);
|
2020-05-17 08:35:08 +00:00
|
|
|
factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL
|
2017-07-16 03:05:40 +00:00
|
|
|
|
2020-06-18 12:52:05 +00:00
|
|
|
factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
2020-06-14 21:33:25 +00:00
|
|
|
factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
2020-06-18 12:52:05 +00:00
|
|
|
factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
|
|
|
factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive);
|
2017-07-16 03:05:40 +00:00
|
|
|
}
|
2010-05-13 16:13:38 +00:00
|
|
|
}
|