Merge pull request #5630 from yandex/sanych73-prepared_statements

Fixes for prepared statements.
This commit is contained in:
alexey-milovidov 2019-06-30 19:11:39 +03:00 committed by GitHub
commit 9ea05b2401
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
82 changed files with 891 additions and 324 deletions

View File

@ -59,6 +59,7 @@
#include <Parsers/parseQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Client/Connection.h>
#include <Common/InterruptListener.h>
#include <Functions/registerFunctions.h>
@ -202,6 +203,9 @@ private:
/// External tables info.
std::list<ExternalTable> external_tables;
/// Dictionary with query parameters for prepared statements.
NameToNameMap query_parameters;
ConnectionParameters connection_parameters;
void initialize(Poco::Util::Application & self)
@ -795,7 +799,6 @@ private:
/// Some parts of a query (result output and formatting) are executed client-side.
/// Thus we need to parse the query.
parsed_query = parsed_query_;
if (!parsed_query)
{
const char * begin = query.data();
@ -900,6 +903,16 @@ private:
/// Process the query that doesn't require transferring data blocks to the server.
void processOrdinaryQuery()
{
/// We will always rewrite query (even if there are no query_parameters) because it will help to find errors in query formatter.
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
ReplaceQueryParameterVisitor visitor(query_parameters);
visitor.visit(parsed_query);
/// Get new query after substitutions. Note that it cannot be done for INSERT query with embedded data.
query = serializeAST(*parsed_query);
}
connection->sendQuery(connection_parameters.timeouts, query, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true);
sendExternalTables();
receiveResult();
@ -1548,7 +1561,8 @@ public:
/** We allow different groups of arguments:
* - common arguments;
* - arguments for any number of external tables each in form "--external args...",
* where possible args are file, name, format, structure, types.
* where possible args are file, name, format, structure, types;
* - param arguments for prepared statements.
* Split these groups before processing.
*/
using Arguments = std::vector<std::string>;
@ -1597,7 +1611,31 @@ public:
else
{
in_external_group = false;
common_arguments.emplace_back(arg);
/// Parameter arg after underline.
if (startsWith(arg, "--param_"))
{
const char * param_continuation = arg + strlen("--param_");
const char * equal_pos = strchr(param_continuation, '=');
if (equal_pos == param_continuation)
throw Exception("Parameter name cannot be empty", ErrorCodes::BAD_ARGUMENTS);
if (equal_pos)
{
/// param_name=value
query_parameters.emplace(String(param_continuation, equal_pos), String(equal_pos + 1));
}
else
{
/// param_name value
++arg_num;
arg = argv[arg_num];
query_parameters.emplace(String(param_continuation), String(arg));
}
}
else
common_arguments.emplace_back(arg);
}
}
@ -1672,6 +1710,7 @@ public:
("structure", po::value<std::string>(), "structure")
("types", po::value<std::string>(), "types")
;
/// Parse main commandline options.
po::parsed_options parsed = po::command_line_parser(common_arguments).options(main_description).run();
po::variables_map options;
@ -1696,6 +1735,7 @@ public:
{
std::cout << main_description << "\n";
std::cout << external_description << "\n";
std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n";
exit(0);
}

View File

@ -475,9 +475,9 @@ void HTTPHandler::processQuery(
settings.readonly = 2;
}
bool isExternalData = startsWith(request.getContentType().data(), "multipart/form-data");
bool has_external_data = startsWith(request.getContentType().data(), "multipart/form-data");
if (isExternalData)
if (has_external_data)
{
/// Skip unneeded parameters to avoid confusing them later with context settings or query parameters.
reserved_param_suffixes.reserve(3);
@ -501,6 +501,12 @@ void HTTPHandler::processQuery(
else if (param_could_be_skipped(key))
{
}
else if (startsWith(key, "param_"))
{
/// Save name and values of substitution in dictionary.
const String parameter_name = key.substr(strlen("param_"));
context.setQueryParameter(parameter_name, value);
}
else
{
/// All other query parameters are treated as settings.
@ -516,7 +522,7 @@ void HTTPHandler::processQuery(
std::string full_query;
/// Support for "external data for query processing".
if (isExternalData)
if (has_external_data)
{
ExternalTablesHandler handler(context, params);
params.load(request, istr, handler);

View File

@ -430,6 +430,8 @@ namespace ErrorCodes
extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES = 453;
extern const int OPENSSL_ERROR = 454;
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY = 455;
extern const int UNKNOWN_QUERY_PARAMETER = 456;
extern const int BAD_QUERY_PARAMETER = 457;
extern const int CANNOT_UNLINK = 458;
extern const int KEEPER_EXCEPTION = 999;

View File

@ -30,6 +30,7 @@ namespace ErrorCodes
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
@ -216,6 +217,12 @@ void DataTypeAggregateFunction::deserializeTextQuoted(IColumn & column, ReadBuff
}
void DataTypeAggregateFunction::deserializeWholeText(IColumn &, ReadBuffer &, const FormatSettings &) const
{
throw Exception("AggregateFunction data type cannot be read from text", ErrorCodes::NOT_IMPLEMENTED);
}
void DataTypeAggregateFunction::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeJSONString(serializeToString(function, column, row_num), ostr, settings);

View File

@ -52,6 +52,8 @@ public:
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -33,6 +33,10 @@ public:
*/
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
/** Text deserialization without quoting or escaping.
*/
virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text serialization with escaping but without quoting.
*/
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;

View File

@ -32,6 +32,13 @@ DataTypeCustomSimpleTextSerialization::~DataTypeCustomSimpleTextSerialization()
{
}
void DataTypeCustomSimpleTextSerialization::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String str;
readString(str, istr);
deserializeFromString(*this, column, str, settings);
}
void DataTypeCustomSimpleTextSerialization::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeEscapedString(serializeToString(*this, column, row_num, settings), ostr);

View File

@ -21,6 +21,10 @@ public:
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0;
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
/** Text deserialization without quoting or escaping.
*/
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
/** Text serialization with escaping but without quoting.
*/
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -16,6 +16,11 @@ void DataTypeDate::serializeText(const IColumn & column, size_t row_num, WriteBu
writeDateText(DayNum(static_cast<const ColumnUInt16 &>(column).getData()[row_num]), ostr);
}
void DataTypeDate::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeTextEscaped(column, istr, settings);
}
void DataTypeDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
DayNum x;

View File

@ -13,6 +13,7 @@ public:
const char * getFamilyName() const override { return "Date"; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -62,6 +62,11 @@ static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings
}
void DataTypeDateTime::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeTextEscaped(column, istr, settings);
}
void DataTypeDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
time_t x;

View File

@ -38,6 +38,7 @@ public:
TypeIndex getTypeId() const override { return TypeIndex::DateTime; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -166,6 +166,14 @@ void DataTypeEnum<Type>::deserializeTextQuoted(IColumn & column, ReadBuffer & is
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
std::string field_name;
readString(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{

View File

@ -96,6 +96,8 @@ public:
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -176,6 +176,12 @@ void DataTypeFixedString::deserializeTextQuoted(IColumn & column, ReadBuffer & i
}
void DataTypeFixedString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringInto(data, istr); });
}
void DataTypeFixedString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
const char * pos = reinterpret_cast<const char *>(&static_cast<const ColumnFixedString &>(column).getChars()[n * row_num]);

View File

@ -50,6 +50,8 @@ public:
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -81,6 +81,11 @@ public:
deserializeImpl(column, &IDataType::deserializeAsTextQuoted, istr, settings);
}
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings);
}
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
{
serializeImpl(column, row_num, &IDataType::serializeAsTextCSV, ostr, settings);

View File

@ -251,6 +251,15 @@ void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr
[this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); });
}
void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
safeDeserialize(column,
[&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); },
[this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); });
}
void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);

