Add BSON input/output format

This commit is contained in:
Mark Polokhov 2022-09-24 16:36:24 +00:00 committed by avogar
parent 66f2ea5ebb
commit 2fff4887ac
13 changed files with 807 additions and 0 deletions

Binary file not shown.

1
programs/server/uuid Normal file
View File

@ -0,0 +1 @@
2e83a9e9-1c5d-4ae0-84d8-cbebb06b63f5

View File

@ -42,6 +42,8 @@ void registerInputFormatJSONColumns(FormatFactory & factory);
void registerOutputFormatJSONColumns(FormatFactory & factory);
void registerInputFormatJSONCompactColumns(FormatFactory & factory);
void registerOutputFormatJSONCompactColumns(FormatFactory & factory);
void registerInputFormatBSONEachRow(FormatFactory & factory);
void registerOutputFormatBSONEachRow(FormatFactory & factory);
void registerInputFormatProtobuf(FormatFactory & factory);
void registerOutputFormatProtobuf(FormatFactory & factory);
void registerInputFormatProtobufList(FormatFactory & factory);
@ -168,6 +170,8 @@ void registerFormats()
registerOutputFormatJSONColumns(factory);
registerInputFormatJSONCompactColumns(factory);
registerOutputFormatJSONCompactColumns(factory);
registerInputFormatBSONEachRow(factory);
registerOutputFormatBSONEachRow(factory);
registerInputFormatProtobuf(factory);
registerOutputFormatProtobufList(factory);
registerInputFormatProtobufList(factory);

View File

@ -0,0 +1,192 @@
#include <IO/ReadBufferFromString.h>
#include <Processors/Formats/Impl/BSONEachRowRowInputFormat.h>
#include <Processors/Formats/Impl/BSONUtils.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int CANNOT_READ_ALL_DATA;
extern const int LOGICAL_ERROR;
}
namespace
{
enum
{
UNKNOWN_FIELD = size_t(-1),
NESTED_FIELD = size_t(-2)
};
}
BSONEachRowRowInputFormat::BSONEachRowRowInputFormat(
ReadBuffer & in_,
const Block & header_,
Params params_,
const FormatSettings & format_settings_,
bool yield_strings_)
: IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()), yield_strings(yield_strings_)
{
size_t num_columns = getPort().getHeader().columns();
for (size_t i = 0; i < num_columns; ++i)
{
const String & column_name = columnName(i);
name_map[column_name] = i;
}
prev_positions.resize(num_columns);
}
const String & BSONEachRowRowInputFormat::columnName(size_t i) const
{
return getPort().getHeader().getByPosition(i).name;
}
inline size_t BSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index)
{
/// Optimization by caching the order of fields (which is almost always the same)
/// and a quick check to match the next expected field, instead of searching the hash table.
if (prev_positions.size() > key_index
&& prev_positions[key_index]
&& name == prev_positions[key_index]->getKey())
{
return prev_positions[key_index]->getMapped();
}
else
{
auto * it = name_map.find(name);
if (it)
{
if (key_index < prev_positions.size())
prev_positions[key_index] = it;
return it->getMapped();
}
else
return UNKNOWN_FIELD;
}
}
StringRef BSONEachRowRowInputFormat::readColumnName(ReadBuffer & buf)
{
// This is just an optimization: try to avoid copying the name into current_column_name
if (nested_prefix_length == 0 && !buf.eof() && buf.position() + 1 < buf.buffer().end())
{
char * next_pos = find_first_symbols<0>(buf.position() + 1, buf.buffer().end());
if (next_pos != buf.buffer().end())
{
/// The most likely option is that there is no escape sequence in the key name, and the entire name is placed in the buffer.
StringRef res(buf.position(), next_pos - buf.position());
buf.position() = next_pos + 1;
return res;
}
}
return current_column_name;
}
void BSONEachRowRowInputFormat::readBSONObject(MutableColumns & columns)
{
UInt32 obj_size;
{
union {
char buf[4];
UInt32 size;
} read_value;
in->read(read_value.buf, 4);
obj_size = read_value.size;
}
UInt32 already_read = BSON_32;
for (size_t key_index = 0; already_read + 1 < obj_size; ++key_index) {
char type;
in->read(type);
StringRef name_ref = readColumnName(*in);
const size_t column_index = columnIndex(name_ref, key_index);
if (unlikely(ssize_t(column_index) < 0))
{
/// name_ref may point directly to the input buffer
/// and input buffer may be filled with new data on next read
/// If we want to use name_ref after another reads from buffer, we must copy it to temporary string.
current_column_name.assign(name_ref.data, name_ref.size);
name_ref = StringRef(current_column_name);
// if (column_index == UNKNOWN_FIELD)
// skipUnknownField(name_ref);
// else if (column_index == NESTED_FIELD)
// readNestedData(name_ref.toString(), columns);
// else
// throw Exception("Logical error: illegal value of column_index", ErrorCodes::LOGICAL_ERROR);
}
if (seen_columns[column_index])
throw Exception("Duplicate field found while parsing BSONEachRow format: " + columnName(column_index), ErrorCodes::INCORRECT_DATA);
seen_columns[column_index] = true;
read_columns[column_index] = BSONUtils::readField(*in, *columns[column_index], type, already_read);
already_read += sizeof(type) + name_ref.size + 1;
}
char eof_check;
in->read(eof_check);
++already_read;
if (eof_check != 0) throw Exception("Wrong BSON syntax", ErrorCodes::INCORRECT_DATA);
}
bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
{
size_t num_columns = columns.size();
read_columns.assign(num_columns, false);
seen_columns.assign(num_columns, false);
nested_prefix_length = 0;
if (in->eof() || in->buffer().size() == 0)
return false;
// LOG_DEBUG(&Poco::Logger::get("<readRow>"), "Reading new object");
readBSONObject(columns);
const auto & header = getPort().getHeader();
/// Fill non-visited columns with the default values.
for (size_t i = 0; i < num_columns; ++i)
if (!seen_columns[i])
header.getByPosition(i).type->insertDefaultInto(*columns[i]);
if (format_settings.defaults_for_omitted_fields)
ext.read_columns = read_columns;
else
ext.read_columns.assign(read_columns.size(), true);
return true;
}
void registerInputFormatBSONEachRow(FormatFactory & factory)
{
factory.registerInputFormat("BSONEachRow", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<BSONEachRowRowInputFormat>(buf, sample, std::move(params), settings, false);
}
);
}
}

