wgtmac commented on code in PR #37400:
URL: https://github.com/apache/arrow/pull/37400#discussion_r1536842511
##########
cpp/src/parquet/column_writer.cc:
##########
@@ -1624,14 +1633,22 @@ class TypedColumnWriterImpl : public ColumnWriterImpl,
public TypedColumnWriter<
if (num_values != num_spaced_values) {
current_value_encoder_->PutSpaced(values,
static_cast<int>(num_spaced_values),
valid_bits, valid_bits_offset);
+ UpdateBloomFilterSpaced(values, num_spaced_values, valid_bits,
valid_bits_offset);
Review Comment:
In the impl of `UpdateBloomFilterSpaced`, perhaps we should also take a
short-cut if (num_values == num_spaced_values)?
##########
cpp/src/parquet/bloom_filter.h:
##########
@@ -167,6 +167,17 @@ class PARQUET_EXPORT BloomFilter {
virtual ~BloomFilter() = default;
+ // Variant of const reference argument to facilitate template
+ uint64_t Hash(const ByteArray& value) const { return Hash(&value); }
Review Comment:
These are new public functions anyway. It would be better to relocate them
to close to their friends and add similar docstrings.
##########
cpp/src/parquet/metadata.cc:
##########
@@ -1805,36 +1805,63 @@ class FileMetaDataBuilder::FileMetaDataBuilderImpl {
}
void SetPageIndexLocation(const PageIndexLocation& location) {
- auto set_index_location =
+ auto set_index_location = [this](size_t row_group_ordinal,
+ const FileIndexLocation&
file_index_location,
+ bool column_index) {
+ auto& row_group_metadata = this->row_groups_.at(row_group_ordinal);
+ auto iter = file_index_location.find(row_group_ordinal);
+ if (iter != file_index_location.cend()) {
+ const auto& row_group_index_location = iter->second;
+ for (size_t i = 0; i < row_group_index_location.size(); ++i) {
+ if (i >= row_group_metadata.columns.size()) {
+ throw ParquetException("Cannot find metadata for column ordinal ",
i);
+ }
+ auto& column_metadata = row_group_metadata.columns.at(i);
+ const auto& index_location = row_group_index_location.at(i);
+ if (index_location.has_value()) {
+ if (column_index) {
+
column_metadata.__set_column_index_offset(index_location->offset);
+
column_metadata.__set_column_index_length(index_location->length);
+ } else {
+
column_metadata.__set_offset_index_offset(index_location->offset);
+
column_metadata.__set_offset_index_length(index_location->length);
+ }
+ }
+ }
+ }
+ };
+
+ for (size_t i = 0; i < row_groups_.size(); ++i) {
+ set_index_location(i, location.column_index_location, true);
+ set_index_location(i, location.offset_index_location, false);
+ }
+ }
+
+ // Update location to all bloom filters in the parquet file.
+ void SetBloomFilterLocation(const BloomFilterLocation& location) {
+ auto set_bloom_filter_location =
[this](size_t row_group_ordinal,
- const PageIndexLocation::FileIndexLocation& file_index_location,
- bool column_index) {
+ const FileIndexLocation& file_bloom_filter_location) {
auto& row_group_metadata = this->row_groups_.at(row_group_ordinal);
- auto iter = file_index_location.find(row_group_ordinal);
- if (iter != file_index_location.cend()) {
- const auto& row_group_index_location = iter->second;
- for (size_t i = 0; i < row_group_index_location.size(); ++i) {
- if (i >= row_group_metadata.columns.size()) {
- throw ParquetException("Cannot find metadata for column
ordinal ", i);
- }
- auto& column_metadata = row_group_metadata.columns.at(i);
- const auto& index_location = row_group_index_location.at(i);
- if (index_location.has_value()) {
- if (column_index) {
-
column_metadata.__set_column_index_offset(index_location->offset);
-
column_metadata.__set_column_index_length(index_location->length);
- } else {
-
column_metadata.__set_offset_index_offset(index_location->offset);
-
column_metadata.__set_offset_index_length(index_location->length);
- }
+ auto iter = file_bloom_filter_location.find(row_group_ordinal);
+ if (iter != file_bloom_filter_location.cend()) {
+ const auto& row_group_bloom_filter_location = iter->second;
+ for (size_t i = 0; i < row_group_bloom_filter_location.size();
++i) {
+ DCHECK(i < row_group_metadata.columns.size());
+ auto& column = row_group_metadata.columns[i];
+ auto& column_metadata = column.meta_data;
+ const auto& bloom_filter_location =
row_group_bloom_filter_location[i];
+ if (bloom_filter_location.has_value()) {
+
column_metadata.__set_bloom_filter_offset(bloom_filter_location->offset);
+ // bloom_filter_length is added in Parquet 2.10
Review Comment:
```suggestion
// bloom_filter_length is added by Parquet format 2.10.0
```
##########
cpp/src/parquet/column_writer.cc:
##########
@@ -1603,6 +1611,7 @@ class TypedColumnWriterImpl : public ColumnWriterImpl,
public TypedColumnWriter<
if (page_statistics_ != nullptr) {
page_statistics_->Update(values, num_values, num_nulls);
}
+ UpdateBloomFilter(values, num_values);
Review Comment:
Does it make sense to add a new internal class for column statistics, bloom
filter and size statistics (which I'm implementing) similar to this:
https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnValueCollector.java
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,82 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/io/type_fwd.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+///
+/// ```
+/// auto bloom_filter_builder = BloomFilterBuilder::Make(schema, properties);
+/// for (int i = 0; i < num_row_groups; i++) {
+/// bloom_filter_builder->AppendRowGroup();
+/// auto* bloom_filter =
+/// bloom_filter_builder->GetOrCreateBloomFilter(bloom_filter_column);
+/// // Add bloom filter entries in `bloom_filter`.
+/// // ...
+/// }
+/// bloom_filter_builder->WriteTo(sink, location);
+/// ```
+class PARQUET_EXPORT BloomFilterBuilder {
+ public:
+ /// \brief API to create a BloomFilterBuilder.
+ static std::unique_ptr<BloomFilterBuilder> Make(const SchemaDescriptor*
schema,
+ const WriterProperties&
properties);
+
+ /// Append a new row group to host all incoming bloom filters.
+ ///
+ /// This method must be called before `GetOrCreateBloomFilter`
+ /// in a row group.
+ virtual void AppendRowGroup() = 0;
+
+ /// \brief Get the BloomFilter from column ordinal.
+ ///
+ /// \param column_ordinal Column ordinal in schema, which is only for leaf
columns.
+ ///
+ /// \return BloomFilter for the column and its memory ownership belongs to
the
+ /// BloomFilterBuilder. It will throw an exception if the BloomFilter is
already
+ /// Finished or column_ordinal is out of bound.
+ ///
+ /// It will return nullptr if bloom filter is not enabled for the column.
Review Comment:
Consolidate this with the `\return` statement above?
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,82 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/io/type_fwd.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+///
+/// ```
+/// auto bloom_filter_builder = BloomFilterBuilder::Make(schema, properties);
+/// for (int i = 0; i < num_row_groups; i++) {
+/// bloom_filter_builder->AppendRowGroup();
+/// auto* bloom_filter =
+/// bloom_filter_builder->GetOrCreateBloomFilter(bloom_filter_column);
+/// // Add bloom filter entries in `bloom_filter`.
+/// // ...
+/// }
+/// bloom_filter_builder->WriteTo(sink, location);
+/// ```
+class PARQUET_EXPORT BloomFilterBuilder {
+ public:
+ /// \brief API to create a BloomFilterBuilder.
+ static std::unique_ptr<BloomFilterBuilder> Make(const SchemaDescriptor*
schema,
+ const WriterProperties&
properties);
+
+ /// Append a new row group to host all incoming bloom filters.
+ ///
+ /// This method must be called before `GetOrCreateBloomFilter`
+ /// in a row group.
+ virtual void AppendRowGroup() = 0;
+
+ /// \brief Get the BloomFilter from column ordinal.
+ ///
+ /// \param column_ordinal Column ordinal in schema, which is only for leaf
columns.
+ ///
+ /// \return BloomFilter for the column and its memory ownership belongs to
the
+ /// BloomFilterBuilder. It will throw an exception if the BloomFilter is
already
Review Comment:
Move the throw exception statement to the end by adding a `\throw` section?
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,82 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/io/type_fwd.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+///
+/// ```
+/// auto bloom_filter_builder = BloomFilterBuilder::Make(schema, properties);
+/// for (int i = 0; i < num_row_groups; i++) {
+/// bloom_filter_builder->AppendRowGroup();
+/// auto* bloom_filter =
+/// bloom_filter_builder->GetOrCreateBloomFilter(bloom_filter_column);
+/// // Add bloom filter entries in `bloom_filter`.
+/// // ...
+/// }
+/// bloom_filter_builder->WriteTo(sink, location);
+/// ```
+class PARQUET_EXPORT BloomFilterBuilder {
+ public:
+ /// \brief API to create a BloomFilterBuilder.
+ static std::unique_ptr<BloomFilterBuilder> Make(const SchemaDescriptor*
schema,
+ const WriterProperties&
properties);
+
+ /// Append a new row group to host all incoming bloom filters.
+ ///
+ /// This method must be called before `GetOrCreateBloomFilter`
+ /// in a row group.
+ virtual void AppendRowGroup() = 0;
+
+ /// \brief Get the BloomFilter from column ordinal.
+ ///
+ /// \param column_ordinal Column ordinal in schema, which is only for leaf
columns.
+ ///
+ /// \return BloomFilter for the column and its memory ownership belongs to
the
+ /// BloomFilterBuilder. It will throw an exception if the BloomFilter is
already
+ /// Finished or column_ordinal is out of bound.
Review Comment:
```suggestion
/// finished or column_ordinal is out of bound.
```
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include "arrow/io/interfaces.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+namespace schema {
+class ColumnPath;
+}
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+class PARQUET_EXPORT BloomFilterBuilder {
Review Comment:
We already have a lot of Builders, so `BloomFilterBuilder` looks fine to me.
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,82 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "arrow/io/type_fwd.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+///
+/// ```
+/// auto bloom_filter_builder = BloomFilterBuilder::Make(schema, properties);
+/// for (int i = 0; i < num_row_groups; i++) {
+/// bloom_filter_builder->AppendRowGroup();
+/// auto* bloom_filter =
+/// bloom_filter_builder->GetOrCreateBloomFilter(bloom_filter_column);
+/// // Add bloom filter entries in `bloom_filter`.
+/// // ...
+/// }
+/// bloom_filter_builder->WriteTo(sink, location);
+/// ```
+class PARQUET_EXPORT BloomFilterBuilder {
+ public:
+ /// \brief API to create a BloomFilterBuilder.
+ static std::unique_ptr<BloomFilterBuilder> Make(const SchemaDescriptor*
schema,
+ const WriterProperties&
properties);
+
+ /// Append a new row group to host all incoming bloom filters.
+ ///
+ /// This method must be called before `GetOrCreateBloomFilter`
+ /// in a row group.
+ virtual void AppendRowGroup() = 0;
+
+ /// \brief Get the BloomFilter from column ordinal.
+ ///
+ /// \param column_ordinal Column ordinal in schema, which is only for leaf
columns.
+ ///
+ /// \return BloomFilter for the column and its memory ownership belongs to
the
+ /// BloomFilterBuilder. It will throw an exception if the BloomFilter is
already
Review Comment:
BTW, it might be better to say throw ParquetException explicitly.
##########
cpp/src/parquet/column_writer.cc:
##########
@@ -1708,7 +1728,7 @@ Status TypedColumnWriterImpl<DType>::WriteArrowDictionary(
AddIfNotNull(rep_levels, offset));
std::shared_ptr<Array> writeable_indices =
indices->Slice(value_offset, batch_num_spaced_values);
- if (page_statistics_) {
+ if (page_statistics_ || bloom_filter_) {
Review Comment:
I did similar thing when I implement SizeStatistics.
##########
cpp/src/parquet/column_writer.cc:
##########
@@ -2322,12 +2347,133 @@ Status
TypedColumnWriterImpl<FLBAType>::WriteArrowDense(
return Status::OK();
}
+template <typename DType>
+void TypedColumnWriterImpl<DType>::UpdateBloomFilter(const T* values,
+ int64_t num_values) {
+ if (bloom_filter_) {
+ std::array<uint64_t, kHashBatchSize> hashes;
+ for (int64_t i = 0; i < num_values; i += kHashBatchSize) {
+ int64_t current_hash_batch_size = std::min(kHashBatchSize, num_values -
i);
+ bloom_filter_->Hashes(values, static_cast<int>(current_hash_batch_size),
+ hashes.data());
+ bloom_filter_->InsertHashes(hashes.data(),
+ static_cast<int>(current_hash_batch_size));
+ }
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<FLBAType>::UpdateBloomFilter(const FLBA* values,
+ int64_t num_values) {
+ if (bloom_filter_) {
+ std::array<uint64_t, kHashBatchSize> hashes;
+ for (int64_t i = 0; i < num_values; i += kHashBatchSize) {
+ int64_t current_hash_batch_size = std::min(kHashBatchSize, num_values -
i);
+ bloom_filter_->Hashes(values, descr_->type_length(),
+ static_cast<int>(current_hash_batch_size),
hashes.data());
+ bloom_filter_->InsertHashes(hashes.data(),
+ static_cast<int>(current_hash_batch_size));
+ }
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<BooleanType>::UpdateBloomFilter(const bool*,
int64_t) {
+ DCHECK(bloom_filter_ == nullptr);
+}
+
+template <typename DType>
+void TypedColumnWriterImpl<DType>::UpdateBloomFilterSpaced(const T* values,
+ int64_t num_values,
+ const uint8_t*
valid_bits,
+ int64_t
valid_bits_offset) {
+ if (bloom_filter_) {
+ std::array<uint64_t, kHashBatchSize> hashes;
+ ::arrow::internal::VisitSetBitRunsVoid(
+ valid_bits, valid_bits_offset, num_values, [&](int64_t position,
int64_t length) {
+ for (int64_t i = 0; i < length; i += kHashBatchSize) {
+ auto current_hash_batch_size = std::min(kHashBatchSize, length -
i);
+ bloom_filter_->Hashes(values + i + position,
+ static_cast<int>(current_hash_batch_size),
+ hashes.data());
+ bloom_filter_->InsertHashes(hashes.data(),
+
static_cast<int>(current_hash_batch_size));
+ }
+ });
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<BooleanType>::UpdateBloomFilterSpaced(const bool*,
int64_t,
+ const
uint8_t*,
+ int64_t) {
+ DCHECK(bloom_filter_ == nullptr);
Review Comment:
Isn't it checked when creating the column writer? It would be enough to make
it an empty function.
##########
cpp/src/parquet/column_writer.cc:
##########
@@ -2319,12 +2343,119 @@ Status
TypedColumnWriterImpl<FLBAType>::WriteArrowDense(
return Status::OK();
}
+template <typename DType>
+void TypedColumnWriterImpl<DType>::UpdateBloomFilter(const T* values,
+ int64_t num_values) {
+ if (bloom_filter_) {
+ // TODO(mwish): Would it allocate too much memory? Would an
std::array<uint64_t, 64>
+ // better here?
+ std::vector<uint64_t> hashes(num_values);
+ bloom_filter_->Hashes(values, static_cast<int>(num_values), hashes.data());
+ bloom_filter_->InsertHashes(hashes.data(), static_cast<int>(num_values));
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<FLBAType>::UpdateBloomFilter(const FLBA* values,
+ int64_t num_values) {
+ if (bloom_filter_) {
+ for (int64_t i = 0; i < num_values; ++i) {
+ bloom_filter_->InsertHash(bloom_filter_->Hash(values + i,
descr_->type_length()));
+ }
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<BooleanType>::UpdateBloomFilter(const bool*,
int64_t) {
+ DCHECK(bloom_filter_ == nullptr);
+}
+
+template <typename DType>
+void TypedColumnWriterImpl<DType>::UpdateBloomFilterSpaced(const T* values,
+ int64_t num_values,
+ const uint8_t*
valid_bits,
+ int64_t
valid_bits_offset) {
+ if (bloom_filter_) {
+ ::arrow::internal::VisitSetBitRunsVoid(
+ valid_bits, valid_bits_offset, num_values, [&](int64_t position,
int64_t length) {
+ for (int64_t i = 0; i < length; i++) {
+ bloom_filter_->InsertHash(bloom_filter_->Hash(values + i +
position));
+ }
+ });
+ }
+}
+
+template <>
+void TypedColumnWriterImpl<BooleanType>::UpdateBloomFilterSpaced(const bool*,
int64_t,
+ const
uint8_t*,
+ int64_t) {
+ DCHECK(bloom_filter_ == nullptr);
+}
+
+template <>
+void TypedColumnWriterImpl<FLBAType>::UpdateBloomFilterSpaced(const FLBA*
values,
+ int64_t
num_values,
+ const uint8_t*
valid_bits,
+ int64_t
valid_bits_offset) {
+ if (bloom_filter_) {
+ ::arrow::internal::VisitSetBitRunsVoid(
+ valid_bits, valid_bits_offset, num_values, [&](int64_t position,
int64_t length) {
+ for (int64_t i = 0; i < length; i++) {
+ bloom_filter_->InsertHash(
+ bloom_filter_->Hash(values + i + position,
descr_->type_length()));
+ }
+ });
+ }
+}
+
+template <typename ArrayType>
+void UpdateBinaryBloomFilter(BloomFilter* bloom_filter, const ArrayType&
array) {
+ PARQUET_THROW_NOT_OK(::arrow::VisitArraySpanInline<typename
ArrayType::TypeClass>(
+ *array.data(),
+ [&](const std::string_view& view) {
Review Comment:
Do you want to revive the fix?
##########
cpp/src/parquet/file_writer.cc:
##########
@@ -140,38 +143,39 @@ class RowGroupSerializer : public
RowGroupWriter::Contents {
const int32_t column_ordinal = next_column_index_++;
const auto& path = col_meta->descr()->path();
+ const ColumnProperties& column_properties =
properties_->column_properties(path);
auto meta_encryptor =
file_encryptor_ ?
file_encryptor_->GetColumnMetaEncryptor(path->ToDotString())
: nullptr;
auto data_encryptor =
file_encryptor_ ?
file_encryptor_->GetColumnDataEncryptor(path->ToDotString())
: nullptr;
- auto ci_builder = page_index_builder_ &&
properties_->page_index_enabled(path) &&
- properties_->statistics_enabled(path)
+ auto ci_builder = page_index_builder_ &&
column_properties.page_index_enabled() &&
+ column_properties.statistics_enabled()
?
page_index_builder_->GetColumnIndexBuilder(column_ordinal)
: nullptr;
- auto oi_builder = page_index_builder_ &&
properties_->page_index_enabled(path)
+ auto oi_builder = page_index_builder_ &&
column_properties.page_index_enabled()
?
page_index_builder_->GetOffsetIndexBuilder(column_ordinal)
: nullptr;
- auto codec_options = properties_->codec_options(path)
- ? properties_->codec_options(path).get()
- : nullptr;
-
- std::unique_ptr<PageWriter> pager;
+ const CodecOptions* codec_options = column_properties.codec_options()
+ ?
column_properties.codec_options().get()
+ : nullptr;
+ BloomFilter* bloom_filter =
+ bloom_filter_builder_ && column_properties.bloom_filter_enabled()
+ ? bloom_filter_builder_->GetOrCreateBloomFilter(column_ordinal)
+ : nullptr;
+
+ CodecOptions default_codec_options;
Review Comment:
Perhaps using a separate patch? I think it will be faster to get merged than
this patch.
##########
cpp/src/parquet/bloom_filter_builder.h:
##########
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#pragma once
+
+#include "arrow/io/interfaces.h"
+#include "parquet/types.h"
+
+namespace parquet {
+
+class BloomFilter;
+class SchemaDescriptor;
+struct BloomFilterOptions;
+struct BloomFilterLocation;
+
+namespace schema {
+class ColumnPath;
+}
+
+/// \brief Interface for collecting bloom filter of a parquet file.
+class PARQUET_EXPORT BloomFilterBuilder {
Review Comment:
BTW, do we really need a separate file for the new builder? If yes, should
we make it an internal header? My rationale is that this kind of builder is
used only internally by the writer and users do not have to deal with them at
all. However, we have already exposed builders like FileMetaDataBuilder,
RowGroupMetaDataBuilder, ColumnChunkMetaDataBuilder, and PageIndexBuilder
(unfortunately it was added by myself).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]