-
Notifications
You must be signed in to change notification settings - Fork 13.7k
[FLINK-12846][table-common] Carry primary key information in TableSchema #10213
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
…ion in TableSchema
|
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit f98778e (Wed Dec 04 15:23:43 UTC 2019) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
|
I dropped unique key information, because we only need to support primary key for TPC-DS optimizations. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I put some comments to the code already, but wanted to ask if those changes are part of any accepted FLIP?
I have some concerns if the concept of PRIMARY KEYS was announced/discussed publicly. I remember reading a document(I think it was from you @wuchong) about different source/sink concepts such as primary key/changeflag etc., but I could not find any references.
| * <p>Concepts about primary key and unique key:</p> | ||
| * <ul> | ||
| * <li> | ||
| * Primary key and unique key can consist of single or multiple columns (fields). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * Primary key and unique key can consist of single or multiple columns (fields). | |
| * A primary key and a unique key can consist of a single or multiple columns (fields). |
| * A primary key or unique key on source will be simply trusted, we won't validate the | ||
| * constraint. The primary key and unique key information will then be used for query | ||
| * optimization. If a bounded or unbounded table source defines any primary key or | ||
| * unique key, it must contain a unique value for each row of data. You cannot have | ||
| * two records having the same value of that field(s). Otherwise, the result of query | ||
| * might be wrong. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe?:
"The primary and unique keys' constraints of a source table are not validated. They are assumed to be correct and used for query optimization. If a bounded or unbounded table source defines any primary or
unique key, it must contain a unique value for each row of data. You cannot have
two records having the same value of that field(s). Otherwise, the result of the query
might be wrong."
What about retract/changelog streams? Do we just assume that changeflag + timestamp are part of every key? I understand this will be handled correctly in the runtime, but I think this should be also explained in this javadoc.
| * might be wrong. | ||
| * </li> | ||
| * <li> | ||
| * A primary key or unique key on sink is a weak constraint. Currently, we won't validate |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't necessarily understand the first sentence. What do you mean by weak constraint? Is it weak because we do not support that yet?
| * compile time. | ||
| * </li> | ||
| * <li> | ||
| * The difference between primary key and unique key is that there can be only one primary |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * The difference between primary key and unique key is that there can be only one primary | |
| * The difference between a primary key and a unique key is that there can only be one primary |
| * </li> | ||
| * <li> | ||
| * The difference between primary key and unique key is that there can be only one primary | ||
| * key and there can be more than one unique key. And a primary key doesn't need to be |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * key and there can be more than one unique key. And a primary key doesn't need to be | |
| * key whereas there can be more than one unique key. A primary key doesn't need to be |
| @Test | ||
| public void testInvalidMultiPrimaryKey() { | ||
| thrown.expectMessage( | ||
| "A primary key [a] have been defined, can not define another primary key [b]"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| "A primary key [a] have been defined, can not define another primary key [b]"); | |
| "A primary key [a] has already been defined. Can not define another primary key [b]"); |
| @Test | ||
| public void testInvalidPrimaryKeyNestedFieldName() { | ||
| thrown.expectMessage( | ||
| "The primary key field 'c.q1' is not existed in the schema"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| "The primary key field 'c.q1' is not existed in the schema"); | |
| "The primary key field 'c.q1' does not exist in the schema"); |
|
|
||
| @Test | ||
| public void testInvalidPrimaryKeyFieldName() { | ||
| thrown.expectMessage("The primary key field 'd' is not existed in the schema"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| thrown.expectMessage("The primary key field 'd' is not existed in the schema"); | |
| thrown.expectMessage("The primary key field 'd' does not exist in the schema"); |
| /** | ||
| * Utilities for {@link org.apache.flink.table.api.TableSchema} validation. | ||
| */ | ||
| public final class TableSchemaValidation { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@Internal
| .collect(Collectors.toMap(TableColumn::getName, Function.identity())); | ||
|
|
||
| // validate primary key | ||
| for (String key : primaryKey) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we validate fields in a key are unique? I think it does not make sense to define the same field in a key multiple times.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes. I think we should validate the key fields are unique.
|
Thanks @dawidwys for the reviewing. I agree with that the concepts of primary key on source/sink should be discussed and accepted publicly. But the background of this issue is that we would like to support more optimizations for TPC-DS where the sources have primary key information in Hive metastore. So we would like to only expose the primary key interface to But I'm not sure whether this still requires a FLIP. What do you think? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I added a few small suggestions to the javadoc. I believe this looks good to use in future docs.
| * Primary key can be consist of single or multiple columns (fields). | ||
| * </li> | ||
| * <li> | ||
| * A primary key on source will be simply trusted, we won't validate the constraint. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * A primary key on source will be simply trusted, we won't validate the constraint. | |
| * A primary key on a source table will simply be trusted, we won't validate the constraint. |
| * A primary key on source will be simply trusted, we won't validate the constraint. | ||
| * The primary key information will then be used for query optimization. If a bounded | ||
| * or unbounded table source defines any primary key, it must contain a unique value | ||
| * for each row of data. You cannot have two records having the same value of that field(s). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * for each row of data. You cannot have two records having the same value of that field(s). | |
| * for each row of data. If your table contains multiple rows with the same primary key values |
| * The primary key information will then be used for query optimization. If a bounded | ||
| * or unbounded table source defines any primary key, it must contain a unique value | ||
| * for each row of data. You cannot have two records having the same value of that field(s). | ||
| * Otherwise, the result of query might be wrong. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * Otherwise, the result of query might be wrong. | |
| * then the result of the query might be wrong. |
| * Otherwise, the result of query might be wrong. | ||
| * </li> | ||
| * <li> | ||
| * A primary key on sink is a weak constraint. Currently, we won't validate the constraint, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * A primary key on sink is a weak constraint. Currently, we won't validate the constraint, | |
| * A primary key on sink is a weak constraint and is not validated by Flink. However, |
| * </li> | ||
| * <li> | ||
| * A primary key on sink is a weak constraint. Currently, we won't validate the constraint, | ||
| * but we may add some check in the future to validate whether the primary key of the query |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| * but we may add some check in the future to validate whether the primary key of the query | |
| * we may add some checks in the future to validate whether the primary key of the query |
|
Hi @wuchong. I did a bit of research on the |
What is the purpose of the change
The primary key is a standard meta information in SQL. And they are important information for optimization, for example, AggregateRemove, AggregateReduceGrouping and state layout optimization for TopN and Join.
So in this PR, we extend TableSchema to carry more information about primary key.
Bridge the primary key of TableSchema to the planner metadata will be done in other PRs.
Brief change log
TableSchemawith some Javadocs.Verifying this change
TableSchemaTestto check the primary key.Does this pull request potentially affect one of the following parts:
@Public(Evolving): (no)Documentation