Skip to content

Commit

Permalink
Support scala 2.13 console in thin client
Browse files Browse the repository at this point in the history
In order to make the console task work with scala 2.13 and the thin
client, we need to provide a way for the scala repl to use an sbt
provided jline3 terminal instead of the default terminal typically built
by the repl. We also need to put jline 3 higher up in the classloading
hierarchy to ensure that two versions of jline 3 are not loaded (which
makes it impossible to share the sbt terminal with the scala terminal).

One impact of this change is the decoupling of the version of
jline-terminal used by the in process scala console and the version
of jline-terminal specified by the scala version itself. It is possible
to override this by setting the `useScalaReplJLine` flag to true. When
that is set, the scala REPL will run in a fully isolated classloader. That
will ensure that the versions are consistent. It will, however, for sure
break the thin client and may interfere with the embedded shell ui.

As part of this work, I also discovered that jline 3 Terminal.getSize is
very slow. In jline 2, the terminal attributes were automatically cached with a
timeout of, I think, 1 second so it wasn't a big deal to call
Terminal.getAttributes. The getSize method in jline 3 is not cached and
it shells out to run a tty command. This caused a significant
performance regression in sbt because when progress is enabled, we call
Terminal.getSize whenever we log any messages. I added caching of
getSize at the TerminalImpl level to address this. The timeout is 1
second, which seems responsive enough for most use cases. We could also
move the calculation onto a background thread and have it periodically
updated, but that seems like overkill.
  • Loading branch information
