Omega359 commented on code in PR #9019:
URL: https://github.com/apache/arrow-datafusion/pull/9019#discussion_r1497529417


##########
datafusion-examples/examples/to_date.rs:
##########
@@ -0,0 +1,60 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::arrow::array::StringArray;
+use datafusion::arrow::datatypes::{DataType, Field, Schema};
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::prelude::*;
+
+/// This example demonstrates how to use the to_timestamp series

Review Comment:
   to_timestamp -> to_date



##########
datafusion/physical-expr/src/datetime_expressions.rs:
##########
@@ -424,6 +425,84 @@ fn to_timestamp_impl<T: ArrowTimestampType + 
ScalarType<i64>>(
     }
 }
 
+/// # Examples
+///
+/// ```
+/// use std::sync::Arc;
+
+/// use datafusion::arrow::array::StringArray;
+/// use datafusion::arrow::datatypes::{DataType, Field, Schema};
+/// use datafusion::arrow::record_batch::RecordBatch;
+/// use datafusion::error::Result;
+/// use datafusion::prelude::*;
+
+/// #[tokio::main]
+/// async fn main() -> Result<()> {
+///     // define a schema.
+///     let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, 
false)]));
+
+///     // define data.
+///     let batch = RecordBatch::try_new(
+///         schema,
+///         vec![Arc::new(StringArray::from(vec![
+///             "2020-09-08T13:42:29Z",
+///             "2020-09-08T13:42:29.190855-05:00",
+///             "2020-08-09 12:13:29",
+///             "2020-01-02",
+///         ]))],
+///     )?;
+
+///     // declare a new context. In spark API, this corresponds to a new 
spark SQLsession
+///     let ctx = SessionContext::new();
+
+///     // declare a table in memory. In spark API, this corresponds to 
createDataFrame(...).
+///     ctx.register_batch("t", batch)?;
+///     let df = ctx.table("t").await?;
+
+///     // use to_timestamp function to convert col 'a' to timestamp type 
using the default parsing
+///     let df = df.with_column("a", to_date(vec![col("a")]))?;
+
+///     let df = df.select_columns(&["a"])?;
+
+///     // print the results
+///     df.show().await?;
+
+///     Ok(())
+/// }
+/// ```
+pub fn to_date(args: &[ColumnarValue]) -> Result<ColumnarValue> {
+    match args.len() {
+        1 => handle::<Date32Type, _, Date32Type>(
+            args,
+            |s| {
+                string_to_timestamp_nanos_shim(s)
+                    .map(|n| n / (1_000_000 * 24 * 60 * 60 * 1_000))
+                    .and_then(|v| {
+                        v.try_into().map_err(|_| {
+                            internal_datafusion_err!("Unable to cast to Date32 
for converting from i64 to i32 failed")
+                        })
+                    })
+            },
+            "to_date",
+        ),
+        n if n >= 2 => handle_multiple::<Date32Type, _, Date32Type, _>(
+            args,
+            |s, format| {
+                string_to_timestamp_nanos_formatted(s, format)
+                    .map(|n| n / (1_000_000 * 24 * 60 * 60 * 1_000))
+                    .and_then(|v| {
+                        v.try_into().map_err(|_| {
+                            internal_datafusion_err!("Unable to cast to Date32 
for converting from i64 to i32 failed")
+                        })
+                    })
+            },
+            |n| n,
+            "to_date",
+        ),
+        _ => internal_err!("Unsupported 0 argument count for function 
to_date"),

Review Comment:
   This should be an exec_err (See 
https://github.com/apache/arrow-datafusion/pull/9241 for details)



##########
datafusion-examples/examples/to_date.rs:
##########
@@ -0,0 +1,60 @@
+// 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.
+
+use std::sync::Arc;
+
+use datafusion::arrow::array::StringArray;
+use datafusion::arrow::datatypes::{DataType, Field, Schema};
+use datafusion::arrow::record_batch::RecordBatch;
+use datafusion::error::Result;
+use datafusion::prelude::*;
+
+/// This example demonstrates how to use the to_timestamp series
+/// of functions in the DataFrame API as well as via sql.
+#[tokio::main]
+async fn main() -> Result<()> {
+    // define a schema.
+    let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, 
false)]));
+
+    // define data.
+    let batch = RecordBatch::try_new(
+        schema,
+        vec![Arc::new(StringArray::from(vec![
+            "2020-09-08T13:42:29Z",
+            "2020-09-08T13:42:29.190855-05:00",
+            "2020-08-09 12:13:29",
+            "2020-01-02",
+        ]))],
+    )?;
+
+    // declare a new context. In spark API, this corresponds to a new spark 
SQLsession
+    let ctx = SessionContext::new();
+
+    // declare a table in memory. In spark API, this corresponds to 
createDataFrame(...).
+    ctx.register_batch("t", batch)?;
+    let df = ctx.table("t").await?;
+
+    // use to_timestamp function to convert col 'a' to timestamp type using 
the default parsing

Review Comment:
   to_timestamp -> to_date, convert 'a to date type



##########
datafusion/physical-expr/src/datetime_expressions.rs:
##########
@@ -424,6 +425,84 @@ fn to_timestamp_impl<T: ArrowTimestampType + 
ScalarType<i64>>(
     }
 }
 
+/// # Examples
+///
+/// ```
+/// use std::sync::Arc;
+
+/// use datafusion::arrow::array::StringArray;
+/// use datafusion::arrow::datatypes::{DataType, Field, Schema};
+/// use datafusion::arrow::record_batch::RecordBatch;
+/// use datafusion::error::Result;
+/// use datafusion::prelude::*;

Review Comment:
   Unfortunately we'll have to add an ignore for this because of 
https://github.com/apache/arrow-datafusion/issues/9277 See 
https://github.com/apache/arrow-datafusion/pull/9279/files#diff-67ae8808785b2e651767d7ff67dd7c53be04ca098857b52c82ed19927e071cdaR514
 for an example. I believe once the functions are move to the new 
dataframe-functions crate we should be able to re-enable.



##########
datafusion/physical-expr/src/datetime_expressions.rs:
##########
@@ -424,6 +425,84 @@ fn to_timestamp_impl<T: ArrowTimestampType + 
ScalarType<i64>>(
     }
 }
 
+/// # Examples
+///
+/// ```
+/// use std::sync::Arc;
+
+/// use datafusion::arrow::array::StringArray;
+/// use datafusion::arrow::datatypes::{DataType, Field, Schema};
+/// use datafusion::arrow::record_batch::RecordBatch;
+/// use datafusion::error::Result;
+/// use datafusion::prelude::*;
+
+/// #[tokio::main]
+/// async fn main() -> Result<()> {
+///     // define a schema.
+///     let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, 
false)]));
+
+///     // define data.
+///     let batch = RecordBatch::try_new(
+///         schema,
+///         vec![Arc::new(StringArray::from(vec![
+///             "2020-09-08T13:42:29Z",
+///             "2020-09-08T13:42:29.190855-05:00",
+///             "2020-08-09 12:13:29",
+///             "2020-01-02",
+///         ]))],
+///     )?;
+
+///     // declare a new context. In spark API, this corresponds to a new 
spark SQLsession
+///     let ctx = SessionContext::new();
+
+///     // declare a table in memory. In spark API, this corresponds to 
createDataFrame(...).
+///     ctx.register_batch("t", batch)?;
+///     let df = ctx.table("t").await?;
+
+///     // use to_timestamp function to convert col 'a' to timestamp type 
using the default parsing

Review Comment:
   timestamp => date



##########
datafusion/physical-expr/src/datetime_expressions.rs:
##########
@@ -1567,6 +1646,36 @@ fn validate_to_timestamp_data_types(
     None
 }
 
+/// to_date SQL function implementation
+pub fn to_date_invoke(args: &[ColumnarValue]) -> Result<ColumnarValue> {
+    if args.is_empty() {
+        return exec_err!(
+            "to_date function requires 1 or more arguments, got {}",
+            args.len()
+        );
+    }
+
+    // validate that any args after the first one are Utf8
+    if args.len() > 1 {
+        if let Some(value) = validate_to_timestamp_data_types(args, "to_date") 
{
+            return value;
+        }
+    }
+
+    match args[0].data_type() {
+        DataType::Int32
+        | DataType::Int64
+        | DataType::Null
+        | DataType::Float64
+        | DataType::Date32
+        | DataType::Date64 => cast_column(&args[0], &DataType::Date32, None),
+        DataType::Utf8 => to_date(args),
+        other => {
+            internal_err!("Unsupported data type {:?} for function to_date", 
other)

Review Comment:
   exec_err



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

Reply via email to