Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-10-17 22:59:03 +03:00
commit bdd48e79f3
52 changed files with 3208 additions and 1213 deletions

View File

@ -12,3 +12,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Meetups ## Upcoming Meetups
* [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1) * [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1)
* [Amsterdam on November 15](https://events.yandex.com/events/meetings/15-11-2018/)

View File

@ -0,0 +1,646 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/FieldToDataType.h>
#include <DataStreams/LazyBlockInputStream.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <Storages/StorageSet.h>
#include <Parsers/ASTFunction.h>
#include <Common/typeid_cast.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/ProjectionManipulation.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Set.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/interpretSubquery.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_IDENTIFIER;
extern const int NOT_AN_AGGREGATE;
extern const int UNEXPECTED_EXPRESSION;
extern const int TYPE_MISMATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/// defined in ExpressionAnalyser.cpp
NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols);
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets)
{
const IAST & args = *node->arguments;
if (args.children.size() != 2)
throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTPtr & left_arg = args.children.at(0);
const ASTPtr & right_arg = args.children.at(1);
auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr
{
auto ast_function = typeid_cast<const ASTFunction *>(tuple_ast.get());
if (ast_function && ast_function->name == "tuple" && !ast_function->arguments->children.empty())
{
/// Won't parse all values of outer tuple.
auto element = ast_function->arguments->children.at(0);
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(element, context);
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
}
return evaluateConstantExpression(tuple_ast, context).second;
};
const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type;
const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg);
std::function<size_t(const DataTypePtr &)> getTupleDepth;
getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t
{
if (auto tuple_type = typeid_cast<const DataTypeTuple *>(type.get()))
return 1 + (tuple_type->getElements().empty() ? 0 : getTupleDepth(tuple_type->getElements().at(0)));
return 0;
};
size_t left_tuple_depth = getTupleDepth(left_arg_type);
size_t right_tuple_depth = getTupleDepth(right_arg_type);
DataTypes set_element_types = {left_arg_type};
auto left_tuple_type = typeid_cast<const DataTypeTuple *>(left_arg_type.get());
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
set_element_types = left_tuple_type->getElements();
for (auto & element_type : set_element_types)
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
element_type = low_cardinality_type->getDictionaryType();
ASTPtr elements_ast = nullptr;
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
if (left_tuple_depth == right_tuple_depth)
{
ASTPtr exp_list = std::make_shared<ASTExpressionList>();
exp_list->children.push_back(right_arg);
elements_ast = exp_list;
}
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
else if (left_tuple_depth + 1 == right_tuple_depth)
{
ASTFunction * set_func = typeid_cast<ASTFunction *>(right_arg.get());
if (!set_func || set_func->name != "tuple")
throw Exception("Incorrect type of 2nd argument for function " + node->name
+ ". Must be subquery or set of elements with type " + left_arg_type->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
elements_ast = set_func->arguments;
}
else
throw Exception("Invalid types for IN function: "
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set);
set->createFromAST(set_element_types, elements_ast, context);
prepared_sets[right_arg->range] = std::move(set);
}
static String getUniqueName(const Block & block, const String & prefix)
{
int i = 1;
while (block.has(prefix + toString(i)))
++i;
return prefix + toString(i);
}
ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_)
: context(context_)
{
stack.emplace_back();
stack.back().actions = actions;
const Block & sample_block = actions->getSampleBlock();
for (size_t i = 0, size = sample_block.columns(); i < size; ++i)
stack.back().new_columns.insert(sample_block.getByPosition(i).name);
}
void ScopeStack::pushLevel(const NamesAndTypesList & input_columns)
{
stack.emplace_back();
Level & prev = stack[stack.size() - 2];
ColumnsWithTypeAndName all_columns;
NameSet new_names;
for (NamesAndTypesList::const_iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
all_columns.emplace_back(nullptr, it->type, it->name);
new_names.insert(it->name);
stack.back().new_columns.insert(it->name);
}
const Block & prev_sample_block = prev.actions->getSampleBlock();
for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i)
{
const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i);
if (!new_names.count(col.name))
all_columns.push_back(col);
}
stack.back().actions = std::make_shared<ExpressionActions>(all_columns, context);
}
size_t ScopeStack::getColumnLevel(const std::string & name)
{
for (int i = static_cast<int>(stack.size()) - 1; i >= 0; --i)
if (stack[i].new_columns.count(name))
return i;
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
}
void ScopeStack::addAction(const ExpressionAction & action)
{
size_t level = 0;
Names required = action.getNeededColumns();
for (size_t i = 0; i < required.size(); ++i)
level = std::max(level, getColumnLevel(required[i]));
Names added;
stack[level].actions->add(action, added);
stack[level].new_columns.insert(added.begin(), added.end());
for (size_t i = 0; i < added.size(); ++i)
{
const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(added[i]);
for (size_t j = level + 1; j < stack.size(); ++j)
stack[j].actions->addInput(col);
}
}
ExpressionActionsPtr ScopeStack::popLevel()
{
ExpressionActionsPtr res = stack.back().actions;
stack.pop_back();
return res;
}
const Block & ScopeStack::getSampleBlock() const
{
return stack.back().actions->getSampleBlock();
}
ActionsVisitor::ActionsVisitor(
const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_,
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_)
: context(context_),
set_size_limit(set_size_limit_),
subquery_depth(subquery_depth_),
source_columns(source_columns_),
prepared_sets(prepared_sets_),
subqueries_for_sets(subqueries_for_sets_),
no_subqueries(no_subqueries_),
only_consts(only_consts_),
no_storage_or_local(no_storage_or_local_),
visit_depth(0),
ostr(ostr_),
actions_stack(actions, context)
{
if (is_conditional_tree)
projection_manipulator = std::make_shared<ConditionalTree>(actions_stack, context);
else
projection_manipulator = std::make_shared<DefaultProjectionManipulator>(actions_stack);
}
void ActionsVisitor::visit(const ASTPtr & ast)
{
DumpASTNode dump(*ast, ostr, visit_depth, "getActions");
String ast_column_name;
auto getColumnName = [&ast, &ast_column_name]()
{
if (ast_column_name.empty())
ast_column_name = ast->getColumnName();
return ast_column_name;
};
/// If the result of the calculation already exists in the block.
if ((typeid_cast<ASTFunction *>(ast.get()) || typeid_cast<ASTLiteral *>(ast.get()))
&& projection_manipulator->tryToGetFromUpperProjection(getColumnName()))
return;
if (typeid_cast<ASTIdentifier *>(ast.get()))
{
if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName()))
{
/// The requested column is not in the block.
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
bool found = false;
for (const auto & column_name_type : source_columns)
if (column_name_type.name == getColumnName())
found = true;
if (found)
throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.",
ErrorCodes::NOT_AN_AGGREGATE);
}
}
else if (ASTFunction * node = typeid_cast<ASTFunction *>(ast.get()))
{
if (node->name == "lambda")
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
/// Function arrayJoin.
if (node->name == "arrayJoin")
{
if (node->arguments->children.size() != 1)
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
ASTPtr arg = node->arguments->children.at(0);
visit(arg);
if (!only_consts)
{
String result_name = projection_manipulator->getColumnName(getColumnName());
actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name));
NameSet joined_columns;
joined_columns.insert(result_name);
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context));
}
return;
}
if (functionIsInOrGlobalInOperator(node->name))
{
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
visit(node->arguments->children.at(0));
if (!no_subqueries)
{
/// Transform tuple or subquery into a set.
makeSet(node, actions_stack.getSampleBlock());
}
else
{
if (!only_consts)
{
/// We are in the part of the tree that we are not going to compute. You just need to define types.
/// Do not subquery and create sets. We treat "IN" as "ignore" function.
actions_stack.addAction(ExpressionAction::applyFunction(
FunctionFactory::instance().get("ignore", context),
{ node->arguments->children.at(0)->getColumnName() },
projection_manipulator->getColumnName(getColumnName()),
projection_manipulator->getProjectionSourceColumn()));
}
return;
}
}
/// A special function `indexHint`. Everything that is inside it is not calculated
/// (and is used only for index analysis, see KeyCondition).
if (node->name == "indexHint")
{
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
projection_manipulator->getColumnName(getColumnName())), projection_manipulator->getProjectionSourceColumn(), false));
return;
}
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
return;
/// Context object that we pass to function should live during query.
const Context & function_context = context.hasQueryContext()
? context.getQueryContext()
: context;
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, function_context);
auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, getColumnName(), function_context);
Names argument_names;
DataTypes argument_types;
bool arguments_present = true;
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
bool has_lambda_arguments = false;
for (size_t arg = 0; arg < node->arguments->children.size(); ++arg)
{
auto & child = node->arguments->children[arg];
auto child_column_name = child->getColumnName();
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
if (lambda && lambda->name == "lambda")
{
/// If the argument is a lambda expression, just remember its approximate type.
if (lambda->arguments->children.size() != 2)
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
has_lambda_arguments = true;
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
/// Select the name in the next cycle.
argument_names.emplace_back();
}
else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
{
ColumnWithTypeAndName column;
column.type = std::make_shared<DataTypeSet>();
const SetPtr & set = prepared_sets[child->range];
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
/// so that sets with the same literal representation do not fuse together (they can have different types).
if (!set->empty())
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
else
column.name = child_column_name;
column.name = projection_manipulator->getColumnName(column.name);
if (!actions_stack.getSampleBlock().has(column.name))
{
column.column = ColumnSet::create(1, set);
actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false));
}
argument_types.push_back(column.type);
argument_names.push_back(column.name);
}
else
{
/// If the argument is not a lambda expression, call it recursively and find out its type.
projection_action->preArgumentAction();
visit(child);
std::string name = projection_manipulator->getColumnName(child_column_name);
projection_action->postArgumentAction(child_column_name);
if (actions_stack.getSampleBlock().has(name))
{
argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type);
argument_names.push_back(name);
}
else
{
if (only_consts)
{
arguments_present = false;
}
else
{
throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER);
}
}
}
}
if (only_consts && !arguments_present)
return;
if (has_lambda_arguments && !only_consts)
{
function_builder->getLambdaArgumentTypes(argument_types);
/// Call recursively for lambda expressions.
for (size_t i = 0; i < node->arguments->children.size(); ++i)
{
ASTPtr child = node->arguments->children[i];
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
if (lambda && lambda->name == "lambda")
{
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(lambda->arguments->children.at(0).get());
ASTs lambda_arg_asts = lambda_args_tuple->arguments->children;
NamesAndTypesList lambda_arguments;
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
{
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(lambda_arg_asts[j].get());
if (!identifier)
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
String arg_name = identifier->name;
lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]);
}
projection_action->preArgumentAction();
actions_stack.pushLevel(lambda_arguments);
visit(lambda->arguments->children.at(1));
ExpressionActionsPtr lambda_actions = actions_stack.popLevel();
String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName());
lambda_actions->finalize(Names(1, result_name));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
Names captured;
Names required = lambda_actions->getRequiredColumns();
for (const auto & required_arg : required)
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
captured.push_back(required_arg);
/// We can not name `getColumnName()`,
/// because it does not uniquely define the expression (the types of arguments can be different).
String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda");
auto function_capture = std::make_shared<FunctionCapture>(
lambda_actions, captured, lambda_arguments, result_type, result_name);
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name,
projection_manipulator->getProjectionSourceColumn()));
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
argument_names[i] = lambda_name;
projection_action->postArgumentAction(lambda_name);
}
}
}
if (only_consts)
{
for (const auto & argument_name : argument_names)
{
if (!actions_stack.getSampleBlock().has(argument_name))
{
arguments_present = false;
break;
}
}
}
if (arguments_present)
{
projection_action->preCalculation();
if (projection_action->isCalculationRequired())
{
actions_stack.addAction(
ExpressionAction::applyFunction(function_builder,
argument_names,
projection_manipulator->getColumnName(getColumnName()),
projection_manipulator->getProjectionSourceColumn()));
}
}
}
else if (ASTLiteral * literal = typeid_cast<ASTLiteral *>(ast.get()))
{
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
ColumnWithTypeAndName column;
column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type));
column.type = type;
column.name = getColumnName();
actions_stack.addAction(ExpressionAction::addColumn(column, "", false));
projection_manipulator->tryToGetFromUpperProjection(column.name);
}
else
{
for (auto & child : ast->children)
{
/// Do not go to FROM, JOIN, UNION.
if (!typeid_cast<const ASTTableExpression *>(child.get())
&& !typeid_cast<const ASTSelectQuery *>(child.get()))
visit(child);
}
}
}
void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block)
{
/** You need to convert the right argument to a set.
* This can be a table name, a value, a value enumeration, or a subquery.
* The enumeration of values is parsed as a function `tuple`.
*/
const IAST & args = *node->arguments;
const ASTPtr & arg = args.children.at(1);
/// Already converted.
if (prepared_sets.count(arg->range))
return;
/// If the subquery or table name for SELECT.
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(arg.get());
if (typeid_cast<const ASTSubquery *>(arg.get()) || identifier)
{
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
String set_id = arg->getColumnName();
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).
if (identifier)
{
auto database_table = getDatabaseAndTableNameFromIdentifier(*identifier);
StoragePtr table = context.tryGetTable(database_table.first, database_table.second);
if (table)
{
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
if (storage_set)
{
prepared_sets[arg->range] = storage_set->getSet();
return;
}
}
}
SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id];
/// If you already created a Set with the same subquery / table.
if (subquery_for_set.set)
{
prepared_sets[arg->range] = subquery_for_set.set;
return;
}
SetPtr set = std::make_shared<Set>(set_size_limit, false);
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
* in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table.
* - this function shows the expression IN_data1.
*/
if (!subquery_for_set.source && no_storage_or_local)
{
auto interpreter = interpretSubquery(arg, context, subquery_depth, {});
subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; });
/** Why is LazyBlockInputStream used?
*
* The fact is that when processing a query of the form
* SELECT ... FROM remote_test WHERE column GLOBAL IN (subquery),
* if the distributed remote_test table contains localhost as one of the servers,
* the query will be interpreted locally again (and not sent over TCP, as in the case of a remote server).
*
* The query execution pipeline will be:
* CreatingSets
* subquery execution, filling the temporary table with _data1 (1)
* CreatingSets
* reading from the table _data1, creating the set (2)
* read from the table subordinate to remote_test.
*
* (The second part of the pipeline under CreateSets is a reinterpretation of the query inside StorageDistributed,
* the query differs in that the database name and tables are replaced with subordinates, and the subquery is replaced with _data1.)
*
* But when creating the pipeline, when creating the source (2), it will be found that the _data1 table is empty
* (because the query has not started yet), and empty source will be returned as the source.
* And then, when the query is executed, an empty set will be created in step (2).
*
* Therefore, we make the initialization of step (2) lazy
* - so that it does not occur until step (1) is completed, on which the table will be populated.
*
* Note: this solution is not very good, you need to think better.
*/
}
subquery_for_set.set = set;
prepared_sets[arg->range] = set;
}
else
{
/// An explicit enumeration of values in parentheses.
makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets);
}
}
}