View File

@ -53,6 +53,7 @@ public:
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;

View File

@ -244,6 +244,12 @@ static inline void read(IColumn & column, Reader && reader)
}
void DataTypeString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
read(column, [&](ColumnString::Chars & data) { readStringInto(data, istr); });
}
void DataTypeString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); });

View File

@ -30,6 +30,7 @@ public:
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;

View File

@ -32,6 +32,11 @@ protected:
serializeText(column, row_num, ostr, settings);
}
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);
}
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
{
deserializeText(column, istr, settings);

View File

@ -142,121 +142,89 @@ void IDataType::insertDefaultInto(IColumn & column) const
void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeTextEscaped(column, row_num, ostr, settings);
}
else
{
serializeTextEscaped(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->deserializeTextEscaped(column, istr, settings);
}
else
{
deserializeTextEscaped(column, istr, settings);
}
}
void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeTextQuoted(column, row_num, ostr, settings);
}
else
{
serializeTextQuoted(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->deserializeTextQuoted(column, istr, settings);
}
else
{
deserializeTextQuoted(column, istr, settings);
}
}
void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeTextCSV(column, row_num, ostr, settings);
}
else
{
serializeTextCSV(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->deserializeTextCSV(column, istr, settings);
}
else
{
deserializeTextCSV(column, istr, settings);
}
}
void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeText(column, row_num, ostr, settings);
}
else
{
serializeText(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (custom_text_serialization)
custom_text_serialization->deserializeWholeText(column, istr, settings);
else
deserializeWholeText(column, istr, settings);
}
void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeTextJSON(column, row_num, ostr, settings);
}
else
{
serializeTextJSON(column, row_num, ostr, settings);
}
}
void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->deserializeTextJSON(column, istr, settings);
}
else
{
deserializeTextJSON(column, istr, settings);
}
}
void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
if (custom_text_serialization)
{
custom_text_serialization->serializeTextXML(column, row_num, ostr, settings);
}
else
{
serializeTextXML(column, row_num, ostr, settings);
}
}
void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const

View File

@ -222,76 +222,60 @@ public:
/// If method will throw an exception, then column will be in same state as before call to method.
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
/** Text serialization with escaping but without quoting.
*/
virtual void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization as a literal that may be inserted into a query.
*/
virtual void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for the CSV format.
*/
virtual void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
* Without escaping or quoting.
*/
virtual void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
/** Text serialization intended for using in JSON format.
*/
virtual void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for putting into the XML format.
*/
virtual void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
/** Serialize to a protobuf. */
virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0;
virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0;
protected:
virtual String doGetName() const;
/** Text serialization with escaping but without quoting.
*/
public: // used somewhere in arcadia
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
protected:
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization as a literal that may be inserted into a query.
*/
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for the CSV format.
*/
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
void deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
* Without escaping or quoting.
*/
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
/** Text deserialization in case when buffer contains only one value, without any escaping and delimiters.
*/
void deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization intended for using in JSON format.
* force_quoting_64bit_integers parameter forces to brace UInt64 and Int64 types into quotes.
*/
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
/** Text serialization for putting into the XML format.
*/
void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
protected:
virtual String doGetName() const;
/// Default implementations of text serialization in case of 'custom_text_serialization' is not set.
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
@ -471,7 +455,6 @@ private:
public:
const IDataTypeCustomName * getCustomName() const { return custom_name.get(); }
const IDataTypeCustomTextSerialization * getCustomTextSerialization() const { return custom_text_serialization.get(); }
};

View File

@ -86,7 +86,7 @@ DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, con
bool DatabaseDictionary::empty(const Context & context) const
{
return context.getExternalDictionaries().getNumberOfNames() == 0;
return !context.getExternalDictionaries().hasCurrentlyLoadedObjects();
}
StoragePtr DatabaseDictionary::detachTable(const String & /*table_name*/)

View File

@ -1846,6 +1846,25 @@ Context::SampleBlockCache & Context::getSampleBlockCache() const
}
bool Context::hasQueryParameters() const
{
return !query_parameters.empty();
}
const NameToNameMap & Context::getQueryParameters() const
{
return query_parameters;
}
void Context::setQueryParameter(const String & name, const String & value)
{
if (!query_parameters.emplace(name, value).second)
throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS);
}
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> Context::getCompiledExpressionCache() const

View File

@ -144,6 +144,9 @@ private:
using DatabasePtr = std::shared_ptr<IDatabase>;
using Databases = std::map<String, std::shared_ptr<IDatabase>>;
NameToNameMap query_parameters; /// Dictionary with query parameters for prepared statements.
/// (key=name, value)
IHostContextPtr host_context; /// Arbitrary object that may used to attach some host specific information to query context,
/// when using ClickHouse as a library in some project. For example, it may contain host
/// logger, some query identification information, profiling guards, etc. This field is
@ -464,6 +467,11 @@ public:
SampleBlockCache & getSampleBlockCache() const;
/// Query parameters for prepared statements.
bool hasQueryParameters() const;
const NameToNameMap & getQueryParameters() const;
void setQueryParameter(const String & name, const String & value);
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> getCompiledExpressionCache() const;
void setCompiledExpressionCache(size_t cache_size);

View File

