Merge branch 'master' into aavdonkin-master

This commit is contained in:
Alexey Milovidov 2017-12-27 21:50:01 +03:00
commit f5b1ad1fa2
23 changed files with 84 additions and 52 deletions

View File

@ -63,5 +63,6 @@ done
# Even more platform-specific headers
for src_file in $(ls -1 $SOURCE_PATH/contrib/libboost/boost_1_65_0/boost/smart_ptr/detail/*);
do
mkdir -p "$DST/$(echo $src_file | sed -r -e 's/\/[^/]*$/\//')";
cp "$src_file" "$DST/$src_file";
done

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54326-testing)
set(VERSION_REVISION 54326)
set(VERSION_DESCRIBE v1.1.54328-testing)
set(VERSION_REVISION 54328)
# end of autochange
set (VERSION_MAJOR 1)

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

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

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

View File

@ -1545,7 +1545,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
{
makeExplicitSet(func, sample_block, true);
}
catch (const DB::Exception & e)
catch (const Exception & e)
{
/// in `sample_block` there are no columns that add `getActions`
if (e.code() != ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK)
@ -1674,7 +1674,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
/** NOTE If tuple in left hand side specified non-explicitly
* Example: identity((a, b)) IN ((1, 2), (3, 4))
* instead of (a, b)) IN ((1, 2), (3, 4))
* then set creation of set doesn't work correctly.
* then set creation doesn't work correctly.
*/
if (left_arg_tuple && left_arg_tuple->name == "tuple")
{

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

@ -864,7 +864,7 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, co
break;
}
/// Compute the function.
/// Apply the function.
DataTypePtr new_type;
if (!key_range_transformed.left.isNull())
applyFunction(func, current_type, key_range_transformed.left, new_type, key_range_transformed.left);

View File

@ -87,12 +87,14 @@ def main(args):
args.shard = True
base_dir = os.path.abspath(args.queries)
tmp_dir = os.path.abspath(args.tmp)
failures_total = 0
# Keep same default values as in queries/0_stateless/00000_sh_lib.sh
os.environ.setdefault("CLICKHOUSE_BINARY", args.binary)
os.environ.setdefault("CLICKHOUSE_CLIENT", args.client)
os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir)
for suite in sorted(os.listdir(base_dir)):
if SERVER_DIED:
@ -102,6 +104,11 @@ def main(args):
suite_re_obj = re.search('^[0-9]+_(.*)$', suite)
if not suite_re_obj: #skip .gitignore and so on
continue
suite_tmp_dir = os.path.join(tmp_dir, suite)
if not os.path.exists(suite_tmp_dir):
os.makedirs(suite_tmp_dir)
suite = suite_re_obj.group(1)
if os.path.isdir(suite_dir):
print("\nRunning {} tests.\n".format(suite))
@ -154,15 +161,15 @@ def main(args):
print(MSG_SKIPPED + " - no shard")
else:
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:
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'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
if ext == '.sql':
command = "{0} --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file)
@ -287,9 +294,10 @@ def main(args):
if __name__ == '__main__':
parser = ArgumentParser(description = 'ClickHouse functional tests')
parser.add_argument('-q', '--queries', default = 'queries', help = 'Path to queries dir')
parser.add_argument('--tmp', default = 'queries', help = 'Path to tmp dir')
parser.add_argument('-b', '--binary', default = 'clickhouse', help = 'Main clickhouse binary')
parser.add_argument('-c', '--client', help = 'Client program')
parser.add_argument('--client_config', help = 'Client config (if you use not default ports)')
parser.add_argument('--clientconfig', help = 'Client config (if you use not default ports)')
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')
@ -308,6 +316,6 @@ if __name__ == '__main__':
args = parser.parse_args()
if args.client is None:
args.client = args.binary + '-client'
if args.client_config:
args.client += ' -c' + args.client_config
if args.clientconfig:
args.client += ' -c' + args.clientconfig
main(args)

View File