View File

@ -0,0 +1,77 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
namespace DB
{
class ReadBuffer;
class BSONEachRowRowInputFormat final : public IRowInputFormat
{
public:
BSONEachRowRowInputFormat(
ReadBuffer & in_,
const Block & header_,
Params params_,
const FormatSettings & format_settings_,
bool yield_strings_);
String getName() const override { return "BSONEachRowRowInputFormat"; }
void resetParser() override {}
private:
void readPrefix() override {}
void readSuffix() override {}
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override {}
const String & columnName(size_t i) const;
size_t columnIndex(const StringRef & name, size_t key_index);
StringRef readColumnName(ReadBuffer & buf);
void readField(size_t index, MutableColumns & columns);
void readBSONObject(MutableColumns & columns);
const FormatSettings format_settings;
/// Buffer for the read from the stream field name. Used when you have to copy it.
/// Also, if processing of Nested data is in progress, it holds the common prefix
/// of the nested column names (so that appending the field name to it produces
/// the full column name)
String current_column_name;
/// If processing Nested data, holds the length of the common prefix
/// of the names of related nested columns. For example, for a table
/// created as follows
/// CREATE TABLE t (n Nested (i Int32, s String))
/// the nested column names are 'n.i' and 'n.s' and the nested prefix is 'n.'
size_t nested_prefix_length = 0;
/// Set of columns for which the values were read. The rest will be filled with default values.
std::vector<UInt8> read_columns;
/// Set of columns which already met in row. Exception is thrown if there are more than one column with the same name.
std::vector<UInt8> seen_columns;
/// These sets may be different, because if null_as_default=1 read_columns[i] will be false and seen_columns[i] will be true
/// for row like {..., "non-nullable column name" : null, ...}
/// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map.
using NameMap = HashMap<StringRef, size_t, StringRefHash>;
NameMap name_map;
/// Cached search results for previous row (keyed as index in JSON object) - used as a hint.
std::vector<NameMap::LookupResult> prev_positions;
/// This flag is needed to know if data is in square brackets.
bool data_in_square_brackets = false;
bool allow_new_rows = true;
bool yield_strings;
};
}

View File