@ -9,10 +9,12 @@
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/addTypeConversionToAST.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataTypes/DataTypeAggregateFunction.h>
namespace DB
{
@ -23,22 +25,6 @@ namespace ErrorCodes
}
static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String & type_name)
{
auto func = std::make_shared<ASTFunction>();
ASTPtr res = func;
func->alias = ast->alias;
func->prefer_alias_to_column_name = ast->prefer_alias_to_column_name;
ast->alias.clear();
func->name = "CAST";
auto exp_list = std::make_shared<ASTExpressionList>();
func->arguments = exp_list;
func->children.push_back(func->arguments);
exp_list->children.emplace_back(ast.release());
exp_list->children.emplace_back(std::make_shared<ASTLiteral>(type_name));
return res;
}
bool ExecuteScalarSubqueriesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child)
{
/// Processed
@ -110,7 +96,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
lit->alias = subquery.alias;
lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = addTypeConversion(std::move(lit), block.safeGetByPosition(0).type->getName());
ast = addTypeConversionToAST(std::move(lit), block.safeGetByPosition(0).type->getName());
}
else
{
@ -125,7 +111,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr
exp_list->children.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversion(
exp_list->children[i] = addTypeConversionToAST(
std::make_unique<ASTLiteral>((*block.safeGetByPosition(i).column)[0]),
block.safeGetByPosition(i).type->getName());
}

View File

@ -343,25 +343,6 @@ public:
enable_async_loading = enable;
}
/// Returns the names of all the objects in the configuration (loaded or not).
std::vector<String> getNames() const
{
std::lock_guard lock{mutex};
std::vector<String> all_names;
for (const auto & name_and_info : infos)
{
const String & name = name_and_info.first;
all_names.emplace_back(name);
}
return all_names;
}
size_t getNumberOfNames() const
{
std::lock_guard lock{mutex};
return infos.size();
}
/// Returns the status of the object.
/// If the object has not been loaded yet then the function returns Status::NOT_LOADED.
/// If the specified name isn't found in the configuration then the function returns Status::NOT_EXIST.
@ -419,6 +400,15 @@ public:
return count;
}
bool hasCurrentlyLoadedObjects() const
{
std::lock_guard lock{mutex};
for (auto & [name, info] : infos)
if (info.loaded())
return true;
return false;
}
/// Starts loading of a specified object.
void load(const String & name)
{
@ -1008,14 +998,9 @@ void ExternalLoader::enablePeriodicUpdates(bool enable_, const ExternalLoaderUpd
periodic_updater->enable(enable_, settings_);
}
std::vector<String> ExternalLoader::getNames() const
bool ExternalLoader::hasCurrentlyLoadedObjects() const
{
return loading_dispatcher->getNames();
}
size_t ExternalLoader::getNumberOfNames() const
{
return loading_dispatcher->getNumberOfNames();
return loading_dispatcher->hasCurrentlyLoadedObjects();
}
ExternalLoader::Status ExternalLoader::getCurrentStatus(const String & name) const

View File

@ -107,10 +107,6 @@ public:
/// Sets settings for periodic updates.
void enablePeriodicUpdates(bool enable, const ExternalLoaderUpdateSettings & settings = {});
/// Returns the names of all the objects in the configuration (loaded or not).
std::vector<String> getNames() const;
size_t getNumberOfNames() const;
/// Returns the status of the object.
/// If the object has not been loaded yet then the function returns Status::NOT_LOADED.
/// If the specified name isn't found in the configuration then the function returns Status::NOT_EXIST.
@ -133,6 +129,9 @@ public:
Loadables getCurrentlyLoadedObjects(const FilterByNameFunction & filter_by_name) const;
size_t getNumberOfCurrentlyLoadedObjects() const;
/// Returns true if any object was loaded.
bool hasCurrentlyLoadedObjects() const;
static constexpr Duration NO_TIMEOUT = Duration::max();
/// Starts loading of a specified object.

View File

@ -1,9 +1,6 @@
#include <memory>
#include <boost/range/join.hpp>
#include <Poco/File.h>
#include <Poco/FileStream.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
@ -44,10 +41,10 @@
#include <Databases/DatabaseFactory.h>
#include <Databases/IDatabase.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/addTypeConversionToAST.h>
namespace DB
@ -281,19 +278,25 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
/// add column to postprocessing if there is a default_expression specified
if (col_decl.default_expression)
{
/** for columns with explicitly-specified type create two expressions:
* 1. default_expression aliased as column name with _tmp suffix
* 2. conversion of expression (1) to explicitly-specified type alias as column name */
/** For columns with explicitly-specified type create two expressions:
* 1. default_expression aliased as column name with _tmp suffix
* 2. conversion of expression (1) to explicitly-specified type alias as column name
*/
if (col_decl.type)
{
const auto & final_column_name = col_decl.name;
const auto tmp_column_name = final_column_name + "_tmp";
const auto data_type_ptr = column_names_and_types.back().type.get();
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(data_type_ptr->getName())), final_column_name));
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
default_expr_list->children.emplace_back(
setAlias(addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
final_column_name));
default_expr_list->children.emplace_back(
setAlias(
col_decl.default_expression->clone(),
tmp_column_name));
}
else
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
@ -332,7 +335,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
column.type = name_type_it->type;
if (!column.type->equals(*deduced_type))
default_expr = makeASTFunction("CAST", default_expr, std::make_shared<ASTLiteral>(column.type->getName()));
default_expr = addTypeConversionToAST(std::move(default_expr), column.type->getName());
}
else
column.type = defaults_sample_block.getByName(column.name).type;

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
@ -20,6 +21,7 @@ namespace ErrorCodes
{
extern const int TOO_DEEP_AST;
extern const int CYCLIC_ALIASES;
extern const int UNKNOWN_QUERY_PARAMETER;
}
@ -227,14 +229,16 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
data.current_alias = my_alias;
}
if (auto * node = ast->as<ASTFunction>())
visit(*node, ast, data);
if (auto * node = ast->as<ASTIdentifier>())
visit(*node, ast, data);
if (auto * node = ast->as<ASTTablesInSelectQueryElement>())
visit(*node, ast, data);
if (auto * node = ast->as<ASTSelectQuery>())
visit(*node, ast, data);
if (auto * node_func = ast->as<ASTFunction>())
visit(*node_func, ast, data);
else if (auto * node_id = ast->as<ASTIdentifier>())
visit(*node_id, ast, data);
else if (auto * node_tables = ast->as<ASTTablesInSelectQueryElement>())
visit(*node_tables, ast, data);
else if (auto * node_select = ast->as<ASTSelectQuery>())
visit(*node_select, ast, data);
else if (auto * node_param = ast->as<ASTQueryParameter>())
throw Exception("Query parameter " + backQuote(node_param->name) + " was not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER);
/// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias.
if (ast.get() != initial_ast.get())

View File

@ -0,0 +1,64 @@
#include <boost/algorithm/string/replace.hpp>
#include <Common/typeid_cast.h>
#include <Columns/IColumn.h>
#include <Core/Field.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeFactory.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadBufferFromString.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTQueryParameter.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/addTypeConversionToAST.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_QUERY_PARAMETER;
extern const int BAD_QUERY_PARAMETER;
}
void ReplaceQueryParameterVisitor::visit(ASTPtr & ast)
{
for (auto & child : ast->children)
{
if (child->as<ASTQueryParameter>())
visitQueryParameter(child);
else
visit(child);
}
}
const String & ReplaceQueryParameterVisitor::getParamValue(const String & name)
{
auto search = query_parameters.find(name);
if (search != query_parameters.end())
return search->second;
else
throw Exception("Substitution " + backQuote(name) + " is not set", ErrorCodes::UNKNOWN_QUERY_PARAMETER);
}
void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast)
{
const auto & ast_param = ast->as<ASTQueryParameter &>();
const String & value = getParamValue(ast_param.name);
const String & type_name = ast_param.type;
const auto data_type = DataTypeFactory::instance().get(type_name);
auto temp_column_ptr = data_type->createColumn();
IColumn & temp_column = *temp_column_ptr;
ReadBufferFromString read_buffer{value};
FormatSettings format_settings;
data_type->deserializeAsWholeText(temp_column, read_buffer, format_settings);
if (!read_buffer.eof())
throw Exception("Value " + value + " cannot be parsed as " + type_name + " for query parameter '" + ast_param.name + "'", ErrorCodes::BAD_QUERY_PARAMETER);
ast = addTypeConversionToAST(std::make_shared<ASTLiteral>(temp_column[0]), type_name);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Core/Names.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class ASTQueryParameter;
/// Visit substitutions in a query, replace ASTQueryParameter with ASTLiteral.
class ReplaceQueryParameterVisitor
{
public:
ReplaceQueryParameterVisitor(const NameToNameMap & parameters)
: query_parameters(parameters)
{}
void visit(ASTPtr & ast);
private:
const NameToNameMap & query_parameters;
const String & getParamValue(const String & name);
void visitQueryParameter(ASTPtr & ast);
};
}

