#include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; } InterpreterInsertQuery::InterpreterInsertQuery( const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_, bool no_destination_) : query_ptr(query_ptr_) , context(context_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) , no_destination(no_destination_) { checkStackSize(); } StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) { if (query.table_function) { const auto * table_function = query.table_function->as(); const auto & factory = TableFunctionFactory::instance(); TableFunctionPtr table_function_ptr = factory.get(table_function->name, context); return table_function_ptr->execute(query.table_function, context, table_function_ptr->getName()); } query.table_id = context.resolveStorageID(query.table_id); return DatabaseCatalog::instance().getTable(query.table_id); } Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) { Block table_sample_non_materialized = table->getSampleBlockNonMaterialized(); /// If the query does not include information about columns if (!query.columns) { if (no_destination) return table->getSampleBlockWithVirtuals(); else return table_sample_non_materialized; } Block table_sample = table->getSampleBlock(); /// Form the block based on the column names from the query Block res; for (const auto & identifier : query.columns->children) { std::string current_name = identifier->getColumnName(); /// The table does not have a column with that name if (!table_sample.has(current_name)) throw Exception("No such column " + current_name + " in table " + query.table_id.getNameForLogs(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); if (!allow_materialized && !table_sample_non_materialized.has(current_name)) throw Exception("Cannot insert column " + current_name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); if (res.has(current_name)) throw Exception("Column " + current_name + " specified more than once", ErrorCodes::DUPLICATE_COLUMN); res.insert(ColumnWithTypeAndName(table_sample.getByName(current_name).type, current_name)); } return res; } BlockIO InterpreterInsertQuery::execute() { const Settings & settings = context.getSettingsRef(); auto & query = query_ptr->as(); BlockIO res; StoragePtr table = getTable(query); auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId()); auto query_sample_block = getSampleBlock(query, table); if (!query.table_function) context.checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); BlockInputStreams in_streams; BlockOutputStreams out_streams; bool is_distributed_insert_select = false; if (query.select && table->isRemote() && settings.distributed_insert_select) { // Distributed INSERT SELECT std::shared_ptr storage_src; auto & select_ = query.select->as(); auto new_query = dynamic_pointer_cast(query.clone()); if (select_.list_of_selects->children.size() == 1) { auto & select_query = select_.list_of_selects->children.at(0)->as(); JoinedTables joined_tables(Context(context), select_query); if (joined_tables.tablesCount() == 1) { storage_src = dynamic_pointer_cast(joined_tables.getLeftTableStorage()); if (storage_src) { const auto select_with_union_query = std::make_shared(); select_with_union_query->list_of_selects = std::make_shared(); auto new_select_query = dynamic_pointer_cast(select_query.clone()); select_with_union_query->list_of_selects->children.push_back(new_select_query); new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); new_query->select = select_with_union_query; } } } auto storage_dst = dynamic_pointer_cast(table); if (storage_src && storage_dst && storage_src->cluster_name == storage_dst->cluster_name) { is_distributed_insert_select = true; const auto & cluster = storage_src->getCluster(); const auto & shards_info = cluster->getShardsInfo(); String new_query_str = queryToString(new_query); for (size_t shard_index : ext::range(0, shards_info.size())) { const auto & shard_info = shards_info[shard_index]; if (shard_info.isLocal()) { InterpreterInsertQuery interpreter(new_query, context); auto block_io = interpreter.execute(); in_streams.push_back(block_io.in); } else { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); auto connections = shard_info.pool->getMany(timeouts, &settings, PoolMode::GET_ONE); if (connections.empty() || connections.front().isNull()) throw Exception( "Expected exactly one connection for shard " + toString(shard_info.shard_num), ErrorCodes::LOGICAL_ERROR); /// INSERT SELECT querie return empty block auto in_stream = std::make_shared(*connections.front(), new_query_str, Block{}, context); in_streams.push_back(in_stream); } out_streams.push_back(std::make_shared(Block())); } } } if (!is_distributed_insert_select) { size_t out_streams_size = 1; if (query.select) { /// Passing 1 as subquery_depth will disable limiting size of intermediate result. InterpreterSelectWithUnionQuery interpreter_select{ query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; if (table->supportsParallelInsert() && settings.max_insert_threads > 1) { in_streams = interpreter_select.executeWithMultipleStreams(res.pipeline); out_streams_size = std::min(size_t(settings.max_insert_threads), in_streams.size()); } else { res = interpreter_select.execute(); in_streams.emplace_back(res.in); res.in = nullptr; res.out = nullptr; } } for (size_t i = 0; i < out_streams_size; i++) { /// We create a pipeline of several streams, into which we will write data. BlockOutputStreamPtr out; /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) out = table->write(query_ptr, context); else out = std::make_shared(table, context, query_ptr, no_destination); /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash) { out = std::make_shared( out, out->getHeader(), context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); } /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) out = std::make_shared( out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); if (const auto & constraints = table->getConstraints(); !constraints.empty()) out = std::make_shared( query.table_id, out, query_sample_block, table->getConstraints(), context); auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); out_streams.emplace_back(std::move(out)); } } /// What type of query: INSERT or INSERT SELECT? if (query.select) { for (auto & in_stream : in_streams) { in_stream = std::make_shared( context, in_stream, out_streams.at(0)->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position); } Block in_header = in_streams.at(0)->getHeader(); if (in_streams.size() > 1) { for (size_t i = 1; i < in_streams.size(); ++i) assertBlocksHaveEqualStructure(in_streams[i]->getHeader(), in_header, "INSERT SELECT"); } res.in = std::make_shared(in_streams, out_streams); if (!allow_materialized) { for (const auto & column : table->getColumns()) if (column.default_desc.kind == ColumnDefaultKind::Materialized && in_header.has(column.name)) throw Exception("Cannot insert column " + column.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); } } else if (query.data && !query.has_tail) /// can execute without additional data { // res.out = std::move(out_streams.at(0)); res.in = std::make_shared(query_ptr, nullptr, query_sample_block, context, nullptr); res.in = std::make_shared(res.in, out_streams.at(0)); } else res.out = std::move(out_streams.at(0)); res.pipeline.addStorageHolder(table); return res; } StorageID InterpreterInsertQuery::getDatabaseTable() const { return query_ptr->as().table_id; } }