mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into break_some_tests
This commit is contained in:
commit
1e5f8ed9cf
@ -43,10 +43,10 @@ CREATE DATABASE mysql ENGINE = MaterializedMySQL('localhost:3306', 'db', 'user',
|
||||
|
||||
**Settings on MySQL-server side**
|
||||
|
||||
For the correct work of `MaterializeMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set:
|
||||
For the correct work of `MaterializedMySQL`, there are few mandatory `MySQL`-side configuration settings that should be set:
|
||||
|
||||
- `default_authentication_plugin = mysql_native_password` since `MaterializeMySQL` can only authorize with this method.
|
||||
- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializeMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`.
|
||||
- `default_authentication_plugin = mysql_native_password` since `MaterializedMySQL` can only authorize with this method.
|
||||
- `gtid_mode = on` since GTID based logging is a mandatory for providing correct `MaterializedMySQL` replication. Pay attention that while turning this mode `On` you should also specify `enforce_gtid_consistency = on`.
|
||||
|
||||
## Virtual columns {#virtual-columns}
|
||||
|
||||
|
@ -125,6 +125,44 @@ Result:
|
||||
└───────────────────────────┘
|
||||
```
|
||||
|
||||
## subBitmap {#subBitmap}
|
||||
|
||||
Creates a subset of bitmap limit the results to `cardinality_limit` with offset of `offset`.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
subBitmap(bitmap, offset, cardinality_limit)
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild).
|
||||
- `offset` – the number of offsets. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
|
||||
- `cardinality_limit` – The subset cardinality upper limit. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
|
||||
|
||||
**Returned value**
|
||||
|
||||
The subset.
|
||||
|
||||
Type: `Bitmap object`.
|
||||
|
||||
**Example**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─res─────────────────────────────┐
|
||||
│ [10,11,12,13,14,15,16,17,18,19] │
|
||||
└─────────────────────────────────┘
|
||||
```
|
||||
|
||||
## bitmapContains {#bitmap_functions-bitmapcontains}
|
||||
|
||||
Checks whether the bitmap contains an element.
|
||||
|
@ -88,6 +88,30 @@ SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12
|
||||
│ [30,31,32,33,100,200,500] │
|
||||
└───────────────────────────┘
|
||||
|
||||
## subBitmap {#subBitmap}
|
||||
|
||||
将位图跳过`offset`个元素,限制大小为`limit`个的结果转换为另一个位图。
|
||||
|
||||
subBitmap(bitmap, offset, limit)
|
||||
|
||||
**参数**
|
||||
|
||||
- `bitmap` – 位图对象.
|
||||
- `offset` – 跳过多少个元素.
|
||||
- `limit` – 子位图基数上限.
|
||||
|
||||
**示例**
|
||||
|
||||
``` sql
|
||||
SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res
|
||||
```
|
||||
|
||||
```text
|
||||
┌─res─────────────────────────────┐
|
||||
│ [10,11,12,13,14,15,16,17,18,19] │
|
||||
└─────────────────────────────────┘
|
||||
```
|
||||
|
||||
## bitmapContains {#bitmapcontains}
|
||||
|
||||
检查位图是否包含指定元素。
|
||||
|
@ -1,3 +1,6 @@
|
||||
#include <string>
|
||||
#include "Common/MemoryTracker.h"
|
||||
#include "Columns/ColumnsNumber.h"
|
||||
#include "ConnectionParameters.h"
|
||||
#include "QueryFuzzer.h"
|
||||
#include "Suggest.h"
|
||||
@ -100,6 +103,14 @@
|
||||
#pragma GCC optimize("-fno-var-tracking-assignments")
|
||||
#endif
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric Revision;
|
||||
extern const Metric VersionInteger;
|
||||
extern const Metric MemoryTracking;
|
||||
extern const Metric MaxDDLEntryID;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
@ -524,6 +535,18 @@ private:
|
||||
{
|
||||
UseSSL use_ssl;
|
||||
|
||||
MainThreadStatus::getInstance();
|
||||
|
||||
/// Limit on total memory usage
|
||||
size_t max_client_memory_usage = config().getInt64("max_memory_usage_in_client", 0 /*default value*/);
|
||||
|
||||
if (max_client_memory_usage != 0)
|
||||
{
|
||||
total_memory_tracker.setHardLimit(max_client_memory_usage);
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
}
|
||||
|
||||
registerFormats();
|
||||
registerFunctions();
|
||||
registerAggregateFunctions();
|
||||
@ -2581,6 +2604,7 @@ public:
|
||||
("opentelemetry-tracestate", po::value<std::string>(), "OpenTelemetry tracestate header as described by W3C Trace Context recommendation")
|
||||
("history_file", po::value<std::string>(), "path to history file")
|
||||
("no-warnings", "disable warnings when client connects to server")
|
||||
("max_memory_usage_in_client", po::value<int>(), "sets memory limit in client")
|
||||
;
|
||||
|
||||
Settings cmd_settings;
|
||||
|
@ -68,7 +68,7 @@
|
||||
html, body
|
||||
{
|
||||
/* Personal choice. */
|
||||
font-family: Sans-Serif;
|
||||
font-family: Liberation Sans, DejaVu Sans, sans-serif, Noto Color Emoji, Apple Color Emoji, Segoe UI Emoji;
|
||||
background: var(--background-color);
|
||||
color: var(--text-color);
|
||||
}
|
||||
@ -96,11 +96,16 @@
|
||||
.monospace
|
||||
{
|
||||
/* Prefer fonts that have full hinting info. This is important for non-retina displays.
|
||||
Also I personally dislike "Ubuntu" font due to the similarity of 'r' and 'г' (it looks very ignorant).
|
||||
*/
|
||||
Also I personally dislike "Ubuntu" font due to the similarity of 'r' and 'г' (it looks very ignorant). */
|
||||
font-family: Liberation Mono, DejaVu Sans Mono, MonoLisa, Consolas, Monospace;
|
||||
}
|
||||
|
||||
.monospace-table
|
||||
{
|
||||
/* Liberation is worse than DejaVu for block drawing characters. */
|
||||
font-family: DejaVu Sans Mono, Liberation Mono, MonoLisa, Consolas, Monospace;
|
||||
}
|
||||
|
||||
.shadow
|
||||
{
|
||||
box-shadow: 0 0 1rem var(--shadow-color);
|
||||
@ -325,8 +330,8 @@
|
||||
<span id="toggle-dark">🌑</span><span id="toggle-light">🌞</span>
|
||||
</div>
|
||||
<div id="data_div">
|
||||
<table class="monospace shadow" id="data-table"></table>
|
||||
<pre class="monospace shadow" id="data-unparsed"></pre>
|
||||
<table class="monospace-table shadow" id="data-table"></table>
|
||||
<pre class="monospace-table shadow" id="data-unparsed"></pre>
|
||||
</div>
|
||||
<svg id="graph" fill="none"></svg>
|
||||
<p id="error" class="monospace shadow">
|
||||
@ -367,7 +372,7 @@
|
||||
const server_address = document.getElementById('url').value;
|
||||
|
||||
const url = server_address +
|
||||
(server_address.indexOf('?') >= 0 ? '&' : '?') +
|
||||
(server_address.indexOf('?') >= 0 ? '&' : '?') +
|
||||
/// Ask server to allow cross-domain requests.
|
||||
'add_http_cors_header=1' +
|
||||
'&user=' + encodeURIComponent(user) +
|
||||
|
@ -579,6 +579,37 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 rb_offset_limit(UInt64 offset, UInt64 limit, RoaringBitmapWithSmallSet & r1) const
|
||||
{
|
||||
if (limit == 0 || offset >= size())
|
||||
return 0;
|
||||
|
||||
if (isSmall())
|
||||
{
|
||||
UInt64 count = 0;
|
||||
UInt64 offset_count = 0;
|
||||
auto it = small.begin();
|
||||
for (;it != small.end() && offset_count < offset; ++it)
|
||||
++offset_count;
|
||||
|
||||
for (;it != small.end() && count < limit; ++it, ++count)
|
||||
r1.add(it->getValue());
|
||||
return count;
|
||||
}
|
||||
else
|
||||
{
|
||||
UInt64 count = 0;
|
||||
UInt64 offset_count = 0;
|
||||
auto it = rb->begin();
|
||||
for (;it != rb->end() && offset_count < offset; ++it)
|
||||
++offset_count;
|
||||
|
||||
for (;it != rb->end() && count < limit; ++it, ++count)
|
||||
r1.add(*it);
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 rb_min() const
|
||||
{
|
||||
if (isSmall())
|
||||
|
@ -187,6 +187,7 @@ public:
|
||||
* So LC(Nullable(T)) would return true, LC(U) -- false.
|
||||
*/
|
||||
bool nestedIsNullable() const { return isColumnNullable(*dictionary.getColumnUnique().getNestedColumn()); }
|
||||
bool nestedCanBeInsideNullable() const { return dictionary.getColumnUnique().getNestedColumn()->canBeInsideNullable(); }
|
||||
void nestedToNullable() { dictionary.getColumnUnique().nestedToNullable(); }
|
||||
void nestedRemoveNullable() { dictionary.getColumnUnique().nestedRemoveNullable(); }
|
||||
|
||||
|
@ -44,6 +44,13 @@ void Block::initializeIndexByName()
|
||||
}
|
||||
|
||||
|
||||
void Block::reserve(size_t count)
|
||||
{
|
||||
index_by_name.reserve(count);
|
||||
data.reserve(count);
|
||||
}
|
||||
|
||||
|
||||
void Block::insert(size_t position, ColumnWithTypeAndName elem)
|
||||
{
|
||||
if (position > data.size())
|
||||
@ -287,6 +294,7 @@ std::string Block::dumpIndex() const
|
||||
Block Block::cloneEmpty() const
|
||||
{
|
||||
Block res;
|
||||
res.reserve(data.size());
|
||||
|
||||
for (const auto & elem : data)
|
||||
res.insert(elem.cloneEmpty());
|
||||
@ -364,6 +372,8 @@ Block Block::cloneWithColumns(MutableColumns && columns) const
|
||||
Block res;
|
||||
|
||||
size_t num_columns = data.size();
|
||||
res.reserve(num_columns);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
res.insert({ std::move(columns[i]), data[i].type, data[i].name });
|
||||
|
||||
@ -381,6 +391,8 @@ Block Block::cloneWithColumns(const Columns & columns) const
|
||||
throw Exception("Cannot clone block with columns because block has " + toString(num_columns) + " columns, "
|
||||
"but " + toString(columns.size()) + " columns given.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
res.reserve(num_columns);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
res.insert({ columns[i], data[i].type, data[i].name });
|
||||
|
||||
@ -393,6 +405,8 @@ Block Block::cloneWithoutColumns() const
|
||||
Block res;
|
||||
|
||||
size_t num_columns = data.size();
|
||||
res.reserve(num_columns);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
res.insert({ nullptr, data[i].type, data[i].name });
|
||||
|
||||
|
@ -152,6 +152,7 @@ public:
|
||||
private:
|
||||
void eraseImpl(size_t position);
|
||||
void initializeIndexByName();
|
||||
void reserve(size_t count);
|
||||
|
||||
/// This is needed to allow function execution over data.
|
||||
/// It is safe because functions does not change column names, so index is unaffected.
|
||||
|
@ -31,6 +31,10 @@ SRCS(
|
||||
MySQL/PacketsProtocolText.cpp
|
||||
MySQL/PacketsReplication.cpp
|
||||
NamesAndTypes.cpp
|
||||
PostgreSQL/Connection.cpp
|
||||
PostgreSQL/PoolWithFailover.cpp
|
||||
PostgreSQL/Utils.cpp
|
||||
PostgreSQL/insertPostgreSQLValue.cpp
|
||||
PostgreSQLProtocol.cpp
|
||||
QueryProcessingStage.cpp
|
||||
Settings.cpp
|
||||
|
@ -49,6 +49,7 @@ SRCS(
|
||||
TTLUpdateInfoAlgorithm.cpp
|
||||
copyData.cpp
|
||||
finalizeBlock.cpp
|
||||
formatBlock.cpp
|
||||
materializeBlock.cpp
|
||||
narrowBlockInputStreams.cpp
|
||||
|
||||
|
@ -13,6 +13,7 @@ void registerFunctionsBitmap(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionBitmapToArray>();
|
||||
factory.registerFunction<FunctionBitmapSubsetInRange>();
|
||||
factory.registerFunction<FunctionBitmapSubsetLimit>();
|
||||
factory.registerFunction<FunctionBitmapSubsetOffsetLimit>();
|
||||
factory.registerFunction<FunctionBitmapTransform>();
|
||||
|
||||
factory.registerFunction<FunctionBitmapSelfCardinality>();
|
||||
|
@ -460,9 +460,24 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
struct BitmapSubsetOffsetLimitImpl
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "subBitmap";
|
||||
template <typename T>
|
||||
static void apply(
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0,
|
||||
UInt64 range_start,
|
||||
UInt64 range_end,
|
||||
AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
|
||||
{
|
||||
bitmap_data_0.rbs.rb_offset_limit(range_start, range_end, bitmap_data_2.rbs);
|
||||
}
|
||||
};
|
||||
|
||||
using FunctionBitmapSubsetInRange = FunctionBitmapSubset<BitmapSubsetInRangeImpl>;
|
||||
using FunctionBitmapSubsetLimit = FunctionBitmapSubset<BitmapSubsetLimitImpl>;
|
||||
|
||||
using FunctionBitmapSubsetOffsetLimit = FunctionBitmapSubset<BitmapSubsetOffsetLimitImpl>;
|
||||
|
||||
class FunctionBitmapTransform : public IFunction
|
||||
{
|
||||
|
@ -1091,7 +1091,14 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
|
||||
{
|
||||
sensor_file->rewind();
|
||||
Int64 temperature = 0;
|
||||
readText(temperature, *sensor_file);
|
||||
try
|
||||
{
|
||||
readText(temperature, *sensor_file);
|
||||
}
|
||||
catch (const ErrnoException & e)
|
||||
{
|
||||
LOG_DEBUG(&Poco::Logger::get("AsynchronousMetrics"), "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", hwmon_name, sensor_name, e.getErrno());
|
||||
}
|
||||
|
||||
if (sensor_name.empty())
|
||||
new_values[fmt::format("Temperature_{}", hwmon_name)] = temperature * 0.001;
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
@ -105,25 +106,57 @@ DataTypePtr convertTypeToNullable(const DataTypePtr & type)
|
||||
return type;
|
||||
}
|
||||
|
||||
/// Convert column to nullable. If column LowCardinality or Const, convert nested column.
|
||||
/// Returns nullptr if conversion cannot be performed.
|
||||
static ColumnPtr tryConvertColumnToNullable(const ColumnPtr & col)
|
||||
{
|
||||
if (isColumnNullable(*col) || col->canBeInsideNullable())
|
||||
return makeNullable(col);
|
||||
|
||||
if (col->lowCardinality())
|
||||
{
|
||||
auto mut_col = IColumn::mutate(std::move(col));
|
||||
ColumnLowCardinality * col_lc = assert_cast<ColumnLowCardinality *>(mut_col.get());
|
||||
if (col_lc->nestedIsNullable())
|
||||
{
|
||||
return mut_col;
|
||||
}
|
||||
else if (col_lc->nestedCanBeInsideNullable())
|
||||
{
|
||||
col_lc->nestedToNullable();
|
||||
return mut_col;
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(*col))
|
||||
{
|
||||
const auto & nested = col_const->getDataColumnPtr();
|
||||
if (nested->isNullable() || nested->canBeInsideNullable())
|
||||
{
|
||||
return makeNullable(col);
|
||||
}
|
||||
else if (nested->lowCardinality())
|
||||
{
|
||||
ColumnPtr nested_nullable = tryConvertColumnToNullable(nested);
|
||||
if (nested_nullable)
|
||||
return ColumnConst::create(nested_nullable, col_const->size());
|
||||
}
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void convertColumnToNullable(ColumnWithTypeAndName & column)
|
||||
{
|
||||
column.type = convertTypeToNullable(column.type);
|
||||
|
||||
if (!column.column)
|
||||
{
|
||||
column.type = convertTypeToNullable(column.type);
|
||||
return;
|
||||
|
||||
if (column.column->lowCardinality())
|
||||
{
|
||||
/// Convert nested to nullable, not LowCardinality itself
|
||||
auto mut_col = IColumn::mutate(std::move(column.column));
|
||||
ColumnLowCardinality * col_as_lc = assert_cast<ColumnLowCardinality *>(mut_col.get());
|
||||
if (!col_as_lc->nestedIsNullable())
|
||||
col_as_lc->nestedToNullable();
|
||||
column.column = std::move(mut_col);
|
||||
}
|
||||
else if (column.column->canBeInsideNullable())
|
||||
|
||||
ColumnPtr nullable_column = tryConvertColumnToNullable(column.column);
|
||||
if (nullable_column)
|
||||
{
|
||||
column.column = makeNullable(column.column);
|
||||
column.type = convertTypeToNullable(column.type);
|
||||
column.column = std::move(nullable_column);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -21,6 +21,7 @@ SRCS(
|
||||
ASTCreateRowPolicyQuery.cpp
|
||||
ASTCreateSettingsProfileQuery.cpp
|
||||
ASTCreateUserQuery.cpp
|
||||
ASTDatabaseOrNone.cpp
|
||||
ASTDictionary.cpp
|
||||
ASTDictionaryAttributeDeclaration.cpp
|
||||
ASTDropAccessEntityQuery.cpp
|
||||
@ -95,6 +96,7 @@ SRCS(
|
||||
ParserCreateSettingsProfileQuery.cpp
|
||||
ParserCreateUserQuery.cpp
|
||||
ParserDataType.cpp
|
||||
ParserDatabaseOrNone.cpp
|
||||
ParserDescribeTableQuery.cpp
|
||||
ParserDictionary.cpp
|
||||
ParserDictionaryAttributeDeclaration.cpp
|
||||
|
@ -1166,6 +1166,23 @@ void WindowTransform::appendChunk(Chunk & chunk)
|
||||
// Write out the aggregation results.
|
||||
writeOutCurrentRow();
|
||||
|
||||
if (isCancelled())
|
||||
{
|
||||
// Good time to check if the query is cancelled. Checking once
|
||||
// per block might not be enough in severe quadratic cases.
|
||||
// Just leave the work halfway through and return, the 'prepare'
|
||||
// method will figure out what to do. Note that this doesn't
|
||||
// handle 'max_execution_time' and other limits, because these
|
||||
// limits are only updated between blocks. Eventually we should
|
||||
// start updating them in background and canceling the processor,
|
||||
// like we do for Ctrl+C handling.
|
||||
//
|
||||
// This class is final, so the check should hopefully be
|
||||
// devirtualized and become a single never-taken branch that is
|
||||
// basically free.
|
||||
return;
|
||||
}
|
||||
|
||||
// Move to the next row. The frame will have to be recalculated.
|
||||
// The peer group start is updated at the beginning of the loop,
|
||||
// because current_row might now be past-the-end.
|
||||
@ -1255,10 +1272,12 @@ IProcessor::Status WindowTransform::prepare()
|
||||
// next_output_block_number, first_not_ready_row, first_block_number,
|
||||
// blocks.size());
|
||||
|
||||
if (output.isFinished())
|
||||
if (output.isFinished() || isCancelled())
|
||||
{
|
||||
// The consumer asked us not to continue (or we decided it ourselves),
|
||||
// so we abort.
|
||||
// so we abort. Not sure what the difference between the two conditions
|
||||
// is, but it seemed that output.isFinished() is not enough to cancel on
|
||||
// Ctrl+C. Test manually if you change it.
|
||||
input.close();
|
||||
return Status::Finished;
|
||||
}
|
||||
|
@ -80,8 +80,10 @@ struct RowNumber
|
||||
* the order of input data. This property also trivially holds for the ROWS and
|
||||
* GROUPS frames. For the RANGE frame, the proof requires the additional fact
|
||||
* that the ranges are specified in terms of (the single) ORDER BY column.
|
||||
*
|
||||
* `final` is so that the isCancelled() is devirtualized, we call it every row.
|
||||
*/
|
||||
class WindowTransform : public IProcessor /* public ISimpleTransform */
|
||||
class WindowTransform final : public IProcessor
|
||||
{
|
||||
public:
|
||||
WindowTransform(
|
||||
|
@ -7,14 +7,8 @@ PEERDIR(
|
||||
clickhouse/src/Common
|
||||
contrib/libs/msgpack
|
||||
contrib/libs/protobuf
|
||||
contrib/libs/arrow
|
||||
)
|
||||
|
||||
ADDINCL(
|
||||
contrib/libs/arrow/src
|
||||
)
|
||||
|
||||
CFLAGS(-DUSE_ARROW=1)
|
||||
|
||||
SRCS(
|
||||
Chunk.cpp
|
||||
@ -31,11 +25,6 @@ SRCS(
|
||||
Formats/IOutputFormat.cpp
|
||||
Formats/IRowInputFormat.cpp
|
||||
Formats/IRowOutputFormat.cpp
|
||||
Formats/Impl/ArrowBlockInputFormat.cpp
|
||||
Formats/Impl/ArrowBlockOutputFormat.cpp
|
||||
Formats/Impl/ArrowBufferedStreams.cpp
|
||||
Formats/Impl/ArrowColumnToCHColumn.cpp
|
||||
Formats/Impl/CHColumnToArrowColumn.cpp
|
||||
Formats/Impl/BinaryRowInputFormat.cpp
|
||||
Formats/Impl/BinaryRowOutputFormat.cpp
|
||||
Formats/Impl/CSVRowInputFormat.cpp
|
||||
|
@ -141,6 +141,7 @@ SRCS(
|
||||
StorageMerge.cpp
|
||||
StorageMergeTree.cpp
|
||||
StorageMongoDB.cpp
|
||||
StorageMongoDBSocketFactory.cpp
|
||||
StorageMySQL.cpp
|
||||
StorageNull.cpp
|
||||
StorageReplicatedMergeTree.cpp
|
||||
|
@ -647,7 +647,7 @@ def run_tests_array(all_tests_with_params):
|
||||
failures_chain += 1
|
||||
status += MSG_FAIL
|
||||
status += print_test_time(total_time)
|
||||
status += " - having exception:\n{}\n".format(
|
||||
status += " - having exception in stdout:\n{}\n".format(
|
||||
'\n'.join(stdout.split('\n')[:100]))
|
||||
status += 'Database: ' + testcase_args.testcase_database
|
||||
elif reference_file is None:
|
||||
|
@ -91,6 +91,14 @@ tag4 [0,1,2,3,4,5,6,7,8,9] [5,999,2] [2,888,20] [0,1,3,4,6,7,8,9,20]
|
||||
[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]
|
||||
[30,31,32,33,100,200,500]
|
||||
[100,200,500]
|
||||
[]
|
||||
[]
|
||||
[1,5,7,9]
|
||||
[5,7,9]
|
||||
[5,7]
|
||||
[0,1,2,3,4,5,6,7,8,9]
|
||||
[30,31,32,33,100,200,500]
|
||||
[100,200,500]
|
||||
0
|
||||
0
|
||||
0
|
||||
|
@ -286,6 +286,25 @@ select bitmapToArray(bitmapSubsetLimit(bitmapBuild([
|
||||
0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,
|
||||
100,200,500]), toUInt32(100), toUInt16(200)));
|
||||
|
||||
-- subBitmap:
|
||||
---- Empty
|
||||
SELECT bitmapToArray(subBitmap(bitmapBuild(emptyArrayUInt32()), toUInt8(0), toUInt32(10)));
|
||||
SELECT bitmapToArray(subBitmap(bitmapBuild(emptyArrayUInt16()), toUInt32(0), toUInt64(10)));
|
||||
---- Small
|
||||
select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt8(0), toUInt32(4)));
|
||||
select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt32(1), toUInt64(4)));
|
||||
select bitmapToArray(subBitmap(bitmapBuild([1,5,7,9]), toUInt16(1), toUInt32(2)));
|
||||
---- Large
|
||||
select bitmapToArray(subBitmap(bitmapBuild([
|
||||
0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,
|
||||
100,200,500]), toUInt32(0), toUInt32(10)));
|
||||
select bitmapToArray(subBitmap(bitmapBuild([
|
||||
0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,
|
||||
100,200,500]), toUInt32(30), toUInt32(200)));
|
||||
select bitmapToArray(subBitmap(bitmapBuild([
|
||||
0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,
|
||||
100,200,500]), toUInt32(34), toUInt16(3)));
|
||||
|
||||
-- bitmapMin:
|
||||
---- Empty
|
||||
SELECT bitmapMin(bitmapBuild(emptyArrayUInt8()));
|
||||
|
@ -0,0 +1,3 @@
|
||||
Started
|
||||
Sent kill request
|
||||
Exit 138
|
36
tests/queries/0_stateless/01572_kill_window_function.sh
Executable file
36
tests/queries/0_stateless/01572_kill_window_function.sh
Executable file
@ -0,0 +1,36 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
set -e -o pipefail
|
||||
|
||||
# Run a test query that takes very long to run.
|
||||
query_id="01572_kill_window_function-$CLICKHOUSE_DATABASE"
|
||||
$CLICKHOUSE_CLIENT --query_id="$query_id" --query "SELECT count(1048575) OVER (PARTITION BY intDiv(NULL, number) ORDER BY number DESC NULLS FIRST ROWS BETWEEN CURRENT ROW AND 1048575 FOLLOWING) FROM numbers(255, 1048575)" >/dev/null 2>&1 &
|
||||
client_pid=$!
|
||||
echo Started
|
||||
|
||||
# Use one query to both kill the test query and verify that it has started,
|
||||
# because if we try to kill it before it starts, the test will fail.
|
||||
while [ -z "$($CLICKHOUSE_CLIENT --query "kill query where query_id = '$query_id' and current_database = currentDatabase()")" ]
|
||||
do
|
||||
# If we don't yet see the query in the process list, the client should still
|
||||
# be running. The query is very long.
|
||||
kill -0 -- $client_pid
|
||||
sleep 1
|
||||
done
|
||||
echo Sent kill request
|
||||
|
||||
# Wait for the client to terminate.
|
||||
client_exit_code=0
|
||||
wait $client_pid || client_exit_code=$?
|
||||
|
||||
echo "Exit $client_exit_code"
|
||||
|
||||
# We have tested for Ctrl+C.
|
||||
# The following client flags don't cancel, but should: --max_execution_time,
|
||||
# --receive_timeout. Probably needs asynchonous calculation of query limits, as
|
||||
# discussed with Nikolay on TG: https://t.me/c/1214350934/21492
|
||||
|
40
tests/queries/0_stateless/02003_memory_limit_in_client.expect
Executable file
40
tests/queries/0_stateless/02003_memory_limit_in_client.expect
Executable file
@ -0,0 +1,40 @@
|
||||
#!/usr/bin/expect -f
|
||||
|
||||
# This is a test for system.warnings. Testing in interactive mode is necessary,
|
||||
# as we want to see certain warnings from client
|
||||
|
||||
log_user 0
|
||||
set timeout 60
|
||||
match_max 100000
|
||||
|
||||
# A default timeout action is to do nothing, change it to fail
|
||||
expect_after {
|
||||
timeout {
|
||||
exit 1
|
||||
}
|
||||
}
|
||||
|
||||
set basedir [file dirname $argv0]
|
||||
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1"
|
||||
expect ":) "
|
||||
|
||||
send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r"
|
||||
expect "Code: 241"
|
||||
|
||||
expect ":) "
|
||||
|
||||
# Exit.
|
||||
send -- "\4"
|
||||
expect eof
|
||||
|
||||
set basedir [file dirname $argv0]
|
||||
spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1"
|
||||
expect ":) "
|
||||
|
||||
send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r"
|
||||
expect "60000"
|
||||
expect ":) "
|
||||
|
||||
# Exit.
|
||||
send -- "\4"
|
||||
expect eof
|
8
tests/queries/0_stateless/02007_join_use_nulls.reference
Normal file
8
tests/queries/0_stateless/02007_join_use_nulls.reference
Normal file
@ -0,0 +1,8 @@
|
||||
1 2 3 1 3
|
||||
1 UInt8 2 UInt8 3 Nullable(UInt8)
|
||||
1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 3 LowCardinality(Nullable(UInt8))
|
||||
1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 1 LowCardinality(Nullable(UInt8))
|
||||
1 UInt8 2 UInt8 3 Nullable(UInt8)
|
||||
1 UInt8 2 UInt8 1 Nullable(UInt8) 3 Nullable(UInt8)
|
||||
1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 3 LowCardinality(Nullable(UInt8))
|
||||
1 LowCardinality(UInt8) 2 LowCardinality(UInt8) 1 LowCardinality(Nullable(UInt8)) 3 LowCardinality(Nullable(UInt8))
|
11
tests/queries/0_stateless/02007_join_use_nulls.sql
Normal file
11
tests/queries/0_stateless/02007_join_use_nulls.sql
Normal file
@ -0,0 +1,11 @@
|
||||
SET join_use_nulls = 1;
|
||||
|
||||
SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;
|
||||
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) , d.values, toTypeName(d.values) FROM ( SELECT 1 AS id, 2 AS value ) a SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id;
|
||||
SELECT id, toTypeName(id), value, toTypeName(value), d.id, toTypeName(d.id) , d.values, toTypeName(d.values) FROM ( SELECT toLowCardinality(1) AS id, toLowCardinality(2) AS value ) a SEMI LEFT JOIN ( SELECT toLowCardinality(1) AS id, toLowCardinality(3) AS values ) AS d USING id;
|
@ -2,19 +2,9 @@
|
||||
<div class="container text-center">
|
||||
<h2 id="success-stories" class="display-4">Success stories</h2>
|
||||
<div class="row">
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-lb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">HTTP and DNS analytics</h4>
|
||||
<p class="card-text text-muted">by Cloudflare</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
</div>
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="docs/en/introduction/history/" class="text-reset text-decoration-none">
|
||||
<div class="card py-3 dots-cc">
|
||||
<div class="card py-3 dots-lb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">Yandex Metrica</h4>
|
||||
<p class="card-text text-muted">The original usecase</p>
|
||||
@ -23,11 +13,21 @@
|
||||
</a>
|
||||
</div>
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<a href="https://eng.uber.com/logging/" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-cc">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">Log Platform</h4>
|
||||
<p class="card-text text-muted">at Uber</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
</div>
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://tech.ebayinc.com/engineering/ou-online-analytical-processing/" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-rb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">Experimentation</h4>
|
||||
<p class="card-text text-muted">at Spotify</p>
|
||||
<h4 class="card-title">Analytical Processing</h4>
|
||||
<p class="card-text text-muted">at eBay</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
@ -35,31 +35,31 @@
|
||||
</div>
|
||||
<div class="row">
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fwww.jianshu.com%2Fp%2F4c86a2478cca&lang=zh-en" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<a href="https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-rb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">OLAP contest victory</h4>
|
||||
<p class="card-text text-muted">by Analysys</p>
|
||||
<h4 class="card-title">HTTP and DNS analytics</h4>
|
||||
<p class="card-text text-muted">at Cloudflare</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
</div>
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://translate.yandex.com/translate?url=http%3A%2F%2Fsouslecapot.net%2F2018%2F11%2F21%2Fpatrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante%2F&lang=fr-en" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<a href="https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-rhb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">Digital experience analytics</h4>
|
||||
<p class="card-text text-muted">by ContentSquare</p>
|
||||
<h4 class="card-title">Experimentation</h4>
|
||||
<p class="card-text text-muted">at Spotify</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
</div>
|
||||
<div class="col-lg-4 py-3">
|
||||
<a href="https://translate.yandex.com/translate?url=https%3A%2F%2Ftech.geniee.co.jp%2Fentry%2F2017%2F07%2F20%2F160100" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<a href="https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf" class="text-reset text-decoration-none" rel="external nofollow noreferrer" target="_blank">
|
||||
<div class="card py-3 dots-lb">
|
||||
<div class="card-body">
|
||||
<h4 class="card-title">Speeding up Report API</h4>
|
||||
<p class="card-text text-muted">at Geniee</p>
|
||||
<h4 class="card-title">Business Intelligence</h4>
|
||||
<p class="card-text text-muted">at Deutsche Bank</p>
|
||||
</div>
|
||||
</div>
|
||||
</a>
|
||||
|
Loading…
Reference in New Issue
Block a user