View File

@ -0,0 +1,26 @@
#include "addTypeConversionToAST.h"
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTWithAlias.h>
namespace DB
{
ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name)
{
auto func = makeASTFunction("CAST", ast, std::make_shared<ASTLiteral>(type_name));
if (ASTWithAlias * ast_with_alias = dynamic_cast<ASTWithAlias *>(ast.get()))
{
func->alias = ast_with_alias->alias;
func->prefer_alias_to_column_name = ast_with_alias->prefer_alias_to_column_name;
ast_with_alias->alias.clear();
}
return func;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Core/Types.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
/// It will produce an expression with CAST to get an AST with the required type.
ASTPtr addTypeConversionToAST(ASTPtr && ast, const String & type_name);
}

View File

@ -26,6 +26,7 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include "DNSCacheUpdater.h"
@ -180,7 +181,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
insert_query->has_tail = has_query_tail;
}
else
{
query_end = end;
}
}
catch (...)
{
@ -200,6 +203,17 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
try
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
if (context.hasQueryParameters())
{
ReplaceQueryParameterVisitor visitor(context.getQueryParameters());
visitor.visit(ast);
}
/// Get new query after substitutions.
if (context.hasQueryParameters())
query = serializeAST(*ast);
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
/// Check the limits.

View File

@ -82,6 +82,13 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
col_decl->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::COMMENT_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
settings.ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : "");
@ -172,13 +179,6 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::COMMENT_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
settings.ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : "");

View File

@ -21,18 +21,18 @@ ASTPtr ASTColumnDeclaration::clone() const
res->children.push_back(res->default_expression);
}
if (codec)
{
res->codec = codec->clone();
res->children.push_back(res->codec);
}
if (comment)
{
res->comment = comment->clone();
res->children.push_back(res->comment);
}
if (codec)
{
res->codec = codec->clone();
res->children.push_back(res->codec);
}
if (ttl)
{
res->ttl = ttl->clone();

View File

@ -15,8 +15,8 @@ public:
ASTPtr type;
String default_specifier;
ASTPtr default_expression;
ASTPtr codec;
ASTPtr comment;
ASTPtr codec;
ASTPtr ttl;
String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); }

View File

@ -26,9 +26,10 @@ public:
ASTPtr clone() const override { return std::make_shared<ASTExplainQuery>(*this); }
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind) << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind) << (settings.hilite ? hilite_none : "") << " ";
children.at(0)->formatImpl(settings, state, frame);
}
private:
@ -38,8 +39,8 @@ private:
{
switch (kind)
{
case ParsedAST: return "ParsedAST";
case AnalyzedSyntax: return "AnalyzedSyntax";
case ParsedAST: return "AST";
case AnalyzedSyntax: return "ANALYZE";
}
__builtin_unreachable();

View File

@ -126,6 +126,9 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
{
if (0 == strcmp(name.c_str(), func[0]))
{
if (frame.need_parens)
settings.ostr << '(';
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
/** A particularly stupid case. If we have a unary minus before a literal that is a negative number
@ -138,6 +141,9 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
arguments->formatImpl(settings, state, nested_need_parens);
written = true;
if (frame.need_parens)
settings.ostr << ')';
}
}
}
@ -209,11 +215,17 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (!written && 0 == strcmp(name.c_str(), "arrayElement"))
{
if (frame.need_parens)
settings.ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
arguments->children[1]->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : "");
written = true;
if (frame.need_parens)
settings.ostr << ')';
}
if (!written && 0 == strcmp(name.c_str(), "tupleElement"))
@ -223,10 +235,16 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
{
if (lit->value.getType() == Field::Types::UInt64)
{
if (frame.need_parens)
settings.ostr << '(';
arguments->children[0]->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << "." << (settings.hilite ? hilite_none : "");
arguments->children[1]->formatImpl(settings, state, nested_need_parens);
arguments->children[1]->formatImpl(settings, state, nested_dont_need_parens);
written = true;
if (frame.need_parens)
settings.ostr << ')';
}
}
}

View File

@ -0,0 +1,24 @@
#include <Parsers/ASTQueryParameter.h>
#include <IO/WriteHelpers.h>
namespace DB
{
void ASTQueryParameter::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr
<< (settings.hilite ? hilite_substitution : "") << '{'
<< (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(name)
<< (settings.hilite ? hilite_substitution : "") << ':'
<< (settings.hilite ? hilite_identifier : "") << type
<< (settings.hilite ? hilite_substitution : "") << '}'
<< (settings.hilite ? hilite_none : "");
}
void ASTQueryParameter::appendColumnNameImpl(WriteBuffer & ostr) const
{
writeString(name, ostr);
}
}

View File

@ -0,0 +1,29 @@
#pragma once
#include <Parsers/ASTWithAlias.h>
namespace DB
{
/// Parameter in query with name and type of substitution ({name:type}).
/// Example: SELECT * FROM table WHERE id = {pid:UInt16}.
class ASTQueryParameter : public ASTWithAlias
{
public:
String name;
String type;
ASTQueryParameter(const String & name_, const String & type_) : name(name_), type(type_) {}
/** Get the text that identifies this element. */
String getID(char delim) const override { return String("QueryParameter") + delim + name + ':' + type; }
ASTPtr clone() const override { return std::make_shared<ASTQueryParameter>(*this); }
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
};
}

View File

@ -0,0 +1,14 @@
#include <Parsers/ASTQueryWithTableAndOutput.h>
namespace DB
{
void ASTQueryWithTableAndOutput::formatHelper(const FormatSettings & settings, const char * name) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "");
settings.ostr << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
}
}

View File

@ -9,7 +9,7 @@ namespace DB
/** Query specifying table name and, possibly, the database and the FORMAT section.
*/
*/
class ASTQueryWithTableAndOutput : public ASTQueryWithOutput
{
public:
@ -18,11 +18,7 @@ public:
bool temporary{false};
protected:
void formatHelper(const FormatSettings & settings, const char * name) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << name << " " << (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
}
void formatHelper(const FormatSettings & settings, const char * name) const;
};
@ -43,7 +39,7 @@ public:
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
{
formatHelper(settings, AstIDAndQueryNames::Query);
formatHelper(settings, temporary ? AstIDAndQueryNames::QueryTemporary : AstIDAndQueryNames::Query);
}
};

View File

@ -0,0 +1,37 @@
#include <iomanip>
#include <Parsers/ASTShowTablesQuery.h>
namespace DB
{
ASTPtr ASTShowTablesQuery::clone() const
{
auto res = std::make_shared<ASTShowTablesQuery>(*this);
res->children.clear();
cloneOutputOptions(*res);
return res;
}
void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
if (databases)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << "TABLES" << (settings.hilite ? hilite_none : "");
if (!from.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(from);
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
<< std::quoted(like, '\'');
}
}
}

View File

