Merge branch 'master' into dictionary-invalidate-query

This commit is contained in:
Nikolai Kochetov 2017-05-22 13:49:45 +03:00
commit 7762204b86
94 changed files with 1722 additions and 697 deletions

View File

@ -1,18 +0,0 @@
## [1.1.54189](https://github.com/yandex/Clickhouse/tree/v1.1.54189-testing) (2017-03-17)
[Full Changelog](https://github.com/yandex/Clickhouse/compare/v1.1.54188-stable...v1.1.54189-testing)
- Config: Allow define several graphite blocks, graphite.interval=60 option added. use_graphite option deleted.
- Configuration elements can now be loaded from ZooKeeper (see [documentation](https://clickhouse.yandex/reference_en.html#Configuration%20files))
## [1.1.54181](https://github.com/yandex/Clickhouse/tree/v1.1.54181-testing) (2017-03-10)
[Full Changelog](https://github.com/yandex/Clickhouse/compare/v1.1.54165-stable...v1.1.54181-testing)
- https server:
to enable: get/generate keys, uncomment in config.xml: `<https_port>8443</https_port>` and tune `<openSSL>` section
- listen_host by default changed to ::1 and 127.0.0.1.
If you want use connections from other computers write to config.xml: `<listen_host>::</listen_host>`

View File

@ -255,7 +255,7 @@ btrie_insert_a6(btrie_t *tree, const uint8_t *key, const uint8_t *mask,
uintptr_t value) uintptr_t value)
{ {
uint8_t bit; uint8_t bit;
uint i; unsigned int i;
btrie_node_t *node, *next; btrie_node_t *node, *next;
i = 0; i = 0;
@ -337,7 +337,7 @@ int
btrie_delete_a6(btrie_t *tree, const uint8_t *key, const uint8_t *mask) btrie_delete_a6(btrie_t *tree, const uint8_t *key, const uint8_t *mask)
{ {
uint8_t bit; uint8_t bit;
uint i; unsigned int i;
btrie_node_t *node; btrie_node_t *node;
i = 0; i = 0;
@ -411,7 +411,7 @@ btrie_find_a6(btrie_t *tree, const uint8_t *key)
{ {
uint8_t bit; uint8_t bit;
uintptr_t value; uintptr_t value;
uint i; unsigned int i;
btrie_node_t *node; btrie_node_t *node;
i = 0; i = 0;

View File

@ -1,6 +1,6 @@
#This strings autochanged from release_lib.sh : #This strings autochanged from release_lib.sh :
set(VERSION_DESCRIBE v1.1.54234-testing) set(VERSION_DESCRIBE v1.1.54235-testing)
set(VERSION_REVISION 54234) set(VERSION_REVISION 54235)
#===end of autochange #===end of autochange
set (VERSION_MAJOR 1) set (VERSION_MAJOR 1)

View File

@ -35,8 +35,8 @@ try
auto system_database = std::make_shared<DatabaseMemory>("system"); auto system_database = std::make_shared<DatabaseMemory>("system");
context.addDatabase("system", system_database); context.addDatabase("system", system_database);
system_database->attachTable("one", StorageSystemOne::create("one")); system_database->attachTable("one", StorageSystemOne::create("one"));
system_database->attachTable("numbers", StorageSystemNumbers::create("numbers")); system_database->attachTable("numbers", StorageSystemNumbers::create("numbers"));
context.setCurrentDatabase("system"); context.setCurrentDatabase("system");
AnalyzeLambdas analyze_lambdas; AnalyzeLambdas analyze_lambdas;

View File

@ -12,14 +12,11 @@ namespace DB
CastTypeBlockInputStream::CastTypeBlockInputStream( CastTypeBlockInputStream::CastTypeBlockInputStream(
const Context & context_, const Context & context_,
BlockInputStreamPtr input_, const BlockInputStreamPtr & input_,
const Block & in_sample_, const Block & reference_definition_)
const Block & out_sample_) : context(context_), ref_defenition(reference_definition_)
: context(context_)
{ {
collectDifferent(in_sample_, out_sample_); children.emplace_back(input_);
cast_functions.resize(in_sample_.columns());
children.push_back(input_);
} }
String CastTypeBlockInputStream::getName() const String CastTypeBlockInputStream::getName() const
@ -29,98 +26,91 @@ String CastTypeBlockInputStream::getName() const
String CastTypeBlockInputStream::getID() const String CastTypeBlockInputStream::getID() const
{ {
std::stringstream res; return "CastType(" + children.back()->getID() + ")";
res << "CastType(" << children.back()->getID() << ")";
return res.str();
} }
Block CastTypeBlockInputStream::readImpl() Block CastTypeBlockInputStream::readImpl()
{ {
Block block = children.back()->read(); Block block = children.back()->read();
if (!block || cast_types.empty()) if (!block)
return block; return block;
size_t block_size = block.columns(); if (!initialized)
if (block_size != cast_types.size())
{ {
LOG_ERROR(log, "Number of columns do not match, skipping cast"); initialized = true;
return block; initialize(block);
} }
if (cast_description.empty())
return block;
size_t num_columns = block.columns();
Block res; Block res;
for (size_t i = 0; i < block_size; ++i) for (size_t col = 0; col < num_columns; ++col)
{ {
const auto & elem = block.getByPosition(i); const auto & src_column = block.getByPosition(col);
auto it = cast_description.find(col);
if (bool(cast_types[i])) if (it == cast_description.end())
{ {
const auto & type = cast_types[i]->type; res.insert(src_column);
Block temporary_block
{
{
elem.column,
elem.type,
elem.name
},
{
std::make_shared<ColumnConstString>(1, type->getName()),
std::make_shared<DataTypeString>(),
""
},
{
nullptr,
cast_types[i]->type,
""
}
};
FunctionPtr & cast_function = cast_functions[i];
/// Initialize function.
if (!cast_function)
{
cast_function = FunctionFactory::instance().get("CAST", context);
DataTypePtr unused_return_type;
ColumnsWithTypeAndName arguments{ temporary_block.getByPosition(0), temporary_block.getByPosition(1) };
std::vector<ExpressionAction> unused_prerequisites;
/// Prepares function to execution. TODO It is not obvious.
cast_function->getReturnTypeAndPrerequisites(arguments, unused_return_type, unused_prerequisites);
}
cast_function->execute(temporary_block, {0, 1}, 2);
res.insert({
temporary_block.getByPosition(2).column,
cast_types[i]->type,
cast_types[i]->name});
} }
else else
{ {
res.insert(elem); CastElement & cast_element = it->second;
size_t tmp_col = cast_element.tmp_col_offset;
ColumnNumbers arguments{tmp_col, tmp_col + 1};
tmp_conversion_block.getByPosition(tmp_col).column = src_column.column;
cast_element.function->execute(tmp_conversion_block, arguments, tmp_col + 2);
res.insert(tmp_conversion_block.getByPosition(tmp_col + 2));
} }
} }
return res; return res;
} }
void CastTypeBlockInputStream::collectDifferent(const Block & in_sample, const Block & out_sample)
CastTypeBlockInputStream::CastElement::CastElement(std::shared_ptr<IFunction> && function_, size_t tmp_col_offset_)
: function(std::move(function_)), tmp_col_offset(tmp_col_offset_) {}
void CastTypeBlockInputStream::initialize(const Block & src_block)
{ {
size_t in_size = in_sample.columns(); for (size_t src_col = 0; src_col < src_block.columns(); ++src_col)
cast_types.resize(in_size);
for (size_t i = 0; i < in_size; ++i)
{ {
const auto & in_elem = in_sample.getByPosition(i); const auto & src_column = src_block.getByPosition(src_col);
const auto & out_elem = out_sample.getByPosition(i);
/// Skip, if it is a problem, it will be detected on the next pipeline stage
if (!ref_defenition.has(src_column.name))
continue;
const auto & ref_column = ref_defenition.getByName(src_column.name);
/// Force conversion if source and destination types is different. /// Force conversion if source and destination types is different.
if (!out_elem.type->equals(*in_elem.type)) if (!ref_column.type->equals(*src_column.type))
{ {
cast_types[i] = NameAndTypePair(out_elem.name, out_elem.type); ColumnWithTypeAndName src_columnn_copy = src_column.cloneEmpty();
ColumnWithTypeAndName alias_column(std::make_shared<ColumnConstString>(1, ref_column.type->getName()), std::make_shared<DataTypeString>(), "");
ColumnWithTypeAndName result_column(nullptr, ref_column.type->clone(), src_column.name);
DataTypePtr unused_return_type;
std::vector<ExpressionAction> unused_prerequisites;
ColumnsWithTypeAndName arguments{src_columnn_copy, alias_column};
/// Prepares function to execution. TODO It is not obvious.
auto cast_function = FunctionFactory::instance().get("CAST", context);
cast_function->getReturnTypeAndPrerequisites(arguments, unused_return_type, unused_prerequisites);
tmp_conversion_block.insert(src_column);
tmp_conversion_block.insert(alias_column);
tmp_conversion_block.insert(result_column);
size_t tmp_col_offset = cast_description.size() * 3;
cast_description.emplace(src_col, CastElement(std::move(cast_function), tmp_col_offset));
} }
} }
} }

View File

@ -2,10 +2,6 @@
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <common/logger_useful.h>
#include <experimental/optional>
#include <vector>
namespace DB namespace DB
{ {
@ -16,10 +12,9 @@ class IFunction;
class CastTypeBlockInputStream : public IProfilingBlockInputStream class CastTypeBlockInputStream : public IProfilingBlockInputStream
{ {
public: public:
CastTypeBlockInputStream(const Context & context_, CastTypeBlockInputStream(const Context & context,
BlockInputStreamPtr input_, const BlockInputStreamPtr & input,
const Block & in_sample_, const Block & reference_definition);
const Block & out_sample_);
String getName() const override; String getName() const override;
@ -28,14 +23,25 @@ public:
protected: protected:
Block readImpl() override; Block readImpl() override;
private:
void collectDifferent(const Block & in_sample, const Block & out_sample);
private: private:
const Context & context; const Context & context;
std::vector<std::experimental::optional<NameAndTypePair>> cast_types; Block ref_defenition;
std::vector<std::shared_ptr<IFunction>> cast_functions; /// Used to perform type conversions.
Logger * log = &Logger::get("CastTypeBlockInputStream"); void initialize(const Block & src_block);
bool initialized = false;
struct CastElement
{
std::shared_ptr<IFunction> function;
size_t tmp_col_offset;
CastElement(std::shared_ptr<IFunction> && function_, size_t tmp_col_offset_);
};
/// Describes required conversions on source block
std::map<size_t, CastElement> cast_description;
/// Auxiliary block, stores arguments and results of required CAST calls
Block tmp_conversion_block;
}; };
} }

View File

@ -94,8 +94,8 @@ Block GraphiteRollupSortedBlockInputStream::readImpl()
if (i != time_column_num && i != value_column_num && i != version_column_num) if (i != time_column_num && i != value_column_num && i != version_column_num)
unmodified_column_numbers.push_back(i); unmodified_column_numbers.push_back(i);
if (selected_row.empty()) if (current_selected_row.empty())
selected_row.columns.resize(num_columns); current_selected_row.columns.resize(num_columns);
} }
if (has_collation) if (has_collation)
@ -112,111 +112,114 @@ void GraphiteRollupSortedBlockInputStream::merge(ColumnPlainPtrs & merged_column
{ {
const DateLUTImpl & date_lut = DateLUT::instance(); const DateLUTImpl & date_lut = DateLUT::instance();
size_t merged_rows = 0; size_t started_rows = 0; /// Number of times startNextRow() has been called.
/// Take rows in needed order and put them into `merged_block` until we get `max_block_size` rows.
///
/// Variables starting with current_* refer to the rows previously popped from the queue that will
/// contribute towards current output row.
/// Variables starting with next_* refer to the row at the top of the queue.
/// Take rows in needed order and put them into `merged_block` until rows no more than `max_block_size`
while (!queue.empty()) while (!queue.empty())
{ {
TSortCursor current = queue.top(); TSortCursor next_cursor = queue.top();
next_path = current->all_columns[path_column_num]->getDataAt(current->pos); StringRef next_path = next_cursor->all_columns[path_column_num]->getDataAt(next_cursor->pos);
next_time = current->all_columns[time_column_num]->get64(current->pos);
auto prev_pattern = current_pattern;
bool path_differs = is_first || next_path != current_path; bool path_differs = is_first || next_path != current_path;
is_first = false; is_first = false;
time_t next_time = next_cursor->all_columns[time_column_num]->get64(next_cursor->pos);
/// Is new key before rounding. /// Is new key before rounding.
bool is_new_key = path_differs || next_time != current_time; bool is_new_key = path_differs || next_time != current_time;
UInt64 current_version = current->all_columns[version_column_num]->get64(current->pos);
if (is_new_key) if (is_new_key)
{ {
current_path = next_path; /// Accumulate the row that has maximum version in the previous group of rows wit the same key:
current_time = next_time; if (started_rows)
accumulateRow(current_selected_row);
/// For previous group of rows with same key, accumulate a row that has maximum version.
if (merged_rows)
accumulateRow(selected_row);
const Graphite::Pattern * next_pattern = current_pattern;
if (path_differs) if (path_differs)
current_pattern = selectPatternForPath(next_path); next_pattern = selectPatternForPath(next_path);
if (current_pattern) time_t next_time_rounded;
if (next_pattern)
{ {
UInt32 precision = selectPrecision(current_pattern->retentions, next_time); UInt32 precision = selectPrecision(next_pattern->retentions, next_time);
next_time_rounded = roundTimeToPrecision(date_lut, next_time, precision); next_time_rounded = roundTimeToPrecision(date_lut, next_time, precision);
} }
/// If no patterns has matched - it means that no need to do rounding. else
{
/// If no pattern has matched - take the value as-is.
next_time_rounded = next_time;
}
/// Key will be new after rounding. It means new result row. /// Key will be new after rounding. It means new result row.
bool will_be_new_key = path_differs || next_time_rounded != current_time_rounded; bool will_be_new_key = path_differs || next_time_rounded != current_time_rounded;
if (will_be_new_key) if (will_be_new_key)
{ {
/// This is not the first row in block. if (started_rows)
if (merged_rows)
{ {
finishCurrentRow(merged_columns); finishCurrentRow(merged_columns);
/// if we have enough rows /// We have enough rows - return, but don't advance the loop. At the beginning of the
if (merged_rows >= max_block_size) /// next call to merge() the same next_cursor will be processed once more and
/// the next output row will be created from it.
if (started_rows >= max_block_size)
return; return;
} }
startNextRow(merged_columns, current); /// At this point previous row has been fully processed, so we can advance the loop
/// (substitute current_* values for next_*, advance the cursor).
startNextRow(merged_columns, next_cursor, next_pattern);
++started_rows;
current_time_rounded = next_time_rounded; current_time_rounded = next_time_rounded;
if (prev_pattern)
prev_pattern->function->destroy(place_for_aggregate_state.data());
if (current_pattern)
current_pattern->function->create(place_for_aggregate_state.data());
++merged_rows;
} }
current_path = next_path;
current_time = next_time;
} }
/// Within all rows with same key, we should leave only one row with maximum version; /// Within all rows with same key, we should leave only one row with maximum version;
/// and for rows with same maximum version - only last row. /// and for rows with same maximum version - only last row.
if (is_new_key || current_version >= current_max_version) UInt64 next_version = next_cursor->all_columns[version_column_num]->get64(next_cursor->pos);
if (is_new_key || next_version >= current_max_version)
{ {
current_max_version = current_version; current_max_version = next_version;
setRowRef(selected_row, current); setRowRef(current_selected_row, next_cursor);
} }
queue.pop(); queue.pop();
if (!current->isLast()) if (!next_cursor->isLast())
{ {
current->next(); next_cursor->next();
queue.push(current); queue.push(next_cursor);
} }
else else
{ {
/// We get the next block from the appropriate source, if there is one. /// We get the next block from the appropriate source, if there is one.
fetchNextBlock(current, queue); fetchNextBlock(next_cursor, queue);
} }
} }
/// Write result row for the last group. /// Write result row for the last group.
++merged_rows; if (started_rows)
accumulateRow(selected_row); {
finishCurrentRow(merged_columns); accumulateRow(current_selected_row);
finishCurrentRow(merged_columns);
}
finished = true; finished = true;
if (current_pattern)
{
current_pattern->function->destroy(place_for_aggregate_state.data());
current_pattern = nullptr;
}
} }
template <class TSortCursor> template <class TSortCursor>
void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor) void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor, const Graphite::Pattern * next_pattern)
{ {
/// Copy unmodified column values. /// Copy unmodified column values.
for (size_t i = 0, size = unmodified_column_numbers.size(); i < size; ++i) for (size_t i = 0, size = unmodified_column_numbers.size(); i < size; ++i)
@ -225,8 +228,13 @@ void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged
merged_columns[j]->insertFrom(*cursor->all_columns[j], cursor->pos); merged_columns[j]->insertFrom(*cursor->all_columns[j], cursor->pos);
} }
if (!current_pattern) if (next_pattern)
merged_columns[value_column_num]->insertFrom(*cursor->all_columns[value_column_num], cursor->pos); {
next_pattern->function->create(place_for_aggregate_state.data());
aggregate_state_created = true;
}
current_pattern = next_pattern;
} }
@ -236,14 +244,21 @@ void GraphiteRollupSortedBlockInputStream::finishCurrentRow(ColumnPlainPtrs & me
merged_columns[time_column_num]->insert(UInt64(current_time_rounded)); merged_columns[time_column_num]->insert(UInt64(current_time_rounded));
merged_columns[version_column_num]->insert(current_max_version); merged_columns[version_column_num]->insert(current_max_version);
if (current_pattern) if (aggregate_state_created)
{
current_pattern->function->insertResultInto(place_for_aggregate_state.data(), *merged_columns[value_column_num]); current_pattern->function->insertResultInto(place_for_aggregate_state.data(), *merged_columns[value_column_num]);
current_pattern->function->destroy(place_for_aggregate_state.data());
aggregate_state_created = false;
}
else
merged_columns[value_column_num]->insertFrom(
*current_selected_row.columns[value_column_num], current_selected_row.row_num);
} }
void GraphiteRollupSortedBlockInputStream::accumulateRow(RowRef & row) void GraphiteRollupSortedBlockInputStream::accumulateRow(RowRef & row)
{ {
if (current_pattern) if (aggregate_state_created)
current_pattern->function->add(place_for_aggregate_state.data(), &row.columns[value_column_num], row.row_num, nullptr); current_pattern->function->add(place_for_aggregate_state.data(), &row.columns[value_column_num], row.row_num, nullptr);
} }

View File

@ -154,7 +154,7 @@ public:
~GraphiteRollupSortedBlockInputStream() ~GraphiteRollupSortedBlockInputStream()
{ {
if (current_pattern) if (aggregate_state_created)
current_pattern->function->destroy(place_for_aggregate_state.data()); current_pattern->function->destroy(place_for_aggregate_state.data());
} }
@ -179,19 +179,17 @@ private:
/// All data has been read. /// All data has been read.
bool finished = false; bool finished = false;
RowRef selected_row; /// Last row with maximum version for current primary key. RowRef current_selected_row; /// Last row with maximum version for current primary key.
UInt64 current_max_version = 0; UInt64 current_max_version = 0;
bool is_first = true; bool is_first = true;
StringRef current_path; StringRef current_path;
time_t current_time = 0; time_t current_time = 0;
time_t current_time_rounded = 0; time_t current_time_rounded = 0;
StringRef next_path;
time_t next_time = 0;
time_t next_time_rounded = 0;
const Graphite::Pattern * current_pattern = nullptr; const Graphite::Pattern * current_pattern = nullptr;
std::vector<char> place_for_aggregate_state; std::vector<char> place_for_aggregate_state;
bool aggregate_state_created = false; /// Invariant: if true then current_pattern is not NULL.
const Graphite::Pattern * selectPatternForPath(StringRef path) const; const Graphite::Pattern * selectPatternForPath(StringRef path) const;
UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const; UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const;
@ -202,7 +200,7 @@ private:
/// Insert the values into the resulting columns, which will not be changed in the future. /// Insert the values into the resulting columns, which will not be changed in the future.
template <class TSortCursor> template <class TSortCursor>
void startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor); void startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor, const Graphite::Pattern * next_pattern);
/// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows. /// Insert the calculated `time`, `value`, `version` values into the resulting columns by the last group of rows.
void finishCurrentRow(ColumnPlainPtrs & merged_columns); void finishCurrentRow(ColumnPlainPtrs & merged_columns);

View File

@ -82,6 +82,9 @@ void DataTypeNumberBase<T>::serializeTextJSON(const IColumn & column, size_t row
template <typename T> template <typename T>
void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{ {
static constexpr bool is_uint8 = std::is_same<T, UInt8>::value;
static constexpr bool is_int8 = std::is_same<T, Int8>::value;
bool has_quote = false; bool has_quote = false;
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without. if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
{ {
@ -101,7 +104,24 @@ void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & i
} }
else else
{ {
readText(x, istr); if (is_uint8 || is_int8)
{
// extra conditions to parse true/false strings into 1/0
if (istr.eof())
throwReadAfterEOF();
if (*istr.position() == 't' || *istr.position() == 'f')
{
bool tmp = false;
readBoolTextWord(tmp, istr);
x = tmp;
}
else
readText(x, istr);
}
else
{
readText(x, istr);
}
if (has_quote) if (has_quote)
assertChar('"', istr); assertChar('"', istr);

View File

@ -16,6 +16,7 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionIPv4ToIPv6>(); factory.registerFunction<FunctionIPv4ToIPv6>();
factory.registerFunction<FunctionMACNumToString>(); factory.registerFunction<FunctionMACNumToString>();
factory.registerFunction<FunctionMACStringToNum>(); factory.registerFunction<FunctionMACStringToNum>();
factory.registerFunction<FunctionMACStringToOUI>();
factory.registerFunction<FunctionUUIDNumToString>(); factory.registerFunction<FunctionUUIDNumToString>();
factory.registerFunction<FunctionUUIDStringToNum>(); factory.registerFunction<FunctionUUIDStringToNum>();
factory.registerFunction<FunctionHex>(); factory.registerFunction<FunctionHex>();

View File

@ -1189,6 +1189,103 @@ public:
} }
}; };
class FunctionMACStringToOUI : public IFunction
{
public:
static constexpr auto name = "MACStringToOUI";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionMACStringToOUI>(); }
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!typeid_cast<const DataTypeString *>(&*arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt64>();
}
static UInt64 parseMAC(const char * pos)
{
/// get integer value for a hexademical char digit, or -1
const auto number_by_char = [] (const char ch)
{
if ('A' <= ch && ch <= 'F')
return 10 + ch - 'A';
if ('a' <= ch && ch <= 'f')
return 10 + ch - 'a';
if ('0' <= ch && ch <= '9')
return ch - '0';
return -1;
};
UInt64 res = 0;
for (int offset = 40; offset >= 0; offset -= 8)
{
UInt64 value = 0;
size_t len = 0;
int val = 0;
while ((val = number_by_char(*pos)) >= 0 && len <= 2)
{
value = value * 16 + val;
++len;
++pos;
}
if (len == 0 || value > 255 || (offset > 0 && *pos != ':'))
return 0;
res |= value << offset;
++pos;
}
if (*(pos - 1) != '\0')
return 0;
res = res >> 24;
return res;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr & column = block.safeGetByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
{
auto col_res = std::make_shared<ColumnUInt64>();
block.safeGetByPosition(result).column = col_res;
ColumnUInt64::Container_t & vec_res = col_res->getData();
vec_res.resize(col->size());
const ColumnString::Chars_t & vec_src = col->getChars();
const ColumnString::Offsets_t & offsets_src = col->getOffsets();
size_t prev_offset = 0;
for (size_t i = 0; i < vec_res.size(); ++i)
{
vec_res[i] = parseMAC(reinterpret_cast<const char *>(&vec_src[prev_offset]));
prev_offset = offsets_src[i];
}
}
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
{
auto col_res = std::make_shared<ColumnConst<UInt64>>(col->size(), parseMAC(col->getData().c_str()));
block.safeGetByPosition(result).column = col_res;
}
else
throw Exception("Illegal column " + block.safeGetByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
class FunctionUUIDNumToString : public IFunction class FunctionUUIDNumToString : public IFunction
{ {

View File

@ -255,6 +255,44 @@ public:
}; };
/// Returns global default value for type of passed argument (example: 0 for numeric types, '' for String).
class FunctionDefaultValueOfArgumentType : public IFunction
{
public:
static constexpr auto name = "defaultValueOfArgumentType";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDefaultValueOfArgumentType>();
}
String getName() const override
{
return name;
}
bool hasSpecialSupportForNulls() const override
{
return true;
}
size_t getNumberOfArguments() const override
{
return 1;
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return arguments[0]->clone();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IDataType & type = *block.getByPosition(arguments[0]).type;
block.getByPosition(result).column = type.createConstColumn(block.rows(), type.getDefault());
}
};
class FunctionBlockSize : public IFunction class FunctionBlockSize : public IFunction
{ {
public: public:
@ -1762,6 +1800,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
factory.registerFunction<FunctionVisibleWidth>(); factory.registerFunction<FunctionVisibleWidth>();
factory.registerFunction<FunctionToTypeName>(); factory.registerFunction<FunctionToTypeName>();
factory.registerFunction<FunctionToColumnTypeName>(); factory.registerFunction<FunctionToColumnTypeName>();
factory.registerFunction<FunctionDefaultValueOfArgumentType>();
factory.registerFunction<FunctionBlockSize>(); factory.registerFunction<FunctionBlockSize>();
factory.registerFunction<FunctionBlockNumber>(); factory.registerFunction<FunctionBlockNumber>();
factory.registerFunction<FunctionRowNumberInBlock>(); factory.registerFunction<FunctionRowNumberInBlock>();

View File

@ -18,7 +18,7 @@ namespace DB
/** Функции округления: /** Функции округления:
* roundToExp2 - вниз до ближайшей степени двойки; * roundToExp2 - вниз до ближайшей степени двойки;
* roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000; * roundDuration - вниз до ближайшего из: 0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000;
* roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45. * roundAge - вниз до ближайшего из: 0, 18, 25, 35, 45, 55.
* *
* round(x, N) - арифметическое округление (N = 0 по умолчанию). * round(x, N) - арифметическое округление (N = 0 по умолчанию).
* ceil(x, N) - наименьшее число, которое не меньше x (N = 0 по умолчанию). * ceil(x, N) - наименьшее число, которое не меньше x (N = 0 по умолчанию).
@ -101,7 +101,8 @@ struct RoundAgeImpl
: (x < 25 ? 18 : (x < 25 ? 18
: (x < 35 ? 25 : (x < 35 ? 25
: (x < 45 ? 35 : (x < 45 ? 35
: 45)))); : (x < 55 ? 45
: 55)))));
} }
}; };

View File

@ -216,6 +216,23 @@ inline void readBoolText(bool & x, ReadBuffer & buf)
x = tmp != '0'; x = tmp != '0';
} }
inline void readBoolTextWord(bool & x, ReadBuffer & buf)
{
if (buf.eof())
throwReadAfterEOF();
if (*buf.position() == 't')
{
assertString("true", buf);
x = true;
}
else
{
assertString("false", buf);
x = false;
}
}
template <typename T, typename ReturnType = void> template <typename T, typename ReturnType = void>
ReturnType readIntTextImpl(T & x, ReadBuffer & buf) ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
{ {

View File

@ -70,7 +70,7 @@ BlockIO InterpreterAlterQuery::execute()
case PartitionCommand::RESHARD_PARTITION: case PartitionCommand::RESHARD_PARTITION:
table->reshardPartitions(query_ptr, database_name, command.partition, command.last_partition, table->reshardPartitions(query_ptr, database_name, command.partition, command.last_partition,
command.weighted_zookeeper_paths, command.sharding_key_expr, command.do_copy, command.weighted_zookeeper_paths, command.sharding_key_expr, command.do_copy,
command.coordinator, context.getSettingsRef()); command.coordinator, context);
break; break;
case PartitionCommand::DROP_COLUMN: case PartitionCommand::DROP_COLUMN:

View File

@ -33,7 +33,7 @@ namespace ErrorCodes
} }
InterpreterInsertQuery::InterpreterInsertQuery(ASTPtr query_ptr_, Context & context_) InterpreterInsertQuery::InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_)
: query_ptr(query_ptr_), context(context_) : query_ptr(query_ptr_), context(context_)
{ {
ProfileEvents::increment(ProfileEvents::InsertQuery); ProfileEvents::increment(ProfileEvents::InsertQuery);
@ -124,7 +124,7 @@ BlockIO InterpreterInsertQuery::execute()
res.in = interpreter_select.execute().in; res.in = interpreter_select.execute().in;
res.in = std::make_shared<NullableAdapterBlockInputStream>(res.in, res.in_sample, res.out_sample); res.in = std::make_shared<NullableAdapterBlockInputStream>(res.in, res.in_sample, res.out_sample);
res.in = std::make_shared<CastTypeBlockInputStream>(context, res.in, res.in_sample, res.out_sample); res.in = std::make_shared<CastTypeBlockInputStream>(context, res.in, res.out_sample);
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, out); res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, out);
} }

View File

@ -15,7 +15,7 @@ namespace DB
class InterpreterInsertQuery : public IInterpreter class InterpreterInsertQuery : public IInterpreter
{ {
public: public:
InterpreterInsertQuery(ASTPtr query_ptr_, Context & context_); InterpreterInsertQuery(const ASTPtr & query_ptr_, const Context & context_);
/** Подготовить запрос к выполнению. Вернуть потоки блоков /** Подготовить запрос к выполнению. Вернуть потоки блоков
* - поток, в который можно писать данные для выполнения запроса, если INSERT; * - поток, в который можно писать данные для выполнения запроса, если INSERT;

View File

@ -185,7 +185,7 @@ public:
* It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). * It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP).
*/ */
virtual BlockOutputStreamPtr write( virtual BlockOutputStreamPtr write(
ASTPtr query, const ASTPtr & query,
const Settings & settings) const Settings & settings)
{ {
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
@ -251,11 +251,12 @@ public:
/** Run the RESHARD PARTITION query. /** Run the RESHARD PARTITION query.
*/ */
virtual void reshardPartitions(ASTPtr query, const String & database_name, virtual void reshardPartitions(
const ASTPtr & query, const String & database_name,
const Field & first_partition, const Field & last_partition, const Field & first_partition, const Field & last_partition,
const WeightedZooKeeperPaths & weighted_zookeeper_paths, const WeightedZooKeeperPaths & weighted_zookeeper_paths,
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
const Settings & settings) Context & context)
{ {
throw Exception("Method reshardPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); throw Exception("Method reshardPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
} }

View File

@ -150,11 +150,11 @@ try
std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end()); std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end());
auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr
: std::make_shared<MergeTreeBlockSizePredictor>(data_part, columns, pre_columns); : std::make_unique<MergeTreeBlockSizePredictor>(data_part, ordered_names, data_part->storage.getSampleBlock());
task = std::make_unique<MergeTreeReadTask>(data_part, remaining_mark_ranges, part_index_in_query, ordered_names, task = std::make_unique<MergeTreeReadTask>(data_part, remaining_mark_ranges, part_index_in_query, ordered_names,
column_name_set, columns, pre_columns, remove_prewhere_column, should_reorder, column_name_set, columns, pre_columns, remove_prewhere_column, should_reorder,
size_predictor); std::move(size_predictor));
if (!reader) if (!reader)
{ {

View File

@ -67,30 +67,29 @@ MergeTreeReadTask::MergeTreeReadTask(
const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const std::size_t part_index_in_query, const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const std::size_t part_index_in_query,
const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns, const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns,
const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder, const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder,
const MergeTreeBlockSizePredictorPtr & size_predictor) MergeTreeBlockSizePredictorPtr && size_predictor)
: data_part{data_part}, mark_ranges{mark_ranges}, part_index_in_query{part_index_in_query}, : data_part{data_part}, mark_ranges{mark_ranges}, part_index_in_query{part_index_in_query},
ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns}, ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns},
remove_prewhere_column{remove_prewhere_column}, should_reorder{should_reorder}, size_predictor{size_predictor} remove_prewhere_column{remove_prewhere_column}, should_reorder{should_reorder}, size_predictor{std::move(size_predictor)}
{} {}
MergeTreeReadTask::~MergeTreeReadTask() = default; MergeTreeReadTask::~MergeTreeReadTask() = default;
MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor( MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor(
const MergeTreeData::DataPartPtr & data_part_, const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block)
const NamesAndTypesList & columns,
const NamesAndTypesList & pre_columns)
: data_part(data_part_) : data_part(data_part_)
{ {
auto add_column = [&] (const NameAndTypePair & column) for (const String & column_name : columns)
{ {
ColumnPtr column_data = column.type->createColumn(); const auto column_checksum = data_part->tryGetBinChecksum(column_name);
const auto column_checksum = data_part->tryGetBinChecksum(column.name);
/// There are no data files, column will be const /// There are no column data files, column will be const
if (!column_checksum) if (!column_checksum || !data_part->hasColumnFiles(column_name))
return; return;
const ColumnPtr & column_data = sample_block.getByName(column_name).column;
if (column_data->isFixed()) if (column_data->isFixed())
{ {
fixed_columns_bytes_per_row += column_data->sizeOfField(); fixed_columns_bytes_per_row += column_data->sizeOfField();
@ -98,19 +97,13 @@ MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor(
else else
{ {
ColumnInfo info; ColumnInfo info;
info.name = column.name; info.name = column_name;
info.bytes_per_row_global = column_checksum->uncompressed_size; info.bytes_per_row_global = column_checksum->uncompressed_size;
dynamic_columns_infos.emplace_back(info); dynamic_columns_infos.emplace_back(info);
} }
}; };
for (const NameAndTypePair & column : pre_columns)
add_column(column);
for (const NameAndTypePair & column : columns)
add_column(column);
size_t rows_approx = data_part->getExactSizeRows(); size_t rows_approx = data_part->getExactSizeRows();
bytes_per_row_global = fixed_columns_bytes_per_row; bytes_per_row_global = fixed_columns_bytes_per_row;
@ -133,7 +126,7 @@ void MergeTreeBlockSizePredictor::startBlock()
} }
/// FIXME: add last_read_row_in_part parameter to take into account gaps between adjacent ranges /// TODO: add last_read_row_in_part parameter to take into account gaps between adjacent ranges
void MergeTreeBlockSizePredictor::update(const Block & block, double decay) void MergeTreeBlockSizePredictor::update(const Block & block, double decay)
{ {
size_t new_rows = block.rows(); size_t new_rows = block.rows();

View File

@ -10,7 +10,7 @@ struct MergeTreeReadTask;
struct MergeTreeBlockSizePredictor; struct MergeTreeBlockSizePredictor;
using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>; using MergeTreeReadTaskPtr = std::unique_ptr<MergeTreeReadTask>;
using MergeTreeBlockSizePredictorPtr = std::shared_ptr<MergeTreeBlockSizePredictor>; using MergeTreeBlockSizePredictorPtr = std::unique_ptr<MergeTreeBlockSizePredictor>;
/** If some of the requested columns are not in the part, /** If some of the requested columns are not in the part,
@ -50,7 +50,7 @@ struct MergeTreeReadTask
const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const std::size_t part_index_in_query, const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const std::size_t part_index_in_query,
const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns, const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns,
const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder, const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder,
const MergeTreeBlockSizePredictorPtr & size_predictor); MergeTreeBlockSizePredictorPtr && size_predictor);
virtual ~MergeTreeReadTask(); virtual ~MergeTreeReadTask();
}; };
@ -58,10 +58,7 @@ struct MergeTreeReadTask
struct MergeTreeBlockSizePredictor struct MergeTreeBlockSizePredictor
{ {
MergeTreeBlockSizePredictor( MergeTreeBlockSizePredictor(const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block);
const MergeTreeData::DataPartPtr & data_part_,
const NamesAndTypesList & columns,
const NamesAndTypesList & pre_columns);
/// Reset some values for correct statistics calculating /// Reset some values for correct statistics calculating
void startBlock(); void startBlock();

View File

@ -108,12 +108,12 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const std::size_t min_marks_to_r
} }
auto curr_task_size_predictor = !per_part_size_predictor[part_idx] ? nullptr auto curr_task_size_predictor = !per_part_size_predictor[part_idx] ? nullptr
: std::make_shared<MergeTreeBlockSizePredictor>(*per_part_size_predictor[part_idx]); /// make a copy : std::make_unique<MergeTreeBlockSizePredictor>(*per_part_size_predictor[part_idx]); /// make a copy
return std::make_unique<MergeTreeReadTask>( return std::make_unique<MergeTreeReadTask>(
part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names, part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names,
per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx], per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx],
per_part_remove_prewhere_column[part_idx], per_part_should_reorder[part_idx], curr_task_size_predictor); per_part_remove_prewhere_column[part_idx], per_part_should_reorder[part_idx], std::move(curr_task_size_predictor));
} }
@ -163,6 +163,7 @@ std::vector<std::size_t> MergeTreeReadPool::fillPerPartInfo(
const bool check_columns) const bool check_columns)
{ {
std::vector<std::size_t> per_part_sum_marks; std::vector<std::size_t> per_part_sum_marks;
Block sample_block = data.getSampleBlock();
for (const auto i : ext::range(0, parts.size())) for (const auto i : ext::range(0, parts.size()))
{ {
@ -247,8 +248,8 @@ std::vector<std::size_t> MergeTreeReadPool::fillPerPartInfo(
if (predict_block_size_bytes) if (predict_block_size_bytes)
{ {
per_part_size_predictor.emplace_back(std::make_shared<MergeTreeBlockSizePredictor>( per_part_size_predictor.emplace_back(std::make_unique<MergeTreeBlockSizePredictor>(
part.data_part, per_part_columns.back(), per_part_pre_columns.back())); part.data_part, column_names, sample_block));
} }
else else
per_part_size_predictor.emplace_back(nullptr); per_part_size_predictor.emplace_back(nullptr);

View File

@ -31,11 +31,11 @@ struct ReplicatedMergeTreeLogEntryData
{ {
enum Type enum Type
{ {
EMPTY, /// Not used. EMPTY, /// Not used.
GET_PART, /// Get the part from another replica. GET_PART, /// Get the part from another replica.
MERGE_PARTS, /// Merge the parts. MERGE_PARTS, /// Merge the parts.
DROP_RANGE, /// Delete the parts in the specified month in the specified number range. DROP_RANGE, /// Delete the parts in the specified month in the specified number range.
ATTACH_PART, /// Move a part from the `detached` or `unreplicated` directory. ATTACH_PART, /// Move a part from the `detached` or `unreplicated` directory.
}; };
String typeToString() const String typeToString() const
@ -43,9 +43,9 @@ struct ReplicatedMergeTreeLogEntryData
switch (type) switch (type)
{ {
case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART"; case ReplicatedMergeTreeLogEntryData::GET_PART: return "GET_PART";
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
default: default:
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR); throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR);
} }
@ -63,7 +63,8 @@ struct ReplicatedMergeTreeLogEntryData
/// The name of resulting part. /// The name of resulting part.
/// For DROP_RANGE, the name of a non-existent part. You need to remove all the parts covered by it. /// For DROP_RANGE, the name of a non-existent part. You need to remove all the parts covered by it.
String new_part_name; String new_part_name;
String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/).
mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'.
Strings parts_to_merge; Strings parts_to_merge;
bool deduplicate = false; /// Do deduplicate on merge bool deduplicate = false; /// Do deduplicate on merge

View File

@ -448,6 +448,60 @@ void ReplicatedMergeTreeQueue::removeGetsAndMergesInRange(zkutil::ZooKeeperPtr z
} }
bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & new_part_name, String & out_reason)
{
/// mutex should been already acquired
/// Let's check if the same part is now being created by another action.
if (future_parts.count(new_part_name))
{
out_reason = "Not executing log entry for part " + new_part_name
+ " because another log entry for the same part is being processed. This shouldn't happen often.";
return false;
/** When the corresponding action is completed, then `isNotCoveredByFuturePart` next time, will succeed,
* and queue element will be processed.
* Immediately in the `executeLogEntry` function it will be found that we already have a part,
* and queue element will be immediately treated as processed.
*/
}
/// A more complex check is whether another part is currently created by other action that will cover this part.
/// NOTE The above is redundant, but left for a more convenient message in the log.
ActiveDataPartSet::Part result_part;
ActiveDataPartSet::parsePartName(new_part_name, result_part);
/// It can slow down when the size of `future_parts` is large. But it can not be large, since `BackgroundProcessingPool` is limited.
for (const auto & future_part_name : future_parts)
{
ActiveDataPartSet::Part future_part;
ActiveDataPartSet::parsePartName(future_part_name, future_part);
if (future_part.contains(result_part))
{
out_reason = "Not executing log entry for part " + new_part_name
+ " because another log entry for covering part " + future_part_name + " is being processed.";
return false;
}
}
return true;
}
bool ReplicatedMergeTreeQueue::addFuturePartIfNotCoveredByThem(const String & part_name, const LogEntry & entry, String & reject_reason)
{
std::lock_guard<std::mutex> lock(mutex);
if (isNotCoveredByFuturePartsImpl(part_name, reject_reason))
{
CurrentlyExecuting::setActualPartName(entry, part_name, *this);
return true;
}
return false;
}
bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
const LogEntry & entry, const LogEntry & entry,
String & out_postpone_reason, String & out_postpone_reason,
@ -458,41 +512,10 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART)
{ {
/// Let's check if the same part is now being created by another action. if (!isNotCoveredByFuturePartsImpl(entry.new_part_name, out_postpone_reason))
if (future_parts.count(entry.new_part_name))
{ {
String reason = "Not executing log entry for part " + entry.new_part_name LOG_DEBUG(log, out_postpone_reason);
+ " because another log entry for the same part is being processed. This shouldn't happen often.";
LOG_DEBUG(log, reason);
out_postpone_reason = reason;
return false; return false;
/** When the corresponding action is completed, then `shouldExecuteLogEntry` next time, will succeed,
* and queue element will be processed.
* Immediately in the `executeLogEntry` function it will be found that we already have a part,
* and queue element will be immediately treated as processed.
*/
}
/// A more complex check is whether another part is currently created by other action that will cover this part.
/// NOTE The above is redundant, but left for a more convenient message in the log.
ActiveDataPartSet::Part result_part;
ActiveDataPartSet::parsePartName(entry.new_part_name, result_part);
/// It can slow down when the size of `future_parts` is large. But it can not be large, since `BackgroundProcessingPool` is limited.
for (const auto & future_part_name : future_parts)
{
ActiveDataPartSet::Part future_part;
ActiveDataPartSet::parsePartName(future_part_name, future_part);
if (future_part.contains(result_part))
{
String reason = "Not executing log entry for part " + entry.new_part_name
+ " because another log entry for covering part " + future_part_name + " is being processed.";
LOG_DEBUG(log, reason);
out_postpone_reason = reason;
return false;
}
} }
} }
@ -560,6 +583,24 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::CurrentlyExecuting(ReplicatedMerge
throw Exception("Tagging already tagged future part " + entry->new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); throw Exception("Tagging already tagged future part " + entry->new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
} }
void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName(const ReplicatedMergeTreeLogEntry & entry,
const String & actual_part_name, ReplicatedMergeTreeQueue & queue)
{
if (!entry.actual_new_part_name.empty())
throw Exception("Entry actual part isn't empty yet. This is a bug.", ErrorCodes::LOGICAL_ERROR);
entry.actual_new_part_name = actual_part_name;
/// Check if it is the same (and already added) part.
if (entry.actual_new_part_name == entry.new_part_name)
return;
if (!queue.future_parts.insert(entry.actual_new_part_name).second)
throw Exception("Attaching already exsisting future part " + entry.actual_new_part_name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
}
ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
{ {
std::lock_guard<std::mutex> lock(queue.mutex); std::lock_guard<std::mutex> lock(queue.mutex);
@ -569,6 +610,14 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting()
if (!queue.future_parts.erase(entry->new_part_name)) if (!queue.future_parts.erase(entry->new_part_name))
LOG_ERROR(queue.log, "Untagging already untagged future part " + entry->new_part_name + ". This is a bug."); LOG_ERROR(queue.log, "Untagging already untagged future part " + entry->new_part_name + ". This is a bug.");
if (!entry->actual_new_part_name.empty())
{
if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name))
LOG_ERROR(queue.log, "Untagging already untagged future part " + entry->actual_new_part_name + ". This is a bug.");
entry->actual_new_part_name.clear();
}
} }
@ -578,7 +627,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP
LogEntryPtr entry; LogEntryPtr entry;
for (Queue::iterator it = queue.begin(); it != queue.end(); ++it) for (auto it = queue.begin(); it != queue.end(); ++it)
{ {
if ((*it)->currently_executing) if ((*it)->currently_executing)
continue; continue;

View File

@ -88,6 +88,11 @@ private:
*/ */
bool shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason, MergeTreeDataMerger & merger, MergeTreeData & data); bool shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason, MergeTreeDataMerger & merger, MergeTreeData & data);
/** Check that part isn't in currently generating parts and isn't covered by them.
* Should be called under queue's mutex.
*/
bool isNotCoveredByFuturePartsImpl(const String & new_part_name, String & out_reason);
/// After removing the queue element, update the insertion times in the RAM. Running under queue_mutex. /// After removing the queue element, update the insertion times in the RAM. Running under queue_mutex.
/// Returns information about what times have changed - this information can be passed to updateTimesInZooKeeper. /// Returns information about what times have changed - this information can be passed to updateTimesInZooKeeper.
void updateTimesOnRemoval(const LogEntryPtr & entry, bool & min_unprocessed_insert_time_changed, bool & max_processed_insert_time_changed); void updateTimesOnRemoval(const LogEntryPtr & entry, bool & min_unprocessed_insert_time_changed, bool & max_processed_insert_time_changed);
@ -107,6 +112,10 @@ private:
/// Created only in the selectEntryToProcess function. It is called under mutex. /// Created only in the selectEntryToProcess function. It is called under mutex.
CurrentlyExecuting(ReplicatedMergeTreeQueue::LogEntryPtr & entry, ReplicatedMergeTreeQueue & queue); CurrentlyExecuting(ReplicatedMergeTreeQueue::LogEntryPtr & entry, ReplicatedMergeTreeQueue & queue);
/// In case of fetch, we determine actual part during the execution, so we need to update entry. It is called under mutex.
static void setActualPartName(const ReplicatedMergeTreeLogEntry & entry, const String & actual_part_name,
ReplicatedMergeTreeQueue & queue);
public: public:
~CurrentlyExecuting(); ~CurrentlyExecuting();
}; };
@ -165,6 +174,11 @@ public:
/// Prohibit merges in the specified range. /// Prohibit merges in the specified range.
void disableMergesInRange(const String & part_name); void disableMergesInRange(const String & part_name);
/** Check that part isn't in currently generating parts and isn't covered by them and add it to future_parts.
* Locks queue's mutex.
*/
bool addFuturePartIfNotCoveredByThem(const String & part_name, const LogEntry & entry, String & reject_reason);
/// Count the number of merges in the queue. /// Count the number of merges in the queue.
size_t countMerges(); size_t countMerges();

View File

@ -347,7 +347,7 @@ private:
}; };
BlockOutputStreamPtr StorageBuffer::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & query, const Settings & settings)
{ {
return std::make_shared<BufferBlockOutputStream>(*this); return std::make_shared<BufferBlockOutputStream>(*this);
} }

View File

@ -76,7 +76,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
/// Resets all buffers to the subordinate table. /// Resets all buffers to the subordinate table.
void shutdown() override; void shutdown() override;

View File

@ -253,7 +253,7 @@ BlockInputStreams StorageDistributed::read(
} }
BlockOutputStreamPtr StorageDistributed::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Settings & settings)
{ {
auto cluster = context.getCluster(cluster_name); auto cluster = context.getCluster(cluster_name);
@ -293,11 +293,12 @@ void StorageDistributed::shutdown()
} }
void StorageDistributed::reshardPartitions(ASTPtr query, const String & database_name, void StorageDistributed::reshardPartitions(
const ASTPtr & query, const String & database_name,
const Field & first_partition, const Field & last_partition, const Field & first_partition, const Field & last_partition,
const WeightedZooKeeperPaths & weighted_zookeeper_paths, const WeightedZooKeeperPaths & weighted_zookeeper_paths,
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
const Settings & settings) Context & context)
{ {
auto & resharding_worker = context.getReshardingWorker(); auto & resharding_worker = context.getReshardingWorker();
if (!resharding_worker.isStarted()) if (!resharding_worker.isStarted())
@ -377,7 +378,7 @@ void StorageDistributed::reshardPartitions(ASTPtr query, const String & database
ClusterProxy::AlterQueryConstructor alter_query_constructor; ClusterProxy::AlterQueryConstructor alter_query_constructor;
BlockInputStreams streams = ClusterProxy::Query{alter_query_constructor, cluster, alter_query_ptr, BlockInputStreams streams = ClusterProxy::Query{alter_query_constructor, cluster, alter_query_ptr,
context, settings, enable_shard_multiplexing}.execute(); context, context.getSettingsRef(), enable_shard_multiplexing}.execute();
/// This callback is called if an exception has occurred while attempting to read /// This callback is called if an exception has occurred while attempting to read
/// a block from a shard. This is to avoid a potential deadlock if other shards are /// a block from a shard. This is to avoid a potential deadlock if other shards are
@ -399,7 +400,7 @@ void StorageDistributed::reshardPartitions(ASTPtr query, const String & database
}; };
streams[0] = std::make_shared<UnionBlockInputStream<>>( streams[0] = std::make_shared<UnionBlockInputStream<>>(
streams, nullptr, settings.max_distributed_connections, exception_callback); streams, nullptr, context.getSettingsRef().max_distributed_connections, exception_callback);
streams.resize(1); streams.resize(1);
auto stream_ptr = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]); auto stream_ptr = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]);

