mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse into nvartolomei-nv/getStructureOfRemoteTable-try-all-shards
This commit is contained in:
commit
2344d4715e
@ -8,14 +8,15 @@ ClickHouse is an open-source column-oriented database management system that all
|
||||
* [Tutorial](https://clickhouse.tech/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster.
|
||||
* [Documentation](https://clickhouse.tech/docs/en/) provides more in-depth information.
|
||||
* [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format.
|
||||
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/enQtOTUzMjM4ODQwNTc5LWJmMjE3Yjc2YmI1ZDBlZmI4ZTc3OWY3ZTIwYTljYzY4MzBlODM3YzBjZTc1YmYyODRlZTJkYTgzYzBiNTA2Yjk) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time.
|
||||
* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-d2zxkf9e-XyxDa_ucfPxzuH4SJIm~Ng) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time.
|
||||
* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events.
|
||||
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
|
||||
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
|
||||
|
||||
## Upcoming Events
|
||||
|
||||
* [ClickHouse Online Meetup (in Russian)](https://events.yandex.ru/events/click-house-onlajn-vs-03-04-2020) on April 3, 2020.
|
||||
* [ClickHouse in Avito (online in Russian)](https://avitotech.timepad.ru/event/1290051/) on April 9, 2020.
|
||||
* [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on TBD date.
|
||||
* [Talks on Saint HighLoad++ in St. Petersburg](https://www.highload.ru/spb/2020/abstracts/6647) on TBD date.
|
||||
* [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on TBD date.
|
||||
* [ClickHouse in Avito (online in Russian)](https://avitotech.timepad.ru/event/1290051/) on April 9, 2020.
|
||||
|
@ -51,6 +51,10 @@ public:
|
||||
if (!has_nullable_types)
|
||||
throw Exception("Aggregate function combinator 'Null' requires at least one argument to be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (nested_function)
|
||||
if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params))
|
||||
return adapter;
|
||||
|
||||
/// Special case for 'count' function. It could be called with Nullable arguments
|
||||
/// - that means - count number of calls, when all arguments are not NULL.
|
||||
if (nested_function && nested_function->getName() == "count")
|
||||
@ -71,9 +75,9 @@ public:
|
||||
else
|
||||
{
|
||||
if (return_type_is_nullable)
|
||||
return std::make_shared<AggregateFunctionNullVariadic<true>>(nested_function, arguments, params);
|
||||
return std::make_shared<AggregateFunctionNullVariadic<true, true>>(nested_function, arguments, params);
|
||||
else
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false>>(nested_function, arguments, params);
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, true>>(nested_function, arguments, params);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -204,12 +204,12 @@ public:
|
||||
};
|
||||
|
||||
|
||||
template <bool result_is_nullable>
|
||||
class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable>>
|
||||
template <bool result_is_nullable, bool null_is_skipped>
|
||||
class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable, null_is_skipped>>
|
||||
{
|
||||
public:
|
||||
AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params)
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable>>(std::move(nested_function_), arguments, params),
|
||||
: AggregateFunctionNullBase<result_is_nullable, AggregateFunctionNullVariadic<result_is_nullable, null_is_skipped>>(std::move(nested_function_), arguments, params),
|
||||
number_of_arguments(arguments.size())
|
||||
{
|
||||
if (number_of_arguments == 1)
|
||||
@ -233,7 +233,7 @@ public:
|
||||
if (is_nullable[i])
|
||||
{
|
||||
const ColumnNullable & nullable_col = assert_cast<const ColumnNullable &>(*columns[i]);
|
||||
if (nullable_col.isNullAt(row_num))
|
||||
if (null_is_skipped && nullable_col.isNullAt(row_num))
|
||||
{
|
||||
/// If at least one column has a null value in the current row,
|
||||
/// we don't process this row.
|
||||
|
@ -11,7 +11,7 @@
|
||||
#include <Common/ArenaAllocator.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/AggregateFunctionNull.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -186,6 +186,14 @@ private:
|
||||
{
|
||||
return event_idx + 1;
|
||||
}
|
||||
else if (strict_order && first_event && events_timestamp[event_idx - 1] == -1)
|
||||
{
|
||||
for (size_t event = 0; event < events_timestamp.size(); ++event)
|
||||
{
|
||||
if (events_timestamp[event] == -1)
|
||||
return event;
|
||||
}
|
||||
}
|
||||
else if (events_timestamp[event_idx - 1] >= 0 && timestamp <= events_timestamp[event_idx - 1] + window)
|
||||
{
|
||||
events_timestamp[event_idx] = events_timestamp[event_idx - 1];
|
||||
@ -232,6 +240,11 @@ public:
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
AggregateFunctionPtr getOwnNullAdapter(const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override
|
||||
{
|
||||
return std::make_shared<AggregateFunctionNullVariadic<false, false>>(nested_function, arguments, params);
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override
|
||||
{
|
||||
bool has_event = false;
|
||||
|
@ -31,6 +31,8 @@ using DataTypes = std::vector<DataTypePtr>;
|
||||
using AggregateDataPtr = char *;
|
||||
using ConstAggregateDataPtr = const char *;
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
|
||||
/** Aggregate functions interface.
|
||||
* Instances of classes with this interface do not contain the data itself for aggregation,
|
||||
@ -149,6 +151,17 @@ public:
|
||||
virtual void addBatchArray(
|
||||
size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) const = 0;
|
||||
|
||||
/** By default all NULLs are skipped during aggregation.
|
||||
* If it returns nullptr, the default one will be used.
|
||||
* If an aggregate function wants to use something instead of the default one, it overrides this function and returns its own null adapter.
|
||||
* nested_function is a smart pointer to this aggregate function itself.
|
||||
* arguments and params are for nested_function.
|
||||
*/
|
||||
virtual AggregateFunctionPtr getOwnNullAdapter(const AggregateFunctionPtr & /*nested_function*/, const DataTypes & /*arguments*/, const Array & /*params*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
const DataTypes & getArgumentTypes() const { return argument_types; }
|
||||
const Array & getParameters() const { return parameters; }
|
||||
|
||||
@ -244,6 +257,4 @@ public:
|
||||
};
|
||||
|
||||
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
|
||||
}
|
||||
|
@ -40,7 +40,7 @@ namespace
|
||||
if (overrun_count)
|
||||
{
|
||||
/// But pass with some frequency to avoid drop of all traces.
|
||||
if (write_trace_iteration % overrun_count == 0)
|
||||
if (write_trace_iteration % (overrun_count + 1) == 0)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::QueryProfilerSignalOverruns, overrun_count);
|
||||
}
|
||||
|
@ -421,6 +421,18 @@ void DDLWorker::processTasks()
|
||||
{
|
||||
processTask(task, zookeeper);
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
if (server_startup && e.code == Coordination::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(log, "ZooKeeper NONODE error during startup. Ignoring entry " <<
|
||||
task.entry_name << " (" << task.entry.query << ") : " << getCurrentExceptionMessage(true));
|
||||
}
|
||||
else
|
||||
{
|
||||
throw;
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_WARNING(log, "An error occurred while processing task " << task.entry_name << " (" << task.entry.query << ") : "
|
||||
|
@ -53,7 +53,11 @@ bool SubqueryForSet::insertJoinedBlock(Block & block)
|
||||
void SubqueryForSet::setTotals()
|
||||
{
|
||||
if (join && source)
|
||||
join->setTotals(source->getTotals());
|
||||
{
|
||||
Block totals = source->getTotals();
|
||||
renameColumns(totals);
|
||||
join->setTotals(totals);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -42,6 +42,7 @@ struct SubqueryForSet
|
||||
private:
|
||||
NamesWithAliases joined_block_aliases; /// Rename column from joined block from this list.
|
||||
|
||||
/// Rename source right table column names into qualified column names if they conflicts with left table ones.
|
||||
void renameColumns(Block & block);
|
||||
};
|
||||
|
||||
|
@ -26,7 +26,7 @@ void ExpressionTransform::transform(Chunk & chunk)
|
||||
{
|
||||
initialized = true;
|
||||
|
||||
if (expression->resultIsAlwaysEmpty())
|
||||
if (expression->resultIsAlwaysEmpty() && !on_totals)
|
||||
{
|
||||
stopReading();
|
||||
chunk.clear();
|
||||
|
@ -25,7 +25,7 @@ void InflatingExpressionTransform::transform(Chunk & chunk)
|
||||
{
|
||||
initialized = true;
|
||||
|
||||
if (expression->resultIsAlwaysEmpty())
|
||||
if (expression->resultIsAlwaysEmpty() && !on_totals)
|
||||
{
|
||||
stopReading();
|
||||
chunk.clear();
|
||||
|
@ -71,7 +71,7 @@ Pipes StorageMySQL::read(
|
||||
{
|
||||
check(column_names_);
|
||||
String query = transformQueryForExternalDatabase(
|
||||
*query_info_.query, getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, remote_database_name, remote_table_name, context_);
|
||||
query_info_, getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, remote_database_name, remote_table_name, context_);
|
||||
|
||||
Block sample_block;
|
||||
for (const String & column_name : column_names_)
|
||||
|
@ -3336,7 +3336,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
}
|
||||
else if (rc == Coordination::ZBADVERSION)
|
||||
{
|
||||
if (dynamic_cast<const Coordination::SetResponse &>(*results[0]).error)
|
||||
if (results[0]->error)
|
||||
throw Exception("Metadata on replica is not up to date with common metadata in Zookeeper. Cannot alter", ErrorCodes::CANNOT_ASSIGN_ALTER);
|
||||
|
||||
continue;
|
||||
|
@ -74,7 +74,7 @@ std::function<void(std::ostream &)> StorageXDBC::getReadPOSTDataCallback(const N
|
||||
QueryProcessingStage::Enum & /*processed_stage*/,
|
||||
size_t /*max_block_size*/) const
|
||||
{
|
||||
String query = transformQueryForExternalDatabase(*query_info.query,
|
||||
String query = transformQueryForExternalDatabase(query_info,
|
||||
getColumns().getOrdinary(),
|
||||
bridge_helper->getIdentifierQuotingStyle(),
|
||||
remote_database_name,
|
||||
|
@ -49,7 +49,10 @@ static void check(const std::string & query, const std::string & expected, const
|
||||
{
|
||||
ParserSelectQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, query, 1000);
|
||||
std::string transformed_query = transformQueryForExternalDatabase(*ast, columns, IdentifierQuotingStyle::DoubleQuotes, "test", "table", context);
|
||||
SelectQueryInfo query_info;
|
||||
query_info.syntax_analyzer_result = SyntaxAnalyzer(context).analyzeSelect(ast, columns);
|
||||
query_info.query = ast;
|
||||
std::string transformed_query = transformQueryForExternalDatabase(query_info, columns, IdentifierQuotingStyle::DoubleQuotes, "test", "table", context);
|
||||
|
||||
EXPECT_EQ(transformed_query, expected);
|
||||
}
|
||||
|
@ -134,16 +134,15 @@ bool isCompatible(const IAST & node)
|
||||
|
||||
|
||||
String transformQueryForExternalDatabase(
|
||||
const IAST & query,
|
||||
const SelectQueryInfo & query_info,
|
||||
const NamesAndTypesList & available_columns,
|
||||
IdentifierQuotingStyle identifier_quoting_style,
|
||||
const String & database,
|
||||
const String & table,
|
||||
const Context & context)
|
||||
{
|
||||
auto clone_query = query.clone();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyzeSelect(clone_query, available_columns);
|
||||
const Names used_columns = syntax_result->requiredSourceColumns();
|
||||
auto clone_query = query_info.query->clone();
|
||||
const Names used_columns = query_info.syntax_analyzer_result->requiredSourceColumns();
|
||||
|
||||
auto select = std::make_shared<ASTSelectQuery>();
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Parsers/IdentifierQuotingStyle.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -23,7 +24,7 @@ class Context;
|
||||
* Compatible expressions are comparisons of identifiers, constants, and logical operations on them.
|
||||
*/
|
||||
String transformQueryForExternalDatabase(
|
||||
const IAST & query,
|
||||
const SelectQueryInfo & query_info,
|
||||
const NamesAndTypesList & available_columns,
|
||||
IdentifierQuotingStyle identifier_quoting_style,
|
||||
const String & database,
|
||||
|
@ -99,3 +99,24 @@ def test_clickhouse_dml_for_mysql_database(started_cluster):
|
||||
assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '10000'
|
||||
|
||||
mysql_node.query("DROP DATABASE test_database")
|
||||
|
||||
|
||||
def test_clickhouse_join_for_mysql_database(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node:
|
||||
mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'")
|
||||
mysql_node.query("CREATE TABLE test.t1_mysql_local ("
|
||||
"pays VARCHAR(55) DEFAULT 'FRA' NOT NULL,"
|
||||
"service VARCHAR(5) DEFAULT '' NOT NULL,"
|
||||
"opco CHAR(3) DEFAULT '' NOT NULL"
|
||||
")")
|
||||
mysql_node.query("CREATE TABLE test.t2_mysql_local ("
|
||||
"service VARCHAR(5) DEFAULT '' NOT NULL,"
|
||||
"opco VARCHAR(5) DEFAULT ''"
|
||||
")")
|
||||
clickhouse_node.query("CREATE TABLE default.t1_remote_mysql AS mysql('mysql1:3306','test','t1_mysql_local','root','clickhouse')")
|
||||
clickhouse_node.query("CREATE TABLE default.t2_remote_mysql AS mysql('mysql1:3306','test','t2_mysql_local','root','clickhouse')")
|
||||
assert clickhouse_node.query("SELECT s.pays "
|
||||
"FROM default.t1_remote_mysql AS s "
|
||||
"LEFT JOIN default.t1_remote_mysql AS s_ref "
|
||||
"ON (s_ref.opco = s.opco AND s_ref.service = s.service)") == ''
|
||||
mysql_node.query("DROP DATABASE test")
|
||||
|
@ -36,3 +36,24 @@
|
||||
[4, 2]
|
||||
[5, 2]
|
||||
[6, 1]
|
||||
[7, 1]
|
||||
[1, 2]
|
||||
[2, 2]
|
||||
[3, 0]
|
||||
[4, 0]
|
||||
[1, 2]
|
||||
[2, 1]
|
||||
[3, 0]
|
||||
[4, 0]
|
||||
[1, 0]
|
||||
[2, 0]
|
||||
[3, 1]
|
||||
[4, 0]
|
||||
[1, 0]
|
||||
[2, 0]
|
||||
[3, 1]
|
||||
[4, 2]
|
||||
[1, 0]
|
||||
[2, 0]
|
||||
[3, 1]
|
||||
[4, 1]
|
||||
|
@ -63,4 +63,19 @@ insert into funnel_test_strict_order values (1, 6, 'c') (2, 6, 'c') (3, 6, 'b')
|
||||
select user, windowFunnel(86400)(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
select user, windowFunnel(86400, 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
select user, windowFunnel(86400, 'strict', 'strict_order')(dt, event='a', event='b', event='c') as s from funnel_test_strict_order group by user order by user format JSONCompactEachRow;
|
||||
insert into funnel_test_strict_order values (1, 7, 'a') (2, 7, 'c') (3, 7, 'b');
|
||||
select user, windowFunnel(10, 'strict_order')(dt, event = 'a', event = 'b', event = 'c') as s from funnel_test_strict_order where user = 7 group by user format JSONCompactEachRow;
|
||||
drop table funnel_test_strict_order;
|
||||
|
||||
drop table if exists funnel_test_non_null;
|
||||
create table funnel_test_non_null (`dt` DateTime, `u` int, `a` Nullable(String), `b` Nullable(String)) engine = MergeTree() partition by dt order by u;
|
||||
insert into funnel_test_non_null values (1, 1, 'a1', 'b1') (2, 1, 'a2', 'b2');
|
||||
insert into funnel_test_non_null values (1, 2, 'a1', null) (2, 2, 'a2', null);
|
||||
insert into funnel_test_non_null values (1, 3, null, null);
|
||||
insert into funnel_test_non_null values (1, 4, null, 'b1') (2, 4, 'a2', null) (3, 4, null, 'b3');
|
||||
select u, windowFunnel(86400)(dt, a = 'a1', a = 'a2') as s from funnel_test_non_null group by u order by u format JSONCompactEachRow;
|
||||
select u, windowFunnel(86400)(dt, a = 'a1', b = 'b2') as s from funnel_test_non_null group by u order by u format JSONCompactEachRow;
|
||||
select u, windowFunnel(86400)(dt, a is null and b is null) as s from funnel_test_non_null group by u order by u format JSONCompactEachRow;
|
||||
select u, windowFunnel(86400)(dt, a is null, b = 'b3') as s from funnel_test_non_null group by u order by u format JSONCompactEachRow;
|
||||
select u, windowFunnel(86400, 'strict_order')(dt, a is null, b = 'b3') as s from funnel_test_non_null group by u order by u format JSONCompactEachRow;
|
||||
drop table funnel_test_non_null;
|
||||
|
@ -46,15 +46,12 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;"
|
||||
sleep 1
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;"
|
||||
wait_for_mutation "indices_mutaions1" "0000000000" "test"
|
||||
wait_for_mutation "indices_mutaions2" "0000000000" "test"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
|
@ -97,7 +97,7 @@ done
|
||||
echo "Equal number of columns"
|
||||
|
||||
# This alter will finish all previous, but replica 1 maybe still not up-to-date
|
||||
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_1 MODIFY COLUMN value0 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
while [[ $(timeout 30 $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_1 MODIFY COLUMN value0 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
sleep 1
|
||||
done
|
||||
|
||||
|
@ -96,7 +96,7 @@ done
|
||||
|
||||
|
||||
# This alter will finish all previous, but replica 1 maybe still not up-to-date
|
||||
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
while [[ $(timeout 30 $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
sleep 1
|
||||
done
|
||||
|
||||
|
@ -101,7 +101,7 @@ wait
|
||||
echo "Finishing alters"
|
||||
|
||||
# This alter will finish all previous, but replica 1 maybe still not up-to-date
|
||||
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
while [[ $(timeout 30 $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
|
||||
sleep 1
|
||||
done
|
||||
|
||||
|
@ -0,0 +1,10 @@
|
||||
|
||||
0
|
||||
|
||||
0
|
||||
|
||||
0
|
||||
|
||||
0
|
||||
|
||||
0 0 0
|
@ -0,0 +1,38 @@
|
||||
DROP TABLE IF EXISTS t;
|
||||
CREATE TABLE t (item_id UInt64, price_sold Float32, date Date) ENGINE MergeTree ORDER BY item_id;
|
||||
|
||||
SELECT item_id
|
||||
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) l
|
||||
FULL JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) r
|
||||
USING (item_id);
|
||||
|
||||
SELECT id
|
||||
FROM (SELECT item_id AS id FROM t GROUP BY id WITH TOTALS) l
|
||||
FULL JOIN (SELECT item_id AS id FROM t GROUP BY id WITH TOTALS) r
|
||||
USING (id);
|
||||
|
||||
SELECT item_id
|
||||
FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) l
|
||||
INNER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) r
|
||||
USING (item_id);
|
||||
|
||||
SELECT id
|
||||
FROM (SELECT item_id AS id FROM t GROUP BY id WITH TOTALS) l
|
||||
INNER JOIN (SELECT item_id AS id FROM t GROUP BY id WITH TOTALS) r
|
||||
USING (id);
|
||||
|
||||
SELECT id, yago, recent
|
||||
FROM (
|
||||
SELECT item_id AS id, SUM(price_sold) AS recent
|
||||
FROM t WHERE (date BETWEEN '2019-12-16' AND '2020-03-08')
|
||||
GROUP BY id WITH TOTALS
|
||||
) ll
|
||||
FULL JOIN
|
||||
(
|
||||
SELECT item_id AS id, SUM(price_sold) AS yago
|
||||
FROM t WHERE (date BETWEEN '2018-12-17' AND '2019-03-10')
|
||||
GROUP BY id WITH TOTALS
|
||||
) rr
|
||||
USING (id);
|
||||
|
||||
DROP TABLE t;
|
@ -100,6 +100,30 @@ Example: `regionToCountry(toUInt32(213)) = 225` converts Moscow (213) to Russia
|
||||
Converts a region to a continent. In every other way, this function is the same as ‘regionToCity’.
|
||||
Example: `regionToContinent(toUInt32(213)) = 10001` converts Moscow (213) to Eurasia (10001).
|
||||
|
||||
### regionToTopContinent (#regiontotopcontinent)
|
||||
|
||||
Finds the highest continent in the hierarchy for the region.
|
||||
|
||||
**Syntax**
|
||||
|
||||
```sql
|
||||
regionToTopContinent(id[, geobase]);
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
- `id` — Region ID from the Yandex geobase. [UInt32](../../data_types/int_uint.md).
|
||||
- `geobase` — Dictionary key. See [Multiple Geobases](#multiple-geobases). [String](../../data_types/string.md). Optional.
|
||||
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Identifier of the top level continent (the latter when you climb the hierarchy of regions).
|
||||
- 0, if there is none.
|
||||
|
||||
Type: `UInt32`.
|
||||
|
||||
|
||||
### regionToPopulation(id\[, geobase\]) {#regiontopopulationid-geobase}
|
||||
|
||||
Gets the population for a region.
|
||||
|
@ -101,6 +101,28 @@ LIMIT 15
|
||||
Переводит регион в континент. В остальном, аналогично функции regionToCity.
|
||||
Пример: `regionToContinent(toUInt32(213)) = 10001` - преобразовали Москву (213) в Евразию (10001).
|
||||
|
||||
### regionToTopContinent (#regiontotopcontinent)
|
||||
|
||||
Находит для региона верхний в иерархии континент.
|
||||
|
||||
**Синтаксис**
|
||||
|
||||
```sql
|
||||
regionToTopContinent(id[, geobase]);
|
||||
```
|
||||
|
||||
**Параметры**
|
||||
|
||||
- `id` — Идентификатор региона из геобазы Яндекса. [UInt32](../../data_types/int_uint.md).
|
||||
- `geobase` — Ключ словаря. Смотрите [Множественные геобазы](#multiple-geobases). [String](../../data_types/string.md). Опциональный параметр.
|
||||
|
||||
**Возвращаемое значение**
|
||||
|
||||
- Идентификатор континента верхнего уровня (последний при подъеме по иерархии регионов).
|
||||
- 0, если его нет.
|
||||
|
||||
Тип: `UInt32`.
|
||||
|
||||
### regionToPopulation(id\[, geobase\]) {#regiontopopulationid-geobase}
|
||||
|
||||
Получает население для региона.
|
||||
|
@ -65,4 +65,5 @@ unixODBC позволял указать путь для подключения
|
||||
Некорректная конфигурация в deb пакете могла привести к неавторизованному доступу к базе данных.
|
||||
|
||||
Обнаружено благодаря: the UK’s National Cyber Security Centre (NCSC)
|
||||
[Оригинальная статья](https://clickhouse.tech/docs/ru/security_changelog/) <!--hide-->
|
||||
|
||||
{## [Оригинальная статья](https://clickhouse.tech/docs/ru/security_changelog/) ##}
|
||||
|
@ -38,5 +38,5 @@ The easiest way to see the result is to use `--livereload=8888` argument of buil
|
||||
|
||||
At the moment there’s no easy way to do just that, but you can consider:
|
||||
|
||||
- To hit the “Watch” button on top of GitHub web interface to know as early as possible, even during pull request. Alternative to this is `#github-activity` channel of [public ClickHouse Slack](https://join.slack.com/t/clickhousedb/shared_invite/enQtOTUzMjM4ODQwNTc5LWJmMjE3Yjc2YmI1ZDBlZmI4ZTc3OWY3ZTIwYTljYzY4MzBlODM3YzBjZTc1YmYyODRlZTJkYTgzYzBiNTA2Yjk).
|
||||
- To hit the “Watch” button on top of GitHub web interface to know as early as possible, even during pull request. Alternative to this is `#github-activity` channel of [public ClickHouse Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-d2zxkf9e-XyxDa_ucfPxzuH4SJIm~Ng).
|
||||
- Some search engines allow to subscribe on specific website changes via email and you can opt-in for that for https://clickhouse.tech.
|
||||
|
@ -1,17 +1,15 @@
|
||||
#!/usr/bin/env python3
|
||||
|
||||
from __future__ import unicode_literals
|
||||
|
||||
import argparse
|
||||
import datetime
|
||||
import http.server
|
||||
import logging
|
||||
import multiprocessing
|
||||
import os
|
||||
import shutil
|
||||
import socketserver
|
||||
import subprocess
|
||||
import sys
|
||||
import threading
|
||||
import time
|
||||
|
||||
import jinja2
|
||||
@ -218,54 +216,64 @@ def build_single_page_version(lang, args, cfg):
|
||||
os.path.join(site_temp, 'single'),
|
||||
single_page_output_path
|
||||
)
|
||||
|
||||
logging.info(f'Re-building single page for {lang} pdf/test')
|
||||
with util.temp_dir() as test_dir:
|
||||
single_page_pdf = os.path.abspath(
|
||||
os.path.join(single_page_output_path, f'clickhouse_{lang}.pdf')
|
||||
)
|
||||
extra['single_page'] = False
|
||||
cfg.load_dict({
|
||||
'docs_dir': docs_temp_lang,
|
||||
'site_dir': test_dir,
|
||||
'extra': extra,
|
||||
'nav': [
|
||||
{cfg.data.get('site_name'): 'single.md'}
|
||||
]
|
||||
})
|
||||
mkdocs_build.build(cfg)
|
||||
|
||||
if not args.skip_pdf:
|
||||
with util.temp_dir() as test_dir:
|
||||
single_page_pdf = os.path.abspath(
|
||||
os.path.join(single_page_output_path, f'clickhouse_{lang}.pdf')
|
||||
)
|
||||
extra['single_page'] = False
|
||||
cfg.load_dict({
|
||||
'docs_dir': docs_temp_lang,
|
||||
'site_dir': test_dir,
|
||||
'extra': extra,
|
||||
'nav': [
|
||||
{cfg.data.get('site_name'): 'single.md'}
|
||||
]
|
||||
})
|
||||
mkdocs_build.build(cfg)
|
||||
|
||||
css_in = ' '.join(website.get_css_in(args))
|
||||
js_in = ' '.join(website.get_js_in(args))
|
||||
subprocess.check_call(f'cat {css_in} > {test_dir}/css/base.css', shell=True)
|
||||
subprocess.check_call(f'cat {js_in} > {test_dir}/js/base.js', shell=True)
|
||||
css_in = ' '.join(website.get_css_in(args))
|
||||
js_in = ' '.join(website.get_js_in(args))
|
||||
subprocess.check_call(f'cat {css_in} > {test_dir}/css/base.css', shell=True)
|
||||
subprocess.check_call(f'cat {js_in} > {test_dir}/js/base.js', shell=True)
|
||||
if not args.skip_pdf:
|
||||
port_for_pdf = util.get_free_port()
|
||||
with socketserver.TCPServer(
|
||||
('', port_for_pdf), http.server.SimpleHTTPRequestHandler
|
||||
) as httpd:
|
||||
logging.info(f"serving for pdf at port {port_for_pdf}")
|
||||
thread = threading.Thread(target=httpd.serve_forever)
|
||||
with util.cd(test_dir):
|
||||
thread.start()
|
||||
create_pdf_command = [
|
||||
'wkhtmltopdf',
|
||||
'--print-media-type',
|
||||
'--no-stop-slow-scripts',
|
||||
'--log-level', 'warn',
|
||||
f'http://localhost:{port_for_pdf}/single/', single_page_pdf
|
||||
]
|
||||
try:
|
||||
if args.save_raw_single_page:
|
||||
shutil.copytree(test_dir, args.save_raw_single_page)
|
||||
logging.info(' '.join(create_pdf_command))
|
||||
subprocess.check_call(' '.join(create_pdf_command), shell=True)
|
||||
finally:
|
||||
httpd.shutdown()
|
||||
thread.join(timeout=5.0)
|
||||
httpd = socketserver.TCPServer(
|
||||
('', port_for_pdf), http.server.SimpleHTTPRequestHandler
|
||||
)
|
||||
logging.info(f"Serving for {lang} pdf at port {port_for_pdf}")
|
||||
process = multiprocessing.Process(target=httpd.serve_forever)
|
||||
with util.cd(test_dir):
|
||||
process.start()
|
||||
create_pdf_command = [
|
||||
'wkhtmltopdf',
|
||||
'--print-media-type',
|
||||
'--disable-javascript',
|
||||
# TODO: restore '--log-level', 'warn',
|
||||
f'http://localhost:{port_for_pdf}/single/', single_page_pdf
|
||||
]
|
||||
try:
|
||||
if args.save_raw_single_page:
|
||||
shutil.copytree(test_dir, args.save_raw_single_page)
|
||||
logging.info(' '.join(create_pdf_command))
|
||||
subprocess.check_call(' '.join(create_pdf_command), shell=True)
|
||||
finally:
|
||||
logging.info(f'Stop serving for {lang} pdf at port {port_for_pdf}')
|
||||
process.kill()
|
||||
while True:
|
||||
time.sleep(0.25)
|
||||
try:
|
||||
process.close()
|
||||
break
|
||||
except ValueError:
|
||||
logging.info(f'Waiting for {lang} httpd at port {port_for_pdf} to stop')
|
||||
|
||||
if not args.version_prefix: # maybe enable in future
|
||||
test.test_single_page(
|
||||
os.path.join(test_dir, 'single', 'index.html'), lang)
|
||||
|
||||
if not args.version_prefix: # maybe enable in future
|
||||
logging.info(f'Running tests for {lang}')
|
||||
test.test_single_page(
|
||||
os.path.join(test_dir, 'single', 'index.html'), lang)
|
||||
|
||||
logging.info(f'Finished building single page version for {lang}')
|
||||
|
||||
|
@ -14,9 +14,8 @@ def concatenate(lang, docs_path, single_page_file):
|
||||
files_to_concatenate = [(l[l.index(':') + 1:]).strip(" '\n") for l in cfg_file
|
||||
if '.md' in l and 'single_page' not in l]
|
||||
|
||||
logging.info(
|
||||
str(len(files_to_concatenate)) +
|
||||
' files will be concatenated into single md-file.')
|
||||
files_count = len(files_to_concatenate)
|
||||
logging.info(f'{files_count} files will be concatenated into single md-file for {lang}.')
|
||||
logging.debug('Concatenating: ' + ', '.join(files_to_concatenate))
|
||||
|
||||
for path in files_to_concatenate:
|
||||
|
@ -11,6 +11,7 @@ import macros.plugin
|
||||
|
||||
import slugify as slugify_impl
|
||||
|
||||
|
||||
class ClickHouseLinkMixin(object):
|
||||
|
||||
def handleMatch(self, m, data):
|
||||
|
@ -44,16 +44,19 @@ def get_free_port():
|
||||
|
||||
|
||||
def run_function_in_parallel(func, args_list, threads=False):
|
||||
processes = []
|
||||
exit_code = 0
|
||||
for task in args_list:
|
||||
cls = threading.Thread if threads else multiprocessing.Process
|
||||
processes.append(cls(target=func, args=task))
|
||||
processes[-1].start()
|
||||
for process in processes:
|
||||
process.join()
|
||||
if not threads:
|
||||
if process.exitcode and not exit_code:
|
||||
exit_code = process.exitcode
|
||||
if exit_code:
|
||||
sys.exit(exit_code)
|
||||
func(*task)
|
||||
# TODO: back to parallel
|
||||
# processes = []
|
||||
# exit_code = 0
|
||||
# for task in args_list:
|
||||
# cls = threading.Thread if threads else multiprocessing.Process
|
||||
# processes.append(cls(target=func, args=task))
|
||||
# processes[-1].start()
|
||||
# for process in processes:
|
||||
# process.join()
|
||||
# if not threads:
|
||||
# if process.exitcode and not exit_code:
|
||||
# exit_code = process.exitcode
|
||||
# if exit_code:
|
||||
# sys.exit(exit_code)
|
||||
|
@ -125,8 +125,7 @@ def minify_website(args):
|
||||
'--compilation_level', 'SIMPLE',
|
||||
'--dependency_mode', 'NONE',
|
||||
'--third_party', '--use_types_for_optimization',
|
||||
'--isolation_mode', 'IIFE',
|
||||
'--create_source_map', '%outname%.map'
|
||||
'--isolation_mode', 'IIFE'
|
||||
]
|
||||
logging.info(closure_args)
|
||||
if closure.run(*closure_args):
|
||||
|
15
website/css/bootstrap.css
vendored
15
website/css/bootstrap.css
vendored
@ -7978,15 +7978,14 @@ button.bg-dark:focus {
|
||||
z-index: 1030;
|
||||
}
|
||||
|
||||
@supports ((position: -webkit-sticky) or (position: sticky)) {
|
||||
.sticky-top {
|
||||
position: -webkit-sticky;
|
||||
position: sticky;
|
||||
top: 0;
|
||||
z-index: 1020;
|
||||
}
|
||||
.sticky-top {
|
||||
position: -webkit-sticky;
|
||||
position: sticky;
|
||||
top: 0;
|
||||
z-index: 1020;
|
||||
}
|
||||
|
||||
|
||||
.sr-only {
|
||||
position: absolute;
|
||||
width: 1px;
|
||||
@ -10221,4 +10220,4 @@ a.text-dark:hover, a.text-dark:focus {
|
||||
border-color: #dee2e6;
|
||||
}
|
||||
}
|
||||
/*# sourceMappingURL=bootstrap.css.map */
|
||||
/*# sourceMappingURL=bootstrap.css.map */
|
||||
|
Loading…
Reference in New Issue
Block a user