@ -23,34 +23,10 @@ public:
/** Get the text that identifies this element. */
String getID(char) const override { return "ShowTables"; }
ASTPtr clone() const override
{
auto res = std::make_shared<ASTShowTablesQuery>(*this);
res->children.clear();
cloneOutputOptions(*res);
return res;
}
ASTPtr clone() const override;
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
{
if (databases)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW DATABASES" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW TABLES" << (settings.hilite ? hilite_none : "");
if (!from.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(from);
if (!like.empty())
settings.ostr << (settings.hilite ? hilite_keyword : "") << " LIKE " << (settings.hilite ? hilite_none : "")
<< std::quoted(like, '\'');
}
}
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -81,6 +81,7 @@ ASTPtr ASTTablesInSelectQuery::clone() const
void ASTTableExpression::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.current_select = this;
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
if (database_and_table_name)
@ -183,14 +184,14 @@ void ASTTableJoin::formatImplAfterTable(const FormatSettings & settings, FormatS
if (using_expression_list)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "USING " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " USING " << (settings.hilite ? hilite_none : "");
settings.ostr << "(";
using_expression_list->formatImpl(settings, state, frame);
settings.ostr << ")";
}
else if (on_expression)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ON " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "");
on_expression->formatImpl(settings, state, frame);
}
}
@ -226,7 +227,6 @@ void ASTTablesInSelectQueryElement::formatImpl(const FormatSettings & settings,
}
table_expression->formatImpl(settings, state, frame);
settings.ostr << " ";
if (table_join)
table_join->as<ASTTableJoin &>().formatImplAfterTable(settings, state, frame);

View File

@ -16,27 +16,27 @@ void ASTWithAlias::writeAlias(const String & name, const FormatSettings & settin
void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!alias.empty())
/// If we have previously output this node elsewhere in the query, now it is enough to output only the alias.
/// This is needed because the query can become extraordinary large after substitution of aliases.
if (!alias.empty() && !state.printed_asts_with_alias.emplace(frame.current_select, alias, getTreeHash()).second)
{
/// If we have previously output this node elsewhere in the query, now it is enough to output only the alias.
if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second)
{
settings.writeIdentifier(alias);
return;
}
settings.writeIdentifier(alias);
}
/// If there is an alias, then parentheses are required around the entire expression, including the alias. Because a record of the form `0 AS x + 0` is syntactically invalid.
if (frame.need_parens && !alias.empty())
settings.ostr <<'(';
formatImplWithoutAlias(settings, state, frame);
if (!alias.empty())
else
{
writeAlias(alias, settings);
if (frame.need_parens)
settings.ostr <<')';
/// If there is an alias, then parentheses are required around the entire expression, including the alias.
/// Because a record of the form `0 AS x + 0` is syntactically invalid.
if (frame.need_parens && !alias.empty())
settings.ostr << '(';
formatImplWithoutAlias(settings, state, frame);
if (!alias.empty())
{
writeAlias(alias, settings);
if (frame.need_parens)
settings.ostr << ')';
}
}
}

View File

@ -15,6 +15,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSubquery.h>
@ -1199,6 +1200,52 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
}
bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::OpeningCurlyBrace)
return false;
++pos;
if (pos->type != TokenType::BareWord)
{
expected.add(pos, "substitution name (identifier)");
return false;
}
String name(pos->begin, pos->end);
++pos;
if (pos->type != TokenType::Colon)
{
expected.add(pos, "colon between name and type");
return false;
}
++pos;
auto old_pos = pos;
ParserIdentifierWithOptionalParameters type_parser;
if (!type_parser.ignore(pos, expected))
{
expected.add(pos, "substitution type");
return false;
}
String type(old_pos->begin, pos->begin);
if (pos->type != TokenType::ClosingCurlyBrace)
{
expected.add(pos, "closing curly brace");
return false;
}
++pos;
node = std::make_shared<ASTQueryParameter>(name, type);
return true;
}
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserSubquery().parse(pos, node, expected)
@ -1218,7 +1265,8 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|| ParserFunction().parse(pos, node, expected)
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|| ParserAsterisk().parse(pos, node, expected)
|| ParserCompoundIdentifier().parse(pos, node, expected);
|| ParserCompoundIdentifier().parse(pos, node, expected)
|| ParserSubstitution().parse(pos, node, expected);
}

View File

@ -242,6 +242,17 @@ private:
};
/** Prepared statements.
* Parse query with parameter expression {name:type}.
*/
class ParserSubstitution : public IParserBase
{
protected:
const char * getName() const { return "substitution"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
*/
class ParserExpressionElement : public IParserBase

View File

@ -17,12 +17,13 @@ namespace ErrorCodes
}
const char * IAST::hilite_keyword = "\033[1m";
const char * IAST::hilite_identifier = "\033[0;36m";
const char * IAST::hilite_function = "\033[0;33m";
const char * IAST::hilite_operator = "\033[1;33m";
const char * IAST::hilite_alias = "\033[0;32m";
const char * IAST::hilite_none = "\033[0m";
const char * IAST::hilite_keyword = "\033[1m";
const char * IAST::hilite_identifier = "\033[0;36m";
const char * IAST::hilite_function = "\033[0;33m";
const char * IAST::hilite_operator = "\033[1;33m";
const char * IAST::hilite_alias = "\033[0;32m";
const char * IAST::hilite_substitution = "\033[1;36m";
const char * IAST::hilite_none = "\033[0m";
String backQuoteIfNeed(const String & x)

View File

@ -161,6 +161,13 @@ public:
nl_or_ws = one_line ? ' ' : '\n';
}
FormatSettings(std::ostream & ostr_, const FormatSettings & other)
: ostr(ostr_), hilite(other.hilite), one_line(other.one_line),
always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style)
{
nl_or_ws = one_line ? ' ' : '\n';
}
void writeIdentifier(const String & name) const;
};
@ -170,7 +177,10 @@ public:
/** The SELECT query in which the alias was found; identifier of a node with such an alias.
* It is necessary that when the node has met again, output only the alias.
*/
std::set<std::pair<const IAST *, std::string>> printed_asts_with_alias;
std::set<std::tuple<
const IAST * /* SELECT query node */,
std::string /* alias */,
Hash /* printed content */>> printed_asts_with_alias;
};
/// The state that is copied when each node is formatted. For example, nesting level.
@ -201,6 +211,7 @@ public:
static const char * hilite_function;
static const char * hilite_operator;
static const char * hilite_alias;
static const char * hilite_substitution;
static const char * hilite_none;
private:

View File

@ -173,7 +173,10 @@ Token Lexer::nextTokenImpl()
return Token(TokenType::OpeningSquareBracket, token_begin, ++pos);
case ']':
return Token(TokenType::ClosingSquareBracket, token_begin, ++pos);
case '{':
return Token(TokenType::OpeningCurlyBrace, token_begin, ++pos);
case '}':
return Token(TokenType::ClosingCurlyBrace, token_begin, ++pos);
case ',':
return Token(TokenType::Comma, token_begin, ++pos);
case ';':

View File

@ -23,6 +23,9 @@ namespace DB
M(OpeningSquareBracket) \
M(ClosingSquareBracket) \
\
M(OpeningCurlyBrace) \
M(ClosingCurlyBrace) \
\
M(Comma) \
M(Semicolon) \
M(Dot) /** Compound identifiers, like a.b or tuple access operator a.1, (x, y).2. */ \

