-
Notifications
You must be signed in to change notification settings - Fork 2.8k
/
SparkScala212Interpreter.scala
326 lines (287 loc) · 12.4 KB
/
SparkScala212Interpreter.scala
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
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
/*
* 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.zeppelin.spark
import org.apache.spark.SparkConf
import org.apache.spark.repl.SparkILoop
import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion
import org.apache.zeppelin.interpreter.util.InterpreterOutputStream
import org.apache.zeppelin.interpreter.{InterpreterContext, InterpreterException, InterpreterGroup, InterpreterResult}
import org.apache.zeppelin.kotlin.KotlinInterpreter
import org.slf4j.{Logger, LoggerFactory}
import java.io.{BufferedReader, File, PrintStream}
import java.net.URLClassLoader
import java.nio.file.Paths
import java.util.Properties
import scala.collection.JavaConverters._
import scala.tools.nsc.Settings
import scala.tools.nsc.interpreter.ILoop.loopToInterpreter
import scala.tools.nsc.interpreter._
/**
* SparkInterpreter for scala-2.12.
* It is used by both Spark 2.x and 3.x
*/
class SparkScala212Interpreter(conf: SparkConf,
depFiles: java.util.List[String],
properties: Properties,
interpreterGroup: InterpreterGroup,
sparkInterpreterClassLoader: URLClassLoader,
outputDir: File) extends AbstractSparkScalaInterpreter(conf, properties, depFiles) {
private lazy val LOGGER: Logger = LoggerFactory.getLogger(getClass)
private var sparkILoop: SparkILoop = _
private var scalaCompletion: Completion = _
private val interpreterOutput = new InterpreterOutputStream(LOGGER)
private val sparkMaster: String = conf.get(SparkStringConstants.MASTER_PROP_NAME,
SparkStringConstants.DEFAULT_MASTER_VALUE)
override def interpret(code: String, context: InterpreterContext): InterpreterResult = {
val originalOut = System.out
val printREPLOutput = context.getStringLocalProperty("printREPLOutput", "true").toBoolean
def _interpret(code: String): scala.tools.nsc.interpreter.Results.Result = {
Console.withOut(interpreterOutput) {
System.setOut(Console.out)
if (printREPLOutput) {
interpreterOutput.setInterpreterOutput(context.out)
} else {
interpreterOutput.setInterpreterOutput(null)
}
interpreterOutput.ignoreLeadingNewLinesFromScalaReporter()
val status = scalaInterpret(code) match {
case success@scala.tools.nsc.interpreter.IR.Success =>
success
case scala.tools.nsc.interpreter.IR.Error =>
val errorMsg = new String(interpreterOutput.getInterpreterOutput.toByteArray)
if (errorMsg.contains("value toDF is not a member of org.apache.spark.rdd.RDD") ||
errorMsg.contains("value toDS is not a member of org.apache.spark.rdd.RDD")) {
// prepend "import sqlContext.implicits._" due to
// https://issues.scala-lang.org/browse/SI-6649
context.out.clear()
scalaInterpret("import sqlContext.implicits._\n" + code)
} else {
scala.tools.nsc.interpreter.IR.Error
}
case scala.tools.nsc.interpreter.IR.Incomplete =>
// add print("") at the end in case the last line is comment which lead to INCOMPLETE
scalaInterpret(code + "\nprint(\"\")")
}
context.out.flush()
status
}
}
// reset the java stdout
System.setOut(originalOut)
context.out.write("")
val lastStatus = _interpret(code) match {
case scala.tools.nsc.interpreter.IR.Success =>
InterpreterResult.Code.SUCCESS
case scala.tools.nsc.interpreter.IR.Error =>
InterpreterResult.Code.ERROR
case scala.tools.nsc.interpreter.IR.Incomplete =>
InterpreterResult.Code.INCOMPLETE
}
lastStatus match {
case InterpreterResult.Code.INCOMPLETE => new InterpreterResult(lastStatus, "Incomplete expression")
case _ => new InterpreterResult(lastStatus)
}
}
override def completion(buf: String,
cursor: Int,
context: InterpreterContext): java.util.List[InterpreterCompletion] = {
scalaCompletion.complete(buf.substring(0, cursor), cursor)
.candidates
.map(e => new InterpreterCompletion(e, e, null))
.asJava
}
private def bind(name: String, tpe: String, value: Object, modifier: List[String]): Unit = {
sparkILoop.beQuietDuring {
val result = sparkILoop.bind(name, tpe, value, modifier)
if (result != IR.Success) {
throw new RuntimeException("Fail to bind variable: " + name)
}
}
}
override def bind(name: String,
tpe: String,
value: Object,
modifier: java.util.List[String]): Unit =
bind(name, tpe, value, modifier.asScala.toList)
def scalaInterpret(code: String): scala.tools.nsc.interpreter.IR.Result =
sparkILoop.interpret(code)
@throws[InterpreterException]
def scalaInterpretQuietly(code: String): Unit = {
scalaInterpret(code) match {
case scala.tools.nsc.interpreter.Results.Success =>
// do nothing
case scala.tools.nsc.interpreter.Results.Error =>
throw new InterpreterException("Fail to run code: " + code)
case scala.tools.nsc.interpreter.Results.Incomplete =>
throw new InterpreterException("Incomplete code: " + code)
}
}
override def getScalaShellClassLoader: ClassLoader = {
sparkILoop.classLoader
}
// Used by KotlinSparkInterpreter
override def delegateInterpret(interpreter: KotlinInterpreter,
code: String,
context: InterpreterContext): InterpreterResult = {
val out = context.out
val newOut = if (out != null) new PrintStream(out) else null
Console.withOut(newOut) {
interpreter.interpret(code, context)
}
}
def interpret(code: String): InterpreterResult =
interpret(code, InterpreterContext.get())
override def close(): Unit = {
super.close()
if (sparkILoop != null) {
sparkILoop.closeInterpreter()
}
}
override def createSparkILoop(): Unit = {
if (sparkMaster == "yarn-client") {
System.setProperty("SPARK_YARN_MODE", "true")
}
LOGGER.info("Scala shell repl output dir: " + outputDir.getAbsolutePath)
conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath)
val settings = new Settings()
settings.processArguments(List("-Yrepl-class-based",
"-Yrepl-outdir", s"${outputDir.getAbsolutePath}"), true)
settings.embeddedDefaults(sparkInterpreterClassLoader)
settings.usejavacp.value = true
val userJars = getUserJars()
LOGGER.info("UserJars: " + userJars.mkString(File.pathSeparator))
settings.classpath.value = userJars.mkString(File.pathSeparator)
val printReplOutput = properties.getProperty("zeppelin.spark.printREPLOutput", "true").toBoolean
val replOut = if (printReplOutput) {
new JPrintWriter(interpreterOutput, true)
} else {
new JPrintWriter(Console.out, true)
}
sparkILoop = new SparkILoop(None, replOut)
sparkILoop.settings = settings
sparkILoop.createInterpreter()
val in0 = getDeclareField(sparkILoop, "in0").asInstanceOf[Option[BufferedReader]]
val reader = in0.fold(sparkILoop.chooseReader(settings))(r => SimpleReader(r, replOut, interactive = true))
sparkILoop.in = reader
sparkILoop.initializeSynchronous()
SparkScala212Interpreter.loopPostInit(this)
this.scalaCompletion = reader.completion
}
override def createZeppelinContext(): Unit = {
val sparkShims = SparkShims.getInstance(sc.version, properties, sparkSession)
sparkShims.setupSparkListener(sc.master, sparkUrl, InterpreterContext.get)
z = new SparkZeppelinContext(sc, sparkShims,
interpreterGroup.getInterpreterHookRegistry,
properties.getProperty("zeppelin.spark.maxResult", "1000").toInt)
bind("z", z.getClass.getCanonicalName, z, List("""@transient"""))
}
private def getDeclareField(obj: Object, name: String): Object = {
val field = obj.getClass.getDeclaredField(name)
field.setAccessible(true)
field.get(obj)
}
private def callMethod(obj: Object, name: String,
parameterTypes: Array[Class[_]],
parameters: Array[Object]): Object = {
val method = obj.getClass.getMethod(name, parameterTypes: _ *)
method.setAccessible(true)
method.invoke(obj, parameters: _ *)
}
private def getUserJars(): Seq[String] = {
var classLoader = Thread.currentThread().getContextClassLoader
var extraJars = Seq.empty[String]
while (classLoader != null) {
if (classLoader.getClass.getCanonicalName ==
"org.apache.spark.util.MutableURLClassLoader") {
extraJars = classLoader.asInstanceOf[URLClassLoader].getURLs()
// Check if the file exists.
.filter { u => u.getProtocol == "file" && new File(u.getPath).isFile }
// Some bad spark packages depend on the wrong version of scala-reflect. Blacklist it.
.filterNot {
u => Paths.get(u.toURI).getFileName.toString.contains("org.scala-lang_scala-reflect")
}
.map(url => url.toString).toSeq
classLoader = null
} else {
classLoader = classLoader.getParent
}
}
extraJars ++= sparkInterpreterClassLoader.getURLs().map(_.getPath())
LOGGER.debug("User jar for spark repl: " + extraJars.mkString(","))
extraJars
}
}
private object SparkScala212Interpreter {
/**
* This is a hack to call `loopPostInit` at `ILoop`. At higher version of Scala such
* as 2.11.12, `loopPostInit` became a nested function which is inaccessible. Here,
* we redefine `loopPostInit` at Scala's 2.11.8 side and ignore `loadInitFiles` being called at
* Scala 2.11.12 since here we do not have to load files.
*
* Both methods `loopPostInit` and `unleashAndSetPhase` are redefined, and `phaseCommand` and
* `asyncMessage` are being called via reflection since both exist in Scala 2.11.8 and 2.11.12.
*
* Please see the codes below:
* https://github.com/scala/scala/blob/v2.11.8/src/repl/scala/tools/nsc/interpreter/ILoop.scala
* https://github.com/scala/scala/blob/v2.11.12/src/repl/scala/tools/nsc/interpreter/ILoop.scala
*
* See also ZEPPELIN-3810.
*/
private def loopPostInit(interpreter: SparkScala212Interpreter): Unit = {
import StdReplTags._
import scala.reflect.{classTag, io}
val sparkILoop = interpreter.sparkILoop
val intp = sparkILoop.intp
val power = sparkILoop.power
val in = sparkILoop.in
def loopPostInit() {
// Bind intp somewhere out of the regular namespace where
// we can get at it in generated code.
intp.quietBind(NamedParam[IMain]("$intp", intp)(tagOfIMain, classTag[IMain]))
// Auto-run code via some setting.
(replProps.replAutorunCode.option
flatMap (f => io.File(f).safeSlurp())
foreach (intp quietRun _)
)
// classloader and power mode setup
intp.setContextClassLoader()
if (isReplPower) {
replProps.power setValue true
unleashAndSetPhase()
asyncMessage(power.banner)
}
// SI-7418 Now, and only now, can we enable TAB completion.
in.postInit()
}
def unleashAndSetPhase() = if (isReplPower) {
power.unleash()
intp beSilentDuring phaseCommand("typer") // Set the phase to "typer"
}
def phaseCommand(name: String): Results.Result = {
interpreter.callMethod(
sparkILoop,
"scala$tools$nsc$interpreter$ILoop$$phaseCommand",
Array(classOf[String]),
Array(name)).asInstanceOf[Results.Result]
}
def asyncMessage(msg: String): Unit = {
interpreter.callMethod(
sparkILoop, "asyncMessage", Array(classOf[String]), Array(msg))
}
loopPostInit()
}
}