This is an automated email from the ASF dual-hosted git repository.

tustvold pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-rs.git


The following commit(s) were added to refs/heads/master by this push:
     new 9f889aa3df Fix test feature selection so all feature combinations work 
as expected (#6626)
9f889aa3df is described below

commit 9f889aa3df225de3034ea9905d974a6661820bb5
Author: June <[email protected]>
AuthorDate: Tue Oct 29 02:49:54 2024 -0600

    Fix test feature selection so all feature combinations work as expected 
(#6626)
    
    * Fix test feature selection so all feature combinations work as expected
    
    * Consolidate cfg
---
 parquet/src/file/metadata/reader.rs    | 17 ++++++++---------
 parquet/tests/arrow_reader/bad_data.rs |  1 +
 parquet/tests/arrow_reader/checksum.rs |  6 ++++--
 3 files changed, 13 insertions(+), 11 deletions(-)

diff --git a/parquet/src/file/metadata/reader.rs 
b/parquet/src/file/metadata/reader.rs
index c803d6013d..2a927f15fb 100644
--- a/parquet/src/file/metadata/reader.rs
+++ b/parquet/src/file/metadata/reader.rs
@@ -31,7 +31,7 @@ use crate::schema::types;
 use crate::schema::types::SchemaDescriptor;
 use crate::thrift::{TCompactSliceInputProtocol, TSerializable};
 
-#[cfg(feature = "async")]
+#[cfg(all(feature = "async", feature = "arrow"))]
 use crate::arrow::async_reader::MetadataFetch;
 
 /// Reads the [`ParquetMetaData`] from a byte stream.
@@ -321,7 +321,7 @@ impl ParquetMetaDataReader {
     ///
     /// See [`Self::with_prefetch_hint`] for a discussion of how to reduce the 
number of fetches
     /// performed by this function.
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     pub async fn load_and_finish<F: MetadataFetch>(
         mut self,
         fetch: F,
@@ -336,7 +336,7 @@ impl ParquetMetaDataReader {
     ///
     /// See [`Self::with_prefetch_hint`] for a discussion of how to reduce the 
number of fetches
     /// performed by this function.
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     pub async fn try_load<F: MetadataFetch>(
         &mut self,
         mut fetch: F,
@@ -357,12 +357,12 @@ impl ParquetMetaDataReader {
 
     /// Asynchronously fetch the page index structures when a 
[`ParquetMetaData`] has already
     /// been obtained. See [`Self::new_with_metadata()`].
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     pub async fn load_page_index<F: MetadataFetch>(&mut self, fetch: F) -> 
Result<()> {
         self.load_page_index_with_remainder(fetch, None).await
     }
 
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     async fn load_page_index_with_remainder<F: MetadataFetch>(
         &mut self,
         mut fetch: F,
@@ -513,7 +513,7 @@ impl ParquetMetaDataReader {
     /// Return the number of bytes to read in the initial pass. If 
`prefetch_size` has
     /// been provided, then return that value if it is larger than the size of 
the Parquet
     /// file footer (8 bytes). Otherwise returns `8`.
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     fn get_prefetch_size(&self) -> usize {
         if let Some(prefetch) = self.prefetch_hint {
             if prefetch > FOOTER_SIZE {
@@ -523,7 +523,7 @@ impl ParquetMetaDataReader {
         FOOTER_SIZE
     }
 
-    #[cfg(feature = "async")]
+    #[cfg(all(feature = "async", feature = "arrow"))]
     async fn load_metadata<F: MetadataFetch>(
         fetch: &mut F,
         file_size: usize,
@@ -851,8 +851,7 @@ mod tests {
     }
 }
 
-#[cfg(feature = "async")]
-#[cfg(test)]
+#[cfg(all(feature = "async", feature = "arrow", test))]
 mod async_tests {
     use super::*;
     use bytes::Bytes;
diff --git a/parquet/tests/arrow_reader/bad_data.rs 
b/parquet/tests/arrow_reader/bad_data.rs
index e2975c17c8..7434203143 100644
--- a/parquet/tests/arrow_reader/bad_data.rs
+++ b/parquet/tests/arrow_reader/bad_data.rs
@@ -111,6 +111,7 @@ fn test_arrow_rs_gh_6229_dict_header() {
 }
 
 #[test]
+#[cfg(feature = "snap")]
 fn test_arrow_rs_gh_6229_dict_levels() {
     let err = read_file("ARROW-RS-GH-6229-LEVELS.parquet").unwrap_err();
     assert_eq!(
diff --git a/parquet/tests/arrow_reader/checksum.rs 
b/parquet/tests/arrow_reader/checksum.rs
index c60908d8b9..b500b7cb1d 100644
--- a/parquet/tests/arrow_reader/checksum.rs
+++ b/parquet/tests/arrow_reader/checksum.rs
@@ -26,10 +26,10 @@ use parquet::arrow::arrow_reader::ArrowReaderBuilder;
 fn test_datapage_v1_corrupt_checksum() {
     let errors = 
read_file_batch_errors("datapage_v1-corrupt-checksum.parquet");
     assert_eq!(errors, [
-        Err("Parquet argument error: Parquet error: Page CRC checksum 
mismatch".to_string()), 
+        Err("Parquet argument error: Parquet error: Page CRC checksum 
mismatch".to_string()),
         Ok(()),
         Ok(()),
-        Err("Parquet argument error: Parquet error: Page CRC checksum 
mismatch".to_string()), 
+        Err("Parquet argument error: Parquet error: Page CRC checksum 
mismatch".to_string()),
         Err("Parquet argument error: Parquet error: Not all children array 
length are the same!".to_string())
     ]);
 }
@@ -41,6 +41,7 @@ fn test_datapage_v1_uncompressed_checksum() {
 }
 
 #[test]
+#[cfg(feature = "snap")]
 fn test_datapage_v1_snappy_compressed_checksum() {
     let errors = 
read_file_batch_errors("datapage_v1-snappy-compressed-checksum.parquet");
     assert_eq!(errors, [Ok(()), Ok(()), Ok(()), Ok(()), Ok(())]);
@@ -52,6 +53,7 @@ fn test_plain_dict_uncompressed_checksum() {
     assert_eq!(errors, [Ok(())]);
 }
 #[test]
+#[cfg(feature = "snap")]
 fn test_rle_dict_snappy_checksum() {
     let errors = read_file_batch_errors("rle-dict-snappy-checksum.parquet");
     assert_eq!(errors, [Ok(())]);

Reply via email to