View File

@ -75,7 +75,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void drop() override {} void drop() override {}
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; } void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
@ -85,11 +85,12 @@ public:
void shutdown() override; void shutdown() override;
void reshardPartitions(ASTPtr query, const String & database_name, void reshardPartitions(
const ASTPtr & query, const String & database_name,
const Field & first_partition, const Field & last_partition, const Field & first_partition, const Field & last_partition,
const WeightedZooKeeperPaths & weighted_zookeeper_paths, const WeightedZooKeeperPaths & weighted_zookeeper_paths,
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
const Settings & settings) override; Context & context) override;
/// From each replica, get a description of the corresponding local table. /// From each replica, get a description of the corresponding local table.
BlockInputStreams describe(const Context & context, const Settings & settings); BlockInputStreams describe(const Context & context, const Settings & settings);
@ -145,7 +146,7 @@ private:
String remote_database; String remote_database;
String remote_table; String remote_table;
Context & context; const Context & context;
Logger * log = &Logger::get("StorageDistributed"); Logger * log = &Logger::get("StorageDistributed");
/// Used to implement TableFunctionRemote. /// Used to implement TableFunctionRemote.

View File

@ -218,7 +218,7 @@ private:
}; };
BlockOutputStreamPtr StorageFile::write( BlockOutputStreamPtr StorageFile::write(
ASTPtr query, const ASTPtr & query,
const Settings & settings) const Settings & settings)
{ {
return std::make_shared<StorageFileBlockOutputStream>(*this); return std::make_shared<StorageFileBlockOutputStream>(*this);

View File

@ -82,7 +82,7 @@ public:
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write( BlockOutputStreamPtr write(
ASTPtr query, const ASTPtr & query,
const Settings & settings) override; const Settings & settings) override;
void drop() override; void drop() override;

View File

@ -919,7 +919,7 @@ BlockInputStreams StorageLog::read(
BlockOutputStreamPtr StorageLog::write( BlockOutputStreamPtr StorageLog::write(
ASTPtr query, const Settings & settings) const ASTPtr & query, const Settings & settings)
{ {
loadMarks(); loadMarks();
return std::make_shared<LogBlockOutputStream>(*this); return std::make_shared<LogBlockOutputStream>(*this);

View File

@ -78,7 +78,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;

View File

@ -119,7 +119,7 @@ BlockInputStreams StorageMaterializedView::read(
return getInnerTable()->read(column_names, query, context, settings, processed_stage, max_block_size, threads); return getInnerTable()->read(column_names, query, context, settings, processed_stage, max_block_size, threads);
} }
BlockOutputStreamPtr StorageMaterializedView::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Settings & settings)
{ {
return getInnerTable()->write(query, settings); return getInnerTable()->write(query, settings);
} }

View File

@ -37,7 +37,7 @@ public:
bool supportsParallelReplicas() const override { return getInnerTable()->supportsParallelReplicas(); } bool supportsParallelReplicas() const override { return getInnerTable()->supportsParallelReplicas(); }
bool supportsIndexForIn() const override { return getInnerTable()->supportsIndexForIn(); } bool supportsIndexForIn() const override { return getInnerTable()->supportsIndexForIn(); }
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void drop() override; void drop() override;
bool optimize(const String & partition, bool final, bool deduplicate, const Settings & settings) override; bool optimize(const String & partition, bool final, bool deduplicate, const Settings & settings) override;

View File

@ -149,7 +149,7 @@ BlockInputStreams StorageMemory::read(
BlockOutputStreamPtr StorageMemory::write( BlockOutputStreamPtr StorageMemory::write(
ASTPtr query, const Settings & settings) const ASTPtr & query, const Settings & settings)
{ {
return std::make_shared<MemoryBlockOutputStream>(*this); return std::make_shared<MemoryBlockOutputStream>(*this);
} }

View File

@ -54,7 +54,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void drop() override; void drop() override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; } void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }

View File

@ -183,7 +183,7 @@ BlockInputStreams StorageMerge::read(
for (auto & stream : source_streams) for (auto & stream : source_streams)
{ {
/// will throw if some columns not convertible /// will throw if some columns not convertible
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, table->getSampleBlock(), getSampleBlock()); stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
} }
} }
else else
@ -212,7 +212,7 @@ BlockInputStreams StorageMerge::read(
if (!streams.empty()) if (!streams.empty())
{ {
/// will throw if some columns not convertible /// will throw if some columns not convertible
stream = std::make_shared<CastTypeBlockInputStream>(context, stream, table->getSampleBlock(), getSampleBlock()); stream = std::make_shared<CastTypeBlockInputStream>(context, stream, getSampleBlock());
} }
return stream; return stream;
})); }));

