-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28139][SQL] Add v2 ALTER TABLE implementation. #24937
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
|
@dongjoon-hyun, this PR implements |
|
Test build #106796 has finished for PR 24937 at commit
|
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.
how about
import org.apache.spark.sql.catalyst.plans.logical.sql._
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.
No, I've noted in several places that this is a bad practice because it causes namespace problems (duplicate names accidentally imported) and commit conflicts.
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
Outdated
Show resolved
Hide resolved
|
@jose-torres, this is the alter table PR, if you have time to review. |
|
Test build #106976 has finished for PR 24937 at commit
|
jose-torres
left a comment
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.
Looks generally good to me. Only one major question.
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.
nit: I'd prefer to have a test for compatible-but-truncating types too, even if they currently have the same behavior.
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.
Can you give an example?
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'm thinking cases like LONG to INT, but maybe I'm missing context on how they're abstracted.
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.
There is a test case for this that rejects unsafe casts: AlterTable: update column type must be compatible. Long to int is not allowed.
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.
General question more related to the parser than here: are there tests ensuring that something dangerous doesn't happen if you try to rename/alter/whatever a special identifier like "points.value" directly?
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.
It is possible to alter points.value, so the parser and analysis checks allow it. For example, the "AlterTable: update column map value type" test updates a map's value type from int to long.
Implementations may also support renaming here, since some formats, like Parquet, use an array of structs to store map data. (Similarly, Parquet has a struct to capture nullability of array elements.) So it may be valid to rename points.value, but most implementations would reject it.
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.
Makes sense.
|
Test build #107246 has finished for PR 24937 at commit
|
b7df0e6 to
c40115e
Compare
|
Test build #107291 has finished for PR 24937 at commit
|
c40115e to
c022b37
Compare
|
Test build #107410 has finished for PR 24937 at commit
|
|
|
||
| case (Seq("element", names @ _*), array @ ArrayType(elementStruct: StructType, _)) => | ||
| Some(field.copy(dataType = array.copy(elementType = replace(elementStruct, names, update)))) | ||
|
|
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.
This looks a pretty cool feature to me. Can we document it in TableChange? Tell users that they can use a.b.mapField.key to change map type's key type, etc.
| * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and | ||
| * the table identifier does not include a catalog. | ||
| */ | ||
| object ResolveAlterTable extends Rule[LogicalPlan] { |
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.
can you remind me where we handle the v1 code path for ALTER TABLE?
|
|
||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| try { | ||
| catalog.alterTable(ident, changes: _*) |
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.
not related to this PR but just a thought: for API that's implemented by users and called by Spark, it seems not a good idea to use var-length argument list, as Spark always call this API with all the parameters at hand.
| } | ||
| } | ||
|
|
||
| test("AlterTable: table does not exist") { |
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.
can we create a new test suite for alter table? I'm afraid this test suite will get too big if we put all the DDL tests here.
|
LGTM, can you resolve the conflict please? |
brkyvz
left a comment
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.
LGTM. I have a couple questions around how case sensitivity is handled in column name resolution. I also think that ALTER TABLE SET LOCATION merits its own TableChange operation (minor)
| } | ||
| } | ||
|
|
||
| interface ColumnChange extends TableChange { |
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.
Could you add docs for this please?
| val pos = struct.getFieldIndex(fieldNames.head) | ||
| .getOrElse(throw new IllegalArgumentException(s"Cannot find field: ${fieldNames.head}")) | ||
| val field = struct.fields(pos) | ||
| val replacement: Option[StructField] = if (fieldNames.tail.isEmpty) { |
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 you need to perform name resolution according to case sensitivity instead of strict equality when calling getFieldIndex?
| import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ | ||
| override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { | ||
| case alter @ AlterTableAddColumnsStatement( | ||
| CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => |
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 have a JIRA to follow up on changing these to use the V2SessionCatalog if a catalog is not defined?
| AlterTable( | ||
| v2Catalog.asTableCatalog, ident, | ||
| UnresolvedRelation(alter.tableName), | ||
| Seq(TableChange.setProperty("location", newLoc))) |
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.
Hmm, I feel changing the location of a table deserves its own special TableChange operator, like ChangeLocation rather than a simple setProperty. When I think property, I think TBLPROPERTIES, and the location is separate from that
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.
Not all tables have locations and we're using the convention of passing optional metadata as table properties. I think that given that we don't want to build special support for everything, this is the right way to pass the change.
| val table = alter.table | ||
| def findField(operation: String, fieldName: Array[String]): StructField = { | ||
| // include collections because structs nested in maps and arrays may be altered | ||
| val field = table.schema.findNestedField(fieldName, includeCollections = true) |
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.
how is case sensitivity handled here?
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 believe it uses the resolver, so it should be case sensitive if the analyzer is.
| case _ => | ||
| } | ||
|
|
||
| case alter: AlterTable if alter.childrenResolved => |
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.
Is there a reason that some of these checks are here, and some in the resolved of AlterTable? It'd be nice to hide the complexity inside AlterTable if possible, and do a simpler check here on whether AlterTable is resolved.
However, I would much prefer the case where we can throw a better error message.
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. The meaning of resolved is that references are satisfied and there are no unresolved placeholders -- that's why resolved checks that the referenced fields exist. This checks whether that resolved plan has valid and reasonable changes, like whether the change updates a column from a string to a boolean.
|
Test build #107545 has finished for PR 24937 at commit
|
|
Since it's a pretty big PR and I'd like to merge it first. @rdblue please send a followup PR to add docs to |
|
Thanks, @cloud-fan! |
## What changes were proposed in this pull request? Implement `ALTER TABLE` for v2 tables: * Add `AlterTable` logical plan and `AlterTableExec` physical plan * Convert `ALTER TABLE` parsed plans to `AlterTable` when a v2 catalog is responsible for an identifier * Validate that columns to alter exist in analyzer checks * Fix nested type handling in `CatalogV2Util` ## How was this patch tested? * Add extensive tests in `DataSourceV2SQLSuite` Closes apache#24937 from rdblue/SPARK-28139-add-v2-alter-table. Lead-authored-by: Ryan Blue <[email protected]> Co-authored-by: Ryan Blue <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
What changes were proposed in this pull request?
Implement
ALTER TABLEfor v2 tables:AlterTablelogical plan andAlterTableExecphysical planALTER TABLEparsed plans toAlterTablewhen a v2 catalog is responsible for an identifierCatalogV2UtilHow was this patch tested?
DataSourceV2SQLSuite