Merge pull request #32745 from ClickHouse/fix_special_build_check

Better control build artifacts
This commit is contained in:
alesapin 2021-12-15 14:51:02 +03:00 committed by GitHub
commit d962d92896
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 69 additions and 39 deletions

View File

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

View 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;
}
}

View 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');
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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()");
} }

View File

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

View File

@ -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));
} }

View File

@ -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_)
{ {
} }

View File

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

View File

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

View File

@ -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": {