View File

@ -124,7 +124,7 @@ BlockInputStreams StorageMergeTree::read(
return reader.read(column_names, query, context, settings, processed_stage, max_block_size, threads, nullptr, 0); return reader.read(column_names, query, context, settings, processed_stage, max_block_size, threads, nullptr, 0);
} }
BlockOutputStreamPtr StorageMergeTree::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & query, const Settings & settings)
{ {
return std::make_shared<MergeTreeBlockOutputStream>(*this); return std::make_shared<MergeTreeBlockOutputStream>(*this);
} }

View File

@ -83,7 +83,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
/** Perform the next step in combining the parts. /** Perform the next step in combining the parts.
*/ */

View File

@ -46,7 +46,7 @@ public:
return { std::make_shared<NullBlockInputStream>() }; return { std::make_shared<NullBlockInputStream>() };
} }
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override
{ {
return std::make_shared<NullBlockOutputStream>(); return std::make_shared<NullBlockOutputStream>();
} }

View File

@ -1251,8 +1251,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
if (do_fetch) if (do_fetch)
{ {
String covering_part; String replica = findReplicaHavingCoveringPart(entry, true);
String replica = findReplicaHavingCoveringPart(entry.new_part_name, true, covering_part);
static std::atomic_uint total_fetches {0}; static std::atomic_uint total_fetches {0};
if (data.settings.replicated_max_parallel_fetches && total_fetches >= data.settings.replicated_max_parallel_fetches) if (data.settings.replicated_max_parallel_fetches && total_fetches >= data.settings.replicated_max_parallel_fetches)
@ -1422,7 +1421,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
try try
{ {
if (!fetchPart(covering_part, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
return false; return false;
} }
catch (Exception & e) catch (Exception & e)
@ -1565,13 +1564,19 @@ void StorageReplicatedMergeTree::queueUpdatingThread()
{ {
setThreadName("ReplMTQueueUpd"); setThreadName("ReplMTQueueUpd");
bool update_in_progress = false;
while (!shutdown_called) while (!shutdown_called)
{ {
last_queue_update_attempt_time.store(time(nullptr)); if (!update_in_progress)
{
last_queue_update_start_time.store(time(nullptr));
update_in_progress = true;
}
try try
{ {
pullLogsToQueue(queue_updating_event); pullLogsToQueue(queue_updating_event);
last_successful_queue_update_attempt_time.store(time(nullptr)); last_queue_update_finish_time.store(time(nullptr));
update_in_progress = false;
queue_updating_event->wait(); queue_updating_event->wait();
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
@ -1965,7 +1970,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam
} }
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const String & part_name, bool active, String & out_covering_part_name) String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const LogEntry & entry, bool active)
{ {
auto zookeeper = getZooKeeper(); auto zookeeper = getZooKeeper();
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
@ -1985,10 +1990,9 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const String &
Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts"); Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
for (const String & part_on_replica : parts) for (const String & part_on_replica : parts)
{ {
if (part_on_replica == part_name || ActiveDataPartSet::contains(part_on_replica, part_name)) if (part_on_replica == entry.new_part_name || ActiveDataPartSet::contains(part_on_replica, entry.new_part_name))
{ {
if (largest_part_found.empty() if (largest_part_found.empty() || ActiveDataPartSet::contains(part_on_replica, largest_part_found))
|| ActiveDataPartSet::contains(part_on_replica, largest_part_found))
{ {
largest_part_found = part_on_replica; largest_part_found = part_on_replica;
} }
@ -1997,7 +2001,23 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const String &
if (!largest_part_found.empty()) if (!largest_part_found.empty())
{ {
out_covering_part_name = largest_part_found; bool the_same_part = largest_part_found == entry.new_part_name;
/// Make a check in case if selected part differs from source part
if (!the_same_part)
{
String reject_reason;
if (!queue.addFuturePartIfNotCoveredByThem(largest_part_found, entry, reject_reason))
{
LOG_INFO(log, "Will not fetch part " << largest_part_found << " covering " << entry.new_part_name << ". " << reject_reason);
return {};
}
}
else
{
entry.actual_new_part_name = entry.new_part_name;
}
return replica; return replica;
} }
} }
@ -2389,7 +2409,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const
} }
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & query, const Settings & settings)
{ {
assertNotReadonly(); assertNotReadonly();
@ -3261,16 +3281,16 @@ time_t StorageReplicatedMergeTree::getAbsoluteDelay() const
time_t max_processed_insert_time = 0; time_t max_processed_insert_time = 0;
queue.getInsertTimes(min_unprocessed_insert_time, max_processed_insert_time); queue.getInsertTimes(min_unprocessed_insert_time, max_processed_insert_time);
/// Load in reverse order to preserve consistency (successful update time must be after update start time). /// Load start time, then finish time to avoid reporting false delay when start time is updated
/// Probably doesn't matter because pullLogsToQueue() acts as a barrier. /// between loading of two variables.
time_t successful_queue_update_time = last_successful_queue_update_attempt_time.load(); time_t queue_update_start_time = last_queue_update_start_time.load();
time_t queue_update_time = last_queue_update_attempt_time.load(); time_t queue_update_finish_time = last_queue_update_finish_time.load();
time_t current_time = time(nullptr); time_t current_time = time(nullptr);
if (!queue_update_time) if (!queue_update_finish_time)
{ {
/// We have not even tried to update queue yet (perhaps replica is readonly). /// We have not updated queue even once yet (perhaps replica is readonly).
/// As we have no info about the current state of replication log, return effectively infinite delay. /// As we have no info about the current state of replication log, return effectively infinite delay.
return current_time; return current_time;
} }
@ -3279,12 +3299,12 @@ time_t StorageReplicatedMergeTree::getAbsoluteDelay() const
/// There are some unprocessed insert entries in queue. /// There are some unprocessed insert entries in queue.
return (current_time > min_unprocessed_insert_time) ? (current_time - min_unprocessed_insert_time) : 0; return (current_time > min_unprocessed_insert_time) ? (current_time - min_unprocessed_insert_time) : 0;
} }
else if (queue_update_time > successful_queue_update_time) else if (queue_update_start_time > queue_update_finish_time)
{ {
/// Queue is empty, but there are some in-flight or failed queue update attempts /// Queue is empty, but there are some in-flight or failed queue update attempts
/// (likely because of problems with connecting to ZooKeeper). /// (likely because of problems with connecting to ZooKeeper).
/// Return the time passed since last attempt. /// Return the time passed since last attempt.
return (current_time > queue_update_time) ? (current_time - queue_update_time) : 0; return (current_time > queue_update_start_time) ? (current_time - queue_update_start_time) : 0;
} }
else else
{ {
@ -3523,11 +3543,12 @@ void StorageReplicatedMergeTree::freezePartition(const Field & partition, const
} }
void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String & database_name, void StorageReplicatedMergeTree::reshardPartitions(
const ASTPtr & query, const String & database_name,
const Field & first_partition, const Field & last_partition, const Field & first_partition, const Field & last_partition,
const WeightedZooKeeperPaths & weighted_zookeeper_paths, const WeightedZooKeeperPaths & weighted_zookeeper_paths,
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
const Settings & settings) Context & context)
{ {
auto & resharding_worker = context.getReshardingWorker(); auto & resharding_worker = context.getReshardingWorker();
if (!resharding_worker.isStarted()) if (!resharding_worker.isStarted())

View File

@ -131,7 +131,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
bool optimize(const String & partition, bool final, bool deduplicate, const Settings & settings) override; bool optimize(const String & partition, bool final, bool deduplicate, const Settings & settings) override;
@ -142,11 +142,12 @@ public:
void fetchPartition(const Field & partition, const String & from, const Settings & settings) override; void fetchPartition(const Field & partition, const String & from, const Settings & settings) override;
void freezePartition(const Field & partition, const String & with_name, const Settings & settings) override; void freezePartition(const Field & partition, const String & with_name, const Settings & settings) override;
void reshardPartitions(ASTPtr query, const String & database_name, void reshardPartitions(
const ASTPtr & query, const String & database_name,
const Field & first_partition, const Field & last_partition, const Field & first_partition, const Field & last_partition,
const WeightedZooKeeperPaths & weighted_zookeeper_paths, const WeightedZooKeeperPaths & weighted_zookeeper_paths,
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
const Settings & settings) override; Context & context) override;
/** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper.
*/ */
@ -245,8 +246,8 @@ private:
* In ZK entries in chronological order. Here it is not necessary. * In ZK entries in chronological order. Here it is not necessary.
*/ */
ReplicatedMergeTreeQueue queue; ReplicatedMergeTreeQueue queue;
std::atomic<time_t> last_queue_update_attempt_time{0}; std::atomic<time_t> last_queue_update_start_time{0};
std::atomic<time_t> last_successful_queue_update_attempt_time{0}; std::atomic<time_t> last_queue_update_finish_time{0};
/** /replicas/me/is_active. /** /replicas/me/is_active.
*/ */
@ -445,10 +446,10 @@ private:
/** Find replica having specified part or any part that covers it. /** Find replica having specified part or any part that covers it.
* If active = true, consider only active replicas. * If active = true, consider only active replicas.
* If found, returns replica name and set 'out_covering_part_name' to name of found largest covering part. * If found, returns replica name and set 'entry->actual_new_part_name' to name of found largest covering part.
* If not found, returns empty string. * If not found, returns empty string.
*/ */
String findReplicaHavingCoveringPart(const String & part_name, bool active, String & out_covering_part_name); String findReplicaHavingCoveringPart(const LogEntry & entry, bool active);
/** Download the specified part from the specified replica. /** Download the specified part from the specified replica.
* If `to_detached`, the part is placed in the `detached` directory. * If `to_detached`, the part is placed in the `detached` directory.
@ -457,11 +458,11 @@ private:
*/ */
bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum); bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum);
/// Required only to avoid races between executeLogEntry and fetchPartition
std::unordered_set<String> currently_fetching_parts; std::unordered_set<String> currently_fetching_parts;
std::mutex currently_fetching_parts_mutex; std::mutex currently_fetching_parts_mutex;
/** With the quorum being tracked, add a replica to the quorum for the part. /// With the quorum being tracked, add a replica to the quorum for the part.
*/
void updateQuorum(const String & part_name); void updateQuorum(const String & part_name);
AbandonableLockInZooKeeper allocateBlockNumber(const String & month_name); AbandonableLockInZooKeeper allocateBlockNumber(const String & month_name);

View File

@ -66,7 +66,7 @@ void SetOrJoinBlockOutputStream::writeSuffix()
BlockOutputStreamPtr StorageSetOrJoinBase::write(ASTPtr query, const Settings & settings) BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & query, const Settings & settings)
{ {
++increment; ++increment;
return std::make_shared<SetOrJoinBlockOutputStream>(*this, path, path + "tmp/", toString(increment) + ".bin"); return std::make_shared<SetOrJoinBlockOutputStream>(*this, path, path + "tmp/", toString(increment) + ".bin");

View File

@ -25,7 +25,7 @@ public:
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
protected: protected:
StorageSetOrJoinBase( StorageSetOrJoinBase(

View File

@ -283,7 +283,7 @@ BlockInputStreams StorageStripeLog::read(
BlockOutputStreamPtr StorageStripeLog::write( BlockOutputStreamPtr StorageStripeLog::write(
ASTPtr query, const Settings & settings) const ASTPtr & query, const Settings & settings)
{ {
return std::make_shared<StripeLogBlockOutputStream>(*this); return std::make_shared<StripeLogBlockOutputStream>(*this);
} }

View File

@ -53,7 +53,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;

View File

@ -363,7 +363,7 @@ void TinyLogBlockOutputStream::addStream(const String & name, const IDataType &
void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column,
OffsetColumns & offset_columns, size_t level) OffsetColumns & offset_columns, size_t level)
{ {
if (type.isNullable()) if (type.isNullable())
{ {
@ -557,7 +557,7 @@ BlockInputStreams StorageTinyLog::read(
BlockOutputStreamPtr StorageTinyLog::write( BlockOutputStreamPtr StorageTinyLog::write(
ASTPtr query, const Settings & settings) const ASTPtr & query, const Settings & settings)
{ {
return std::make_shared<TinyLogBlockOutputStream>(*this); return std::make_shared<TinyLogBlockOutputStream>(*this);
} }

View File

@ -53,7 +53,7 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE, size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override; unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
void drop() override; void drop() override;

View File

@ -72,8 +72,8 @@ StoragePtr TableFunctionMerge::execute(ASTPtr ast_function, Context & context) c
args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[0], context); args[0] = evaluateConstantExpressionOrIdentidierAsLiteral(args[0], context);
args[1] = evaluateConstantExpressionAsLiteral(args[1], context); args[1] = evaluateConstantExpressionAsLiteral(args[1], context);
String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>(); String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String table_name_regexp = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>(); String table_name_regexp = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
return StorageMerge::create( return StorageMerge::create(
getName(), getName(),

View File

@ -30,213 +30,213 @@ MSG_SKIPPED = OP_SQUARE_BRACKET + colored(" SKIPPED ", "cyan", attrs=['bold']) +
def main(args): def main(args):
SERVER_DIED = False SERVER_DIED = False
def is_data_present(): def is_data_present():
proc = Popen(args.client, stdin=PIPE, stdout=PIPE, stderr=PIPE) proc = Popen(args.client, stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = proc.communicate("EXISTS TABLE test.hits") (stdout, stderr) = proc.communicate("EXISTS TABLE test.hits")
if proc.returncode != 0: if proc.returncode != 0:
raise CalledProcessError(proc.returncode, args.client, stderr) raise CalledProcessError(proc.returncode, args.client, stderr)
return stdout.startswith('1') return stdout.startswith('1')
def dump_report(destination, suite, test_case, report): def dump_report(destination, suite, test_case, report):
if destination is not None: if destination is not None:
destination_file = os.path.join(destination, suite, test_case + ".xml") destination_file = os.path.join(destination, suite, test_case + ".xml")
destination_dir = os.path.dirname(destination_file) destination_dir = os.path.dirname(destination_file)
if not os.path.exists(destination_dir): if not os.path.exists(destination_dir):
os.makedirs(destination_dir) os.makedirs(destination_dir)
with open(destination_file, 'w') as report_file: with open(destination_file, 'w') as report_file:
report_root = et.Element("testsuites", attrib = {'name': 'ClickHouse Tests'}) report_root = et.Element("testsuites", attrib = {'name': 'ClickHouse Tests'})
report_suite = et.Element("testsuite", attrib = {"name": suite}) report_suite = et.Element("testsuite", attrib = {"name": suite})
report_suite.append(report) report_suite.append(report)
report_root.append(report_suite) report_root.append(report_suite)
report_file.write(et.tostring(report_root, encoding = "UTF-8", xml_declaration=True, pretty_print=True)) report_file.write(et.tostring(report_root, encoding = "UTF-8", xml_declaration=True, pretty_print=True))
if args.zookeeper is None: if args.zookeeper is None:
try: try:
check_call(['grep', '-q', '<zookeeper', '/etc/clickhouse-server/config-preprocessed.xml'], ) check_call(['grep', '-q', '<zookeeper', '/etc/clickhouse-server/config-preprocessed.xml'], )
args.zookeeper = True args.zookeeper = True
except CalledProcessError: except CalledProcessError:
args.zookeeper = False args.zookeeper = False
if args.shard is None: if args.shard is None:
try: try:
check_call(['grep', '-qE', '"127.0.0.2|<listen_host>::</listen_host>"', '/etc/clickhouse-server/config-preprocessed.xml'], ) check_call(['grep', '-qE', '"127.0.0.2|<listen_host>::</listen_host>"', '/etc/clickhouse-server/config-preprocessed.xml'], )
args.shard = True args.shard = True
except CalledProcessError: except CalledProcessError:
# TODO: false here after setting <listen_host>::1</listen_host> # TODO: false here after setting <listen_host>::1</listen_host>
args.shard = True args.shard = True
base_dir = os.path.abspath(args.queries) base_dir = os.path.abspath(args.queries)
failures_total = 0 failures_total = 0
for suite in sorted(os.listdir(base_dir)): for suite in sorted(os.listdir(base_dir)):
if SERVER_DIED: if SERVER_DIED:
break break
suite_dir = os.path.join(base_dir, suite) suite_dir = os.path.join(base_dir, suite)
suite_re_obj = re.search('^[0-9]+_(.*)$', suite) suite_re_obj = re.search('^[0-9]+_(.*)$', suite)
if not suite_re_obj: #skip .gitignore and so on if not suite_re_obj: #skip .gitignore and so on
continue continue
suite = suite_re_obj.group(1) suite = suite_re_obj.group(1)
if os.path.isdir(suite_dir): if os.path.isdir(suite_dir):
print("\nRunning {} tests.\n".format(suite)) print("\nRunning {} tests.\n".format(suite))
failures = 0 failures = 0
if 'stateful' in suite and not is_data_present(): if 'stateful' in suite and not is_data_present():
print("Won't run stateful tests because test data wasn't loaded. See README.txt.") print("Won't run stateful tests because test data wasn't loaded. See README.txt.")
continue continue
for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir))): for case in sorted(filter(lambda case: re.search(args.test, case) if args.test else True, os.listdir(suite_dir))):
if SERVER_DIED: if SERVER_DIED:
break break
case_file = os.path.join(suite_dir, case) case_file = os.path.join(suite_dir, case)
if os.path.isfile(case_file) and (case.endswith('.sh') or case.endswith('.py') or case.endswith('.sql')): if os.path.isfile(case_file) and (case.endswith('.sh') or case.endswith('.py') or case.endswith('.sql')):
(name, ext) = os.path.splitext(case) (name, ext) = os.path.splitext(case)
report_testcase = et.Element("testcase", attrib = {"name": name}) report_testcase = et.Element("testcase", attrib = {"name": name})
print "{0:70}".format(name + ": "), print "{0:70}".format(name + ": "),
sys.stdout.flush() sys.stdout.flush()
if not args.zookeeper and 'zookeeper' in name: if not args.zookeeper and 'zookeeper' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"})) report_testcase.append(et.Element("skipped", attrib = {"message": "no zookeeper"}))
print(MSG_SKIPPED + " - no zookeeper") print(MSG_SKIPPED + " - no zookeeper")
elif not args.shard and 'shard' in name: elif not args.shard and 'shard' in name:
report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"})) report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"}))
print(MSG_SKIPPED + " - no shard") print(MSG_SKIPPED + " - no shard")
else: else:
reference_file = os.path.join(suite_dir, name) + '.reference' reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_dir, name) + '.stdout' stdout_file = os.path.join(suite_dir, name) + '.stdout'
stderr_file = os.path.join(suite_dir, name) + '.stderr' stderr_file = os.path.join(suite_dir, name) + '.stderr'
if ext == '.sql': if ext == '.sql':
command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file) command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file)
else: else:
command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file) command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file)
proc = Popen(command, shell = True) proc = Popen(command, shell = True)
start_time = datetime.now() start_time = datetime.now()
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None: while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0) sleep(0)
if proc.returncode is None: if proc.returncode is None:
try: try:
proc.kill() proc.kill()
except OSError as e: except OSError as e:
if e.errno != ESRCH: if e.errno != ESRCH:
raise raise
failure = et.Element("failure", attrib = {"message": "Timeout"}) failure = et.Element("failure", attrib = {"message": "Timeout"})
report_testcase.append(failure) report_testcase.append(failure)
failures = failures + 1 failures = failures + 1
print("{0} - Timeout!".format(MSG_FAIL)) print("{0} - Timeout!".format(MSG_FAIL))
else: else:
stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else '' stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else ''
stdout = unicode(stdout, errors='replace', encoding='utf-8') stdout = unicode(stdout, errors='replace', encoding='utf-8')
stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else '' stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else ''
stderr = unicode(stderr, errors='replace', encoding='utf-8') stderr = unicode(stderr, errors='replace', encoding='utf-8')
if proc.returncode != 0: if proc.returncode != 0:
failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)}) failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)})
report_testcase.append(failure) report_testcase.append(failure)
stdout_element = et.Element("system-out") stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout) stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element) report_testcase.append(stdout_element)
failures = failures + 1 failures = failures + 1
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode)) print("{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr: if stderr:
stderr_element = et.Element("system-err") stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr) stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element) report_testcase.append(stderr_element)
print(stderr) print(stderr)
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True SERVER_DIED = True
elif stderr: elif stderr:
failure = et.Element("failure", attrib = {"message": "having stderror"}) failure = et.Element("failure", attrib = {"message": "having stderror"})
report_testcase.append(failure) report_testcase.append(failure)
stderr_element = et.Element("system-err") stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr) stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element) report_testcase.append(stderr_element)
failures = failures + 1 failures = failures + 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8'))) print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout: elif 'Exception' in stdout:
failure = et.Element("error", attrib = {"message": "having exception"}) failure = et.Element("error", attrib = {"message": "having exception"})
report_testcase.append(failure) report_testcase.append(failure)
stdout_element = et.Element("system-out") stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout) stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element) report_testcase.append(stdout_element)
failures = failures + 1 failures = failures + 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8'))) print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file): elif not os.path.isfile(reference_file):
skipped = et.Element("skipped", attrib = {"message": "no reference file"}) skipped = et.Element("skipped", attrib = {"message": "no reference file"})
report_testcase.append(skipped) report_testcase.append(skipped)
print("{0} - no reference file".format(MSG_UNKNOWN)) print("{0} - no reference file".format(MSG_UNKNOWN))
else: else:
result_is_different = subprocess.call(['cmp', '-s', reference_file, stdout_file], stdout = PIPE) result_is_different = subprocess.call(['cmp', '-s', reference_file, stdout_file], stdout = PIPE)
if result_is_different: if result_is_different:
(diff, _) = Popen(['diff', '--side-by-side', reference_file, stdout_file], stdout = PIPE).communicate() (diff, _) = Popen(['diff', '--side-by-side', reference_file, stdout_file], stdout = PIPE).communicate()
diff = unicode(diff, errors='replace', encoding='utf-8') diff = unicode(diff, errors='replace', encoding='utf-8')
failure = et.Element("failure", attrib = {"message": "result differs with reference"}) failure = et.Element("failure", attrib = {"message": "result differs with reference"})
report_testcase.append(failure) report_testcase.append(failure)
stdout_element = et.Element("system-out") stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(diff) stdout_element.text = et.CDATA(diff)
report_testcase.append(stdout_element) report_testcase.append(stdout_element)
failures = failures + 1 failures = failures + 1
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff.encode('utf-8'))) print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff.encode('utf-8')))
else: else:
print(MSG_OK) print(MSG_OK)
if os.path.exists(stdout_file): if os.path.exists(stdout_file):
os.remove(stdout_file) os.remove(stdout_file)
if os.path.exists(stderr_file): if os.path.exists(stderr_file):
os.remove(stderr_file) os.remove(stderr_file)
dump_report(args.output, suite, name, report_testcase) dump_report(args.output, suite, name, report_testcase)
failures_total = failures_total + failures failures_total = failures_total + failures
if failures_total > 0: if failures_total > 0:
print(colored("\nHaving {0} errors!".format(failures_total), "red", attrs=["bold"])) print(colored("\nHaving {0} errors!".format(failures_total), "red", attrs=["bold"]))
sys.exit(1) sys.exit(1)
else: else:
print(colored("\nAll tests passed.", "green", attrs=["bold"])) print(colored("\nAll tests passed.", "green", attrs=["bold"]))
sys.exit(0) sys.exit(0)
if __name__ == '__main__': if __name__ == '__main__':
parser = ArgumentParser(description = 'ClickHouse functional tests') parser = ArgumentParser(description = 'ClickHouse functional tests')
parser.add_argument('-q', '--queries', default = 'queries', help = 'Path to queries dir') parser.add_argument('-q', '--queries', default = 'queries', help = 'Path to queries dir')
parser.add_argument('-c', '--client', default = 'clickhouse-client', help = 'Client program') parser.add_argument('-c', '--client', default = 'clickhouse-client', help = 'Client program')
parser.add_argument('-o', '--output', help = 'Output xUnit compliant test report directory') parser.add_argument('-o', '--output', help = 'Output xUnit compliant test report directory')
parser.add_argument('-t', '--timeout', type = int, default = 600, help = 'Timeout for each test case in seconds') parser.add_argument('-t', '--timeout', type = int, default = 600, help = 'Timeout for each test case in seconds')
parser.add_argument('test', nargs = '?', help = 'Optional test case name regex') parser.add_argument('test', nargs = '?', help = 'Optional test case name regex')
group = parser.add_mutually_exclusive_group(required = False) group = parser.add_mutually_exclusive_group(required = False)
group.add_argument('--zookeeper', action = 'store_true', default = None, dest = 'zookeeper', help = 'Run zookeeper related tests') group.add_argument('--zookeeper', action = 'store_true', default = None, dest = 'zookeeper', help = 'Run zookeeper related tests')
group.add_argument('--no-zookeeper', action = 'store_false', default = None, dest = 'zookeeper', help = 'Do not run zookeeper related tests') group.add_argument('--no-zookeeper', action = 'store_false', default = None, dest = 'zookeeper', help = 'Do not run zookeeper related tests')
group.add_argument('--shard', action = 'store_true', default = None, dest = 'shard', help = 'Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--shard', action = 'store_true', default = None, dest = 'shard', help = 'Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)')
group.add_argument('--no-shard', action = 'store_false', default = None, dest = 'shard', help = 'Do not run shard related tests') group.add_argument('--no-shard', action = 'store_false', default = None, dest = 'shard', help = 'Do not run shard related tests')
group.add_argument('--stop', action = 'store_true', default = None, dest = 'stop', help = 'Stop on network errors ') group.add_argument('--stop', action = 'store_true', default = None, dest = 'stop', help = 'Stop on network errors ')
args = parser.parse_args() args = parser.parse_args()
main(args) main(args)

