You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "parkma99 (via GitHub)" <gi...@apache.org> on 2023/07/31 17:15:33 UTC

[GitHub] [arrow-datafusion] parkma99 opened a new pull request, #7160: support primary key aother syntax

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

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #7152 
   
   # Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are these changes tested?
   Yes , add a test case in datafusion/sql/tests/sql_integration.rs
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   # Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->


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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #7160: feat: support primary key aother syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   I am not sure this means a multi part primary key -- postgres errors on this syntax
   
   ```sql
   postgres=# create table person (id int, name varchar primary key, primary key(id));
   ERROR:  multiple primary keys for table "person" are not allowed
   LINE 1: ...e table person (id int, name varchar primary key, primary ke...
   ```
   
   I think the way this is supposed to be expressed is like
   
   ```sql
   create table person (id int, name varchar,  primary key(name,  id));
   ```
   
   



##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";

Review Comment:
   What do you think about adding unique constraints as well
   
   ```sql
   create table person (id int unique, name string)
   ```



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


[GitHub] [arrow-datafusion] parkma99 commented on pull request #7160: feat: support primary key aother syntax

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

   Thanks @jonahgao , I will update this scenarios recently.


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


[GitHub] [arrow-datafusion] alamb commented on pull request #7160: feat: support primary key alternate syntax

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

   Marking as draft per https://github.com/apache/arrow-datafusion/pull/7160#discussion_r1287654603


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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   @parkma99  is this comment still outstanding? 



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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #7160: feat: support primary key aother syntax

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


##########
datafusion/sql/src/statement.rs:
##########
@@ -132,75 +132,93 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                 if_not_exists,
                 or_replace,
                 ..
