You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/10/08 10:54:56 UTC

[GitHub] [arrow] alamb opened a new pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

alamb opened a new pull request #8397:
URL: https://github.com/apache/arrow/pull/8397


   
   This PR makes `array_value_to_string` available to all arrow builds. Currently it is only available if the `feature = "prettyprint"` is enabled. The full `print_batches` and `pretty_format_batches` (and the libraries they depend on) are still only available of the feature flag is set.
   
   The rationale for making this change is that I want to be able to use `array_value_to_string` to write tests (such as on https://github.com/apache/arrow/pull/8346) but currently it is only available when `feature = "prettyprint"` is enabled.
   
   It appears that @nevi-mi made prettyprint compilation optional so that arrow could be compiled for wasm in https://github.com/apache/arrow/pull/7400. https://issues.apache.org/jira/browse/ARROW-9088 explains that this is due to some dependency of pretty-table;   `array_value_to_string` has no needed dependencies.
   
   Note I tried to compile ARROW again using the `wasm32-unknown-unknown` target on master and it fails (perhaps due to a new dependency that was added?): 
   
   <details>
     <summary>Click to expand!</summary>
   
   ```
   alamb@ip-192-168-0-182 rust % git log | head -n 1
   git log | head -n 1
   commit d4cbc4b7aab5d37262b83e972af4bd7cb44c7a5c
   alamb@ip-192-168-0-182 rust % git status
   git status
   On branch master
   Your branch is up to date with 'upstream/master'.
   
   nothing to commit, working tree clean
   alamb@ip-192-168-0-182 rust % 
   
   alamb@ip-192-168-0-182 rust % cargo build --target=wasm32-unknown-unknown
   cargo build --target=wasm32-unknown-unknown
      Compiling cfg-if v0.1.10
      Compiling lazy_static v1.4.0
      Compiling futures-core v0.3.5
      Compiling slab v0.4.2
      Compiling futures-sink v0.3.5
      Compiling once_cell v1.4.0
      Compiling pin-utils v0.1.0
      Compiling futures-io v0.3.5
      Compiling itoa v0.4.5
      Compiling bytes v0.5.4
      Compiling fnv v1.0.7
      Compiling iovec v0.1.4
      Compiling unicode-width v0.1.7
      Compiling pin-project-lite v0.1.7
      Compiling ppv-lite86 v0.2.8
      Compiling atty v0.2.14
      Compiling dirs v1.0.5
      Compiling smallvec v1.4.0
      Compiling regex-syntax v0.6.18
      Compiling encode_unicode v0.3.6
      Compiling hex v0.4.2
      Compiling tower-service v0.3.0
   error[E0433]: failed to resolve: could not find `unix` in `os`
     --> /Users/alamb/.cargo/registry/src/github.com-1ecc6299db9ec823/dirs-1.0.5/src/lin.rs:41:18
      |
   41 |     use std::os::unix::ffi::OsStringExt;
      |                  ^^^^ could not find `unix` in `os`
   
   error[E0432]: unresolved import `unix`
    --> /Users/alamb/.cargo/registry/src/github.com-1ecc6299db9ec823/dirs-1.0.5/src/lin.rs:6:5
     |
   6 | use unix;
     |     ^^^^ no `unix` in the root
   
      Compiling alloc-no-stdlib v2.0.1
      Compiling adler32 v1.0.4
   error[E0599]: no function or associated item named `from_vec` found for struct `std::ffi::OsString` in the current scope
     --> /Users/alamb/.cargo/registry/src/github.com-1ecc6299db9ec823/dirs-1.0.5/src/lin.rs:48:34
      |
   48 |     Some(PathBuf::from(OsString::from_vec(out)))
      |                                  ^^^^^^^^ function or associated item not found in `std::ffi::OsString`
      |
      = help: items from traits can only be used if the trait is in scope
      = note: the following trait is implemented but not in scope; perhaps add a `use` for it:
              `use std::sys_common::os_str_bytes::OsStringExt;`
   
   error: aborting due to 3 previous errors
   
   Some errors have detailed explanations: E0432, E0433, E0599.
   For more information about an error, try `rustc --explain E0432`.
   error: could not compile `dirs`.
   
   To learn more, run the command again with --verbose.
   warning: build failed, waiting for other jobs to finish...
   error: build failed
   alamb@ip-192-168-0-182 rust % ```
   
   </details>
   


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

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



[GitHub] [arrow] alamb commented on a change in pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#discussion_r501717377



##########
File path: rust/arrow/src/util/display.rs
##########
@@ -0,0 +1,135 @@
+// 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.
+
+//! Functions for printing array values, as strings, for debugging
+//! purposes. See the `pretty` crate for additional functions for
+//! record batch pretty printing.
+
+use crate::array;
+use crate::array::{Array, PrimitiveArrayOps};
+use crate::datatypes::{
+    ArrowNativeType, ArrowPrimitiveType, DataType, Int16Type, Int32Type, Int64Type,
+    Int8Type, TimeUnit, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
+};
+
+use array::DictionaryArray;
+
+use crate::error::{ArrowError, Result};
+
+macro_rules! make_string {
+    ($array_type:ty, $column: ident, $row: ident) => {{
+        let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
+
+        let s = if array.is_null($row) {
+            "".to_string()
+        } else {
+            array.value($row).to_string()
+        };
+
+        Ok(s)
+    }};
+}
+
+/// Get the value at the given row in an array as a String.
+///
+/// Note this function is quite inefficient and is unlikely to be
+/// suitable for converting large arrays or record batches.
+pub fn array_value_to_string(column: &array::ArrayRef, row: usize) -> Result<String> {
+    match column.data_type() {
+        DataType::Utf8 => make_string!(array::StringArray, column, row),
+        DataType::Boolean => make_string!(array::BooleanArray, column, row),
+        DataType::Int8 => make_string!(array::Int8Array, column, row),
+        DataType::Int16 => make_string!(array::Int16Array, column, row),
+        DataType::Int32 => make_string!(array::Int32Array, column, row),
+        DataType::Int64 => make_string!(array::Int64Array, column, row),
+        DataType::UInt8 => make_string!(array::UInt8Array, column, row),
+        DataType::UInt16 => make_string!(array::UInt16Array, column, row),
+        DataType::UInt32 => make_string!(array::UInt32Array, column, row),
+        DataType::UInt64 => make_string!(array::UInt64Array, column, row),
+        DataType::Float16 => make_string!(array::Float32Array, column, row),
+        DataType::Float32 => make_string!(array::Float32Array, column, row),
+        DataType::Float64 => make_string!(array::Float64Array, column, row),
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Second => {
+            make_string!(array::TimestampSecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::TimestampMillisecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::TimestampMicrosecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::TimestampNanosecondArray, column, row)
+        }
+        DataType::Date32(_) => make_string!(array::Date32Array, column, row),
+        DataType::Date64(_) => make_string!(array::Date64Array, column, row),
+        DataType::Time32(unit) if *unit == TimeUnit::Second => {
+            make_string!(array::Time32SecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::Time32MillisecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::Time64MicrosecondArray, column, row)
+        }
+        DataType::Time64(unit) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::Time64NanosecondArray, column, row)
+        }
+        DataType::Dictionary(index_type, _value_type) => match **index_type {
+            DataType::Int8 => dict_array_value_to_string::<Int8Type>(column, row),
+            DataType::Int16 => dict_array_value_to_string::<Int16Type>(column, row),
+            DataType::Int32 => dict_array_value_to_string::<Int32Type>(column, row),
+            DataType::Int64 => dict_array_value_to_string::<Int64Type>(column, row),
+            DataType::UInt8 => dict_array_value_to_string::<UInt8Type>(column, row),
+            DataType::UInt16 => dict_array_value_to_string::<UInt16Type>(column, row),
+            DataType::UInt32 => dict_array_value_to_string::<UInt32Type>(column, row),
+            DataType::UInt64 => dict_array_value_to_string::<UInt64Type>(column, row),
+            _ => Err(ArrowError::InvalidArgumentError(format!(
+                "Pretty printing not supported for {:?} due to index type",
+                column.data_type()
+            ))),
+        },
+        _ => Err(ArrowError::InvalidArgumentError(format!(
+            "Pretty printing not implemented for {:?} type",
+            column.data_type()
+        ))),
+    }
+}
+
+/// Converts the value of the dictionary array at `row` to a String
+fn dict_array_value_to_string<K: ArrowPrimitiveType>(
+    colum: &array::ArrayRef,
+    row: usize,
+) -> Result<String> {
+    let dict_array = colum.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
+
+    let keys_array = dict_array.keys_array();
+
+    if keys_array.is_null(row) {
+        return Ok(String::from(""));
+    }
+
+    let dict_index = keys_array.value(row).to_usize().ok_or_else(|| {
+        ArrowError::InvalidArgumentError(format!(
+            "Can not convert value {:?} at index {:?} to usize for repl.",

Review comment:
       ```suggestion
               "Can not convert value {:?} at index {:?} to usize for string conversion.",
   ```




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

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



[GitHub] [arrow] jorgecarleitao closed pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
jorgecarleitao closed pull request #8397:
URL: https://github.com/apache/arrow/pull/8397


   


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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#issuecomment-705497672


   https://issues.apache.org/jira/browse/ARROW-10233


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

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



[GitHub] [arrow] nevi-me commented on a change in pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
nevi-me commented on a change in pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#discussion_r501678317



##########
File path: rust/arrow/src/util/display.rs
##########
@@ -0,0 +1,135 @@
+// 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.
+
+//! Functions for printing array values, as strings, for debugging
+//! purposes. See the `pretty` crate for additional functions for
+//! record batch pretty printing.
+
+use crate::array;
+use crate::array::{Array, PrimitiveArrayOps};
+use crate::datatypes::{
+    ArrowNativeType, ArrowPrimitiveType, DataType, Int16Type, Int32Type, Int64Type,
+    Int8Type, TimeUnit, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
+};
+
+use array::DictionaryArray;
+
+use crate::error::{ArrowError, Result};
+
+macro_rules! make_string {
+    ($array_type:ty, $column: ident, $row: ident) => {{
+        let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
+
+        let s = if array.is_null($row) {
+            "".to_string()
+        } else {
+            array.value($row).to_string()
+        };
+
+        Ok(s)
+    }};
+}
+
+/// Get the value at the given row in an array as a String.
+///
+/// Note this function is quite inefficient and is unlikely to be
+/// suitable for converting large arrays or record batches.
+pub fn array_value_to_string(column: &array::ArrayRef, row: usize) -> Result<String> {
+    match column.data_type() {
+        DataType::Utf8 => make_string!(array::StringArray, column, row),
+        DataType::Boolean => make_string!(array::BooleanArray, column, row),
+        DataType::Int8 => make_string!(array::Int8Array, column, row),
+        DataType::Int16 => make_string!(array::Int16Array, column, row),
+        DataType::Int32 => make_string!(array::Int32Array, column, row),
+        DataType::Int64 => make_string!(array::Int64Array, column, row),
+        DataType::UInt8 => make_string!(array::UInt8Array, column, row),
+        DataType::UInt16 => make_string!(array::UInt16Array, column, row),
+        DataType::UInt32 => make_string!(array::UInt32Array, column, row),
+        DataType::UInt64 => make_string!(array::UInt64Array, column, row),
+        DataType::Float16 => make_string!(array::Float32Array, column, row),
+        DataType::Float32 => make_string!(array::Float32Array, column, row),
+        DataType::Float64 => make_string!(array::Float64Array, column, row),
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Second => {
+            make_string!(array::TimestampSecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::TimestampMillisecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::TimestampMicrosecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::TimestampNanosecondArray, column, row)
+        }
+        DataType::Date32(_) => make_string!(array::Date32Array, column, row),
+        DataType::Date64(_) => make_string!(array::Date64Array, column, row),
+        DataType::Time32(unit) if *unit == TimeUnit::Second => {
+            make_string!(array::Time32SecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::Time32MillisecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::Time64MicrosecondArray, column, row)
+        }
+        DataType::Time64(unit) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::Time64NanosecondArray, column, row)
+        }
+        DataType::Dictionary(index_type, _value_type) => match **index_type {
+            DataType::Int8 => dict_array_value_to_string::<Int8Type>(column, row),
+            DataType::Int16 => dict_array_value_to_string::<Int16Type>(column, row),
+            DataType::Int32 => dict_array_value_to_string::<Int32Type>(column, row),
+            DataType::Int64 => dict_array_value_to_string::<Int64Type>(column, row),
+            DataType::UInt8 => dict_array_value_to_string::<UInt8Type>(column, row),
+            DataType::UInt16 => dict_array_value_to_string::<UInt16Type>(column, row),
+            DataType::UInt32 => dict_array_value_to_string::<UInt32Type>(column, row),
+            DataType::UInt64 => dict_array_value_to_string::<UInt64Type>(column, row),
+            _ => Err(ArrowError::InvalidArgumentError(format!(
+                "Pretty printing not supported for {:?} due to index type",
+                column.data_type()
+            ))),
+        },
+        _ => Err(ArrowError::InvalidArgumentError(format!(
+            "Pretty printing not implemented for {:?} type",
+            column.data_type()
+        ))),
+    }
+}
+
+/// Converts the value of the dictionary array at `row` to a String
+fn dict_array_value_to_string<K: ArrowPrimitiveType>(
+    colum: &array::ArrayRef,
+    row: usize,
+) -> Result<String> {
+    let dict_array = colum.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
+
+    let keys_array = dict_array.keys_array();
+
+    if keys_array.is_null(row) {
+        return Ok(String::from(""));
+    }
+
+    let dict_index = keys_array.value(row).to_usize().ok_or_else(|| {
+        ArrowError::InvalidArgumentError(format!(
+            "Can not convert value {:?} at index {:?} to usize for repl.",

Review comment:
       nit: remove `for repl`.




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

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



[GitHub] [arrow] alamb commented on a change in pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#discussion_r501717377



##########
File path: rust/arrow/src/util/display.rs
##########
@@ -0,0 +1,135 @@
+// 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.
+
+//! Functions for printing array values, as strings, for debugging
+//! purposes. See the `pretty` crate for additional functions for
+//! record batch pretty printing.
+
+use crate::array;
+use crate::array::{Array, PrimitiveArrayOps};
+use crate::datatypes::{
+    ArrowNativeType, ArrowPrimitiveType, DataType, Int16Type, Int32Type, Int64Type,
+    Int8Type, TimeUnit, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
+};
+
+use array::DictionaryArray;
+
+use crate::error::{ArrowError, Result};
+
+macro_rules! make_string {
+    ($array_type:ty, $column: ident, $row: ident) => {{
+        let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
+
+        let s = if array.is_null($row) {
+            "".to_string()
+        } else {
+            array.value($row).to_string()
+        };
+
+        Ok(s)
+    }};
+}
+
+/// Get the value at the given row in an array as a String.
+///
+/// Note this function is quite inefficient and is unlikely to be
+/// suitable for converting large arrays or record batches.
+pub fn array_value_to_string(column: &array::ArrayRef, row: usize) -> Result<String> {
+    match column.data_type() {
+        DataType::Utf8 => make_string!(array::StringArray, column, row),
+        DataType::Boolean => make_string!(array::BooleanArray, column, row),
+        DataType::Int8 => make_string!(array::Int8Array, column, row),
+        DataType::Int16 => make_string!(array::Int16Array, column, row),
+        DataType::Int32 => make_string!(array::Int32Array, column, row),
+        DataType::Int64 => make_string!(array::Int64Array, column, row),
+        DataType::UInt8 => make_string!(array::UInt8Array, column, row),
+        DataType::UInt16 => make_string!(array::UInt16Array, column, row),
+        DataType::UInt32 => make_string!(array::UInt32Array, column, row),
+        DataType::UInt64 => make_string!(array::UInt64Array, column, row),
+        DataType::Float16 => make_string!(array::Float32Array, column, row),
+        DataType::Float32 => make_string!(array::Float32Array, column, row),
+        DataType::Float64 => make_string!(array::Float64Array, column, row),
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Second => {
+            make_string!(array::TimestampSecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::TimestampMillisecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::TimestampMicrosecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::TimestampNanosecondArray, column, row)
+        }
+        DataType::Date32(_) => make_string!(array::Date32Array, column, row),
+        DataType::Date64(_) => make_string!(array::Date64Array, column, row),
+        DataType::Time32(unit) if *unit == TimeUnit::Second => {
+            make_string!(array::Time32SecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::Time32MillisecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::Time64MicrosecondArray, column, row)
+        }
+        DataType::Time64(unit) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::Time64NanosecondArray, column, row)
+        }
+        DataType::Dictionary(index_type, _value_type) => match **index_type {
+            DataType::Int8 => dict_array_value_to_string::<Int8Type>(column, row),
+            DataType::Int16 => dict_array_value_to_string::<Int16Type>(column, row),
+            DataType::Int32 => dict_array_value_to_string::<Int32Type>(column, row),
+            DataType::Int64 => dict_array_value_to_string::<Int64Type>(column, row),
+            DataType::UInt8 => dict_array_value_to_string::<UInt8Type>(column, row),
+            DataType::UInt16 => dict_array_value_to_string::<UInt16Type>(column, row),
+            DataType::UInt32 => dict_array_value_to_string::<UInt32Type>(column, row),
+            DataType::UInt64 => dict_array_value_to_string::<UInt64Type>(column, row),
+            _ => Err(ArrowError::InvalidArgumentError(format!(
+                "Pretty printing not supported for {:?} due to index type",
+                column.data_type()
+            ))),
+        },
+        _ => Err(ArrowError::InvalidArgumentError(format!(
+            "Pretty printing not implemented for {:?} type",
+            column.data_type()
+        ))),
+    }
+}
+
+/// Converts the value of the dictionary array at `row` to a String
+fn dict_array_value_to_string<K: ArrowPrimitiveType>(
+    colum: &array::ArrayRef,
+    row: usize,
+) -> Result<String> {
+    let dict_array = colum.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
+
+    let keys_array = dict_array.keys_array();
+
+    if keys_array.is_null(row) {
+        return Ok(String::from(""));
+    }
+
+    let dict_index = keys_array.value(row).to_usize().ok_or_else(|| {
+        ArrowError::InvalidArgumentError(format!(
+            "Can not convert value {:?} at index {:?} to usize for repl.",

Review comment:
       ```suggestion
               "Can not convert value {:?} at index {:?} to usize for string conversion.",
   ```




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

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



[GitHub] [arrow] nevi-me commented on a change in pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
nevi-me commented on a change in pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#discussion_r501678317



##########
File path: rust/arrow/src/util/display.rs
##########
@@ -0,0 +1,135 @@
+// 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.
+
+//! Functions for printing array values, as strings, for debugging
+//! purposes. See the `pretty` crate for additional functions for
+//! record batch pretty printing.
+
+use crate::array;
+use crate::array::{Array, PrimitiveArrayOps};
+use crate::datatypes::{
+    ArrowNativeType, ArrowPrimitiveType, DataType, Int16Type, Int32Type, Int64Type,
+    Int8Type, TimeUnit, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
+};
+
+use array::DictionaryArray;
+
+use crate::error::{ArrowError, Result};
+
+macro_rules! make_string {
+    ($array_type:ty, $column: ident, $row: ident) => {{
+        let array = $column.as_any().downcast_ref::<$array_type>().unwrap();
+
+        let s = if array.is_null($row) {
+            "".to_string()
+        } else {
+            array.value($row).to_string()
+        };
+
+        Ok(s)
+    }};
+}
+
+/// Get the value at the given row in an array as a String.
+///
+/// Note this function is quite inefficient and is unlikely to be
+/// suitable for converting large arrays or record batches.
+pub fn array_value_to_string(column: &array::ArrayRef, row: usize) -> Result<String> {
+    match column.data_type() {
+        DataType::Utf8 => make_string!(array::StringArray, column, row),
+        DataType::Boolean => make_string!(array::BooleanArray, column, row),
+        DataType::Int8 => make_string!(array::Int8Array, column, row),
+        DataType::Int16 => make_string!(array::Int16Array, column, row),
+        DataType::Int32 => make_string!(array::Int32Array, column, row),
+        DataType::Int64 => make_string!(array::Int64Array, column, row),
+        DataType::UInt8 => make_string!(array::UInt8Array, column, row),
+        DataType::UInt16 => make_string!(array::UInt16Array, column, row),
+        DataType::UInt32 => make_string!(array::UInt32Array, column, row),
+        DataType::UInt64 => make_string!(array::UInt64Array, column, row),
+        DataType::Float16 => make_string!(array::Float32Array, column, row),
+        DataType::Float32 => make_string!(array::Float32Array, column, row),
+        DataType::Float64 => make_string!(array::Float64Array, column, row),
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Second => {
+            make_string!(array::TimestampSecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::TimestampMillisecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::TimestampMicrosecondArray, column, row)
+        }
+        DataType::Timestamp(unit, _) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::TimestampNanosecondArray, column, row)
+        }
+        DataType::Date32(_) => make_string!(array::Date32Array, column, row),
+        DataType::Date64(_) => make_string!(array::Date64Array, column, row),
+        DataType::Time32(unit) if *unit == TimeUnit::Second => {
+            make_string!(array::Time32SecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Millisecond => {
+            make_string!(array::Time32MillisecondArray, column, row)
+        }
+        DataType::Time32(unit) if *unit == TimeUnit::Microsecond => {
+            make_string!(array::Time64MicrosecondArray, column, row)
+        }
+        DataType::Time64(unit) if *unit == TimeUnit::Nanosecond => {
+            make_string!(array::Time64NanosecondArray, column, row)
+        }
+        DataType::Dictionary(index_type, _value_type) => match **index_type {
+            DataType::Int8 => dict_array_value_to_string::<Int8Type>(column, row),
+            DataType::Int16 => dict_array_value_to_string::<Int16Type>(column, row),
+            DataType::Int32 => dict_array_value_to_string::<Int32Type>(column, row),
+            DataType::Int64 => dict_array_value_to_string::<Int64Type>(column, row),
+            DataType::UInt8 => dict_array_value_to_string::<UInt8Type>(column, row),
+            DataType::UInt16 => dict_array_value_to_string::<UInt16Type>(column, row),
+            DataType::UInt32 => dict_array_value_to_string::<UInt32Type>(column, row),
+            DataType::UInt64 => dict_array_value_to_string::<UInt64Type>(column, row),
+            _ => Err(ArrowError::InvalidArgumentError(format!(
+                "Pretty printing not supported for {:?} due to index type",
+                column.data_type()
+            ))),
+        },
+        _ => Err(ArrowError::InvalidArgumentError(format!(
+            "Pretty printing not implemented for {:?} type",
+            column.data_type()
+        ))),
+    }
+}
+
+/// Converts the value of the dictionary array at `row` to a String
+fn dict_array_value_to_string<K: ArrowPrimitiveType>(
+    colum: &array::ArrayRef,
+    row: usize,
+) -> Result<String> {
+    let dict_array = colum.as_any().downcast_ref::<DictionaryArray<K>>().unwrap();
+
+    let keys_array = dict_array.keys_array();
+
+    if keys_array.is_null(row) {
+        return Ok(String::from(""));
+    }
+
+    let dict_index = keys_array.value(row).to_usize().ok_or_else(|| {
+        ArrowError::InvalidArgumentError(format!(
+            "Can not convert value {:?} at index {:?} to usize for repl.",

Review comment:
       nit: remove `for repl`.




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8397:
URL: https://github.com/apache/arrow/pull/8397#issuecomment-705497672


   https://issues.apache.org/jira/browse/ARROW-10233


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

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



[GitHub] [arrow] jorgecarleitao closed pull request #8397: ARROW-10233: [Rust] Make array_value_to_string available in all Arrow builds

Posted by GitBox <gi...@apache.org>.
jorgecarleitao closed pull request #8397:
URL: https://github.com/apache/arrow/pull/8397


   


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

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