View File

@ -1,6 +1,8 @@
1 0 1 0
2 0 2 0
45 0 45 0
PASSED 1000
PASSED 2000
250500 250500 1000 250500 250500 1000
PASSED
PASSED

View File

@ -10,6 +10,25 @@ clickhouse-client --preferred_block_size_bytes=52 -q "SELECT DISTINCT blockSize(
clickhouse-client --preferred_block_size_bytes=90 -q "SELECT DISTINCT blockSize(), ignore(p) FROM test.preferred_block_size_bytes" clickhouse-client --preferred_block_size_bytes=90 -q "SELECT DISTINCT blockSize(), ignore(p) FROM test.preferred_block_size_bytes"
clickhouse-client -q "DROP TABLE IF EXISTS test.preferred_block_size_bytes" clickhouse-client -q "DROP TABLE IF EXISTS test.preferred_block_size_bytes"
# PREWHERE using empty column
clickhouse-client -q "DROP TABLE IF EXISTS test.pbs"
clickhouse-client -q "CREATE TABLE test.pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree(p, p, 100)"
clickhouse-client -q "INSERT INTO test.pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000"
clickhouse-client -q "ALTER TABLE test.pbs ADD COLUMN s UInt8 DEFAULT 0"
clickhouse-client --preferred_block_size_bytes=100000 -q "SELECT count() FROM test.pbs PREWHERE s = 0"
clickhouse-client -q "INSERT INTO test.pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000"
clickhouse-client --preferred_block_size_bytes=100000 -q "SELECT count() FROM test.pbs PREWHERE s = 0"
clickhouse-client -q "DROP TABLE test.pbs"
# Nullable PREWHERE
clickhouse-client -q "DROP TABLE IF EXISTS test.nullable_prewhere"
clickhouse-client -q "CREATE TABLE test.nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree(p, p, 8)"
clickhouse-client -q "INSERT INTO test.nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001"
clickhouse-client -q "SELECT sum(d), sum(f), max(d) FROM test.nullable_prewhere PREWHERE NOT isNull(f)"
clickhouse-client -q "DROP TABLE IF EXISTS test.nullable_prewhere"
# Depend on 00282_merging test # Depend on 00282_merging test
pushd `dirname $0` > /dev/null pushd `dirname $0` > /dev/null
@ -25,11 +44,3 @@ cat "$SCRIPTDIR"/00282_merging.sql | clickhouse-client --preferred_block_size_by
cmp "$SCRIPTDIR"/00282_merging.reference preferred_block_size_bytes.stdout && echo PASSED || echo FAILED cmp "$SCRIPTDIR"/00282_merging.reference preferred_block_size_bytes.stdout && echo PASSED || echo FAILED
rm preferred_block_size_bytes.stdout rm preferred_block_size_bytes.stdout
# Nullable PREWHERE
clickhouse-client -q "DROP TABLE IF EXISTS test.nullable_prewhere"
clickhouse-client -q "CREATE TABLE test.nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree(p, p, 8)"
clickhouse-client -q "INSERT INTO test.nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001"
clickhouse-client -q "SELECT sum(d), sum(f), max(d) FROM test.nullable_prewhere PREWHERE NOT isNull(f)"
clickhouse-client -q "DROP TABLE IF EXISTS test.nullable_prewhere"

View File

@ -26,8 +26,8 @@
1 1
1 1
Array(UInt32) | Array(UInt64) Array(UInt32) | Array(UInt64)
[1] [1] [0]
[1] [1] [0]
[4294967290] [4294967290] [4294967290]
[4294967290] [4294967290] [4294967290]
[4294967299] [4294967299] [4294967299]

