mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 02:52:13 +00:00
Merge branch 'master' of https://github.com/yandex/ClickHouse
This commit is contained in:
commit
8e868dd2fe
@ -12,7 +12,6 @@ ClickHouse is an open-source column-oriented database management system that all
|
||||
* 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 Community Meetup in Beijing](https://www.huodongxing.com/event/2483759276200) on June 8.
|
||||
* [ClickHouse on HighLoad++ Siberia](https://www.highload.ru/siberia/2019/abstracts/5348) on June 24-25.
|
||||
* [ClickHouse Community Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20.
|
||||
* [ClickHouse Community Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27.
|
||||
|
@ -117,11 +117,11 @@ namespace
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined);
|
||||
}
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ namespace detail
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace detail
|
||||
}
|
||||
|
||||
|
||||
template <typename Key, UInt8 K>
|
||||
@ -231,4 +231,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -60,4 +60,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -21,4 +21,4 @@ private:
|
||||
Block readImpl() override { return {}; }
|
||||
};
|
||||
|
||||
} /// namespace DB
|
||||
}
|
||||
|
@ -85,4 +85,4 @@ public:
|
||||
String getName() const override { return name; }
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -98,7 +98,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||
{
|
||||
@ -115,4 +115,4 @@ void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||
});
|
||||
}
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -23,7 +23,7 @@ static void deserializeFromString(const DataTypeCustomSimpleTextSerialization &
|
||||
domain.deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -85,4 +85,4 @@ void DataTypeCustomSimpleTextSerialization::serializeTextXML(const IColumn & col
|
||||
writeXMLString(serializeToString(*this, column, row_num, settings), ostr);
|
||||
}
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -50,4 +50,4 @@ public:
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ namespace
|
||||
return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)};
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
|
||||
AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type)
|
||||
|
@ -39,7 +39,7 @@ const DB::IColumn & unwrapNullableColumn(const DB::IColumn & column)
|
||||
return column;
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -187,7 +187,7 @@ inline Encoded base32Decode(const char * encoded_string, size_t encoded_length)
|
||||
return result;
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -227,6 +227,6 @@ void geohashDecode(const char * encoded_string, size_t encoded_len, Float64 * lo
|
||||
*latitude = decodeCoordinate(lat_encoded, -90, 90, singleCoordBitsPrecision(precision, LATITUDE));
|
||||
}
|
||||
|
||||
} // namespace GeoUtils
|
||||
}
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -234,8 +234,8 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
{
|
||||
throw Exception(
|
||||
getName() + ": Passed array number " + std::to_string(array_num) + " depth ("
|
||||
+ std::to_string(arrays_depths.depths[array_num]) + ") is more than the actual array depth (" + std::to_string(col_depth)
|
||||
+ ").",
|
||||
+ std::to_string(arrays_depths.depths[array_num]) + ") is more than the actual array depth ("
|
||||
+ std::to_string(col_depth) + ").",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
@ -255,7 +255,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeImpl(
|
||||
executeMethodImpl(offsets_by_depth, data_columns, arrays_depths, res_values);
|
||||
|
||||
ColumnPtr result_nested_array = std::move(res_nested);
|
||||
for (int depth = arrays_depths.max_array_depth - 1; depth >= 0; --depth)
|
||||
for (ssize_t depth = arrays_depths.max_array_depth - 1; depth >= 0; --depth)
|
||||
result_nested_array = ColumnArray::create(std::move(result_nested_array), offsetsptr_by_depth[depth]);
|
||||
|
||||
block.getByPosition(result).column = result_nested_array;
|
||||
@ -321,6 +321,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
|
||||
std::vector<size_t> indices_by_depth(arrays_depths.max_array_depth);
|
||||
std::vector<size_t> current_offset_n_by_depth(arrays_depths.max_array_depth);
|
||||
std::vector<size_t> last_offset_by_depth(arrays_depths.max_array_depth, 0); // For skipping empty arrays
|
||||
|
||||
UInt32 rank = 0;
|
||||
|
||||
@ -330,6 +331,24 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
{
|
||||
bool want_clear = false;
|
||||
|
||||
/// Skipping offsets if no data in this array
|
||||
if (prev_off == off)
|
||||
{
|
||||
want_clear = true;
|
||||
++indices_by_depth[0];
|
||||
|
||||
for (ssize_t depth = current_offset_depth - 1; depth >= 0; --depth)
|
||||
{
|
||||
const auto offsets_by_depth_size = offsets_by_depth[depth]->size();
|
||||
while (last_offset_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
{
|
||||
if (current_offset_n_by_depth[depth] + 1 >= offsets_by_depth_size)
|
||||
break; // only one empty array: SELECT arrayEnumerateUniqRanked([]);
|
||||
++current_offset_n_by_depth[depth];
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// For each element at the depth we want to look.
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
@ -356,14 +375,21 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
|
||||
// Debug: DUMP(off, prev_off, j, columns_indices, res_values[j], columns);
|
||||
|
||||
for (int depth = current_offset_depth - 1; depth >= 0; --depth)
|
||||
for (ssize_t depth = current_offset_depth - 1; depth >= 0; --depth)
|
||||
{
|
||||
/// Skipping offsets for empty arrays
|
||||
while (last_offset_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
{
|
||||
++current_offset_n_by_depth[depth];
|
||||
}
|
||||
|
||||
++indices_by_depth[depth];
|
||||
|
||||
if (indices_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
{
|
||||
if (static_cast<int>(arrays_depths.clear_depth) == depth + 1)
|
||||
want_clear = true;
|
||||
last_offset_by_depth[depth] = (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]];
|
||||
++current_offset_n_by_depth[depth];
|
||||
}
|
||||
else
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayEnumerateUniqRanked : public FunctionArrayEnumerateRankedExtended<FunctionArrayEnumerateUniqRanked>
|
||||
{
|
||||
using Base = FunctionArrayEnumerateRankedExtended<FunctionArrayEnumerateUniqRanked>;
|
||||
|
@ -49,4 +49,4 @@ private:
|
||||
bool put_delimiter = false;
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -207,6 +207,11 @@ static bool isSupportedAlterType(int type)
|
||||
ASTAlterCommand::DROP_PARTITION,
|
||||
ASTAlterCommand::DELETE,
|
||||
ASTAlterCommand::UPDATE,
|
||||
ASTAlterCommand::COMMENT_COLUMN,
|
||||
ASTAlterCommand::MODIFY_ORDER_BY,
|
||||
ASTAlterCommand::MODIFY_TTL,
|
||||
ASTAlterCommand::ADD_INDEX,
|
||||
ASTAlterCommand::DROP_INDEX,
|
||||
};
|
||||
|
||||
return supported_alter_types.count(type) != 0;
|
||||
|
@ -127,7 +127,7 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
|
||||
return expr_list->children.at(0)->getColumnName();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
const ASTPtr & query_ptr_,
|
||||
|
@ -46,7 +46,7 @@ String qualifiedName(ASTIdentifier * identifier, const String & prefix)
|
||||
return identifier->getAliasOrColumnName();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
|
||||
ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_)
|
||||
|
@ -27,7 +27,7 @@ void callWithType(AsofRowRefs::Type which, F && f)
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
|
||||
AsofRowRefs::AsofRowRefs(Type type)
|
||||
|
@ -655,7 +655,7 @@ void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace
|
||||
}
|
||||
|
||||
|
||||
SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
|
@ -10,4 +10,4 @@ class IAST;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using ASTs = std::vector<ASTPtr>;
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -30,4 +30,4 @@ private:
|
||||
bool broken = true, claimed = false;
|
||||
};
|
||||
|
||||
} // namespace DB
|
||||
}
|
||||
|
@ -62,7 +62,7 @@ namespace
|
||||
conf.set(key_name, config.getString(key_path));
|
||||
}
|
||||
}
|
||||
} // namespace
|
||||
}
|
||||
|
||||
StorageKafka::StorageKafka(
|
||||
const std::string & table_name_,
|
||||
|
@ -178,3 +178,50 @@ arrayEnumerateUniq(a1, a2) =
|
||||
[1,2]
|
||||
[1,1]
|
||||
[[[[[[[[[[1]]]]]]]]]]
|
||||
[[1],[1]]
|
||||
[[],[1],[1]]
|
||||
[[],[],[],[1],[],[1]]
|
||||
[[],[],[],[],[1],[1]]
|
||||
[[1],[],[1]]
|
||||
[[1],[],[],[1]]
|
||||
[[1],[],[],[1],[1]]
|
||||
-- no order
|
||||
[[],[1,2,3,4]] [[],[1,1,1,1]]
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
-- order no arr
|
||||
[[1,2,3,4]] [[1,1,1,1]]
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
-- order two arr
|
||||
[[],[1,2,3,4]] [[],[1,1,1,1]]
|
||||
[[],[3,4,5]] [[],[1,1,1]]
|
||||
-- order non empt
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
[[6],[1,2,3,4]] [[1],[1,1,1,1]]
|
||||
-- order
|
||||
[[],[1,2,3,4]] [[],[1,1,1,1]]
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
--
|
||||
[[1,1,1,1],[1,1,1,1]]
|
||||
[[],[1,1,1,1],[1,1,1,1]]
|
||||
[[],[1,1,1,1],[],[],[1,1,1,1]]
|
||||
[[1,1,1,1],[],[],[1,1,1,1]]
|
||||
[[1],[1]]
|
||||
[[],[1],[1]]
|
||||
[[],[4]] [[],[1]]
|
||||
[[4]] [[1]]
|
||||
--
|
||||
[[],[1,2,3,4]] [[],[1,1,1,1]]
|
||||
[[],[3,4,5]] [[],[1,1,1]]
|
||||
--
|
||||
[[],[1,2,3,4]] [[],[1,1,1,1]]
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
--
|
||||
[[],[],[1,2,3,4]] [[],[],[1,1,1,1]]
|
||||
[[3,4,5]] [[1,1,1]]
|
||||
--
|
||||
[[],[],[1,2,3,4]] [[],[],[1,1,1,1]]
|
||||
[[],[],[3,4,5]] [[],[],[1,1,1]]
|
||||
--
|
||||
[[],[],[1,2,1,4]] [[],[],[1,1,2,1]]
|
||||
[[],[],[3,4,5,4]] [[],[],[1,1,1,2]]
|
||||
--
|
||||
|
@ -180,3 +180,44 @@ SELECT arrayEnumerateDenseRanked(1.1, [10,20,10,30]); -- { serverError 170 }
|
||||
SELECT arrayEnumerateDenseRanked([10,20,10,30], 0.4); -- { serverError 170 }
|
||||
SELECT arrayEnumerateDenseRanked([10,20,10,30], 1.8); -- { serverError 170 }
|
||||
SELECT arrayEnumerateUniqRanked(1, [], 1000000000); -- { serverError 36 }
|
||||
|
||||
|
||||
-- skipping empty arrays
|
||||
SELECT arrayEnumerateUniqRanked(2, [[3], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [3], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [], [], [3], [], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [], [], [], [3], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[3], [], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[3], [], [], [3]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[3], [], [], [3], [3]]);
|
||||
|
||||
|
||||
select '-- no order';
|
||||
SELECT * FROM (SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ) ) ORDER BY a ASC;
|
||||
select '-- order no arr';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- order two arr';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [1, 2, 3, 4]] AS a UNION ALL SELECT [[], [3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- order non empt';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[6], [1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- order';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT arrayEnumerateUniqRanked(2,[[1, 2, 3, 4], [3, 4, 5, 6]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [1, 2, 3, 4], [3, 4, 5, 6]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [1, 2, 3, 4], [], [], [3, 4, 5, 6]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[1, 2, 3, 4], [], [], [3, 4, 5, 6]]);
|
||||
SELECT arrayEnumerateUniqRanked(2,[[1], [1]]);
|
||||
SELECT arrayEnumerateUniqRanked(2, [[], [1], [1]]);
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [4]] AS a UNION ALL SELECT [[4]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [1, 2, 3, 4]] AS a UNION ALL SELECT [[], [3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [], [1, 2, 3, 4]] AS a UNION ALL SELECT [[3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [], [1, 2, 3, 4]] AS a UNION ALL SELECT [[], [], [3, 4, 5]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
SELECT a, arrayEnumerateUniqRanked(a) FROM ( SELECT * FROM ( SELECT [[], [], [1, 2, 1, 4]] AS a UNION ALL SELECT [[], [], [3, 4, 5, 4]] AS a ) ORDER BY a ASC );
|
||||
select '-- ';
|
||||
|
@ -117,7 +117,7 @@ There are ordinary functions and aggregate functions. For aggregate functions, s
|
||||
|
||||
Ordinary functions don't change the number of rows – they work as if they are processing each row independently. In fact, functions are not called for individual rows, but for `Block`'s of data to implement vectorized query execution.
|
||||
|
||||
There are some miscellaneous functions, like `blockSize`, `rowNumberInBlock`, and `runningAccumulate`, that exploit block processing and violate the independence of rows.
|
||||
There are some miscellaneous functions, like [blockSize](../query_language/functions/other_functions.md#function-blocksize), [rowNumberInBlock](../query_language/functions/other_functions.md#function-rownumberinblock), and [runningAccumulate](../query_language/functions/other_functions.md#function-runningaccumulate), that exploit block processing and violate the independence of rows.
|
||||
|
||||
ClickHouse has strong typing, so implicit type conversion doesn't occur. If a function doesn't support a specific combination of types, an exception will be thrown. But functions can work (be overloaded) for many different combinations of types. For example, the `plus` function (to implement the `+` operator) works for any combination of numeric types: `UInt8` + `Float32`, `UInt16` + `Int8`, and so on. Also, some variadic functions can accept any number of arguments, such as the `concat` function.
|
||||
|
||||
|
@ -11,9 +11,9 @@ Distributed sorting is one of the main causes of reduced performance when runnin
|
||||
|
||||
Most MapReduce implementations allow you to execute arbitrary code on a cluster. But a declarative query language is better suited to OLAP in order to run experiments quickly. For example, Hadoop has Hive and Pig. Also consider Cloudera Impala or Shark (outdated) for Spark, as well as Spark SQL, Presto, and Apache Drill. Performance when running such tasks is highly sub-optimal compared to specialized systems, but relatively high latency makes it unrealistic to use these systems as the backend for a web interface.
|
||||
|
||||
## What to do if I have a problem with encodings when using Oracle through ODBC? {#oracle-odbc-encodings}
|
||||
## What If I Have a Problem with Encodings When Using Oracle Through ODBC? {#oracle-odbc-encodings}
|
||||
|
||||
If you use Oracle through ODBC driver as a source of external dictionaries, you need to set up correctly value for the `NLS_LANG` variable in the `/etc/default/clickhouse`. For more details see the [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html).
|
||||
If you use Oracle through the ODBC driver as a source of external dictionaries, you need to set the correct value for the `NLS_LANG` environment variable in `/etc/default/clickhouse`. For more information, see the [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html).
|
||||
|
||||
**Example**
|
||||
|
||||
|
@ -666,13 +666,13 @@ Path to the file that contains:
|
||||
|
||||
## zookeeper {#server-settings_zookeeper}
|
||||
|
||||
Contains settings that allow ClickHouse to interact with [ZooKeeper](http://zookeeper.apache.org/) cluster.
|
||||
Contains settings that allow ClickHouse to interact with a [ZooKeeper](http://zookeeper.apache.org/) cluster.
|
||||
|
||||
ClickHouse uses ZooKeeper for storing metadata of replicas when using replicated tables. If replicated tables are not used, this parameter section can be omitted.
|
||||
ClickHouse uses ZooKeeper for storing metadata of replicas when using replicated tables. If replicated tables are not used, this section of parameters can be omitted.
|
||||
|
||||
This parameter section contains the following parameters:
|
||||
This section contains the following parameters:
|
||||
|
||||
- `node` — ZooKeeper endpoint. You can set a few endpoints.
|
||||
- `node` — ZooKeeper endpoint. You can set multiple endpoints.
|
||||
|
||||
For example:
|
||||
|
||||
@ -683,12 +683,11 @@ This parameter section contains the following parameters:
|
||||
</node>
|
||||
```
|
||||
|
||||
The `index` attribute is not used in ClickHouse. The only reason for this attribute is to allow some other programs to use the same configuraton.
|
||||
The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster.
|
||||
|
||||
- `session_timeout_ms` — Maximum timeout for client session in milliseconds (default: 30000).
|
||||
- `operation_timeout_ms` — Maximum timeout for operation in milliseconds (default: 10000).
|
||||
- `root` — ZNode, that is used as root for znodes used by ClickHouse server. Optional.
|
||||
- `identity` — User and password, required by ZooKeeper to give access to requested znodes. Optional.
|
||||
- `session_timeout` — Maximum timeout for the client session in milliseconds.
|
||||
- `root` — The [znode](http://zookeeper.apache.org/doc/r3.5.5/zookeeperOver.html#Nodes+and+ephemeral+nodes) that is used as the root for znodes used by the ClickHouse server. Optional.
|
||||
- `identity` — User and password, that can be required by ZooKeeper to give access to requested znodes. Optional.
|
||||
|
||||
**Example configuration**
|
||||
|
||||
|
@ -1,4 +1,3 @@
|
||||
|
||||
# Function reference
|
||||
|
||||
## count() {#agg_function-count}
|
||||
@ -301,6 +300,98 @@ GROUP BY timeslot
|
||||
└─────────────────────┴──────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## skewPop
|
||||
|
||||
Computes the [skewness](https://en.wikipedia.org/wiki/Skewness) for sequence.
|
||||
|
||||
```
|
||||
skewPop(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
`expr` — [Expression](../syntax.md#syntax-expressions) returning a number.
|
||||
|
||||
**Returned value**
|
||||
|
||||
The skewness of given distribution. Type — [Float64](../../data_types/float.md)
|
||||
|
||||
**Example of Use**
|
||||
|
||||
```sql
|
||||
SELECT skewPop(value) FROM series_with_value_column
|
||||
```
|
||||
|
||||
## skewSamp
|
||||
|
||||
Computes the [sample skewness](https://en.wikipedia.org/wiki/Skewness) for sequence.
|
||||
|
||||
It represents an unbiased estimate of the skewness of a random variable, if passed values form it's sample.
|
||||
|
||||
```
|
||||
skewSamp(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
`expr` — [Expression](../syntax.md#syntax-expressions) returning a number.
|
||||
|
||||
**Returned value**
|
||||
|
||||
The skewness of given distribution. Type — [Float64](../../data_types/float.md). If `n <= 1` (`n` is a size of the sample), then the function returns `nan`.
|
||||
|
||||
**Example of Use**
|
||||
|
||||
```sql
|
||||
SELECT skewSamp(value) FROM series_with_value_column
|
||||
```
|
||||
|
||||
## kurtPop
|
||||
|
||||
Computes the [kurtosis](https://en.wikipedia.org/wiki/Kurtosis) for sequence.
|
||||
|
||||
```
|
||||
kurtPop(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
`expr` — [Expression](../syntax.md#syntax-expressions) returning a number.
|
||||
|
||||
**Returned value**
|
||||
|
||||
The kurtosis of given distribution. Type — [Float64](../../data_types/float.md)
|
||||
|
||||
**Example of Use**
|
||||
|
||||
```sql
|
||||
SELECT kurtPop(value) FROM series_with_value_column
|
||||
```
|
||||
|
||||
## kurtSamp
|
||||
|
||||
Computes the [sample kurtosis](https://en.wikipedia.org/wiki/Kurtosis) for sequence.
|
||||
|
||||
It represents an unbiased estimate of the kurtosis of a random variable, if passed values form it's sample.
|
||||
|
||||
```
|
||||
kurtSamp(expr)
|
||||
```
|
||||
|
||||
**Parameters**
|
||||
|
||||
`expr` — [Expression](../syntax.md#syntax-expressions) returning a number.
|
||||
|
||||
**Returned value**
|
||||
|
||||
The kurtosis of given distribution. Type — [Float64](../../data_types/float.md). If `n <= 1` (`n` is a size of the sample), then the function returns `nan`.
|
||||
|
||||
**Example of Use**
|
||||
|
||||
```sql
|
||||
SELECT kurtSamp(value) FROM series_with_value_column
|
||||
```
|
||||
|
||||
## timeSeriesGroupSum(uid, timestamp, value) {#agg_function-timeseriesgroupsum}
|
||||
timeSeriesGroupSum can aggregate different time series that sample timestamp not alignment.
|
||||
It will use linear interpolation between two sample timestamp and then sum time-series together.
|
||||
@ -514,7 +605,7 @@ All the quantile functions also have corresponding quantiles functions: `quantil
|
||||
|
||||
Calculates the amount `Σ((x - x̅)^2) / (n - 1)`, where `n` is the sample size and `x̅`is the average value of `x`.
|
||||
|
||||
It represents an unbiased estimate of the variance of a random variable, if the values passed to the function are a sample of this random amount.
|
||||
It represents an unbiased estimate of the variance of a random variable, if passed values form it's sample.
|
||||
|
||||
Returns `Float64`. When `n <= 1`, returns `+∞`.
|
||||
|
||||
@ -595,7 +686,7 @@ Parameters:
|
||||
|
||||
Returned values:
|
||||
|
||||
Parameters `(a, b)` of the resulting line `x = a*y + b`.
|
||||
Parameters `(a, b)` of the resulting line `y = a*x + b`.
|
||||
|
||||
**Examples**
|
||||
|
||||
|
@ -147,35 +147,35 @@ SELECT
|
||||
└─────────────────────────────────────┴─────────────────────┘
|
||||
```
|
||||
|
||||
## IPv4CIDRtoIPv4Range(ipv4, cidr),
|
||||
## IPv4CIDRToRange(ipv4, cidr),
|
||||
|
||||
Accepts an IPv4 and an UInt8 value containing the CIDR. Return a tuple with two IPv4 containing the lower range and the higher range of the subnet.
|
||||
|
||||
|
||||
```sql
|
||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 16)
|
||||
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 16)
|
||||
```
|
||||
|
||||
```
|
||||
┌─IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 16)─┐
|
||||
┌─IPv4CIDRToRange(toIPv4('192.168.5.2'), 16)─┐
|
||||
│ ('192.168.0.0','192.168.255.255') │
|
||||
└────────────────────────────────────────────────┘
|
||||
└────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
## IPv6CIDRtoIPv6Range(ipv6, cidr),
|
||||
## IPv6CIDRToRange(ipv6, cidr),
|
||||
|
||||
Accepts an IPv6 and an UInt8 value containing the CIDR. Return a tuple with two IPv6 containing the lower range and the higher range of the subnet.
|
||||
|
||||
|
||||
```sql
|
||||
SELECT IPv6CIDRtoIPv6Range(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32);
|
||||
SELECT IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32);
|
||||
```
|
||||
|
||||
```
|
||||
┌─IPv6CIDRtoIPv6Range(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
|
||||
┌─IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
|
||||
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff') │
|
||||
└────────────────────────────────────────────────────────────────────────────┘
|
||||
└────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## toIPv4(string)
|
||||
|
@ -74,7 +74,7 @@ Returns a string containing the type name of the passed argument.
|
||||
|
||||
If `NULL` is passed to the function as input, then it returns the `Nullable(Nothing)` type, which corresponds to an internal `NULL` representation in ClickHouse.
|
||||
|
||||
## blockSize()
|
||||
## blockSize() {#function-blocksize}
|
||||
|
||||
Gets the size of the block.
|
||||
In ClickHouse, queries are always run on blocks (sets of column parts). This function allows getting the size of the block that you called it for.
|
||||
@ -303,7 +303,7 @@ Returns the timezone of the server.
|
||||
|
||||
Returns the sequence number of the data block where the row is located.
|
||||
|
||||
## rowNumberInBlock
|
||||
## rowNumberInBlock {#function-rownumberinblock}
|
||||
|
||||
Returns the ordinal number of the row in the data block. Different data blocks are always recalculated.
|
||||
|
||||
@ -639,7 +639,7 @@ Returns the capacity information of the disk, in bytes. This information is eval
|
||||
|
||||
Takes state of aggregate function. Returns result of aggregation (finalized state).
|
||||
|
||||
## runningAccumulate
|
||||
## runningAccumulate {#function-runningaccumulate}
|
||||
|
||||
Takes the states of the aggregate function and returns a column with values, are the result of the accumulation of these states for a set of block lines, from the first to the current line.
|
||||
For example, takes state of aggregate function (example runningAccumulate(uniqState(UserID))), and for each row of block, return result of aggregate function on merge of states of all previous rows and current row.
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
## Почему бы не использовать системы типа MapReduce?
|
||||
|
||||
Системами типа MapReduce будем называть системы распределённых вычислений, в которых операция reduce сделана на основе распределённой сортировки. Наиболее распространённым opensource решением данного класса является [Apache Hadoop](http://hadoop.apache.org), а в Яндексе используется внутрення разработка — YT.
|
||||
Системами типа MapReduce будем называть системы распределённых вычислений, в которых операция reduce сделана на основе распределённой сортировки. Наиболее распространённым opensource решением данного класса является [Apache Hadoop](http://hadoop.apache.org). Яндекс использует собственное решение — YT.
|
||||
|
||||
Такие системы не подходят для онлайн запросов в силу слишком большой latency. То есть, не могут быть использованы в качестве бэкенда для веб-интерфейса.
|
||||
Такие системы не подходят для обновления данных в реальном времени.
|
||||
@ -10,4 +10,15 @@
|
||||
Распределённая сортировка является основной причиной тормозов при выполнении несложных map-reduce задач.
|
||||
|
||||
Большинство реализаций MapReduce позволяют выполнять произвольный код на кластере. Но для OLAP задач лучше подходит декларативный язык запросов, который позволяет быстро проводить исследования. Для примера, для Hadoop существует Hive и Pig. Также смотрите Cloudera Impala, Shark (устаревший) для Spark, а также Spark SQL, Presto, Apache Drill. Впрочем, производительность при выполнении таких задач является сильно неоптимальной по сравнению со специализированными системами, а сравнительно высокая latency не позволяет использовать эти системы в качестве бэкенда для веб-интерфейса.
|
||||
[Оригинальная статья](https://clickhouse.yandex/docs/ru/faq/general/) <!--hide-->
|
||||
|
||||
## Что делать, если у меня проблема с кодировками при использовании Oracle через ODBC? {#oracle-odbc-encodings}
|
||||
|
||||
Если вы используете Oracle через драйвер ODBC в качестве источника внешних словарей, необходимо задать правильное значение для переменной окружения `NLS_LANG` в `/etc/default/clickhouse`. Подробнее читайте в [Oracle NLS_LANG FAQ](https://www.oracle.com/technetwork/products/globalization/nls-lang-099431.html).
|
||||
|
||||
**Пример**
|
||||
|
||||
```
|
||||
NLS_LANG=RUSSIAN_RUSSIA.UTF8
|
||||
```
|
||||
|
||||
[Оригинальная статья ](https://clickhouse.yandex/docs/en/faq/general/) <!--hide-->
|
||||
|
@ -664,22 +664,17 @@ TCP порт для защищённого обмена данными с кли
|
||||
<users_config>users.xml</users_config>
|
||||
```
|
||||
|
||||
## zookeeper {#server-settings_zookeeper}
|
||||
|
||||
## zookeeper
|
||||
Содержит параметры, позволяющие ClickHouse взаимодействовать с кластером [ZooKeeper](http://zookeeper.apache.org/).
|
||||
|
||||
Конфигурация серверов ZooKeeper.
|
||||
ClickHouse использует ZooKeeper для хранения метаданных о репликах при использовании реплицированных таблиц. Если реплицированные таблицы не используются, этот раздел параметров может отсутствовать.
|
||||
|
||||
Содержит параметры для взаимодействия ClickHouse с кластером [ZooKeeper](http://zookeeper.apache.org/).
|
||||
Раздел содержит следующие параметры:
|
||||
|
||||
ClickHouse использует ZooKeeper для хранения метаданных о репликах при использовании реплицированных таблиц.
|
||||
- `node` — адрес ноды (сервера) ZooKeeper. Можно сконфигурировать несколько нод.
|
||||
|
||||
Параметр можно не указывать, если реплицированные таблицы не используются.
|
||||
|
||||
Содержит следующие параметры:
|
||||
|
||||
- `node` — нода ZooKeeper. Может содержать несколько узлов.
|
||||
|
||||
Пример:
|
||||
Например:
|
||||
|
||||
```xml
|
||||
<node index="1">
|
||||
@ -688,14 +683,13 @@ ClickHouse использует ZooKeeper для хранения метадан
|
||||
</node>
|
||||
```
|
||||
|
||||
Аттрибут `index` не используется в ClickHouse. Он присутствует в примере в связи с тем, что на серверах могут быть установлены другие программы, которые могут считывать тот же конфигурационный файл.
|
||||
Атрибут `index` задает порядок опроса нод при попытках подключиться к кластеру ZooKeeper.
|
||||
|
||||
- `session_timeout_ms` — максимальный таймаут для сессии в миллисекундах (default: 30000).
|
||||
- `operation_timeout_ms` — максимальный таймаут для операции в миллисекундах (default: 10000).
|
||||
- `root` — корневая znode, которая используется сервером ClickHouse для всех остальных znode. Опционально.
|
||||
- `identity` — пара `usename:password` для авторизации в кластере ZooKeeper. Опционально.
|
||||
- `session_timeout` — максимальный таймаут клиентской сессии в миллисекундах.
|
||||
- `root` — [znode](http://zookeeper.apache.org/doc/r3.5.5/zookeeperOver.html#Nodes+and+ephemeral+nodes), который используется как корневой для всех znode, которые использует сервер ClickHouse. Необязательный.
|
||||
- `identity` — пользователь и пароль, которые может потребовать ZooKeeper для доступа к запрошенным znode. Необязательный.
|
||||
|
||||
**Пример**
|
||||
**Пример конфигурации**
|
||||
|
||||
```xml
|
||||
<zookeeper>
|
||||
@ -708,7 +702,6 @@ ClickHouse использует ZooKeeper для хранения метадан
|
||||
<port>2181</port>
|
||||
</node>
|
||||
<session_timeout_ms>30000</session_timeout_ms>
|
||||
<operation_timeout_ms>10000</operation_timeout_ms>
|
||||
<!-- Optional. Chroot suffix. Should exist. -->
|
||||
<root>/path/to/zookeeper/node</root>
|
||||
<!-- Optional. Zookeeper digest ACL string. -->
|
||||
@ -716,9 +709,9 @@ ClickHouse использует ZooKeeper для хранения метадан
|
||||
</zookeeper>
|
||||
```
|
||||
|
||||
**См. также:**
|
||||
**Смотрите также**
|
||||
|
||||
- [Репликация](../../operations/table_engines/replication.md).
|
||||
- [Репликация](../../operations/table_engines/replication.md)
|
||||
- [ZooKeeper Programmer's Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html)
|
||||
|
||||
## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper}
|
||||
|
@ -173,6 +173,7 @@ default_expression String - выражение для значения по ум
|
||||
- `database (String)` - имя базы данных;
|
||||
- `table (String)` - имя таблицы;
|
||||
- `engine (String)` - имя движка таблицы, без параметров.
|
||||
- `is_frozen (UInt8)` – Признак, показывающий существование бэкапа партиции. 1, бэкап есть. 0, бэкапа нет. Смотрите раздел [FREEZE PARTITION](../query_language/alter.md#alter_freeze-partition)
|
||||
|
||||
## system.part_log {#system_tables-part-log}
|
||||
|
||||
|
@ -121,6 +121,8 @@
|
||||
|
||||
ClickHouse получает от ODBC-драйвера информацию о квотировании и квотирует настройки в запросах к драйверу, поэтому имя таблицы нужно указывать в соответствии с регистром имени таблицы в базе данных.
|
||||
|
||||
Если у вас есть проблемы с кодировками при использовании Oracle, ознакомьтесь с соответствущим разделом [FAQ](../../faq/general.md#oracle-odbc-encodings).
|
||||
|
||||
### Выявленная уязвимость в функционировании ODBC словарей
|
||||
|
||||
!!! attention
|
||||
|
@ -22,7 +22,7 @@ SELECT visibleWidth(NULL)
|
||||
|
||||
Если на вход функции передать `NULL`, то она вернёт тип `Nullable(Nothing)`, что соответствует внутреннему представлению `NULL` в ClickHouse.
|
||||
|
||||
## blockSize()
|
||||
## blockSize() {#function-blocksize}
|
||||
Получить размер блока.
|
||||
В ClickHouse выполнение запроса всегда идёт по блокам (наборам кусочков столбцов). Функция позволяет получить размер блока, для которого её вызвали.
|
||||
|
||||
@ -225,6 +225,10 @@ SELECT
|
||||
## version()
|
||||
Возвращает версию сервера в виде строки.
|
||||
|
||||
## rowNumberInBlock {#function-rownumberinblock}
|
||||
|
||||
Возвращает порядковый номер строки в блоке данных. Для каждого блока данных нумерация начинается с 0.
|
||||
|
||||
## rowNumberInAllBlocks()
|
||||
Возвращает порядковый номер строки в блоке данных. Функция учитывает только задействованные блоки данных.
|
||||
|
||||
@ -540,4 +544,8 @@ SELECT replicate(1, ['a', 'b', 'c'])
|
||||
└───────────────────────────────┘
|
||||
```
|
||||
|
||||
## runningAccumulate {#function-runningaccumulate}
|
||||
|
||||
Принимает на вход состояния агрегатной функции и возвращает столбец со значениями, которые представляют собой результат мёржа этих состояний для выборки строк из блока от первой до текущей строки. Например, принимает состояние агрегатной функции (например, `runningAccumulate(uniqState(UserID))`), и для каждой строки блока возвращает результат агрегатной функции после мёржа состояний функции для всех предыдущих строк и текущей. Таким образом, результат зависит от разбиения данных по блокам и от порядка данных в блоке.
|
||||
|
||||
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/other_functions/) <!--hide-->
|
||||
|
@ -147,35 +147,35 @@ SELECT
|
||||
└─────────────────────────────────────┴─────────────────────┘
|
||||
```
|
||||
|
||||
## IPv4CIDRtoIPv4Range(ipv4, cidr),
|
||||
## IPv4CIDRToRange(ipv4, cidr),
|
||||
|
||||
接受一个IPv4地址以及一个UInt8类型的CIDR。返回包含子网最低范围以及最高范围的元组。
|
||||
|
||||
|
||||
```sql
|
||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 16)
|
||||
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 16)
|
||||
```
|
||||
|
||||
```
|
||||
┌─IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 16)─┐
|
||||
┌─IPv4CIDRToRange(toIPv4('192.168.5.2'), 16)─┐
|
||||
│ ('192.168.0.0','192.168.255.255') │
|
||||
└────────────────────────────────────────────────┘
|
||||
└────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
|
||||
## IPv6CIDRtoIPv6Range(ipv6, cidr),
|
||||
## IPv6CIDRToRange(ipv6, cidr),
|
||||
|
||||
接受一个IPv6地址以及一个UInt8类型的CIDR。返回包含子网最低范围以及最高范围的元组。
|
||||
|
||||
|
||||
```sql
|
||||
SELECT IPv6CIDRtoIPv6Range(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32);
|
||||
SELECT IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32);
|
||||
```
|
||||
|
||||
```
|
||||
┌─IPv6CIDRtoIPv6Range(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
|
||||
┌─IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
|
||||
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff') │
|
||||
└────────────────────────────────────────────────────────────────────────────┘
|
||||
└────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## toIPv4(string)
|
||||
|
@ -20,3 +20,8 @@ find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' |
|
||||
# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character | missing whitespace after for/if/while... before opening brace | whitespaces inside braces
|
||||
grep -v -P '(//|:\s+\*|\$\(\()| \)"'
|
||||
# single-line comment | continuation of a multiline comment | a typical piece of embedded shell code | something like ending of raw string literal
|
||||
|
||||
# // namespace comments are unneeded
|
||||
find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' |
|
||||
grep -vP 'Compiler|build' |
|
||||
xargs grep $@ -P '}\s*//+\s*namespace\s*'
|
||||
|
@ -94,7 +94,7 @@
|
||||
</div>
|
||||
<div id="announcement" class="colored-block">
|
||||
<div class="page">
|
||||
Upcoming ClickHouse Meetup: <a class="announcement-link" href="https://www.huodongxing.com/event/2483759276200" rel="external nofollow" target="_blank">Beijing</a> on June 8
|
||||
Upcoming ClickHouse Meetup: <a class="announcement-link" href="https://www.huodongxing.com/event/3483759917300" rel="external nofollow" target="_blank">Shenzhen</a> on October 20
|
||||
</div>
|
||||
</div>
|
||||
<div class="page">
|
||||
|
Loading…
Reference in New Issue
Block a user