Merge remote-tracking branch 'ck/master' into 20200409_bug_fix_mysql_handshake_scramble

This commit is contained in:
BohuTANG 2020-04-09 10:59:04 +08:00
commit 691fe42586
8 changed files with 36 additions and 77 deletions

View File

@ -3,10 +3,10 @@ compilers and build settings. Correctly configured Docker daemon is single depen
Usage:
Build deb package with `gcc-8` in `debug` mode:
Build deb package with `gcc-9` in `debug` mode:
```
$ mkdir deb/test_output
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=gcc-8 --build-type=debug
$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=gcc-9 --build-type=debug
$ ls -l deb/test_output
-rw-r--r-- 1 root root 3730 clickhouse-client_18.14.2+debug_all.deb
-rw-r--r-- 1 root root 84221888 clickhouse-common-static_18.14.2+debug_amd64.deb
@ -18,11 +18,11 @@ $ ls -l deb/test_output
```
Build ClickHouse binary with `clang-6.0` and `address` sanitizer in `relwithdebuginfo`
Build ClickHouse binary with `clang-9.0` and `address` sanitizer in `relwithdebuginfo`
mode:
```
$ mkdir $HOME/some_clickhouse
$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-6.0 --sanitizer=address
$ ./packager --output-dir=$HOME/some_clickhouse --package-type binary --compiler=clang-9.0 --sanitizer=address
$ ls -l $HOME/some_clickhouse
-rwxr-xr-x 1 root root 787061952 clickhouse
lrwxrwxrwx 1 root root 10 clickhouse-benchmark -> clickhouse

View File

@ -1,4 +0,0 @@
Vagrant.configure("2") do |config|
config.vm.box = "robot-clickhouse/clickhouse-freebsd"
config.vm.synced_folder ".", "/vagrant", disabled: true
end

View File

@ -11,48 +11,8 @@ SCRIPT_PATH = os.path.realpath(__file__)
IMAGE_MAP = {
"deb": "yandex/clickhouse-deb-builder",
"binary": "yandex/clickhouse-binary-builder",
"freebsd": os.path.join(os.path.dirname(SCRIPT_PATH), "freebsd"),
}
class Vagrant(object):
def __init__(self, path_to_vagrant_file):
self.prefix = "VAGRANT_CWD=" + path_to_vagrant_file
def __enter__(self):
subprocess.check_call("{} vagrant up".format(self.prefix), shell=True)
self.ssh_path = "/tmp/vagrant-ssh"
subprocess.check_call("{} vagrant ssh-config > {}".format(self.prefix, self.ssh_path), shell=True)
return self
def copy_to_image(self, local_path, remote_path):
cmd = "scp -F {ssh} -r {lpath} default:{rpath}".format(ssh=self.ssh_path, lpath=local_path, rpath=remote_path)
logging.info("Copying to image %s", cmd)
subprocess.check_call(
cmd,
shell=True
)
def copy_from_image(self, remote_path, local_path):
cmd = "scp -F {ssh} -r default:{rpath} {lpath}".format(ssh=self.ssh_path, rpath=remote_path, lpath=local_path)
logging.info("Copying from image %s", cmd)
subprocess.check_call(
cmd,
shell=True
)
def execute_cmd(self, cmd):
cmd = '{} vagrant ssh -c "{}"'.format(self.prefix, cmd)
logging.info("Executin cmd %s", cmd)
subprocess.check_call(
cmd,
shell=True
)
def __exit__(self, exc_type, exc_val, exc_tb):
logging.info("Destroying image")
subprocess.check_call("{} vagrant destroy --force".format(self.prefix), shell=True)
def check_image_exists_locally(image_name):
try:
output = subprocess.check_output("docker images -q {} 2> /dev/null".format(image_name), shell=True)
@ -94,15 +54,6 @@ def run_docker_image_with_env(image_name, output, env_variables, ch_root, ccache
subprocess.check_call(cmd, shell=True)
def run_vagrant_box_with_env(image_path, output_dir, ch_root):
with Vagrant(image_path) as vagrant:
logging.info("Copying folder to vagrant machine")
vagrant.copy_to_image(ch_root, "~/ClickHouse")
logging.info("Running build")
vagrant.execute_cmd("cd ~/ClickHouse && cmake . && ninja")
logging.info("Copying binary back")
vagrant.copy_from_image("~/ClickHouse/programs/clickhouse", output_dir)
def parse_env_variables(build_type, compiler, sanitizer, package_type, image_type, cache, distcc_hosts, unbundled, split_binary, clang_tidy, version, author, official, alien_pkgs, with_coverage):
CLANG_PREFIX = "clang"
DARWIN_SUFFIX = "-darwin"
@ -210,7 +161,7 @@ if __name__ == "__main__":
logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s')
parser = argparse.ArgumentParser(description="ClickHouse building script using prebuilt Docker image")
# 'performance' creates a combined .tgz with server and configs to be used for performance test.
parser.add_argument("--package-type", choices=['deb', 'binary', 'performance', 'freebsd'], required=True)
parser.add_argument("--package-type", choices=['deb', 'binary', 'performance'], required=True)
parser.add_argument("--clickhouse-repo-path", default="../../")
parser.add_argument("--output-dir", required=True)
parser.add_argument("--build-type", choices=("debug", ""), default="")
@ -252,9 +203,5 @@ if __name__ == "__main__":
args.build_type, args.compiler, args.sanitizer, args.package_type, image_type,
args.cache, args.distcc_hosts, args.unbundled, args.split_binary, args.clang_tidy,
args.version, args.author, args.official, args.alien_pkgs, args.with_coverage)
if image_type != "freebsd":
run_docker_image_with_env(image_name, args.output_dir, env_prepared, ch_root, args.ccache_dir)
else:
logging.info("Running freebsd build, arguments will be ignored")
run_vagrant_box_with_env(image_name, args.output_dir, ch_root)
run_docker_image_with_env(image_name, args.output_dir, env_prepared, ch_root, args.ccache_dir)
logging.info("Output placed into {}".format(args.output_dir))

View File

@ -12,6 +12,8 @@
#include <IO/WriteBufferFromString.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <DataStreams/RemoteBlockOutputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/createBlockSelector.h>
#include <Interpreters/ExpressionActions.h>
@ -59,6 +61,26 @@ namespace ErrorCodes
extern const int CANNOT_LINK;
}
static void writeBlockConvert(const Context & context, const BlockOutputStreamPtr & out, const Block & block, const size_t repeats)
{
if (!blocksHaveEqualStructure(out->getHeader(), block))
{
ConvertingBlockInputStream convert(context,
std::make_shared<OneBlockInputStream>(block),
out->getHeader(),
ConvertingBlockInputStream::MatchColumnsMode::Name);
auto adopted_block = convert.read();
for (size_t i = 0; i < repeats; ++i)
out->write(adopted_block);
}
else
{
for (size_t i = 0; i < repeats; ++i)
out->write(block);
}
}
DistributedBlockOutputStream::DistributedBlockOutputStream(
const Context & context_, StorageDistributed & storage_, const ASTPtr & query_ast_, const ClusterPtr & cluster_,
@ -306,14 +328,12 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
InterpreterInsertQuery interp(query_ast, *job.local_context);
auto block_io = interp.execute();
assertBlocksHaveEqualStructure(block_io.out->getHeader(), shard_block, "flushing shard block for " + storage.getStorageID().getNameForLogs());
job.stream = block_io.out;
job.stream->writePrefix();
}
size_t num_repetitions = shard_info.getLocalNodeCount();
for (size_t i = 0; i < num_repetitions; ++i)
job.stream->write(shard_block);
writeBlockConvert(context, job.stream, shard_block, shard_info.getLocalNodeCount());
}
job.blocks_written += 1;
@ -547,13 +567,8 @@ void DistributedBlockOutputStream::writeToLocal(const Block & block, const size_
auto block_io = interp.execute();
assertBlocksHaveEqualStructure(block_io.out->getHeader(), block, "flushing " + storage.getStorageID().getNameForLogs());
block_io.out->writePrefix();
for (size_t i = 0; i < repeats; ++i)
block_io.out->write(block);
writeBlockConvert(context, block_io.out, block, repeats);
block_io.out->writeSuffix();
}

View File

@ -147,11 +147,11 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
/// That is, do not insert the same data to the same partition twice.
block_id = part->info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
LOG_DEBUG(log, "Wrote block with ID '" << block_id << "', " << block.rows() << " rows");
LOG_DEBUG(log, "Wrote block with ID '" << block_id << "', " << current_block.block.rows() << " rows");
}
else
{
LOG_DEBUG(log, "Wrote block with " << block.rows() << " rows");
LOG_DEBUG(log, "Wrote block with " << current_block.block.rows() << " rows");
}
try

View File

@ -4,6 +4,6 @@ DROP TABLE IF EXISTS underlying_00967;
CREATE TABLE dist_00967 (key UInt64) Engine=Distributed('test_shard_localhost', currentDatabase(), underlying_00967);
-- fails for TinyLog()/MergeTree()/... but not for Memory()
CREATE TABLE underlying_00967 (key Nullable(UInt64)) Engine=TinyLog();
INSERT INTO dist_00967 SELECT toUInt64(number) FROM system.numbers LIMIT 1; -- { serverError 171; }
INSERT INTO dist_00967 SELECT toUInt64(number) FROM system.numbers LIMIT 1;
SELECT * FROM dist_00967;

View File

@ -1 +1 @@
CREATE DATABASE conv_main ENGINE = MySQL('127.0.0.1:3456', conv_main, 'metrika', 'password'); -- { serverError 1000 }
CREATE DATABASE conv_main ENGINE = MySQL('127.0.0.1:3456', conv_main, 'metrika', 'password'); -- { serverError 501 }