mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Implement COLUMNS clause
This commit is contained in:
parent
6082354d3c
commit
bb40d4729c
@ -4,6 +4,7 @@
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,9 +25,11 @@ struct AsteriskSemantic
|
||||
|
||||
static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
static void setAliases(ASTColumnsClause & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); }
|
||||
|
||||
static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
static RevertedAliasesPtr getAliases(const ASTColumnsClause & node) { return node.semantic ? node.semantic->aliases : nullptr; }
|
||||
|
||||
private:
|
||||
static std::shared_ptr<AsteriskSemanticImpl> makeSemantic(const RevertedAliasesPtr & aliases)
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <Interpreters/FindIdentifierBestTableVisitor.h>
|
||||
#include <Interpreters/ExtractFunctionDataVisitor.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -412,7 +413,7 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast
|
||||
|
||||
for (const auto & projection_column : select_query->select()->children)
|
||||
{
|
||||
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>())
|
||||
if (projection_column->as<ASTAsterisk>() || projection_column->as<ASTQualifiedAsterisk>() || projection_column->as<ASTColumnsClause>())
|
||||
{
|
||||
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
|
||||
|
||||
@ -483,8 +484,20 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
|
||||
throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto block = storage->getSampleBlock();
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
||||
if (const auto * asterisk_pattern = asterisk->as<ASTColumnsClause>())
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
{
|
||||
auto & col = block.getByPosition(idx);
|
||||
if (asterisk_pattern->isColumnMatching(col.name))
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(col.name));
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t idx = 0; idx < block.columns(); idx++)
|
||||
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
|
||||
}
|
||||
}
|
||||
}
|
||||
return projection_columns;
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <iostream>
|
||||
#include "../Parsers/ASTColumnsClause.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -166,7 +167,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
bool has_asterisk = false;
|
||||
for (const auto & child : node.children)
|
||||
{
|
||||
if (child->as<ASTAsterisk>())
|
||||
if (child->as<ASTAsterisk>() || child->as<ASTColumnsClause>())
|
||||
{
|
||||
if (tables_with_columns.empty())
|
||||
throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -207,6 +208,23 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
first_table = false;
|
||||
}
|
||||
}
|
||||
else if (const auto * asterisk_pattern = child->as<ASTColumnsClause>())
|
||||
{
|
||||
bool first_table = true;
|
||||
for (const auto & [table, table_columns] : tables_with_columns)
|
||||
{
|
||||
for (const auto & column_name : table_columns)
|
||||
{
|
||||
if (asterisk_pattern->isColumnMatching(column_name) && (first_table || !data.join_using_columns.count(column_name)))
|
||||
{
|
||||
String table_name = table.getQualifiedNamePrefix(false);
|
||||
addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern));
|
||||
}
|
||||
}
|
||||
|
||||
first_table = false;
|
||||
}
|
||||
}
|
||||
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
||||
|
22
dbms/src/Parsers/ASTColumnsClause.cpp
Normal file
22
dbms/src/Parsers/ASTColumnsClause.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "ASTColumnsClause.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTPtr ASTColumnsClause::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTColumnsClause>(*this);
|
||||
clone->cloneChildren();
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsClause::appendColumnName(WriteBuffer & ostr) const { writeString(originalPattern, ostr); }
|
||||
|
||||
void ASTColumnsClause::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << " '" << originalPattern << "'";
|
||||
}
|
||||
|
||||
}
|
51
dbms/src/Parsers/ASTColumnsClause.h
Normal file
51
dbms/src/Parsers/ASTColumnsClause.h
Normal file
@ -0,0 +1,51 @@
|
||||
#pragma once
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <re2/re2.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
}
|
||||
|
||||
struct AsteriskSemantic;
|
||||
struct AsteriskSemanticImpl;
|
||||
|
||||
|
||||
class ASTColumnsClause : public IAST
|
||||
{
|
||||
public:
|
||||
|
||||
String getID(char) const override { return "ColumnsClause"; }
|
||||
ASTPtr clone() const override;
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void setPattern(String pattern)
|
||||
{
|
||||
originalPattern = pattern;
|
||||
columnMatcher = std::make_shared<RE2>(pattern, RE2::Quiet);
|
||||
if (!columnMatcher->ok())
|
||||
throw DB::Exception("COLUMNS pattern " + originalPattern + " cannot be compiled: " + columnMatcher->error(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
|
||||
}
|
||||
bool isColumnMatching(String columnName) const
|
||||
{
|
||||
return RE2::FullMatch(columnName, *columnMatcher);
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<RE2> columnMatcher;
|
||||
String originalPattern;
|
||||
std::shared_ptr<AsteriskSemanticImpl> semantic; /// pimpl
|
||||
|
||||
friend struct AsteriskSemantic;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -29,6 +29,7 @@
|
||||
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include "ASTColumnsClause.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -1168,6 +1169,25 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
|
||||
|
||||
bool ParserColumnsClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword columns("COLUMNS");
|
||||
ParserStringLiteral regex;
|
||||
|
||||
if (!columns.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr regex_node;
|
||||
if (!regex.parse(pos, regex_node, expected))
|
||||
return false;
|
||||
|
||||
auto res = std::make_shared<ASTColumnsClause>();
|
||||
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
|
||||
res->children.push_back(regex_node);
|
||||
node = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
|
||||
{
|
||||
if (pos->type == TokenType::Asterisk)
|
||||
@ -1265,6 +1285,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
|| ParserFunction().parse(pos, node, expected)
|
||||
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|
||||
|| ParserAsterisk().parse(pos, node, expected)
|
||||
|| ParserColumnsClause().parse(pos, node, expected)
|
||||
|| ParserCompoundIdentifier().parse(pos, node, expected)
|
||||
|| ParserSubstitution().parse(pos, node, expected);
|
||||
}
|
||||
|
@ -56,7 +56,6 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
/// Just *
|
||||
class ParserAsterisk : public IParserBase
|
||||
{
|
||||
@ -65,7 +64,6 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
/** Something like t.* or db.table.*
|
||||
*/
|
||||
class ParserQualifiedAsterisk : public IParserBase
|
||||
@ -75,6 +73,14 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/** COLUMNS '<regular expression>'
|
||||
*/
|
||||
class ParserColumnsClause : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "COLUMNS clause"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
/** A function, for example, f(x, y + 1, g(z)).
|
||||
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
|
||||
|
@ -0,0 +1,2 @@
|
||||
100 10
|
||||
120 8
|
3
dbms/tests/queries/0_stateless/00969_columns_clause.sql
Normal file
3
dbms/tests/queries/0_stateless/00969_columns_clause.sql
Normal file
@ -0,0 +1,3 @@
|
||||
CREATE TABLE IF NOT EXISTS ColumnsClauseTest (product_price Int64, product_weight Int16, amount Int64) Engine=TinyLog;
|
||||
INSERT INTO ColumnsClauseTest VALUES (100, 10, 324), (120, 8, 23);
|
||||
SELECT COLUMNS 'product.*' from ColumnsClauseTest ORDER BY product_price;
|
Loading…
Reference in New Issue
Block a user