-            } if table_properties.is_empty() && with_options.is_empty() => match query {
-                Some(query) => {
-                    let plan = self.query_to_plan(*query, planner_context)?;
-                    let input_schema = plan.schema();
-
-                    let plan = if !columns.is_empty() {
-                        let schema = self.build_schema(columns)?.to_dfschema_ref()?;
-                        if schema.fields().len() != input_schema.fields().len() {
-                            return Err(DataFusionError::Plan(format!(
+            } if table_properties.is_empty() && with_options.is_empty() => {
+                let mut constraints = constraints;
+                for column in &columns {
+                    for option in &column.options {
+                        if let ast::ColumnOption::Unique { is_primary } = option.option {
+                            constraints.push(ast::TableConstraint::Unique {
+                                name: None,
+                                columns: vec![column.name.clone()],
+                                is_primary,
+                            })
+                        }
+                        // TODO (parkma99) handle ForeignKey etc.

Review Comment:
   Can you please add code here that returns a `Err(DataFusionError::NotImplemented)` for other types of constraints -- I think this code will silently ignore them
   
   Maybe it could be something like
   
   ```rust
   match option.option {
     ast::ColumnOption::Unique { is_primary } => ... 
     option => return Err(DataFusionError::NotImplemented(format!("Constraint {} not implemented", option))),
   }
   ```
   



##########
datafusion/sql/src/statement.rs:
##########
@@ -132,75 +132,93 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                 if_not_exists,
                 or_replace,
                 ..
-            } if table_properties.is_empty() && with_options.is_empty() => match query {
-                Some(query) => {
-                    let plan = self.query_to_plan(*query, planner_context)?;
-                    let input_schema = plan.schema();
-
-                    let plan = if !columns.is_empty() {
-                        let schema = self.build_schema(columns)?.to_dfschema_ref()?;
-                        if schema.fields().len() != input_schema.fields().len() {
-                            return Err(DataFusionError::Plan(format!(
+            } if table_properties.is_empty() && with_options.is_empty() => {
+                let mut constraints = constraints;
+                for column in &columns {
+                    for option in &column.options {
+                        if let ast::ColumnOption::Unique { is_primary } = option.option {
+                            constraints.push(ast::TableConstraint::Unique {
+                                name: None,
+                                columns: vec![column.name.clone()],
+                                is_primary,
+                            })
+                        }
+                        // TODO (parkma99) handle ForeignKey etc.

Review Comment:
   However, given that the current code will also ignore such errors I don't think this change is required



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


[GitHub] [arrow-datafusion] mustafasrepo commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   > I am not sure this means a multi part primary key -- postgres errors on this syntax
   > 
   > ```sql
   > postgres=# create table person (id int, name varchar primary key, primary key(id));
   > ERROR:  multiple primary keys for table "person" are not allowed
   > LINE 1: ...e table person (id int, name varchar primary key, primary ke...
   > ```
   > 
   > I think the way this is supposed to be expressed is like
   > 
   > ```sql
   > create table person (id int, name varchar,  primary key(name,  id));
   > ```
   
   I guess it is better to follow postgres in this case. Postgres supports multiple unique constraints. Hence following query
   ```sql
   create table person (id int, name string primary key, primary key(id))
   ```
   can be written as 
   ```sql
   create table person (id int, name string unique not null, primary key(id))
   ```
   to define unique constraint. For writing composite primary key one have to use following syntax
   ```sql
   create table person (id int, name varchar,  primary key(name,  id));
   ```



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


[GitHub] [arrow-datafusion] mustafasrepo commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   > I am not sure this means a multi part primary key -- postgres errors on this syntax
   > 
   > ```sql
   > postgres=# create table person (id int, name varchar primary key, primary key(id));
   > ERROR:  multiple primary keys for table "person" are not allowed
   > LINE 1: ...e table person (id int, name varchar primary key, primary ke...
   > ```
   > 
   > I think the way this is supposed to be expressed is like
   > 
   > ```sql
   > create table person (id int, name varchar,  primary key(name,  id));
   > ```
   
   I would expect that `name` is primary key, and `id` is primary key (they each consists of unique values, unlike their tuple version is unique which is also the case.) for the query
   ```sql
   create table person (id int, name string primary key, primary key(id))
   ```
    In short, when one want to use composite primary key, they have to use 
   ```sql
   create table person (id int, name varchar,  primary key(name,  id));
   ```
   syntax. However, for defining single primary keys, they have alternative version. However, if this behaviour is misleading, or counter intuitive; We can follow postgres and give an error. However, It seems to me that, there is no reason to restrain. 



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


[GitHub] [arrow-datafusion] mustafasrepo commented on pull request #7160: feat: support primary key aother syntax

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

   > Perhaps we should consider scenarios where primary keys are defined multiple times.
   > 
   > For example:
   > 
   > ```shell
   > psql=> create table foo(
   >         a int primary key, 
   >         b int, 
   >         c int, 
   >         primary key(b,c)
   > );
   > 
   > ERROR:  multiple primary keys for table "foo" are not allowed
   > LINE 5:         primary key(b,c)
   > ```
   > 
   > It will fail in PostgreSQL.
   
   I think we may choose not to fail, for these cases as long as both `a` and `(b, c)` satisfies primary key constraint (which is the responsibility of the user). 


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


[GitHub] [arrow-datafusion] jonahgao commented on pull request #7160: feat: support primary key aother syntax

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

   Perhaps we should consider scenarios where primary keys are defined multiple times.
   
   For example:
   ```sh
   psql=> create table foo(
           a int primary key, 
           b int, 
           c int, 
           primary key(b,c)
   );
   
   ERROR:  multiple primary keys for table "foo" are not allowed
   LINE 5:         primary key(b,c)
   ```
   It will fail in PostgreSQL.


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


[GitHub] [arrow-datafusion] mustafasrepo commented on pull request #7160: feat: support primary key alternate syntax

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

   Sorry for the late response. I was at vacation, and didn't have time to look. Thanks @parkma99 for this PR. Thanks @alamb for the 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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   Sorry, I am at home this week.



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


[GitHub] [arrow-datafusion] parkma99 commented on pull request #7160: feat: support primary key aother syntax

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

   cc @mustafasrepo @jonahgao @alamb 


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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";

Review Comment:
   I add a test case about unique constraints.



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


[GitHub] [arrow-datafusion] mustafasrepo commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   > I am not sure this means a multi part primary key -- postgres errors on this syntax
   > 
   > ```sql
   > postgres=# create table person (id int, name varchar primary key, primary key(id));
   > ERROR:  multiple primary keys for table "person" are not allowed
   > LINE 1: ...e table person (id int, name varchar primary key, primary ke...
   > ```
   > 
   > I think the way this is supposed to be expressed is like
   > 
   > ```sql
   > create table person (id int, name varchar,  primary key(name,  id));
   > ```
   
   I would expect that `name` is primary key, and `id` is primary key (they each consists of unique values, unlike their tuple version is unique which is also the case.) for the query
   ```sql
   create table person (id int, name string primary key, primary key(id))
   ```
    In short, when one want to use composite primary key, they have to use 
   ```sql
   create table person (id int, name varchar,  primary key(name,  id));
   ```
   syntax. However, for defining single primary keys, they have alternative version. However, if this behaviour is misleading, or counter intuitive. We can follow postgres and give an error. However, It seems to me that, there is no reason to restrain. 



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


[GitHub] [arrow-datafusion] parkma99 commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/src/statement.rs:
##########
@@ -132,75 +132,93 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                 if_not_exists,
                 or_replace,
                 ..
-            } if table_properties.is_empty() && with_options.is_empty() => match query {
-                Some(query) => {
-                    let plan = self.query_to_plan(*query, planner_context)?;
-                    let input_schema = plan.schema();
-
-                    let plan = if !columns.is_empty() {
-                        let schema = self.build_schema(columns)?.to_dfschema_ref()?;
-                        if schema.fields().len() != input_schema.fields().len() {
-                            return Err(DataFusionError::Plan(format!(
+            } if table_properties.is_empty() && with_options.is_empty() => {
+                let mut constraints = constraints;
+                for column in &columns {
+                    for option in &column.options {
+                        if let ast::ColumnOption::Unique { is_primary } = option.option {
+                            constraints.push(ast::TableConstraint::Unique {
+                                name: None,
+                                columns: vec![column.name.clone()],
+                                is_primary,
+                            })
+                        }
+                        // TODO (parkma99) handle ForeignKey etc.

Review Comment:
   I think it should not return `Error` because of without any constraints is 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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   no worries and no pressure! I was just trying to understand what the state of this PR was -- it looks like you are aware of the feedback and just haven't had a chance to implement it. I wanted to make sure you weren't waiting on additional feedback
   
   I'll mark it as draft so it doesn't appear on the list of PRs needing 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


[GitHub] [arrow-datafusion] alamb merged pull request #7160: feat: support primary key alternate syntax

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


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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #7160: feat: support primary key alternate syntax

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


##########
datafusion/sql/tests/sql_integration.rs:
##########
@@ -233,6 +233,25 @@ fn plan_create_table_with_pk() {
     let sql = "create table person (id int, name string, primary key(id))";
     let plan = r#"
 CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+
+    let sql = "create table person (id int primary key, name string)";
+    let plan = r#"
+CreateMemoryTable: Bare { table: "person" } constraints=[PrimaryKey([0])]
+  EmptyRelation
+    "#
+    .trim();
+    quick_test(sql, plan);
+}
+
+#[test]
+fn plan_create_table_with_multi_pk() {
+    let sql = "create table person (id int, name string primary key, primary key(id))";

Review Comment:
   I still think two primary keys on the same column doesn't really make sense, but we can always make this check more strict as a follow on PR I think



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