mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Revert "Add return flag for preFinalize."
This reverts commit 3261378ef8
.
This commit is contained in:
parent
3261378ef8
commit
4aa6e4d683
@ -169,7 +169,6 @@ void SerializationAggregateFunction::deserializeTextQuoted(IColumn & column, Rea
|
||||
{
|
||||
String s;
|
||||
readQuotedStringWithSQLStyle(s, istr);
|
||||
std::cerr << "==========\n" << s << std::endl << "====\n";
|
||||
deserializeFromString(function, column, s, version);
|
||||
}
|
||||
|
||||
|
@ -110,14 +110,7 @@ public:
|
||||
/// and start sending data asynchronously. It may improve writing performance in case you have
|
||||
/// multiple files to finalize. Mainly, for blob storage, finalization has high latency,
|
||||
/// and calling preFinalize in a loop may parallelize it.
|
||||
///
|
||||
/// Returns true if preFinalize did something useful and some data was sent asynchronously indeed.
|
||||
/// Otherwise, we can call finalize() and remove a buffer to decrease memory usage.
|
||||
[[nodiscard]] virtual bool preFinalize()
|
||||
{
|
||||
next();
|
||||
return false;
|
||||
}
|
||||
virtual void preFinalize() { next(); }
|
||||
|
||||
/// Write the last data.
|
||||
void finalize()
|
||||
|
@ -16,7 +16,7 @@ void WriteBufferFromFileDecorator::finalizeImpl()
|
||||
next();
|
||||
|
||||
if (!is_prefinalized)
|
||||
std::ignore = WriteBufferFromFileDecorator::preFinalize();
|
||||
WriteBufferFromFileDecorator::preFinalize();
|
||||
|
||||
impl->finalize();
|
||||
}
|
||||
|
@ -17,12 +17,11 @@ public:
|
||||
|
||||
std::string getFileName() const override;
|
||||
|
||||
bool preFinalize() override
|
||||
void preFinalize() override
|
||||
{
|
||||
next();
|
||||
auto res = impl->preFinalize();
|
||||
impl->preFinalize();
|
||||
is_prefinalized = true;
|
||||
return res;
|
||||
}
|
||||
|
||||
const WriteBuffer & getImpl() const { return *impl; }
|
||||
|
@ -148,7 +148,7 @@ WriteBufferFromS3::~WriteBufferFromS3()
|
||||
#endif
|
||||
}
|
||||
|
||||
bool WriteBufferFromS3::preFinalize()
|
||||
void WriteBufferFromS3::preFinalize()
|
||||
{
|
||||
next();
|
||||
|
||||
@ -163,13 +163,12 @@ bool WriteBufferFromS3::preFinalize()
|
||||
}
|
||||
|
||||
is_prefinalized = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
void WriteBufferFromS3::finalizeImpl()
|
||||
{
|
||||
if (!is_prefinalized)
|
||||
std::ignore = preFinalize();
|
||||
preFinalize();
|
||||
|
||||
waitForAllBackGroundTasks();
|
||||
|
||||
|
@ -55,7 +55,7 @@ public:
|
||||
|
||||
void nextImpl() override;
|
||||
|
||||
bool preFinalize() override;
|
||||
void preFinalize() override;
|
||||
|
||||
private:
|
||||
void allocateBuffer();
|
||||
|
@ -139,10 +139,8 @@ IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::s
|
||||
out_hashing.next();
|
||||
out_checksums.files[file_name].file_size = out_hashing.count();
|
||||
out_checksums.files[file_name].file_hash = out_hashing.getHash();
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
return written_files;
|
||||
|
@ -39,8 +39,6 @@ public:
|
||||
Columns releaseIndexColumns();
|
||||
const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; }
|
||||
|
||||
bool hasAsynchronousWritingBuffers() const { return has_asynchronous_writing_buffers; }
|
||||
|
||||
protected:
|
||||
|
||||
const MergeTreeMutableDataPartPtr data_part;
|
||||
@ -50,7 +48,6 @@ protected:
|
||||
const MergeTreeWriterSettings settings;
|
||||
MergeTreeIndexGranularity index_granularity;
|
||||
const bool with_final_mark;
|
||||
bool has_asynchronous_writing_buffers = false;
|
||||
|
||||
MutableColumns index_columns;
|
||||
};
|
||||
|
@ -558,10 +558,7 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const
|
||||
auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns);
|
||||
global_ctx->checksums_gathered_columns.add(std::move(changed_checksums));
|
||||
|
||||
if (ctx->column_to->hasAsynchronousWritingBuffers())
|
||||
ctx->delayed_streams.emplace_back(std::move(ctx->column_to));
|
||||
else
|
||||
ctx->column_to->finish(ctx->need_sync);
|
||||
ctx->delayed_streams.emplace_back(std::move(ctx->column_to));
|
||||
|
||||
while (ctx->delayed_streams.size() > ctx->max_delayed_streams)
|
||||
{
|
||||
|
@ -286,8 +286,8 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Check
|
||||
marks_file_hashing->next();
|
||||
addToChecksums(checksums);
|
||||
|
||||
has_asynchronous_writing_buffers |= plain_file->preFinalize();
|
||||
has_asynchronous_writing_buffers |= marks_file->preFinalize();
|
||||
plain_file->preFinalize();
|
||||
marks_file->preFinalize();
|
||||
}
|
||||
|
||||
void MergeTreeDataPartWriterCompact::finishDataSerialization(bool sync)
|
||||
|
@ -11,7 +11,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
bool MergeTreeDataPartWriterOnDisk::Stream::preFinalize()
|
||||
void MergeTreeDataPartWriterOnDisk::Stream::preFinalize()
|
||||
{
|
||||
compressed_hashing.next();
|
||||
compressor.next();
|
||||
@ -25,18 +25,16 @@ bool MergeTreeDataPartWriterOnDisk::Stream::preFinalize()
|
||||
|
||||
marks_hashing.next();
|
||||
|
||||
bool has_asynchronous_writing_buffers = false;
|
||||
has_asynchronous_writing_buffers |= plain_file->preFinalize();
|
||||
has_asynchronous_writing_buffers |= marks_file->preFinalize();
|
||||
plain_file->preFinalize();
|
||||
marks_file->preFinalize();
|
||||
|
||||
is_prefinalized = true;
|
||||
return has_asynchronous_writing_buffers;
|
||||
}
|
||||
|
||||
void MergeTreeDataPartWriterOnDisk::Stream::finalize()
|
||||
{
|
||||
if (!is_prefinalized)
|
||||
std::ignore = preFinalize();
|
||||
preFinalize();
|
||||
|
||||
plain_file->finalize();
|
||||
marks_file->finalize();
|
||||
@ -362,7 +360,7 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat
|
||||
}
|
||||
checksums.files[index_name].file_size = index_file_hashing_stream->count();
|
||||
checksums.files[index_name].file_hash = index_file_hashing_stream->getHash();
|
||||
has_asynchronous_writing_buffers |= index_file_stream->preFinalize();
|
||||
index_file_stream->preFinalize();
|
||||
}
|
||||
}
|
||||
|
||||
@ -394,7 +392,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data
|
||||
|
||||
for (auto & stream : skip_indices_streams)
|
||||
{
|
||||
has_asynchronous_writing_buffers |= stream->preFinalize();
|
||||
stream->preFinalize();
|
||||
stream->addToChecksums(checksums);
|
||||
}
|
||||
}
|
||||
|
@ -80,7 +80,7 @@ public:
|
||||
|
||||
bool is_prefinalized = false;
|
||||
|
||||
[[nodiscard]] bool preFinalize();
|
||||
void preFinalize();
|
||||
|
||||
void finalize();
|
||||
|
||||
|
@ -589,7 +589,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum
|
||||
|
||||
for (auto & stream : column_streams)
|
||||
{
|
||||
has_asynchronous_writing_buffers |= stream.second->preFinalize();
|
||||
stream.second->preFinalize();
|
||||
stream.second->addToChecksums(checksums);
|
||||
}
|
||||
}
|
||||
|
@ -418,7 +418,7 @@ std::unique_ptr<WriteBufferFromFileBase> MergeTreePartition::store(const Block &
|
||||
out_hashing.next();
|
||||
checksums.files["partition.dat"].file_size = out_hashing.count();
|
||||
checksums.files["partition.dat"].file_hash = out_hashing.getHash();
|
||||
std::ignore = out->preFinalize();
|
||||
out->preFinalize();
|
||||
return out;
|
||||
}
|
||||
|
||||
|
@ -213,10 +213,8 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
count_out_hashing.next();
|
||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
||||
if (count_out->preFinalize())
|
||||
written_files.emplace_back(std::move(count_out));
|
||||
else
|
||||
count_out->finalize();
|
||||
count_out->preFinalize();
|
||||
written_files.emplace_back(std::move(count_out));
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -228,10 +226,8 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
writeUUIDText(new_part->uuid, out_hashing);
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
@ -257,10 +253,8 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
count_out_hashing.next();
|
||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
||||
if (count_out->preFinalize())
|
||||
written_files.emplace_back(std::move(count_out));
|
||||
else
|
||||
count_out->finalize();
|
||||
count_out->preFinalize();
|
||||
written_files.emplace_back(std::move(count_out));
|
||||
}
|
||||
}
|
||||
|
||||
@ -272,10 +266,8 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
new_part->ttl_infos.write(out_hashing);
|
||||
checksums.files["ttl.txt"].file_size = out_hashing.count();
|
||||
checksums.files["ttl.txt"].file_hash = out_hashing.getHash();
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
if (!new_part->getSerializationInfos().empty())
|
||||
@ -285,30 +277,24 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
new_part->getSerializationInfos().writeJSON(out_hashing);
|
||||
checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count();
|
||||
checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash();
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
{
|
||||
/// Write a file with a description of columns.
|
||||
auto out = new_part->getDataPartStorage().writeFile("columns.txt", 4096, write_settings);
|
||||
new_part->getColumns().writeText(*out);
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
if (default_codec != nullptr)
|
||||
{
|
||||
auto out = new_part->getDataPartStorage().writeFile(IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096, write_settings);
|
||||
DB::writeText(queryToString(default_codec->getFullCodecDesc()), *out);
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -320,10 +306,8 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis
|
||||
/// Write file with checksums.
|
||||
auto out = new_part->getDataPartStorage().writeFile("checksums.txt", 4096, write_settings);
|
||||
checksums.write(*out);
|
||||
if (out->preFinalize())
|
||||
written_files.emplace_back(std::move(out));
|
||||
else
|
||||
out->finalize();
|
||||
out->preFinalize();
|
||||
written_files.emplace_back(std::move(out));
|
||||
}
|
||||
|
||||
return written_files;
|
||||
|
@ -29,7 +29,6 @@ public:
|
||||
MergeTreeData::DataPart::Checksums
|
||||
fillChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums);
|
||||
|
||||
bool hasAsynchronousWritingBuffers() const { return writer->hasAsynchronousWritingBuffers(); }
|
||||
void finish(bool sync);
|
||||
|
||||
private:
|
||||
|
Loading…
Reference in New Issue
Block a user