Minor improvements, better docs

This commit is contained in:
avogar 2022-11-14 20:05:01 +00:00
parent 098dfcff56
commit 842d25c358
6 changed files with 38 additions and 36 deletions

View File

@ -13,7 +13,7 @@ The supported formats are:
| Format | Input | Output |
|-------------------------------------------------------------------------------------------|------|--------|
| [TabSeparated](#tabseparated) | ✔ | ✔ |
| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ |
| [TabSeparatedRaon Big-Endian platformsw](#tabseparatedraw) | ✔ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [TabSeparatedRawWithNames](#tabseparatedrawwithnames) | ✔ | ✔ |
@ -1213,8 +1213,8 @@ SELECT * FROM json_each_row_nested
## BSONEachRow {#bsoneachrow}
In this format, ClickHouse formats/parses each row as a separated BSON Document.
Each column is formatted as a single BSON field with column name as a key.
In this format, ClickHouse formats/parses data as a sequence of BSON documents without any separator between them.
Each row is formatted as a single document and each column is formatted as a single BSON document field with column name as a key.
For output it uses the following correspondence between ClickHouse types and BSON types:
@ -1265,6 +1265,8 @@ For input it uses the following correspondence between BSON types and ClickHouse
Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8).
Big integers and decimals (Int128/UInt128/Int256/UInt256/Decimal128/Decimal256) can be parsed from BSON Binary value with `\x00` binary subtype. In this case this format will validate that the size of binary data equals the size of expected value.
Note: this format don't work properly on Big-Endian platforms.
### BSON format settings {#bson-format-settings}
- [output_format_bson_string_as_string](../operations/settings/settings.md#output_format_bson_string_as_string) - use BSON String type instead of Binary for String columns. Default value - `false`.

View File

@ -8,8 +8,9 @@ namespace DB
static const uint8_t BSON_DOCUMENT_END = 0x00;
using BSON_SIZE_TYPE = uint32_t;
static const BSON_SIZE_TYPE MAX_BSON_SIZE = UINT32_MAX;
static const BSONSizeT MAX_BSON_SIZE = std::numeric_limits<BSONSizeT>::max();
/// See details on https://bsonspec.org/spec.html
enum class BSONType
{
DOUBLE = 0x01,

View File

@ -52,10 +52,10 @@ BSONEachRowRowInputFormat::BSONEachRowRowInputFormat(
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, in_, std::move(params_))
, format_settings(format_settings_)
, name_map(header_.getNamesToIndexesMap())
, prev_positions(header_.columns())
, types(header_.getDataTypes())
{
name_map = getPort().getHeader().getNamesToIndexesMap();
}
inline size_t BSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index)
@ -114,7 +114,7 @@ static UInt8 readBSONType(ReadBuffer & in)
static size_t readBSONSize(ReadBuffer & in)
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
return size;
}
@ -330,7 +330,7 @@ void BSONEachRowRowInputFormat::readArray(IColumn & column, const DataTypePtr &
auto & nested_column = array_column.getData();
size_t document_start = in->count();
BSON_SIZE_TYPE document_size;
BSONSizeT document_size;
readBinary(document_size, *in);
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
{
@ -357,7 +357,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr &
size_t read_nested_columns = 0;
size_t document_start = in->count();
BSON_SIZE_TYPE document_size;
BSONSizeT document_size;
readBinary(document_size, *in);
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
{
@ -415,7 +415,7 @@ void BSONEachRowRowInputFormat::readMap(IColumn & column, const DataTypePtr & da
auto & offsets = column_map.getNestedColumn().getOffsets();
size_t document_start = in->count();
BSON_SIZE_TYPE document_size;
BSONSizeT document_size;
readBinary(document_size, *in);
while (in->count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
{
@ -631,7 +631,7 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
case BSONType::SYMBOL: [[fallthrough]];
case BSONType::STRING:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size);
break;
@ -639,14 +639,14 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
case BSONType::DOCUMENT: [[fallthrough]];
case BSONType::ARRAY:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size - sizeof(size));
break;
}
case BSONType::BINARY:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size + 1);
break;
@ -671,14 +671,14 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
}
case BSONType::DB_POINTER:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size + 12);
break;
}
case BSONType::JAVA_SCRIPT_CODE_W_SCOPE:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size - sizeof(size));
break;
@ -793,7 +793,7 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
case BSONType::SYMBOL: [[fallthrough]];
case BSONType::STRING:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
in.ignore(size);
return makeNullable(std::make_shared<DataTypeString>());
@ -842,7 +842,7 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
}
case BSONType::BINARY:
{
BSON_SIZE_TYPE size;
BSONSizeT size;
readBinary(size, in);
auto subtype = getBSONBinarySubtype(readBSONType(in));
in.ignore(size);
@ -877,7 +877,7 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
NamesAndTypesList BSONEachRowSchemaReader::getDataTypesFromBSONDocument(bool allow_to_skip_unsupported_types)
{
size_t document_start = in.count();
BSON_SIZE_TYPE document_size;
BSONSizeT document_size;
readBinary(document_size, in);
NamesAndTypesList names_and_types;
while (in.count() - document_start + sizeof(BSON_DOCUMENT_END) != document_size)
@ -923,7 +923,7 @@ fileSegmentationEngineBSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t
while (!in.eof() && memory.size() < min_bytes && number_of_rows < max_rows)
{
BSON_SIZE_TYPE document_size;
BSONSizeT document_size;
readBinary(document_size, in);
size_t old_size = memory.size();
memory.resize(old_size + document_size);

View File

@ -40,6 +40,8 @@ namespace DB
* Big integers and decimals Int128/UInt128/Int256/UInt256/Decimal128/Decimal256
* can be parsed from BSON Binary value with \x00 binary subtype. In this case
* we validate that the size of binary data equals the size of expected value.
*
* Note: this format will not work on Big-Endian platforms.
*/
class ReadBuffer;
@ -92,7 +94,7 @@ private:
DataTypes types;
size_t current_document_start;
BSON_SIZE_TYPE current_document_size;
BSONSizeT current_document_size;
};
class BSONEachRowSchemaReader : public IRowWithNamesSchemaReader

View File

@ -57,7 +57,7 @@ static void writeBSONSize(size_t size, WriteBuffer & buf)
if (size > MAX_BSON_SIZE)
throw Exception(ErrorCodes::INCORRECT_DATA, "Too large document/value size: {}. Maximum allowed size: {}.", size, MAX_BSON_SIZE);
writePODBinary<BSON_SIZE_TYPE>(BSON_SIZE_TYPE(size), buf);
writePODBinary<BSONSizeT>(BSONSizeT(size), buf);
}
template <typename Type>
@ -150,27 +150,27 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co
case TypeIndex::UInt128: [[fallthrough]];
case TypeIndex::Decimal128:
{
return size + sizeof(BSON_SIZE_TYPE) + 1 + sizeof(UInt128); // Size of a binary + binary subtype + 16 bytes of value
return size + sizeof(BSONSizeT) + 1 + sizeof(UInt128); // Size of a binary + binary subtype + 16 bytes of value
}
case TypeIndex::Int256: [[fallthrough]];
case TypeIndex::UInt256: [[fallthrough]];
case TypeIndex::Decimal256:
{
return size + sizeof(BSON_SIZE_TYPE) + 1 + sizeof(UInt256); // Size of a binary + binary subtype + 32 bytes of value
return size + sizeof(BSONSizeT) + 1 + sizeof(UInt256); // Size of a binary + binary subtype + 32 bytes of value
}
case TypeIndex::String:
{
const auto & string_column = assert_cast<const ColumnString &>(column);
return size + sizeof(BSON_SIZE_TYPE) + string_column.getDataAt(row_num).size + 1; // Size of data + data + \0 or BSON subtype (in case of BSON binary)
return size + sizeof(BSONSizeT) + string_column.getDataAt(row_num).size + 1; // Size of data + data + \0 or BSON subtype (in case of BSON binary)
}
case TypeIndex::FixedString:
{
const auto & string_column = assert_cast<const ColumnFixedString &>(column);
return size + sizeof(BSON_SIZE_TYPE) + string_column.getN() + 1; // Size of data + data + \0 or BSON subtype (in case of BSON binary)
return size + sizeof(BSONSizeT) + string_column.getN() + 1; // Size of data + data + \0 or BSON subtype (in case of BSON binary)
}
case TypeIndex::UUID:
{
return size + sizeof(BSON_SIZE_TYPE) + 1 + sizeof(UUID); // Size of data + BSON binary subtype + 16 bytes of value
return size + sizeof(BSONSizeT) + 1 + sizeof(UUID); // Size of data + BSON binary subtype + 16 bytes of value
}
case TypeIndex::LowCardinality:
{
@ -190,7 +190,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co
}
case TypeIndex::Array:
{
size += sizeof(BSON_SIZE_TYPE); // Size of a document
size += sizeof(BSONSizeT); // Size of a document
const auto & nested_type = assert_cast<const DataTypeArray *>(data_type.get())->getNestedType();
const ColumnArray & column_array = assert_cast<const ColumnArray &>(column);
@ -206,7 +206,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co
}
case TypeIndex::Tuple:
{
size += sizeof(BSON_SIZE_TYPE); // Size of a document
size += sizeof(BSONSizeT); // Size of a document
const auto * tuple_type = assert_cast<const DataTypeTuple *>(data_type.get());
const auto & nested_types = tuple_type->getElements();
@ -225,7 +225,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co
}
case TypeIndex::Map:
{
size += sizeof(BSON_SIZE_TYPE); // Size of a document
size += sizeof(BSONSizeT); // Size of a document
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
if (!isStringOrFixedString(map_type.getKeyType()))
@ -407,7 +407,7 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da
writeBSONTypeAndKeyName(BSONType::ARRAY, name, out);
size_t document_size = sizeof(BSON_SIZE_TYPE);
size_t document_size = sizeof(BSONSizeT);
for (size_t i = 0; i < array_size; ++i)
document_size += countBSONFieldSize(nested_column, nested_type, offset + i, std::to_string(i)); // Add size of each value from array
document_size += sizeof(BSON_DOCUMENT_END); // Add final \0
@ -432,7 +432,7 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da
BSONType bson_type = have_explicit_names ? BSONType::DOCUMENT : BSONType::ARRAY;
writeBSONTypeAndKeyName(bson_type, name, out);
size_t document_size = sizeof(BSON_SIZE_TYPE);
size_t document_size = sizeof(BSONSizeT);
for (size_t i = 0; i < nested_columns.size(); ++i)
{
String key_name = have_explicit_names ? toValidUTF8String(nested_names[i]) : std::to_string(i);
@ -466,7 +466,7 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da
writeBSONTypeAndKeyName(BSONType::DOCUMENT, name, out);
size_t document_size = sizeof(BSON_SIZE_TYPE);
size_t document_size = sizeof(BSONSizeT);
for (size_t i = 0; i < map_size; ++i)
{
String key = toValidUTF8String(key_column->getDataAt(offset + i).toString());
@ -493,7 +493,7 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da
void BSONEachRowRowOutputFormat::write(const Columns & columns, size_t row_num)
{
/// We should calculate and write document size before its content
size_t document_size = sizeof(BSON_SIZE_TYPE);
size_t document_size = sizeof(BSONSizeT);
for (size_t i = 0; i != columns.size(); ++i)
document_size += countBSONFieldSize(*columns[i], fields[i].type, row_num, fields[i].name);
document_size += sizeof(BSON_DOCUMENT_END);

View File

@ -40,6 +40,7 @@ namespace DB
* Named Tuple | \x03 document
* Map (with String keys) | \x03 document
*
* Note: on Big-Endian platforms this format will not work properly.
*/
class BSONEachRowRowOutputFormat final : public IRowOutputFormat
@ -54,10 +55,6 @@ private:
void write(const Columns & columns, size_t row_num) override;
void writeField(const IColumn &, const ISerialization &, size_t) override { }
/// No totals and extremes.
void consumeTotals(Chunk) override { }
void consumeExtremes(Chunk) override { }
void serializeField(const IColumn & column, const DataTypePtr & data_type, size_t row_num, const String & name);
/// Count field size in bytes that we will get after serialization in BSON format.