mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
insert defaults (first working version)
This commit is contained in:
parent
2e4c2328af
commit
bac1f77620
@ -865,11 +865,12 @@ private:
|
||||
|
||||
/// Receive description of table structure.
|
||||
Block sample;
|
||||
if (receiveSampleBlock(sample))
|
||||
TableMetaInfo table_meta(parsed_insert_query.database, parsed_insert_query.table);
|
||||
if (receiveSampleBlock(sample, table_meta))
|
||||
{
|
||||
/// If structure was received (thus, server has not thrown an exception),
|
||||
/// send our data with that structure.
|
||||
sendData(sample);
|
||||
sendData(sample, table_meta);
|
||||
receivePacket();
|
||||
}
|
||||
}
|
||||
@ -907,7 +908,7 @@ private:
|
||||
}
|
||||
|
||||
|
||||
void sendData(Block & sample)
|
||||
void sendData(Block & sample, const TableMetaInfo & table_meta)
|
||||
{
|
||||
/// If INSERT data must be sent.
|
||||
const ASTInsertQuery * parsed_insert_query = typeid_cast<const ASTInsertQuery *>(&*parsed_query);
|
||||
@ -918,35 +919,28 @@ private:
|
||||
{
|
||||
/// Send data contained in the query.
|
||||
ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data);
|
||||
sendDataFrom(data_in, sample);
|
||||
sendDataFrom(data_in, sample, table_meta);
|
||||
}
|
||||
else if (!is_interactive)
|
||||
{
|
||||
/// Send data read from stdin.
|
||||
sendDataFrom(std_in, sample);
|
||||
sendDataFrom(std_in, sample, table_meta);
|
||||
}
|
||||
else
|
||||
throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT);
|
||||
}
|
||||
|
||||
|
||||
void sendDataFrom(ReadBuffer & buf, Block & sample)
|
||||
void sendDataFrom(ReadBuffer & buf, Block & sample, const TableMetaInfo & table_meta)
|
||||
{
|
||||
String current_format = insert_format;
|
||||
ColumnDefaults column_defaults;
|
||||
const ColumnDefaults & column_defaults = table_meta.column_defaults;
|
||||
|
||||
/// Data format can be specified in the INSERT query.
|
||||
if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*parsed_query))
|
||||
{
|
||||
if (!insert->format.empty())
|
||||
current_format = insert->format;
|
||||
|
||||
if (context.isTableExist(insert->database, insert->table))
|
||||
{
|
||||
StoragePtr table = context.getTable(insert->database, insert->table);
|
||||
if (table)
|
||||
column_defaults = table->getColumns().defaults;
|
||||
}
|
||||
}
|
||||
|
||||
BlockInputStreamPtr block_input = context.getInputFormat(
|
||||
@ -1071,7 +1065,7 @@ private:
|
||||
|
||||
|
||||
/// Receive the block that serves as an example of the structure of table where data will be inserted.
|
||||
bool receiveSampleBlock(Block & out)
|
||||
bool receiveSampleBlock(Block & out, TableMetaInfo & table_meta)
|
||||
{
|
||||
Connection::Packet packet = connection->receivePacket();
|
||||
|
||||
@ -1087,8 +1081,8 @@ private:
|
||||
return false;
|
||||
|
||||
case Protocol::Server::CapnProto:
|
||||
loadContextBlock(packet.block, context);
|
||||
return receiveSampleBlock(out);
|
||||
loadTableMetaInfo(packet.block, table_meta);
|
||||
return receiveSampleBlock(out, table_meta);
|
||||
|
||||
default:
|
||||
throw NetException("Unexpected packet from server (expected Data, got "
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ExpressionElementParser.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Core/Block.h>
|
||||
@ -109,7 +109,7 @@ namespace DB
|
||||
return proto_column;
|
||||
}
|
||||
|
||||
static void loadContext(const ColumnWithTypeAndName & proto_column, Context & context)
|
||||
static void loadTableMetaInfo(const ColumnWithTypeAndName & proto_column, TableMetaInfo & table_meta)
|
||||
{
|
||||
StringRef plain_data = proto_column.column->getDataAt(0);
|
||||
size_t data_size = proto_column.column->byteSize();
|
||||
@ -120,23 +120,15 @@ namespace DB
|
||||
for (auto proto_database : proto_context.getDatabases())
|
||||
{
|
||||
const String & database_name = proto_database.getName().cStr();
|
||||
if (!context.isDatabaseExist(database_name))
|
||||
{
|
||||
// TODO
|
||||
}
|
||||
if (database_name != table_meta.database)
|
||||
continue;
|
||||
|
||||
for (auto proto_table : proto_database.getTables())
|
||||
{
|
||||
String table_name = proto_table.getName().cStr();
|
||||
if (!context.isTableExist(database_name, table_name))
|
||||
{
|
||||
// TODO
|
||||
}
|
||||
if (table_name != table_meta.table)
|
||||
continue;
|
||||
|
||||
StoragePtr table = context.tryGetTable(database_name, table_name);
|
||||
// TODO: throw on fail
|
||||
|
||||
ColumnsDescription column_description;
|
||||
for (auto column : proto_table.getColumns())
|
||||
{
|
||||
String column_name = column.getName().cStr();
|
||||
@ -144,10 +136,8 @@ namespace DB
|
||||
ColumnDefaultKind expression_kind = static_cast<ColumnDefaultKind>(column.getDefault().getKind());
|
||||
|
||||
ASTPtr ast = parseQuery(parser, expression, expression.size());
|
||||
column_description.defaults[column_name] = ColumnDefault{expression_kind, ast};
|
||||
table_meta.column_defaults.emplace(column_name, ColumnDefault{expression_kind, ast});
|
||||
}
|
||||
|
||||
table->setColumns(column_description);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -164,9 +154,9 @@ namespace DB
|
||||
return block;
|
||||
}
|
||||
|
||||
void loadContextBlock(const Block & block, Context & context)
|
||||
void loadTableMetaInfo(const Block & block, TableMetaInfo & table_meta)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = block.getByName(contextColumnName());
|
||||
loadContext(column, context);
|
||||
loadTableMetaInfo(column, table_meta);
|
||||
}
|
||||
}
|
||||
|
@ -1,11 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/ColumnDefault.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class Block;
|
||||
|
||||
struct TableMetaInfo
|
||||
{
|
||||
TableMetaInfo(const String & database_, const String & table_)
|
||||
: database(database_), table(table_)
|
||||
{}
|
||||
|
||||
const String & database;
|
||||
const String & table;
|
||||
ColumnDefaults column_defaults;
|
||||
};
|
||||
|
||||
Block storeContextBlock(Context & context);
|
||||
void loadContextBlock(const Block & block, Context & context);
|
||||
void loadTableMetaInfo(const Block & block, TableMetaInfo & table_meta);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user