You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "tshauck (via GitHub)" <gi...@apache.org> on 2024/03/21 18:38:21 UTC

[PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

tshauck opened a new pull request, #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729

   ## Which issue does this PR close?
   
   Closes #533
   
   ## Rationale for this change
   
   Being able to control the underlying parser is useful to be able to add capabilities between the SQL and logical plan layer, which can propagate downstream in the plan.
   
   This approach seems to work nicely insofar as modifying how the SQL becomes a user-defined Statement. I.e. here I'm wrapping DF's `Statement` with my own, and creating it based on the underlying result from `parse_copy`.
   
   However, I'm unsure on the next steps. E.g. with this strategy of wrapping the statement, I can't then use DF's machinery to generate the `LogicalPlan`, like using `sql_statement_to_plan`. It's not too bad in the basic case, as it'd be easy to generate a user defined logical node from this enum (or fall back to `sql_statement_to_plan`), but in more complex cases there's probably a better way, e.g. maybe to make `Statement` a Trait with a visit method, or something along those lines? There's probably a better/simpler approach :).
   
   Going to open this in draft, and will come back later today/tomorrow to see what the basic strategy for constructing a LogicalPlan would look like and see if that's a reasonable first chunk.
   
   Open to feedback on any of it, thanks!  
   
   ## What changes are included in this PR?
   
   Makes the parser on DFParser public and adds an example.
   
   ## Are these changes tested?
   
   Manually ran the example. Seems to perform as expected. Though not sure what the best path is next.
   
   ## Are there any user-facing changes?
   
   Not breaking, but `parser` is available on `DFParser`.
   


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "tshauck (via GitHub)" <gi...@apache.org>.
tshauck commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2018566584

   I'm gonna move this out of draft since it's only that small change, but happy to continue the convo.


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2016450858

   Thanks for working on this @tshauck  -- it is really cool to see
   
   > Going to open this in draft, and will come back later today/tomorrow to see what the basic strategy for constructing a LogicalPlan would look like and see if that's a reasonable first chunk.
   
   The simplest strategy I can think of is to implement your own version to `sql_statement_to_plan` but for `MyStatement`
   
   So something like
   
   ```
       pub async fn my_statement_to_plan(
           ctx: &SessionContext,
           statement: MyStatement,
       ) -> Result<LogicalPlan> {
      ...
      // call statement_to_plan here and implement the special Copy logic
   }
   ````
   
   
   
   > So I guess my questions are 1) does an Extension statement seem reasonable to you? 2) and if so, any advice on the implementation to mitigate object safety issues.
   
   I don't fully understand the need for an Extension statement -- if the idea is to wrap `DFStatement` with `MyStatement` I think you can just implement the planning in terms of MyStatement as above. 
   
   If the idea is to avoid repetition with whatever statement_to_plan is doing, maybe we can factor out the common functionality into a module. 
   
   


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "tshauck (via GitHub)" <gi...@apache.org>.
tshauck commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2018565479

   Thanks for the feedback, I reverted the extension changes, and now it only has the example plus making `Parser` public on `DFParser`. To your point later, there's quite a bit within `sql_statement_to_plan` (and functions it calls) that would be nice to not have to replicate, but perhaps going through the work of using this change in my code will make it clearer what those changes should be. Things like resolving name, handling subqueries, etc.


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#discussion_r1540989878


##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way
+#[tokio::main]
+async fn main() -> Result<()> {
+    let mut my_parser =
+        MyParser::new("COPY source_table TO 'file.fasta' STORED AS FASTA")?;

Review Comment:
   😸 



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way
+#[tokio::main]
+async fn main() -> Result<()> {
+    let mut my_parser =
+        MyParser::new("COPY source_table TO 'file.fasta' STORED AS FASTA")?;
+
+    let my_statement = my_parser.parse_statement()?;
+
+    match my_statement {
+        MyStatement::DFStatement(s) => println!("df: {}", s),
+        MyStatement::MyCopyTo(s) => println!("my_copy: {}", s),
+    }
+
+    Ok(())
+}
+
+struct MyParser<'a> {
+    df_parser: DFParser<'a>,
+}
+
+impl MyParser<'_> {
+    fn new(sql: &str) -> Result<Self> {
+        let df_parser = DFParser::new(sql)?;
+        Ok(Self { df_parser })
+    }
+
+    pub fn parse_statement(&mut self) -> Result<MyStatement, ParserError> {

Review Comment:
   ```suggestion
       /// This is the entry point to our parser -- it handles `COPY` statements specially
       /// but otherwise delegates to the existing DataFusion parser. 
       pub fn parse_statement(&mut self) -> Result<MyStatement, ParserError> {
   ```



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way
+#[tokio::main]
+async fn main() -> Result<()> {
+    let mut my_parser =
+        MyParser::new("COPY source_table TO 'file.fasta' STORED AS FASTA")?;
+
+    let my_statement = my_parser.parse_statement()?;
+
+    match my_statement {
+        MyStatement::DFStatement(s) => println!("df: {}", s),
+        MyStatement::MyCopyTo(s) => println!("my_copy: {}", s),
+    }
+
+    Ok(())
+}
+
+struct MyParser<'a> {

Review Comment:
   ```suggestion
   /// Here we define a Parser for our new SQL dialect that wraps the existing `DFParser`
   struct MyParser<'a> {
   ```



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way
+#[tokio::main]
+async fn main() -> Result<()> {
+    let mut my_parser =
+        MyParser::new("COPY source_table TO 'file.fasta' STORED AS FASTA")?;
+
+    let my_statement = my_parser.parse_statement()?;
+
+    match my_statement {
+        MyStatement::DFStatement(s) => println!("df: {}", s),
+        MyStatement::MyCopyTo(s) => println!("my_copy: {}", s),
+    }
+
+    Ok(())
+}
+
+struct MyParser<'a> {
+    df_parser: DFParser<'a>,
+}
+
+impl MyParser<'_> {
+    fn new(sql: &str) -> Result<Self> {
+        let df_parser = DFParser::new(sql)?;
+        Ok(Self { df_parser })
+    }
+
+    pub fn parse_statement(&mut self) -> Result<MyStatement, ParserError> {
+        match self.df_parser.parser.peek_token().token {
+            Token::Word(w) => {
+                match w.keyword {
+                    Keyword::COPY => {
+                        self.df_parser.parser.next_token(); // COPY
+                        let df_statement = self.df_parser.parse_copy()?;
+
+                        if let Statement::CopyTo(s) = df_statement {
+                            Ok(MyStatement::from(s))
+                        } else {
+                            Ok(MyStatement::DFStatement(Box::from(df_statement)))
+                        }
+                    }
+                    _ => {
+                        // use sqlparser-rs parser

Review Comment:
   ```suggestion
                           // Otherwise, delegate back to the DFParser directly
   ```



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way

Review Comment:
   ```suggestion
   /// This example demonstrates how to use the DFParser to parse a statement in a custom way
   ///
   /// This technique can be used to implement a custom SQL dialect, for example. 
   ```



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   What do you think about calling this file `sql_dialect` rather than `sql_parser` to hint it is an example of how to implement a custom dialect?



##########
datafusion-examples/examples/sql_parsing.rs:
##########
@@ -0,0 +1,133 @@
+// 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::fmt::Display;
+
+use datafusion::error::Result;
+use datafusion_sql::{
+    parser::{CopyToSource, CopyToStatement, DFParser, Statement},
+    sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token},
+};
+
+/// This example demonstrates how to use the DFParser to parse a statement in a custom way
+#[tokio::main]
+async fn main() -> Result<()> {
+    let mut my_parser =
+        MyParser::new("COPY source_table TO 'file.fasta' STORED AS FASTA")?;
+
+    let my_statement = my_parser.parse_statement()?;
+
+    match my_statement {
+        MyStatement::DFStatement(s) => println!("df: {}", s),
+        MyStatement::MyCopyTo(s) => println!("my_copy: {}", s),
+    }
+
+    Ok(())
+}
+
+struct MyParser<'a> {
+    df_parser: DFParser<'a>,
+}
+
+impl MyParser<'_> {
+    fn new(sql: &str) -> Result<Self> {
+        let df_parser = DFParser::new(sql)?;
+        Ok(Self { df_parser })
+    }
+
+    pub fn parse_statement(&mut self) -> Result<MyStatement, ParserError> {
+        match self.df_parser.parser.peek_token().token {
+            Token::Word(w) => {
+                match w.keyword {
+                    Keyword::COPY => {
+                        self.df_parser.parser.next_token(); // COPY

Review Comment:
   ```suggestion
                           // Invoke special COPY dialect parsing
                           // and fall back to the DataFusion parser to parse the body
                           self.df_parser.parser.next_token(); // COPY
   ```



-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "tshauck (via GitHub)" <gi...@apache.org>.
tshauck commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2015642603

   @alamb, would you be able to please give me some feedback here?
   
   In https://github.com/apache/arrow-datafusion/pull/9729/commits/a7a7808d5a5088540e80308a4c0a0a39a22a899a, I made parser public and added an example, and that seemed to go well insofar as I could create a new `MyStatement` which is an enum of a DF Statement and a custom one. 
   
   That was good, but then it can't work with df functionality like `sql_statement_to_plan`, so in [86588e4](https://github.com/apache/arrow-datafusion/pull/9729/commits/86588e4513fc787c01c567c40bb76cb598a55ad1) I started down added an Extension enum variant to `Statement`, based on how the LogicalPlan extensions work, but I'm having trouble integrating it into DF writ large... e.g. I'm running into many object safety issues...
   
   <img width="916" alt="image" src="https://github.com/apache/arrow-datafusion/assets/421839/d512090f-b2b4-48df-98e0-4c8f6cf2c243">
   
   So I guess my questions are 1) does an Extension statement seem reasonable to you? 2) and if so, any advice on the implementation to mitigate object safety issues.
   


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "tshauck (via GitHub)" <gi...@apache.org>.
tshauck commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2023124662

   @alamb Thanks for all the feedback! The `matches!` approach is really nice... requesting review.


-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "tshauck (via GitHub)" <gi...@apache.org>.
tshauck commented on code in PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#discussion_r1541366411


##########
datafusion-examples/README.md:
##########
@@ -42,36 +42,37 @@ cargo run --example csv_sql
 
 ## Single Process
 
+- [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF)
+- [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF)
+- [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF)
 - [`avro_sql.rs`](examples/avro_sql.rs): Build and run a query plan from a SQL statement against a local AVRO file
+- [`catalog.rs`](examples/catalog.rs): Register the table into a custom catalog
 - [`csv_sql.rs`](examples/csv_sql.rs): Build and run a query plan from a SQL statement against a local CSV file
 - [`csv_sql_streaming.rs`](examples/csv_sql_streaming.rs): Build and run a streaming query plan from a SQL statement against a local CSV file
-- [`catalog.rs`](examples/catalog.rs): Register the table into a custom catalog
 - [`custom_datasource.rs`](examples/custom_datasource.rs): Run queries against a custom datasource (TableProvider)
-- [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame against a local parquet file
 - [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 and writing back to s3
-- [`dataframe_output.rs`](examples/dataframe_output.rs): Examples of methods which write data out from a DataFrame
+- [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame against a local parquet file
 - [`dataframe_in_memory.rs`](examples/dataframe_in_memory.rs): Run a query using a DataFrame against data in memory
+- [`dataframe_output.rs`](examples/dataframe_output.rs): Examples of methods which write data out from a DataFrame
 - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde
 - [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify and analyze `Expr`s
 - [`flight_sql_server.rs`](examples/flight/flight_sql_server.rs): Run DataFusion as a standalone process and execute SQL queries from JDBC clients
 - [`function_factory.rs`](examples/function_factory.rs): Register `CREATE FUNCTION` handler to implement SQL macros
 - [`make_date.rs`](examples/make_date.rs): Examples of using the make_date function
 - [`memtable.rs`](examples/memtable.rs): Create an query data in memory using SQL and `RecordBatch`es
-- [`pruning.rs`](examples/parquet_sql.rs): Use pruning to rule out files based on statistics
 - [`parquet_sql.rs`](examples/parquet_sql.rs): Build and run a query plan from a SQL statement against a local Parquet file
 - [`parquet_sql_multiple_files.rs`](examples/parquet_sql_multiple_files.rs): Build and run a query plan from a SQL statement against multiple local Parquet files
+- [`pruning.rs`](examples/parquet_sql.rs): Use pruning to rule out files based on statistics
 - [`query-aws-s3.rs`](examples/external_dependency/query-aws-s3.rs): Configure `object_store` and run a query against files stored in AWS S3
 - [`query-http-csv.rs`](examples/query-http-csv.rs): Configure `object_store` and run a query against files vi HTTP
 - [`regexp.rs`](examples/regexp.rs): Examples of using regular expression functions
 - [`rewrite_expr.rs`](examples/rewrite_expr.rs): Define and invoke a custom Query Optimizer pass
-- [`to_char.rs`](examples/to_char.rs): Examples of using the to_char function
-- [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using to_timestamp functions
-- [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF)
-- [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF)
 - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF)
-- [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF)
+- [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF)
 - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF)
-- [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF)
+- [`sql_dialect.rs`](examples/sql_dialect.rs): Examples of using the SQL Dialect

Review Comment:
   Here's `sql_dialect.rs` -- I also alphabetized the list, hope that's ok.



-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#discussion_r1541462957


##########
datafusion-examples/README.md:
##########
@@ -42,36 +42,37 @@ cargo run --example csv_sql
 
 ## Single Process
 
+- [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF)

Review Comment:
   ❤️  -- very nice drive by cleanup 🙏 



-- 
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-unsubscribe@arrow.apache.org

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


Re: [PR] Expose `parser` on DFParser to enable user controlled parsing [arrow-datafusion]

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #9729:
URL: https://github.com/apache/arrow-datafusion/pull/9729#issuecomment-2024294610

   Thanks again @tshauck 


-- 
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-unsubscribe@arrow.apache.org

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