View File

@ -0,0 +1,121 @@
#pragma once
#include <Parsers/StringRange.h>
namespace DB
{
class Context;
class ASTFunction;
struct ProjectionManipulatorBase;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
class Set;
using SetPtr = std::shared_ptr<Set>;
/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range.
/// It should be taken into account when we want to change AST part which contains sets.
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
class Join;
using JoinPtr = std::shared_ptr<Join>;
/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
struct SubqueryForSet
{
/// The source is obtained using the InterpreterSelectQuery subquery.
BlockInputStreamPtr source;
/// If set, build it from result.
SetPtr set;
JoinPtr join;
/// Apply this actions to joined block.
ExpressionActionsPtr joined_block_actions;
/// Rename column from joined block from this list.
NamesWithAliases joined_block_aliases;
/// If set, put the result into the table.
/// This is a temporary table for transferring to remote servers for distributed query processing.
StoragePtr table;
};
/// ID of subquery -> what to do with it.
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
/// The case of an explicit enumeration of values.
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set,
const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.
* The new action should be added to the highest possible level.
* For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)"
* calculation of the product must be done outside the lambda expression (it does not depend on x),
* and the calculation of the sum is inside (depends on x).
*/
struct ScopeStack
{
struct Level
{
ExpressionActionsPtr actions;
NameSet new_columns;
};
using Levels = std::vector<Level>;
Levels stack;
const Context & context;
ScopeStack(const ExpressionActionsPtr & actions, const Context & context_);
void pushLevel(const NamesAndTypesList & input_columns);
size_t getColumnLevel(const std::string & name);
void addAction(const ExpressionAction & action);
ExpressionActionsPtr popLevel();
const Block & getSampleBlock() const;
};
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
/// After AST is visited source ExpressionActions should be updated with popActionsLevel() method.
class ActionsVisitor
{
public:
ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_,
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr);
void visit(const ASTPtr & ast);
ExpressionActionsPtr popActionsLevel() { return actions_stack.popLevel(); }
private:
const Context & context;
SizeLimits set_size_limit;
size_t subquery_depth;
const NamesAndTypesList & source_columns;
PreparedSets & prepared_sets;
SubqueriesForSets & subqueries_for_sets;
const bool no_subqueries;
const bool only_consts;
const bool no_storage_or_local;
mutable size_t visit_depth;
std::ostream * ostr;
ScopeStack actions_stack;
std::shared_ptr<ProjectionManipulatorBase> projection_manipulator;
void makeSet(const ASTFunction * node, const Block & sample_block);
};
}

View File

@ -0,0 +1,94 @@
#pragma once
namespace DB
{
/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that.
class ArrayJoinedColumnsVisitor
{
public:
ArrayJoinedColumnsVisitor(NameToNameMap & array_join_name_to_alias_,
NameToNameMap & array_join_alias_to_name_,
NameToNameMap & array_join_result_to_source_)
: array_join_name_to_alias(array_join_name_to_alias_),
array_join_alias_to_name(array_join_alias_to_name_),
array_join_result_to_source(array_join_result_to_source_)
{}
void visit(ASTPtr & ast) const
{
if (!tryVisit<ASTTablesInSelectQuery>(ast) &&
!tryVisit<ASTIdentifier>(ast))
visitChildren(ast);
}
private:
NameToNameMap & array_join_name_to_alias;
NameToNameMap & array_join_alias_to_name;
NameToNameMap & array_join_result_to_source;
void visit(const ASTTablesInSelectQuery *, ASTPtr &) const
{}
void visit(const ASTIdentifier * node, ASTPtr &) const
{
if (node->general())
{
auto splitted = Nested::splitName(node->name); /// ParsedParams, Key1
if (array_join_alias_to_name.count(node->name))
{
/// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1
array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1
}
else if (array_join_alias_to_name.count(splitted.first) && !splitted.second.empty())
{
/// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP
array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1
= Nested::concatenateName(array_join_alias_to_name[splitted.first], splitted.second);
}
else if (array_join_name_to_alias.count(node->name))
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1.
* That is, the query uses the original array, replicated by itself.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
array_join_name_to_alias[node->name]] = node->name;
}
else if (array_join_name_to_alias.count(splitted.first) && !splitted.second.empty())
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
*/
array_join_result_to_source[ /// PP.Key1 -> ParsedParams.Key1
Nested::concatenateName(array_join_name_to_alias[splitted.first], splitted.second)] = node->name;
}
}
}
void visit(const ASTSubquery *, ASTPtr &) const
{}
void visit(const ASTSelectQuery *, ASTPtr &) const
{}
void visitChildren(ASTPtr & ast) const
{
for (auto & child : ast->children)
if (!tryVisit<ASTSubquery>(child) &&
!tryVisit<ASTSelectQuery>(child))
visit(child);
}
template <typename T>
bool tryVisit(ASTPtr & ast) const
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
return true;
}
return false;
}
};
}

View File