@ -0,0 +1,217 @@
#include <Processors/Formats/Impl/BSONEachRowRowOutputFormat.h>
#include <Processors/Formats/Impl/BSONUtils.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeDateTime64.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int CANNOT_READ_ALL_DATA;
extern const int LOGICAL_ERROR;
}
BSONEachRowRowOutputFormat::BSONEachRowRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSettings & settings_)
: IRowOutputFormat(header_, out_, params_),
settings(settings_)
{
const auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
fields.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
WriteBufferFromString buf(fields[i]);
writeString(sample.getByPosition(i).name.c_str(), buf);
}
}
void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, size_t row_num, const String & name) {
switch(column.getDataType())
{
case TypeIndex::Float64:
{
writeChar(0x01, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<Float64>(assert_cast<const ColumnFloat64 &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_64;
break;
}
case TypeIndex::String:
{
writeChar(0x02, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<UInt32>(assert_cast<const ColumnString &>(column).getDataAt(row_num).size + BSON_ZERO, out);
writeString(assert_cast<const ColumnString &>(column).getDataAt(row_num), out);
writeChar(0x00, out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_32 + assert_cast<const ColumnString &>(column).getDataAt(row_num).size + BSON_ZERO;
break;
}
// case TypeIndex::EMBDOCUMENT - Not supported
case TypeIndex::Array:
{
writeChar(0x04, out);
writeString(name, out);
writeChar(0x00, out);
BufferBase::Position array_begin;
array_begin = out.position();
writePODBinary<UInt32>(0, out);
UInt32 array_size = BSON_32;
const ColumnArray & column_array = assert_cast<const ColumnArray &>(column);
const IColumn & nested_column = column_array.getData();
const ColumnArray::Offsets & offsets = column_array.getOffsets();
size_t offset = offsets[row_num - 1];
size_t size = offsets[row_num] - offset;
for (UInt32 i = 0; i < size; ++i) {
auto size_before_field = obj_size;
serializeField(nested_column, offset + i, std::to_string(i));
array_size += obj_size - size_before_field;
}
writeChar(0x00, out);
++array_size;
BufferBase::Position array_end = out.position();
out.position() = array_begin;
writePODBinary<UInt32>(array_size, out);
out.position() = array_end;
obj_size += BSON_TYPE + BSON_32 + name.size() + BSON_ZERO + BSON_ZERO;
break;
}
// case TypeIndex::Binary: - Not supported
// case TypeIndex::UNDEFINED - Depricated
case TypeIndex::UUID:
{
writeChar(0x07, out);
writeString(name, out);
writeChar(0x00, out);
writeString(assert_cast<const ColumnUUID &>(column).getDataAt(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_128;
break;
}
case TypeIndex::UInt8:
{
writeChar(0x08, out);
writeString(name, out);
writeChar(0x00, out);
writeString(assert_cast<const ColumnUInt8 &>(column).getDataAt(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_8;
break;
}
case TypeIndex::DateTime64:
{
writeChar(0x09, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<UInt64>(assert_cast<const DataTypeDateTime64::ColumnType &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_64;
break;
}
case TypeIndex::Nullable:
{
writeChar(0x0A, out);
writeString(name, out);
writeChar(0x00, out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO;
break;
}
// case TypeIndex::Regexp - Not supported
// case TypeIndex::DBPointer - Depricated
// case TypeIndex::JSCODE - Not supported
// case TypeIndex::Symbol - Depricated
// case TypeIndex::JSCODE_w_scope - Depricated
case TypeIndex::Int32:
{
writeChar(0x10, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<Int32>(assert_cast<const ColumnInt32 &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_32;
break;
}
case TypeIndex::UInt64:
{
writeChar(0x11, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<UInt64>(assert_cast<const ColumnUInt64 &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_64;
break;
}
case TypeIndex::Int64:
{
writeChar(0x12, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<Int64>(assert_cast<const ColumnInt64 &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_64;
break;
}
case TypeIndex::Decimal128:
{
writeChar(0x13, out);
writeString(name, out);
writeChar(0x00, out);
writePODBinary<Decimal128>(assert_cast<const ColumnDecimal<Decimal128> &>(column).getElement(row_num), out);
obj_size += BSON_TYPE + name.size() + BSON_ZERO + BSON_128;
break;
}
// case TypeIndex::MinKey - Not supported
// case TypeIndex::MaxKey - Not supported
default:
throw Exception("Unrecognized Type", ErrorCodes::INCORRECT_DATA);
}
++row_num;
}
void BSONEachRowRowOutputFormat::write(const Columns & columns, size_t row_num)
{
BufferBase::Position buf_begin;
buf_begin = out.position();
writePODBinary<UInt32>(0, out);
obj_size += BSON_32;
size_t columns_size = columns.size();
for (size_t i = 0; i < columns_size; ++i) {
BSONEachRowRowOutputFormat::serializeField(*columns[i], row_num, fields[i]);
}
writeChar(0x00, out);
++obj_size;
BufferBase::Position buf_end = out.position();
out.position() = buf_begin;
writePODBinary<UInt32>(obj_size, out);
out.position() = buf_end;
}
void registerOutputFormatBSONEachRow(FormatFactory & factory) {
factory.registerOutputFormat("BSONEachRow", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & _format_settings)
{
return std::make_shared<BSONEachRowRowOutputFormat>(buf, sample, params, _format_settings);
}
);
factory.markOutputFormatSupportsParallelFormatting("BSONEachRow");
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Core/Block.h>
#include <IO/WriteBuffer.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class BSONEachRowRowOutputFormat final : public IRowOutputFormat
{
public:
BSONEachRowRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSettings & settings_);
String getName() const override { return "BSONEachRowRowOutputFormat"; }
protected:
void write(const Columns & columns, size_t row_num) override;
void writeField(const IColumn &, const ISerialization &, size_t) override {}
void serializeField(const IColumn & column, size_t row_num, const String & name);
/// No totals and extremes.
void consumeTotals(Chunk) override {}
void consumeExtremes(Chunk) override {}
private:
Names fields;
UInt32 obj_size = 0;
FormatSettings settings;
};
}

View File

@ -0,0 +1,212 @@
#include <Processors/Formats/Impl/BSONUtils.h>
#include <Common/assert_cast.h>
#include <IO/ReadHelpers.h>
#include <Formats/FormatFactory.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <base/strong_typedef.h>
namespace DB
{
namespace BSONUtils
{
bool readField(
ReadBuffer & in,
IColumn & column,
char type,
UInt32 & already_read
)
{
try
{
switch (type)
{
case BSONDataTypeIndex::DOUBLE:
{
union {
char buf[BSON_64];
Float64 value;
} read_value;
in.read(read_value.buf, BSON_64);
assert_cast<ColumnFloat64 &>(column).insertValue(read_value.value);
already_read += BSON_64;
return true;
}
case BSONDataTypeIndex::STRING:
{
union {
char buf[BSON_32];
UInt32 size;
} read_value;
in.read(read_value.buf, BSON_32);
already_read += BSON_32;
if (read_value.size != 0) {
String str;
str.resize(read_value.size - 1);
for (size_t i = 0; i + 1 < read_value.size; ++i)
{
in.read(str[i]);
}
assert_cast<ColumnString &>(column).insertData(str.c_str(), read_value.size - 1);
already_read += read_value.size;
char str_end;
in.read(str_end);
if (str_end != 0) throw Exception("Wrong BSON syntax", ErrorCodes::INCORRECT_DATA);
}
return true;
}
// case BSONDataTypeIndex::EMB_DOCUMENT: - Not supported
case BSONDataTypeIndex::ARRAY:
{
if (in.eof())
return false;
auto pre_array_size = already_read;
UInt32 arr_size;
{
union {
char buf[BSON_32];
UInt32 size;
} read_value;
in.read(read_value.buf, BSON_32);
arr_size = read_value.size;
}
already_read += BSON_32 + BSON_TYPE;
char nested_type;
in.read(nested_type);
ColumnArray & column_array = assert_cast<ColumnArray &>(column);
ColumnArray::Offsets & offsets = column_array.getOffsets();
IColumn & nested_column = column_array.getData();
size_t key_index = 0;
for (; already_read - pre_array_size < arr_size; ++key_index) {
char name_byte = 1;
while(name_byte != 0) {
in.read(name_byte);
++already_read;
}
BSONUtils::readField(in, nested_column, nested_type, already_read);
if (already_read - pre_array_size + 1 >= arr_size) {
in.read(nested_type);
++already_read;
}
}
offsets.push_back(offsets.back() + key_index);
return true;
}
// case BSONDataTypeIndex::BINARY: - Not supported
// case BSONDataTypeIndex::UNDEFINED: - Depricated
case BSONDataTypeIndex::OID:
{
union {
char buf[BSON_128];
UInt128 value;
} read_value;
in.read(read_value.buf, BSON_128);
assert_cast<ColumnUUID &>(column).insertValue(StrongTypedef<UInt128, struct UUIDTag>(read_value.value));
already_read += BSON_128;
return true;
}
case BSONDataTypeIndex::BOOL:
{
union {
char f;
UInt8 value;
} read_value;
in.read(read_value.f);
assert_cast<ColumnUInt8 &>(column).insertValue(read_value.value);
already_read += BSON_8;
return true;
}
case BSONDataTypeIndex::DATETIME:
{
union {
char buf[BSON_64];
Int64 value;
} read_value;
in.read(read_value.buf, BSON_64);
assert_cast<DataTypeDateTime64::ColumnType &>(column).insertValue(read_value.value);
already_read += BSON_64;
return true;
}
case BSONDataTypeIndex::NULLVALUE:
{
assert_cast<ColumnNullable &>(column).insertData(nullptr, 0);
return true;
}
// case BSONDataTypeIndex::REGEXP: - Not supported
// case BSONDataTypeIndex::DBPOINTER: - Depricated
// case BSONDataTypeIndex::JAVASCRIPT_CODE: - Not supported
// case BSONDataTypeIndex::SYMBOL: - Depricated
// case BSONDataTypeIndex::JAVASCRIPT_CODE_WITH_SCOPE: - Depricated
case BSONDataTypeIndex::INT32:
{
union {
char buf[BSON_32];
Int32 value;
} read_value;
in.read(read_value.buf, BSON_32);
assert_cast<ColumnInt32 &>(column).insertValue(read_value.value);
already_read += BSON_32;
return true;
}
case BSONDataTypeIndex::UINT64:
{
union {
char buf[BSON_64];
UInt64 value;
} read_value;
in.read(read_value.buf, BSON_64);
assert_cast<ColumnUInt64 &>(column).insertValue(read_value.value);
already_read += BSON_64;
return true;
}
case BSONDataTypeIndex::INT64:
{
union {
char buf[BSON_64];
Int64 value;
} read_value;
in.read(read_value.buf, BSON_64);
assert_cast<ColumnInt64 &>(column).insertValue(read_value.value);
already_read += BSON_64;
return true;
}
case BSONDataTypeIndex::DECIMAL128:
{
union {
char buf[BSON_128];
Decimal128 value;
} read_value;
in.read(read_value.buf, BSON_128);
assert_cast<ColumnDecimal<Decimal128> &>(column).insertValue(read_value.value);
already_read += BSON_128;
return true;
}
// case BSONDataTypeIndex::MIN_KEY: - Not supported
// case BSONDataTypeIndex::MAX_KEY: - Not supported
}
}
catch (Exception &)
{
throw;
}
return false;
}
}
}

View File

@ -0,0 +1,54 @@
#include <IO/ReadBuffer.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
namespace DB
{
enum BSONDataTypeIndex
{
DOUBLE = 0x01,
STRING = 0x02,
EMB_DOCUMENT = 0x03,
ARRAY = 0x04,
BINARY = 0x05,
UNDEFINED = 0x06,
OID = 0x07,
BOOL = 0x08,
DATETIME = 0x09,
NULLVALUE = 0x0A,
REGEXP = 0x0B,
DBPOINTER = 0x0C,
JAVASCRIPT_CODE = 0x0D,
SYMBOL = 0x0E,
JAVASCRIPT_CODE_WITH_SCOPE = 0x0F,
INT32 = 0x10,
TIMESTAMP = 0x11, // The same as UINT64
UINT64 = 0x11, // The same as TIMESTAMP
INT64 = 0x12,
DECIMAL128 = 0x13,
MIN_KEY = 0xFF,
MAX_KEY = 0x7F,
};
enum {
BSON_TYPE = 1,
BSON_ZERO = 1,
BSON_8 = 1,
BSON_32 = 4,
BSON_64 = 8,
BSON_128 = 16,
};
namespace BSONUtils
{
bool readField(
ReadBuffer & in,
IColumn & column,
char type,
UInt32 & already_read
);
}
}

View File

@ -0,0 +1 @@
{"fl":50.36,"str":"hello","date":"2282-12-31 00:00:00.000","num1":42,"num2":"43","num3":"44","arr":[42,13],"nothing":null,"bool":1,"id":"61f0c404-5cb3-11e7-907b-a6006ad3dba0"}

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
# Tags: no-fasttest, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "select toFloat64(50.36) as fl, toString('hello') as str, toDateTime64(247000000000000000, 3) as date, toInt32(42) as num1, toInt64(43) as num2, toUInt64(44) as num3, [toInt32(42), toInt32(13)] as arr, Null as nothing, True as bool, toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0') as id format BSONEachRow" > /home/ubuntu/ClickHouse/programs/server/user_files/test.bson
$CLICKHOUSE_CLIENT -q "select * from file(test.bson, BSONEachRow, 'fl Float64, str String, date DateTime64, num1 Int32, num2 Int64, num3 UInt64, arr Array(Int32), nothing Nullable(Nothing), bool UInt8, id UUID') format JSONEachRow"

Binary file not shown.