[GitHub] [spark] dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation

2019-06-05 Thread GitBox
dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] 
from_avro shouldn't produces same value when converted to local relation
URL: https://github.com/apache/spark/pull/24805#discussion_r290843329
 
 

 ##
 File path: 
external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
 ##
 @@ -1491,4 +1494,38 @@ class AvroSuite extends QueryTest with SharedSQLContext 
with SQLTestUtils {
   |}
 """.stripMargin)
   }
+
+  test("SPARK-27798: from_avro produces same value when converted to local 
relation") {
+val simpleSchema =
+  """
+|{
+|"type": "record",
 
 Review comment:
   nit. indentation?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation

2019-06-05 Thread GitBox
dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] 
from_avro shouldn't produces same value when converted to local relation
URL: https://github.com/apache/spark/pull/24805#discussion_r290844064
 
 

 ##
 File path: 
external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
 ##
 @@ -1491,4 +1494,38 @@ class AvroSuite extends QueryTest with SharedSQLContext 
with SQLTestUtils {
   |}
 """.stripMargin)
   }
+
+  test("SPARK-27798: from_avro produces same value when converted to local 
relation") {
+val simpleSchema =
+  """
+|{
+|"type": "record",
+|"name" : "Payload",
+|"fields" : [ {"name" : "message", "type" : "string" } ]
+|}
+  """.stripMargin
+
+def generateBinary(message: String, avroSchema: String): Array[Byte] = {
+  val schema = new Schema.Parser().parse(avroSchema)
+  val out = new ByteArrayOutputStream()
+  val writer = new GenericDatumWriter[GenericRecord](schema)
+  val encoder = EncoderFactory.get().binaryEncoder(out, null)
+  val rootRecord = new GenericRecordBuilder(schema).set("message", 
message).build()
+  writer.write(rootRecord, encoder)
+  encoder.flush()
+  out.toByteArray
+}
+
+// This bug is hit when the rule `ConvertToLocalRelation` is run. But the 
rule was excluded
+// in `SharedSparkSession`.
 
 Review comment:
   Thank you for this notes.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation

2019-06-05 Thread GitBox
dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] 
from_avro shouldn't produces same value when converted to local relation
URL: https://github.com/apache/spark/pull/24805#discussion_r290845722
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ##
 @@ -1420,9 +1420,9 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
 case Project(projectList, LocalRelation(output, data, isStreaming))
 if !projectList.exists(hasUnevaluableExpr) =>
-  val projection = new InterpretedProjection(projectList, output)
+  val projection = new InterpretedMutableProjection(projectList, output)
   projection.initialize(0)
-  LocalRelation(projectList.map(_.toAttribute), data.map(projection), 
isStreaming)
+  LocalRelation(projectList.map(_.toAttribute), 
data.map(projection(_).copy()), isStreaming)
 
 Review comment:
   It's good to have `AvroSuite` test case because it's a reported instance.
   However, since this is `catalyst` Optimizer issue, we had better a test 
coverage in `catalyst` module. Could you add a test case in `catalyst` module 
with other data sources, please?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation

2019-06-05 Thread GitBox
dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] 
from_avro shouldn't produces same value when converted to local relation
URL: https://github.com/apache/spark/pull/24805#discussion_r290845856
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ##
 @@ -1420,9 +1420,9 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transform {
 case Project(projectList, LocalRelation(output, data, isStreaming))
 if !projectList.exists(hasUnevaluableExpr) =>
-  val projection = new InterpretedProjection(projectList, output)
+  val projection = new InterpretedMutableProjection(projectList, output)
   projection.initialize(0)
-  LocalRelation(projectList.map(_.toAttribute), data.map(projection), 
isStreaming)
+  LocalRelation(projectList.map(_.toAttribute), 
data.map(projection(_).copy()), isStreaming)
 
 Review comment:
   cc @gatorsmile 


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] from_avro shouldn't produces same value when converted to local relation

2019-06-06 Thread GitBox
dongjoon-hyun commented on a change in pull request #24805: [SPARK-27798][SQL] 
from_avro shouldn't produces same value when converted to local relation
URL: https://github.com/apache/spark/pull/24805#discussion_r291440164
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala
 ##
 @@ -70,4 +72,36 @@ class ConvertToLocalRelationSuite extends PlanTest {
 
 comparePlans(optimized, correctAnswer)
   }
+
+  test("SPARK-27798: Expression reusing output shouldn't override values in 
local relation") {
 
 Review comment:
   Thank you for adding this, @viirya .


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:
us...@infra.apache.org


With regards,
Apache Git Services

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