@ -35,7 +35,7 @@ static ASTPtr addTypeConversion(std::unique_ptr<ASTLiteral> && ast, const String
return res; return res;
} }
void ExecuteScalarSubqueriesVisitor::visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const void ExecuteScalarSubqueriesVisitor::visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode &) const
{ {
Context subquery_context = context; Context subquery_context = context;
Settings subquery_settings = context.getSettings(); Settings subquery_settings = context.getSettings();
@ -101,12 +101,12 @@ void ExecuteScalarSubqueriesVisitor::visit(ASTSubquery * subquery, ASTPtr & ast,
} }
void ExecuteScalarSubqueriesVisitor::visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const void ExecuteScalarSubqueriesVisitor::visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const
{ {
/// Don't descend into subqueries in FROM section. /// Don't descend into subqueries in FROM section.
} }
void ExecuteScalarSubqueriesVisitor::visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const void ExecuteScalarSubqueriesVisitor::visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const
{ {
/// Don't descend into subqueries in arguments of IN operator. /// Don't descend into subqueries in arguments of IN operator.
/// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them. /// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them.

View File

@ -54,9 +54,9 @@ private:
mutable size_t visit_depth; mutable size_t visit_depth;
std::ostream * ostr; std::ostream * ostr;
void visit(ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const; void visit(const ASTSubquery * subquery, ASTPtr & ast, const DumpASTNode & dump) const;
void visit(ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const; void visit(const ASTFunction * func, ASTPtr & ast, const DumpASTNode &) const;
void visit(ASTTableExpression *, ASTPtr &, const DumpASTNode &) const; void visit(const ASTTableExpression *, ASTPtr &, const DumpASTNode &) const;
void visitChildren(ASTPtr & ast) const void visitChildren(ASTPtr & ast) const
{ {
@ -67,7 +67,7 @@ private:
template <typename T> template <typename T>
bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const bool tryVisit(ASTPtr & ast, const DumpASTNode & dump) const
{ {
if (T * t = typeid_cast<T *>(ast.get())) if (const T * t = typeid_cast<const T *>(ast.get()))
{ {
visit(t, ast, dump); visit(t, ast, dump);
return true; return true;

File diff suppressed because it is too large Load Diff

View File

@ -2,10 +2,10 @@
#include <Interpreters/AggregateDescription.h> #include <Interpreters/AggregateDescription.h>
#include <Interpreters/Settings.h> #include <Interpreters/Settings.h>
#include <Core/Block.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/ProjectionManipulation.h> #include <Interpreters/ActionsVisitor.h>
#include <Parsers/StringRange.h>
#include <Core/Block.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
namespace DB namespace DB
@ -16,18 +16,9 @@ class Context;
class ExpressionActions; class ExpressionActions;
struct ExpressionActionsChain; struct ExpressionActionsChain;
class Join;
using JoinPtr = std::shared_ptr<Join>;
class IAST; class IAST;
using ASTPtr = std::shared_ptr<IAST>; using ASTPtr = std::shared_ptr<IAST>;
class Set;
using SetPtr = std::shared_ptr<Set>;
/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range.
/// It should be taken into account when we want to change AST part which contains sets.
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointersHash, StringRangePointersEqualTo>;
class IBlockInputStream; class IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>; using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
@ -39,58 +30,12 @@ class ASTFunction;
class ASTExpressionList; class ASTExpressionList;
class ASTSelectQuery; class ASTSelectQuery;
struct ProjectionManipulatorBase;
using ProjectionManipulatorPtr = std::shared_ptr<ProjectionManipulatorBase>;
/** Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section. inline SizeLimits getSetSizeLimits(const Settings & settings)
*/
struct SubqueryForSet
{ {
/// The source is obtained using the InterpreterSelectQuery subquery. return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode);
BlockInputStreamPtr source; }
/// If set, build it from result.
SetPtr set;
JoinPtr join;
/// Apply this actions to joined block.
ExpressionActionsPtr joined_block_actions;
/// Rename column from joined block from this list.
NamesWithAliases joined_block_aliases;
/// If set, put the result into the table.
/// This is a temporary table for transferring to remote servers for distributed query processing.
StoragePtr table;
};
/// ID of subquery -> what to do with it.
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
struct ScopeStack
{
struct Level
{
ExpressionActionsPtr actions;
NameSet new_columns;
};
using Levels = std::vector<Level>;
Levels stack;
const Context & context;
ScopeStack(const ExpressionActionsPtr & actions, const Context & context_);
void pushLevel(const NamesAndTypesList & input_columns);
size_t getColumnLevel(const std::string & name);
void addAction(const ExpressionAction & action);
ExpressionActionsPtr popLevel();
const Block & getSampleBlock() const;
};
/** Transforms an expression from a syntax tree into a sequence of actions to execute it. /** Transforms an expression from a syntax tree into a sequence of actions to execute it.
* *
@ -303,7 +248,6 @@ private:
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries. /// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
Tables external_tables; Tables external_tables;
size_t external_table_id = 1;
/// Predicate optimizer overrides the sub queries /// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false; bool rewrite_subqueries = false;
@ -341,21 +285,14 @@ private:
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast); void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast);
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const; bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
void makeSet(const ASTFunction * node, const Block & sample_block);
/// Adds a list of ALIAS columns from the table. /// Adds a list of ALIAS columns from the table.
void addAliasColumns(); void addAliasColumns();
/// Replacing scalar subqueries with constant values. /// Replacing scalar subqueries with constant values.
void executeScalarSubqueries(); void executeScalarSubqueries();
void executeScalarSubqueriesImpl(ASTPtr & ast);
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables(); void initGlobalSubqueriesAndExternalTables();
void initGlobalSubqueries(ASTPtr & ast);
/// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables.
void findExternalTables(ASTPtr & ast);
/** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section, /** Initialize InterpreterSelectQuery for a subquery in the GLOBAL IN/JOIN section,
* create a temporary table of type Memory and store it in the external_tables dictionary. * create a temporary table of type Memory and store it in the external_tables dictionary.
@ -363,20 +300,16 @@ private:
void addExternalStorage(ASTPtr & subquery_or_table_name); void addExternalStorage(ASTPtr & subquery_or_table_name);
void getArrayJoinedColumns(); void getArrayJoinedColumns();
void getArrayJoinedColumnsImpl(const ASTPtr & ast);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const; void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
bool isThereArrayJoin(const ASTPtr & ast); bool isThereArrayJoin(const ASTPtr & ast);
void getActionsImpl(const ASTPtr & ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack,
ProjectionManipulatorPtr projection_manipulator);
/// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns.
void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions);
void getRootActions(const ASTPtr & ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions); void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries); void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries);
@ -389,26 +322,12 @@ private:
void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions); void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions);
void assertNoAggregates(const ASTPtr & ast, const char * description); void assertNoAggregates(const ASTPtr & ast, const char * description);
/** Get a set of necessary columns to read from the table.
* In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified.
* The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table.
* Put in required_joined_columns the set of columns available from JOIN and needed.
*/
void getRequiredSourceColumnsImpl(const ASTPtr & ast,
const NameSet & available_columns, NameSet & required_source_columns, NameSet & ignored_names,
const NameSet & available_joined_columns, NameSet & required_joined_columns);
/// columns - the columns that are present before the transformations begin. /// columns - the columns that are present before the transformations begin.
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
void assertSelect() const; void assertSelect() const;
void assertAggregation() const; void assertAggregation() const;
/** Create Set from an explicit enumeration of values in the query.
* If create_ordered_set = true - create a data structure suitable for using the index.
*/
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set);
/** /**
* Create Set from a subuqery or a table expression in the query. The created set is suitable for using the index. * Create Set from a subuqery or a table expression in the query. The created set is suitable for using the index.
* The set will not be created if its size hits the limit. * The set will not be created if its size hits the limit.
@ -427,6 +346,8 @@ private:
* This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result. * This is the case when we have DISTINCT or arrayJoin: we require more columns in SELECT even if we need less columns in result.
*/ */
void removeUnneededColumnsFromSelectClause(); void removeUnneededColumnsFromSelectClause();
bool isRemoteStorage() const;
}; };
} }

View File

@ -0,0 +1,47 @@
#pragma once
namespace DB
{
/// Finds in the query the usage of external tables (as table identifiers). Fills in external_tables.
class ExternalTablesVisitor
{
public:
ExternalTablesVisitor(const Context & context_, Tables & tables)
: context(context_),
external_tables(tables)
{}
void visit(ASTPtr & ast) const
{
/// Traverse from the bottom. Intentionally go into subqueries.
for (auto & child : ast->children)
visit(child);
tryVisit<ASTIdentifier>(ast);
}
private:
const Context & context;
Tables & external_tables;
void visit(const ASTIdentifier * node, ASTPtr &) const
{
if (node->special())
if (StoragePtr external_storage = context.tryGetExternalTable(node->name))
external_tables[node->name] = external_storage;
}
template <typename T>
bool tryVisit(ASTPtr & ast) const
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
return true;
}
return false;
}
};
}

View File

@ -0,0 +1,167 @@
#pragma once
namespace DB
{
/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr.
class GlobalSubqueriesVisitor
{
public:
GlobalSubqueriesVisitor(const Context & context_, size_t subquery_depth_, bool is_remote_,
Tables & tables, SubqueriesForSets & subqueries_for_sets_, bool & has_global_subqueries_)
: context(context_),
subquery_depth(subquery_depth_),
is_remote(is_remote_),
external_table_id(1),
external_tables(tables),
subqueries_for_sets(subqueries_for_sets_),
has_global_subqueries(has_global_subqueries_)
{}
void visit(ASTPtr & ast) const
{
/// Recursive calls. We do not go into subqueries.
for (auto & child : ast->children)
if (!typeid_cast<ASTSelectQuery *>(child.get()))
visit(child);
/// Bottom-up actions.
if (tryVisit<ASTFunction>(ast) ||
tryVisit<ASTTablesInSelectQueryElement>(ast))
{}
}
private:
const Context & context;
size_t subquery_depth;
bool is_remote;
mutable size_t external_table_id = 1;
Tables & external_tables;
SubqueriesForSets & subqueries_for_sets;
bool & has_global_subqueries;
/// GLOBAL IN
void visit(ASTFunction * func, ASTPtr &) const
{
if (func->name == "globalIn" || func->name == "globalNotIn")
{
addExternalStorage(func->arguments->children.at(1));
has_global_subqueries = true;
}
}
/// GLOBAL JOIN
void visit(ASTTablesInSelectQueryElement * table_elem, ASTPtr &) const
{
if (table_elem->table_join
&& static_cast<const ASTTableJoin &>(*table_elem->table_join).locality == ASTTableJoin::Locality::Global)
{
addExternalStorage(table_elem->table_expression);
has_global_subqueries = true;
}
}
template <typename T>
bool tryVisit(ASTPtr & ast) const
{
if (T * t = typeid_cast<T *>(ast.get()))
{
visit(t, ast);
return true;
}
return false;
}
void addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) const
{
/// With nondistributed queries, creating temporary tables does not make sense.
if (!is_remote)
return;
ASTPtr subquery;
ASTPtr table_name;
ASTPtr subquery_or_table_name;
if (typeid_cast<const ASTIdentifier *>(subquery_or_table_name_or_table_expression.get()))
{
table_name = subquery_or_table_name_or_table_expression;
subquery_or_table_name = table_name;
}
else if (auto ast_table_expr = typeid_cast<const ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
{
if (ast_table_expr->database_and_table_name)
{
table_name = ast_table_expr->database_and_table_name;
subquery_or_table_name = table_name;
}
else if (ast_table_expr->subquery)
{
subquery = ast_table_expr->subquery;
subquery_or_table_name = subquery;
}
}
else if (typeid_cast<const ASTSubquery *>(subquery_or_table_name_or_table_expression.get()))
{
subquery = subquery_or_table_name_or_table_expression;
subquery_or_table_name = subquery;
}
if (!subquery_or_table_name)
throw Exception("Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method",
ErrorCodes::LOGICAL_ERROR);
if (table_name)
{
/// If this is already an external table, you do not need to add anything. Just remember its presence.
if (external_tables.end() != external_tables.find(static_cast<const ASTIdentifier &>(*table_name).name))
return;
}
/// Generate the name for the external table.
String external_table_name = "_data" + toString(external_table_id);
while (external_tables.count(external_table_name))
{
++external_table_id;
external_table_name = "_data" + toString(external_table_id);
}
auto interpreter = interpretSubquery(subquery_or_table_name, context, subquery_depth, {});
Block sample = interpreter->getSampleBlock();
NamesAndTypesList columns = sample.getNamesAndTypesList();
StoragePtr external_storage = StorageMemory::create(external_table_name, ColumnsDescription{columns});
external_storage->startup();
/** We replace the subquery with the name of the temporary table.
* It is in this form, the request will go to the remote server.
* This temporary table will go to the remote server, and on its side,
* instead of doing a subquery, you just need to read it.
*/
auto database_and_table_name = ASTIdentifier::createSpecial(external_table_name);
if (auto ast_table_expr = typeid_cast<ASTTableExpression *>(subquery_or_table_name_or_table_expression.get()))
{
ast_table_expr->subquery.reset();
ast_table_expr->database_and_table_name = database_and_table_name;
ast_table_expr->children.clear();
ast_table_expr->children.emplace_back(database_and_table_name);
}
else
subquery_or_table_name_or_table_expression = database_and_table_name;
external_tables[external_table_name] = external_storage;
subqueries_for_sets[external_table_name].source = interpreter->execute().in;
subqueries_for_sets[external_table_name].table = external_storage;
/** NOTE If it was written IN tmp_table - the existing temporary (but not external) table,
* then a new temporary table will be created (for example, _data1),
* and the data will then be copied to it.
* Maybe this can be avoided.
*/
}
};
}

View File

@ -5,7 +5,7 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ActionsVisitor.h>
#include <Interpreters/ProjectionManipulation.h> #include <Interpreters/ProjectionManipulation.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>

View File

@ -1,13 +1,18 @@
#pragma once #pragma once
#include <string> #include <string>
#include <vector>
#include <memory>
#include <unordered_map>
namespace DB namespace DB
{ {
class ExpressionAnalyzer;
class ExpressionAnalyzer;
class Context;
struct ScopeStack; struct ScopeStack;
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND;

View File

@ -0,0 +1,136 @@
#pragma once
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
/** Get a set of necessary columns to read from the table.
* In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified.
* The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table.
* Put in required_joined_columns the set of columns available from JOIN and needed.
*/
class RequiredSourceColumnsVisitor
{
public:
RequiredSourceColumnsVisitor(const NameSet & available_columns_, NameSet & required_source_columns_, NameSet & ignored_names_,
const NameSet & available_joined_columns_, NameSet & required_joined_columns_)
: available_columns(available_columns_),
required_source_columns(required_source_columns_),
ignored_names(ignored_names_),
available_joined_columns(available_joined_columns_),
required_joined_columns(required_joined_columns_)
{}
/** Find all the identifiers in the query.
* We will use depth first search in AST.
* In this case
* - for lambda functions we will not take formal parameters;
* - do not go into subqueries (they have their own identifiers);
* - there is some exception for the ARRAY JOIN clause (it has a slightly different identifiers);
* - we put identifiers available from JOIN in required_joined_columns.
*/
void visit(const ASTPtr & ast) const
{
if (!tryVisit<ASTIdentifier>(ast) &&
!tryVisit<ASTFunction>(ast))
visitChildren(ast);
}
private:
const NameSet & available_columns;
NameSet & required_source_columns;
NameSet & ignored_names;
const NameSet & available_joined_columns;
NameSet & required_joined_columns;
void visit(const ASTIdentifier * node, const ASTPtr &) const
{
if (node->general()
&& !ignored_names.count(node->name)
&& !ignored_names.count(Nested::extractTableName(node->name)))
{
if (!available_joined_columns.count(node->name)
|| available_columns.count(node->name)) /// Read column from left table if has.
required_source_columns.insert(node->name);
else
required_joined_columns.insert(node->name);
}
}
void visit(const ASTFunction * node, const ASTPtr & ast) const
{
if (node->name == "lambda")
{
if (node->arguments->children.size() != 2)
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTFunction * lambda_args_tuple = typeid_cast<ASTFunction *>(node->arguments->children.at(0).get());
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
/// You do not need to add formal parameters of the lambda expression in required_source_columns.
Names added_ignored;
for (auto & child : lambda_args_tuple->arguments->children)
{
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(child.get());
if (!identifier)
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
String & name = identifier->name;
if (!ignored_names.count(name))
{
ignored_names.insert(name);
added_ignored.push_back(name);
}
}
visit(node->arguments->children.at(1));
for (size_t i = 0; i < added_ignored.size(); ++i)
ignored_names.erase(added_ignored[i]);
return;
}
/// A special function `indexHint`. Everything that is inside it is not calculated
/// (and is used only for index analysis, see KeyCondition).
if (node->name == "indexHint")
return;
visitChildren(ast);
}
void visitChildren(const ASTPtr & ast) const
{
for (auto & child : ast->children)
{
/** We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns.
* There, `collectUsedColumns` will send us separately.
*/
if (!typeid_cast<const ASTSelectQuery *>(child.get())
&& !typeid_cast<const ASTArrayJoin *>(child.get())
&& !typeid_cast<const ASTTableExpression *>(child.get())
&& !typeid_cast<const ASTTableJoin *>(child.get()))
visit(child);
}
}
template <typename T>
bool tryVisit(const ASTPtr & ast) const
{
if (const T * t = typeid_cast<const T *>(ast.get()))
{
visit(t, ast);
return true;
}
return false;
}
};
}

View File

@ -0,0 +1,130 @@
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <Storages/IStorage.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/evaluateQualified.h>
namespace DB
{
std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)
{
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
const ASTSubquery * subquery = typeid_cast<const ASTSubquery *>(table_expression.get());
const ASTFunction * function = typeid_cast<const ASTFunction *>(table_expression.get());
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(table_expression.get());
if (!subquery && !table && !function)
throw Exception("Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery." , ErrorCodes::LOGICAL_ERROR);
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query.
* Constraints work instead
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query).
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr query;
if (table || function)
{
/// create ASTSelectQuery for "SELECT * FROM table" as if written by hand
const auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
query = select_with_union_query;
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
const auto select_query = std::make_shared<ASTSelectQuery>();
select_with_union_query->list_of_selects->children.push_back(select_query);
const auto select_expression_list = std::make_shared<ASTExpressionList>();
select_query->select_expression_list = select_expression_list;
select_query->children.emplace_back(select_query->select_expression_list);
NamesAndTypesList columns;
/// get columns list for target table
if (function)
{
auto query_context = const_cast<Context *>(&context.getQueryContext());
const auto & storage = query_context->executeTableFunction(table_expression);
columns = storage->getColumns().ordinary;
select_query->addTableFunction(*const_cast<ASTPtr *>(&table_expression));
}
else
{
auto database_table = getDatabaseAndTableNameFromIdentifier(*table);
const auto & storage = context.getTable(database_table.first, database_table.second);
columns = storage->getColumns().ordinary;
select_query->replaceDatabaseAndTable(database_table.first, database_table.second);
}
select_expression_list->children.reserve(columns.size());
/// manually substitute column names in place of asterisk
for (const auto & column : columns)
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
}
else
{
query = subquery->children.at(0);
/** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t
* This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns.
* Saving to the table is required for GLOBAL subqueries.
*
* To avoid this situation, we will rename the same columns.
*/
std::set<std::string> all_column_names;
std::set<std::string> assigned_column_names;
if (ASTSelectWithUnionQuery * select_with_union = typeid_cast<ASTSelectWithUnionQuery *>(query.get()))
{
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(select_with_union->list_of_selects->children.at(0).get()))
{
for (auto & expr : select->select_expression_list->children)
all_column_names.insert(expr->getAliasOrColumnName());
for (auto & expr : select->select_expression_list->children)
{
auto name = expr->getAliasOrColumnName();
if (!assigned_column_names.insert(name).second)
{
size_t i = 1;
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
++i;
name = name + "_" + toString(i);
expr = expr->clone(); /// Cancels fuse of the same expressions in the tree.
expr->setAlias(name);
all_column_names.insert(name);
assigned_column_names.insert(name);
}
}
}
}
}
return std::make_shared<InterpreterSelectWithUnionQuery>(
query, subquery_context, required_source_columns, QueryProcessingStage::Complete, subquery_depth + 1);
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
namespace DB
{
class Context;
std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns);
}

View File

@ -7,15 +7,15 @@ nav:
- '性能': 'introduction/performance.md' - '性能': 'introduction/performance.md'
- 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md' - 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md'
- '起步': - '入门指南':
- '部署运行': 'getting_started/index.md' - '部署运行': 'getting_started/index.md'
- '示例数据集': - '示例数据集':
- 'OnTime': 'getting_started/example_datasets/ontime.md' - '航班飞行数据': 'getting_started/example_datasets/ontime.md'
- 'New York Taxi data': 'getting_started/example_datasets/nyc_taxi.md' - '纽约市出租车数据': 'getting_started/example_datasets/nyc_taxi.md'
- 'AMPLab Big Data Benchmark': 'getting_started/example_datasets/amplab_benchmark.md' - 'AMPLab大数据基准测试': 'getting_started/example_datasets/amplab_benchmark.md'
- 'WikiStat': 'getting_started/example_datasets/wikistat.md' - '维基访问数据': 'getting_started/example_datasets/wikistat.md'
- 'Terabyte click logs from Criteo': 'getting_started/example_datasets/criteo.md' - 'Criteo TB级别点击日志': 'getting_started/example_datasets/criteo.md'
- 'Star Schema Benchmark': 'getting_started/example_datasets/star_schema.md' - 'Star Schema基准测试': 'getting_started/example_datasets/star_schema.md'
- '客户端': - '客户端':
- '介绍': 'interfaces/index.md' - '介绍': 'interfaces/index.md'

View File

@ -64,7 +64,7 @@
} }
body { body {
font: 400 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif;
} }
body.md-lang-zh { body.md-lang-zh {

View File

@ -7,6 +7,10 @@
"footer.next": "Next", "footer.next": "Next",
"meta.comments": "Comments", "meta.comments": "Comments",
"meta.source": "Source", "meta.source": "Source",
"nav.multi_page": "Multi page version",
"nav.pdf": "PDF version",
"nav.single_page": "Single page version",
"nav.source": "ClickHouse source code",
"search.placeholder": "Search", "search.placeholder": "Search",
"search.result.placeholder": "Type to start searching", "search.result.placeholder": "Type to start searching",
"search.result.none": "No matching documents", "search.result.none": "No matching documents",

View File

@ -8,6 +8,10 @@
"footer.next": "بعدی", "footer.next": "بعدی",
"meta.comments": "نظرات", "meta.comments": "نظرات",
"meta.source": "منبع", "meta.source": "منبع",
"nav.multi_page": "نسخه چند صفحه ای",
"nav.pdf": "نسخه PDF",
"nav.single_page": "نسخه تک صفحه",
"nav.source": "کد منبع کلیک",
"search.language": "", "search.language": "",
"search.pipeline.stopwords": false, "search.pipeline.stopwords": false,
"search.pipeline.trimmer": false, "search.pipeline.trimmer": false,

View File

@ -7,6 +7,10 @@
"footer.next": "Вперед", "footer.next": "Вперед",
"meta.comments": "Комментарии", "meta.comments": "Комментарии",
"meta.source": "Исходный код", "meta.source": "Исходный код",
"nav.multi_page": "Многостраничная версия",
"nav.pdf": "PDF версия",
"nav.single_page": "Одностраничная версия",
"nav.source": "Исходный код ClickHouse",
"search.placeholder": "Поиск", "search.placeholder": "Поиск",
"search.result.placeholder": "Начните печатать для поиска", "search.result.placeholder": "Начните печатать для поиска",
"search.result.none": "Совпадений не найдено", "search.result.none": "Совпадений не найдено",

View File

@ -7,6 +7,10 @@
"footer.next": "前进", "footer.next": "前进",
"meta.comments": "评论", "meta.comments": "评论",
"meta.source": "来源", "meta.source": "来源",
"nav.multi_page": "多页版本",
"nav.pdf": "PDF版本",
"nav.single_page": "单页版本",
"nav.source": "ClickHouse源代码",
"search.placeholder": "搜索", "search.placeholder": "搜索",
"search.result.placeholder": "键入以开始搜索", "search.result.placeholder": "键入以开始搜索",
"search.result.none": "没有找到符合条件的结果", "search.result.none": "没有找到符合条件的结果",

View File

@ -15,35 +15,19 @@
<ul id="md-extra-nav" class="md-nav__list" data-md-scrollfix> <ul id="md-extra-nav" class="md-nav__list" data-md-scrollfix>
<li class="md-nav__item md-nav__item--active"> <li class="md-nav__item md-nav__item--active">
{% if config.theme.language == 'ru' %}
{% if config.extra.single_page %} {% if config.extra.single_page %}
<a href="{{ base_url }}" class="md-nav__link md-nav__link--active">Многостраничная версия</a> <a href="{{ base_url }}" class="md-nav__link md-nav__link--active">{{ lang.t("nav.multi_page") }}</a>
{% else %} {% else %}
<a href="{{ base_url }}/single/" class="md-nav__link md-nav__link--active">Одностраничная версия</a> <a href="{{ base_url }}/single/" class="md-nav__link md-nav__link--active">{{ lang.t("nav.single_page") }}</a>
{% endif %}
{% else %}
{% if config.extra.single_page %}
<a href="{{ base_url }}" class="md-nav__link md-nav__link--active">Multi page version</a>
{% else %}
<a href="{{ base_url }}/single/" class="md-nav__link md-nav__link--active">Single page version</a>
{% endif %}
{% endif %} {% endif %}
</li> </li>
<li class="md-nav__item md-nav__item--active"> <li class="md-nav__item md-nav__item--active">
{% if config.theme.language == 'ru' %} <a href="{{ base_url }}/single/clickhouse_{{ config.theme.language }}.pdf" class="md-nav__link md-nav__link--active">{{ lang.t("nav.pdf") }}</a>
<a href="{{ base_url }}/single/clickhouse_{{ config.theme.language }}.pdf" class="md-nav__link md-nav__link--active">PDF версия</a>
{% else %}
<a href="{{ base_url }}/single/clickhouse_{{ config.theme.language }}.pdf" class="md-nav__link md-nav__link--active">PDF version</a>
{% endif %}
</li> </li>
{% if config.repo_url %} {% if config.repo_url %}
<li class="md-nav__item md-nav__item--active"> <li class="md-nav__item md-nav__item--active">
<a href="{{ config.repo_url }}" rel="external nofollow" target="_blank" class="md-nav__link"> <a href="{{ config.repo_url }}" rel="external nofollow" target="_blank" class="md-nav__link">
{% if config.theme.language == 'ru' %} {{ lang.t("nav.source") }}
Исходники ClickHouse
{% else %}
ClickHouse sources
{% endif %}
</a> </a>
</li> </li>
{% endif %} {% endif %}

View File

@ -1 +0,0 @@
../../en/data_types/array.md

View File

@ -0,0 +1,8 @@
<a name="data_type-array"></a>
# ArrayT
一个包含类型 T 的 数组Array。T 类型可以是任意类型,包括数组类型。
我们不推荐使用多维数组,因为多维数组并没有得到很好支持(比如,不能在 MergeTree 引擎的表中存储多维数组)。
[来源文章](https://clickhouse.yandex/docs/en/data_types/array/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/boolean.md

View File

@ -0,0 +1,5 @@
# Boolean values
没有单独的类型来存储 boolean 值。可以使用 UInt8 类型,取值限制为 0 或 1。
[来源文章](https://clickhouse.yandex/docs/en/data_types/boolean/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/date.md

View File

@ -0,0 +1,8 @@
# Date
Date 用两个字节来存储从 1970-01-01 到现在的日期值无符号的值。Date 允许存储的值 UNIX 纪元开始后的时间值,这个值上限会在编译阶段作为一个常量存储(当前只能到 2038 年,但可以拓展到 2106 年)。
Date 最小的值为 0000-00-00 00:00:00。
Date 中没有存储时区信息。
[来源文章](https://clickhouse.yandex/docs/en/data_types/date/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/datetime.md

View File

@ -0,0 +1,16 @@
# DateTime
包含时间和日期。用4个字节来存储 Unix 时间戳(无符号的值)。允许存储的时间范围同 Date 类型。最小的值为 0000-00-00 00:00:00。
DateTime 类型值精确到秒(不考虑闰秒)
## Time zones
在客户端或服务器启动时,会使用系统的时区来将时间日期从文本(划分成多个部分)到二进制之间相互转化。在文本格式中,有关夏令时信息不作存储。
默认情况下,客户端连接到服务的时候会使用服务端时区。可以通过客户端命令行参数 `--use_client_time_zone` 来设置使用客户端时区。
在所有的时区内只支持部分与UTC相差整数小时不考虑闰秒的时区。
因此,在处理文本日期时(例如,在保存文本转储时),请记住在夏时制期间可能会出现歧义,如果时区发生变化,可能会出现与数据匹配的问题。
[来源文章](https://clickhouse.yandex/docs/en/data_types/datetime/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/decimal.md

View File

@ -0,0 +1,100 @@
<a name="data_type-decimal"></a>
# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S)
有符号的定点数,在进行加减乘操作的过程中都会保留精度。对于除法,最低有效数字被丢弃(不舍入)。
## 参数
- P - 精度。范围值: [ 1 : 38 ],决定可以有多少个十进制数字(包括分数)。
- S - 范围。范围值: [ 0 : P ],决定小数的位数。
对于不同的 P 参数值 Decimal 表示,以下例子都是同义的:
- [ 1 : 9 ] 的 P - for Decimal32(S)
- [ 10 : 18 ] 的 P - for Decimal64(S)
- [ 19 : 38 ] 的 P - for Decimal128(S)
## 十进制值范围
- Decimal32(S) - ( -1 * 10^(9 - S), 1 * 10^(9 - S) )
- Decimal64(S) - ( -1 * 10^(18 - S), 1 * 10^(18 - S) )
- Decimal128(S) - ( -1 * 10^(38 - S), 1 * 10^(38 - S) )
例如, Decimal32(4) 可以表示 -99999.9999 到 99999.9999 范围内步数为 0.0001 的值。
## 内部表示方式
数据采用与自身位宽相同的有符号整数存储。这个数在内存中实际范围会高于上述范围,从 String 转换到十进制数的时候会做对应的检查。
Decimal32/Decimal64 通常处理速度要高于Decimal128这是因为当前通用CPU不支持128位的操作导致的。
## 运算以及结果的类型
对Decimal的二进制运算导致更宽的结果类型具有任何参数顺序
- Decimal64(S1) <op> Decimal32(S2) -> Decimal64(S)
- Decimal128(S1) <op> Decimal32(S2) -> Decimal128(S)
- Decimal128(S1) <op> Decimal64(S2) -> Decimal128(S)
精度变化的规则:
- 加,减: S = max(S1, S2).
- 相乘: S = S1 + S2.
- 相除S = S1.
对于 Decimal 和整数之间的类似操作,结果为一样参数值的 Decimal。
没有定义 Decimal 和 Float32/Float64 的操作。如果你真的需要他们,你可以某一个参数明确地转换为 toDecimal32toDecimal64 toDecimal128 或 toFloat32 toFloat64。注意这个操作会丢失精度并且类型转换是一个代价昂贵的操作。
有一些函数对 Decimal 进行操作后是返回 Float64 的例如var 或 stddev。计算的结果可能仍在 Decimal 中执行,这可能导致 Float64 和具有相同值的 Decimal 输入计算后的结果不同。
## 溢出检查
在对 Decimal 计算的过程中,数值会有可能溢出。分数中的过多数字被丢弃(不是舍入的)。 整数中的过多数字将导致异常。
```
SELECT toDecimal32(2, 4) AS x, x / 3
```
```
┌──────x─┬─divide(toDecimal32(2, 4), 3)─┐
│ 2.0000 │ 0.6666 │
└────────┴──────────────────────────────┘
```
```
SELECT toDecimal32(4.2, 8) AS x, x * x
```
```
DB::Exception: Scale is out of bounds.
```
```
SELECT toDecimal32(4.2, 8) AS x, 6 * x
```
```
DB::Exception: Decimal math overflow.
```
溢出检查会导致操作减慢。 如果已知溢出不可能,则使用`decimal_check_overflow`设置禁用检查是有意义的。 禁用检查并发生溢出时,结果将不正确:
```
SET decimal_check_overflow = 0;
SELECT toDecimal32(4.2, 8) AS x, 6 * x
```
```
┌──────────x─┬─multiply(6, toDecimal32(4.2, 8))─┐
│ 4.20000000 │ -17.74967296 │
└────────────┴──────────────────────────────────┘
```
溢出检查不仅会在数学运算中进行,还会在值比较中进行:
```
SELECT toDecimal32(1, 8) < 100
```
```
DB::Exception: Can't compare.
```
[来源文章](https://clickhouse.yandex/docs/en/data_types/decimal/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/enum.md

View File

@ -0,0 +1,34 @@
# Enum
`Enum8` 或者 `Enum16`。 一组有限的字符串值,比 `String` 类型的存储更加有效。
示例:
```text
Enum8('hello' = 1, 'world' = 2)
```
- 一个类型可以表示两个值: 'hello' and 'world'。
`Enum8` 类型的每个值范围是 `-128 ... 127``Enum16` 类型的每个值范围是 `-32768 ... 32767`。所有的字符串或者数字都必须是不一样的。允许存在空字符串。如果某个 Enum 类型被指定了(在表定义的时候),数字可以是任意顺序。然而,顺序并不重要。
(译者注:如 `Enum8('he o' = 3, 'wld' = 1)` 也是合法的)
在内存中Enum 类型当做 `Int8` or `Int16` 对应的数值来存储。
当以文本方式读取的时候ClickHouse 将值解析成字符串然后去 Enum 值的集合中搜索对应字符串。如果没有找到,会抛出异常。当读取文本格式的时候,会根据读取到的字符串去找对应的数值。如果没有找到,会抛出异常。
当以文本方式写入的时候ClickHouse 将值解析成字符串写入。如果 column 数据包含垃圾不是从有用集合含有的数值会抛弃异常。Enum 类型以二进制读取和写入的方式与 Int8 和 Int16 类型一样的。
隐式默认值是对应类型的最小值。
`ORDER BY`, `GROUP BY`, `IN`, `DISTINCT`Enums 和对应数值是一样的工作方式。比如, ORDER BY 会将它们按数值排序。对 Enums 类型使用相同和比较操作符都与操作它们隐含的数值是一样的。
Enum 值不能和数值比较大小。Enums 可以和一个常量字符串比较大小。如果字符串不是一个可用的 Enum 值,会抛出异常。可以使用 IN 运算符来判断一个 Enum 是否存在于某个 Enum 集合中,其中集合中的 Enum 需要用字符串表示。
大部分数字运算和字符串运算都没有给 Enum 类型定义比如Enum 类型不能和一个数相加,或进行字符串连接的操作,但是可以通过 toString 方法返回它对应的字符串。
Enum 值使用 `toT` 函数可以转换成数值类型,其中 T 是一个数值类型。若 `T` 恰好对应 Enum 的底层数值类型,这个转换是零消耗的。
Enum 类型可以被 `ALTER` 无成本地修改对应集合的值。可以通过 `ALTER` 操作来增加或删除 Enum 的成员(只要表没有用到该值,删除都是安全的)。作为安全保障,改变之前使用过的 Enum 成员将抛出异常。
通过 `ALTER` 操作,可以将 `Enum8` 转成 `Enum16`,反之亦然,就像 `Int8``Int16`一样。
[来源文章](https://clickhouse.yandex/docs/en/data_types/enum/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/fixedstring.md

View File

@ -0,0 +1,9 @@
# FixedString(N)
固定长度 N 的字符串。N 必须是严格的正自然数。
当服务端读取长度小于 N 的字符串时候(譬如解析插入的数据),字符串通过在末尾添加空字节来达到 N 字节长度。
当服务端读取长度大于 N 的字符串时候,会返回一个错误。
当服务端写入一个字符串的时候(譬如写入数据到 SELECT 查询结果中),末尾的空字节会被修剪掉。
注意这种方式与 MYSQL 的 CHAR 类型是不一样的MYSQL 的字符串会以空格填充,然后输出的时候空格会被修剪)。
很少函数会使用 `FixedString(N)` 来代替 `String`,因此它并不是很方便。

View File

@ -1 +0,0 @@
../../en/data_types/float.md

View File

@ -0,0 +1,72 @@
# Float32, Float64
[浮点数](https://en.wikipedia.org/wiki/IEEE_754)。
类型与以下 C 类型是相同的:
- `Float32` - `float`
- `Float64` - ` double`
我们建议,如果可能的话尽量用整形来存储数据。比如,将一定精度的浮点数转换成整形,譬如货币金额或者毫秒单位的加载时间。
## 使用浮点数
- 对浮点数进行计算可能引起四舍五入的误差。
```sql
SELECT 1 - 0.9
```
```
┌───────minus(1, 0.9)─┐
│ 0.09999999999999998 │
└─────────────────────┘
```
- 计算的结果取决于计算方式(处理器类型和计算机系统架构)
- 浮点数计算可能出现这样的结果,比如 "infinity" `Inf` 和 "not-a-number" `NaN`)。对浮点数计算的时候应该考虑到这点。
- 当一行行阅读浮点数的时候,浮点数的结果可能不是机器最近显示的数值。
## NaN 和 Inf
相比于 SQLClickHouse 支持以下几种浮点数分类:
- `Inf` 正无穷。
```sql
SELECT 0.5 / 0
```
```
┌─divide(0.5, 0)─┐
│ inf │
└────────────────┘
```
- `-Inf` 负无穷。
```sql
SELECT -0.5 / 0
```
```
┌─divide(-0.5, 0)─┐
│ -inf │
└─────────────────┘
```
- `NaN` 非数字。
```
SELECT 0 / 0
```
```
┌─divide(0, 0)─┐
│ nan │
└──────────────┘
```
可以在[ORDER BY 子句](../query_language/select.md#query_language-queries-order_by) 查看更多关于 ` NaN` 排序的规则。
[来源文章](https://clickhouse.yandex/docs/en/data_types/float/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/index.md

View File

@ -0,0 +1,9 @@
<a name="data_types"></a>
# 数据类型
ClickHouse 可以在数据表中存储多种数据类型。
本节将描述 ClickHouse 支持的数据类型以及在使用 与/或 运算实现他们时候的特殊考虑(如果有)。
[来源文章](https://clickhouse.yandex/docs/en/data_types/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/int_uint.md

View File

@ -0,0 +1,22 @@
<a name="data_type-int"></a>
# UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64
固定长度的整型,包括有符号整型或无符号整型。
## 整型范围
- Int8 - [-128 : 127]
- Int16 - [-32768 : 32767]
- Int32 - [-2147483648 : 2147483647]
- Int64 - [-9223372036854775808 : 9223372036854775807]
## 无符号整型范围
- UInt8 - [0 : 255]
- UInt16 - [0 : 65535]
- UInt32 - [0 : 4294967295]
- UInt64 - [0 : 18446744073709551615]
[来源文章](https://clickhouse.yandex/docs/en/data_types/int_uint/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/nested_data_structures/aggregatefunction.md

View File

@ -0,0 +1,5 @@
# AggregateFunction(name, types_of_arguments...)
表示聚合函数中的中间状态。可以在聚合函数中通过 '-State' 后缀来访问它。更多信息,参考 "AggregatingMergeTree"。
[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/nested_data_structures/index.md

View File

@ -0,0 +1,4 @@
# 嵌套数据类型
[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/nested_data_structures/nested.md

View File

@ -0,0 +1,99 @@
# Nested(Name1 Type1, Name2 Type2, ...)
类似嵌套表的嵌套数据结构。嵌套数据结构的参数(列名和列类型)与 CREATE 查询类似。每个表可以对应任意多行嵌套数据结构。
示例:
```sql
CREATE TABLE test.visits
(
CounterID UInt32,
StartDate Date,
Sign Int8,
IsNew UInt8,
VisitID UInt64,
UserID UInt64,
...
Goals Nested
(
ID UInt32,
Serial UInt32,
EventTime DateTime,
Price Int64,
OrderID String,
CurrencyID UInt32
),
...
) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)
```
上述示例声明了 `Goals` 这种嵌套数据结构,它包含访客转化相关的数据(访客达到的目标)。在 'visits' 表中每一行都可以对应零个或者任意个转化数据。
只支持一级嵌套。嵌套结构的列中若列的类型是数组类型那么该列其实和多维数组是相同的所以目前嵌套层级的支持很局限MergeTree 引擎中不支持存储这样的列)
大多数情况下,处理嵌套数据结构时,会指定一个单独的列。为了这样实现,列的名称会与点号连接起来。这些列构成了一组匹配类型。在同一条嵌套数据中,所有的列都具有相同的长度。
示例:
```sql
SELECT
Goals.ID,
Goals.EventTime
FROM test.visits
WHERE CounterID = 101500 AND length(Goals.ID) < 5
LIMIT 10
```
```text
┌─Goals.ID───────────────────────┬─Goals.EventTime───────────────────────────────────────────────────────────────────────────┐
│ [1073752,591325,591325] │ ['2014-03-17 16:38:10','2014-03-17 16:38:48','2014-03-17 16:42:27'] │
│ [1073752] │ ['2014-03-17 00:28:25'] │
│ [1073752] │ ['2014-03-17 10:46:20'] │
│ [1073752,591325,591325,591325] │ ['2014-03-17 13:59:20','2014-03-17 22:17:55','2014-03-17 22:18:07','2014-03-17 22:18:51'] │
│ [] │ [] │
│ [1073752,591325,591325] │ ['2014-03-17 11:37:06','2014-03-17 14:07:47','2014-03-17 14:36:21'] │
│ [] │ [] │
│ [] │ [] │
│ [591325,1073752] │ ['2014-03-17 00:46:05','2014-03-17 00:46:05'] │
│ [1073752,591325,591325,591325] │ ['2014-03-17 13:28:33','2014-03-17 13:30:26','2014-03-17 18:51:21','2014-03-17 18:51:45'] │
└────────────────────────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘
```
所以可以简单地把嵌套数据结构当做是所有列都是相同长度的多列数组。
SELECT 查询只有在使用 ARRAY JOIN 的时候才可以指定整个嵌套数据结构的名称。更多信息,参考 "ARRAY JOIN 子句"。示例:
```sql
SELECT
Goal.ID,
Goal.EventTime
FROM test.visits
ARRAY JOIN Goals AS Goal
WHERE CounterID = 101500 AND length(Goals.ID) < 5
LIMIT 10
```
```text
┌─Goal.ID─┬──────Goal.EventTime─┐
│ 1073752 │ 2014-03-17 16:38:10 │
│ 591325 │ 2014-03-17 16:38:48 │
│ 591325 │ 2014-03-17 16:42:27 │
│ 1073752 │ 2014-03-17 00:28:25 │
│ 1073752 │ 2014-03-17 10:46:20 │
│ 1073752 │ 2014-03-17 13:59:20 │
│ 591325 │ 2014-03-17 22:17:55 │
│ 591325 │ 2014-03-17 22:18:07 │
│ 591325 │ 2014-03-17 22:18:51 │
│ 1073752 │ 2014-03-17 11:37:06 │
└─────────┴─────────────────────┘
```
不能对整个嵌套数据结构进行 SELECT 。只能显式地列出它的一部分列。
对于 INSERT 查询,可以单独地传入所有嵌套数据结构中的列数组(假如它们是单独的列数组)。在插入过程中,系统会检查它们是否有相同的长度。
对于 DESCRIBE 查询,嵌套数据结构中的列会以相同的方式分别列出来。
ALTER 查询对嵌套数据结构的操作非常局限。
[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/nested/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/nullable.md

View File

@ -0,0 +1,59 @@
<a name="data_type-nullable"></a>
# Nullable(TypeName)
允许加上一个符号 ([NULL](../query_language/syntax.md#null-literal)) 表示“缺失值”和 `TypeName` 允许的正常值。 例如,`NullableInt8` 类型列可以存储 `Int8` 类型值,而没有值的行将存储 `NULL`
对于`TypeName`,您不能使用复合数据类型 [Array](array.md#data_type is array) 和 [Tuple](tuple.md#data_type-tuple)。 复合数据类型可以包含 `Nullable` 类型值,例如 `Array (Nullable(Int8))`
`Nullable` 类型字段不能包含在表索引中。
除非在 ClickHouse 服务器配置中另有说明,否则 `NULL` 是任何 `Nullable` 类型的默认值。
## 存储特性
要在表的列中存储 `Nullable` 类型值ClickHouse 除了使用带有值的普通文件外,还使用带有 `NULL` 掩码的单独文件。 掩码文件中的条目允许 ClickHouse 区分每个表行的 `NULL` 和相应数据类型的默认值。 由于附加了新文件,`Nullable` 列与类似的普通文件相比消耗额外的存储空间。
!!! info "Note"
使用 `Nullable` 几乎总是会对性能产生负面影响,在设计数据库时请记住这一点。
## 使用案例
```
:) CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog
CREATE TABLE t_null
(
x Int8,
y Nullable(Int8)
)
ENGINE = TinyLog
Ok.
0 rows in set. Elapsed: 0.012 sec.
:) INSERT INTO t_null VALUES (1, NULL)
INSERT INTO t_null VALUES
Ok.
1 rows in set. Elapsed: 0.007 sec.
:) SELECT x + y FROM t_null
SELECT x + y
FROM t_null
┌─plus(x, y)─┐
│ ᴺᵁᴸᴸ │
│ 5 │
└────────────┘
2 rows in set. Elapsed: 0.144 sec.
```
[来源文章](https://clickhouse.yandex/docs/en/data_types/nullable/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/special_data_types/expression.md

View File

@ -0,0 +1,5 @@
# Expression
用在高阶函数中的 lambda 表达式中。
[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/expression/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/special_data_types/index.md

View File

@ -0,0 +1,5 @@
# Special Data Types
特殊数据类型的值既不能存在表中也不能在结果中输出,但是可以被用于表示查询过程中的中间结果。
[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/special_data_types/nothing.md

View File

@ -0,0 +1,24 @@
<a name="special_data_type-nothing"></a>
# Nothing
此数据类型的唯一目的是表示不是期望值的情况。 所以你不能创建一个` Nothing` 类型的值。
例如, [NULL](../../query_language/syntax.md#null-literal) 有 `Nullable(Nothing)`。更多参见[Nullable](../../data_types/nullable.md#data_type-nullable).
`Nothing` 类型也可用于表示空数组:
```bash
:) SELECT toTypeName(array())
SELECT toTypeName([])
┌─toTypeName(array())─┐
│ Array(Nothing) │
└─────────────────────┘
1 rows in set. Elapsed: 0.062 sec.
```
[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/nothing/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/data_types/special_data_types/set.md

View File

@ -0,0 +1,5 @@
# Set
可以用在 IN 表达式的右半边。
[来源文章](https://clickhouse.yandex/docs/en/data_types/special_data_types/set/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/string.md

View File

@ -0,0 +1,13 @@
# String
字符串可以任意长度的。它可以包含任意的字节组合,包括空字符。
String 类型替代了其他 DBMSs 中的 VARCHAR BLOB CLOB 等类型。
## 编码
ClickHouse 没有编码的概念。字符串可以是任意的字节组合,按它们原本的方式进行存储和输出。
若需存储文本,我们建议使用 UTF-8 编码。至少如果你的终端使用UTF-8推荐这样读写就不需要进行任何的转换了。
同样,对不同的编码文本 ClickHouse 会有不同处理字符串的函数。
比如,`length` 函数可以计算字符串包含的字节数组的长度,然而 `lengthUTF8` 函数是假设字符串以 UTF-8 编码,计算的是字符串包含的 Unicode 字符的长度。
[来源文章](https://clickhouse.yandex/docs/en/data_types/string/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/data_types/tuple.md

View File

@ -0,0 +1,53 @@
# Tuple(T1, T2, ...)
元组,元组中的每个元素都有独立的[类型](index.md#data_types)。
元组不能写入表中(除 Memory 表外。它们会用于临时列分组。在某个查询中IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,参见 [IN 操作符] (../query_language/select.md#in_operators) 和 [高阶函数](../query_language/functions/higher_order_functions.md#higher_order_functions)。
## 创建元组
可以用下面函数来创建元组:
```
tuple(T1, T2, ...)
```
创建元组的示例:
```
:) SELECT tuple(1,'a') AS x, toTypeName(x)
SELECT
(1, 'a') AS x,
toTypeName(x)
┌─x───────┬─toTypeName(tuple(1, 'a'))─┐
│ (1,'a') │ Tuple(UInt8, String) │
└─────────┴───────────────────────────┘
1 rows in set. Elapsed: 0.021 sec.
```
## 元组中的数据类型
当动态创建元组时ClickHouse 会自动为元组的每一个参数赋予最小可表达的类型。如果该参数是 [NULL](../query_language/syntax.md#null-literal),那么这个元组对应元素就是 [Nullable](nullable.md#data_type-nullable)。
自动识别数据类型的示例:
```
SELECT tuple(1, NULL) AS x, toTypeName(x)
SELECT
(1, NULL) AS x,
toTypeName(x)
┌─x────────┬─toTypeName(tuple(1, NULL))──────┐
│ (1,NULL) │ Tuple(UInt8, Nullable(Nothing)) │
└──────────┴─────────────────────────────────┘
1 rows in set. Elapsed: 0.002 sec.
```
[来源文章](https://clickhouse.yandex/docs/en/data_types/tuple/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/amplab_benchmark.md

View File

@ -0,0 +1,123 @@
# AMPLab 大数据基准测试
参考 <https://amplab.cs.berkeley.edu/benchmark/>
需要您在<https://aws.amazon.com>注册一个免费的账号。注册时需要您提供信用卡、邮箱、电话等信息。之后可以在<https://console.aws.amazon.com/iam/home?nc2=h_m_sc#security_credential>获取新的访问密钥
在控制台运行以下命令:
```bash
sudo apt-get install s3cmd
mkdir tiny; cd tiny;
s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/tiny/ .
cd ..
mkdir 1node; cd 1node;
s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/1node/ .
cd ..
mkdir 5nodes; cd 5nodes;
s3cmd sync s3://big-data-benchmark/pavlo/text-deflate/5nodes/ .
cd ..
```
在ClickHouse运行如下查询
``` sql
CREATE TABLE rankings_tiny
(
pageURL String,
pageRank UInt32,
avgDuration UInt32
) ENGINE = Log;
CREATE TABLE uservisits_tiny
(
sourceIP String,
destinationURL String,
visitDate Date,
adRevenue Float32,
UserAgent String,
cCode FixedString(3),
lCode FixedString(6),
searchWord String,
duration UInt32
) ENGINE = MergeTree(visitDate, visitDate, 8192);
CREATE TABLE rankings_1node
(
pageURL String,
pageRank UInt32,
avgDuration UInt32
) ENGINE = Log;
CREATE TABLE uservisits_1node
(
sourceIP String,
destinationURL String,
visitDate Date,
adRevenue Float32,
UserAgent String,
cCode FixedString(3),
lCode FixedString(6),
searchWord String,
duration UInt32
) ENGINE = MergeTree(visitDate, visitDate, 8192);
CREATE TABLE rankings_5nodes_on_single
(
pageURL String,
pageRank UInt32,
avgDuration UInt32
) ENGINE = Log;
CREATE TABLE uservisits_5nodes_on_single
(
sourceIP String,
destinationURL String,
visitDate Date,
adRevenue Float32,
UserAgent String,
cCode FixedString(3),
lCode FixedString(6),
searchWord String,
duration UInt32
) ENGINE = MergeTree(visitDate, visitDate, 8192);
```
回到控制台运行如下命令:
```bash
for i in tiny/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_tiny FORMAT CSV"; done
for i in tiny/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_tiny FORMAT CSV"; done
for i in 1node/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_1node FORMAT CSV"; done
for i in 1node/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_1node FORMAT CSV"; done
for i in 5nodes/rankings/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO rankings_5nodes_on_single FORMAT CSV"; done
for i in 5nodes/uservisits/*.deflate; do echo $i; zlib-flate -uncompress < $i | clickhouse-client --host=example-perftest01j --query="INSERT INTO uservisits_5nodes_on_single FORMAT CSV"; done
```
简单的查询示例:
``` sql
SELECT pageURL, pageRank FROM rankings_1node WHERE pageRank > 1000
SELECT substring(sourceIP, 1, 8), sum(adRevenue) FROM uservisits_1node GROUP BY substring(sourceIP, 1, 8)
SELECT
sourceIP,
sum(adRevenue) AS totalRevenue,
avg(pageRank) AS pageRank
FROM rankings_1node ALL INNER JOIN
(
SELECT
sourceIP,
destinationURL AS pageURL,
adRevenue
FROM uservisits_1node
WHERE (visitDate > '1980-01-01') AND (visitDate < '1980-04-01')
) USING pageURL
GROUP BY sourceIP
ORDER BY totalRevenue DESC
LIMIT 1
```
[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/amplab_benchmark/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/criteo.md

View File

@ -0,0 +1,75 @@
# Criteo TB级别点击日志
可以从<http://labs.criteo.com/downloads/download-terabyte-click-logs/>上下载数据
创建原始数据对应的表结构:
``` sql
CREATE TABLE criteo_log (date Date, clicked UInt8, int1 Int32, int2 Int32, int3 Int32, int4 Int32, int5 Int32, int6 Int32, int7 Int32, int8 Int32, int9 Int32, int10 Int32, int11 Int32, int12 Int32, int13 Int32, cat1 String, cat2 String, cat3 String, cat4 String, cat5 String, cat6 String, cat7 String, cat8 String, cat9 String, cat10 String, cat11 String, cat12 String, cat13 String, cat14 String, cat15 String, cat16 String, cat17 String, cat18 String, cat19 String, cat20 String, cat21 String, cat22 String, cat23 String, cat24 String, cat25 String, cat26 String) ENGINE = Log
```
下载数据:
```bash
for i in {00..23}; do echo $i; zcat datasets/criteo/day_${i#0}.gz | sed -r 's/^/2000-01-'${i/00/24}'\t/' | clickhouse-client --host=example-perftest01j --query="INSERT INTO criteo_log FORMAT TabSeparated"; done
```
创建转换后的数据对应的表结构:
``` sql
CREATE TABLE criteo
(
date Date,
clicked UInt8,
int1 Int32,
int2 Int32,
int3 Int32,
int4 Int32,
int5 Int32,
int6 Int32,
int7 Int32,
int8 Int32,
int9 Int32,
int10 Int32,
int11 Int32,
int12 Int32,
int13 Int32,
icat1 UInt32,
icat2 UInt32,
icat3 UInt32,
icat4 UInt32,
icat5 UInt32,
icat6 UInt32,
icat7 UInt32,
icat8 UInt32,
icat9 UInt32,
icat10 UInt32,
icat11 UInt32,
icat12 UInt32,
icat13 UInt32,
icat14 UInt32,
icat15 UInt32,
icat16 UInt32,
icat17 UInt32,
icat18 UInt32,
icat19 UInt32,
icat20 UInt32,
icat21 UInt32,
icat22 UInt32,
icat23 UInt32,
icat24 UInt32,
icat25 UInt32,
icat26 UInt32
) ENGINE = MergeTree(date, intHash32(icat1), (date, intHash32(icat1)), 8192)
```
将第一张表中的原始数据转化写入到第二张表中去:
``` sql
INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log;
DROP TABLE criteo_log;
```
[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/criteo/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/nyc_taxi.md

File diff suppressed because one or more lines are too long

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/ontime.md

View File

@ -0,0 +1,318 @@
<a name="example_datasets-ontime"></a>
# 航班飞行数据
下载数据:
```bash
for s in `seq 1987 2017`
do
for m in `seq 1 12`
do
wget http://transtats.bts.gov/PREZIP/On_Time_On_Time_Performance_${s}_${m}.zip
done
done
```
(引用 <https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh> )
创建表结构:
```sql
CREATE TABLE `ontime` (
`Year` UInt16,
`Quarter` UInt8,
`Month` UInt8,
`DayofMonth` UInt8,
`DayOfWeek` UInt8,
`FlightDate` Date,
`UniqueCarrier` FixedString(7),
`AirlineID` Int32,
`Carrier` FixedString(2),
`TailNum` String,
`FlightNum` String,
`OriginAirportID` Int32,
`OriginAirportSeqID` Int32,
`OriginCityMarketID` Int32,
`Origin` FixedString(5),
`OriginCityName` String,
`OriginState` FixedString(2),
`OriginStateFips` String,
`OriginStateName` String,
`OriginWac` Int32,
`DestAirportID` Int32,
`DestAirportSeqID` Int32,
`DestCityMarketID` Int32,
`Dest` FixedString(5),
`DestCityName` String,
`DestState` FixedString(2),
`DestStateFips` String,
`DestStateName` String,
`DestWac` Int32,
`CRSDepTime` Int32,
`DepTime` Int32,
`DepDelay` Int32,
`DepDelayMinutes` Int32,
`DepDel15` Int32,
`DepartureDelayGroups` String,
`DepTimeBlk` String,
`TaxiOut` Int32,
`WheelsOff` Int32,
`WheelsOn` Int32,
`TaxiIn` Int32,
`CRSArrTime` Int32,
`ArrTime` Int32,
`ArrDelay` Int32,
`ArrDelayMinutes` Int32,
`ArrDel15` Int32,
`ArrivalDelayGroups` Int32,
`ArrTimeBlk` String,
`Cancelled` UInt8,
`CancellationCode` FixedString(1),
`Diverted` UInt8,
`CRSElapsedTime` Int32,
`ActualElapsedTime` Int32,
`AirTime` Int32,
`Flights` Int32,
`Distance` Int32,
`DistanceGroup` UInt8,
`CarrierDelay` Int32,
`WeatherDelay` Int32,
`NASDelay` Int32,
`SecurityDelay` Int32,
`LateAircraftDelay` Int32,
`FirstDepTime` String,
`TotalAddGTime` String,
`LongestAddGTime` String,
`DivAirportLandings` String,
`DivReachedDest` String,
`DivActualElapsedTime` String,
`DivArrDelay` String,
`DivDistance` String,
`Div1Airport` String,
`Div1AirportID` Int32,
`Div1AirportSeqID` Int32,
`Div1WheelsOn` String,
`Div1TotalGTime` String,
`Div1LongestGTime` String,
`Div1WheelsOff` String,
`Div1TailNum` String,
`Div2Airport` String,
`Div2AirportID` Int32,
`Div2AirportSeqID` Int32,
`Div2WheelsOn` String,
`Div2TotalGTime` String,
`Div2LongestGTime` String,
`Div2WheelsOff` String,
`Div2TailNum` String,
`Div3Airport` String,
`Div3AirportID` Int32,
`Div3AirportSeqID` Int32,
`Div3WheelsOn` String,
`Div3TotalGTime` String,
`Div3LongestGTime` String,
`Div3WheelsOff` String,
`Div3TailNum` String,
`Div4Airport` String,
`Div4AirportID` Int32,
`Div4AirportSeqID` Int32,
`Div4WheelsOn` String,
`Div4TotalGTime` String,
`Div4LongestGTime` String,
`Div4WheelsOff` String,
`Div4TailNum` String,
`Div5Airport` String,
`Div5AirportID` Int32,
`Div5AirportSeqID` Int32,
`Div5WheelsOn` String,
`Div5TotalGTime` String,
`Div5LongestGTime` String,
`Div5WheelsOff` String,
`Div5TailNum` String
) ENGINE = MergeTree(FlightDate, (Year, FlightDate), 8192)
```
加载数据:
```bash
for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done
```
查询:
Q0.
```sql
select avg(c1) from (select Year, Month, count(*) as c1 from ontime group by Year, Month);
```
Q1. 查询从2000年到2008年每天的航班数
```sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC;
```
Q2. 查询从2000年到2008年每周延误超过10分钟的航班数。
```sql
SELECT DayOfWeek, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY DayOfWeek ORDER BY c DESC
```
Q3. 查询2000年到2008年每个机场延误超过10分钟以上的次数
```sql
SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Year <= 2008 GROUP BY Origin ORDER BY c DESC LIMIT 10
```
Q4. 查询2007年各航空公司延误超过10分钟以上的次数
```sql
SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC
```
Q5. 查询2007年各航空公司延误超过10分钟以上的百分比
```sql
SELECT Carrier, c, c2, c*1000/c2 as c3
FROM
(
SELECT
Carrier,
count(*) AS c
FROM ontime
WHERE DepDelay>10
AND Year=2007
GROUP BY Carrier
)
ANY INNER JOIN
(
SELECT
Carrier,
count(*) AS c2
FROM ontime
WHERE Year=2007
GROUP BY Carrier
) USING Carrier
ORDER BY c3 DESC;
```
更好的查询版本:
```sql
SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier
```
Q6. 同上一个查询一致,只是查询范围扩大到2000年到2008年
```sql
SELECT Carrier, c, c2, c*1000/c2 as c3
FROM
(
SELECT
Carrier,
count(*) AS c
FROM ontime
WHERE DepDelay>10
AND Year >= 2000 AND Year <= 2008
GROUP BY Carrier
)
ANY INNER JOIN
(
SELECT
Carrier,
count(*) AS c2
FROM ontime
WHERE Year >= 2000 AND Year <= 2008
GROUP BY Carrier
) USING Carrier
ORDER BY c3 DESC;
```
更好的查询版本:
```sql
SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier
```
Q7. 每年航班延误超过10分钟的百分比
```sql
SELECT Year, c1/c2
FROM
(
select
Year,
count(*)*1000 as c1
from ontime
WHERE DepDelay>10
GROUP BY Year
)
ANY INNER JOIN
(
select
Year,
count(*) as c2
from ontime
GROUP BY Year
) USING (Year)
ORDER BY Year
```
更好的查询版本:
```sql
SELECT Year, avg(DepDelay > 10) FROM ontime GROUP BY Year ORDER BY Year
```
Q8. 每年更受人们喜爱的目的地
```sql
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime WHERE Year >= 2000 and Year <= 2010 GROUP BY DestCityName ORDER BY u DESC LIMIT 10;
```
Q9.
```sql
select Year, count(*) as c1 from ontime group by Year;
```
Q10.
```sql
select
min(Year), max(Year), Carrier, count(*) as cnt,
sum(ArrDelayMinutes>30) as flights_delayed,
round(sum(ArrDelayMinutes>30)/count(*),2) as rate
FROM ontime
WHERE
DayOfWeek not in (6,7) and OriginState not in ('AK', 'HI', 'PR', 'VI')
and DestState not in ('AK', 'HI', 'PR', 'VI')
and FlightDate < '2010-01-01'
GROUP by Carrier
HAVING cnt > 100000 and max(Year) > 1990
ORDER by rate DESC
LIMIT 1000;
```
Bonus:
```sql
SELECT avg(cnt) FROM (SELECT Year,Month,count(*) AS cnt FROM ontime WHERE DepDel15=1 GROUP BY Year,Month)
select avg(c1) from (select Year,Month,count(*) as c1 from ontime group by Year,Month)
SELECT DestCityName, uniqExact(OriginCityName) AS u FROM ontime GROUP BY DestCityName ORDER BY u DESC LIMIT 10;
SELECT OriginCityName, DestCityName, count() AS c FROM ontime GROUP BY OriginCityName, DestCityName ORDER BY c DESC LIMIT 10;
SELECT OriginCityName, count() AS c FROM ontime GROUP BY OriginCityName ORDER BY c DESC LIMIT 10;
```
这个性能测试由Vadim Tkachenko提供。参考
- <https://www.percona.com/blog/2009/10/02/analyzing-air-traffic-performance-with-infobright-and-monetdb/>
- <https://www.percona.com/blog/2009/10/26/air-traffic-queries-in-luciddb/>
- <https://www.percona.com/blog/2009/11/02/air-traffic-queries-in-infinidb-early-alpha/>
- <https://www.percona.com/blog/2014/04/21/using-apache-hadoop-and-impala-together-with-mysql-for-data-analysis/>
- <https://www.percona.com/blog/2016/01/07/apache-spark-with-air-ontime-performance-data/>
- <http://nickmakos.blogspot.ru/2012/08/analyzing-air-traffic-performance-with.html>

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/star_schema.md

View File

@ -0,0 +1,87 @@
# Star Schema 基准测试
编译 dbgen: <https://github.com/vadimtk/ssb-dbgen>
```bash
git clone git@github.com:vadimtk/ssb-dbgen.git
cd ssb-dbgen
make
```
在编译过程中可能会有一些警告,这是正常的。
将`dbgen`和`dists.dss`放在一个可用容量大于800GB的磁盘中。
开始生成数据:
```bash
./dbgen -s 1000 -T c
./dbgen -s 1000 -T l
```
在ClickHouse中创建表结构
``` sql
CREATE TABLE lineorder (
LO_ORDERKEY UInt32,
LO_LINENUMBER UInt8,
LO_CUSTKEY UInt32,
LO_PARTKEY UInt32,
LO_SUPPKEY UInt32,
LO_ORDERDATE Date,
LO_ORDERPRIORITY String,
LO_SHIPPRIORITY UInt8,
LO_QUANTITY UInt8,
LO_EXTENDEDPRICE UInt32,
LO_ORDTOTALPRICE UInt32,
LO_DISCOUNT UInt8,
LO_REVENUE UInt32,
LO_SUPPLYCOST UInt32,
LO_TAX UInt8,
LO_COMMITDATE Date,
LO_SHIPMODE String
)Engine=MergeTree(LO_ORDERDATE,(LO_ORDERKEY,LO_LINENUMBER,LO_ORDERDATE),8192);
CREATE TABLE customer (
C_CUSTKEY UInt32,
C_NAME String,
C_ADDRESS String,
C_CITY String,
C_NATION String,
C_REGION String,
C_PHONE String,
C_MKTSEGMENT String,
C_FAKEDATE Date
)Engine=MergeTree(C_FAKEDATE,(C_CUSTKEY,C_FAKEDATE),8192);
CREATE TABLE part (
P_PARTKEY UInt32,
P_NAME String,
P_MFGR String,
P_CATEGORY String,
P_BRAND String,
P_COLOR String,
P_TYPE String,
P_SIZE UInt8,
P_CONTAINER String,
P_FAKEDATE Date
)Engine=MergeTree(P_FAKEDATE,(P_PARTKEY,P_FAKEDATE),8192);
CREATE TABLE lineorderd AS lineorder ENGINE = Distributed(perftest_3shards_1replicas, default, lineorder, rand());
CREATE TABLE customerd AS customer ENGINE = Distributed(perftest_3shards_1replicas, default, customer, rand());
CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, default, part, rand());
```
如果是在单节点中进行的测试那么只需要创建对应的MergeTree表。
如果是在多节点中进行的测试,您需要在配置文件中配置`perftest_3shards_1replicas`集群的信息。
然后在每个节点中同时创建MergeTree表和Distributed表。
下载数据(如果您是分布式测试的话将'customer'更改为'customerd'
```bash
cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INTO customer FORMAT CSV"
cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV"
```
[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/star_schema/) <!--hide-->

View File

@ -1 +0,0 @@
../../../en/getting_started/example_datasets/wikistat.md

View File

@ -0,0 +1,29 @@
# 维基访问数据
参考: <http://dumps.wikimedia.org/other/pagecounts-raw/>
创建表结构:
``` sql
CREATE TABLE wikistat
(
date Date,
time DateTime,
project String,
subproject String,
path String,
hits UInt64,
size UInt64
) ENGINE = MergeTree(date, (path, time), 8192);
```
加载数据:
```bash
for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt
cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done
ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done
```
[Original article](https://clickhouse.yandex/docs/en/getting_started/example_datasets/wikistat/) <!--hide-->

View File

@ -1 +0,0 @@
../../en/getting_started/index.md

View File

@ -0,0 +1,141 @@
# 入门指南
## 系统要求
如果从官方仓库安装需要确保您使用的是x86_64处理器构架的Linux并且支持SSE 4.2指令集
检查是否支持SSE 4.2
```bash
grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported"
```
我们推荐使用Ubuntu或者Debian。终端必须使用UTF-8编码。
基于rpm的系统,你可以使用第三方的安装包https://packagecloud.io/altinity/clickhouse 或者直接安装debian安装包。
ClickHouse还可以在FreeBSD与Mac OS X上工作。同时它可以在不支持SSE 4.2的x86_64构架和AArch64 CPUs上编译。
## 安装
为了测试和开发系统可以安装在单个服务器或普通PC机上。
### 为Debian/Ubuntu安装
在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库:
```text
deb http://repo.yandex.ru/clickhouse/deb/stable/ main/
```
如果你想使用最新的测试版本,请使用'testing'替换'stable'。
然后运行:
```bash
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-get update
sudo apt-get install clickhouse-client clickhouse-server
```
你也可以从这里手动下载安装包:<https://repo.yandex.ru/clickhouse/deb/stable/main/>
ClickHouse包含访问控制配置它们位于`users.xml`文件中(与'config.xml'同目录)。
默认情况下允许从任何地方使用默认的default用户无密码的访问ClickHouse。参考user/default/networks
有关更多信息,请参考"Configuration files"部分。
### 使用源码安装
具体编译方式可以参考build.md。
你可以编译并安装它们。
你也可以直接使用而不进行安装。
```text
Client: dbms/programs/clickhouse-client
Server: dbms/programs/clickhouse-server
```
在服务器中为数据创建如下目录:
```text
/opt/clickhouse/data/default/
/opt/clickhouse/metadata/default/
```
(它们可以在server config中配置。)
为需要的用户运行chown
日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。
### 其他的安装方法
Docker image<https://hub.docker.com/r/yandex/clickhouse-server/>
CentOS或RHEL安装包<https://github.com/Altinity/clickhouse-rpm-install>
Gentoo`emerge clickhouse`
## 启动
可以运行如下命令在后台启动服务:
```bash
sudo service clickhouse-server start
```
可以在`/var/log/clickhouse-server/`目录中查看日志。
如果服务没有启动,请检查配置文件 `/etc/clickhouse-server/config.xml`
你也可以在控制台中直接启动服务:
```bash
clickhouse-server --config-file=/etc/clickhouse-server/config.xml
```
在这种情况下,日志将被打印到控制台中,这在开发过程中很方便。
如果配置文件在当前目录中,你可以不指定‘--config-file参数。它默认使用./config.xml
你可以使用命令行客户端连接到服务:
```bash
clickhouse-client
```
默认情况下它使用default用户无密码的与localhost:9000服务建立连接。
客户端也可以用于连接远程服务,例如:
```bash
clickhouse-client --host=example.com
```
有关更多信息,请参考"Command-line client"部分。
检查系统是否工作:
```bash
milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client
ClickHouse client version 0.0.18749.
Connecting to localhost:9000.
Connected to ClickHouse server version 0.0.18749.
:) SELECT 1
SELECT 1
┌─1─┐
│ 1 │
└───┘
1 rows in set. Elapsed: 0.003 sec.
:)
```
**恭喜,系统已经工作了!**
为了继续进行实验,你可以尝试下载测试数据集。
[Original article](https://clickhouse.yandex/docs/en/getting_started/) <!--hide-->

View File

@ -173,7 +173,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA
解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下它们被解析为逗号或换行符CR 或 LF。在解析不带引号的行时若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 UnixLFWindowsCR LF和 Mac OS ClassicCR LF 解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下它们被解析为逗号或换行符CR 或 LF。在解析不带引号的行时若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 UnixLFWindowsCR LF和 Mac OS ClassicCR LF
`NULL` is formatted as `\N`. `NULL` 将输出为 `\N`
CSV 格式是和 TabSeparated 一样的方式输出总数和极值。 CSV 格式是和 TabSeparated 一样的方式输出总数和极值。
@ -511,7 +511,7 @@ test: string with 'quotes' and with some special
characters characters
``` ```
Compare with the Vertical format: 和 Vertical 格式相比:
``` ```
:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical; :) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical;

View File

@ -65,7 +65,7 @@
body { body {
background: #fff; background: #fff;
font: 400 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif;
margin: 0; margin: 0;
padding: 0; padding: 0;
} }
@ -215,7 +215,7 @@ a:hover, a:active {
#announcement { #announcement {
margin: 0 0 60px 0; margin: 0 0 60px 0;
padding: 20px 0; padding: 20px 0;
font: 400 140%/133% 'Yandex Sans Display Web',Arial,sans-serif; font: 400 125%/133% 'Yandex Sans Display Web',Arial,sans-serif;
} }
.announcement-link { .announcement-link {
color: #000!important; color: #000!important;

View File

@ -92,7 +92,8 @@
</div> </div>
<div id="announcement" class="colored-block"> <div id="announcement" class="colored-block">
<div class="page"> <div class="page">
Upcoming meetup in <a class="announcement-link" href="http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1" rel="external nofollow" target="_blank">Beijing on October 28</a> Upcoming meetups in <a class="announcement-link" href="http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1" rel="external nofollow" target="_blank">Beijing on October 28</a>
and <a class="announcement-link" href="https://events.yandex.com/events/meetings/15-11-2018/" rel="external nofollow" target="_blank">Amsterdam on November 15</a>
</div> </div>
</div> </div>
<div class="page"> <div class="page">
@ -199,6 +200,8 @@
rel="external nofollow" target="_blank">ClickHouse vs Amazon RedShift Benchmark</a> by Altinity</li> rel="external nofollow" target="_blank">ClickHouse vs Amazon RedShift Benchmark</a> by Altinity</li>
<li><a href="https://carto.com/blog/inside/geospatial-processing-with-clickhouse" <li><a href="https://carto.com/blog/inside/geospatial-processing-with-clickhouse"
rel="external nofollow" target="_blank">Geospatial processing with Clickhouse</a> by Carto</li> rel="external nofollow" target="_blank">Geospatial processing with Clickhouse</a> by Carto</li>
<li><a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fwww.clickhouse.com.cn%2Ftopic%2F5a72e8ab9d28dfde2ddc5ea2F&amp;lang=zh-en"
rel="external nofollow" target="_blank">ClickHouse and Vertica comparison</a> by zhtsh <span class="grey">(machine translation from Chinese)</span></li>
<li><a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fverynull.com%2F2016%2F08%2F22%2Finfinidb%E4%B8%8Eclickhouse%E5%AF%B9%E6%AF%94%2F&amp;lang=zh-en" <li><a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fverynull.com%2F2016%2F08%2F22%2Finfinidb%E4%B8%8Eclickhouse%E5%AF%B9%E6%AF%94%2F&amp;lang=zh-en"
rel="external nofollow" target="_blank">ClickHouse and InfiniDB comparison</a> by RamboLau <span class="grey">(machine translation from Chinese)</span></li> rel="external nofollow" target="_blank">ClickHouse and InfiniDB comparison</a> by RamboLau <span class="grey">(machine translation from Chinese)</span></li>
</ul> </ul>
@ -306,15 +309,20 @@
<h2 id="success-stories">Success Stories</h2> <h2 id="success-stories">Success Stories</h2>
<ul class="dashed"> <ul class="dashed">
<li><a href="docs/en/introduction/ya_metrika_task.html">Yandex.Metrica</a></li> <li><a href="docs/en/introduction/ya_metrika_task/">Yandex.Metrica</a></li>
<li><a href="https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/" <li><a href="https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/"
rel="external nofollow" target="_blank">CloudFlare DNS Analytics</a></li> rel="external nofollow" target="_blank">HTTP Analytics</a> and <a href="https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/"
rel="external nofollow" target="_blank">DNS Analytics</a> at CloudFlare</li>
<li><a href="https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173"
rel="external nofollow" target="_blank">ClickHouse for Experimentation</a> at Spotify</li>
<li><a href="https://translate.yandex.com/translate?url=https%3A%2F%2Fhabrahabr.ru%2Fpost%2F322620%2F&amp;lang=ru-en" <li><a href="https://translate.yandex.com/translate?url=https%3A%2F%2Fhabrahabr.ru%2Fpost%2F322620%2F&amp;lang=ru-en"
rel="external nofollow" target="_blank">Migrating to Yandex ClickHouse</a> by LifeStreet <span class="grey">(machine translation from Russian)</span></li> rel="external nofollow" target="_blank">Migrating to Yandex ClickHouse</a> by LifeStreet <span class="grey">(machine translation from Russian)</span></li>
<li><a href="https://translate.yandex.com/translate?url=https%3A%2F%2Fhabrahabr.ru%2Fcompany%2Fsmi2%2Fblog%2F314558%2F&amp;lang=ru-en" <li><a href="https://translate.yandex.com/translate?url=https%3A%2F%2Fhabrahabr.ru%2Fcompany%2Fsmi2%2Fblog%2F314558%2F&amp;lang=ru-en"
rel="external nofollow" target="_blank">How to start ClickHouse up and win the jackpot</a> by SMI2 <span class="grey">(machine translation from Russian)</span></li> rel="external nofollow" target="_blank">How to start ClickHouse up and win the jackpot</a> by SMI2 <span class="grey">(machine translation from Russian)</span></li>
<li><a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fwww.jianshu.com%2Fp%2F4c86a2478cca&amp;lang=zh-en" <li><a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fwww.jianshu.com%2Fp%2F4c86a2478cca&amp;lang=zh-en"
rel="external nofollow" target="_blank">First place at Analysys OLAP algorithm contest</a> <span class="grey">(machine translation from Chinese)</span></li> rel="external nofollow" target="_blank">First place at Analysys OLAP algorithm contest</a> <span class="grey">(machine translation from Chinese)</span></li>
<li><a href="https://translate.yandex.com/translate?url=https%3A%2F%2Ftech.geniee.co.jp%2Fentry%2F2017%2F07%2F20%2F160100"
rel="external nofollow" target="_blank">Speeding up Report API</a> at Geniee <span class="grey">(machine translation from Japanese)</span></li>
<li><a href="https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/" <li><a href="https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/"
rel="external nofollow" target="_blank">LHCb experiment</a> by CERN</li> rel="external nofollow" target="_blank">LHCb experiment</a> by CERN</li>
</ul> </ul>
@ -428,6 +436,9 @@ clickhouse-client
rel="external nofollow" target="_blank">English</a> or in rel="external nofollow" target="_blank">English</a> or in
<a href="https://telegram.me/clickhouse_ru" <a href="https://telegram.me/clickhouse_ru"
rel="external nofollow" target="_blank">Russian</a>.</li> rel="external nofollow" target="_blank">Russian</a>.</li>
<li>Follow official <a
href="https://twitter.com/ClickHouseDB"
rel="external nofollow" target="_blank">Twitter account</a>.</li>
</ul> </ul>
<p>Or email ClickHouse team at Yandex directly: <p>Or email ClickHouse team at Yandex directly:
@ -485,16 +496,6 @@ clickhouse-client
if (hostParts.length > 2 && hostParts[0] != 'test') { if (hostParts.length > 2 && hostParts[0] != 'test') {
window.location.host = hostParts[0] + '.' + hostParts[1]; window.location.host = hostParts[0] + '.' + hostParts[1];
} }
/* var available_distributives = ['xenial', 'trusty', 'precise'];
available_distributives.forEach(function (name) {
$('#ubuntu_' + name).on('click', function () {
$('#distributive').html(name);
available_distributives.forEach(function (distr) {
$('#ubuntu_' + distr).attr('class', (name == distr) ? 'distributive_selected' : 'distributive_not_selected');
});
});
});*/
}); });
</script> </script>