Jinchul81 commented on code in PR #317:
URL: https://github.com/apache/iceberg-cpp/pull/317#discussion_r2553981036


##########
src/iceberg/manifest_adapter.cc:
##########
@@ -161,6 +164,110 @@ ManifestEntryAdapter::~ManifestEntryAdapter() {
   }
 }
 
+Status ManifestEntryAdapter::AddEntry(ManifestEntry& entry) {
+  ICEBERG_RETURN_UNEXPECTED(CheckDataFile(*entry.data_file));
+  entry.status = ManifestStatus::kAdded;
+  entry.snapshot_id = snapshot_id_;
+  if (entry.sequence_number.has_value() &&
+      entry.sequence_number.value() < TableMetadata::kInitialSequenceNumber) {
+    entry.sequence_number = std::nullopt;
+  }
+  entry.file_sequence_number = std::nullopt;

Review Comment:
   Why is `file_sequence_number` initialized here?



##########
src/iceberg/metrics.h:
##########
@@ -32,12 +32,12 @@ namespace iceberg {
 /// \brief Iceberg file format metrics
 struct ICEBERG_EXPORT Metrics {
   int64_t row_count = 0;
-  std::unordered_map<int64_t, int64_t> column_sizes;
-  std::unordered_map<int64_t, int64_t> value_counts;
-  std::unordered_map<int64_t, int64_t> null_value_counts;
-  std::unordered_map<int64_t, int64_t> nan_value_counts;
-  std::unordered_map<int64_t, Literal> lower_bounds;
-  std::unordered_map<int64_t, Literal> upper_bounds;
+  std::unordered_map<int32_t, int64_t> column_sizes;

Review Comment:
   What's the reason to shrink the size?



##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {
+ public:
+  explicit PartitionFieldStats(const std::shared_ptr<Type>& type) : 
type_(type) {}
+
+  Status Update(const Literal& value);
+
+  PartitionFieldSummary Finish() const;
+
+ private:
+  std::shared_ptr<Type> type_{nullptr};
+  bool contains_null_{false};
+  bool contains_nan_{false};
+  std::optional<Literal> lower_bound_;
+  std::optional<Literal> upper_bound_;
+};
+
+class PartitionSummary {
+ public:
+  /// \brief Update the partition summary with partition.
+  Status Update(const std::vector<Literal>& partition);
+
+  /// \brief Get the list of partition field summaries.
+  std::vector<PartitionFieldSummary> Summaries() const;
+
+  /// \brief Create a PartitionSummary from the partition type.
+  /// \param partition_type The partition type.
+  /// \return A Result containing a unique pointer to the PartitionSummary.
+  static Result<std::unique_ptr<PartitionSummary>> Make(const StructType& 
partition_type);
+
+ private:
+  /// \brief Create a PartitionSummary with the given field stats.
+  explicit PartitionSummary(std::vector<PartitionFieldStats> field_stats)

Review Comment:
   Why should we hide the ctor? Please add a comment for reasoning.



##########
src/iceberg/manifest_adapter.cc:
##########
@@ -161,6 +164,110 @@ ManifestEntryAdapter::~ManifestEntryAdapter() {
   }
 }
 
+Status ManifestEntryAdapter::AddEntry(ManifestEntry& entry) {
+  ICEBERG_RETURN_UNEXPECTED(CheckDataFile(*entry.data_file));
+  entry.status = ManifestStatus::kAdded;
+  entry.snapshot_id = snapshot_id_;
+  if (entry.sequence_number.has_value() &&
+      entry.sequence_number.value() < TableMetadata::kInitialSequenceNumber) {
+    entry.sequence_number = std::nullopt;

Review Comment:
   Please add a comment why `sequence_number` should be initialized here.



##########
src/iceberg/manifest_adapter.cc:
##########
@@ -161,6 +164,110 @@ ManifestEntryAdapter::~ManifestEntryAdapter() {
   }
 }
 
+Status ManifestEntryAdapter::AddEntry(ManifestEntry& entry) {
+  ICEBERG_RETURN_UNEXPECTED(CheckDataFile(*entry.data_file));
+  entry.status = ManifestStatus::kAdded;
+  entry.snapshot_id = snapshot_id_;
+  if (entry.sequence_number.has_value() &&
+      entry.sequence_number.value() < TableMetadata::kInitialSequenceNumber) {
+    entry.sequence_number = std::nullopt;
+  }
+  entry.file_sequence_number = std::nullopt;
+  return AddEntryInternal(entry);
+}
+
+Status ManifestEntryAdapter::AddDeleteEntry(ManifestEntry& entry) {
+  ICEBERG_RETURN_UNEXPECTED(CheckDataFile(*entry.data_file));
+  entry.status = ManifestStatus::kDeleted;
+  entry.snapshot_id = snapshot_id_;
+  return AddEntryInternal(entry);
+}
+
+Status ManifestEntryAdapter::AddExistingEntry(ManifestEntry& entry) {
+  ICEBERG_RETURN_UNEXPECTED(CheckDataFile(*entry.data_file));
+  entry.status = ManifestStatus::kExisting;
+  return AddEntryInternal(entry);
+}
+
+ManifestFile ManifestEntryAdapter::ToManifestFile() const {
+  ManifestFile manifest_file;
+  manifest_file.partition_spec_id = partition_spec_->spec_id();
+  manifest_file.content = content_;
+  // sequence_number and min_sequence_number with kInvalidSequenceNumber will 
be
+  // replace with real sequence number in `ManifestListWriter`.
+  manifest_file.sequence_number = TableMetadata::kInvalidSequenceNumber;
+  manifest_file.min_sequence_number =
+      min_sequence_number_.value_or(TableMetadata::kInvalidSequenceNumber);
+  manifest_file.existing_files_count = existing_files_count_;
+  manifest_file.added_snapshot_id = 
snapshot_id_.value_or(Snapshot::kInvalidSnapshotId);
+  manifest_file.added_files_count = add_files_count_;
+  manifest_file.existing_files_count = existing_files_count_;
+  manifest_file.deleted_files_count = delete_files_count_;
+  manifest_file.added_rows_count = add_rows_count_;
+  manifest_file.existing_rows_count = existing_rows_count_;
+  manifest_file.deleted_rows_count = delete_rows_count_;
+  manifest_file.partitions = std::move(partition_summary_->Summaries());

Review Comment:
   Please clarify the ownership movement. Please add a comment why it is 
required.



##########
src/iceberg/manifest_list.h:
##########
@@ -72,17 +72,17 @@ struct ICEBERG_EXPORT PartitionFieldSummary {
   static const StructType& Type();
 };
 
+/// \brief The type of files tracked by the manifest, either data or delete 
files; 0 for
+/// all v1 manifests
+enum class ManifestContent {

Review Comment:
   I would prefer `ManifestFileContent` rather than `ManifestContent`.



##########
src/iceberg/manifest_reader_internal.cc:
##########
@@ -367,13 +367,16 @@ Status ParseDataFile(const std::shared_ptr<StructType>& 
data_file_schema,
           return InvalidManifest("Field:{} should be a struct.", field_name);
         }
         if (view_of_file_field->n_children > 0) {
-          auto view_of_partition = view_of_file_field->children[0];
-          for (int64_t row_idx = 0; row_idx < view_of_partition->length; 
row_idx++) {
-            if (ArrowArrayViewIsNull(view_of_partition, row_idx)) {
-              break;
+          for (int64_t partition_idx = 0; partition_idx < 
view_of_file_field->n_children;
+               partition_idx++) {
+            auto view_of_partition = 
view_of_file_field->children[partition_idx];
+            for (int64_t row_idx = 0; row_idx < view_of_partition->length; 
row_idx++) {
+              if (ArrowArrayViewIsNull(view_of_partition, row_idx)) {

Review Comment:
   Can you please add a comment why we stop here if finding null?



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();
+  }
+  if (upper_bound_) {
+    summary.upper_bound = upper_bound_->Serialize().value();
+  }
+  return summary;
+}
+
+Status PartitionSummary::Update(const std::vector<Literal>& partition) {
+  if (partition.size() != field_stats_.size()) {

Review Comment:
   Missing `[[unlikely]]`.



##########
src/iceberg/manifest_writer.cc:
##########
@@ -29,18 +29,39 @@
 
 namespace iceberg {
 
-Status ManifestWriter::Add(const ManifestEntry& entry) {
+Status ManifestWriter::AddEntry(const ManifestEntry& entry) {
   if (adapter_->size() >= kBatchSize) {
     ICEBERG_ASSIGN_OR_RAISE(auto array, adapter_->FinishAppending());
     ICEBERG_RETURN_UNEXPECTED(writer_->Write(array));
     ICEBERG_RETURN_UNEXPECTED(adapter_->StartAppending());
   }
-  return adapter_->Append(entry);
+  auto copy = entry.Copy();
+  return adapter_->AddEntry(copy);
+}
+
+Status ManifestWriter::AddDelete(const ManifestEntry& entry) {
+  if (adapter_->size() >= kBatchSize) {
+    ICEBERG_ASSIGN_OR_RAISE(auto array, adapter_->FinishAppending());
+    ICEBERG_RETURN_UNEXPECTED(writer_->Write(array));
+    ICEBERG_RETURN_UNEXPECTED(adapter_->StartAppending());
+  }
+  auto copy = entry.Copy();
+  return adapter_->AddDeleteEntry(copy);
+}
+
+Status ManifestWriter::AddExisting(const ManifestEntry& entry) {
+  if (adapter_->size() >= kBatchSize) {

Review Comment:
   Please refactor code. The following code snippets are redundant in 
`AddEntry`, `AddDelete`, and `AddExisting`.
   ```
     if (adapter_->size() >= kBatchSize) {
       ICEBERG_ASSIGN_OR_RAISE(auto array, adapter_->FinishAppending());
       ICEBERG_RETURN_UNEXPECTED(writer_->Write(array));
       ICEBERG_RETURN_UNEXPECTED(adapter_->StartAppending());
     }
     auto copy = entry.Copy();
     ```



##########
src/iceberg/manifest_writer.h:
##########
@@ -103,6 +124,10 @@ class ICEBERG_EXPORT ManifestWriter {
   static constexpr int64_t kBatchSize = 1024;
   std::unique_ptr<Writer> writer_;
   std::unique_ptr<ManifestEntryAdapter> adapter_;
+  bool closed_{false};
+  std::string manifest_location_;
+  // TODO(zhjwpku): consider passing key metadata when makeing ManifestWriter.

Review Comment:
   Please file an issue and use the tracking id instead of your id. I know 
other code already used personal id, but it's not helpful to follow what to do.



##########
src/iceberg/manifest_list.h:
##########
@@ -218,21 +224,21 @@ struct ICEBERG_EXPORT ManifestList {
 };
 
 /// \brief Get the relative manifest content type name
-ICEBERG_EXPORT constexpr std::string_view ToString(ManifestFile::Content type) 
noexcept {
+ICEBERG_EXPORT constexpr std::string_view ToString(ManifestContent type) 
noexcept {

Review Comment:
   Please make it a static function.



##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {
+ public:
+  explicit PartitionFieldStats(const std::shared_ptr<Type>& type) : 
type_(type) {}
+
+  Status Update(const Literal& value);
+
+  PartitionFieldSummary Finish() const;
+
+ private:
+  std::shared_ptr<Type> type_{nullptr};
+  bool contains_null_{false};
+  bool contains_nan_{false};
+  std::optional<Literal> lower_bound_;
+  std::optional<Literal> upper_bound_;
+};
+
+class PartitionSummary {

Review Comment:
   Missing the class description.



##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {
+ public:
+  explicit PartitionFieldStats(const std::shared_ptr<Type>& type) : 
type_(type) {}
+
+  Status Update(const Literal& value);
+
+  PartitionFieldSummary Finish() const;

Review Comment:
   Please add a comment.



##########
src/iceberg/manifest_list.h:
##########
@@ -218,21 +224,21 @@ struct ICEBERG_EXPORT ManifestList {
 };
 
 /// \brief Get the relative manifest content type name
-ICEBERG_EXPORT constexpr std::string_view ToString(ManifestFile::Content type) 
noexcept {
+ICEBERG_EXPORT constexpr std::string_view ToString(ManifestContent type) 
noexcept {
   switch (type) {
-    case ManifestFile::Content::kData:
+    case ManifestContent::kData:
       return "data";
-    case ManifestFile::Content::kDeletes:
+    case ManifestContent::kDeletes:
       return "deletes";
   }
   std::unreachable();
 }
 
 /// \brief Get the relative manifest content type from name
-ICEBERG_EXPORT constexpr Result<ManifestFile::Content> 
ManifestFileContentFromString(
+ICEBERG_EXPORT constexpr Result<ManifestContent> ManifestContentFromString(

Review Comment:
   Please make it a static function.



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");

Review Comment:
   Please put more details such as actual and expected type ids.



##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {

Review Comment:
   Missing the class description.



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {

Review Comment:
   Missing `[[unlikely]]`.



##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {
+ public:
+  explicit PartitionFieldStats(const std::shared_ptr<Type>& type) : 
type_(type) {}
+
+  Status Update(const Literal& value);

Review Comment:
   Please add a comment.



##########
src/iceberg/v3_metadata.cc:
##########
@@ -189,7 +188,12 @@ Status ManifestFileAdapterV3::Init() {
   return ToArrowSchema(*manifest_list_schema_, &schema_);
 }
 
-Status ManifestFileAdapterV3::Append(const ManifestFile& file) {
+Status ManifestFileAdapterV3::Append(ManifestFile& file) {
+  // TODO(zhjwpku): Should we set sequence_number/first_row_id here?

Review Comment:
   Please file an issue and use the tracking id instead of your id. I know 
other code already used personal id, but it's not helpful to follow what to do.
   



##########
src/iceberg/v2_metadata.cc:
##########
@@ -154,7 +151,9 @@ Status ManifestFileAdapterV2::Init() {
   return ToArrowSchema(*manifest_list_schema_, &schema_);
 }
 
-Status ManifestFileAdapterV2::Append(const ManifestFile& file) {
+Status ManifestFileAdapterV2::Append(ManifestFile& file) {
+  // TODO(zhjwpku): Should we set sequence_number here?

Review Comment:
   Please file an issue and use the tracking id instead of your id. I know 
other code already used personal id, but it's not helpful to follow what to do.
   



##########
src/iceberg/test/manifest_reader_writer_test.cc:
##########
@@ -66,7 +68,20 @@ class ManifestReaderWriterTestBase : public TempFileTestBase 
{
     auto read_result = manifest_reader->Entries();
     ASSERT_TRUE(read_result.has_value()) << read_result.error().message;
     ASSERT_EQ(read_result.value().size(), expected_entries.size());
-    ASSERT_EQ(read_result.value(), expected_entries);
+
+    // Make a copy of expected_entries but set snapshot_id if provided
+    if (snapshot_id.has_value()) {
+      std::vector<ManifestEntry> adjusted_expected_entries;
+      for (const auto& entry : expected_entries) {
+        ManifestEntry copy = entry;
+        copy.snapshot_id = snapshot_id;
+        adjusted_expected_entries.push_back(copy);
+      }
+      ASSERT_EQ(read_result.value(), adjusted_expected_entries);
+      return;

Review Comment:
   Why should we stop here?



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to