Merge branch 'master' of https://github.com/vzakaznikov/ClickHouse into liveview

This commit is contained in:
Vitaliy Zakaznikov 2019-06-13 15:23:20 -04:00
commit 117f2d191e
39 changed files with 338 additions and 109 deletions

View File

@ -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.

View File

@ -117,11 +117,11 @@ namespace
__builtin_unreachable();
}
} // namespace
}
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
{
factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined);
}
} // namespace DB
}

View File

@ -63,7 +63,7 @@ namespace detail
}
};
} // namespace detail
}
template <typename Key, UInt8 K>
@ -231,4 +231,4 @@ public:
}
};
} // namespace DB
}

View File

@ -60,4 +60,4 @@ public:
}
};
} // namespace DB
}

View File

@ -21,4 +21,4 @@ private:
Block readImpl() override { return {}; }
};
} /// namespace DB
}

View File

@ -85,4 +85,4 @@ public:
String getName() const override { return name; }
};
} // namespace DB
}

View File

@ -98,7 +98,7 @@ public:
}
};
} // namespace
}
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
{
@ -115,4 +115,4 @@ void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
});
}
} // namespace DB
}

View File

@ -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
}

View File

@ -50,4 +50,4 @@ public:
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
};
} // namespace DB
}

View File

@ -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)

View File

@ -39,7 +39,7 @@ const DB::IColumn & unwrapNullableColumn(const DB::IColumn & column)
return column;
}
} // namespace
}
namespace DB
{

View File

@ -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
}

View File

@ -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

View File

@ -4,6 +4,7 @@
namespace DB
{
class FunctionArrayEnumerateUniqRanked : public FunctionArrayEnumerateRankedExtended<FunctionArrayEnumerateUniqRanked>
{
using Base = FunctionArrayEnumerateRankedExtended<FunctionArrayEnumerateUniqRanked>;

View File

@ -49,4 +49,4 @@ private:
bool put_delimiter = false;
};
} // namespace DB
}

View File

@ -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;

View File

@ -127,7 +127,7 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
return expr_list->children.at(0)->getColumnName();
}
} // namespace
}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,

View File

@ -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_)

View File

@ -27,7 +27,7 @@ void callWithType(AsofRowRefs::Type which, F && f)
__builtin_unreachable();
}
} // namespace
}
AsofRowRefs::AsofRowRefs(Type type)

View File

@ -655,7 +655,7 @@ void replaceJoinedTable(const ASTTablesInSelectQueryElement* join)
}
}
} // namespace
}
SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(

View File

@ -10,4 +10,4 @@ class IAST;
using ASTPtr = std::shared_ptr<IAST>;
using ASTs = std::vector<ASTPtr>;
} // namespace DB
}

View File

@ -30,4 +30,4 @@ private:
bool broken = true, claimed = false;
};
} // namespace DB
}

View File

@ -62,7 +62,7 @@ namespace
conf.set(key_name, config.getString(key_path));
}
}
} // namespace
}
StorageKafka::StorageKafka(
const std::string & table_name_,

View File

@ -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]]
--

View File

@ -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 '-- ';

View File

@ -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.

View File

@ -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**

View File

@ -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**

View File

@ -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**
@ -654,11 +745,11 @@ To predict we use function `evalMLMethod`, which takes a state as an argument as
param2 Float64,
target Float64
) ENGINE = Memory;
CREATE TABLE your_model ENGINE = Memory AS SELECT
stochasticLinearRegressionState(0.1, 0.0, 5, 'SGD')(target, param1, param2)
AS state FROM train_data;
```
Here we also need to insert data into `train_data` table. The number of parameters is not fixed, it depends only on number of arguments, passed into `linearRegressionState`. They all must be numeric values.
Note that the column with target value(which we would like to learn to predict) is inserted as the first argument.
@ -671,7 +762,7 @@ To predict we use function `evalMLMethod`, which takes a state as an argument as
evalMLMethod(model, param1, param2) FROM test_data
```
The query will return a column of predicted values. Note that first argument of `evalMLMethod` is `AggregateFunctionState` object, next are columns of features.
`test_data` is a table like `train_data` but may not contain target value.
**Some notes**
@ -681,12 +772,12 @@ To predict we use function `evalMLMethod`, which takes a state as an argument as
SELECT state1 + state2 FROM your_models
```
where `your_models` table contains both models. This query will return new `AggregateFunctionState` object.
2. User may fetch weights of the created model for its own purposes without saving the model if no `-State` combinator is used.
```sql
SELECT stochasticLinearRegression(0.01)(target, param1, param2) FROM train_data
```
Such query will fit the model and return its weights - first are weights, which correspond to the parameters of the model, the last one is bias. So in the example above the query will return a column with 3 values.
Such query will fit the model and return its weights - first are weights, which correspond to the parameters of the model, the last one is bias. So in the example above the query will return a column with 3 values.
## logisticRegression
@ -696,8 +787,8 @@ This function implements stochastic logistic regression. It can be used for bina
#### Parameters
Parameters are exactly the same as in stochasticLinearRegression:
`learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`.
Parameters are exactly the same as in stochasticLinearRegression:
`learning rate`, `l2 regularization coefficient`, `mini-batch size`, `method for updating weights`.
For more information see [parameters](#parameters).
```text
stochasticLogisticRegression(1.0, 1.0, 10, 'SGD')
@ -706,9 +797,9 @@ stochasticLogisticRegression(1.0, 1.0, 10, 'SGD')
1. *Fitting*
See *stochasticLinearRegression.Fitting*
Predicted labels have to be in {-1, 1}.
2. *Predicting*
Using saved state we can predict probability of object having label *1*.

