westonpace commented on code in PR #13245:
URL: https://github.com/apache/arrow/pull/13245#discussion_r886245878


##########
cpp/src/arrow/ipc/reader.cc:
##########
@@ -1121,6 +1121,29 @@ class ARROW_EXPORT SelectiveIpcFileRecordBatchGenerator {
   int index_;
 };
 
+struct AtomicReadStats {
+  std::atomic<int64_t> num_messages{0};
+  std::atomic<int64_t> num_record_batches{0};
+  std::atomic<int64_t> num_dictionary_batches{0};
+  std::atomic<int64_t> num_dictionary_deltas{0};
+  std::atomic<int64_t> num_replaced_dictionaries{0};
+
+  /// \brief Capture a copy of the current counters
+  ///
+  /// It's possible to get inconsistent values.  For example, if
+  /// this method is called in the middle of a read you might have
+  /// a case where num_messages != num_record_batches + num_dictionary_batches

Review Comment:
   Looking at this further I'm more convinced this isn't needed.  For example, 
when reading rows, the number of messages will be incremented but the number of 
record batches will not.  So this isn't a very reliable invariant anyways.
   
   I went ahead and removed the comment itself to avoid confusion in the future.
   
   I'm still open to using a mutex instead of an atomic if we want to.  I only 
chose atomics for simplicity and not for performance.
   
   There does appear to be a slight impact to performance for very simple IPC 
reads from buffers which is unfortunate:
   
   Before:
   ```
   ReadBuffer/num_cols:1/is_partial:0/real_time_mean         2372 ns         
2372 ns          100 bytes_per_second=411.778G/s
   ```
   After:
   ```
   ReadBuffer/num_cols:1/is_partial:0/real_time_mean         2525 ns         
2525 ns          100 bytes_per_second=386.79G/s
   ```



-- 
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: github-unsubscr...@arrow.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to