Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2017-12-27 01:56:30 +03:00
commit 09262ae0d7
25 changed files with 225 additions and 129 deletions

2
contrib/librdkafka vendored

@ -1 +1 @@
Subproject commit 3401fa1e45605b5ae806f94905c92f5f546a607b
Subproject commit 3a986afbb977fa13582991ce8f2c0b2045ffaa33

2
contrib/zstd vendored

@ -1 +1 @@
Subproject commit f4340f46b2387bc8de7d5320c0b83bb1499933ad
Subproject commit f3a8bd553a865c59f1bd6e1f68bf182cf75a8f00

View File

@ -8,6 +8,7 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
/** This is simple, not numerically stable

View File

@ -1,5 +1,11 @@
#pragma once
#include <Common/PODArray.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <Core/Types.h>
#include <IO/VarInt.h>
namespace DB
{

View File

@ -1,6 +1,11 @@
#pragma once
#include <cmath>
#include <Common/RadixSort.h>
#include <Common/PODArray.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/VarInt.h>
namespace DB

View File

@ -2,6 +2,12 @@
#include <Common/HashTable/Hash.h>
#include <Common/MemoryTracker.h>
#include <Common/PODArray.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/likely.h>
namespace DB

View File

@ -76,7 +76,7 @@ protected:
}
template <typename ... TAllocatorParams>
void alloc(size_t bytes, TAllocatorParams ... allocator_params)
void alloc(size_t bytes, TAllocatorParams &&... allocator_params)
{
c_start = c_end = reinterpret_cast<char *>(TAllocator::alloc(bytes, std::forward<TAllocatorParams>(allocator_params)...));
c_end_of_storage = c_start + bytes - pad_right;
@ -91,7 +91,7 @@ protected:
}
template <typename ... TAllocatorParams>
void realloc(size_t bytes, TAllocatorParams ... allocator_params)
void realloc(size_t bytes, TAllocatorParams &&... allocator_params)
{
if (c_start == nullptr)
{
@ -119,10 +119,15 @@ protected:
}
template <typename ... TAllocatorParams>
void reserveForNextSize(TAllocatorParams ... allocator_params)
void reserveForNextSize(TAllocatorParams &&... allocator_params)
{
if (size() == 0)
realloc(std::max(INITIAL_SIZE, minimum_memory_for_elements(1)), std::forward<TAllocatorParams>(allocator_params)...);
{
// The allocated memory should be multiplication of sizeof(T) to hold the element, otherwise,
// memory issue such as corruption could appear in edge case.
realloc(std::max(((INITIAL_SIZE - 1) / sizeof(T) + 1) * sizeof(T), minimum_memory_for_elements(1)),
std::forward<TAllocatorParams>(allocator_params)...);
}
else
realloc(allocated_bytes() * 2, std::forward<TAllocatorParams>(allocator_params)...);
}
@ -207,14 +212,14 @@ public:
const_iterator cend() const { return t_end(); }
template <typename ... TAllocatorParams>
void reserve(size_t n, TAllocatorParams ... allocator_params)
void reserve(size_t n, TAllocatorParams &&... allocator_params)
{
if (n > capacity())
realloc(roundUpToPowerOfTwoOrZero(minimum_memory_for_elements(n)), std::forward<TAllocatorParams>(allocator_params)...);
}
template <typename ... TAllocatorParams>
void resize(size_t n, TAllocatorParams ... allocator_params)
void resize(size_t n, TAllocatorParams &&... allocator_params)
{
reserve(n, std::forward<TAllocatorParams>(allocator_params)...);
resize_assume_reserved(n);
@ -254,7 +259,7 @@ public:
}
template <typename ... TAllocatorParams>
void push_back(const T & x, TAllocatorParams ... allocator_params)
void push_back(const T & x, TAllocatorParams &&... allocator_params)
{
if (unlikely(c_end == c_end_of_storage))
reserveForNextSize(std::forward<TAllocatorParams>(allocator_params)...);
@ -283,7 +288,7 @@ public:
/// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated.
template <typename It1, typename It2, typename ... TAllocatorParams>
void insert(It1 from_begin, It2 from_end, TAllocatorParams ... allocator_params)
void insert(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params)
{
size_t required_capacity = size() + (from_end - from_begin);
if (required_capacity > capacity())

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/UInt128.h>
#include <common/strong_typedef.h>
namespace DB
{

View File

@ -1,12 +1,12 @@
#pragma once
#include <DataStreams/IRowInputStream.h>
#include <Core/Block.h>
namespace DB
{
class Block;
class ReadBuffer;

View File

@ -54,7 +54,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
if (dict_struct.id)
block.insert(ColumnWithTypeAndName{
ColumnUInt64::create(1), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
ColumnUInt64::create(1, 0), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
if (dict_struct.key)
{
@ -70,7 +70,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
if (dict_struct.range_min)
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
block.insert(ColumnWithTypeAndName{
ColumnUInt16::create(1), std::make_shared<DataTypeDate>(), attribute->name});
ColumnUInt16::create(1, 0), std::make_shared<DataTypeDate>(), attribute->name});
for (const auto & attribute : dict_struct.attributes)
{

View File

@ -8,6 +8,7 @@
#include <Dictionaries/DictionaryBlockInputStreamBase.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/RangeHashedDictionary.h>
#include <ext/range.h>
namespace DB
@ -57,6 +58,9 @@ private:
Block fillBlock(const PaddedPODArray<Key> & ids,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
PaddedPODArray<UInt16> makeDateKey(
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
DictionatyPtr dictionary;
Names column_names;
PaddedPODArray<Key> ids;
@ -88,8 +92,8 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
for (auto idx : ext::range(start, start + length))
{
block_ids.push_back(ids[idx]);
block_start_dates.push_back(block_start_dates[idx]);
block_end_dates.push_back(block_end_dates[idx]);
block_start_dates.push_back(start_dates[idx]);
block_end_dates.push_back(end_dates[idx]);
}
return fillBlock(block_ids, block_start_dates, block_end_dates);
@ -144,6 +148,23 @@ void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
}
}
template <typename DictionaryType, typename Key>
PaddedPODArray<UInt16> RangeDictionaryBlockInputStream<DictionaryType, Key>::makeDateKey(
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const
{
PaddedPODArray<UInt16> key(start_dates.size());
for (size_t i = 0; i < key.size(); ++i)
{
if (RangeHashedDictionary::Range::isCorrectDate(start_dates[i]))
key[i] = start_dates[i];
else
key[i] = end_dates[i];
}
return key;
}
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key>& ids,
@ -158,6 +179,8 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
addSpecialColumn(structure.range_min, std::make_shared<DataTypeDate>(), "Range Start", names, start_dates, columns);
addSpecialColumn(structure.range_max, std::make_shared<DataTypeDate>(), "Range End", names, end_dates, columns);
auto date_key = makeDateKey(start_dates, end_dates);
for (const auto idx : ext::range(0, structure.attributes.size()))
{
const DictionaryAttribute& attribute = structure.attributes[idx];
@ -165,7 +188,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
{
ColumnPtr column;
#define GET_COLUMN_FORM_ATTRIBUTE(TYPE)\
column = getColumnFromAttribute<TYPE>(&DictionaryType::get##TYPE, ids, start_dates, attribute, *dictionary)
column = getColumnFromAttribute<TYPE>(&DictionaryType::get##TYPE, ids, date_key, attribute, *dictionary)
switch (attribute.underlying_type)
{
case AttributeUnderlyingType::UInt8:
@ -202,7 +225,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
GET_COLUMN_FORM_ATTRIBUTE(Float64);
break;
case AttributeUnderlyingType::String:
column = getColumnFromAttributeString(ids, start_dates, attribute, *dictionary);
column = getColumnFromAttributeString(ids, date_key, attribute, *dictionary);
break;
}

View File

@ -391,10 +391,11 @@ void RangeHashedDictionary::getIdsAndDates(const Attribute& attribute, PaddedPOD
start_dates.reserve(attr.size());
end_dates.reserve(attr.size());
for (const auto & key : attr) {
ids.push_back(key.first);
for (const auto & key : attr)
{
for (const auto & value : key.second)
{
ids.push_back(key.first);
start_dates.push_back(value.range.first);
end_dates.push_back(value.range.second);
}

View File

@ -82,11 +82,12 @@ public:
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
struct Range : std::pair<UInt16, UInt16>
{
using std::pair<UInt16, UInt16>::pair;
static bool isCorrectDate(const UInt16 date) { return 0 < date && date <= DATE_LUT_MAX_DAY_NUM; }
bool contains(const UInt16 date) const
{
const auto & left = first;
@ -95,8 +96,8 @@ private:
if (left <= date && date <= right)
return true;
const auto has_left_bound = 0 < left && left <= DATE_LUT_MAX_DAY_NUM;
const auto has_right_bound = 0 < right && right <= DATE_LUT_MAX_DAY_NUM;
const auto has_left_bound = isCorrectDate(left);
const auto has_right_bound = isCorrectDate(right);
if ((!has_left_bound || left <= date) && (!has_right_bound || date <= right))
return true;
@ -105,6 +106,7 @@ private:
}
};
private:
template <typename T>
struct Value final
{

View File

@ -1,6 +1,7 @@
#pragma once
#include <thread>
#include <functional>
#include <common/MultiVersion.h>
#include <Poco/Event.h>

View File

@ -14,7 +14,8 @@ public:
#ifdef __SIZEOF_INT128__
using BigNum = __uint128_t; /// Must contain the result of multiplying two UInt64.
#else
#error "No uint128_t type. Sampling ratios cannot work correctly."
#warning "No uint128_t type. Sampling ratios cannot work correctly."
using BigNum = uint64_t;
#endif
struct Rational

View File

@ -1,5 +1,4 @@
#include <Storages/MergeTree/DataPartsExchange.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/CurrentMetrics.h>
#include <Common/NetException.h>
#include <Common/typeid_cast.h>
@ -129,12 +128,12 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ABORTED && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM)
dynamic_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
data.reportBrokenPart(part_name);
throw;
}
catch (...)
{
dynamic_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
data.reportBrokenPart(part_name);
throw;
}
}

View File

@ -1,6 +1,7 @@
#include <sstream>
#include <Parsers/ASTCreateQuery.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/CacheDictionary.h>
@ -87,9 +88,9 @@ NamesAndTypes StorageDictionary::getNamesAndTypes(const DictionaryStructure & di
if (dictionary_structure.id)
dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared<DataTypeUInt64>());
if (dictionary_structure.range_min)
dictionary_names_and_types.emplace_back(dictionary_structure.range_min->name, std::make_shared<DataTypeUInt16>());
dictionary_names_and_types.emplace_back(dictionary_structure.range_min->name, std::make_shared<DataTypeDate>());
if (dictionary_structure.range_max)
dictionary_names_and_types.emplace_back(dictionary_structure.range_max->name, std::make_shared<DataTypeUInt16>());
dictionary_names_and_types.emplace_back(dictionary_structure.range_max->name, std::make_shared<DataTypeDate>());
if (dictionary_structure.key)
for (const auto & attribute : *dictionary_structure.key)
dictionary_names_and_types.emplace_back(attribute.name, attribute.type);

View File

@ -153,107 +153,114 @@ def main(args):
report_testcase.append(et.Element("skipped", attrib = {"message": "no shard"}))
print(MSG_SKIPPED + " - no shard")
else:
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_dir, name) + '.stdout'
stderr_file = os.path.join(suite_dir, name) + '.stderr'
if ext == '.sql':
command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file)
disabled_file = os.path.join(suite_dir, name) + '.disabled'
if os.path.exists(disabled_file) and not args.disabled:
message = open(disabled_file, 'r').read()
report_testcase.append(et.Element("skipped", attrib = {"message": message}))
print(MSG_SKIPPED + " - " + message)
else:
command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file)
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_dir, name) + '.stdout'
stderr_file = os.path.join(suite_dir, name) + '.stderr'
proc = Popen(command, shell = True)
start_time = datetime.now()
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0)
if proc.returncode is None:
try:
proc.kill()
except OSError as e:
if e.errno != ESRCH:
raise
failure = et.Element("failure", attrib = {"message": "Timeout"})
report_testcase.append(failure)
failures = failures + 1
print("{0} - Timeout!".format(MSG_FAIL))
else:
stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else ''
stdout = unicode(stdout, errors='replace', encoding='utf-8')
stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else ''
stderr = unicode(stderr, errors='replace', encoding='utf-8')
if proc.returncode != 0:
failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures = failures + 1
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr:
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
print(stderr)
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True
elif stderr:
failure = et.Element("failure", attrib = {"message": "having stderror"})
report_testcase.append(failure)
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
failures = failures + 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout:
failure = et.Element("error", attrib = {"message": "having exception"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures = failures + 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file):
skipped = et.Element("skipped", attrib = {"message": "no reference file"})
report_testcase.append(skipped)
print("{0} - no reference file".format(MSG_UNKNOWN))
if ext == '.sql':
command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file)
else:
result_is_different = subprocess.call(['cmp', '-s', reference_file, stdout_file], stdout = PIPE)
command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file)
if result_is_different:
(diff, _) = Popen(['diff', '--side-by-side', reference_file, stdout_file], stdout = PIPE).communicate()
diff = unicode(diff, errors='replace', encoding='utf-8')
proc = Popen(command, shell = True)
start_time = datetime.now()
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0)
failure = et.Element("failure", attrib = {"message": "result differs with reference"})
if proc.returncode is None:
try:
proc.kill()
except OSError as e:
if e.errno != ESRCH:
raise
failure = et.Element("failure", attrib = {"message": "Timeout"})
report_testcase.append(failure)
failures = failures + 1
print("{0} - Timeout!".format(MSG_FAIL))
else:
stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else ''
stdout = unicode(stdout, errors='replace', encoding='utf-8')
stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else ''
stderr = unicode(stderr, errors='replace', encoding='utf-8')
if proc.returncode != 0:
failure = et.Element("failure", attrib = {"message": "return code {}".format(proc.returncode)})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
try:
stdout_element.text = et.CDATA(diff)
except:
stdout_element.text = et.CDATA(remove_control_characters(diff))
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures = failures + 1
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff.encode('utf-8')))
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr:
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
print(stderr)
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True
elif stderr:
failure = et.Element("failure", attrib = {"message": "having stderror"})
report_testcase.append(failure)
stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element)
failures = failures + 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout:
failure = et.Element("error", attrib = {"message": "having exception"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
stdout_element.text = et.CDATA(stdout)
report_testcase.append(stdout_element)
failures = failures + 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file):
skipped = et.Element("skipped", attrib = {"message": "no reference file"})
report_testcase.append(skipped)
print("{0} - no reference file".format(MSG_UNKNOWN))
else:
print(MSG_OK)
if os.path.exists(stdout_file):
os.remove(stdout_file)
if os.path.exists(stderr_file):
os.remove(stderr_file)
result_is_different = subprocess.call(['cmp', '-s', reference_file, stdout_file], stdout = PIPE)
if result_is_different:
(diff, _) = Popen(['diff', '--side-by-side', reference_file, stdout_file], stdout = PIPE).communicate()
diff = unicode(diff, errors='replace', encoding='utf-8')
failure = et.Element("failure", attrib = {"message": "result differs with reference"})
report_testcase.append(failure)
stdout_element = et.Element("system-out")
try:
stdout_element.text = et.CDATA(diff)
except:
stdout_element.text = et.CDATA(remove_control_characters(diff))
report_testcase.append(stdout_element)
failures = failures + 1
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff.encode('utf-8')))
else:
print(MSG_OK)
if os.path.exists(stdout_file):
os.remove(stdout_file)
if os.path.exists(stderr_file):
os.remove(stderr_file)
except KeyboardInterrupt as e:
print(colored("Break tests execution", "red"))
raise e
@ -286,6 +293,7 @@ if __name__ == '__main__':
parser.add_argument('-o', '--output', help = 'Output xUnit compliant test report directory')
parser.add_argument('-t', '--timeout', type = int, default = 600, help = 'Timeout for each test case in seconds')
parser.add_argument('test', nargs = '?', help = 'Optional test case name regex')
parser.add_argument('-d', '--disabled', action = 'store_true', default = False, help = 'Also run disabled tests')
parser.add_argument('--stop', action = 'store_true', default = None, dest = 'stop', help = 'Stop on network errors')
parser.add_argument('--order', default = 'desc', help = 'Run order (asc, desc, random)')
parser.add_argument('--testname', action = 'store_true', default = None, dest = 'testname', help = 'Make query with test name before test run')

View File

@ -1,4 +1,4 @@
0 0 0 0 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0
1 1 1 1 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0
2 2 2 2 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1
3 3 3 3 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2
0 0 0 0 0000-00-00 0000-00-00 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0
1 1 1 1 1970-01-02 0000-00-00 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0
2 2 2 2 0000-00-00 1970-01-02 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1
3 3 3 3 1990-01-03 1990-01-03 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2
1 0 0 0 0 0000-00-00 0000-00-00 50 13874 980694578 980694579 50 13874 980694578 980694579 0 0 4761183170873013810 2007-12-27 1970-01-02 06:51:14 0
2 1 1 1 1 1970-01-02 0000-00-00 48 57392 4083802160 4083802161 48 -8144 -211165136 -211165135 1.5 1.5 10577349846663553072 2037-06-02 1970-01-02 09:50:24 0
3 2 2 2 2 0000-00-00 1970-01-02 69 35909 1447922757 1447922758 69 -29627 1447922757 1447922758 3 3 18198135717204167749 1978-08-08 1970-01-02 03:52:21 1
4 3 3 3 3 1990-01-03 1990-01-03 250 1274 1029309690 1029309691 -6 1274 1029309690 1029309691 4.5 4.5 9624464864560415994 1973-06-28 1970-01-02 03:21:14 2

View File

@ -43,6 +43,9 @@ def generate_structure():
# Complex key dictionaries with (String, UInt8) key
['clickhouse_complex_mixed_key_hashed', 2, False],
['clickhouse_complex_mixed_key_cache', 2, False],
# Range hashed dictionary
['clickhouse_range_hashed', 3, False],
]
@ -102,6 +105,7 @@ def generate_dictionaries(path, structure):
layout_cache = '<cache><size_in_cells>128</size_in_cells></cache>'
layout_complex_key_hashed = '<complex_key_hashed />'
layout_complex_key_cache = '<complex_key_cache><size_in_cells>128</size_in_cells></complex_key_cache>'
layout_range_hashed = '<range_hashed />'
key_simple = '''
<id>
@ -135,7 +139,19 @@ def generate_dictionaries(path, structure):
</key>
'''
keys = [key_simple, key_complex_integers, key_complex_mixed]
key_range_hashed = '''
<id>
<name>id</name>
</id>
<range_min>
<name>StartDate</name>
</range_min>
<range_max>
<name>EndDate</name>
</range_max>
'''
keys = [key_simple, key_complex_integers, key_complex_mixed, key_range_hashed]
parent_attribute = '''
<attribute>
@ -158,6 +174,9 @@ def generate_dictionaries(path, structure):
# Complex key dictionaries with (String, UInt8) key
[source_clickhouse, layout_complex_key_hashed],
[source_clickhouse, layout_complex_key_cache],
# Range hashed dictionary
[source_clickhouse, layout_range_hashed],
]
file_names = []
@ -176,6 +195,7 @@ def generate_dictionaries(path, structure):
class DictionaryTestTable:
def __init__(self, source_file_name):
self.structure = '''id UInt64, key0 UInt8, key0_str String, key1 UInt8,
StartDate Date, EndDate Date,
UInt8_ UInt8, UInt16_ UInt16, UInt32_ UInt32, UInt64_ UInt64,
Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64,
Float32_ Float32, Float64_ Float64,
@ -183,8 +203,8 @@ class DictionaryTestTable:
Date_ Date, DateTime_ DateTime, Parent UInt64'''
self.names_and_types = map(str.split, self.structure.split(','))
self.keys_names_and_types = self.names_and_types[:4]
self.values_names_and_types = self.names_and_types[4:]
self.keys_names_and_types = self.names_and_types[:6]
self.values_names_and_types = self.names_and_types[6:]
self.source_file_name = source_file_name
self.rows = None
@ -213,6 +233,7 @@ class DictionaryTestTable:
return '(' + ','.join(map(wrap_value, zip(row, types))) + ')'
values = ','.join(map(make_tuple, lines))
print query % (self.structure, values)
instance.query(query % (self.structure, values))
def get_structure_for_keys(self, keys, enable_parent=True):

View File

@ -45,10 +45,12 @@ def started_cluster():
('clickhouse_flat', ('id',), True),
('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False),
('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False),
('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False),
],
ids=['clickhouse_hashed', 'clickhouse_flat',
'clickhouse_complex_integers_key_hashed',
'clickhouse_complex_mixed_key_hashed']
'clickhouse_complex_mixed_key_hashed',
'clickhouse_range_hashed']
)
def dictionary_structure(started_cluster, request):
return request.param

8
debian/.pbuilderrc vendored
View File

@ -52,8 +52,8 @@ UBUNTU_SUITES=("bionic" "artful" "zesty" "xenial" "trusty" "devel")
DEBIAN_MIRROR="deb.debian.org"
#UBUNTU_MIRROR="mirrors.kernel.org"
UBUNTU_MIRROR="mirror.yandex.ru"
#DEBIAN_MIRROR="mirror.yandex.ru"
UBUNTU_MIRROR="mirror.yandex.ru"
# Set a default distribution if none is used. Note that you can set your own default (i.e. ${DIST:="unstable"}).
: ${DIST:="$(lsb_release --short --codename)"}
@ -81,9 +81,11 @@ if [ -n "${ARCH}" ]; then
DEBOOTSTRAPOPTS=("--arch" "$ARCH" "${DEBOOTSTRAPOPTS[@]}")
fi
BASETGZ="/var/cache/pbuilder/$NAME-base.tgz"
BASETGZ=${SET_BASETGZ}
BASETGZ=${BASETGZ:="/var/cache/pbuilder/$NAME-base.tgz"}
DISTRIBUTION="$DIST"
BUILDRESULT="/var/cache/pbuilder/$NAME/result/"
BUILDRESULT=${SET_BUILDRESULT}
BUILDRESULT=${BUILDRESULT:="/var/cache/pbuilder/$NAME/result/"}
APTCACHE="/var/cache/pbuilder/$NAME/aptcache/"
BUILDPLACE="/var/cache/pbuilder/build/"

View File

@ -13,6 +13,12 @@ ReplicatedAggregatingMergeTree
ReplicatedSummingMergeTree
~~~~~~~~~~~~~~~~~~~~~~~~~~
ReplicatedReplacingMergeTree
~~~~~~~~~~~~~~~~~~~~~~~~~~~~
ReplicatedGraphiteMergeTree
~~~~~~~~~~~~~~~~~~~~~~~~~~~
Replication is only supported for tables in the MergeTree family. Replication works at the level of an individual table, not the entire server. A server can store both replicated and non-replicated tables at the same time.
INSERT and ALTER are replicated (for more information, see ALTER). Compressed data is replicated, not query texts.

View File

@ -10,6 +10,10 @@
## ReplicatedSummingMergeTree
## ReplicatedReplacingMergeTree
## ReplicatedGraphiteMergeTree
Репликация поддерживается только для таблиц семейства MergeTree. Репликация работает на уровне отдельных таблиц, а не всего сервера. То есть, на сервере могут быть расположены одновременно реплицируемые и не реплицируемые таблицы.
Реплицируются INSERT, ALTER (см. подробности в описании запроса ALTER). Реплицируются сжатые данные, а не тексты запросов.

View File

@ -3,6 +3,7 @@
#include <cstring>
#include <iostream>
#include <type_traits>
#include <vector>
#include <mysqlxx/Types.h>
#include <mysqlxx/Row.h>