1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
|
/*
* 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.hive.execution
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
import org.scalatest.exceptions.TestFailedException
import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode}
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.types.StringType
class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton {
import hiveContext.implicits._
private val noSerdeIOSchema = HiveScriptIOSchema(
inputRowFormat = Seq.empty,
outputRowFormat = Seq.empty,
inputSerdeClass = None,
outputSerdeClass = None,
inputSerdeProps = Seq.empty,
outputSerdeProps = Seq.empty,
recordReaderClass = None,
recordWriterClass = None,
schemaLess = false
)
private val serdeIOSchema = noSerdeIOSchema.copy(
inputSerdeClass = Some(classOf[LazySimpleSerDe].getCanonicalName),
outputSerdeClass = Some(classOf[LazySimpleSerDe].getCanonicalName)
)
test("cat without SerDe") {
val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a")
checkAnswer(
rowsDf,
(child: SparkPlan) => new ScriptTransformation(
input = Seq(rowsDf.col("a").expr),
script = "cat",
output = Seq(AttributeReference("a", StringType)()),
child = child,
ioschema = noSerdeIOSchema
),
rowsDf.collect())
}
test("cat with LazySimpleSerDe") {
val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a")
checkAnswer(
rowsDf,
(child: SparkPlan) => new ScriptTransformation(
input = Seq(rowsDf.col("a").expr),
script = "cat",
output = Seq(AttributeReference("a", StringType)()),
child = child,
ioschema = serdeIOSchema
),
rowsDf.collect())
}
test("script transformation should not swallow errors from upstream operators (no serde)") {
val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a")
val e = intercept[TestFailedException] {
checkAnswer(
rowsDf,
(child: SparkPlan) => new ScriptTransformation(
input = Seq(rowsDf.col("a").expr),
script = "cat",
output = Seq(AttributeReference("a", StringType)()),
child = ExceptionInjectingOperator(child),
ioschema = noSerdeIOSchema
),
rowsDf.collect())
}
assert(e.getMessage().contains("intentional exception"))
}
test("script transformation should not swallow errors from upstream operators (with serde)") {
val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a")
val e = intercept[TestFailedException] {
checkAnswer(
rowsDf,
(child: SparkPlan) => new ScriptTransformation(
input = Seq(rowsDf.col("a").expr),
script = "cat",
output = Seq(AttributeReference("a", StringType)()),
child = ExceptionInjectingOperator(child),
ioschema = serdeIOSchema
),
rowsDf.collect())
}
assert(e.getMessage().contains("intentional exception"))
}
}
private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode {
override protected def doExecute(): RDD[InternalRow] = {
child.execute().map { x =>
assert(TaskContext.get() != null) // Make sure that TaskContext is defined.
Thread.sleep(1000) // This sleep gives the external process time to start.
throw new IllegalArgumentException("intentional exception")
}
}
override def output: Seq[Attribute] = child.output
}
|