Skip to content

Commit 03eb611

Browse files
committed
[SPARK-21164][SQL] Remove isTableSample from Sample and isGenerated from Alias and AttributeReference
## What changes were proposed in this pull request? `isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by #11148 and #11050 Since SQL Generation is removed, we do not need to keep `isTableSample`. ## How was this patch tested? The existing test cases Author: Xiao Li <[email protected]> Closes #18379 from gatorsmile/CleanSample.
1 parent 13c2a4f commit 03eb611

File tree

15 files changed

+40
-54
lines changed

15 files changed

+40
-54
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -874,7 +874,7 @@ class Analyzer(
874874

875875
def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = {
876876
expressions.map {
877-
case a: Alias => Alias(a.child, a.name)(isGenerated = a.isGenerated)
877+
case a: Alias => Alias(a.child, a.name)()
878878
case other => other
879879
}
880880
}
@@ -1368,7 +1368,7 @@ class Analyzer(
13681368
val aggregatedCondition =
13691369
Aggregate(
13701370
grouping,
1371-
Alias(havingCondition, "havingCondition")(isGenerated = true) :: Nil,
1371+
Alias(havingCondition, "havingCondition")() :: Nil,
13721372
child)
13731373
val resolvedOperator = execute(aggregatedCondition)
13741374
def resolvedAggregateFilter =
@@ -1424,7 +1424,7 @@ class Analyzer(
14241424
try {
14251425
val unresolvedSortOrders = sortOrder.filter(s => !s.resolved || containsAggregate(s))
14261426
val aliasedOrdering =
1427-
unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")(isGenerated = true))
1427+
unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")())
14281428
val aggregatedOrdering = aggregate.copy(aggregateExpressions = aliasedOrdering)
14291429
val resolvedAggregate: Aggregate = execute(aggregatedOrdering).asInstanceOf[Aggregate]
14301430
val resolvedAliasedOrdering: Seq[Alias] =
@@ -1935,7 +1935,7 @@ class Analyzer(
19351935
leafNondeterministic.distinct.map { e =>
19361936
val ne = e match {
19371937
case n: NamedExpression => n
1938-
case _ => Alias(e, "_nondeterministic")(isGenerated = true)
1938+
case _ => Alias(e, "_nondeterministic")()
19391939
}
19401940
e -> ne
19411941
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala

Lines changed: 12 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -81,9 +81,6 @@ trait NamedExpression extends Expression {
8181
/** Returns the metadata when an expression is a reference to another expression with metadata. */
8282
def metadata: Metadata = Metadata.empty
8383

84-
/** Returns true if the expression is generated by Catalyst */
85-
def isGenerated: java.lang.Boolean = false
86-
8784
/** Returns a copy of this expression with a new `exprId`. */
8885
def newInstance(): NamedExpression
8986

@@ -128,13 +125,11 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn
128125
* qualified way. Consider the examples tableName.name, subQueryAlias.name.
129126
* tableName and subQueryAlias are possible qualifiers.
130127
* @param explicitMetadata Explicit metadata associated with this alias that overwrites child's.
131-
* @param isGenerated A flag to indicate if this alias is generated by Catalyst
132128
*/
133129
case class Alias(child: Expression, name: String)(
134130
val exprId: ExprId = NamedExpression.newExprId,
135131
val qualifier: Option[String] = None,
136-
val explicitMetadata: Option[Metadata] = None,
137-
override val isGenerated: java.lang.Boolean = false)
132+
val explicitMetadata: Option[Metadata] = None)
138133
extends UnaryExpression with NamedExpression {
139134

140135
// Alias(Generator, xx) need to be transformed into Generate(generator, ...)
@@ -159,13 +154,11 @@ case class Alias(child: Expression, name: String)(
159154
}
160155

161156
def newInstance(): NamedExpression =
162-
Alias(child, name)(
163-
qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated)
157+
Alias(child, name)(qualifier = qualifier, explicitMetadata = explicitMetadata)
164158

165159
override def toAttribute: Attribute = {
166160
if (resolved) {
167-
AttributeReference(name, child.dataType, child.nullable, metadata)(
168-
exprId, qualifier, isGenerated)
161+
AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifier)
169162
} else {
170163
UnresolvedAttribute(name)
171164
}
@@ -174,7 +167,7 @@ case class Alias(child: Expression, name: String)(
174167
override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix"
175168

176169
override protected final def otherCopyArgs: Seq[AnyRef] = {
177-
exprId :: qualifier :: explicitMetadata :: isGenerated :: Nil
170+
exprId :: qualifier :: explicitMetadata :: Nil
178171
}
179172

180173
override def hashCode(): Int = {
@@ -207,16 +200,14 @@ case class Alias(child: Expression, name: String)(
207200
* @param qualifier An optional string that can be used to referred to this attribute in a fully
208201
* qualified way. Consider the examples tableName.name, subQueryAlias.name.
209202
* tableName and subQueryAlias are possible qualifiers.
210-
* @param isGenerated A flag to indicate if this reference is generated by Catalyst
211203
*/
212204
case class AttributeReference(
213205
name: String,
214206
dataType: DataType,
215207
nullable: Boolean = true,
216208
override val metadata: Metadata = Metadata.empty)(
217209
val exprId: ExprId = NamedExpression.newExprId,
218-
val qualifier: Option[String] = None,
219-
override val isGenerated: java.lang.Boolean = false)
210+
val qualifier: Option[String] = None)
220211
extends Attribute with Unevaluable {
221212

222213
/**
@@ -253,8 +244,7 @@ case class AttributeReference(
253244
}
254245

255246
override def newInstance(): AttributeReference =
256-
AttributeReference(name, dataType, nullable, metadata)(
257-
qualifier = qualifier, isGenerated = isGenerated)
247+
AttributeReference(name, dataType, nullable, metadata)(qualifier = qualifier)
258248

259249
/**
260250
* Returns a copy of this [[AttributeReference]] with changed nullability.
@@ -263,15 +253,15 @@ case class AttributeReference(
263253
if (nullable == newNullability) {
264254
this
265255
} else {
266-
AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier, isGenerated)
256+
AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier)
267257
}
268258
}
269259

270260
override def withName(newName: String): AttributeReference = {
271261
if (name == newName) {
272262
this
273263
} else {
274-
AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier, isGenerated)
264+
AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier)
275265
}
276266
}
277267

@@ -282,24 +272,24 @@ case class AttributeReference(
282272
if (newQualifier == qualifier) {
283273
this
284274
} else {
285-
AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier, isGenerated)
275+
AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier)
286276
}
287277
}
288278

289279
def withExprId(newExprId: ExprId): AttributeReference = {
290280
if (exprId == newExprId) {
291281
this
292282
} else {
293-
AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier, isGenerated)
283+
AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier)
294284
}
295285
}
296286

297287
override def withMetadata(newMetadata: Metadata): Attribute = {
298-
AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier, isGenerated)
288+
AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier)
299289
}
300290

301291
override protected final def otherCopyArgs: Seq[AnyRef] = {
302-
exprId :: qualifier :: isGenerated :: Nil
292+
exprId :: qualifier :: Nil
303293
}
304294

305295
/** Used to signal the column used to calculate an eventTime watermark (e.g. a#1-T{delayMs}) */

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -134,7 +134,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] {
134134
// Aggregation strategy can handle queries with a single distinct group.
135135
if (distinctAggGroups.size > 1) {
136136
// Create the attributes for the grouping id and the group by clause.
137-
val gid = AttributeReference("gid", IntegerType, nullable = false)(isGenerated = true)
137+
val gid = AttributeReference("gid", IntegerType, nullable = false)()
138138
val groupByMap = a.groupingExpressions.collect {
139139
case ne: NamedExpression => ne -> ne.toAttribute
140140
case e => e -> AttributeReference(e.sql, e.dataType, e.nullable)()

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -627,7 +627,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
627627
validate(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
628628
s"Sampling fraction ($fraction) must be on interval [0, 1]",
629629
ctx)
630-
Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
630+
Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)
631631
}
632632

633633
ctx.sampleType.getType match {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -80,12 +80,12 @@ object PhysicalOperation extends PredicateHelper {
8080
expr.transform {
8181
case a @ Alias(ref: AttributeReference, name) =>
8282
aliases.get(ref)
83-
.map(Alias(_, name)(a.exprId, a.qualifier, isGenerated = a.isGenerated))
83+
.map(Alias(_, name)(a.exprId, a.qualifier))
8484
.getOrElse(a)
8585

8686
case a: AttributeReference =>
8787
aliases.get(a)
88-
.map(Alias(_, a.name)(a.exprId, a.qualifier, isGenerated = a.isGenerated)).getOrElse(a)
88+
.map(Alias(_, a.name)(a.exprId, a.qualifier)).getOrElse(a)
8989
}
9090
}
9191
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -200,7 +200,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT
200200
// normalize that for equality testing, by assigning expr id from 0 incrementally. The
201201
// alias name doesn't matter and should be erased.
202202
val normalizedChild = QueryPlan.normalizeExprId(a.child, allAttributes)
203-
Alias(normalizedChild, "")(ExprId(id), a.qualifier, isGenerated = a.isGenerated)
203+
Alias(normalizedChild, "")(ExprId(id), a.qualifier)
204204

205205
case ar: AttributeReference if allAttributes.indexOf(ar.exprId) == -1 =>
206206
// Top level `AttributeReference` may also be used for output like `Alias`, we should

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -221,7 +221,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with QueryPlanConstrai
221221
nameParts: Seq[String],
222222
resolver: Resolver,
223223
attribute: Attribute): Option[(Attribute, List[String])] = {
224-
if (!attribute.isGenerated && resolver(attribute.name, nameParts.head)) {
224+
if (resolver(attribute.name, nameParts.head)) {
225225
Option((attribute.withName(nameParts.head), nameParts.tail.toList))
226226
} else {
227227
None

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -807,15 +807,13 @@ case class SubqueryAlias(
807807
* @param withReplacement Whether to sample with replacement.
808808
* @param seed the random seed
809809
* @param child the LogicalPlan
810-
* @param isTableSample Is created from TABLESAMPLE in the parser.
811810
*/
812811
case class Sample(
813812
lowerBound: Double,
814813
upperBound: Double,
815814
withReplacement: Boolean,
816815
seed: Long,
817-
child: LogicalPlan)(
818-
val isTableSample: java.lang.Boolean = false) extends UnaryNode {
816+
child: LogicalPlan) extends UnaryNode {
819817

820818
val eps = RandomSampler.roundingEpsilon
821819
val fraction = upperBound - lowerBound
@@ -842,8 +840,6 @@ case class Sample(
842840
// Don't propagate column stats, because we don't know the distribution after a sample operation
843841
Statistics(sizeInBytes, sampledRowCount, hints = childStats.hints)
844842
}
845-
846-
override protected def otherCopyArgs: Seq[AnyRef] = isTableSample :: Nil
847843
}
848844

849845
/**

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -573,7 +573,7 @@ class AnalysisErrorSuite extends AnalysisTest {
573573
val plan5 = Filter(
574574
Exists(
575575
Sample(0.0, 0.5, false, 1L,
576-
Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b)))().select('b)
576+
Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b))).select('b)
577577
),
578578
LocalRelation(a))
579579
assertAnalysisError(plan5,

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -491,7 +491,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
491491

492492
// Other unary operations
493493
testUnaryOperatorInStreamingPlan(
494-
"sample", Sample(0.1, 1, true, 1L, _)(), expectedMsg = "sampling")
494+
"sample", Sample(0.1, 1, true, 1L, _), expectedMsg = "sampling")
495495
testUnaryOperatorInStreamingPlan(
496496
"window", Window(Nil, Nil, Nil, _), expectedMsg = "non-time-based windows")
497497

0 commit comments

Comments
 (0)