Revert "Add return flag for preFinalize."

This reverts commit 3261378ef8.
This commit is contained in:
Nikolai Kochetov 2023-02-09 19:51:09 +00:00
parent 3261378ef8
commit 4aa6e4d683
16 changed files with 37 additions and 74 deletions

View File

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

View File

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

View File

@ -16,7 +16,7 @@ void WriteBufferFromFileDecorator::finalizeImpl()
next();
if (!is_prefinalized)
std::ignore = WriteBufferFromFileDecorator::preFinalize();
WriteBufferFromFileDecorator::preFinalize();
impl->finalize();
}

View File

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

View File

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

View File

@ -55,7 +55,7 @@ public:
void nextImpl() override;
bool preFinalize() override;
void preFinalize() override;
private:
void allocateBuffer();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -80,7 +80,7 @@ public:
bool is_prefinalized = false;
[[nodiscard]] bool preFinalize();
void preFinalize();
void finalize();

View File

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

View File

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

View File

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

View File

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