View File

@ -10,24 +10,28 @@ struct ASTExistsQueryIDAndQueryNames
{
static constexpr auto ID = "ExistsQuery";
static constexpr auto Query = "EXISTS TABLE";
static constexpr auto QueryTemporary = "EXISTS TEMPORARY TABLE";
};
struct ASTShowCreateTableQueryIDAndQueryNames
{
static constexpr auto ID = "ShowCreateTableQuery";
static constexpr auto Query = "SHOW CREATE TABLE";
static constexpr auto QueryTemporary = "SHOW CREATE TEMPORARY TABLE";
};
struct ASTShowCreateDatabaseQueryIDAndQueryNames
{
static constexpr auto ID = "ShowCreateDatabaseQuery";
static constexpr auto Query = "SHOW CREATE DATABASE";
static constexpr auto QueryTemporary = "SHOW CREATE TEMPORARY DATABASE";
};
struct ASTDescribeQueryExistsQueryIDAndQueryNames
{
static constexpr auto ID = "DescribeQuery";
static constexpr auto Query = "DESCRIBE TABLE";
static constexpr auto QueryTemporary = "DESCRIBE TEMPORARY TABLE";
};
using ASTExistsQuery = ASTQueryWithTableAndOutputImpl<ASTExistsQueryIDAndQueryNames>;

View File

@ -28,6 +28,8 @@ std::map<TokenType, const char *> hilite =
{TokenType::ClosingRoundBracket, "\033[1;33m"},
{TokenType::OpeningSquareBracket, "\033[1;33m"},
{TokenType::ClosingSquareBracket, "\033[1;33m"},
{TokenType::OpeningCurlyBrace, "\033[1;33m"},
{TokenType::ClosingCurlyBrace, "\033[1;33m"},
{TokenType::Comma, "\033[1;33m"},
{TokenType::Semicolon, "\033[1;33m"},

View File

@ -54,22 +54,22 @@ void check(const std::string & query, const std::string & expected, const Contex
TEST(TransformQueryForExternalDatabase, InWithSingleElement)
{
check("SELECT column FROM test.table WHERE 1 IN (1)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE 1 IN (1)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE 1 IN (1)",
state().context, state().columns);
check("SELECT column FROM test.table WHERE column IN (1, 2)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" IN (1, 2)",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" IN (1, 2)",
state().context, state().columns);
check("SELECT column FROM test.table WHERE column NOT IN ('hello', 'world')",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" NOT IN ('hello', 'world')",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" NOT IN ('hello', 'world')",
state().context, state().columns);
}
TEST(TransformQueryForExternalDatabase, Like)
{
check("SELECT column FROM test.table WHERE column LIKE '%hello%'",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" LIKE '%hello%'",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" LIKE '%hello%'",
state().context, state().columns);
check("SELECT column FROM test.table WHERE column NOT LIKE 'w%rld'",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" NOT LIKE 'w%rld'",
"SELECT \"column\" FROM \"test\".\"table\" WHERE \"column\" NOT LIKE 'w%rld'",
state().context, state().columns);
}

View File

@ -24,11 +24,6 @@ DROP DATABASE IF EXISTS test_DatabaseMemory;
CREATE DATABASE test_DatabaseMemory ENGINE = Memory;
CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null;
-- Just in case
DROP DATABASE IF EXISTS test_DatabaseDictionary;
CREATE DATABASE test_DatabaseDictionary ENGINE = Dictionary;
SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables;
DROP DATABASE test_DatabaseDictionary; -- { serverError 48 }
DROP DATABASE test_DatabaseMemory;

View File

@ -4,59 +4,59 @@
1
2000-01-01 1 test string 1 1
-------Forbid push down-------
SELECT count()\nFROM \n(\n SELECT \n [number] AS a, \n [number * 2] AS b\n FROM system.numbers \n LIMIT 1\n) AS t \nARRAY JOIN \n a, \n b\nWHERE NOT ignore(a + b)
SELECT count()\nFROM \n(\n SELECT \n [number] AS a, \n [number * 2] AS b\n FROM system.numbers\n LIMIT 1\n) AS t\nARRAY JOIN \n a, \n b\nWHERE NOT ignore(a + b)
1
SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n) \nANY LEFT JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n) \nANY RIGHT JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n) \nANY FULL OUTER JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n) \nANY FULL OUTER JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n)\nANY LEFT JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n)\nANY RIGHT JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT 1 AS a\n)\nANY FULL OUTER JOIN \n(\n SELECT \n 1 AS a, \n 1 AS b\n) USING (a)\nWHERE b = 0
SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b\n)\nANY FULL OUTER JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0
-------Need push down-------
SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n WHERE toString(value) = \'1\'\n) \nWHERE value = \'1\'
SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n WHERE toString(value) = \'1\'\n)\nWHERE value = \'1\'
1
SELECT id\nFROM \n(\n SELECT 1 AS id\n WHERE id = 1\n UNION ALL\n SELECT 2 AS `2`\n WHERE `2` = 1\n) \nWHERE id = 1
SELECT id\nFROM \n(\n SELECT 1 AS id\n WHERE id = 1\n UNION ALL\n SELECT 2 AS `2`\n WHERE `2` = 1\n)\nWHERE id = 1
1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n) \nWHERE id = 1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1
1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n) \nWHERE id = 1
SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1
1
SELECT \n id, \n subquery\nFROM \n(\n SELECT \n 1 AS id, \n CAST(1, \'UInt8\') AS subquery\n WHERE subquery = 1\n) \nWHERE subquery = 1
SELECT \n id, \n subquery\nFROM \n(\n SELECT \n 1 AS id, \n CAST(1, \'UInt8\') AS subquery\n WHERE subquery = 1\n)\nWHERE subquery = 1
1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597 \n HAVING a = 3\n) \nWHERE a = 3
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597\n HAVING a = 3\n)\nWHERE a = 3
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n name, \n value, \n min(id) AS id\n FROM test_00597 \n GROUP BY \n date, \n name, \n value\n HAVING id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n name, \n value, \n min(id) AS id\n FROM test_00597\n GROUP BY \n date, \n name, \n value\n HAVING id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597 AS table_alias \n HAVING b = 3\n) AS outer_table_alias \nWHERE b = 3
SELECT \n a, \n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a, \n b\n FROM test_00597 AS table_alias\n HAVING b = 3\n) AS outer_table_alias\nWHERE b = 3
3 3
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n )\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS b \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n ) AS b\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n )\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) AS b \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n) AS b\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS a \n WHERE id = 1\n) AS b \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n ) AS a\n WHERE id = 1\n) AS b\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n id, \n date, \n value\nFROM \n(\n SELECT \n id, \n date, \n min(value) AS value\n FROM test_00597 \n WHERE id = 1\n GROUP BY \n id, \n date\n) \nWHERE id = 1
SELECT \n id, \n date, \n value\nFROM \n(\n SELECT \n id, \n date, \n min(value) AS value\n FROM test_00597\n WHERE id = 1\n GROUP BY \n id, \n date\n)\nWHERE id = 1
1 2000-01-01 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n UNION ALL\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n UNION ALL\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) USING (id)\nWHERE id = 1
SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n)\nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597\n) USING (id)\nWHERE id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS test_00597 USING (id)\nWHERE value = 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT toInt8(1) AS id\n)\nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597\n) AS test_00597 USING (id)\nWHERE value = 1
1 2000-01-01 test string 1 1
SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n) \nANY LEFT JOIN test_00597 AS b USING (id)\nWHERE value = 1
SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n)\nANY LEFT JOIN test_00597 AS b USING (id)\nWHERE value = 1
1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) \n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597 \n ) USING (id)\n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n date, \n name, \n value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n )\n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597\n ) USING (id)\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.name, \n b.value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS b USING (id)\nWHERE b.id = 1
SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.name, \n b.value\nFROM \n(\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n)\nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597\n) AS b USING (id)\nWHERE b.id = 1
2000-01-01 1 test string 1 1 2000-01-01 test string 1 1
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT \n toInt8(1) AS id, \n toDate(\'2000-01-01\') AS date\n FROM system.numbers \n LIMIT 1\n) \nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597 \n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\')
SELECT \n id, \n date, \n name, \n value\nFROM \n(\n SELECT \n toInt8(1) AS id, \n toDate(\'2000-01-01\') AS date\n FROM system.numbers\n LIMIT 1\n)\nANY LEFT JOIN \n(\n SELECT *\n FROM test_00597\n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\')
1 2000-01-01 test string 1 1
SELECT \n date, \n id, \n name, \n value, \n `b.date`, \n `b.id`, \n `b.name`, \n `b.value`\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.id, \n b.name, \n b.value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597 \n WHERE id = 1\n ) AS a \n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597 \n ) AS b ON id = b.id\n WHERE id = 1\n) \nWHERE id = 1
SELECT \n date, \n id, \n name, \n value, \n `b.date`, \n `b.id`, \n `b.name`, \n `b.value`\nFROM \n(\n SELECT \n date, \n id, \n name, \n value, \n b.date, \n b.id, \n b.name, \n b.value\n FROM \n (\n SELECT \n date, \n id, \n name, \n value\n FROM test_00597\n WHERE id = 1\n ) AS a\n ANY LEFT JOIN \n (\n SELECT *\n FROM test_00597\n ) AS b ON id = b.id\n WHERE id = 1\n)\nWHERE id = 1
2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1

