cloud-fan commented on code in PR #40456:
URL: https://github.com/apache/spark/pull/40456#discussion_r1140294460


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExtractDistributedSequenceID.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
DistributedSequenceID}
+import org.apache.spark.sql.catalyst.plans.logical.{AttachDistributedSequence, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.DISTRIBUTED_SEQUENCE_ID
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Extracts [[DistributedSequenceID]] in logical plans, and replace it to
+ * [[AttachDistributedSequence]] because this expressions requires a shuffle
+ * to generate a sequence that needs the context of the whole data, e.g.,
+ * [[org.apache.spark.rdd.RDD.zipWithIndex]].
+ */
+object ExtractDistributedSequenceID extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    
plan.resolveOperatorsUpWithPruning(_.containsPattern(DISTRIBUTED_SEQUENCE_ID)) {
+      case plan: LogicalPlan if
+          
plan.expressions.exists(_.exists(_.isInstanceOf[DistributedSequenceID])) =>
+        val attr = AttributeReference("distributed_sequence_id", LongType, 
nullable = false)()
+        val newPlan = 
plan.withNewChildren(plan.children.map(AttachDistributedSequence(attr, _)))
+          .transformExpressions { case _: DistributedSequenceID => attr }
+        Project(plan.output, newPlan)

Review Comment:
   We can only call `plan.outout` if `plan` is resolved.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExtractDistributedSequenceID.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
DistributedSequenceID}
+import org.apache.spark.sql.catalyst.plans.logical.{AttachDistributedSequence, 
LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.DISTRIBUTED_SEQUENCE_ID
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Extracts [[DistributedSequenceID]] in logical plans, and replace it to
+ * [[AttachDistributedSequence]] because this expressions requires a shuffle
+ * to generate a sequence that needs the context of the whole data, e.g.,
+ * [[org.apache.spark.rdd.RDD.zipWithIndex]].
+ */
+object ExtractDistributedSequenceID extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    
plan.resolveOperatorsUpWithPruning(_.containsPattern(DISTRIBUTED_SEQUENCE_ID)) {
+      case plan: LogicalPlan if

Review Comment:
   ```suggestion
         case plan: LogicalPlan if plan.resolved &&
   ```



-- 
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: reviews-unsubscr...@spark.apache.org

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


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

Reply via email to