@ -8,7 +8,7 @@ TABLE_HASH="cityHash64(groupArray(cityHash64(*)))"
function pack_unpack_compare()
{
local buf_file="test.buf.'.$3"
local buf_file="${CLICKHOUSE_TMP}/test.buf.'.$3"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf_file"
@ -20,12 +20,12 @@ function pack_unpack_compare()
local res_db_file=$(${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT $TABLE_HASH FROM test.buf_file")
${CLICKHOUSE_CLIENT} --max_threads=1 --query "SELECT * FROM test.buf FORMAT $3" > "$buf_file"
local res_ch_local1=$(${CLICKHOUSE_LOCAL} --structure "$2" --file "$buf_file" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`" 2>stderr || cat stderr 1>&2)
local res_ch_local2=$(${CLICKHOUSE_LOCAL} --structure "$2" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`" < "$buf_file" 2>stderr || cat stderr 1>&2)
local res_ch_local1=$(${CLICKHOUSE_LOCAL} --structure "$2" --file "$buf_file" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`" 2>${CLICKHOUSE_TMP}/stderr || cat stderr 1>&2)
local res_ch_local2=$(${CLICKHOUSE_LOCAL} --structure "$2" --table "my super table" --input-format "$3" --output-format TabSeparated --query "SELECT $TABLE_HASH FROM \`my super table\`" < "$buf_file" 2>${CLICKHOUSE_TMP}/stderr || cat ${CLICKHOUSE_TMP}/stderr 1>&2)
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf"
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.buf_file"
rm -f "$buf_file" stderr
rm -f "${CLICKHOUSE_TMP}/$buf_file" stderr
echo $((res_orig - res_db_file)) $((res_orig - res_ch_local1)) $((res_orig - res_ch_local2))
}

View File

@ -11,29 +11,29 @@ function perform()
echo "performing test: $test_id"
${CLICKHOUSE_CLIENT} --query "$query" 2>/dev/null
if [ "$?" -eq 0 ]; then
cat "./test_into_outfile_$test_id.out"
cat "${CLICKHOUSE_TMP}/test_into_outfile_$test_id.out"
else
echo "query failed"
fi
rm -f "./test_into_outfile_$test_id.out"
rm -f "${CLICKHOUSE_TMP}/test_into_outfile_$test_id.out"
}
perform "select" "SELECT 1, 2, 3 INTO OUTFILE './test_into_outfile_select.out'"
perform "select" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_select.out'"
perform "union_all" "SELECT 1, 2 UNION ALL SELECT 3, 4 INTO OUTFILE './test_into_outfile_union_all.out' FORMAT TSV" | sort --numeric-sort
perform "union_all" "SELECT 1, 2 UNION ALL SELECT 3, 4 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_union_all.out' FORMAT TSV" | sort --numeric-sort
perform "bad_union_all" "SELECT 1, 2 INTO OUTFILE './test_into_outfile_bad_union_all.out' UNION ALL SELECT 3, 4"
perform "bad_union_all" "SELECT 1, 2 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_bad_union_all.out' UNION ALL SELECT 3, 4"
perform "describe_table" "DESCRIBE TABLE system.one INTO OUTFILE './test_into_outfile_describe_table.out'"
perform "describe_table" "DESCRIBE TABLE system.one INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_describe_table.out'"
echo "performing test: clickhouse-local"
echo -e '1\t2' | ${CLICKHOUSE_LOCAL} --structure 'col1 UInt32, col2 UInt32' --query "SELECT col1 + 1, col2 + 1 FROM table INTO OUTFILE './test_into_outfile_clickhouse-local.out'" 2>/dev/null
echo -e '1\t2' | ${CLICKHOUSE_LOCAL} --structure 'col1 UInt32, col2 UInt32' --query "SELECT col1 + 1, col2 + 1 FROM table INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_clickhouse-local.out'" 2>/dev/null
if [ "$?" -eq 0 ]; then
cat "./test_into_outfile_clickhouse-local.out"
cat "${CLICKHOUSE_TMP}/test_into_outfile_clickhouse-local.out"
else
echo "query failed"
fi
rm -f "./test_into_outfile_clickhouse-local.out"
rm -f "${CLICKHOUSE_TMP}/test_into_outfile_clickhouse-local.out"
echo "performing test: http"
echo "SELECT 1, 2 INTO OUTFILE './test_into_outfile_http.out'" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" -d @- --fail || echo "query failed"
echo "SELECT 1, 2 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_http.out'" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" -d @- --fail || echo "query failed"

View File

@ -66,11 +66,11 @@ corner_sizes="1048576 `seq 500000 1000000 3500000`"
# Check HTTP results with $CLICKHOUSE_CLIENT in normal case
function cmp_cli_and_http() {
$CLICKHOUSE_CLIENT -q "`query $1`" > res1
ch_url "buffer_size=$2&wait_end_of_query=0" "$1" > res2
ch_url "buffer_size=$2&wait_end_of_query=1" "$1" > res3
cmp res1 res2 && cmp res1 res3 || echo FAIL
rm -rf res1 res2 res3
$CLICKHOUSE_CLIENT -q "`query $1`" > ${CLICKHOUSE_TMP}/res1
ch_url "buffer_size=$2&wait_end_of_query=0" "$1" > ${CLICKHOUSE_TMP}/res2
ch_url "buffer_size=$2&wait_end_of_query=1" "$1" > ${CLICKHOUSE_TMP}/res3
cmp ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res2 && cmp ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res3 || echo FAIL
rm -rf ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res2 ${CLICKHOUSE_TMP}/res3
}
function check_cli_and_http() {
@ -88,14 +88,14 @@ check_cli_and_http
# Check HTTP internal compression in normal case
function cmp_http_compression() {
$CLICKHOUSE_CLIENT -q "`query $1`" > res0
ch_url 'compress=1' $1 | clickhouse-compressor --decompress > res1
ch_url "compress=1&buffer_size=$2&wait_end_of_query=0" $1 | clickhouse-compressor --decompress > res2
ch_url "compress=1&buffer_size=$2&wait_end_of_query=1" $1 | clickhouse-compressor --decompress > res3
cmp res0 res1
cmp res1 res2
cmp res1 res3
rm -rf res0 res1 res2 res3
$CLICKHOUSE_CLIENT -q "`query $1`" > ${CLICKHOUSE_TMP}/res0
ch_url 'compress=1' $1 | clickhouse-compressor --decompress > ${CLICKHOUSE_TMP}/res1
ch_url "compress=1&buffer_size=$2&wait_end_of_query=0" $1 | clickhouse-compressor --decompress > ${CLICKHOUSE_TMP}/res2
ch_url "compress=1&buffer_size=$2&wait_end_of_query=1" $1 | clickhouse-compressor --decompress > ${CLICKHOUSE_TMP}/res3
cmp ${CLICKHOUSE_TMP}/res0 ${CLICKHOUSE_TMP}/res1
cmp ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res2
cmp ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res3
rm -rf ${CLICKHOUSE_TMP}/res0 ${CLICKHOUSE_TMP}/res1 ${CLICKHOUSE_TMP}/res2 ${CLICKHOUSE_TMP}/res3
}
function check_http_compression() {

View File

@ -41,10 +41,10 @@ popd > /dev/null
#SCRIPTDIR=`dirname "$SCRIPTPATH"`
SCRIPTDIR=$SCRIPTPATH
cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 --merge_tree_uniform_read_distribution=1 -n 2>&1 > preferred_block_size_bytes.stdout
cmp "$SCRIPTDIR"/00282_merging.reference preferred_block_size_bytes.stdout && echo PASSED || echo FAILED
cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 --merge_tree_uniform_read_distribution=1 -n 2>&1 > ${CLICKHOUSE_TMP}/preferred_block_size_bytes.stdout
cmp "$SCRIPTDIR"/00282_merging.reference ${CLICKHOUSE_TMP}/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED
cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 --merge_tree_uniform_read_distribution=0 -n 2>&1 > preferred_block_size_bytes.stdout
cmp "$SCRIPTDIR"/00282_merging.reference preferred_block_size_bytes.stdout && echo PASSED || echo FAILED
cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 --merge_tree_uniform_read_distribution=0 -n 2>&1 > ${CLICKHOUSE_TMP}/preferred_block_size_bytes.stdout
cmp "$SCRIPTDIR"/00282_merging.reference ${CLICKHOUSE_TMP}/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED
rm preferred_block_size_bytes.stdout
rm ${CLICKHOUSE_TMP}/preferred_block_size_bytes.stdout

View File

@ -18,3 +18,5 @@ export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:="8443"}
export CLICKHOUSE_PORT_HTTP_PROTO=${CLICKHOUSE_PORT_HTTP_PROTO:="http"}
export CLICKHOUSE_URL=${CLICKHOUSE_URL:="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/"}
export CLICKHOUSE_CURL=${CLICKHOUSE_CURL:="curl --max-time 5"}
export CLICKHOUSE_TMP=${CLICKHOUSE_TMP:="."}
mkdir -p ${CLICKHOUSE_TMP}

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54326) unstable; urgency=low
clickhouse (1.1.54328) unstable; urgency=low
* Modified source code
-- proller <proller@yandex-team.ru> Mon, 18 Dec 2017 16:14:46 +0300
-- <robot-metrika-test@yandex-team.ru> Wed, 27 Dec 2017 11:09:36 +0300

View File

@ -132,7 +132,7 @@ ZooKeeper
Probably you already have ZooKeeper for other purposes.
It's Ok to use existing ZooKeeper installation if it is not overloaded.
Use recent version of ZooKeeper. At least 3.5 is Ok. Version in your Linux package repository might be outdated.
Use recent version of ZooKeeper. At least 3.4.9 is Ok. Version in your Linux package repository might be outdated.
With default settings, ZooKeeper have time bomb:

View File

@ -105,7 +105,7 @@ XFS также подходит, но не так тщательно проте
Вероятно вы уже используете ZooKeeper для других целей. Можно использовать ту же инсталляцию ZooKeeper, если она не сильно перегружена.
Лучше использовать свежую версию ZooKeeper, как минимум 3.5. Версия в стабильных дистрибутивах Linux может быть устаревшей.
Лучше использовать свежую версию ZooKeeper, как минимум 3.4.9. Версия в стабильных дистрибутивах Linux может быть устаревшей.
С настройками по умолчанию, ZooKeeper является бомбой замедленного действия:

View File

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