mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 04:22:03 +00:00
Add BSON input/output format
This commit is contained in:
parent
66f2ea5ebb
commit
2fff4887ac
BIN
programs/server/coordination/logs/changelog_1_100000.bin.zstd
Normal file
BIN
programs/server/coordination/logs/changelog_1_100000.bin.zstd
Normal file
Binary file not shown.
BIN
programs/server/coordination/state
Normal file
BIN
programs/server/coordination/state
Normal file
Binary file not shown.
1
programs/server/uuid
Normal file
1
programs/server/uuid
Normal file
@ -0,0 +1 @@
|
||||
2e83a9e9-1c5d-4ae0-84d8-cbebb06b63f5
|
@ -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);
|
||||
|
192
src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp
Normal file
192
src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp
Normal 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);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
}
|
77
src/Processors/Formats/Impl/BSONEachRowRowInputFormat.h
Normal file
77
src/Processors/Formats/Impl/BSONEachRowRowInputFormat.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
217
src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp
Normal file
217
src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp
Normal 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");
|
||||
}
|
||||
|
||||
}
|
39
src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.h
Normal file
39
src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
212
src/Processors/Formats/Impl/BSONUtils.cpp
Normal file
212
src/Processors/Formats/Impl/BSONUtils.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
54
src/Processors/Formats/Impl/BSONUtils.h
Normal file
54
src/Processors/Formats/Impl/BSONUtils.h
Normal 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
|
||||
);
|
||||
}
|
||||
|
||||
}
|
@ -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"}
|
10
tests/queries/0_stateless/02453_bson_data_types.sh
Executable file
10
tests/queries/0_stateless/02453_bson_data_types.sh
Executable 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"
|
BIN
tests/queries/0_stateless/test.bson
Normal file
BIN
tests/queries/0_stateless/test.bson
Normal file
Binary file not shown.
Loading…
Reference in New Issue
Block a user