View File

@ -1 +1 @@
CREATE VIEW default.test_view_00599 (`id` UInt64) AS SELECT * FROM default.test_00599 WHERE id = (SELECT 1)
CREATE VIEW default.test_view_00599 (`id` UInt64) AS SELECT * FROM default.test_00599 WHERE id = (SELECT 1)

View File

@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT $settings -q "INSERT INTO merge_tree_table SELECT (intHash64(
$CLICKHOUSE_CLIENT $settings -q "OPTIMIZE TABLE merge_tree_table FINAL;"
toching_many_parts_query="SELECT count() from (SELECT toDayOfWeek(date) as m, id, count() FROM merge_tree_table GROUP BY id, m ORDER BY count() DESC LIMIT 10 SETTINGS max_threads = 1)"
toching_many_parts_query="SELECT count() FROM (SELECT toDayOfWeek(date) AS m, id, count() FROM merge_tree_table GROUP BY id, m ORDER BY count() DESC LIMIT 10 SETTINGS max_threads = 1)"
$CLICKHOUSE_CLIENT $settings -q "$toching_many_parts_query" &> /dev/null
$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS"

View File

@ -1,4 +1,4 @@
CREATE MATERIALIZED VIEW test.t_mv_00751 (`date` Date, `platform` Enum8('a' = 0, 'b' = 1), `app` Enum8('a' = 0, 'b' = 1)) ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS SELECT date, platform, app FROM test.t_00751 WHERE (app = (SELECT min(app) FROM test.u_00751 )) AND (platform = (SELECT (SELECT min(platform) FROM test.v_00751 )))
CREATE MATERIALIZED VIEW test.t_mv_00751 (`date` Date, `platform` Enum8('a' = 0, 'b' = 1), `app` Enum8('a' = 0, 'b' = 1)) ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS SELECT date, platform, app FROM test.t_00751 WHERE (app = (SELECT min(app) FROM test.u_00751)) AND (platform = (SELECT (SELECT min(platform) FROM test.v_00751)))
2000-01-01 a a
2000-01-02 b b
2000-01-03 a a

View File

@ -56,26 +56,26 @@ comma nullable
1 1 1 1
2 2 1 2
cross
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \n, \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a
cross nullable vs not nullable
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b
cross self
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x \nCROSS JOIN t1_00826 AS y \nWHERE (a = y.a) AND (b = y.b)
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x \nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b)
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nCROSS JOIN t1_00826 AS y\nWHERE (a = y.a) AND (b = y.b)
SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b)
cross one table expr
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b
cross multiple ands
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b)
cross and inside and
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 \nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n) AS t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b)))
cross split conjunction
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00826 \n WHERE b > 0\n) AS t2_00826 \nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826 \n WHERE b > 0\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)
SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0)

View File