View File

@ -5,18 +5,18 @@ DROP TABLE IF EXISTS test.u32;
DROP TABLE IF EXISTS test.u64; DROP TABLE IF EXISTS test.u64;
DROP TABLE IF EXISTS test.merge_32_64; DROP TABLE IF EXISTS test.merge_32_64;
CREATE TABLE test.u32 (x UInt32) ENGINE = Memory; CREATE TABLE test.u32 (x UInt32, y UInt32 DEFAULT x) ENGINE = Memory;
CREATE TABLE test.u64 (x UInt64) ENGINE = Memory; CREATE TABLE test.u64 (x UInt64, y UInt64 DEFAULT x) ENGINE = Memory;
CREATE TABLE test.merge_32_64 (x UInt64) ENGINE = Merge(test, 'u32|u64'); CREATE TABLE test.merge_32_64 (x UInt64) ENGINE = Merge(test, 'u32|u64');
INSERT INTO test.u32 VALUES (1); INSERT INTO test.u32 (x) VALUES (1);
INSERT INTO test.u64 VALUES (1); INSERT INTO test.u64 (x) VALUES (1);
INSERT INTO test.u32 VALUES (4294967290); INSERT INTO test.u32 (x) VALUES (4294967290);
INSERT INTO test.u64 VALUES (4294967290); INSERT INTO test.u64 (x) VALUES (4294967290);
--now inserts 3. maybe need out of range check? --now inserts 3. maybe need out of range check?
--INSERT INTO test.u32 VALUES (4294967299); --INSERT INTO test.u32 VALUES (4294967299);
INSERT INTO test.u64 VALUES (4294967299); INSERT INTO test.u64 (x) VALUES (4294967299);
select ' = 1:'; select ' = 1:';
SELECT x FROM test.merge_32_64 WHERE x = 1; SELECT x FROM test.merge_32_64 WHERE x = 1;
@ -117,20 +117,20 @@ DROP TABLE IF EXISTS test.one;
DROP TABLE IF EXISTS test.two; DROP TABLE IF EXISTS test.two;
DROP TABLE IF EXISTS test.merge_one_two; DROP TABLE IF EXISTS test.merge_one_two;
CREATE TABLE test.one (x Array(UInt32)) ENGINE = Memory; CREATE TABLE test.one (x Array(UInt32), z String DEFAULT '', y Array(UInt32)) ENGINE = Memory;
CREATE TABLE test.two (x Array(UInt64)) ENGINE = Memory; CREATE TABLE test.two (x Array(UInt64), z String DEFAULT '', y Array(UInt64)) ENGINE = Memory;
CREATE TABLE test.merge_one_two (x Array(UInt64)) ENGINE = Merge(test, 'one|two'); CREATE TABLE test.merge_one_two (x Array(UInt64), z String, y Array(UInt64)) ENGINE = Merge(test, 'one|two');
INSERT INTO test.one VALUES ([1]); INSERT INTO test.one (x, y) VALUES ([1], [0]);
INSERT INTO test.two VALUES ([1]); INSERT INTO test.two (x, y) VALUES ([1], [0]);
INSERT INTO test.one VALUES ([4294967290]); INSERT INTO test.one (x, y) VALUES ([4294967290], [4294967290]);
INSERT INTO test.two VALUES ([4294967290]); INSERT INTO test.two (x, y) VALUES ([4294967290], [4294967290]);
INSERT INTO test.one VALUES ([4294967299]); INSERT INTO test.one (x, y) VALUES ([4294967299], [4294967299]);
INSERT INTO test.two VALUES ([4294967299]); INSERT INTO test.two (x, y) VALUES ([4294967299], [4294967299]);
SELECT x FROM test.merge_one_two WHERE x IN (1); SELECT x, y FROM test.merge_one_two WHERE x IN (1);
SELECT x FROM test.merge_one_two WHERE x IN (4294967290); SELECT x, y FROM test.merge_one_two WHERE x IN (4294967290);
SELECT x FROM test.merge_one_two WHERE x IN (4294967299); SELECT x, y FROM test.merge_one_two WHERE x IN (4294967299);
DROP TABLE IF EXISTS test.one; DROP TABLE IF EXISTS test.one;
DROP TABLE IF EXISTS test.two; DROP TABLE IF EXISTS test.two;

View File

@ -0,0 +1,138 @@
As of 2017-05-20, we have the following configuration in production:
ZooKeeper version is 3.5.1.
zoo.cfg:
```
# http://hadoop.apache.org/zookeeper/docs/current/zookeeperAdmin.html
# The number of milliseconds of each tick
tickTime=2000
# The number of ticks that the initial
# synchronization phase can take
initLimit=30000
# The number of ticks that can pass between
# sending a request and getting an acknowledgement
syncLimit=10
maxClientCnxns=2000
maxSessionTimeout=60000000
# the directory where the snapshot is stored.
dataDir=/opt/zookeeper/{{ cluster['name'] }}/data
# Place the dataLogDir to a separate physical disc for better performance
dataLogDir=/opt/zookeeper/{{ cluster['name'] }}/logs
autopurge.snapRetainCount=10
autopurge.purgeInterval=1
# To avoid seeks ZooKeeper allocates space in the transaction log file in
# blocks of preAllocSize kilobytes. The default block size is 64M. One reason
# for changing the size of the blocks is to reduce the block size if snapshots
# are taken more often. (Also, see snapCount).
preAllocSize=131072
# Clients can submit requests faster than ZooKeeper can process them,
# especially if there are a lot of clients. To prevent ZooKeeper from running
# out of memory due to queued requests, ZooKeeper will throttle clients so that
# there is no more than globalOutstandingLimit outstanding requests in the
# system. The default limit is 1,000.ZooKeeper logs transactions to a
# transaction log. After snapCount transactions are written to a log file a
# snapshot is started and a new transaction log file is started. The default
# snapCount is 10,000.
snapCount=3000000
# If this option is defined, requests will be will logged to a trace file named
# traceFile.year.month.day.
#traceFile=
# Leader accepts client connections. Default value is "yes". The leader machine
# coordinates updates. For higher update throughput at thes slight expense of
# read throughput the leader can be configured to not accept clients and focus
# on coordination.
leaderServes=yes
standaloneEnabled=false
dynamicConfigFile=/etc/zookeeper-{{ cluster['name'] }}/conf/zoo.cfg.dynamic
```
Java version:
```
Java(TM) SE Runtime Environment (build 1.8.0_25-b17)
Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode)
```
Java parameters:
```
NAME=zookeeper-{{ cluster['name'] }}
ZOOCFGDIR=/etc/$NAME/conf
# TODO this is really ugly
# How to find out, which jars are needed?
# seems, that log4j requires the log4j.properties file to be in the classpath
CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper/zookeeper-3.5.1-metrika.jar:/usr/share/zookeeper/slf4j-log4j12-1.7.5.jar:/usr/share/zookeeper/slf4j-api-1.7.5.jar:/usr/share/zookeeper/servlet-api-2.5-20081211.jar:/usr/share/zookeeper/netty-3.7.0.Final.jar:/usr/share/zookeeper/log4j-1.2.16.jar:/usr/share/zookeeper/jline-2.11.jar:/usr/share/zookeeper/jetty-util-6.1.26.jar:/usr/share/zookeeper/jetty-6.1.26.jar:/usr/share/zookeeper/javacc.jar:/usr/share/zookeeper/jackson-mapper-asl-1.9.11.jar:/usr/share/zookeeper/jackson-core-asl-1.9.11.jar:/usr/share/zookeeper/commons-cli-1.2.jar:/usr/src/java/lib/*.jar:/usr/etc/zookeeper"
ZOOCFG="$ZOOCFGDIR/zoo.cfg"
ZOO_LOG_DIR=/var/log/$NAME
USER=zookeeper
GROUP=zookeeper
PIDDIR=/var/run/$NAME
PIDFILE=$PIDDIR/$NAME.pid
SCRIPTNAME=/etc/init.d/$NAME
JAVA=/usr/bin/java
ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain"
ZOO_LOG4J_PROP="INFO,ROLLINGFILE"
JMXLOCALONLY=false
JAVA_OPTS="-Xms{{ cluster.get('xms','128M') }} \
-Xmx{{ cluster.get('xmx','1G') }} \
-Xloggc:/var/log/$NAME/zookeeper-gc.log \
-XX:+UseGCLogFileRotation \
-XX:NumberOfGCLogFiles=16 \
-XX:GCLogFileSize=16M \
-verbose:gc \
-XX:+PrintGCTimeStamps \
-XX:+PrintGCDateStamps \
-XX:+PrintGCDetails
-XX:+PrintTenuringDistribution \
-XX:+PrintGCApplicationStoppedTime \
-XX:+PrintGCApplicationConcurrentTime \
-XX:+PrintSafepointStatistics \
-XX:+UseParNewGC \
-XX:+UseConcMarkSweepGC \
-XX:+CMSParallelRemarkEnabled"
```
Salt init:
```
description "zookeeper-{{ cluster['name'] }} centralized coordination service"
start on runlevel [2345]
stop on runlevel [!2345]
respawn
limit nofile 8192 8192
pre-start script
[ -r "/etc/zookeeper-{{ cluster['name'] }}/conf/environment" ] || exit 0
. /etc/zookeeper-{{ cluster['name'] }}/conf/environment
[ -d $ZOO_LOG_DIR ] || mkdir -p $ZOO_LOG_DIR
chown $USER:$GROUP $ZOO_LOG_DIR
end script
script
. /etc/zookeeper-{{ cluster['name'] }}/conf/environment
[ -r /etc/default/zookeeper ] && . /etc/default/zookeeper
if [ -z "$JMXDISABLE" ]; then
JAVA_OPTS="$JAVA_OPTS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY"
fi
exec start-stop-daemon --start -c $USER --exec $JAVA --name zookeeper-{{ cluster['name'] }} \
-- -cp $CLASSPATH $JAVA_OPTS -Dzookeeper.log.dir=${ZOO_LOG_DIR} \
-Dzookeeper.root.logger=${ZOO_LOG4J_PROP} $ZOOMAIN $ZOOCFG
end script
```

View File

@ -133,7 +133,7 @@ For production releases, GCC is used.
### Run release script. ### Run release script.
``` ```
rm -f ../clickhouse*.deb rm -f ../clickhouse*.deb
./release --standalone ./release
``` ```
You will find built packages in parent directory. You will find built packages in parent directory.

View File

@ -33,7 +33,7 @@ https://github.com/yandex/ClickHouse/blob/master/doc/build.md
Install required packages. After that let's run the following command from directory with source code of ClickHouse: Install required packages. After that let's run the following command from directory with source code of ClickHouse:
``` ```
~/ClickHouse$ ./release --standalone ~/ClickHouse$ ./release
``` ```
The build successfully completed: The build successfully completed:

View File

@ -33,7 +33,7 @@ https://github.com/yandex/ClickHouse/blob/master/doc/build.md
Установим необходимые пакеты. После этого выполним следующую команду из директории с исходными кодами ClickHouse: Установим необходимые пакеты. После этого выполним следующую команду из директории с исходными кодами ClickHouse:
``` ```
~/ClickHouse$ ./release --standalone ~/ClickHouse$ ./release
``` ```
Сборка успешно завершена: Сборка успешно завершена:

1
doc/presentations Submodule

@ -0,0 +1 @@
Subproject commit b0b67a13a5ab20eac11496393acbe1e6c4f9c925

View File

@ -4,7 +4,7 @@ ARG repository="deb https://repo.yandex.ru/clickhouse/xenial/ dists/stable/main/
ARG version=\* ARG version=\*
RUN apt-get update && \ RUN apt-get update && \
apt-get install -y apt-transport-https && \ apt-get install -y apt-transport-https tzdata && \
mkdir -p /etc/apt/sources.list.d && \ mkdir -p /etc/apt/sources.list.d && \
echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \ echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \
apt-get update && \ apt-get update && \

View File

@ -4,7 +4,7 @@ ARG repository="deb https://repo.yandex.ru/clickhouse/xenial/ dists/stable/main/
ARG version=\* ARG version=\*
RUN apt-get update && \ RUN apt-get update && \
apt-get install -y apt-transport-https && \ apt-get install -y apt-transport-https tzdata && \
mkdir -p /etc/apt/sources.list.d && \ mkdir -p /etc/apt/sources.list.d && \
echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \ echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \
apt-get update && \ apt-get update && \

View File

@ -2,7 +2,7 @@
# #
# You can set these variables from the command line. # You can set these variables from the command line.
SPHINXOPTS = SPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER))
SPHINXBUILD = sphinx-build SPHINXBUILD = sphinx-build
PAPER = PAPER =
BUILDDIR = build BUILDDIR = build
@ -15,11 +15,17 @@ endif
# Internal variables. # Internal variables.
PAPEROPT_a4 = -D latex_paper_size=a4 PAPEROPT_a4 = -D latex_paper_size=a4
PAPEROPT_letter = -D latex_paper_size=letter PAPEROPT_letter = -D latex_paper_size=letter
ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) ru ALLSPHINXOPTS = $(SPHINXOPTS) en
# the i18n builder cannot share the environment and doctrees with the others # the i18n builder cannot share the environment and doctrees with the others
I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) ru I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) ru
.PHONY: help clean html dirhtml singlehtml pickle json htmlhelp qthelp devhelp epub latex latexpdf text man changes linkcheck doctest gettext .PHONY: default help clean html dirhtml singlehtml pickle json htmlhelp qthelp devhelp epub latex latexpdf text man changes linkcheck doctest gettext
default:
$(SPHINXBUILD) -b html $(SPHINXOPTS) ru $(BUILDDIR)/docs/ru
$(SPHINXBUILD) -b html $(SPHINXOPTS) en $(BUILDDIR)/docs/en
SPHINX_TEMPLATES=../_singlehtml_templates $(SPHINXBUILD) -b singlehtml $(SPHINXOPTS) ru $(BUILDDIR)/docs/ru/single
SPHINX_TEMPLATES=../_singlehtml_templates $(SPHINXBUILD) -b singlehtml $(SPHINXOPTS) en $(BUILDDIR)/docs/en/single
help: help:
@echo "Please use \`make <target>' where <target> is one of" @echo "Please use \`make <target>' where <target> is one of"
@ -175,3 +181,4 @@ pseudoxml:
$(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml $(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml
@echo @echo
@echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml." @echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml."

View File

@ -0,0 +1,10 @@
<h3>{{ _('Navigation') }}</h3>
{{ toctree(includehidden=theme_sidebar_includehidden, collapse=False) }}
{% if theme_extra_nav_links %}
<hr />
<ul>
{% for text, uri in theme_extra_nav_links.items() %}
<li class="toctree-l1"><a href="{{ uri }}">{{ text }}</a></li>
{% endfor %}
</ul>
{% endif %}

88
docs/_static/custom.css vendored Normal file
View File

@ -0,0 +1,88 @@
div.sphinxsidebarwrapper {
padding: 2px 10px;
}
div.sphinxsidebarwrapper p.logo {
float: left;
text-align: left;
margin: -4px 4px 0 0px;
}
div.sphinxsidebar a {
border-bottom: none;
color: #000;
}
pre {
padding: 4px;
}
input {
display: block;
margin-bottom: 4px;
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot);
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'),
url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg');
font-weight: 700;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot);
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'),
url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot);
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'),
url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot);
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'),
url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot);
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'),
url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}

1
docs/_static/logo.svg vendored Symbolic link
View File

@ -0,0 +1 @@
/Users/blinkov/ClickHouse/website/logo.svg

View File

@ -12,8 +12,9 @@
# All configuration values have a default; values that are commented out # All configuration values have a default; values that are commented out
# serve to show the default. # serve to show the default.
import sys import collections
import os import os
import sys
# If extensions (or modules to document with autodoc) are in another directory, # If extensions (or modules to document with autodoc) are in another directory,
# add these directories to sys.path here. If the directory is relative to the # add these directories to sys.path here. If the directory is relative to the
@ -29,11 +30,14 @@ import os
# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom # extensions coming with Sphinx (named 'sphinx.ext.*') or your custom
# ones. # ones.
extensions = [ extensions = [
'sphinx.ext.mathjax', 'alabaster'
] ]
# Add any paths that contain templates here, relative to this directory. # Add any paths that contain templates here, relative to this directory.
templates_path = ['_templates'] templates_path = [
'../_templates',
os.getenv('SPHINX_TEMPLATES', '')
]
# The suffix of source filenames. # The suffix of source filenames.
source_suffix = '.rst' source_suffix = '.rst'
@ -46,20 +50,20 @@ master_doc = 'index'
# General information about the project. # General information about the project.
project = u'ClickHouse' project = u'ClickHouse'
copyright = u'2017, Alexey Milovidov' copyright = u'20162017 Yandex LLC'
# The version info for the project you're documenting, acts as replacement for # The version info for the project you're documenting, acts as replacement for
# |version| and |release|, also used in various other places throughout the # |version| and |release|, also used in various other places throughout the
# built documents. # built documents.
# #
# The short X.Y version. # The short X.Y version.
version = '1' version = ''
# The full version, including alpha/beta/rc tags. # The full version, including alpha/beta/rc tags.
release = '1' release = ''
# The language for content autogenerated by Sphinx. Refer to documentation # The language for content autogenerated by Sphinx. Refer to documentation
# for a list of supported languages. # for a list of supported languages.
#language = None language = 'en'
# There are two options for replacing |today|: either, you set today to some # There are two options for replacing |today|: either, you set today to some
# non-false value, then it is used: # non-false value, then it is used:
@ -100,12 +104,29 @@ pygments_style = 'sphinx'
# The theme to use for HTML and HTML Help pages. See the documentation for # The theme to use for HTML and HTML Help pages. See the documentation for
# a list of builtin themes. # a list of builtin themes.
html_theme = 'default' html_theme = 'alabaster'
# Theme options are theme-specific and customize the look and feel of a theme # Theme options are theme-specific and customize the look and feel of a theme
# further. For a list of options available for each theme, see the # further. For a list of options available for each theme, see the
# documentation. # documentation.
#html_theme_options = {} font_family = '\'Yandex Sans Display Web\', Arial, sans-serif'
html_theme_options = {
'logo': 'logo.svg',
'logo_name': True,
'github_user': 'yandex',
'github_repo': 'ClickHouse',
'github_button': False,
'font_family': font_family,
'head_font_family': font_family,
'caption_font_family': font_family,
'show_powered_by': False,
'extra_nav_links': collections.OrderedDict([
('Documentation in Russian', '/docs/ru/'),
('Single page documentation', '/docs/en/single/'),
('Website home', '/'),
('GitHub', 'https://github.com/yandex/ClickHouse'),
])
}
# Add any paths that contain custom themes here, relative to this directory. # Add any paths that contain custom themes here, relative to this directory.
#html_theme_path = [] #html_theme_path = []
@ -129,7 +150,7 @@ html_theme = 'default'
# Add any paths that contain custom static files (such as style sheets) here, # Add any paths that contain custom static files (such as style sheets) here,
# relative to this directory. They are copied after the builtin static files, # relative to this directory. They are copied after the builtin static files,
# so a file named "default.css" will overwrite the builtin "default.css". # so a file named "default.css" will overwrite the builtin "default.css".
html_static_path = ['_static'] html_static_path = ['../_static']
# Add any extra paths that contain custom files (such as robots.txt or # Add any extra paths that contain custom files (such as robots.txt or
# .htaccess) here, relative to this directory. These files are copied # .htaccess) here, relative to this directory. These files are copied
@ -145,7 +166,14 @@ html_static_path = ['_static']
#html_use_smartypants = True #html_use_smartypants = True
# Custom sidebar templates, maps document names to template names. # Custom sidebar templates, maps document names to template names.
#html_sidebars = {} html_sidebars = {
'**': [
'about.html',
'navigation.html',
'relations.html',
'searchbox.html',
]
}
# Additional templates that should be rendered to pages, maps page names to # Additional templates that should be rendered to pages, maps page names to
# template names. # template names.
@ -161,10 +189,10 @@ html_static_path = ['_static']
#html_split_index = False #html_split_index = False
# If true, links to the reST sources are added to the pages. # If true, links to the reST sources are added to the pages.
#html_show_sourcelink = True html_show_sourcelink = False
# If true, "Created using Sphinx" is shown in the HTML footer. Default is True. # If true, "Created using Sphinx" is shown in the HTML footer. Default is True.
#html_show_sphinx = True html_show_sphinx = False
# If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. # If true, "(C) Copyright ..." is shown in the HTML footer. Default is True.
#html_show_copyright = True #html_show_copyright = True
@ -178,7 +206,7 @@ html_static_path = ['_static']
#html_file_suffix = None #html_file_suffix = None
# Output file base name for HTML help builder. # Output file base name for HTML help builder.
htmlhelp_basename = 'ClickHousedoc' htmlhelp_basename = 'ClickHouse'
# -- Options for LaTeX output --------------------------------------------- # -- Options for LaTeX output ---------------------------------------------
@ -199,7 +227,7 @@ latex_elements = {
# author, documentclass [howto, manual, or own class]). # author, documentclass [howto, manual, or own class]).
latex_documents = [ latex_documents = [
('index', 'ClickHouse.tex', u'ClickHouse Documentation', ('index', 'ClickHouse.tex', u'ClickHouse Documentation',
u'Alexey Milovidov', 'manual'), u'Yandex LLC', 'manual'),
] ]
# The name of an image file (relative to this directory) to place at the top of # The name of an image file (relative to this directory) to place at the top of
@ -229,7 +257,7 @@ latex_documents = [
# (source start file, name, description, authors, manual section). # (source start file, name, description, authors, manual section).
man_pages = [ man_pages = [
('index', 'clickhouse', u'ClickHouse Documentation', ('index', 'clickhouse', u'ClickHouse Documentation',
[u'Alexey Milovidov'], 1) [u'Yandex LLC'], 1)
] ]
# If true, show URL addresses after external links. # If true, show URL addresses after external links.
@ -243,7 +271,7 @@ man_pages = [
# dir menu entry, description, category) # dir menu entry, description, category)
texinfo_documents = [ texinfo_documents = [
('index', 'ClickHouse', u'ClickHouse Documentation', ('index', 'ClickHouse', u'ClickHouse Documentation',
u'Alexey Milovidov', 'ClickHouse', 'One line description of project.', u'Yandex LLC', 'ClickHouse', 'One line description of project.',
'Miscellaneous'), 'Miscellaneous'),
] ]

View File

@ -259,10 +259,14 @@ Example:
└─────────┴─────────────────────┴───────┘ └─────────┴─────────────────────┴───────┘
MACNumToString(num) MACNumToString(num)
~~~~~~~~~~~~~ ~~~~~~~~~~~~~~~~~~~
Takes a UInt64 number. Interprets it as an MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). Takes a UInt64 number. Interprets it as an MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form).
MACStringToNum(s) MACStringToNum(s)
~~~~~~~~ ~~~~~~~~~~~~~~~~~
The reverse function of MACNumToString. If the MAC address has an invalid format, it returns 0. The reverse function of MACNumToString. If the MAC address has an invalid format, it returns 0.
MACStringToOUI(s)
~~~~~~~~~~~~~~~~~
Takes MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). Returns first three octets as UInt64 number. If the MAC address has an invalid format, it returns 0.