View File

@ -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)─┐
│ ('192.168.0.0','192.168.255.255')
└────────────────────────────────────────────────
┌─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)─┐
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff')
└────────────────────────────────────────────────────────────────────────────
┌─IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff') │
└────────────────────────────────────────────────────────────────────────┘
```
## toIPv4(string)

View File

@ -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.

View File

@ -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-->

View File

@ -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}

View File

@ -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}
@ -232,7 +233,7 @@ query_id String - идентификатор запроса, если
!!! note "Внимание"
Таблица не содержит данные, передаваемые в запросах `INSERT`.
Таблица `system.query_log` создаётся только в том случае, если задана серверная настройка [query_log](server_settings/settings.md#server_settings-query-log). Эта настройка определяет правила логирования. Например, с какой периодичностью логи будут записываться в таблицу. Также в этой настройке можно изменить название таблицы.
Чтобы включить логирование запросов, необходимо установить параметр [log_queries](settings/settings.md#settings-log-queries) в 1. Подробнее см. в разделе [Настройки](settings/settings.md).
@ -240,7 +241,7 @@ query_id String - идентификатор запроса, если
Логируются следующие запросы:
1. Запросы, которые были вызваны непосредственно клиентом.
2. Дочерние запросы, которые были вызваны другими запросами (при распределенном выполнении запросов). Для дочерних запросов, информация о родительских запросах содержится в столбцах `initial_*`.
2. Дочерние запросы, которые были вызваны другими запросами (при распределенном выполнении запросов). Для дочерних запросов, информация о родительских запросах содержится в столбцах `initial_*`.
Столбцы:
@ -248,7 +249,7 @@ query_id String - идентификатор запроса, если
- 1 — запуск запроса произошел успешно;
- 2 — запрос выполнен успешно;
- 3 — при выполнении запроса возникла ошибка;
- 4 — перед запуском запроса возникла ошибка.
- 4 — перед запуском запроса возникла ошибка.
- `event_date` (Date) — дата возникновения события;
- `event_time` (DateTime) — время возникновения события;
- `query_start_time` (DateTime) — время запуска запроса;
@ -285,16 +286,16 @@ query_id String - идентификатор запроса, если
- `client_version_minor` (UInt32) — минорная версия [clickhouse-client](../interfaces/cli.md);
- `client_version_patch` (UInt32) — patch-компонент версии [clickhouse-client](../interfaces/cli.md);
- `http_method` (UInt8) — используемый HTTP-метод. Возможные значения:
- 0 — запрос был вызван из TCP интерфейса;
- 0 — запрос был вызван из TCP интерфейса;
- 1 — метод `GET`;
- 2 — метод `POST`.
- `http_user_agent` (String) — содержимое заголовка `UserAgent`;
- `quota_key` (String) — ключ квоты, заданный в настройке [quotas](quotas.md);
- `revision` (UInt32) — ревизия сервера ClickHouse;
- `thread_numbers` (Array(UInt32)) — номера потоков, участвующих в выполнении запроса;
- `ProfileEvents.Names` (Array(String)) — счётчики, измеряющие метрики:
- `ProfileEvents.Names` (Array(String)) — счётчики, измеряющие метрики:
- время, потраченное на чтение и запись по сети;
- чтение и запись на диск;
- чтение и запись на диск;
- количество сетевых ошибок;
- время, затраченное на ожидание, при ограниченной пропускной способности сети.
- `ProfileEvents.Values` (Array(UInt64)) — значения счётчиков, перечисленных в `ProfileEvents.Names`.
@ -309,7 +310,7 @@ query_id String - идентификатор запроса, если
По умолчанию, логи записываются в таблицу с периодичностью в 7,5 секунд. Частоту записи логов можно регулировать настройкой [query_log](server_settings/settings.md#server_settings-query-log) (см. параметр `flush_interval_milliseconds`). Чтобы принудительно пробросить логи из буфера памяти в таблицу, используйте запрос `SYSTEM FLUSH LOGS`.
При ручном удалении таблицы, она будет повторно создана на лету. Логи, которые содержались в таблице до её удаления, не сохраняются.
При ручном удалении таблицы, она будет повторно создана на лету. Логи, которые содержались в таблице до её удаления, не сохраняются.
!!! note "Примечание"
Срок хранения логов в таблице неограничен — они не удаляются автоматически. Об удалении неактуальных логов вам нужно позаботиться самостоятельно.

View File

@ -121,6 +121,8 @@
ClickHouse получает от ODBC-драйвера информацию о квотировании и квотирует настройки в запросах к драйверу, поэтому имя таблицы нужно указывать в соответствии с регистром имени таблицы в базе данных.
Если у вас есть проблемы с кодировками при использовании Oracle, ознакомьтесь с соответствущим разделом [FAQ](../../faq/general.md#oracle-odbc-encodings).
### Выявленная уязвимость в функционировании ODBC словарей
!!! attention

View File

@ -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-->

View File

@ -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)─┐
│ ('192.168.0.0','192.168.255.255')
└────────────────────────────────────────────────
┌─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)─┐
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff')
└────────────────────────────────────────────────────────────────────────────
┌─IPv6CIDRToRange(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32)─┐
│ ('2001:db8::','2001:db8:ffff:ffff:ffff:ffff:ffff:ffff') │
└────────────────────────────────────────────────────────────────────────┘
```
## toIPv4(string)

View File

@ -19,4 +19,9 @@ find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' |
xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|if constexpr|else if constexpr|for|while|catch|switch)\(|\( [^\s\\]|\S \)' |
# 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
# 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*'

View File

@ -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">