You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/02/02 18:37:51 UTC

[GitHub] [spark] imback82 commented on a change in pull request #31440: [SPARK-34331][SQL] Speed up DS v2 metadata col resolution

imback82 commented on a change in pull request #31440:
URL: https://github.com/apache/spark/pull/31440#discussion_r568839896



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -4034,3 +4009,37 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] {
     if (targetLength > charLength) StringRPad(expr, Literal(targetLength)) else expr
   }
 }
+
+/**
+ * This rule removes metadata columns from `DataSourceV2Relation` under 2 cases:
+ *   - A single v2 scan (can be produced by `spark.table`), which is similar to star expansion, and
+ *     metadata columns should only be picked by explicit references.
+ *   - V2 scans under writing commands, as we can't insert into metadata columns.
+ */
+object RemoveMetadataColumns extends Rule[LogicalPlan] {
+  import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._
+
+  private def removeMetaCol(tbl: NamedRelation): NamedRelation = tbl match {
+    case r: DataSourceV2Relation =>
+      if (r.output.exists(_.isMetadataCol)) {
+        r.copy(output = r.output.filterNot(_.isMetadataCol))
+      } else {
+        r
+      }
+    case _ => tbl
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+    case r: DataSourceV2Relation => removeMetaCol(r)
+    case s @ SubqueryAlias(_, r: DataSourceV2Relation) => s.copy(child = removeMetaCol(r))
+    case _ => plan.resolveOperators {
+      case i: InsertIntoStatement => i.copy(i.table match {
+        case r: DataSourceV2Relation => removeMetaCol(r)
+        case other => other
+      })
+      case a: AppendData => a.withNewTable(removeMetaCol(a.table))
+      case o: OverwriteByExpression => o.withNewTable(removeMetaCol(o.table))
+      case o: OverwritePartitionsDynamic => o.withNewTable(removeMetaCol(o.table))

Review comment:
       Can these be replaced with `case v: V2WriteCommand => v.withNewTable(removeMetaCol(v.table))`, or do we need to match these specific types?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org