Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
Rewrite Alias in StreamExecution if necessary
  • Loading branch information
zsxwing committed Jan 12, 2017
commit 13f54a93c0cf31a38455e90aec722e890af980c6
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.execution.command.ExplainCommand
Expand Down Expand Up @@ -495,8 +495,13 @@ class StreamExecution(

// Rewire the plan to use the new attributes that were returned by the source.
val replacementMap = AttributeMap(replacements)
val exprIdMap =
replacements.map { case (oldAttr, newAttr) => (oldAttr.exprId, newAttr.exprId)}.toMap
val triggerLogicalPlan = withNewSources transformAllExpressions {
case a: Attribute if replacementMap.contains(a) => replacementMap(a)
case a: Alias if exprIdMap.contains(a.exprId) =>
// Also rewrite `Alias`s as they may use the same `exprId` of `Attribute`s.
Alias(a.child, a.name)(exprIdMap(a.exprId), a.qualifier, a.explicitMetadata, a.isGenerated)
case ct: CurrentTimestamp =>
CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs,
ct.dataType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -304,6 +304,32 @@ class StreamSuite extends StreamTest {
q.stop()
}
}

test("SPARK-19065 Alia should be replaced as well") {
withTempPath { testPath =>
val data = Seq((1, 2), (2, 3), (3, 4))
data.toDS.write.mode("overwrite").json(testPath.getCanonicalPath)
val schema = spark.read.json(testPath.getCanonicalPath).schema
val query = spark
.readStream
.schema(schema)
.json(testPath.getCanonicalPath)
.dropDuplicates("_1") // dropDuplicates will create an Alias using the same exprId.
.writeStream
.format("memory")
.queryName("testquery")
.outputMode("complete")
.start()
try {
query.processAllAvailable()
if (query.exception.isDefined) {
throw query.exception.get
}
} finally {
query.stop()
}
}
}
}

/**
Expand Down