@ -1,17 +1,17 @@
SELECT a\nFROM t1_00849 \nCROSS JOIN \n(\n SELECT *\n FROM t2_00849 \n) AS t2_00849
SELECT a\nFROM t1_00849 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849 \n) AS t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
SELECT a\nFROM t1_00849 \nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849 \n) AS t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849 \n) AS t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849 \n) AS t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n ) \n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n WHERE (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n ) \n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n WHERE (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = `--t2_00849.b`)\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n WHERE `--t2_00849.a` = `--t1_00849.a`\n ) \n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = `--t1_00849.a`)\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n CROSS JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 \n ) \n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n WHERE (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = `--t1_00849.a`)\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n CROSS JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 \n ) \n CROSS JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 \n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n ) \n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n) \nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849 \n CROSS JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 \n ) \n CROSS JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 \n) \nCROSS JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849 \n CROSS JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 \n ) \n CROSS JOIN \n (\n SELECT *\n FROM t3_00849 \n ) AS t3_00849 \n) \nCROSS JOIN \n(\n SELECT *\n FROM t4_00849 \n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849 \n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849 \n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n) \nCROSS JOIN \n(\n SELECT *\n FROM t3_00849 \n) AS t3_00849
SELECT a\nFROM t1_00849\nCROSS JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a
SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n WHERE (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n WHERE (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = `--t2_00849.b`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n WHERE `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n WHERE (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a)
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849
SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849
SELECT * FROM t1, t2
1 1 1 1
1 1 1 \N

View File

@ -1 +1 @@
SELECT \n a, \n b\nFROM a
SELECT \n a, \n b\nFROM a

View File

@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE cannot_kill_query (x UInt64) ENGINE = MergeT
$CLICKHOUSE_CLIENT -q "INSERT INTO cannot_kill_query SELECT * FROM numbers(10000000)" &> /dev/null
# This SELECT query will run for a long time. It's used as bloker for ALTER query. It will be killed with SYNC kill.
query_for_pending="SELECT count() FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1, max_block_size=1"
query_for_pending="SELECT count() FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads = 1, max_block_size = 1"
$CLICKHOUSE_CLIENT -q "$query_for_pending" &>/dev/null &
sleep 1 # queries should be in strict order
@ -23,7 +23,7 @@ sleep 1
# This SELECT query will also run for a long time. Also it's blocked by ALTER query. It will be killed with ASYNC kill.
# This is main idea which we check -- blocked queries can be killed with ASYNC kill.
query_to_kill="SELECT sum(1) FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads=1"
query_to_kill="SELECT sum(1) FROM cannot_kill_query WHERE NOT ignore(sleep(1)) SETTINGS max_threads = 1"
$CLICKHOUSE_CLIENT -q "$query_to_kill" &>/dev/null &
sleep 1 # just to be sure that kill of $query_to_kill will be executed after $query_to_kill.

View File

@ -1,2 +1,2 @@
CREATE VIEW default.t (`number` UInt64) AS SELECT number FROM system.numbers
CREATE VIEW default.t (`next_number` UInt64) AS SELECT number + 1 AS next_number FROM system.numbers
CREATE VIEW default.t (`number` UInt64) AS SELECT number FROM system.numbers
CREATE VIEW default.t (`next_number` UInt64) AS SELECT number + 1 AS next_number FROM system.numbers

View File

@ -0,0 +1,11 @@
1 Hello, world 2005-05-05 05:05:05
1 Hello, world 2005-05-05 05:05:05
2 test 2005-05-25 15:00:00
2 test 2005-05-25 15:00:00
Code: 456.
abc
abc
Hello, world
Hello, world
0
0

View File

@ -0,0 +1,33 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ps";
$CLICKHOUSE_CLIENT -q "CREATE TABLE ps (i UInt8, s String, d DateTime) ENGINE = Memory";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (1, 'Hello, world', '2005-05-05 05:05:05')";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (2, 'test', '2005-05-25 15:00:00')";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=1 \
-q "SELECT * FROM ps WHERE i = {id:UInt8}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_phrase='Hello, world' \
-q "SELECT * FROM ps WHERE s = {phrase:String}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_date='2005-05-25 15:00:00' \
-q "SELECT * FROM ps WHERE d = {date:DateTime}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_id=2 --param_phrase='test' \
-q "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String}";
$CLICKHOUSE_CLIENT -q "SELECT {s:String}" 2>&1 | grep -oP '^Code: 456\.'
$CLICKHOUSE_CLIENT -q "DROP TABLE ps";
$CLICKHOUSE_CLIENT --param_test abc --query 'SELECT {test:String}'
$CLICKHOUSE_CLIENT --param_test=abc --query 'SELECT {test:String}'
$CLICKHOUSE_CLIENT --param_test 'Hello, world' --query 'SELECT {test:String}'
$CLICKHOUSE_CLIENT --param_test='Hello, world' --query 'SELECT {test:String}'
$CLICKHOUSE_CLIENT --param_test '' --query 'SELECT length({test:String})'
$CLICKHOUSE_CLIENT --param_test='' --query 'SELECT length({test:String})'

View File

@ -0,0 +1,6 @@
(1,'Hello')
(1,('dt',2))
[10,10,10]
[[10],[10],[10]]
[10,10,10] [[10],[10],[10]] (10,'Test') (10,('dt',10)) 2015-02-15
Code: 457.

View File

@ -0,0 +1,40 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
EXCEPTION_TEXT="Code: 457."
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ps";
$CLICKHOUSE_CLIENT -q "CREATE TABLE ps (
a Array(UInt32), da Array(Array(UInt8)),
t Tuple(Int16, String), dt Tuple(UInt8, Tuple(String, UInt8)),
n Nullable(Date)
) ENGINE = Memory";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (
[1, 2], [[1, 1], [2, 2]],
(1, 'Hello'), (1, ('dt', 2)),
NULL)";
$CLICKHOUSE_CLIENT -q "INSERT INTO ps VALUES (
[10, 10, 10], [[10], [10], [10]],
(10, 'Test'), (10, ('dt', 10)),
'2015-02-15')";
$CLICKHOUSE_CLIENT --max_threads=1 --param_aui="[1, 2]" \
-q "SELECT t FROM ps WHERE a = {aui:Array(UInt16)}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_d_a="[[1, 1], [2, 2]]" \
-q "SELECT dt FROM ps WHERE da = {d_a:Array(Array(UInt8))}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_tisd="(10, 'Test')" \
-q "SELECT a FROM ps WHERE t = {tisd:Tuple(Int16, String)}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_d_t="(10, ('dt', 10))" \
-q "SELECT da FROM ps WHERE dt = {d_t:Tuple(UInt8, Tuple(String, UInt8))}";
$CLICKHOUSE_CLIENT --max_threads=1 --param_nd="2015-02-15" \
-q "SELECT * FROM ps WHERE n = {nd:Nullable(Date)}";
# Must throw an exception to avoid SQL injection
$CLICKHOUSE_CLIENT --max_threads=1 --param_injection="[1] OR 1" \
-q "SELECT * FROM ps WHERE a = {injection:Array(UInt32)}" 2>&1 \
| grep -o "$EXCEPTION_TEXT"
$CLICKHOUSE_CLIENT -q "DROP TABLE ps";

View File

@ -0,0 +1,4 @@
1 Hello, world 2005-05-05
1 Hello, world 2005-05-05
2 test 2019-05-25
2 test 2019-05-25

View File

@ -0,0 +1,21 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "DROP TABLE IF EXISTS ps";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "CREATE TABLE ps (i UInt8, s String, d Date) ENGINE = Memory";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (1, 'Hello, world', '2005-05-05')";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "INSERT INTO ps VALUES (2, 'test', '2019-05-25')";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_id=1" \
-d "SELECT * FROM ps WHERE i = {id:UInt8} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_phrase=Hello,+world" \
-d "SELECT * FROM ps WHERE s = {phrase:String} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_date=2019-05-25" \
-d "SELECT * FROM ps WHERE d = {date:Date} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?param_id=2&param_phrase=test" \
-d "SELECT * FROM ps WHERE i = {id:UInt8} and s = {phrase:String} ORDER BY i, s, d";
${CLICKHOUSE_CURL} -sS $CLICKHOUSE_URL -d "DROP TABLE ps";

View File

@ -0,0 +1,7 @@
SELECT
1 AS x,
x.y
FROM
(
SELECT 'Hello, world' AS y
) AS x

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
set -e
format="$CLICKHOUSE_FORMAT"
echo "SELECT 1 AS x, x.y FROM (SELECT 'Hello, world' AS y) AS x" | $format

View File

@ -0,0 +1,9 @@
SELECT
(x.1)[1],
(((x[1]).1)[1]).1,
(NOT x)[1],
-(x[1]),
(-x)[1],
(NOT x).1,
-(x.1),
(-x).1

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
set -e
format="$CLICKHOUSE_FORMAT"
echo "SELECT (x.1)[1], (x[1].1)[1].1, (NOT x)[1], -x[1], (-x)[1], (NOT x).1, -x.1, (-x).1" | $format

View File

@ -0,0 +1,3 @@
SELECT a + b AS x, x
SELECT a + b AS x, a + c AS x
SELECT a + b AS x, x

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
set -e
format="$CLICKHOUSE_FORMAT --oneline"
echo "SELECT a + b AS x, a + b AS x" | $format
echo "SELECT a + b AS x, a + c AS x" | $format
echo "SELECT a + b AS x, x" | $format