Skip to content

Commit

Permalink
Limit num-of row-groups when build
Browse files Browse the repository at this point in the history
  • Loading branch information
mapleFU committed Sep 10, 2024
1 parent fed5fcb commit e5c061d
Showing 1 changed file with 5 additions and 0 deletions.
5 changes: 5 additions & 0 deletions cpp/src/parquet/file_writer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -359,6 +359,11 @@ class FileSerializer : public ParquetFileWriter::Contents {
if (row_group_writer_) {
row_group_writer_->Close();
}
if (num_row_groups_ >= std::numeric_limits<int16_t>::max()) {
// Parquet thrifts using int16 for row group ordinal, so we can't have more than
// 32767 row groups in a file.
throw ParquetException("Too many row groups in the file when writing row group");
}
num_row_groups_++;
auto rg_metadata = metadata_->AppendRowGroup();
if (page_index_builder_) {
Expand Down

0 comments on commit e5c061d

Please sign in to comment.