xudong963 commented on code in PR #18817:
URL: https://github.com/apache/datafusion/pull/18817#discussion_r2576074919
##########
datafusion/datasource-parquet/src/opener.rs:
##########
@@ -752,10 +799,316 @@ fn should_enable_page_index(
.unwrap_or(false)
}
+fn reverse_batch(batch: RecordBatch) -> Result<RecordBatch> {
+ let num_rows = batch.num_rows();
+ if num_rows <= 1 {
+ return Ok(batch);
+ }
+
+ let indices = UInt32Array::from_iter_values((0..num_rows as u32).rev());
+
+ let reversed_columns = batch
+ .columns()
+ .iter()
+ .map(|col| take(col.as_ref(), &indices, None))
+ .collect::<std::result::Result<Vec<ArrayRef>,
arrow::error::ArrowError>>()
+ .map_err(DataFusionError::from)?;
+
+ RecordBatch::try_new(batch.schema(),
reversed_columns).map_err(DataFusionError::from)
+}
+
+/// Stream adapter for reversed parquet reading with row-group-level buffering.
+///
+/// # Architecture
+///
+/// This stream implements a sophisticated buffering strategy to achieve true
reverse
+/// reading of Parquet files while maintaining compatibility with the
underlying
+/// ParquetRecordBatchStream's optimizations (caching, prefetching, etc.).
+///
+/// ## Strategy Overview
+///
+/// 1. **Pre-reversed Row Groups**: Row groups are reversed BEFORE building
the stream
+/// (via `row_group_indexes.reverse()`). This allows the Parquet reader to
read
+/// them in reverse order while still utilizing internal optimizations.
+///
+/// 2. **Row-Group-Level Buffering**: As batches arrive from the input stream,
we
+/// track which row group they belong to using cumulative row counts. This
is
+/// the MINIMAL buffering unit required for correctness - we cannot reverse
+/// individual batches without knowing the complete row group context.
+///
+/// 3. **Two-Stage Reversal**: When a complete row group is collected:
+/// - Stage 1: Reverse rows within each batch (using Arrow's take kernel)
+/// - Stage 2: Reverse the order of batches within the row group
+///
+/// 4. **Progressive Output**: Reversed batches are output immediately,
minimizing
+/// memory footprint. We never buffer more than one row group at a time.
+///
+/// ## Memory Characteristics
+///
+/// - **Bounded Memory**: Maximum memory usage = size of largest row group
+/// - **Typical Usage**: ~128MB (default Parquet row group size)
+/// - **Peak Usage**: During reversal of a single row group
+///
+/// ## Why Row-Group-Level Buffering is Necessary
+///
+/// Parquet organizes data into row groups (typically 128MB each), and each
row group
+/// is independently compressed and encoded. When reading in reverse:
+///
+/// - We cannot reverse individual batches in isolation because they may span
+/// row group boundaries or be split arbitrarily by the batch_size parameter
+/// - We must buffer complete row groups to ensure correct ordering semantics
+/// - This is the minimum granularity that maintains correctness
+///
+/// ## Example
+///
+/// Given a file with 3 row groups, each containing 2 batches:
+///
+/// ```text
+/// Normal order: RG0[B0, B1] -> RG1[B0, B1] -> RG2[B0, B1]
+/// Reversed: RG2[B1_rev, B0_rev] -> RG1[B1_rev, B0_rev] -> RG0[B1_rev,
B0_rev]
+/// ^^^^^^^^^^^^ ^^^^^^^^^^^^ ^^^^^^^^^^^^
+/// Output 1st Output 2nd Output 3rd
+/// ```
+///
+/// ## Performance Characteristics
+///
+/// - **Latency**: First batch available after reading complete first
(reversed) row group
+/// - **Throughput**: Near-native speed with ~5-10% overhead for reversal
operations
Review Comment:
How did you get the "~5-10% overhead"?
##########
datafusion/datasource-parquet/src/opener.rs:
##########
@@ -752,10 +799,316 @@ fn should_enable_page_index(
.unwrap_or(false)
}
+fn reverse_batch(batch: RecordBatch) -> Result<RecordBatch> {
+ let num_rows = batch.num_rows();
+ if num_rows <= 1 {
+ return Ok(batch);
+ }
+
+ let indices = UInt32Array::from_iter_values((0..num_rows as u32).rev());
+
+ let reversed_columns = batch
+ .columns()
+ .iter()
+ .map(|col| take(col.as_ref(), &indices, None))
+ .collect::<std::result::Result<Vec<ArrayRef>,
arrow::error::ArrowError>>()
+ .map_err(DataFusionError::from)?;
+
+ RecordBatch::try_new(batch.schema(),
reversed_columns).map_err(DataFusionError::from)
+}
+
+/// Stream adapter for reversed parquet reading with row-group-level buffering.
+///
+/// # Architecture
+///
+/// This stream implements a sophisticated buffering strategy to achieve true
reverse
+/// reading of Parquet files while maintaining compatibility with the
underlying
+/// ParquetRecordBatchStream's optimizations (caching, prefetching, etc.).
+///
+/// ## Strategy Overview
+///
+/// 1. **Pre-reversed Row Groups**: Row groups are reversed BEFORE building
the stream
+/// (via `row_group_indexes.reverse()`). This allows the Parquet reader to
read
+/// them in reverse order while still utilizing internal optimizations.
+///
+/// 2. **Row-Group-Level Buffering**: As batches arrive from the input stream,
we
+/// track which row group they belong to using cumulative row counts. This
is
+/// the MINIMAL buffering unit required for correctness - we cannot reverse
+/// individual batches without knowing the complete row group context.
+///
+/// 3. **Two-Stage Reversal**: When a complete row group is collected:
+/// - Stage 1: Reverse rows within each batch (using Arrow's take kernel)
+/// - Stage 2: Reverse the order of batches within the row group
+///
+/// 4. **Progressive Output**: Reversed batches are output immediately,
minimizing
+/// memory footprint. We never buffer more than one row group at a time.
+///
+/// ## Memory Characteristics
+///
+/// - **Bounded Memory**: Maximum memory usage = size of largest row group
+/// - **Typical Usage**: ~128MB (default Parquet row group size)
+/// - **Peak Usage**: During reversal of a single row group
+///
+/// ## Why Row-Group-Level Buffering is Necessary
+///
+/// Parquet organizes data into row groups (typically 128MB each), and each
row group
+/// is independently compressed and encoded. When reading in reverse:
+///
+/// - We cannot reverse individual batches in isolation because they may span
+/// row group boundaries or be split arbitrarily by the batch_size parameter
+/// - We must buffer complete row groups to ensure correct ordering semantics
+/// - This is the minimum granularity that maintains correctness
+///
+/// ## Example
+///
+/// Given a file with 3 row groups, each containing 2 batches:
+///
+/// ```text
+/// Normal order: RG0[B0, B1] -> RG1[B0, B1] -> RG2[B0, B1]
+/// Reversed: RG2[B1_rev, B0_rev] -> RG1[B1_rev, B0_rev] -> RG0[B1_rev,
B0_rev]
+/// ^^^^^^^^^^^^ ^^^^^^^^^^^^ ^^^^^^^^^^^^
+/// Output 1st Output 2nd Output 3rd
+/// ```
+///
+/// ## Performance Characteristics
+///
+/// - **Latency**: First batch available after reading complete first
(reversed) row group
+/// - **Throughput**: Near-native speed with ~5-10% overhead for reversal
operations
+/// - **Memory**: O(row_group_size), not O(file_size)
+/// TODO should we support max cache size to limit memory usage further? But
if we exceed the cache size we can't reverse properly, so we need to fall back
to normal reading?
+struct ReversedParquetStream<S> {
+ input: S,
+ schema: SchemaRef,
+
+ // Optional limit on the number of rows to output
+ limit: Option<usize>,
+ rows_produced: usize,
+
+ // Current row group being processed
+ current_rg_batches: Vec<RecordBatch>,
+ current_rg_rows_read: usize,
+ current_rg_total_rows: usize,
+
+ // Output buffer for reversed batches
+ output_batches: Vec<RecordBatch>,
+ output_index: usize,
+
+ // Row group metadata (each element is the number of rows in that row
group)
+ // Already in reverse order since row_group_indexes was reversed
+ row_group_metadata: Vec<usize>,
+ current_rg_index: usize,
+
+ done: bool,
+
+ // Metrics
+ row_groups_reversed: Count,
+ batches_reversed: Count,
+ reverse_time: Time,
+
+ /// Pending error from batch reversal
+ pending_error: Option<DataFusionError>,
+}
+
+impl<S> ReversedParquetStream<S> {
+ fn new(
+ stream: S,
+ schema: SchemaRef,
+ row_group_metadata: Vec<usize>,
+ row_groups_reversed: Count,
+ batches_reversed: Count,
+ reverse_time: Time,
+ limit: Option<usize>,
+ ) -> Self {
+ let current_rg_total_rows =
row_group_metadata.first().copied().unwrap_or(0);
+
+ Self {
+ input: stream,
+ schema,
+ limit,
+ rows_produced: 0,
+ current_rg_batches: Vec::new(),
+ current_rg_rows_read: 0,
+ current_rg_total_rows,
+ output_batches: Vec::new(),
+ output_index: 0,
+ row_group_metadata,
+ current_rg_index: 0,
+ done: false,
+ row_groups_reversed,
+ batches_reversed,
+ reverse_time,
+ pending_error: None,
+ }
+ }
+
+ /// Finalizes the current row group by performing the two-stage reversal.
+ ///
+ /// This is called when we've accumulated all batches for a row group.
+ ///
+ /// # Two-Stage Reversal Process
+ ///
+ /// 1. **Stage 1 - Reverse Rows**: For each batch in the row group, reverse
+ /// the order of rows using Arrow's `take` kernel with reversed indices.
+ ///
+ /// 2. **Stage 2 - Reverse Batches**: Reverse the order of batches within
+ /// the row group so that the last batch becomes first.
+ ///
+ /// # Example
+ ///
+ /// Input batches for a row group:
+ /// ```text
+ /// B0: [row0, row1, row2]
+ /// B1: [row3, row4, row5]
+ /// ```
+ ///
+ /// After Stage 1 (reverse rows within each batch):
+ /// ```text
+ /// B0_rev: [row2, row1, row0]
+ /// B1_rev: [row5, row4, row3]
+ /// ```
+ ///
+ /// After Stage 2 (reverse batch order):
+ /// ```text
+ /// Output: B1_rev, B0_rev
+ /// Final sequence: [row5, row4, row3, row2, row1, row0]
+ /// ```
+ fn finalize_current_row_group(&mut self) {
+ if self.current_rg_batches.is_empty() {
+ return;
+ }
+
+ // Start timing
+ let _timer = self.reverse_time.timer();
+ let batch_count = self.current_rg_batches.len();
+
+ // Step 1: Reverse rows within each batch
+ let mut reversed_batches =
Vec::with_capacity(self.current_rg_batches.len());
+ for batch in self.current_rg_batches.drain(..) {
+ match reverse_batch(batch) {
+ Ok(reversed) => reversed_batches.push(reversed),
+ Err(e) => {
+ // Store error and return it on next poll
+ self.pending_error = Some(e);
+ return;
+ }
+ }
+ }
+
+ // Step 2: Reverse the order of batches
+ self.output_batches = reversed_batches.into_iter().rev().collect();
+ self.output_index = 0;
+
+ // Update metrics
+ self.row_groups_reversed.add(1);
+ self.batches_reversed.add(batch_count);
+
+ // Prepare for next row group
+ self.current_rg_rows_read = 0;
+ self.current_rg_index += 1;
+ self.current_rg_total_rows = self
+ .row_group_metadata
+ .get(self.current_rg_index)
+ .copied()
+ .unwrap_or(0);
+ }
+
+ /// Check if we've reached the limit
+ #[inline]
+ fn is_limit_reached(&self) -> bool {
+ self.limit.is_some_and(|limit| self.rows_produced >= limit)
+ }
+}
+
+impl<S> Stream for ReversedParquetStream<S>
+where
+ S: Stream<Item = Result<RecordBatch>> + Unpin,
+{
+ type Item = Result<RecordBatch>;
+
+ fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) ->
Poll<Option<Self::Item>> {
+ let this = self.get_mut(); // Safe: We own the Pin and
ReversedParquetStream is Unpin
+
+ // Check for pending errors first
+ if let Some(err) = this.pending_error.take() {
+ return Poll::Ready(Some(Err(err)));
+ }
+
+ if this.done || this.is_limit_reached() {
+ return Poll::Ready(None);
+ }
+
+ loop {
+ // First, output any ready batches
+ if this.output_index < this.output_batches.len() {
+ let batch = this.output_batches[this.output_index].clone();
+ this.output_index += 1;
+
+ // Apply limit if specified
+ if let Some(limit) = this.limit {
+ let remaining = limit.saturating_sub(this.rows_produced);
+
+ if batch.num_rows() <= remaining {
+ this.rows_produced += batch.num_rows();
+ return Poll::Ready(Some(Ok(batch)));
+ } else {
+ // Slice batch to fit within limit
+ let sliced = batch.slice(0, remaining);
+ this.rows_produced += remaining;
+ this.done = true;
+ return Poll::Ready(Some(Ok(sliced)));
+ }
+ } else {
+ // No limit, return full batch
+ return Poll::Ready(Some(Ok(batch)));
+ }
+ }
+
+ // Need to read more data
+ match ready!(Pin::new(&mut this.input).poll_next(cx)) {
+ Some(Ok(batch)) => {
+ let batch_rows = batch.num_rows();
+ this.current_rg_batches.push(batch);
+ this.current_rg_rows_read += batch_rows;
+
+ // Check if current row group is complete
+ if this.current_rg_rows_read >= this.current_rg_total_rows
{
+ this.finalize_current_row_group();
Review Comment:
When `finalize_current_row_group()` encounters an error (e.g., during
`reverse_batch`), it sets `self.pending_error` and returns early. However, it
is called inside the `loop` in `poll_next`. Here does NOT check if
pending_error was set.
##########
datafusion/datasource-parquet/src/opener.rs:
##########
@@ -752,10 +799,316 @@ fn should_enable_page_index(
.unwrap_or(false)
}
+fn reverse_batch(batch: RecordBatch) -> Result<RecordBatch> {
+ let num_rows = batch.num_rows();
+ if num_rows <= 1 {
+ return Ok(batch);
+ }
+
+ let indices = UInt32Array::from_iter_values((0..num_rows as u32).rev());
+
+ let reversed_columns = batch
+ .columns()
+ .iter()
+ .map(|col| take(col.as_ref(), &indices, None))
+ .collect::<std::result::Result<Vec<ArrayRef>,
arrow::error::ArrowError>>()
+ .map_err(DataFusionError::from)?;
+
+ RecordBatch::try_new(batch.schema(),
reversed_columns).map_err(DataFusionError::from)
+}
+
+/// Stream adapter for reversed parquet reading with row-group-level buffering.
+///
+/// # Architecture
+///
+/// This stream implements a sophisticated buffering strategy to achieve true
reverse
+/// reading of Parquet files while maintaining compatibility with the
underlying
+/// ParquetRecordBatchStream's optimizations (caching, prefetching, etc.).
+///
+/// ## Strategy Overview
+///
+/// 1. **Pre-reversed Row Groups**: Row groups are reversed BEFORE building
the stream
+/// (via `row_group_indexes.reverse()`). This allows the Parquet reader to
read
+/// them in reverse order while still utilizing internal optimizations.
+///
+/// 2. **Row-Group-Level Buffering**: As batches arrive from the input stream,
we
+/// track which row group they belong to using cumulative row counts. This
is
+/// the MINIMAL buffering unit required for correctness - we cannot reverse
+/// individual batches without knowing the complete row group context.
+///
+/// 3. **Two-Stage Reversal**: When a complete row group is collected:
+/// - Stage 1: Reverse rows within each batch (using Arrow's take kernel)
+/// - Stage 2: Reverse the order of batches within the row group
+///
+/// 4. **Progressive Output**: Reversed batches are output immediately,
minimizing
+/// memory footprint. We never buffer more than one row group at a time.
+///
+/// ## Memory Characteristics
+///
+/// - **Bounded Memory**: Maximum memory usage = size of largest row group
+/// - **Typical Usage**: ~128MB (default Parquet row group size)
+/// - **Peak Usage**: During reversal of a single row group
+///
+/// ## Why Row-Group-Level Buffering is Necessary
+///
+/// Parquet organizes data into row groups (typically 128MB each), and each
row group
+/// is independently compressed and encoded. When reading in reverse:
+///
+/// - We cannot reverse individual batches in isolation because they may span
+/// row group boundaries or be split arbitrarily by the batch_size parameter
+/// - We must buffer complete row groups to ensure correct ordering semantics
+/// - This is the minimum granularity that maintains correctness
+///
+/// ## Example
+///
+/// Given a file with 3 row groups, each containing 2 batches:
+///
+/// ```text
+/// Normal order: RG0[B0, B1] -> RG1[B0, B1] -> RG2[B0, B1]
+/// Reversed: RG2[B1_rev, B0_rev] -> RG1[B1_rev, B0_rev] -> RG0[B1_rev,
B0_rev]
+/// ^^^^^^^^^^^^ ^^^^^^^^^^^^ ^^^^^^^^^^^^
+/// Output 1st Output 2nd Output 3rd
+/// ```
+///
+/// ## Performance Characteristics
+///
+/// - **Latency**: First batch available after reading complete first
(reversed) row group
+/// - **Throughput**: Near-native speed with ~5-10% overhead for reversal
operations
+/// - **Memory**: O(row_group_size), not O(file_size)
Review Comment:
I'm a bit confused about the line, what does it mean?
##########
datafusion/datasource/src/file_scan_config.rs:
##########
@@ -766,6 +766,131 @@ impl DataSource for FileScanConfig {
}
}
}
+
+ fn try_pushdown_sort(
+ &self,
+ order: &[PhysicalSortExpr],
+ ) -> Result<Option<Arc<dyn DataSource>>> {
+ let current_ordering = match self.output_ordering.first() {
+ Some(ordering) => ordering.as_ref(),
+ None => return Ok(None),
+ };
+
+ // Only support reverse ordering pushdown
+ if !is_reverse_ordering(order, current_ordering) {
+ return Ok(None);
+ }
+
+ // Ask the file source if it can handle the sort pushdown
+ // (e.g., ParquetSource will enable reverse_scan)
+ let new_file_source = match self.file_source.try_pushdown_sort(order)?
{
+ Some(source) => source,
+ None => return Ok(None),
+ };
+
+ let mut new_config = self.clone();
+
+ // Reverse file groups: when scanning in reverse, we need to read files
+ // in reverse order to maintain the correct global ordering
+ new_config.file_groups = new_config
+ .file_groups
+ .into_iter()
+ .map(|group| {
+ let mut files = group.into_inner();
+ files.reverse();
+ files.into()
+ })
+ .collect();
+
+ // Build the new output ordering by reversing each sort expression's
direction
+ // E.g., [number DESC] becomes [number ASC]
+ let mut reversed_ordering = Vec::new();
+ for sort_expr in current_ordering {
+ reversed_ordering.push(PhysicalSortExpr {
+ expr: Arc::clone(&sort_expr.expr),
+ options: !sort_expr.options,
+ });
+ }
+
+ new_config.output_ordering = vec![LexOrdering::new(reversed_ordering)
+ .ok_or_else(|| {
+ DataFusionError::Plan(
+ "Failed to create ordering: invalid sort
expressions".to_string(),
+ )
+ })?];
+
+ new_config.file_source = new_file_source;
+
+ Ok(Some(Arc::new(new_config)))
+ }
+}
+
+/// Check if the requested ordering can be satisfied by reversing the current
ordering.
+///
+/// This function supports **prefix matching**: if the file has ordering [A
DESC, B ASC]
+/// and we need [A ASC], reversing the scan gives us [A ASC, B DESC], which
satisfies
+/// the requirement since [A ASC] is a prefix.
+///
+/// # Arguments
+/// * `requested` - The ordering required by the query
+/// * `current` - The natural ordering of the data source (e.g., from file
metadata)
+///
+/// # Returns
+/// `true` if reversing the current ordering would satisfy the requested
ordering
+///
+/// # Example
+/// ```text
+/// Current: [number DESC, letter ASC]
+/// Requested: [number ASC]
+/// Reversed: [number ASC, letter DESC] ✓ Prefix match!
+/// ```
+fn is_reverse_ordering(
Review Comment:
I'm wondering if we could leverage the ordering equivalence to do the check
not by string match.
##########
datafusion/datasource-parquet/src/opener.rs:
##########
@@ -752,10 +799,316 @@ fn should_enable_page_index(
.unwrap_or(false)
}
+fn reverse_batch(batch: RecordBatch) -> Result<RecordBatch> {
+ let num_rows = batch.num_rows();
+ if num_rows <= 1 {
+ return Ok(batch);
+ }
+
+ let indices = UInt32Array::from_iter_values((0..num_rows as u32).rev());
Review Comment:
The `UInt32Array` is allocated for every batch, not sure if it could be a
potential cost, if it is, a cache may be good
##########
datafusion/datasource-parquet/src/opener.rs:
##########
@@ -752,10 +799,316 @@ fn should_enable_page_index(
.unwrap_or(false)
}
+fn reverse_batch(batch: RecordBatch) -> Result<RecordBatch> {
+ let num_rows = batch.num_rows();
+ if num_rows <= 1 {
+ return Ok(batch);
+ }
+
+ let indices = UInt32Array::from_iter_values((0..num_rows as u32).rev());
+
+ let reversed_columns = batch
+ .columns()
+ .iter()
+ .map(|col| take(col.as_ref(), &indices, None))
+ .collect::<std::result::Result<Vec<ArrayRef>,
arrow::error::ArrowError>>()
+ .map_err(DataFusionError::from)?;
+
+ RecordBatch::try_new(batch.schema(),
reversed_columns).map_err(DataFusionError::from)
+}
+
+/// Stream adapter for reversed parquet reading with row-group-level buffering.
+///
+/// # Architecture
+///
+/// This stream implements a sophisticated buffering strategy to achieve true
reverse
+/// reading of Parquet files while maintaining compatibility with the
underlying
+/// ParquetRecordBatchStream's optimizations (caching, prefetching, etc.).
+///
+/// ## Strategy Overview
+///
+/// 1. **Pre-reversed Row Groups**: Row groups are reversed BEFORE building
the stream
+/// (via `row_group_indexes.reverse()`). This allows the Parquet reader to
read
+/// them in reverse order while still utilizing internal optimizations.
+///
+/// 2. **Row-Group-Level Buffering**: As batches arrive from the input stream,
we
+/// track which row group they belong to using cumulative row counts. This
is
+/// the MINIMAL buffering unit required for correctness - we cannot reverse
+/// individual batches without knowing the complete row group context.
+///
+/// 3. **Two-Stage Reversal**: When a complete row group is collected:
+/// - Stage 1: Reverse rows within each batch (using Arrow's take kernel)
+/// - Stage 2: Reverse the order of batches within the row group
+///
+/// 4. **Progressive Output**: Reversed batches are output immediately,
minimizing
+/// memory footprint. We never buffer more than one row group at a time.
+///
+/// ## Memory Characteristics
+///
+/// - **Bounded Memory**: Maximum memory usage = size of largest row group
+/// - **Typical Usage**: ~128MB (default Parquet row group size)
+/// - **Peak Usage**: During reversal of a single row group
+///
+/// ## Why Row-Group-Level Buffering is Necessary
+///
+/// Parquet organizes data into row groups (typically 128MB each), and each
row group
+/// is independently compressed and encoded. When reading in reverse:
+///
+/// - We cannot reverse individual batches in isolation because they may span
+/// row group boundaries or be split arbitrarily by the batch_size parameter
+/// - We must buffer complete row groups to ensure correct ordering semantics
+/// - This is the minimum granularity that maintains correctness
+///
+/// ## Example
+///
+/// Given a file with 3 row groups, each containing 2 batches:
+///
+/// ```text
+/// Normal order: RG0[B0, B1] -> RG1[B0, B1] -> RG2[B0, B1]
+/// Reversed: RG2[B1_rev, B0_rev] -> RG1[B1_rev, B0_rev] -> RG0[B1_rev,
B0_rev]
+/// ^^^^^^^^^^^^ ^^^^^^^^^^^^ ^^^^^^^^^^^^
+/// Output 1st Output 2nd Output 3rd
+/// ```
+///
+/// ## Performance Characteristics
+///
+/// - **Latency**: First batch available after reading complete first
(reversed) row group
+/// - **Throughput**: Near-native speed with ~5-10% overhead for reversal
operations
+/// - **Memory**: O(row_group_size), not O(file_size)
+/// TODO should we support max cache size to limit memory usage further? But
if we exceed the cache size we can't reverse properly, so we need to fall back
to normal reading?
Review Comment:
Could you please open a issue to track and descrip the todo?
--
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]