View File

@ -35,4 +35,4 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro
roundAge(num) roundAge(num)
~~~~~~~ ~~~~~~~
Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to numbers from the set: 18, 25, 35, 45. This function is specific to Yandex.Metrica and used for implementing the report on user age. Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to numbers from the set: 18, 25, 35, 45, 55. This function is specific to Yandex.Metrica and used for implementing the report on user age.

View File

@ -84,8 +84,14 @@
Для продакшен использования, указывайте только элементы вида ip (IP-адреса и их маски), так как использование host и host_regexp может вызывать лишние задержки. Для продакшен использования, указывайте только элементы вида ip (IP-адреса и их маски), так как использование host и host_regexp может вызывать лишние задержки.
Далее указывается используемый профиль настроек пользователя (смотрите раздел "Профили настроек"). Вы можете указать профиль по умолчанию - default. Профиль может называться как угодно; один и тот же профиль может быть указан для разных пользователей. Наиболее важная вещь, которую вы можете прописать в профиле настроек - настройку readonly, равную 1, что обеспечивает доступ только на чтение. Далее указывается используемый профиль настроек пользователя (смотрите раздел "Профили настроек"). Вы можете указать профиль по умолчанию - ``default``. Профиль может называться как угодно; один и тот же профиль может быть указан для разных пользователей. Наиболее важная вещь, которую вы можете прописать в профиле настроек - настройку readonly, равную 1, что обеспечивает доступ только на чтение.
Затем указывается используемая квота (смотрите раздел "Квоты"). Вы можете указать квоту по умолчанию - ``default``. Она настроена в конфиге по умолчанию так, что только считает использование ресурсов, но никак их не ограничивает. Квота может называться как угодно; одна и та же квота может быть указана для разных пользователей - в этом случае, подсчёт использования ресурсов делается для каждого пользователя по отдельности. Затем указывается используемая квота (смотрите раздел "Квоты"). Вы можете указать квоту по умолчанию - ``default``. Она настроена в конфиге по умолчанию так, что только считает использование ресурсов, но никак их не ограничивает. Квота может называться как угодно; одна и та же квота может быть указана для разных пользователей - в этом случае, подсчёт использования ресурсов делается для каждого пользователя по отдельности.
Также в необязательном разделе ``<allow_databases>`` можно указать перечень баз, к которым у пользователя будет доступ. По умолчанию пользователю доступны все базы. Можно указать базу данных ``default``, в этом случае пользователь получит доступ к базе данных по умолчанию. Также в необязательном разделе ``<allow_databases>`` можно указать перечень баз, к которым у пользователя будет доступ. По умолчанию пользователю доступны все базы. Можно указать базу данных ``default``, в этом случае пользователь получит доступ к базе данных по умолчанию.
Доступ к БД ``system`` всегда считается разрешённым (так как эта БД используется для выполнения запросов).
Пользователь может получить список всех БД и таблиц в них с помощью запросов ``SHOW`` или системных таблиц, даже если у него нет доступа к отдельным ДБ.
Доступ к БД не связан с настройкой :ref:`query_complexity_readonly`. Невозможно дать полный доступ к одной БД и ``readonly`` к другой.

View File

