ClickHouse/dbms/src/DataTypes/DataTypeNullable.cpp

223 lines
7.8 KiB
C++
Raw Normal View History

#include <DB/DataTypes/DataTypeNullable.h>
2016-07-14 12:58:30 +00:00
#include <DB/DataTypes/NullSymbol.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Common/typeid_cast.h>
2016-07-14 12:58:30 +00:00
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
2016-07-11 16:41:57 +00:00
namespace
{
2016-08-17 16:31:00 +00:00
/// When a column is serialized as a binary data file, its null values are directly
/// represented as null symbols into this file.
/// The template class below provides one method that takes a nullable column being
/// deserialized and looks for the next null symbol from the corresponding binary
/// data file. It updates the null map of the nullable column accordingly. Moreover
/// if a null symbol has been found, a default value is appended to the column data.
2016-07-14 12:58:30 +00:00
template <typename Null>
2016-08-17 16:31:00 +00:00
struct NullDeserializer
2016-07-14 12:58:30 +00:00
{
2016-08-17 16:31:00 +00:00
static bool execute(ColumnNullable & col, ReadBuffer & istr)
2016-07-14 12:58:30 +00:00
{
2016-08-17 16:31:00 +00:00
if (!istr.eof())
2016-07-14 12:58:30 +00:00
{
2016-08-17 16:31:00 +00:00
auto & null_map = static_cast<ColumnUInt8 &>(*col.getNullValuesByteMap()).getData();
if (*istr.position() == Null::name[0])
{
++istr.position();
static constexpr auto length = strlen(Null::name);
if (length > 1)
assertString(&Null::name[length - 1], istr);
null_map.push_back(1);
ColumnPtr & nested_col = col.getNestedColumn();
nested_col->insertDefault();
return true;
}
else
2016-07-14 12:58:30 +00:00
{
2016-08-17 16:31:00 +00:00
null_map.push_back(0);
return false;
2016-07-14 12:58:30 +00:00
}
}
2016-08-17 16:31:00 +00:00
else
return false;
2016-07-14 12:58:30 +00:00
}
};
2016-07-11 16:41:57 +00:00
}
DataTypeNullable::DataTypeNullable(DataTypePtr nested_data_type_)
: nested_data_type{nested_data_type_}
{
}
void DataTypeNullable::serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
nested_data_type->serializeBinary(*col->getNestedColumn(), ostr, offset, limit);
}
void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
nested_data_type->deserializeBinary(*col->getNestedColumn(), istr, limit, avg_value_size_hint);
}
void DataTypeNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
nested_data_type->serializeBinary(*col->getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
nested_data_type->deserializeBinary(*col->getNestedColumn(), istr);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
writeCString(NullSymbol::Escaped::name, ostr);
2016-07-11 14:54:46 +00:00
else
nested_data_type->serializeTextEscaped(*col->getNestedColumn(), row_num, ostr);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (!NullDeserializer<NullSymbol::Escaped>::execute(*col, istr))
nested_data_type->deserializeTextEscaped(*col->getNestedColumn(), istr);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
2016-07-11 14:54:46 +00:00
writeCString(NullSymbol::Quoted::name, ostr);
else
nested_data_type->serializeTextQuoted(*col->getNestedColumn(), row_num, ostr);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (!NullDeserializer<NullSymbol::Quoted>::execute(*col, istr))
nested_data_type->deserializeTextQuoted(*col->getNestedColumn(), istr);
}
2016-07-11 16:37:44 +00:00
void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
2016-07-11 16:37:44 +00:00
writeCString(NullSymbol::Quoted::name, ostr);
else
nested_data_type->serializeTextCSV(*col->getNestedColumn(), row_num, ostr);
}
2016-07-11 16:37:44 +00:00
void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (!NullDeserializer<NullSymbol::Quoted>::execute(*col, istr))
nested_data_type->deserializeTextCSV(*col->getNestedColumn(), istr, delimiter);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
2016-07-11 14:54:46 +00:00
writeCString(NullSymbol::Plain::name, ostr);
else
nested_data_type->serializeText(*col->getNestedColumn(), row_num, ostr);
}
2016-07-11 16:37:44 +00:00
void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
2016-07-11 16:37:44 +00:00
writeCString(NullSymbol::JSON::name, ostr);
else
nested_data_type->serializeTextJSON(*col->getNestedColumn(), row_num, ostr);
}
2016-07-11 16:37:44 +00:00
void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable * col = typeid_cast<ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (!NullDeserializer<NullSymbol::JSON>::execute(*col, istr))
nested_data_type->deserializeTextJSON(*col->getNestedColumn(), istr);
}
2016-07-11 14:54:46 +00:00
void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable * col = typeid_cast<const ColumnNullable *>(&column);
if (col == nullptr)
throw Exception{"Discrepancy between data type and column type", ErrorCodes::LOGICAL_ERROR};
2016-08-17 16:31:00 +00:00
if (col->isNullAt(row_num))
2016-07-11 14:54:46 +00:00
writeCString(NullSymbol::XML::name, ostr);
else
nested_data_type->serializeTextXML(*col->getNestedColumn(), row_num, ostr);
}
ColumnPtr DataTypeNullable::createColumn() const
{
2016-08-16 22:40:05 +00:00
ColumnPtr new_col = nested_data_type->createColumn();
return std::make_shared<ColumnNullable>(new_col, std::make_shared<ColumnUInt8>());
}
ColumnPtr DataTypeNullable::createConstColumn(size_t size, const Field & field) const
{
2016-08-16 22:40:05 +00:00
ColumnPtr new_col = nested_data_type->createConstColumn(size, field);
return std::make_shared<ColumnNullable>(new_col, std::make_shared<ColumnUInt8>(size));
}
}