mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #32745 from ClickHouse/fix_special_build_check
Better control build artifacts
This commit is contained in:
commit
d962d92896
@ -268,7 +268,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version);
|
|||||||
*
|
*
|
||||||
* Whether iconv support is available
|
* Whether iconv support is available
|
||||||
*/
|
*/
|
||||||
#if 1
|
#if 0
|
||||||
#define LIBXML_ICONV_ENABLED
|
#define LIBXML_ICONV_ENABLED
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
17
src/Common/getRandomASCIIString.cpp
Normal file
17
src/Common/getRandomASCIIString.cpp
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
#include <Common/thread_local_rng.h>
|
||||||
|
#include <random>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
String getRandomASCIIString(size_t len, char first, char last)
|
||||||
|
{
|
||||||
|
std::uniform_int_distribution<int> distribution(first, last);
|
||||||
|
String res(len, ' ');
|
||||||
|
for (auto & c : res)
|
||||||
|
c = distribution(thread_local_rng);
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
10
src/Common/getRandomASCIIString.h
Normal file
10
src/Common/getRandomASCIIString.h
Normal file
@ -0,0 +1,10 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <Core/Types.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
/// Slow random string. Useful for random names and things like this. Not for
|
||||||
|
/// generating data.
|
||||||
|
String getRandomASCIIString(size_t len = 32, char first = 'a', char last = 'z');
|
||||||
|
|
||||||
|
}
|
@ -7,6 +7,7 @@
|
|||||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -93,7 +94,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskBlobStorage::writeFile(
|
|||||||
WriteMode mode)
|
WriteMode mode)
|
||||||
{
|
{
|
||||||
auto metadata = readOrCreateMetaForWriting(path, mode);
|
auto metadata = readOrCreateMetaForWriting(path, mode);
|
||||||
auto blob_path = path + "_" + getRandomName(8); /// NOTE: path contains the tmp_* prefix in the blob name
|
auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name
|
||||||
|
|
||||||
LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}",
|
LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}",
|
||||||
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path);
|
mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path);
|
||||||
|
@ -1,4 +1,5 @@
|
|||||||
#include <Disks/RemoteDisksCommon.h>
|
#include <Disks/RemoteDisksCommon.h>
|
||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -8,17 +9,6 @@ namespace ErrorCodes
|
|||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
String getRandomName(size_t len, char first, char last)
|
|
||||||
{
|
|
||||||
std::uniform_int_distribution<int> distribution(first, last);
|
|
||||||
String res(len, ' ');
|
|
||||||
for (auto & c : res)
|
|
||||||
c = distribution(thread_local_rng);
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
||||||
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path)
|
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path)
|
||||||
{
|
{
|
||||||
|
@ -6,13 +6,12 @@
|
|||||||
#include <Common/thread_local_rng.h>
|
#include <Common/thread_local_rng.h>
|
||||||
#include <Disks/IDisk.h>
|
#include <Disks/IDisk.h>
|
||||||
#include <Disks/DiskCacheWrapper.h>
|
#include <Disks/DiskCacheWrapper.h>
|
||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
String getRandomName(size_t len = 32, char first = 'a', char last = 'z');
|
|
||||||
|
|
||||||
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
||||||
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path);
|
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path);
|
||||||
|
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
#include <Common/createHardLink.h>
|
#include <Common/createHardLink.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
#include <Common/thread_local_rng.h>
|
#include <Common/thread_local_rng.h>
|
||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <IO/ReadBufferFromS3.h>
|
#include <IO/ReadBufferFromS3.h>
|
||||||
@ -246,7 +247,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
|||||||
auto metadata = readOrCreateMetaForWriting(path, mode);
|
auto metadata = readOrCreateMetaForWriting(path, mode);
|
||||||
|
|
||||||
/// Path to store new S3 object.
|
/// Path to store new S3 object.
|
||||||
auto s3_path = getRandomName();
|
auto s3_path = getRandomASCIIString();
|
||||||
|
|
||||||
std::optional<ObjectMetadata> object_metadata;
|
std::optional<ObjectMetadata> object_metadata;
|
||||||
if (settings->send_metadata)
|
if (settings->send_metadata)
|
||||||
|
@ -2307,10 +2307,9 @@ namespace
|
|||||||
if (parent_field_descriptor)
|
if (parent_field_descriptor)
|
||||||
out << " field " << quoteString(parent_field_descriptor->full_name()) << " (" << parent_field_descriptor->type_name() << ")";
|
out << " field " << quoteString(parent_field_descriptor->full_name()) << " (" << parent_field_descriptor->type_name() << ")";
|
||||||
|
|
||||||
for (size_t i = 0; i != field_infos.size(); ++i)
|
for (const auto & field_info : field_infos)
|
||||||
{
|
{
|
||||||
out << "\n";
|
out << "\n";
|
||||||
const auto & field_info = field_infos[i];
|
|
||||||
writeIndent(out, indent + 1) << "Columns #";
|
writeIndent(out, indent + 1) << "Columns #";
|
||||||
for (size_t j = 0; j != field_info.column_indices.size(); ++j)
|
for (size_t j = 0; j != field_info.column_indices.size(); ++j)
|
||||||
{
|
{
|
||||||
@ -3017,8 +3016,11 @@ namespace
|
|||||||
if (nested_message_serializer)
|
if (nested_message_serializer)
|
||||||
{
|
{
|
||||||
std::vector<std::string_view> column_names_used;
|
std::vector<std::string_view> column_names_used;
|
||||||
|
column_names_used.reserve(used_column_indices_in_nested.size());
|
||||||
|
|
||||||
for (size_t i : used_column_indices_in_nested)
|
for (size_t i : used_column_indices_in_nested)
|
||||||
column_names_used.emplace_back(nested_column_names[i]);
|
column_names_used.emplace_back(nested_column_names[i]);
|
||||||
|
|
||||||
auto field_serializer = std::make_unique<ProtobufSerializerFlattenedNestedAsArrayOfNestedMessages>(
|
auto field_serializer = std::make_unique<ProtobufSerializerFlattenedNestedAsArrayOfNestedMessages>(
|
||||||
std::move(column_names_used), field_descriptor, std::move(nested_message_serializer), get_root_desc_function);
|
std::move(column_names_used), field_descriptor, std::move(nested_message_serializer), get_root_desc_function);
|
||||||
transformColumnIndices(used_column_indices_in_nested, nested_column_indices);
|
transformColumnIndices(used_column_indices_in_nested, nested_column_indices);
|
||||||
|
@ -68,7 +68,7 @@ bool ReadBufferFromBlobStorage::nextImpl()
|
|||||||
data_capacity = internal_buffer.size();
|
data_capacity = internal_buffer.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t to_read_bytes = std::min(total_size - offset, data_capacity);
|
size_t to_read_bytes = std::min(static_cast<size_t>(total_size - offset), data_capacity);
|
||||||
size_t bytes_read = 0;
|
size_t bytes_read = 0;
|
||||||
|
|
||||||
size_t sleep_time_with_backoff_milliseconds = 100;
|
size_t sleep_time_with_backoff_milliseconds = 100;
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
#include <IO/WriteBufferFromBlobStorage.h>
|
#include <IO/WriteBufferFromBlobStorage.h>
|
||||||
#include <Disks/RemoteDisksCommon.h>
|
#include <Disks/RemoteDisksCommon.h>
|
||||||
|
#include <Common/getRandomASCIIString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -42,7 +43,7 @@ void WriteBufferFromBlobStorage::nextImpl()
|
|||||||
{
|
{
|
||||||
auto part_len = std::min(len - read, max_single_part_upload_size);
|
auto part_len = std::min(len - read, max_single_part_upload_size);
|
||||||
|
|
||||||
auto block_id = getRandomName(64);
|
auto block_id = getRandomASCIIString(64);
|
||||||
block_ids.push_back(block_id);
|
block_ids.push_back(block_id);
|
||||||
|
|
||||||
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + read), part_len);
|
Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast<uint8_t *>(buffer_begin + read), part_len);
|
||||||
|
@ -440,7 +440,7 @@ static ASTPtr tryGetTableOverride(const String & mapped_database, const String &
|
|||||||
if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database))
|
if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database))
|
||||||
{
|
{
|
||||||
auto create_query = database_ptr->getCreateDatabaseQuery();
|
auto create_query = database_ptr->getCreateDatabaseQuery();
|
||||||
if (auto create_database_query = create_query->as<ASTCreateQuery>())
|
if (auto * create_database_query = create_query->as<ASTCreateQuery>())
|
||||||
{
|
{
|
||||||
if (create_database_query->table_overrides)
|
if (create_database_query->table_overrides)
|
||||||
{
|
{
|
||||||
@ -537,8 +537,8 @@ ASTs InterpreterCreateImpl::getRewrittenQueries(
|
|||||||
|
|
||||||
if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table))
|
if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table))
|
||||||
{
|
{
|
||||||
auto override = table_override->as<ASTTableOverride>();
|
auto * override_ast = table_override->as<ASTTableOverride>();
|
||||||
override->applyToCreateTableQuery(rewritten_query.get());
|
override_ast->applyToCreateTableQuery(rewritten_query.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
return ASTs{rewritten_query};
|
return ASTs{rewritten_query};
|
||||||
|
@ -635,7 +635,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
std::unique_ptr<OpenTelemetrySpanHolder> span;
|
std::unique_ptr<OpenTelemetrySpanHolder> span;
|
||||||
if (context->query_trace_context.trace_id != UUID())
|
if (context->query_trace_context.trace_id != UUID())
|
||||||
{
|
{
|
||||||
auto raw_interpreter_ptr = interpreter.get();
|
auto * raw_interpreter_ptr = interpreter.get();
|
||||||
std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr));
|
std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr));
|
||||||
span = std::make_unique<OpenTelemetrySpanHolder>(class_name + "::execute()");
|
span = std::make_unique<OpenTelemetrySpanHolder>(class_name + "::execute()");
|
||||||
}
|
}
|
||||||
|
@ -40,7 +40,7 @@ public:
|
|||||||
String getID(char) const override { return "TableOverrideList"; }
|
String getID(char) const override { return "TableOverrideList"; }
|
||||||
ASTPtr clone() const override;
|
ASTPtr clone() const override;
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
void setTableOverride(const String & name, ASTPtr override);
|
void setTableOverride(const String & name, const ASTPtr ast);
|
||||||
void removeTableOverride(const String & name);
|
void removeTableOverride(const String & name);
|
||||||
ASTPtr tryGetTableOverride(const String & name) const;
|
ASTPtr tryGetTableOverride(const String & name) const;
|
||||||
bool hasOverride(const String & name) const;
|
bool hasOverride(const String & name) const;
|
||||||
|
@ -83,16 +83,16 @@ TEST_P(TableOverrideTest, applyOverrides)
|
|||||||
ASSERT_NE(nullptr, database);
|
ASSERT_NE(nullptr, database);
|
||||||
ASTPtr table_ast;
|
ASTPtr table_ast;
|
||||||
ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0));
|
ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0));
|
||||||
auto table = table_ast->as<ASTCreateQuery>();
|
auto * table = table_ast->as<ASTCreateQuery>();
|
||||||
ASSERT_NE(nullptr, table);
|
ASSERT_NE(nullptr, table);
|
||||||
auto table_name = table->table->as<ASTIdentifier>()->name();
|
auto table_name = table->table->as<ASTIdentifier>()->name();
|
||||||
if (database->table_overrides)
|
if (database->table_overrides)
|
||||||
{
|
{
|
||||||
auto override_ast = database->table_overrides->tryGetTableOverride(table_name);
|
auto override_ast = database->table_overrides->tryGetTableOverride(table_name);
|
||||||
ASSERT_NE(nullptr, override_ast);
|
ASSERT_NE(nullptr, override_ast);
|
||||||
auto override = override_ast->as<ASTTableOverride>();
|
auto * override_table_ast = override_ast->as<ASTTableOverride>();
|
||||||
ASSERT_NE(nullptr, override);
|
ASSERT_NE(nullptr, override_table_ast);
|
||||||
override->applyToCreateTableQuery(table);
|
override_table_ast->applyToCreateTableQuery(table);
|
||||||
}
|
}
|
||||||
EXPECT_EQ(expected_query, serializeAST(*table));
|
EXPECT_EQ(expected_query, serializeAST(*table));
|
||||||
}
|
}
|
||||||
|
@ -24,14 +24,14 @@ static FormatSettings updateFormatSettings(const FormatSettings & settings)
|
|||||||
|
|
||||||
CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
|
CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
|
||||||
const Block & header_,
|
const Block & header_,
|
||||||
ReadBuffer & in_,
|
ReadBuffer & in_buf_,
|
||||||
const Params & params_,
|
const Params & params_,
|
||||||
bool with_names_,
|
bool with_names_,
|
||||||
bool with_types_,
|
bool with_types_,
|
||||||
bool ignore_spaces_,
|
bool ignore_spaces_,
|
||||||
const FormatSettings & format_settings_)
|
const FormatSettings & format_settings_)
|
||||||
: CustomSeparatedRowInputFormat(
|
: CustomSeparatedRowInputFormat(
|
||||||
header_, std::make_unique<PeekableReadBuffer>(in_), params_, with_names_, with_types_, ignore_spaces_, format_settings_)
|
header_, std::make_unique<PeekableReadBuffer>(in_buf_), params_, with_names_, with_types_, ignore_spaces_, format_settings_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -25,7 +25,7 @@ public:
|
|||||||
private:
|
private:
|
||||||
CustomSeparatedRowInputFormat(
|
CustomSeparatedRowInputFormat(
|
||||||
const Block & header_,
|
const Block & header_,
|
||||||
std::unique_ptr<PeekableReadBuffer> in_,
|
std::unique_ptr<PeekableReadBuffer> in_buf_,
|
||||||
const Params & params_,
|
const Params & params_,
|
||||||
bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_);
|
bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_);
|
||||||
using EscapingRule = FormatSettings::EscapingRule;
|
using EscapingRule = FormatSettings::EscapingRule;
|
||||||
|
@ -76,15 +76,23 @@ def get_image_name(build_config):
|
|||||||
return 'clickhouse/deb-builder'
|
return 'clickhouse/deb-builder'
|
||||||
|
|
||||||
|
|
||||||
def build_clickhouse(packager_cmd, logs_path):
|
def build_clickhouse(packager_cmd, logs_path, build_output_path):
|
||||||
build_log_path = os.path.join(logs_path, 'build_log.log')
|
build_log_path = os.path.join(logs_path, 'build_log.log')
|
||||||
with TeePopen(packager_cmd, build_log_path) as process:
|
with TeePopen(packager_cmd, build_log_path) as process:
|
||||||
retcode = process.wait()
|
retcode = process.wait()
|
||||||
|
if os.path.exists(build_output_path):
|
||||||
|
build_results = os.listdir(build_output_path)
|
||||||
|
else:
|
||||||
|
build_results = []
|
||||||
|
|
||||||
if retcode == 0:
|
if retcode == 0:
|
||||||
logging.info("Built successfully")
|
if len(build_results) != 0:
|
||||||
|
logging.info("Built successfully")
|
||||||
|
else:
|
||||||
|
logging.info("Success exit code, but no build artifacts => build failed")
|
||||||
else:
|
else:
|
||||||
logging.info("Build failed")
|
logging.info("Build failed")
|
||||||
return build_log_path, retcode == 0
|
return build_log_path, retcode == 0 and len(build_results) > 0
|
||||||
|
|
||||||
|
|
||||||
def get_build_results_if_exists(s3_helper, s3_prefix):
|
def get_build_results_if_exists(s3_helper, s3_prefix):
|
||||||
@ -159,7 +167,7 @@ if __name__ == "__main__":
|
|||||||
log_url = 'https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20')
|
log_url = 'https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20')
|
||||||
else:
|
else:
|
||||||
build_urls.append('https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20'))
|
build_urls.append('https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20'))
|
||||||
create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, 0, True)
|
create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, 0, len(build_urls) > 0)
|
||||||
sys.exit(0)
|
sys.exit(0)
|
||||||
|
|
||||||
image_name = get_image_name(build_config)
|
image_name = get_image_name(build_config)
|
||||||
@ -203,7 +211,7 @@ if __name__ == "__main__":
|
|||||||
os.makedirs(build_clickhouse_log)
|
os.makedirs(build_clickhouse_log)
|
||||||
|
|
||||||
start = time.time()
|
start = time.time()
|
||||||
log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log)
|
log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log, build_output_path)
|
||||||
elapsed = int(time.time() - start)
|
elapsed = int(time.time() - start)
|
||||||
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True)
|
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True)
|
||||||
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True)
|
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True)
|
||||||
|
@ -179,9 +179,10 @@ CI_CONFIG = {
|
|||||||
"binary_tidy",
|
"binary_tidy",
|
||||||
"binary_splitted",
|
"binary_splitted",
|
||||||
"binary_darwin",
|
"binary_darwin",
|
||||||
"binary_arrach64",
|
"binary_aarch64",
|
||||||
"binary_freebsd",
|
"binary_freebsd",
|
||||||
"binary_darwin_aarch64"
|
"binary_darwin_aarch64",
|
||||||
|
"binary_ppc64le",
|
||||||
],
|
],
|
||||||
},
|
},
|
||||||
"tests_config": {
|
"tests_config": {
|
||||||
|
Loading…
Reference in New Issue
Block a user