@ -12,8 +12,9 @@
# All configuration values have a default; values that are commented out # All configuration values have a default; values that are commented out
# serve to show the default. # serve to show the default.
import sys import collections
import os import os
import sys
# If extensions (or modules to document with autodoc) are in another directory, # If extensions (or modules to document with autodoc) are in another directory,
# add these directories to sys.path here. If the directory is relative to the # add these directories to sys.path here. If the directory is relative to the
@ -29,11 +30,14 @@ import os
# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom # extensions coming with Sphinx (named 'sphinx.ext.*') or your custom
# ones. # ones.
extensions = [ extensions = [
'sphinx.ext.mathjax', 'alabaster'
] ]
# Add any paths that contain templates here, relative to this directory. # Add any paths that contain templates here, relative to this directory.
templates_path = ['_templates'] templates_path = [
'../_templates',
os.getenv('SPHINX_TEMPLATES', '')
]
# The suffix of source filenames. # The suffix of source filenames.
source_suffix = '.rst' source_suffix = '.rst'
@ -46,20 +50,20 @@ master_doc = 'index'
# General information about the project. # General information about the project.
project = u'ClickHouse' project = u'ClickHouse'
copyright = u'2017, Alexey Milovidov' copyright = u'20162017 Yandex LLC'
# The version info for the project you're documenting, acts as replacement for # The version info for the project you're documenting, acts as replacement for
# |version| and |release|, also used in various other places throughout the # |version| and |release|, also used in various other places throughout the
# built documents. # built documents.
# #
# The short X.Y version. # The short X.Y version.
version = '1' version = ''
# The full version, including alpha/beta/rc tags. # The full version, including alpha/beta/rc tags.
release = '1' release = ''
# The language for content autogenerated by Sphinx. Refer to documentation # The language for content autogenerated by Sphinx. Refer to documentation
# for a list of supported languages. # for a list of supported languages.
#language = None language = 'ru'
# There are two options for replacing |today|: either, you set today to some # There are two options for replacing |today|: either, you set today to some
# non-false value, then it is used: # non-false value, then it is used:
@ -100,12 +104,29 @@ pygments_style = 'sphinx'
# The theme to use for HTML and HTML Help pages. See the documentation for # The theme to use for HTML and HTML Help pages. See the documentation for
# a list of builtin themes. # a list of builtin themes.
html_theme = 'default' html_theme = 'alabaster'
# Theme options are theme-specific and customize the look and feel of a theme # Theme options are theme-specific and customize the look and feel of a theme
# further. For a list of options available for each theme, see the # further. For a list of options available for each theme, see the
# documentation. # documentation.
#html_theme_options = {} font_family = '\'Yandex Sans Display Web\', Arial, sans-serif'
html_theme_options = {
'logo': 'logo.svg',
'logo_name': True,
'github_user': 'yandex',
'github_repo': 'ClickHouse',
'github_button': False,
'font_family': font_family,
'head_font_family': font_family,
'caption_font_family': font_family,
'show_powered_by': False,
'extra_nav_links': collections.OrderedDict([
(u'Documentation in English', '/docs/en/'),
(u'Документация на одной странице', '/docs/ru/single/'),
(u'Главная страница сайта', '/'),
(u'GitHub', 'https://github.com/yandex/ClickHouse'),
])
}
# Add any paths that contain custom themes here, relative to this directory. # Add any paths that contain custom themes here, relative to this directory.
#html_theme_path = [] #html_theme_path = []
@ -129,7 +150,7 @@ html_theme = 'default'
# Add any paths that contain custom static files (such as style sheets) here, # Add any paths that contain custom static files (such as style sheets) here,
# relative to this directory. They are copied after the builtin static files, # relative to this directory. They are copied after the builtin static files,
# so a file named "default.css" will overwrite the builtin "default.css". # so a file named "default.css" will overwrite the builtin "default.css".
html_static_path = ['_static'] html_static_path = ['../_static']
# Add any extra paths that contain custom files (such as robots.txt or # Add any extra paths that contain custom files (such as robots.txt or
# .htaccess) here, relative to this directory. These files are copied # .htaccess) here, relative to this directory. These files are copied
@ -145,7 +166,14 @@ html_static_path = ['_static']
#html_use_smartypants = True #html_use_smartypants = True
# Custom sidebar templates, maps document names to template names. # Custom sidebar templates, maps document names to template names.
#html_sidebars = {} html_sidebars = {
'**': [
'about.html',
'navigation.html',
'relations.html',
'searchbox.html',
]
}
# Additional templates that should be rendered to pages, maps page names to # Additional templates that should be rendered to pages, maps page names to
# template names. # template names.
@ -161,10 +189,10 @@ html_static_path = ['_static']
#html_split_index = False #html_split_index = False
# If true, links to the reST sources are added to the pages. # If true, links to the reST sources are added to the pages.
#html_show_sourcelink = True html_show_sourcelink = False
# If true, "Created using Sphinx" is shown in the HTML footer. Default is True. # If true, "Created using Sphinx" is shown in the HTML footer. Default is True.
#html_show_sphinx = True html_show_sphinx = False
# If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. # If true, "(C) Copyright ..." is shown in the HTML footer. Default is True.
#html_show_copyright = True #html_show_copyright = True
@ -178,7 +206,7 @@ html_static_path = ['_static']
#html_file_suffix = None #html_file_suffix = None
# Output file base name for HTML help builder. # Output file base name for HTML help builder.
htmlhelp_basename = 'ClickHousedoc' htmlhelp_basename = 'ClickHouse'
# -- Options for LaTeX output --------------------------------------------- # -- Options for LaTeX output ---------------------------------------------
@ -199,7 +227,7 @@ latex_elements = {
# author, documentclass [howto, manual, or own class]). # author, documentclass [howto, manual, or own class]).
latex_documents = [ latex_documents = [
('index', 'ClickHouse.tex', u'ClickHouse Documentation', ('index', 'ClickHouse.tex', u'ClickHouse Documentation',
u'Alexey Milovidov', 'manual'), u'Yandex LLC', 'manual'),
] ]
# The name of an image file (relative to this directory) to place at the top of # The name of an image file (relative to this directory) to place at the top of
@ -229,7 +257,7 @@ latex_documents = [
# (source start file, name, description, authors, manual section). # (source start file, name, description, authors, manual section).
man_pages = [ man_pages = [
('index', 'clickhouse', u'ClickHouse Documentation', ('index', 'clickhouse', u'ClickHouse Documentation',
[u'Alexey Milovidov'], 1) [u'Yandex LLC'], 1)
] ]
# If true, show URL addresses after external links. # If true, show URL addresses after external links.
@ -243,7 +271,7 @@ man_pages = [
# dir menu entry, description, category) # dir menu entry, description, category)
texinfo_documents = [ texinfo_documents = [
('index', 'ClickHouse', u'ClickHouse Documentation', ('index', 'ClickHouse', u'ClickHouse Documentation',
u'Alexey Milovidov', 'ClickHouse', 'One line description of project.', u'Yandex LLC', 'ClickHouse', 'One line description of project.',
'Miscellaneous'), 'Miscellaneous'),
] ]

View File

@ -1,3 +1,5 @@
.. _configuration_files:
Конфигурационные файлы Конфигурационные файлы
====================== ======================

View File

@ -6,7 +6,7 @@
Словарь может полностью храниться в оперативке и периодически обновляться, или быть частично закэшированным в оперативке и динамически подгружать отсутствующие значения. Словарь может полностью храниться в оперативке и периодически обновляться, или быть частично закэшированным в оперативке и динамически подгружать отсутствующие значения.
Конфигурация внешних словарей находится в отдельном файле или файлах, указанных в конфигурационном параметре dictionaries_config. Конфигурация внешних словарей находится в отдельном файле или файлах, указанных в конфигурационном параметре dictionaries_config.
Этот параметр содержит абсолютный или относительный путь к файлу с конфигурацией словарей. Относительный путь - относительно директории с конфигурационным файлом сервера. Путь может содержать wildcard-ы * и ? - тогда рассматриваются все подходящие файлы. Пример: dictionaries/*.xml. Этот параметр содержит абсолютный или относительный путь к файлу с конфигурацией словарей. Относительный путь - относительно директории с конфигурационным файлом сервера. Путь может содержать wildcard-ы \* и ? - тогда рассматриваются все подходящие файлы. Пример: dictionaries/\*.xml.
Конфигурация словарей, а также множество файлов с конфигурацией, может обновляться без перезапуска сервера. Сервер проверяет обновления каждые 5 секунд. То есть, словари могут подключаться динамически. Конфигурация словарей, а также множество файлов с конфигурацией, может обновляться без перезапуска сервера. Сервер проверяет обновления каждые 5 секунд. То есть, словари могут подключаться динамически.
@ -68,6 +68,18 @@
Для отказоустойчивости, вы можете создать Distributed таблицу на localhost и прописать её. - -> Для отказоустойчивости, вы можете создать Distributed таблицу на localhost и прописать её. - ->
--> -->
<!-- Для <mysql> и <clickhouse> доступен атрибут <where>, позволяющий задать условие выбора
<clickhouse>
<host>example01-01-1</host>
<port>9000</port>
<user>default</user>
<password></password>
<db>default</db>
<table>ids</table>
<where>id=10</where>
</clickhouse>
-->
<!-- или источник - исполняемый файл. Если layout.cache - список нужных ключей будет записан в поток STDIN программы --> <!-- или источник - исполняемый файл. Если layout.cache - список нужных ключей будет записан в поток STDIN программы -->
<executable> <executable>
<!-- Путь или имя программы (если директория есть в переменной окружения PATH) и параметры --> <!-- Путь или имя программы (если директория есть в переменной окружения PATH) и параметры -->
@ -139,6 +151,14 @@
<!-- Можно считать отображение id -> attribute инъективным, чтобы оптимизировать GROUP BY. (по умолчанию, false) --> <!-- Можно считать отображение id -> attribute инъективным, чтобы оптимизировать GROUP BY. (по умолчанию, false) -->
<injective>true</injective> <injective>true</injective>
</attribute> </attribute>
<!-- Атрибут может быть выражением -->
<attribute>
<name>expr</name>
<type>UInt64</type>
<expression>rand64()</expression>
<null_value>0</null_value>
</attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </dictionaries>
@ -167,16 +187,21 @@ range_hashed
------------ ------------
В таблице прописаны какие-то данные для диапазонов дат, для каждого ключа. Дать возможность доставать эти данные для заданного ключа, для заданной даты. В таблице прописаны какие-то данные для диапазонов дат, для каждого ключа. Дать возможность доставать эти данные для заданного ключа, для заданной даты.
Пример: в таблице записаны скидки для каждого рекламодателя в виде:
::
id рекламодателя дата начала действия скидки дата конца величина Пример: таблица содержит скидки для каждого рекламодателя в виде:
123 2015-01-01 2015-01-15 0.15
123 2015-01-16 2015-01-31 0.25
456 2015-01-01 2015-01-15 0.05
Добавляем layout = range_hashed. +------------------+-----------------------------+------------+----------+
При использовании такого layout, в structure должны быть элементы range_min, range_max. | id рекламодателя | дата начала действия скидки | дата конца | величина |
+==================+=============================+============+==========+
| 123 | 2015-01-01 | 2015-01-15 | 0.15 |
+------------------+-----------------------------+------------+----------+
| 123 | 2015-01-16 | 2015-01-31 | 0.25 |
+------------------+-----------------------------+------------+----------+
| 456 | 2015-01-01 | 2015-01-15 | 0.05 |
+------------------+-----------------------------+------------+----------+
Добавляем ``layout = range_hashed``.
При использовании такого layout, в structure должны быть элементы ``range_min``, ``range_max``.
Пример: Пример:

View File

@ -261,10 +261,14 @@ runningDifference(x)
└─────────┴─────────────────────┴───────┘ └─────────┴─────────────────────┴───────┘
MACNumToString(num) MACNumToString(num)
~~~~~~~~~~~~~~~~~~~~ ~~~~~~~~~~~~~~~~~~~
Принимает число типа UInt64. Интерпретирует его, как MAC-адрес в big endian. Возвращает строку, содержащую соответствующий MAC-адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие). Принимает число типа UInt64. Интерпретирует его, как MAC-адрес в big endian. Возвращает строку, содержащую соответствующий MAC-адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие).
MACStringToNum(s) MACStringToNum(s)
~~~~~~~~~~~~~~~~~~ ~~~~~~~~~~~~~~~~~
Функция, обратная к MACNumToString. Если MAC адрес в неправильном формате, то возвращает 0. Функция, обратная к MACNumToString. Если MAC адрес в неправильном формате, то возвращает 0.
MACStringToOUI(s)
~~~~~~~~~~~~~~~~~
Принимает MAC адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие). Возвращает первые три октета как число в формате UInt64. Если MAC адрес в неправильном формате, то возвращает 0.

View File

@ -36,4 +36,4 @@ roundDuration(num)
roundAge(num) roundAge(num)
~~~~~~~~~~~~~ ~~~~~~~~~~~~~
Принимает число. Если число меньше 18 - возвращает 0. Иначе округляет число вниз до чисел из набора: 18, 25, 35, 45. Эта функция специфична для Яндекс.Метрики и предназначена для реализации отчёта по возрасту посетителей. Принимает число. Если число меньше 18 - возвращает 0. Иначе округляет число вниз до чисел из набора: 18, 25, 35, 45, 55. Эта функция специфична для Яндекс.Метрики и предназначена для реализации отчёта по возрасту посетителей.

View File

@ -60,4 +60,4 @@ ClickHouse поддерживает таблицы с первичным клю
14. Репликация данных, поддержка целостности данных на репликах. 14. Репликация данных, поддержка целостности данных на репликах.
---------------------------------------------------------------- ----------------------------------------------------------------
Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики. Система поддерживает полную идентичность данных на разных репликах. Восстановление после сбоя осуществляется автоматически, а в сложных случаях - "по кнопке". Используется асинхронная multimaster репликация. После записи на любую доступную реплику, данные распространяются на все остальные реплики. Система поддерживает полную идентичность данных на разных репликах. Восстановление после сбоя осуществляется автоматически, а в сложных случаях - "по кнопке".
Подробнее смотрите раздел "Репликация данных". Подробнее смотрите раздел :ref:`table_engines-replication`.

View File

@ -65,9 +65,9 @@ CREATE TABLE
``MATERIALIZED expr`` ``MATERIALIZED expr``
Материализованное выражение. Такой столбец не может быть указан при INSERT-е, то есть, он всегда вычисляется. Материализованное выражение. Такой столбец не может быть указан при INSERT, то есть, он всегда вычисляется.
При INSERT-е без указания списка столбцов, такие столбцы не рассматриваются. При INSERT без указания списка столбцов, такие столбцы не рассматриваются.
Также этот столбец не подставляется при использовании звёздочки в запросе SELECT - чтобы сохранить инвариант, что дамп, полученный путём SELECT *, можно вставить обратно в таблицу INSERT-ом без указания списка столбцов. Также этот столбец не подставляется при использовании звёздочки в запросе SELECT - чтобы сохранить инвариант, что дамп, полученный путём ``SELECT *``, можно вставить обратно в таблицу INSERT-ом без указания списка столбцов.
``ALIAS expr`` ``ALIAS expr``
@ -675,7 +675,7 @@ SELECT
``ARRAY JOIN`` - это, по сути, ``INNER JOIN`` с массивом. Пример: ``ARRAY JOIN`` - это, по сути, ``INNER JOIN`` с массивом. Пример:
.. code-block:: sql ::
:) CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory :) CREATE TABLE arrays_test (s String, arr Array(UInt8)) ENGINE = Memory
@ -728,7 +728,7 @@ SELECT
Для массива в секции ARRAY JOIN может быть указан алиас. В этом случае, элемент массива будет доступен под этим алиасом, а сам массив - под исходным именем. Пример: Для массива в секции ARRAY JOIN может быть указан алиас. В этом случае, элемент массива будет доступен под этим алиасом, а сам массив - под исходным именем. Пример:
.. code-block:: sql ::
:) SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a :) SELECT s, arr, a FROM arrays_test ARRAY JOIN arr AS a
@ -748,7 +748,7 @@ SELECT
В секции ARRAY JOIN может быть указано несколько массивов одинаковых размеров через запятую. В этом случае, JOIN делается с ними одновременно (прямая сумма, а не прямое произведение). Пример: В секции ARRAY JOIN может быть указано несколько массивов одинаковых размеров через запятую. В этом случае, JOIN делается с ними одновременно (прямая сумма, а не прямое произведение). Пример:
.. code-block:: sql ::
:) SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped :) SELECT s, arr, a, num, mapped FROM arrays_test ARRAY JOIN arr AS a, arrayEnumerate(arr) AS num, arrayMap(x -> x + 1, arr) AS mapped
@ -784,7 +784,7 @@ SELECT
ARRAY JOIN также работает с вложенными структурами данных. Пример: ARRAY JOIN также работает с вложенными структурами данных. Пример:
.. code-block:: sql ::
:) CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory :) CREATE TABLE nested_test (s String, nest Nested(x UInt8, y UInt32)) ENGINE = Memory
@ -839,7 +839,7 @@ ARRAY JOIN также работает с вложенными структур
При указании имени вложенной структуры данных в ARRAY JOIN, смысл такой же, как ARRAY JOIN со всеми элементами-массивами, из которых она состоит. Пример: При указании имени вложенной структуры данных в ARRAY JOIN, смысл такой же, как ARRAY JOIN со всеми элементами-массивами, из которых она состоит. Пример:
.. code-block:: sql ::
:) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x, nest.y
@ -859,7 +859,7 @@ ARRAY JOIN также работает с вложенными структур
Такой вариант тоже имеет смысл: Такой вариант тоже имеет смысл:
.. code-block:: sql ::
:) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x :) SELECT s, nest.x, nest.y FROM nested_test ARRAY JOIN nest.x
@ -879,7 +879,7 @@ ARRAY JOIN также работает с вложенными структур
Алиас для вложенной структуры данных можно использовать, чтобы выбрать как результат JOIN-а, так и исходный массив. Пример: Алиас для вложенной структуры данных можно использовать, чтобы выбрать как результат JOIN-а, так и исходный массив. Пример:
.. code-block:: sql ::
:) SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n :) SELECT s, n.x, n.y, nest.x, nest.y FROM nested_test ARRAY JOIN nest AS n
@ -899,7 +899,7 @@ ARRAY JOIN также работает с вложенными структур
Пример использования функции arrayEnumerate: Пример использования функции arrayEnumerate:
.. code-block:: sql ::
:) SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num :) SELECT s, n.x, n.y, nest.x, nest.y, num FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(nest.x) AS num
@ -962,7 +962,7 @@ JOIN-ы бывают нескольких видов:
Пример: Пример:
.. code-block:: sql ::
SELECT SELECT
CounterID, CounterID,
@ -1033,7 +1033,7 @@ PREWHERE имеет смысл использовать, если есть ус
Например, полезно писать PREWHERE для запросов, которые вынимают много столбцов, но в которых фильтрация производится лишь по нескольким столбцам. Например, полезно писать PREWHERE для запросов, которые вынимают много столбцов, но в которых фильтрация производится лишь по нескольким столбцам.
PREWHERE поддерживается только таблицами семейства *MergeTree. PREWHERE поддерживается только таблицами семейства ``*MergeTree``.
В запросе могут быть одновременно указаны секции PREWHERE и WHERE. В этом случае, PREWHERE идёт перед WHERE. В запросе могут быть одновременно указаны секции PREWHERE и WHERE. В этом случае, PREWHERE идёт перед WHERE.
@ -1289,7 +1289,7 @@ n и m должны быть неотрицательными целыми чи
Оператор IN и подзапрос могут встречаться в любой части запроса, в том числе в агрегатных и лямбда функциях. Оператор IN и подзапрос могут встречаться в любой части запроса, в том числе в агрегатных и лямбда функциях.
Пример: Пример:
.. code-block:: sql ::
SELECT SELECT
EventDate, EventDate,
@ -1316,11 +1316,17 @@ n и m должны быть неотрицательными целыми чи
за каждый день после 17 марта считаем долю хитов, сделанных посетителями, которые заходили на сайт 17 марта. за каждый день после 17 марта считаем долю хитов, сделанных посетителями, которые заходили на сайт 17 марта.
Подзапрос в секции IN на одном сервере всегда выполняется только один раз. Зависимых подзапросов не существует. Подзапрос в секции IN на одном сервере всегда выполняется только один раз. Зависимых подзапросов не существует.
.. _queries-distributed-subrequests:
Распределённые подзапросы Распределённые подзапросы
""""""""""""""""""""""""" """""""""""""""""""""""""
Существует два варианта IN-ов с подзапросами (аналогично для JOIN-ов): обычный ``IN`` / ``JOIN`` и ``GLOBAL IN`` / ``GLOBAL JOIN``. Они отличаются способом выполнения при распределённой обработке запроса. Существует два варианта IN-ов с подзапросами (аналогично для JOIN-ов): обычный ``IN`` / ``JOIN`` и ``GLOBAL IN`` / ``GLOBAL JOIN``. Они отличаются способом выполнения при распределённой обработке запроса.
.. attention::
Помните, что алгоритмы, описанные ниже, могут работать иначе в зависимости от :ref:`настройки <settings-distributed_product_mode>` ``distributed_product_mode``.
При использовании обычного IN-а, запрос отправляется на удалённые серверы, и на каждом из них выполняются подзапросы в секциях ``IN`` / ``JOIN``. При использовании обычного IN-а, запрос отправляется на удалённые серверы, и на каждом из них выполняются подзапросы в секциях ``IN`` / ``JOIN``.
При использовании ``GLOBAL IN`` / ``GLOBAL JOIN-а``, сначала выполняются все подзапросы для ``GLOBAL IN`` / ``GLOBAL JOIN-ов``, и результаты складываются во временные таблицы. Затем эти временные таблицы передаются на каждый удалённый сервер, и на них выполняются запросы, с использованием этих переданных временных данных. При использовании ``GLOBAL IN`` / ``GLOBAL JOIN-а``, сначала выполняются все подзапросы для ``GLOBAL IN`` / ``GLOBAL JOIN-ов``, и результаты складываются во временные таблицы. Затем эти временные таблицы передаются на каждый удалённый сервер, и на них выполняются запросы, с использованием этих переданных временных данных.
@ -1343,7 +1349,8 @@ n и m должны быть неотрицательными целыми чи
.. code-block:: sql .. code-block:: sql
SELECT uniq(UserID) FROM local_table`` SELECT uniq(UserID) FROM local_table
, выполнен параллельно на каждом из них до стадии, позволяющей объединить промежуточные результаты; затем промежуточные результаты вернутся на сервер-инициатор запроса, будут на нём объединены, и финальный результат будет отправлен клиенту. , выполнен параллельно на каждом из них до стадии, позволяющей объединить промежуточные результаты; затем промежуточные результаты вернутся на сервер-инициатор запроса, будут на нём объединены, и финальный результат будет отправлен клиенту.

View File

@ -1,7 +1,22 @@
Настройки Настройки
========== ==========
Здесь будут рассмотрены настройки, которые можно задать с помощью запроса SET или в конфигурационном файле. Напомню, что эти настройки могут быть выставлены в пределах сессии или глобально. Настройки, которые можно задать только в конфигурационном файле сервера, здесь рассмотрены не будут. Описанные в разделе настройки могут быть заданы следующими способами:
* Глобально.
В конфигурационных файлах сервера.
* Для сессии.
При запуске консольного клиента ClickHouse в интерактивном режиме отправьте запрос ``SET setting=value``.
* Для запроса.
* При запуске консольного клиента ClickHouse в неинтерактивном режиме установите параметр запуска ``--setting=value``.
* При использовании HTTP API передавайте cgi-параметры (``URL?setting_1=value&setting_2=value...``).
Настройки, которые можно задать только в конфигурационном файле сервера, в разделе не рассматриваются.
.. toctree:: .. toctree::
:glob: :glob:

View File

@ -15,6 +15,8 @@
``any (только для group_by_overflow_mode)`` - продолжить агрегацию по ключам, которые успели войти в набор, но не добавлять новые ключи в набор. ``any (только для group_by_overflow_mode)`` - продолжить агрегацию по ключам, которые успели войти в набор, но не добавлять новые ключи в набор.
.. _query_complexity_readonly:
readonly readonly
-------- --------
При значении 0 можно выполнять любые запросы. При значении 0 можно выполнять любые запросы.
@ -90,7 +92,11 @@ result_overflow_mode
Использование break по смыслу похоже на LIMIT. Использование break по смыслу похоже на LIMIT.
max_execution_time max_execution_time
<<<<<<< HEAD
-------------------
=======
------------------ ------------------
>>>>>>> upstream/master
Максимальное время выполнения запроса в секундах. Максимальное время выполнения запроса в секундах.
На данный момент не проверяется при одной из стадий сортировки а также при слиянии и финализации агрегатных функций. На данный момент не проверяется при одной из стадий сортировки а также при слиянии и финализации агрегатных функций.

View File

@ -1,3 +1,47 @@
.. _settings-distributed_product_mode:
distributed_product_mode
------------------------
Изменяет поведение :ref:`распределенных подзапросов <queries-distributed-subrequests>`, т.е. в тех случаях, когда запрос содержит произведение распределённых таблиц.
ClickHouse применяет настройку в том случае, когда в подзапросах на любом уровне встретилась распределенная таблица, которая существует на локальном сервере и имеет больше одного шарда.
Условия применения:
* Только подзапросы для IN, JOIN.
* Только если в секции FROM используется распределённая таблица.
* Не используется в случае табличной функции :ref:`remote <table_functions-remote>`.
Возможные значения:
.. list-table::
:widths: 20 80
:header-rows: 1
* - Значение
- Поведение ClickHouse
* - ``deny`` (по умолчанию)
- Генерирует исключение.
* - ``allow``
- Выполняет запрос без изменения логики.
* - ``global``
- Преобразует ``IN`` в ``GLOBAL IN``, ``JOIN`` в ``GLOBAL JOIN``.
* - ``local``
- Преобразует все вхождения Distributed-таблиц в соответствующие им удалённые таблицы.
.. _settings-settings-fallback_to_stale_replicas_for_distributed_queries:
fallback_to_stale_replicas_for_distributed_queries
--------------------------------------------------
Форсирует запрос в устаревшую реплику в случае, если актуальные данные недоступны. Смотрите :ref:`table_engines-replication`.
Из устаревших реплик таблицы ClickHouse выбирает наиболее актуальную.
Используется при выполнении ``SELECT`` из распределенной таблицы, которая указывает на реплицированные таблицы.
По умолчанию - 1 (включена).
max_block_size max_block_size
-------------- --------------
Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки одного блока достаточно эффективны, но при этом существуют заметные издержки на каждый блок. ``max_block_size`` - это рекомендация, какого размера блоки (в количестве строк) загружать из таблицы. Размер блока должен быть не слишком маленьким, чтобы издержки на каждый блок оставались незаметными, и не слишком большим, чтобы запрос с LIMIT-ом, который завершается уже после первого блока, выполнялся быстро; чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность. Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки одного блока достаточно эффективны, но при этом существуют заметные издержки на каждый блок. ``max_block_size`` - это рекомендация, какого размера блоки (в количестве строк) загружать из таблицы. Размер блока должен быть не слишком маленьким, чтобы издержки на каждый блок оставались незаметными, и не слишком большим, чтобы запрос с LIMIT-ом, который завершается уже после первого блока, выполнялся быстро; чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность.
@ -22,7 +66,21 @@ max_insert_block_size
``По умолчанию - 1 048 576.`` ``По умолчанию - 1 048 576.``
Это намного больше, чем max_block_size. Это сделано, потому что некоторые движки таблиц (*MergeTree) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа *MergeTree, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. Это намного больше, чем ``max_block_size``. Это сделано, потому что некоторые движки таблиц (``*MergeTree``) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа ``*MergeTree``, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке.
.. _settings_settings_max_replica_delay_for_distributed_queries:
max_replica_delay_for_distributed_queries
------------------------------------------
Отключает отстающие реплики при распределенных запросах. Смотрите :ref:`table_engines-replication`.
Устанавливает время в секундах. Если оставание реплики больше установленного значения, то реплика не используется.
Значение по умолчанию: 0 (отключено).
Используется при выполнении ``SELECT`` из распределенной таблицы, которая указывает на реплицированные таблицы.
max_threads max_threads
----------- -----------
@ -48,7 +106,7 @@ max_compress_block_size
min_compress_block_size min_compress_block_size
----------------------- -----------------------
Для таблиц типа *MergeTree. В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min_compress_block_size. По умолчанию - 65 536. Для таблиц типа :ref:`MergeTree <table_engines-mergetree>`. В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min_compress_block_size. По умолчанию - 65 536.
Реальный размер блока, если несжатых данных меньше max_compress_block_size, будет не меньше этого значения и не меньше объёма данных на одну засечку. Реальный размер блока, если несжатых данных меньше max_compress_block_size, будет не меньше этого значения и не меньше объёма данных на одну засечку.
@ -136,6 +194,8 @@ replace_running_query
Эта настройка, выставленная в 1, используется в Яндекс.Метрике для реализации suggest-а значений для условий сегментации. После ввода очередного символа, если старый запрос ещё не выполнился, его следует отменить. Эта настройка, выставленная в 1, используется в Яндекс.Метрике для реализации suggest-а значений для условий сегментации. После ввода очередного символа, если старый запрос ещё не выполнился, его следует отменить.
.. _settings-load_balancing:
load_balancing load_balancing
-------------- --------------
На какие реплики (среди живых реплик) предпочитать отправлять запрос (при первой попытке) при распределённой обработке запроса. На какие реплики (среди живых реплик) предпочитать отправлять запрос (при первой попытке) при распределённой обработке запроса.

View File

@ -0,0 +1,91 @@
GraphiteMergeTree
-----------------
Движок предназначен для rollup (прореживания и агрегирования/усреднения) данных `Graphite <http://graphite.readthedocs.io/en/latest/index.html>`_. Он может быть интересен разработчикам, которые хотят использовать ClickHouse как хранилище данных для Graphite.
Graphite хранит в ClickHouse полные данные, а получать их может следующими способами:
* Без прореживания.
Используется движок :ref:`MergeTree <table_engines-mergetree>`.
* С прореживанием.
Используется движок ``GraphiteMergeTree``.
Движок наследует свойства `MergeTree`. Настройки прореживания данных размещаются в :ref:`общей конфигурации <configuration_files>` ClickHouse (config.xml).
Использование движка
^^^^^^^^^^^^^^^^^^^^
Таблица с данными Graphite должна содержать как минимум следующие поля:
* ``Path`` - имя метрики (сенсора Graphite).
* ``Time`` - время измерения.
* ``Value`` - значение метрики в момент времени Time.
* ``Version`` - настройка, которая определяет какое значение метрики с одинаковыми Path и Time останется в базе.
Шаблон правил rollup: ::
pattern
regexp
function
age -> precision
...
pattern
...
default
function
age -> precision
...
При обработке записи ClickHouse проверит правила в секции ```pattern```. Если имя метрики соответствует шаблону ```regexp```, то применяются правила из ```pattern```, в противном случае из ```default```.
Поля шаблона правил.
+---------------+----------------------------------------------------------------------------------------------------------------------------+
| Поле | Описание |
+===============+============================================================================================================================+
| ``age`` | Минимальный возраст данных в секундах. |
+---------------+----------------------------------------------------------------------------------------------------------------------------+
| ``function`` | Имя агрегирующей функции, которую следует применить к данным, чей возраст оказался в интервале ``[age, age + precision]``. |
+---------------+----------------------------------------------------------------------------------------------------------------------------+
| ``precision`` | Точность определения возраста данных в секундах. |
+---------------+----------------------------------------------------------------------------------------------------------------------------+
| ``regexp`` | Шаблон имени метрики. |
+---------------+----------------------------------------------------------------------------------------------------------------------------+
Пример настройки:
.. code-block:: xml
<graphite_rollup>
<pattern>
<regexp>click_cost</regexp>
<function>any</function>
<retention>
<age>0</age>
<precision>5</precision>
</retention>
<retention>
<age>86400</age>
<precision>60</precision>
</retention>
</pattern>
<default>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup>

View File

@ -1,3 +1,5 @@
.. _table_engines-mergetree:
MergeTree MergeTree
--------- ---------

View File

@ -1,3 +1,5 @@
.. _table_engines-replication:
Репликация данных Репликация данных
----------------- -----------------
@ -45,7 +47,7 @@ ReplicatedSummingMergeTree
Если в конфигурационном файле не настроен ZooKeeper, то вы не сможете создать реплицируемые таблицы, а уже имеющиеся реплицируемые таблицы будут доступны в режиме только на чтение. Если в конфигурационном файле не настроен ZooKeeper, то вы не сможете создать реплицируемые таблицы, а уже имеющиеся реплицируемые таблицы будут доступны в режиме только на чтение.
При запросах SELECT, ZooKeeper не используется. То есть, репликация никак не влияет на производительность SELECT-ов - запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах SELECT, ZooKeeper не используется. То есть, репликация никак не влияет на производительность SELECT-ов - запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах к распределенным реплицированным таблицам поведение ClickHouse регулируется настройками :ref:`settings_settings_max_replica_delay_for_distributed_queries` и :ref:`settings-settings-fallback_to_stale_replicas_for_distributed_queries`.
При каждом запросе INSERT (точнее, на каждый вставляемый блок данных; запрос INSERT содержит один блок, или по блоку на каждые max_insert_block_size = 1048576 строк), делается около десятка записей в ZooKeeper в рамках нескольких транзакций. Это приводит к некоторому увеличению задержек при INSERT-е, по сравнению с нереплицируемыми таблицами. Но если придерживаться обычных рекомендаций - вставлять данные пачками не более одного INSERT-а в секунду, то это не составляет проблем. На всём кластере ClickHouse, использующим для координации один кластер ZooKeeper, может быть в совокупности несколько сотен INSERT-ов в секунду. Пропускная способность при вставке данных (количество строчек в секунду) такая же высокая, как для нереплицируемых таблиц. При каждом запросе INSERT (точнее, на каждый вставляемый блок данных; запрос INSERT содержит один блок, или по блоку на каждые max_insert_block_size = 1048576 строк), делается около десятка записей в ZooKeeper в рамках нескольких транзакций. Это приводит к некоторому увеличению задержек при INSERT-е, по сравнению с нереплицируемыми таблицами. Но если придерживаться обычных рекомендаций - вставлять данные пачками не более одного INSERT-а в секунду, то это не составляет проблем. На всём кластере ClickHouse, использующим для координации один кластер ZooKeeper, может быть в совокупности несколько сотен INSERT-ов в секунду. Пропускная способность при вставке данных (количество строчек в секунду) такая же высокая, как для нереплицируемых таблиц.
@ -72,8 +74,7 @@ ReplicatedSummingMergeTree
Также добавляются два параметра в начало списка параметров - путь к таблице в ZooKeeper, имя реплики в ZooKeeper. Также добавляются два параметра в начало списка параметров - путь к таблице в ZooKeeper, имя реплики в ZooKeeper.
Пример: Пример: ::
::
ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192)
@ -124,6 +125,7 @@ ReplicatedSummingMergeTree
Если обнаруживается, что локальный набор данных слишком сильно отличается от ожидаемого, то срабатывает защитный механизм - сервер сообщает об этом в лог и отказывается запускаться. Это сделано, так как такой случай может свидетельствовать об ошибке конфигурации - например, если реплика одного шарда была случайно сконфигурирована, как реплика другого шарда. Тем не менее, пороги защитного механизма поставлены довольно низкими, и такая ситуация может возникнуть и при обычном восстановлении после сбоя. В этом случае, восстановление делается полуавтоматически - "по кнопке". Если обнаруживается, что локальный набор данных слишком сильно отличается от ожидаемого, то срабатывает защитный механизм - сервер сообщает об этом в лог и отказывается запускаться. Это сделано, так как такой случай может свидетельствовать об ошибке конфигурации - например, если реплика одного шарда была случайно сконфигурирована, как реплика другого шарда. Тем не менее, пороги защитного механизма поставлены довольно низкими, и такая ситуация может возникнуть и при обычном восстановлении после сбоя. В этом случае, восстановление делается полуавтоматически - "по кнопке".
Для запуска восстановления, создайте в ZooKeeper узел ``/path_to_table/replica_name/flags/force_restore_data`` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: Для запуска восстановления, создайте в ZooKeeper узел ``/path_to_table/replica_name/flags/force_restore_data`` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц:
.. code-block:: bash .. code-block:: bash
sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data

View File

@ -1,67 +1,62 @@
.. _table_functions-remote:
remote remote
------ ------
``remote('addresses_expr', db, table[, 'user'[, 'password']])`` Позволяет обратиться к удалённым серверам без создания таблицы типа ``Distributed``.
или Сигнатуры: ::
``remote('addresses_expr', db.table[, 'user'[, 'password']])`` remote('addresses_expr', db, table[, 'user'[, 'password']])
remote('addresses_expr', db.table[, 'user'[, 'password']])
- позволяет обратиться к удалённым серверам без создания таблицы типа Distributed.
``addresses_expr`` - выражение, генерирующее адреса удалённых серверов. ``addresses_expr`` - выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера - это ``хост:порт``, или только ``хост``. Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. Порт - TCP-порт удалённого сервера. Если порт не указан, используется ``tcp_port`` из конфигурационного файла сервера (по умолчанию - 9000).
Это может быть просто один адрес сервера. Адрес сервера - это хост:порт, или только хост. Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. Порт - TCP-порт удалённого сервера. Если порт не указан, используется tcp_port из конфигурационного файла сервера (по умолчанию - 9000). .. important:: С IPv6-адресом обязательно указывать порт.
Замечание: в качестве исключения, при указании IPv6-адреса, обязательно также указывать порт. Примеры: ::
Примеры: example01-01-1
:: example01-01-1:9000
localhost
127.0.0.1
[::]:9000
[2a02:6b8:0:1111::11]:9000
example01-01-1 Адреса можно указать через запятую, в этом случае ClickHouse обработает запрос как распределённый, т.е. отправит его по всем указанным адресам как на шарды с разными данными.
example01-01-1:9000
localhost
127.0.0.1
[::]:9000
[2a02:6b8:0:1111::11]:9000
Могут быть указаны адреса через запятую - в этом случае, запрос пойдёт на все указанные адреса (как на шарды с разными данными) и будет обработан распределённо. Пример: ::
Пример: example01-01-1,example01-02-1
::
example01-01-1,example01-02-1 Часть выражения может быть указана в фигурных скобках. Предыдущий пример может быть записан следующим образом: ::
Часть выражения может быть указана в фигурных скобках. Предыдущий пример может быть записан следующим образом: example01-0{1,2}-1
::
example01-0{1,2}-1 В фигурных скобках может быть указан диапазон (неотрицательных целых) чисел через две точки. В этом случае, диапазон раскрывается в множество значений, генерирующих адреса шардов. Если запись первого числа начинается с нуля, то значения формируются с таким же выравниванием нулями. Предыдущий пример может быть записан следующим образом: ::
В фигурных скобках может быть указан диапазон (неотрицательных целых) чисел через две точки. В этом случае, диапазон раскрывается в множество значений, генерирующих адреса шардов. Если запись первого числа начинается с нуля, то значения формируются с таким же выравниванием нулями. Предыдущий пример может быть записан следующим образом: example01-{01..02}-1
::
example01-{01..02}-1
При наличии нескольких пар фигурных скобок, генерируется прямое произведение соответствующих множеств. При наличии нескольких пар фигурных скобок, генерируется прямое произведение соответствующих множеств.
Адреса или их фрагменты в фигурных скобках, могут быть указаны через символ |. В этом случае, соответствующие множества адресов понимаются как реплики - запрос будет отправлен на первую живую реплику. При этом, реплики перебираются в порядке, согласно текущей настройке load_balancing. Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае, соответствующие множества адресов понимаются как реплики - запрос будет отправлен на первую живую реплику. При этом, реплики перебираются в порядке, согласно текущей настройке :ref:`load_balancing <settings-load_balancing>`.
Пример: Пример: ::
::
example01-{01..02}-{1|2} example01-{01..02}-{1|2}
В этом примере указано два шарда, в каждом из которых имеется две реплики. В этом примере указано два шарда, в каждом из которых имеется две реплики.
Количество генерируемых адресов ограничено некоторой константой - сейчас это 1000 штук. Количество генерируемых адресов ограничено константой - сейчас это 1000 штук.
Использование табличной функции remote менее оптимально, чем создание таблицы типа Distributed, так как в этом случае, соединения с серверами устанавливаются заново при каждом запросе, в случае задания имён хостов, делается резолвинг имён, а также не ведётся подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов, всегда создавайте Distributed таблицу заранее, не используйте табличную функцию remote. Использование табличной функции ``remote`` менее оптимально, чем создание таблицы типа ``Distributed``, так как в этом случае, соединения с серверами устанавливаются заново при каждом запросе, в случае задания имён хостов, делается резолвинг имён, а также не ведётся подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов, всегда создавайте ``Distributed`` таблицу заранее, не используйте табличную функцию ``remote``.
Табличная функция remote может быть полезна для следующих случаев: Табличная функция ``remote`` может быть полезна для следующих случаях:
* обращение на конкретный сервер в целях сравнения данных, отладки и тестирования; * обращение на конкретный сервер в целях сравнения данных, отладки и тестирования;
* запросы между разными кластерами ClickHouse в целях исследований; * запросы между разными кластерами ClickHouse в целях исследований;
* нечастых распределённых запросов, задаваемых вручную; * нечастых распределённых запросов, задаваемых вручную;
* распределённых запросов, где набор серверов определяется каждый раз заново. * распределённых запросов, где набор серверов определяется каждый раз заново.
Имя пользователя может быть не задано - тогда используется имя пользователя 'default'. Если пользователь не задан,то используется ``default``.
Пароль может быть не задан - тогда используется пустой пароль. Если пароль не задан, то используется пустой пароль.

View File

@ -24,6 +24,9 @@ do
gen_revision_author gen_revision_author
git push git push
exit 0 exit 0
elif [[ $1 == '--head' ]]; then
REVISION=`git rev-parse HEAD`
shift
else else
echo "Unknown option $1" echo "Unknown option $1"
exit 2 exit 2

171
website/404.html Normal file
View File

@ -0,0 +1,171 @@
<!DOCTYPE html>
<html lang="ru">
<head>
<meta charset="utf-8"/>
<title>ClickHouse — 404</title>
<style type="text/css">
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot);
src: url(https://yastatic.net/adv-www/_/yy5JveR58JFkc97waf-xp0i6_jM.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/CYblzLEXzCqQIvrYs7QKQe2omRk.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/pUcnOdRwl83MvPPzrNomhyletnA.woff) format('woff'),
url(https://yastatic.net/adv-www/_/vNFEmXOcGYKJ4AAidUprHWoXrLU.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/0w7OcWZM_QLP8x-LQUXFOgXO6dE.svg#YandexSansTextWeb-Bold) format('svg');
font-weight: 700;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot);
src: url(https://yastatic.net/adv-www/_/LI6l3L2RqcgxBe2pXmuUha37czQ.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/z3MYElcut0R2MF_Iw1RDNrstgYs.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/1jvKJ_-hCXl3s7gmFl-y_-UHTaI.woff) format('woff'),
url(https://yastatic.net/adv-www/_/9nzjfpCR2QHvK1EzHpDEIoVFGuY.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/gwyBTpxSwkFCF1looxqs6JokKls.svg#YandexSansTextWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Text Web';
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot);
src: url(https://yastatic.net/adv-www/_/ayAFYoY8swgBLhq_I56tKj2JftU.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/lGQcYklLVV0hyvz1HFmFsUTj8_0.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/f0AAJ9GJ4iiwEmhG-7PWMHk6vUY.woff) format('woff'),
url(https://yastatic.net/adv-www/_/4UDe4nlVvgEJ-VmLWNVq3SxCsA.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/EKLr1STNokPqxLAQa_RyN82pL98.svg#YandexSansTextWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot);
src: url(https://yastatic.net/adv-www/_/H63jN0veW07XQUIA2317lr9UIm8.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/sUYVCPUAQE7ExrvMS7FoISoO83s.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/v2Sve_obH3rKm6rKrtSQpf-eB7U.woff) format('woff'),
url(https://yastatic.net/adv-www/_/PzD8hWLMunow5i3RfJ6WQJAL7aI.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/lF_KG5g4tpQNlYIgA0e77fBSZ5s.svg#YandexSansDisplayWeb-Regular) format('svg');
font-weight: 400;
font-style: normal;
font-stretch: normal
}
@font-face {
font-family: 'Yandex Sans Display Web';
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot);
src: url(https://yastatic.net/adv-www/_/g8_MyyKVquSZ3xEL6tarK__V9Vw.eot?#iefix) format('embedded-opentype'),
url(https://yastatic.net/adv-www/_/LGiRvlfqQHlWR9YKLhsw5e7KGNA.woff2) format('woff2'),
url(https://yastatic.net/adv-www/_/40vXwNl4eYYMgteIVgLP49dwmfc.woff) format('woff'),
url(https://yastatic.net/adv-www/_/X6zG5x_wO8-AtwJ-vDLJcKC5228.ttf) format('truetype'),
url(https://yastatic.net/adv-www/_/ZKhaR0m08c8CRRL77GtFKoHcLYA.svg#YandexSansDisplayWeb-Light) format('svg');
font-weight: 300;
font-style: normal;
font-stretch: normal
}
body {
background: #fff;
font: 300 12pt/150% 'Yandex Sans Text Web', Arial, sans-serif;
}
.page {
width: 900px;
margin: auto;
}
h1 {
font-family: 'Yandex Sans Display Web', Arial, sans-serif;
font-size: 100px;
font-weight: normal;
margin-top: 100px;
margin-bottom: 0;
text-align: center;
padding-top: 27px;
}
.title_link, .title_link:active, .title_link:visited, .title_link:link, .title_link:hover {
text-decoration: none;
color: #000;
}
h2 {
font: normal 50px 'Yandex Sans Display Web', Arial, sans-serif;
text-align: center;
margin-top: 35px;
margin-bottom: 50px;
}
.orange {
fill: #fc0;
}
.red {
fill: #f00
}
</style>
</head>
<body>
<div class="page">
<div>
<div style="float: left; margin-right: -100%; margin-top: 0; margin-left: 3px;">
<a href="/">
<svg xmlns="http://www.w3.org/2000/svg" width="90" height="80" viewBox="0 0 9 8">
<path class="red" d="M0,7 h1 v1 h-1 z"></path>
<path class="orange" d="M0,0 h1 v7 h-1 z"></path>
<path class="orange" d="M2,0 h1 v8 h-1 z"></path>
<path class="orange" d="M4,0 h1 v8 h-1 z"></path>
<path class="orange" d="M6,0 h1 v8 h-1 z"></path>
<path class="orange" d="M8,3.25 h1 v1.5 h-1 z"></path>
</svg>
</a>
</div>
<h1 id="main_title"><a class="title_link" href="/">ClickHouse</a></h1>
<h2>404 Not Found</h2>
<a href="/">&larr; Return to home page</a>
</div>
<!-- Yandex.Metrika counter -->
<script type="text/javascript">
(function (d, w, c) {
(w[c] = w[c] || []).push(function() {
try {
w.yaCounter18343495 = new Ya.Metrika2({
id:18343495,
clickmap:true,
trackLinks:true,
accurateTrackBounce:true,
webvisor:true
});
} catch(e) { }
});
var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script>
<noscript>
<div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=""/></div>
</noscript>
<!-- /Yandex.Metrika counter -->
</body>
</html>

View File

@ -226,16 +226,31 @@ function getParams() {
</script> </script>
<!-- Yandex.Metrika counter --> <!-- Yandex.Metrika counter -->
<script src="https://mc.yandex.ru/metrika/watch.js" type="text/javascript"></script>
<script type="text/javascript"> <script type="text/javascript">
try { var yaCounter18343495 = new Ya.Metrika({id:18343495, (function (d, w, c) {
webvisor:true, (w[c] = w[c] || []).push(function() {
clickmap:true, try {
trackLinks:true, w.yaCounter18343495 = new Ya.Metrika2({
accurateTrackBounce:true, id:18343495,
trackHash:true, clickmap:true,
params: getParams()}); trackLinks:true,
} catch(e) { } accurateTrackBounce:true,
webvisor:true
});
} catch(e) { }
});
var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script> </script>
<noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt="" /></div></noscript> <noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt="" /></div></noscript>
<!-- /Yandex.Metrika counter --> <!-- /Yandex.Metrika counter -->

View File

@ -8,12 +8,15 @@ var htmlmin = require('gulp-htmlmin');
var minifyInline = require('gulp-minify-inline'); var minifyInline = require('gulp-minify-inline');
var del = require('del'); var del = require('del');
var connect = require('gulp-connect'); var connect = require('gulp-connect');
var run = require('gulp-run');
var outputDir = 'public'; var outputDir = 'public';
var docsDir = '../docs/build/docs';
var paths = { var paths = {
htmls: ['*.html', '!reference_ru.html', '!reference_en.html'], htmls: ['*.html', '!reference_ru.html', '!reference_en.html'],
reference: ['reference_ru.html', 'reference_en.html'], reference: ['reference_ru.html', 'reference_en.html'],
docs: [docsDir + '/**'],
scripts: ['*.js', '!gulpfile.js'], scripts: ['*.js', '!gulpfile.js'],
styles: ['*.css'], styles: ['*.css'],
images: ['*.png', '*.ico'], images: ['*.png', '*.ico'],
@ -22,7 +25,7 @@ var paths = {
}; };
gulp.task('clean', function () { gulp.task('clean', function () {
return del([outputDir + '**']); return del([outputDir + '/**']);
}); });
gulp.task('reference', [], function () { gulp.task('reference', [], function () {
@ -32,6 +35,13 @@ gulp.task('reference', [], function () {
.pipe(connect.reload()) .pipe(connect.reload())
}); });
gulp.task('docs', [], function () {
run('cd ' + docsDir + '; make');
return gulp.src(paths.docs)
.pipe(gulp.dest(outputDir + '/docs'))
.pipe(connect.reload())
});
gulp.task('presentations', [], function () { gulp.task('presentations', [], function () {
return gulp.src(paths.presentations) return gulp.src(paths.presentations)
.pipe(gulp.dest(outputDir + '/presentations')) .pipe(gulp.dest(outputDir + '/presentations'))
@ -44,7 +54,7 @@ gulp.task('robotstxt', [], function () {
.pipe(connect.reload()) .pipe(connect.reload())
}); });
gulp.task('htmls', ['reference', 'robotstxt', 'presentations'], function () { gulp.task('htmls', ['reference', 'docs', 'robotstxt', 'presentations'], function () {
return gulp.src(paths.htmls) return gulp.src(paths.htmls)
.pipe(htmlmin({collapseWhitespace: true})) .pipe(htmlmin({collapseWhitespace: true}))
.pipe(minifyInline()) .pipe(minifyInline())

View File

@ -139,9 +139,8 @@
-moz-transform: rotate(-45deg); -moz-transform: rotate(-45deg);
-ms-transform: rotate(-45deg); -ms-transform: rotate(-45deg);
-o-transform: rotate(-45deg); -o-transform: rotate(-45deg);
background: #fc0; background: #fd0;
border: 2px solid #000; border: 2px solid #000;
mix-blend-mode: hard-light;
} }
#github_link { #github_link {
@ -770,12 +769,10 @@
<pre> <pre>
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo mkdir -p /etc/apt/sources.list.d sudo apt-add-repository "deb http://repo.yandex.ru/clickhouse/<span id="distributive">trusty</span> stable main"
echo "deb http://repo.yandex.ru/clickhouse/<span id="distributive">trusty</span> stable main" |
&nbsp;&nbsp;&nbsp;&nbsp;sudo tee /etc/apt/sources.list.d/clickhouse.list
sudo apt-get update sudo apt-get update
sudo apt-get install clickhouse-server-common clickhouse-client sudo apt-get install clickhouse-server-common clickhouse-client -y
sudo service clickhouse-server start sudo service clickhouse-server start
clickhouse-client clickhouse-client
@ -847,6 +844,7 @@ clickhouse-client
$('html, body').animate({ $('html, body').animate({
scrollTop: offset scrollTop: offset
}, 500); }, 500);
window.history.replaceState('', document.title, window.location.href.replace(location.hash, '') + this.hash);
}); });
var available_distributives = ['xenial', 'trusty', 'precise']; var available_distributives = ['xenial', 'trusty', 'precise'];
@ -862,19 +860,31 @@ clickhouse-client
</script> </script>
<!-- Yandex.Metrika counter --> <!-- Yandex.Metrika counter -->
<script src="https://mc.yandex.ru/metrika/watch.js" type="text/javascript"></script>
<script type="text/javascript"> <script type="text/javascript">
try { (function (d, w, c) {
var yaCounter18343495 = new Ya.Metrika({ (w[c] = w[c] || []).push(function() {
id: 18343495, try {
webvisor: true, w.yaCounter18343495 = new Ya.Metrika2({
clickmap: true, id:18343495,
trackLinks: true, clickmap:true,
accurateTrackBounce: true, trackLinks:true,
trackHash: true accurateTrackBounce:true,
webvisor:true
});
} catch(e) { }
}); });
} catch (e) {
} var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script> </script>
<noscript> <noscript>
<div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=""/></div> <div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=""/></div>

Binary file not shown.

Before

Width:  |  Height:  |  Size: 1.0 KiB

After

Width:  |  Height:  |  Size: 377 B

View File

@ -7,15 +7,11 @@ server {
server_name localhost; server_name localhost;
location / { error_page 404 /404.html;
try_files $uri $uri/ /index.html;
error_page 500 502 503 504 /50x.html;
location = /50x.html {
root /usr/share/nginx/www;
} }
#error_page 404 /404.html;
#error_page 500 502 503 504 /50x.html;
#location = /50x.html {
# root /usr/share/nginx/www;
#}
} }

View File

@ -20,16 +20,31 @@ function getParams() {
</script> </script>
<!-- Yandex.Metrica counter --> <!-- Yandex.Metrica counter -->
<script src="https://mc.yandex.ru/metrika/watch.js" type="text/javascript"></script>
<script type="text/javascript"> <script type="text/javascript">
try { var yaCounter18343495 = new Ya.Metrika({id:18343495, (function (d, w, c) {
webvisor:true, (w[c] = w[c] || []).push(function() {
clickmap:true, try {
trackLinks:true, w.yaCounter18343495 = new Ya.Metrika2({
accurateTrackBounce:true, id:18343495,
trackHash:true, clickmap:true,
params: getParams()}); trackLinks:true,
} catch(e) { } accurateTrackBounce:true,
webvisor:true
});
} catch(e) { }
});
var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script> </script>
<noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=" " /></div></noscript> <noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=" " /></div></noscript>
<!-- /Yandex.Metrica counter --> <!-- /Yandex.Metrica counter -->

View File

@ -20,16 +20,31 @@ function getParams() {
</script> </script>
<!-- Yandex.Metrika counter --> <!-- Yandex.Metrika counter -->
<script src="https://mc.yandex.ru/metrika/watch.js" type="text/javascript"></script>
<script type="text/javascript"> <script type="text/javascript">
try { var yaCounter18343495 = new Ya.Metrika({id:18343495, (function (d, w, c) {
webvisor:true, (w[c] = w[c] || []).push(function() {
clickmap:true, try {
trackLinks:true, w.yaCounter18343495 = new Ya.Metrika2({
accurateTrackBounce:true, id:18343495,
trackHash:true, clickmap:true,
params: getParams()}); trackLinks:true,
} catch(e) { } accurateTrackBounce:true,
webvisor:true
});
} catch(e) { }
});
var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script> </script>
<noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt="" /></div></noscript> <noscript><div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt="" /></div></noscript>
<!-- /Yandex.Metrika counter --> <!-- /Yandex.Metrika counter -->

View File

@ -805,20 +805,31 @@ ENGINE = ReplicatedMergeTree(
feedback_email.textContent = feedback_address; feedback_email.textContent = feedback_address;
</script> </script>
<!-- Yandex.Metrika counter --> <!-- Yandex.Metrika counter -->
<script src="https://mc.yandex.ru/metrika/watch.js" type="text/javascript"></script>
<script type="text/javascript"> <script type="text/javascript">
try { (function (d, w, c) {
var yaCounter18343495 = new Ya.Metrika({ (w[c] = w[c] || []).push(function() {
id: 18343495, try {
webvisor: true, w.yaCounter18343495 = new Ya.Metrika2({
clickmap: true, id:18343495,
trackLinks: true, clickmap:true,
accurateTrackBounce: true, trackLinks:true,
trackHash: true, accurateTrackBounce:true,
params: getParams() webvisor:true
});
} catch(e) { }
}); });
} catch (e) {
} var n = d.getElementsByTagName("script")[0],
s = d.createElement("script"),
f = function () { n.parentNode.insertBefore(s, n); };
s.type = "text/javascript";
s.async = true;
s.src = "https://mc.yandex.ru/metrika/tag.js";
if (w.opera == "[object Opera]") {
d.addEventListener("DOMContentLoaded", f, false);
} else { f(); }
})(document, window, "yandex_metrika_callbacks2");
</script> </script>
<noscript> <noscript>
<div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=""/></div> <div><img src="https://mc.yandex.ru/watch/18343495" style="position:absolute; left:-9999px;" alt=""/></div>