eatkins committed Aug 10, 2020
1 parent 6dd69a5 commit 90dacc3
Show file tree
Hide file tree
Showing 27 changed files with 352 additions and 62 deletions.
7 changes: 4 additions & 3 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,7 @@ val completeProj = (project in file("internal") / "util-complete")
testedBaseSettings,
name := "Completion",
libraryDependencies += jline,
libraryDependencies += jline3,
libraryDependencies += jline3Reader,
mimaSettings,
// Parser is used publicly, so we can't break bincompat.
mimaBinaryIssueFilters := Seq(
Expand Down Expand Up @@ -366,7 +366,8 @@ lazy val utilLogging = (project in file("internal") / "util-logging")
libraryDependencies ++=
Seq(
jline,
jline3,
jline3Terminal,
jline3Jansi,
log4jApi,
log4jCore,
disruptor,
Expand Down Expand Up @@ -661,6 +662,7 @@ lazy val actionsProj = (project in file("main-actions"))
testedBaseSettings,
name := "Actions",
libraryDependencies += sjsonNewScalaJson.value,
libraryDependencies += jline3Terminal,
mimaSettings,
mimaBinaryIssueFilters ++= Seq(
// Removed unused private[sbt] nested class
Expand Down Expand Up @@ -1103,7 +1105,6 @@ lazy val sbtClientProj = (project in file("client"))
crossPaths := false,
exportJars := true,
libraryDependencies += jansi,
libraryDependencies += jline3Jansi,
libraryDependencies += scalatest % "test",
/*
* On windows, the raw classpath is too large to be a command argument to an
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/*
* sbt
* Copyright 2011 - 2018, Lightbend, Inc.
* Copyright 2008 - 2010, Mark Harrah
* Licensed under Apache License 2.0 (see LICENSE)
*/

package sbt.internal.util;

import org.jline.terminal.TerminalBuilder;

/**
* This exists to a provide a wrapper to TerminalBuilder.setTerminalOverride that will not emit a
* deprecation warning when called from scala.
*/
public class DeprecatedJLine {
@SuppressWarnings("deprecation")
public static void setTerminalOverride(final org.jline.terminal.Terminal terminal) {
TerminalBuilder.setTerminalOverride(terminal);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@

package sbt.internal.util

import java.io.{ EOFException, InputStream, OutputStream, PrintWriter }
import java.io.{ InputStream, OutputStream, PrintWriter }
import java.nio.charset.Charset
import java.util.{ Arrays, EnumSet }
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference }
Expand All @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._
import scala.util.Try
import java.util.concurrent.LinkedBlockingQueue

private[util] object JLine3 {
private[sbt] object JLine3 {
private val capabilityMap = Capability
.values()
.map { c =>
Expand Down Expand Up @@ -109,18 +109,18 @@ private[util] object JLine3 {
case _ => throw new ClosedException
}
if (res == 4 && term.prompt.render().endsWith(term.prompt.mkPrompt()))
throw new EOFException
throw new ClosedException
res
}
}
override val output: OutputStream = new OutputStream {
override def write(b: Int): Unit = write(Array[Byte](b.toByte))
override def write(b: Array[Byte]): Unit = if (!closed.get) term.withPrintStream { ps =>
ps.write(b)
term.prompt match {
case a: Prompt.AskUser => a.write(b)
case _ =>
}
ps.write(b)
}
override def write(b: Array[Byte], offset: Int, len: Int) =
write(Arrays.copyOfRange(b, offset, offset + len))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import java.util.concurrent.{ Executors, LinkedBlockingQueue, TimeUnit }
import jline.DefaultTerminal2
import jline.console.ConsoleReader
import scala.annotation.tailrec
import scala.concurrent.duration._
import scala.util.Try
import scala.util.control.NonFatal

Expand Down Expand Up @@ -174,10 +175,7 @@ object Terminal {
try Terminal.console.printStream.println(s"[info] $string")
catch { case _: IOException => }
}
private[sbt] def set(terminal: Terminal): Terminal = {
jline.TerminalFactory.set(terminal.toJLine)
activeTerminal.getAndSet(terminal)
}
private[sbt] def set(terminal: Terminal): Terminal = activeTerminal.getAndSet(terminal)
implicit class TerminalOps(private val term: Terminal) extends AnyVal {
def ansi(richString: => String, string: => String): String =
if (term.isAnsiSupported) richString else string
Expand Down Expand Up @@ -500,7 +498,6 @@ object Terminal {
* System.out through the terminal's input and output streams.
*/
private[this] val activeTerminal = new AtomicReference[Terminal](consoleTerminalHolder.get)
jline.TerminalFactory.set(consoleTerminalHolder.get.toJLine)

/**
* The boot input stream allows a remote client to forward input to the sbt process while
Expand Down Expand Up @@ -674,13 +671,13 @@ object Terminal {
if (alive)
try terminal.init()
catch {
case _: InterruptedException =>
case _: InterruptedException | _: java.io.IOError =>
}
override def restore(): Unit =
if (alive)
try terminal.restore()
catch {
case _: InterruptedException =>
case _: InterruptedException | _: java.io.IOError =>
}
override def reset(): Unit =
try terminal.reset()
Expand Down Expand Up @@ -767,10 +764,12 @@ object Terminal {
out: OutputStream
) extends TerminalImpl(in, out, originalErr, "console0") {
private[util] lazy val system = JLine3.system
private[this] def isCI = sys.env.contains("BUILD_NUMBER") || sys.env.contains("CI")
override def getWidth: Int = system.getSize.getColumns
override def getHeight: Int = system.getSize.getRows
override def isAnsiSupported: Boolean = term.isAnsiSupported && !isCI
override private[sbt] def getSizeImpl: (Int, Int) = {
val size = system.getSize
(size.getColumns, size.getRows)
}
private[this] val isCI = sys.env.contains("BUILD_NUMBER") || sys.env.contains("CI")
override lazy val isAnsiSupported: Boolean = term.isAnsiSupported && !isCI
override def isEchoEnabled: Boolean = system.echo()
override def isSuccessEnabled: Boolean = true
override def getBooleanCapability(capability: String, jline3: Boolean): Boolean =
Expand All @@ -785,7 +784,7 @@ object Terminal {
override private[sbt] def restore(): Unit = term.restore()

override private[sbt] def getAttributes: Map[String, String] =
JLine3.toMap(system.getAttributes)
Try(JLine3.toMap(system.getAttributes)).getOrElse(Map.empty)
override private[sbt] def setAttributes(attributes: Map[String, String]): Unit =
system.setAttributes(JLine3.attributesFromMap(attributes))
override private[sbt] def setSize(width: Int, height: Int): Unit =
Expand Down Expand Up @@ -825,6 +824,19 @@ object Terminal {
override val errorStream: OutputStream,
override private[sbt] val name: String
) extends Terminal {
private[sbt] def getSizeImpl: (Int, Int)
private[this] val sizeRefreshPeriod = 1.second
private[this] val size =
new AtomicReference[((Int, Int), Deadline)](((1, 1), Deadline.now - 1.day))
private[this] def setSize() = size.set((Try(getSizeImpl).getOrElse((1, 1)), Deadline.now))
private[this] def getSize = size.get match {
case (s, d) if (d + sizeRefreshPeriod).isOverdue =>
setSize()
size.get._1
case (s, _) => s
}
override def getWidth: Int = getSize._1
override def getHeight: Int = getSize._2
private[this] val rawMode = new AtomicBoolean(false)
private[this] val writeLock = new AnyRef
def throwIfClosed[R](f: => R): R = if (isStopped.get) throw new ClosedChannelException else f
Expand Down
4 changes: 3 additions & 1 deletion main-actions/src/main/scala/sbt/Console.scala
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ package sbt
import java.io.File
import java.nio.channels.ClosedChannelException
import sbt.internal.inc.{ AnalyzingCompiler, PlainVirtualFile }
import sbt.internal.util.Terminal
import sbt.internal.util.{ DeprecatedJLine, Terminal }
import sbt.util.Logger
import xsbti.compile.{ Compilers, Inputs }

Expand Down Expand Up @@ -67,6 +67,8 @@ final class Console(compiler: AnalyzingCompiler) {
try {
sys.props("scala.color") = if (terminal.isColorEnabled) "true" else "false"
terminal.withRawOutput {
jline.TerminalFactory.set(terminal.toJLine)
DeprecatedJLine.setTerminalOverride(sbt.internal.util.JLine3(terminal))
terminal.withRawInput(Run.executeTrapExit(console0, log))
}
} finally {
Expand Down
4 changes: 4 additions & 0 deletions main-command/src/main/scala/sbt/State.scala
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,10 @@ object State {
s get BasicKeys.classLoaderCache getOrElse (throw new IllegalStateException(
"Tried to get classloader cache for uninitialized state."
))
private[sbt] def extendedClassLoaderCache: ClassLoaderCache =
s get BasicKeys.extendedClassLoaderCache getOrElse (throw new IllegalStateException(
"Tried to get extended classloader cache for uninitialized state."
))
def initializeClassLoaderCache: State = {
s.get(BasicKeys.extendedClassLoaderCache).foreach(_.close())
val cache = newClassLoaderCache
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ import java.io.File
import java.lang.management.ManagementFactory
import java.lang.ref.{ Reference, ReferenceQueue, SoftReference }
import java.net.URLClassLoader
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.{ AtomicInteger, AtomicReference }

import sbt.internal.inc.classpath.{
AbstractClassLoaderCache,
Expand All @@ -30,9 +30,12 @@ private object ClassLoaderCache {
private def threadID = new AtomicInteger(0)
}
private[sbt] class ClassLoaderCache(
override val commonParent: ClassLoader,
val parent: ClassLoader,
private val miniProvider: Option[(File, ClassLoader)]
) extends AbstractClassLoaderCache {
private[this] val parentHolder = new AtomicReference(parent)
def commonParent = parentHolder.get()
def setParent(parent: ClassLoader): Unit = parentHolder.set(parent)
def this(commonParent: ClassLoader) = this(commonParent, None)
def this(scalaProvider: ScalaProvider) =
this(scalaProvider.launcher.topLoader, {
Expand All @@ -51,8 +54,9 @@ private[sbt] class ClassLoaderCache(
}
}
private class Key(val fileStamps: Seq[(File, Long)], val parent: ClassLoader) {
def this(files: List[File]) =
this(files.map(f => f -> IO.getModifiedTimeOrZero(f)), commonParent)
def this(files: List[File], parent: ClassLoader) =
this(files.map(f => f -> IO.getModifiedTimeOrZero(f)), parent)
def this(files: List[File]) = this(files, commonParent)
lazy val files: Seq[File] = fileStamps.map(_._1)
lazy val maxStamp: Long = fileStamps.maxBy(_._2)._2
class CachedClassLoader
Expand Down Expand Up @@ -169,10 +173,19 @@ private[sbt] class ClassLoaderCache(
val key = new Key(files, parent)
get(key, mkLoader)
}
override def apply(files: List[File]): ClassLoader = {
val key = new Key(files)
def apply(files: List[File], parent: ClassLoader): ClassLoader = {
val key = new Key(files, parent)
get(key, () => key.toClassLoader)
}
override def apply(files: List[File]): ClassLoader = {
files match {
case d :: s :: Nil if d.getName.startsWith("dotty-library") =>
apply(files, classOf[org.jline.terminal.Terminal].getClassLoader)
case _ =>
val key = new Key(files)
get(key, () => key.toClassLoader)
}
}
override def cachedCustomClassloader(
files: List[File],
mkLoader: () => ClassLoader
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,11 +47,12 @@ import Serialization.{
systemErrFlush,
terminalCapabilities,
terminalCapabilitiesResponse,
terminalGetSize,
terminalPropertiesQuery,
terminalPropertiesResponse,
terminalSetSize,
getTerminalAttributes,
setTerminalAttributes,
setTerminalSize,
}
import NetworkClient.Arguments

Expand Down Expand Up @@ -657,7 +658,13 @@ class NetworkClient(
cchars = attrs.getOrElse("cchars", ""),
)
sendCommandResponse("", response, msg.id)
case (`setTerminalSize`, Some(json)) =>
case (`terminalGetSize`, _) =>
val response = TerminalGetSizeResponse(
Terminal.console.getWidth,
Terminal.console.getHeight,
)
sendCommandResponse("", response, msg.id)
case (`terminalSetSize`, Some(json)) =>
Converter.fromJson[TerminalSetSizeCommand](json) match {
case Success(size) =>
Terminal.console.setSize(size.width, size.height)
Expand Down
6 changes: 4 additions & 2 deletions main-command/src/main/scala/sbt/internal/ui/UITask.scala
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@ private[sbt] object UITask {
override def close(): Unit = {}
}
object Reader {
// Avoid filling the stack trace since it isn't helpful here
object interrupted extends InterruptedException
def terminalReader(parser: Parser[_])(
terminal: Terminal,
state: State
Expand All @@ -59,9 +61,9 @@ private[sbt] object UITask {
val clear = terminal.ansi(ClearPromptLine, "")
@tailrec def impl(): Either[String, String] = {
val thread = Thread.currentThread
if (thread.isInterrupted || closed.get) throw new InterruptedException
if (thread.isInterrupted || closed.get) throw interrupted
val reader = LineReader.createReader(history(state), parser, terminal)
if (thread.isInterrupted || closed.get) throw new InterruptedException
if (thread.isInterrupted || closed.get) throw interrupted
(try reader.readLine(clear + terminal.prompt.mkPrompt())
finally reader.close) match {
case None if terminal == Terminal.console && System.console == null =>
Expand Down
34 changes: 34 additions & 0 deletions main/src/main/java/sbt/internal/JLineLoader.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* sbt
* Copyright 2011 - 2018, Lightbend, Inc.
* Copyright 2008 - 2010, Mark Harrah
* Licensed under Apache License 2.0 (see LICENSE)
*/

package sbt.internal;

import java.net.URL;
import java.net.URLClassLoader;

class JLineLoader extends URLClassLoader {
JLineLoader(final URL[] urls, final ClassLoader parent) {
super(urls, parent);
}

@Override
public String toString() {
final StringBuilder result = new StringBuilder();
result.append("JLineLoader(");
final URL[] urls = getURLs();
for (int i = 0; i < urls.length; ++i) {
result.append(urls[i].toString());
if (i < urls.length - 1) result.append(", ");
}
result.append(")");
return result.toString();
}

static {
registerAsParallelCapable();
}
}

0 comments on commit 90dacc3

Please sign in to comment.