mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge remote-tracking branch 'ck/master' into 20200409_bug_fix_mysql_handshake_scramble
This commit is contained in:
commit
691fe42586
@ -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
|
||||
|
4
docker/packager/freebsd/Vagrantfile
vendored
4
docker/packager/freebsd/Vagrantfile
vendored
@ -1,4 +0,0 @@
|
||||
Vagrant.configure("2") do |config|
|
||||
config.vm.box = "robot-clickhouse/clickhouse-freebsd"
|
||||
config.vm.synced_folder ".", "/vagrant", disabled: true
|
||||
end
|
@ -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))
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -0,0 +1 @@
|
||||
0
|
@ -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;
|
||||
|
@ -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 }
|
||||
|
Loading…
Reference in New Issue
Block a user