diff --git a/.travis.yml b/.travis.yml index f85e14412a..7f384c57f6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,7 +4,6 @@ sudo: false scala: - "2.12.8" - - "2.11.12" - "2.13.0-M5" before_install: diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 0aabfa8800..eb58f9a840 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -206,7 +206,7 @@ validate binary compatibility of incoming pull requests. If your PR fails due to an error like this: ``` -[info] akka-stream: found 1 potential binary incompatibilities while checking against com.typesafe.akka:akka-stream_2.11:2.4.2 (filtered 222) +[info] akka-stream: found 1 potential binary incompatibilities while checking against com.typesafe.akka:akka-stream_2.12:2.4.2 (filtered 222) [error] * method foldAsync(java.lang.Object,scala.Function2)akka.stream.scaladsl.FlowOps in trait akka.stream.scaladsl.FlowOps is present only in current version [error] filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.foldAsync") ``` @@ -439,8 +439,7 @@ Scala has proven the most viable way to do it, as long as you keep the following 1. Have methods in the `javadsl` package delegate to the methods in the Scala API, or the common internal implementation. The Akka Stream Scala instances for example have a `.asJava` method to convert to the `akka.stream.javadsl` counterparts. -1. When using Scala `object` instances, offer a `getInstance()` method and add a sealed abstract class - (to support Scala 2.11) to get the return type. See `akka.Done` for an example. +1. When using Scala `object` instances, offer a `getInstance()` method. See `akka.Done` for an example. 1. When the Scala API contains an `apply` method, use `create` or `of` for Java users. @@ -450,9 +449,6 @@ Scala has proven the most viable way to do it, as long as you keep the following 1. Be careful to convert values within data structures (eg. for `scala.Long` vs. `java.lang.Long`, use `scala.Long.box(value)`) -1. When compiling with both Scala 2.11 and 2.12, some methods considered overloads in 2.11, become ambiguous in - 2.12 as both may be functional interfaces. - 1. Complement any methods with Scala collections with a Java collection version 1. Use the `akka.japi.Pair` class to return tuples @@ -474,8 +470,6 @@ Scala has proven the most viable way to do it, as long as you keep the following both Java and Scala APIs. Make such classes `private[akka]` and also, since that becomes `public` from Java's point of view, annotate with `@InternalApi` and add a scaladoc saying `INTERNAL API` -1. Companion objects (in Scala 2.11) cannot be accessed from Java if their companion is a trait, use an `abstract class` instead - 1. Traits that are part of the Java API should only be used to define pure interfaces, as soon as there are implementations of methods, prefer `abstract class`. diff --git a/RELEASING.md b/RELEASING.md index bc82ae094c..adeb7023a1 100644 --- a/RELEASING.md +++ b/RELEASING.md @@ -2,14 +2,14 @@ ## Prerequisites -### JDK 8 and JDK 9 +### JDK 8 and JDK 11 -Releasing Akka requires running on at least JDK 9, but also having JDK 8 +Releasing Akka requires running on JDK 11, but also having JDK 8 installed. The reason for this is that we want the Akka artifacts to be -usable with JRE 8, but also want to compile some classes with JDK9-specific +usable with JRE 8, but also want to compile some classes with JDK11-specific types. -When we stop supporting Scala 2.11 we might be able to update the build towork +In the future we might be able to update the build to work without having JDK 8 installed, by using the `-release` option. ### MinGW diff --git a/akka-actor-tests/src/test/scala/akka/util/ByteStringSpec.scala b/akka-actor-tests/src/test/scala/akka/util/ByteStringSpec.scala index 5d6145f69d..180d79b1ce 100644 --- a/akka-actor-tests/src/test/scala/akka/util/ByteStringSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/ByteStringSpec.scala @@ -896,8 +896,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers { "serialize correctly" when { "parsing regular ByteString1C as compat" in { val oldSerd = - if (util.Properties.versionNumberString.startsWith("2.11") || util.Properties.versionNumberString.startsWith( - "2.12")) + if (util.Properties.versionNumberString.startsWith("2.12")) "aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e67fa2925150f93468f0200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67" else // The data is the same, but the class hierarchy changed in 2.13: diff --git a/akka-actor-tests/src/test/scala/akka/util/LineNumberSpec.scala b/akka-actor-tests/src/test/scala/akka/util/LineNumberSpec.scala index de949e985a..deb6c0212d 100644 --- a/akka-actor-tests/src/test/scala/akka/util/LineNumberSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/LineNumberSpec.scala @@ -7,12 +7,8 @@ package akka.util import akka.testkit.AkkaSpec import akka.util.LineNumbers._ -import scala.util.Properties - class LineNumberSpec extends AkkaSpec { - private val isScala211 = Properties.versionNumberString.startsWith("2.11") - "LineNumbers" when { "writing Scala" must { @@ -24,11 +20,7 @@ class LineNumberSpec extends AkkaSpec { "work for larger functions" in { val result = LineNumbers(twoline) - if (isScala211) - result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17)) - else - // because how scala 2.12+ does the same as Java Lambdas - result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15)) + result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15)) } "work for partial functions" in { @@ -37,11 +29,7 @@ class LineNumberSpec extends AkkaSpec { "work for `def`" in { val result = LineNumbers(method("foo")) - if (isScala211) - result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27)) - else - // because how scala 2.12 does the same as Java Lambdas - result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27)) + result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27)) } } diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/BehaviorSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/BehaviorSpec.scala index a4d296744e..7e5d840c00 100644 --- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/BehaviorSpec.scala +++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/BehaviorSpec.scala @@ -175,22 +175,6 @@ object BehaviorSpec { /* * function converters for Java, to ease the pain on Scala 2.11 */ - def fs(f: (JActorContext[Command], Signal) => Behavior[Command]) = - new F2[JActorContext[Command], Signal, Behavior[Command]] { - override def apply(context: JActorContext[Command], sig: Signal) = f(context, sig) - } - def fc(f: (JActorContext[Command], Command) => Behavior[Command]) = - new F2[JActorContext[Command], Command, Behavior[Command]] { - override def apply(context: JActorContext[Command], command: Command) = f(context, command) - } - def ps(f: (JActorContext[Command], Signal) => Unit) = - new P2[JActorContext[Command], Signal] { - override def apply(context: JActorContext[Command], sig: Signal) = f(context, sig) - } - def pc(f: (JActorContext[Command], Command) => Unit) = - new P2[JActorContext[Command], Command] { - override def apply(context: JActorContext[Command], command: Command) = f(context, command) - } def pf(f: PFBuilder[Command, Command] => PFBuilder[Command, Command]) = new F1[PFBuilder[Command, Command], PFBuilder[Command, Command]] { override def apply(in: PFBuilder[Command, Command]) = f(in) @@ -199,10 +183,6 @@ object BehaviorSpec { new FI.Apply[Command, Command] { override def apply(in: Command) = f(in) } - def df(f: JActorContext[Command] => Behavior[Command]) = - new F1e[JActorContext[Command], Behavior[Command]] { - override def apply(in: JActorContext[Command]) = f(in) - } trait Lifecycle extends Common { "Lifecycle" must { @@ -547,7 +527,7 @@ class ImmutableWithSignalJavaBehaviorSpec extends Messages with BecomeWithLifecy override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) -> null def behv(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] = JBehaviors.receive( - fc((context, message) => + (context, message) => message match { case GetSelf => monitor ! Self(context.getSelf) @@ -569,40 +549,39 @@ class ImmutableWithSignalJavaBehaviorSpec extends Messages with BecomeWithLifecy SBehaviors.same case Stop => SBehaviors.stopped case _: AuxPing => SBehaviors.unhandled - }), - fs((_, sig) => { + }, + (_, sig) => { monitor ! ReceivedSignal(sig) SBehaviors.same - })) + }) } class ImmutableJavaBehaviorSpec extends Messages with Become with Stoppable { override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) -> null def behv(monitor: ActorRef[Event], state: State): Behavior[Command] = - JBehaviors.receive { - fc((context, message) => - message match { - case GetSelf => - monitor ! Self(context.getSelf) - SBehaviors.same - case Miss => - monitor ! Missed - SBehaviors.unhandled - case Ignore => - monitor ! Ignored - SBehaviors.same - case Ping => - monitor ! Pong - behv(monitor, state) - case Swap => - monitor ! Swapped - behv(monitor, state.next) - case GetState() => - monitor ! state - SBehaviors.same - case Stop => SBehaviors.stopped - case _: AuxPing => SBehaviors.unhandled - }) + JBehaviors.receive { (context, message) => + message match { + case GetSelf => + monitor ! Self(context.getSelf) + SBehaviors.same + case Miss => + monitor ! Missed + SBehaviors.unhandled + case Ignore => + monitor ! Ignored + SBehaviors.same + case Ping => + monitor ! Pong + behv(monitor, state) + case Swap => + monitor ! Swapped + behv(monitor, state.next) + case GetState() => + monitor ! state + SBehaviors.same + case Stop => SBehaviors.stopped + case _: AuxPing => SBehaviors.unhandled + } } } @@ -621,10 +600,10 @@ class DeferredJavaBehaviorSpec extends ImmutableWithSignalJavaBehaviorSpec { override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = { val inbox = TestInbox[Done]("deferredListener") - (JBehaviors.setup(df(_ => { + (JBehaviors.setup(_ => { inbox.ref ! Done super.behavior(monitor)._1 - })), inbox) + }), inbox) } override def checkAux(signal: Signal, aux: Aux): Unit = diff --git a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ActorLoggingSpec.scala b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ActorLoggingSpec.scala index 0b94c0bb8a..79d1b0b9e6 100644 --- a/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ActorLoggingSpec.scala +++ b/akka-actor-typed-tests/src/test/scala/akka/actor/typed/scaladsl/ActorLoggingSpec.scala @@ -43,8 +43,6 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit(""" akka.loggers = ["akka.testkit.TestEventListener"] """) with WordSpecLike { - private val isScala211 = Properties.versionNumberString.startsWith("2.11") - val marker = LogMarker("marker") val cause = new TestException("böö") @@ -77,11 +75,6 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit(""" "contain the class name where the first log was called" in { val eventFilter = EventFilter.custom({ case l: LogEvent if l.logClass == classOf[ActorLoggingSpec] => true - case l: LogEvent if isScala211 => - // TODO remove in Akka 2.6 when we drop Scala 2.11 - // the class with 2.11 is like - // ActorLoggingSpec$$anonfun$1$$anonfun$apply$mcV$sp$26$$anonfun$apply$6$$anonfun$apply$7 - l.logClass.getName.startsWith(classOf[ActorLoggingSpec].getName) case l: LogEvent => println(l.logClass) false @@ -102,11 +95,6 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit(""" "contain the object class name where the first log was called" in { val eventFilter = EventFilter.custom({ case l: LogEvent if l.logClass == WhereTheBehaviorIsDefined.getClass => true - case l: LogEvent if isScala211 => - // TODO remove in Akka 2.6 when we drop Scala 2.11 - // the class with 2.11 is like - // WhereTheBehaviorIsDefined$$anonfun$behavior$1 - l.logClass.getName.startsWith(WhereTheBehaviorIsDefined.getClass.getName) case l: LogEvent => println(l.logClass) false diff --git a/akka-actor/src/main/scala-2.11/akka/compat/Future.scala b/akka-actor/src/main/scala-2.11/akka/compat/Future.scala deleted file mode 100644 index 2b0f1c60a0..0000000000 --- a/akka-actor/src/main/scala-2.11/akka/compat/Future.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright (C) 2009-2019 Lightbend Inc. - */ - -package akka.compat - -import akka.annotation.InternalApi -import scala.concurrent.{ ExecutionContext, Future => SFuture } - -/** - * INTERNAL API - * - * Compatibility wrapper for `scala.concurrent.Future` to be able to compile the same code - * against Scala 2.11, 2.12, 2.13 - * - * Remove these classes as soon as support for Scala 2.11 is dropped! - */ -@InternalApi private[akka] object Future { - def fold[T, R](futures: TraversableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.fold[T, R](futures)(zero)(op)(executor) - - def reduce[T, R >: T](futures: TraversableOnce[SFuture[T]])(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.reduce[T, R](futures)(op)(executor) - - def find[T](futures: TraversableOnce[SFuture[T]])(p: T => Boolean)( - implicit executor: ExecutionContext): SFuture[Option[T]] = - SFuture.find[T](futures)(p)(executor) -} diff --git a/akka-actor/src/main/scala-2.11/akka/util/ByteIterator.scala b/akka-actor/src/main/scala-2.11/akka/util/ByteIterator.scala deleted file mode 100644 index e329baeb60..0000000000 --- a/akka-actor/src/main/scala-2.11/akka/util/ByteIterator.scala +++ /dev/null @@ -1,687 +0,0 @@ -/* - * Copyright (C) 2009-2019 Lightbend Inc. - */ - -package akka.util - -import akka.util.Collections.EmptyImmutableSeq - -import java.nio.{ ByteBuffer, ByteOrder } - -import scala.annotation.tailrec -import scala.collection.LinearSeq -import scala.collection.mutable.ListBuffer -import scala.reflect.ClassTag - -object ByteIterator { - object ByteArrayIterator { - - protected[akka] def apply(array: Array[Byte]): ByteArrayIterator = - new ByteArrayIterator(array, 0, array.length) - - protected[akka] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator = - new ByteArrayIterator(array, from, until) - - val empty: ByteArrayIterator = apply(Array.emptyByteArray) - } - - class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int) - extends ByteIterator { - iterator => - - @inline final def len: Int = until - from - - @inline final def hasNext: Boolean = from < until - - @inline final def head: Byte = array(from) - - final def next(): Byte = { - if (!hasNext) EmptyImmutableSeq.iterator.next() - else { - val i = from; from = from + 1; array(i) - } - } - - def clear(): Unit = { this.array = Array.emptyByteArray; from = 0; until = from } - - final override def length: Int = { val l = len; clear(); l } - - final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match { - case that: ByteIterator => - if (that.isEmpty) this - else if (this.isEmpty) that - else - that match { - case that: ByteArrayIterator => - if ((this.array eq that.array) && (this.until == that.from)) { - this.until = that.until - that.clear() - this - } else { - val result = MultiByteArrayIterator(List(this, that)) - this.clear() - result - } - case that: MultiByteArrayIterator => this ++: that - } - case _ => super.++(that) - } - - final override def clone: ByteArrayIterator = new ByteArrayIterator(array, from, until) - - final override def take(n: Int): this.type = { - if (n < len) until = { if (n > 0) (from + n) else from } - this - } - - final override def drop(n: Int): this.type = { - if (n > 0) from = { if (n < len) (from + n) else until } - this - } - - final override def takeWhile(p: Byte => Boolean): this.type = { - val prev = from - dropWhile(p) - until = from; from = prev - this - } - - final override def dropWhile(p: Byte => Boolean): this.type = { - var stop = false - while (!stop && hasNext) { - if (p(array(from))) { - from = from + 1 - } else { - stop = true - } - } - this - } - - final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = { - val n = 0 max ((xs.length - start) min this.len min len) - Array.copy(this.array, from, xs, start, n) - this.drop(n) - } - - final override def toByteString: ByteString = { - val result = - if ((from == 0) && (until == array.length)) ByteString.ByteString1C(array) - else ByteString.ByteString1(array, from, len) - clear() - result - } - - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = { - if (n <= this.len) { - Array.copy(this.array, this.from, xs, offset, n) - this.drop(n) - } else EmptyImmutableSeq.iterator.next() - } - - private def wrappedByteBuffer: ByteBuffer = ByteBuffer.wrap(array, from, len).asReadOnlyBuffer - - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asShortBuffer.get(xs, offset, n); drop(2 * n) - } - - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asIntBuffer.get(xs, offset, n); drop(4 * n) - } - - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asLongBuffer.get(xs, offset, n); drop(8 * n) - } - - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asFloatBuffer.get(xs, offset, n); drop(4 * n) - } - - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asDoubleBuffer.get(xs, offset, n); drop(8 * n) - } - - def copyToBuffer(buffer: ByteBuffer): Int = { - val copyLength = math.min(buffer.remaining, len) - if (copyLength > 0) { - buffer.put(array, from, copyLength) - drop(copyLength) - } - copyLength - } - - def asInputStream: java.io.InputStream = new java.io.InputStream { - override def available: Int = iterator.len - - def read: Int = if (hasNext) (next().toInt & 0xff) else -1 - - override def read(b: Array[Byte], off: Int, len: Int): Int = { - if ((off < 0) || (len < 0) || (off + len > b.length)) throw new IndexOutOfBoundsException - if (len == 0) 0 - else if (!isEmpty) { - val nRead = math.min(available, len) - copyToArray(b, off, nRead) - nRead - } else -1 - } - - override def skip(n: Long): Long = { - val nSkip = math.min(iterator.len, n.toInt) - iterator.drop(nSkip) - nSkip - } - } - } - - object MultiByteArrayIterator { - protected val clearedList: List[ByteArrayIterator] = List(ByteArrayIterator.empty) - - val empty: MultiByteArrayIterator = new MultiByteArrayIterator(Nil) - - protected[akka] def apply(iterators: LinearSeq[ByteArrayIterator]): MultiByteArrayIterator = - new MultiByteArrayIterator(iterators) - } - - class MultiByteArrayIterator private (private var iterators: LinearSeq[ByteArrayIterator]) extends ByteIterator { - // After normalization: - // * iterators.isEmpty == false - // * (!iterator.head.isEmpty || iterators.tail.isEmpty) == true - private def normalize(): this.type = { - @tailrec def norm(xs: LinearSeq[ByteArrayIterator]): LinearSeq[ByteArrayIterator] = { - if (xs.isEmpty) MultiByteArrayIterator.clearedList - else if (xs.head.isEmpty) norm(xs.tail) - else xs - } - iterators = norm(iterators) - this - } - normalize() - - @inline private def current: ByteArrayIterator = iterators.head - @inline private def dropCurrent(): Unit = { iterators = iterators.tail } - @inline def clear(): Unit = { iterators = MultiByteArrayIterator.empty.iterators } - - @inline final def hasNext: Boolean = current.hasNext - - @inline final def head: Byte = current.head - - final def next(): Byte = { - val result = current.next() - normalize() - result - } - - final override def len: Int = iterators.foldLeft(0) { _ + _.len } - - final override def length: Int = { - val result = len - clear() - result - } - - private[akka] def ++:(that: ByteArrayIterator): this.type = { - iterators = that +: iterators - this - } - - final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match { - case that: ByteIterator => - if (that.isEmpty) this - else if (this.isEmpty) that - else { - that match { - case that: ByteArrayIterator => - iterators = this.iterators :+ that - that.clear() - this - case that: MultiByteArrayIterator => - iterators = this.iterators ++ that.iterators - that.clear() - this - } - } - case _ => super.++(that) - } - - final override def clone: MultiByteArrayIterator = { - val clonedIterators: List[ByteArrayIterator] = iterators.map(_.clone)(collection.breakOut) - new MultiByteArrayIterator(clonedIterators) - } - - /** For performance sensitive code, call take() directly on ByteString (it's optimised there) */ - final override def take(n: Int): this.type = { - var rest = n - val builder = new ListBuffer[ByteArrayIterator] - while ((rest > 0) && !iterators.isEmpty) { - current.take(rest) - if (current.hasNext) { - rest -= current.len - builder += current - } - iterators = iterators.tail - } - iterators = builder.result - normalize() - } - - /** For performance sensitive code, call drop() directly on ByteString (it's optimised there) */ - final override def drop(n: Int): this.type = - if ((n > 0) && !isEmpty) { - val nCurrent = math.min(n, current.len) - current.drop(n) - val rest = n - nCurrent - assert(current.isEmpty || (rest == 0)) - normalize() - drop(rest) - } else this - - final override def takeWhile(p: Byte => Boolean): this.type = { - var stop = false - var builder = new ListBuffer[ByteArrayIterator] - while (!stop && !iterators.isEmpty) { - val lastLen = current.len - current.takeWhile(p) - if (current.hasNext) builder += current - if (current.len < lastLen) stop = true - dropCurrent() - } - iterators = builder.result - normalize() - } - - @tailrec final override def dropWhile(p: Byte => Boolean): this.type = - if (!isEmpty) { - current.dropWhile(p) - val dropMore = current.isEmpty - normalize() - if (dropMore) dropWhile(p) else this - } else this - - final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = { - var pos = start - var rest = len - while ((rest > 0) && !iterators.isEmpty) { - val n = 0 max ((xs.length - pos) min current.len min rest) - current.copyToArray(xs, pos, n) - pos += n - rest -= n - if (current.isEmpty) { - dropCurrent() - } - } - normalize() - } - - override def foreach[@specialized U](f: Byte => U): Unit = { - iterators.foreach { _.foreach(f) } - clear() - } - - final override def toByteString: ByteString = { - if (iterators.tail.isEmpty) iterators.head.toByteString - else { - val result = iterators.foldLeft(ByteString.empty) { _ ++ _.toByteString } - clear() - result - } - } - - @tailrec protected final def getToArray[A](xs: Array[A], offset: Int, n: Int, elemSize: Int)(getSingle: => A)( - getMult: (Array[A], Int, Int) => Unit): this.type = - if (n <= 0) this - else { - if (isEmpty) EmptyImmutableSeq.iterator.next() - val nDone = if (current.len >= elemSize) { - val nCurrent = math.min(n, current.len / elemSize) - getMult(xs, offset, nCurrent) - nCurrent - } else { - xs(offset) = getSingle - 1 - } - normalize() - getToArray(xs, offset + nDone, n - nDone, elemSize)(getSingle)(getMult) - } - - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = - getToArray(xs, offset, n, 1) { getByte } { current.getBytes(_, _, _) } - - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 2) { getShort(byteOrder) } { current.getShorts(_, _, _)(byteOrder) } - - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 4) { getInt(byteOrder) } { current.getInts(_, _, _)(byteOrder) } - - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getLong(byteOrder) } { current.getLongs(_, _, _)(byteOrder) } - - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getFloat(byteOrder) } { current.getFloats(_, _, _)(byteOrder) } - - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getDouble(byteOrder) } { current.getDoubles(_, _, _)(byteOrder) } - - /** For performance sensitive code, call copyToBuffer() directly on ByteString (it's optimised there) */ - override def copyToBuffer(buffer: ByteBuffer): Int = { - // the fold here is better than indexing into the LinearSeq - val n = iterators.foldLeft(0) { _ + _.copyToBuffer(buffer) } - normalize() - n - } - - def asInputStream: java.io.InputStream = new java.io.InputStream { - override def available: Int = current.len - - def read: Int = if (hasNext) (next().toInt & 0xff) else -1 - - override def read(b: Array[Byte], off: Int, len: Int): Int = { - val nRead = current.asInputStream.read(b, off, len) - normalize() - nRead - } - - override def skip(n: Long): Long = { - @tailrec def skipImpl(n: Long, skipped: Long): Long = - if (n > 0) { - if (!isEmpty) { - val m = current.asInputStream.skip(n) - normalize() - val newN = n - m - val newSkipped = skipped + m - if (newN > 0) skipImpl(newN, newSkipped) - else newSkipped - } else 0 - } else 0 - - skipImpl(n, 0) - } - } - } -} - -/** - * An iterator over a ByteString. - */ -abstract class ByteIterator extends BufferedIterator[Byte] { - def len: Int - - def head: Byte - - def next(): Byte - - protected def clear(): Unit - - def ++(that: TraversableOnce[Byte]): ByteIterator = - if (that.isEmpty) this else ByteIterator.ByteArrayIterator(that.toArray) - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def clone: ByteIterator = - throw new UnsupportedOperationException("Method clone is not implemented in ByteIterator") - - override def duplicate: (ByteIterator, ByteIterator) = (this, clone) - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def take(n: Int): this.type = - throw new UnsupportedOperationException("Method take is not implemented in ByteIterator") - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def drop(n: Int): this.type = - throw new UnsupportedOperationException("Method drop is not implemented in ByteIterator") - - override def slice(from: Int, until: Int): this.type = { - if (from > 0) drop(from).take(until - from) - else take(until) - } - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def takeWhile(p: Byte => Boolean): this.type = - throw new UnsupportedOperationException("Method takeWhile is not implemented in ByteIterator") - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def dropWhile(p: Byte => Boolean): this.type = - throw new UnsupportedOperationException("Method dropWhile is not implemented in ByteIterator") - - override def span(p: Byte => Boolean): (ByteIterator, ByteIterator) = { - val that = clone - this.takeWhile(p) - that.drop(this.len) - (this, that) - } - - override def indexWhere(p: Byte => Boolean): Int = indexWhere(p, 0) - def indexWhere(p: Byte => Boolean, from: Int): Int = { - var index = 0 - while (index < from) { - next() - index += 1 - } - var found = false - while (!found && hasNext) if (p(next())) { - found = true - } else { - index += 1 - } - if (found) index else -1 - } - - def indexOf(elem: Byte): Int = indexOf(elem, 0) - def indexOf(elem: Byte, from: Int): Int = indexWhere(_ == elem, from) - - override def indexOf[B >: Byte](elem: B): Int = indexOf[B](elem, 0) - def indexOf[B >: Byte](elem: B, from: Int): Int = indexWhere(_ == elem, from) - - def toByteString: ByteString - - override def toSeq: ByteString = toByteString - - override def foreach[@specialized U](f: Byte => U): Unit = - while (hasNext) f(next()) - - override def foldLeft[@specialized B](z: B)(op: (B, Byte) => B): B = { - var acc = z - foreach { byte => - acc = op(acc, byte) - } - acc - } - - override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = { - val target = new Array[B](len) - copyToArray(target) - target - } - - /** - * Get a single Byte from this iterator. Identical to next(). - */ - def getByte: Byte = next() - - /** - * Get a single Short from this iterator. - */ - def getShort(implicit byteOrder: ByteOrder): Short = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next() & 0xff) << 8 | (next() & 0xff) << 0).toShort - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next() & 0xff) << 0 | (next() & 0xff) << 8).toShort - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a single Int from this iterator. - */ - def getInt(implicit byteOrder: ByteOrder): Int = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next() & 0xff) << 24 - | (next() & 0xff) << 16 - | (next() & 0xff) << 8 - | (next() & 0xff) << 0) - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next() & 0xff) << 0 - | (next() & 0xff) << 8 - | (next() & 0xff) << 16 - | (next() & 0xff) << 24) - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a single Long from this iterator. - */ - def getLong(implicit byteOrder: ByteOrder): Long = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next().toLong & 0xff) << 56 - | (next().toLong & 0xff) << 48 - | (next().toLong & 0xff) << 40 - | (next().toLong & 0xff) << 32 - | (next().toLong & 0xff) << 24 - | (next().toLong & 0xff) << 16 - | (next().toLong & 0xff) << 8 - | (next().toLong & 0xff) << 0) - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next().toLong & 0xff) << 0 - | (next().toLong & 0xff) << 8 - | (next().toLong & 0xff) << 16 - | (next().toLong & 0xff) << 24 - | (next().toLong & 0xff) << 32 - | (next().toLong & 0xff) << 40 - | (next().toLong & 0xff) << 48 - | (next().toLong & 0xff) << 56) - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a Long from this iterator where only the least significant `n` - * bytes were encoded. - */ - def getLongPart(n: Int)(implicit byteOrder: ByteOrder): Long = { - if (byteOrder == ByteOrder.BIG_ENDIAN) { - var x = 0L - (1 to n).foreach(_ => x = (x << 8) | (next() & 0xff)) - x - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - var x = 0L - (0 until n).foreach(i => x |= (next() & 0xff) << 8 * i) - x - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - def getFloat(implicit byteOrder: ByteOrder): Float = - java.lang.Float.intBitsToFloat(getInt(byteOrder)) - - def getDouble(implicit byteOrder: ByteOrder): Double = - java.lang.Double.longBitsToDouble(getLong(byteOrder)) - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < xs.length. - */ - def getBytes(xs: Array[Byte]): this.type = getBytes(xs, 0, xs.length) - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if length < n or if (xs.length - offset) < n. - */ - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < n. - */ - def getBytes(n: Int): Array[Byte] = { - val bytes = new Array[Byte](n) - getBytes(bytes, 0, n) - bytes - } - - /** - * Get a ByteString with specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < n. - */ - def getByteString(n: Int): ByteString = { - val bs = clone.take(n).toByteString - drop(n) - bs - } - - /** - * Get a number of Shorts from this iterator. - */ - def getShorts(xs: Array[Short])(implicit byteOrder: ByteOrder): this.type = - getShorts(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Shorts from this iterator. - */ - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Ints from this iterator. - */ - def getInts(xs: Array[Int])(implicit byteOrder: ByteOrder): this.type = - getInts(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Ints from this iterator. - */ - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Longs from this iterator. - */ - def getLongs(xs: Array[Long])(implicit byteOrder: ByteOrder): this.type = - getLongs(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Longs from this iterator. - */ - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Floats from this iterator. - */ - def getFloats(xs: Array[Float])(implicit byteOrder: ByteOrder): this.type = - getFloats(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Floats from this iterator. - */ - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Doubles from this iterator. - */ - def getDoubles(xs: Array[Double])(implicit byteOrder: ByteOrder): this.type = - getDoubles(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Doubles from this iterator. - */ - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Copy as many bytes as possible to a ByteBuffer, starting from it's - * current position. This method will not overflow the buffer. - * - * @param buffer a ByteBuffer to copy bytes to - * @return the number of bytes actually copied - */ - /** For performance sensitive code, call take() directly on ByteString (it's optimised there) */ - def copyToBuffer(buffer: ByteBuffer): Int - - /** - * Directly wraps this ByteIterator in an InputStream without copying. - * Read and skip operations on the stream will advance the iterator - * accordingly. - */ - def asInputStream: java.io.InputStream -} diff --git a/akka-actor/src/main/scala-2.12/akka/compat/Future.scala b/akka-actor/src/main/scala-2.12/akka/compat/Future.scala index 6ce86517ab..d9d0d2fdd9 100644 --- a/akka-actor/src/main/scala-2.12/akka/compat/Future.scala +++ b/akka-actor/src/main/scala-2.12/akka/compat/Future.scala @@ -15,9 +15,9 @@ import com.github.ghik.silencer.silent * INTERNAL API * * Compatibility wrapper for `scala.concurrent.Future` to be able to compile the same code - * against Scala 2.11, 2.12, 2.13 + * against Scala 2.12, 2.13 * - * Remove these classes as soon as support for Scala 2.11 is dropped! + * Remove these classes as soon as support for Scala 2.12 is dropped! */ @silent @InternalApi private[akka] object Future { def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( diff --git a/akka-actor/src/main/scala-2.13/akka/compat/Future.scala b/akka-actor/src/main/scala-2.13/akka/compat/Future.scala index e600f25174..8144c5a674 100644 --- a/akka-actor/src/main/scala-2.13/akka/compat/Future.scala +++ b/akka-actor/src/main/scala-2.13/akka/compat/Future.scala @@ -12,9 +12,9 @@ import scala.collection.immutable * INTERNAL API * * Compatibility wrapper for `scala.concurrent.Future` to be able to compile the same code - * against Scala 2.11, 2.12, 2.13 + * against Scala 2.12, 2.13 * - * Remove these classes as soon as support for Scala 2.11 is dropped! + * Remove these classes as soon as support for Scala 2.12 is dropped! */ @InternalApi private[akka] object Future { def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( diff --git a/akka-docs/src/main/paradox/java8-compat.md b/akka-docs/src/main/paradox/java8-compat.md index ffd66c4b6f..40d692178f 100644 --- a/akka-docs/src/main/paradox/java8-compat.md +++ b/akka-docs/src/main/paradox/java8-compat.md @@ -14,7 +14,7 @@ and Java counterparts. The artifact can be included in Maven builds using: ``` org.scala-lang.modules - scala-java8-compat_2.11 + scala-java8-compat_2.12 0.7.0 ``` diff --git a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md new file mode 100644 index 0000000000..63b93392f6 --- /dev/null +++ b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md @@ -0,0 +1,5 @@ +# Migration Guide 2.5.x to 2.6.x + +## Scala 2.11 no longer supported + +If you are still using Scala 2.11 then you must upgrade to 2.12 or 2.13 diff --git a/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala index 0a16406621..86a7aa7320 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/DaemonMsgCreateSerializer.scala @@ -33,8 +33,6 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys import ProtobufSerializer.deserializeActorRef import Deploy.NoDispatcherGiven - private val scala212OrLater = !scala.util.Properties.versionNumberString.startsWith("2.11") - private lazy val serialization = SerializationExtension(system) override val includeManifest: Boolean = false @@ -207,8 +205,7 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys "null" } else { val className = m.getClass.getName - if (scala212OrLater && m.isInstanceOf[java.io.Serializable] && m.getClass.isSynthetic && className.contains( - "$Lambda$")) { + if (m.isInstanceOf[java.io.Serializable] && m.getClass.isSynthetic && className.contains("$Lambda$")) { // When the additional-protobuf serializers are not enabled // the serialization of the parameters is based on passing class name instead of // serializerId and manifest as we usually do. With Scala 2.12 the functions are generated as diff --git a/build.sbt b/build.sbt index a07fa4eadb..abfd69abe7 100644 --- a/build.sbt +++ b/build.sbt @@ -1,6 +1,11 @@ -import akka.{AutomaticModuleName, CopyrightHeaderForBuild, ParadoxSupport, ScalafixIgnoreFilePlugin} +import akka.{ AutomaticModuleName, CopyrightHeaderForBuild, ParadoxSupport, ScalafixIgnoreFilePlugin } -enablePlugins(UnidocRoot, TimeStampede, UnidocWithPrValidation, NoPublish, CopyrightHeader, +enablePlugins( + UnidocRoot, + TimeStampede, + UnidocWithPrValidation, + NoPublish, + CopyrightHeader, CopyrightHeaderInPr, ScalafixIgnoreFilePlugin, JavaFormatterPlugin) @@ -10,10 +15,10 @@ addCommandAlias( value = ";scalafixEnable;compile:scalafix;test:scalafix;multi-jvm:scalafix;test:compile;reload") import akka.AkkaBuild._ -import akka.{AkkaBuild, Dependencies, GitHub, OSGi, Protobuf, SigarLoader, VersionGenerator} +import akka.{ AkkaBuild, Dependencies, GitHub, OSGi, Protobuf, SigarLoader, VersionGenerator } import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.MultiJvm import com.typesafe.tools.mima.plugin.MimaPlugin -import sbt.Keys.{initialCommands, parallelExecution} +import sbt.Keys.{ initialCommands, parallelExecution } import spray.boilerplate.BoilerplatePlugin initialize := { @@ -23,66 +28,66 @@ initialize := { } akka.AkkaBuild.buildSettings -shellPrompt := { s => Project.extract(s).currentProject.id + " > " } +shellPrompt := { s => + Project.extract(s).currentProject.id + " > " +} resolverSettings -def isScala211: Boolean = System.getProperty("akka.build.scalaVersion", "").startsWith("2.11") - // When this is updated the set of modules in ActorSystem.allModules should also be updated lazy val aggregatedProjects: Seq[ProjectReference] = List[ProjectReference]( - actor, actorTests, + actor, + actorTests, + actorTestkitTyped, + actorTyped, + actorTypedTests, agent, benchJmh, + benchJmhTyped, camel, - cluster, clusterMetrics, clusterSharding, clusterTools, + cluster, + clusterMetrics, + clusterSharding, + clusterShardingTyped, + clusterTools, + clusterTyped, contrib, + coordination, + discovery, distributedData, + docs, multiNodeTestkit, osgi, - persistence, persistenceQuery, persistenceShared, persistenceTck, + persistence, + persistenceQuery, + persistenceShared, + persistenceTck, + persistenceTyped, protobuf, - remote, remoteTests, + remote, + remoteTests, slf4j, - stream, streamTestkit, streamTests, streamTestsTck, - testkit, - discovery, - coordination -) ++ - (if (isScala211) List.empty[ProjectReference] - else List[ProjectReference]( - docs, - actorTyped, actorTypedTests, actorTestkitTyped, - persistenceTyped, - clusterTyped, clusterShardingTyped, - benchJmhTyped, - streamTyped - )) + stream, + streamTestkit, + streamTests, + streamTestsTck, + streamTyped, + testkit) -lazy val root = Project( - id = "akka", - base = file(".") -).aggregate(aggregatedProjects: _*) - .settings(rootSettings: _*) - .settings(unidocRootIgnoreProjects := - (CrossVersion.partialVersion(scalaVersion.value) match { - case Some((2, n)) if n == 11 => aggregatedProjects // ignore all, don't unidoc when scalaVersion is 2.11 - case _ => Seq(remoteTests, benchJmh, benchJmhTyped, protobuf, akkaScalaNightly, docs) - }) - ) - .settings( - unmanagedSources in(Compile, headerCreate) := (baseDirectory.value / "project").**("*.scala").get - ).enablePlugins(CopyrightHeaderForBuild) +lazy val root = Project(id = "akka", base = file(".")) + .aggregate(aggregatedProjects: _*) + .settings(rootSettings: _*) + .settings(unidocRootIgnoreProjects := Seq(remoteTests, benchJmh, benchJmhTyped, protobuf, akkaScalaNightly, docs)) + .settings(unmanagedSources in (Compile, headerCreate) := (baseDirectory.value / "project").**("*.scala").get) + .enablePlugins(CopyrightHeaderForBuild) lazy val actor = akkaModule("akka-actor") .settings(Dependencies.actor) .settings(OSGi.actor) .settings(AutomaticModuleName.settings("akka.actor")) - .settings( - unmanagedSourceDirectories in Compile += { - val ver = scalaVersion.value.take(4) - (scalaSource in Compile).value.getParentFile / s"scala-$ver" - } - ) + .settings(unmanagedSourceDirectories in Compile += { + val ver = scalaVersion.value.take(4) + (scalaSource in Compile).value.getParentFile / s"scala-$ver" + }) .settings(VersionGenerator.settings) .enablePlugins(BoilerplatePlugin) @@ -100,38 +105,26 @@ lazy val agent = akkaModule("akka-agent") .enablePlugins(ScaladocNoVerificationOfDiagrams) lazy val akkaScalaNightly = akkaModule("akka-scala-nightly") - // remove dependencies that we have to build ourselves (Scala STM) - .aggregate(aggregatedProjects diff List[ProjectReference](agent, docs): _*) +// remove dependencies that we have to build ourselves (Scala STM) + .aggregate(aggregatedProjects.diff(List[ProjectReference](agent, docs)): _*) .disablePlugins(MimaPlugin) .disablePlugins(ValidatePullRequest, MimaPlugin, CopyrightHeaderInPr) lazy val benchJmh = akkaModule("akka-bench-jmh") - .dependsOn( - Seq( - actor, - stream, streamTests, - persistence, distributedData, - testkit - ).map(_ % "compile->compile;compile->test"): _* - ) + .dependsOn(Seq(actor, stream, streamTests, persistence, distributedData, testkit).map( + _ % "compile->compile;compile->test"): _*) .settings(Dependencies.benchJmh) .enablePlugins(JmhPlugin, ScaladocNoVerificationOfDiagrams, NoPublish, CopyrightHeader) .disablePlugins(MimaPlugin, WhiteSourcePlugin, ValidatePullRequest, CopyrightHeaderInPr) // typed benchmarks only on 2.12+ lazy val benchJmhTyped = akkaModule("akka-bench-jmh-typed") - .dependsOn( - Seq( - persistenceTyped, - distributedData, clusterTyped, - testkit, benchJmh - ).map(_ % "compile->compile;compile->test"): _* - ) + .dependsOn(Seq(persistenceTyped, distributedData, clusterTyped, testkit, benchJmh).map( + _ % "compile->compile;compile->test"): _*) .settings(Dependencies.benchJmh) .enablePlugins(JmhPlugin, ScaladocNoVerificationOfDiagrams, NoPublish, CopyrightHeader) .disablePlugins(MimaPlugin, WhiteSourcePlugin, ValidatePullRequest, CopyrightHeaderInPr) - lazy val camel = akkaModule("akka-camel") .dependsOn(actor, slf4j, testkit % "test->test") .settings(Dependencies.camel) @@ -144,13 +137,10 @@ lazy val cluster = akkaModule("akka-cluster") .settings(AutomaticModuleName.settings("akka.cluster")) .settings(OSGi.cluster) .settings(Protobuf.settings) - .settings( - parallelExecution in Test := false - ) + .settings(parallelExecution in Test := false) .configs(MultiJvm) .enablePlugins(MultiNodeScalaTest) - lazy val clusterMetrics = akkaModule("akka-cluster-metrics") .dependsOn(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", slf4j % "test->compile") .settings(OSGi.clusterMetrics) @@ -158,23 +148,20 @@ lazy val clusterMetrics = akkaModule("akka-cluster-metrics") .settings(AutomaticModuleName.settings("akka.cluster.metrics")) .settings(Protobuf.settings) .settings(SigarLoader.sigarSettings) - .settings( - parallelExecution in Test := false - ) + .settings(parallelExecution in Test := false) .configs(MultiJvm) .enablePlugins(MultiNodeScalaTest) lazy val clusterSharding = akkaModule("akka-cluster-sharding") - // TODO akka-persistence dependency should be provided in pom.xml artifact. - // If I only use "provided" here it works, but then we can't run tests. - // Scope "test" is alright in the pom.xml, but would have been nicer with - // provided. +// TODO akka-persistence dependency should be provided in pom.xml artifact. +// If I only use "provided" here it works, but then we can't run tests. +// Scope "test" is alright in the pom.xml, but would have been nicer with +// provided. .dependsOn( cluster % "compile->compile;test->test;multi-jvm->multi-jvm", distributedData, persistence % "compile->compile", - clusterTools % "compile->compile;test->test" - ) + clusterTools % "compile->compile;test->test") .settings(Dependencies.clusterSharding) .settings(AutomaticModuleName.settings("akka.cluster.sharding")) .settings(OSGi.clusterSharding) @@ -183,10 +170,7 @@ lazy val clusterSharding = akkaModule("akka-cluster-sharding") .enablePlugins(MultiNode, ScaladocNoVerificationOfDiagrams) lazy val clusterTools = akkaModule("akka-cluster-tools") - .dependsOn( - cluster % "compile->compile;test->test;multi-jvm->multi-jvm", - coordination - ) + .dependsOn(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", coordination) .settings(Dependencies.clusterTools) .settings(AutomaticModuleName.settings("akka.cluster.tools")) .settings(OSGi.clusterTools) @@ -199,9 +183,8 @@ lazy val contrib = akkaModule("akka-contrib") .settings(Dependencies.contrib) .settings(AutomaticModuleName.settings("akka.contrib")) .settings(OSGi.contrib) - .settings( - description := - """| + .settings(description := + """| |This subproject provides a home to modules contributed by external |developers which may or may not move into the officially supported code |base over time. A module in this subproject doesn't have to obey the rule @@ -210,8 +193,7 @@ lazy val contrib = akkaModule("akka-contrib") |simplify based on your feedback. A module may be dropped in any release |without prior deprecation. The Lightbend subscription does not cover |support for these modules. - |""".stripMargin - ) + |""".stripMargin) .configs(MultiJvm) .enablePlugins(MultiNode, ScaladocNoVerificationOfDiagrams) .disablePlugins(MimaPlugin) @@ -227,7 +209,17 @@ lazy val distributedData = akkaModule("akka-distributed-data") lazy val docs = akkaModule("akka-docs") .dependsOn( - actor, cluster, clusterMetrics, slf4j, agent, osgi, persistenceTck, persistenceQuery, distributedData, stream, actorTyped, + actor, + cluster, + clusterMetrics, + slf4j, + agent, + osgi, + persistenceTck, + persistenceQuery, + distributedData, + stream, + actorTyped, camel % "compile->compile;test->test", clusterTools % "compile->compile;test->test", clusterSharding % "compile->compile;test->test", @@ -239,42 +231,43 @@ lazy val docs = akkaModule("akka-docs") clusterTyped % "compile->compile;test->test", clusterShardingTyped % "compile->compile;test->test", actorTypedTests % "compile->compile;test->test", - streamTestkit % "compile->compile;test->test" - ) + streamTestkit % "compile->compile;test->test") .settings(Dependencies.docs) .settings( - name in(Compile, paradox) := "Akka", + name in (Compile, paradox) := "Akka", Compile / paradoxProperties ++= Map( - "canonical.base_url" -> "https://doc.akka.io/docs/akka/current", - "github.base_url" -> GitHub.url(version.value), // for links like this: @github[#1](#1) or @github[83986f9](83986f9) - "extref.akka.http.base_url" -> "https://doc.akka.io/docs/akka-http/current/%s", - "extref.wikipedia.base_url" -> "https://en.wikipedia.org/wiki/%s", - "extref.github.base_url" -> (GitHub.url(version.value) + "/%s"), // for links to our sources - "extref.samples.base_url" -> "https://developer.lightbend.com/start/?group=akka&project=%s", - "extref.ecs.base_url" -> "https://example.lightbend.com/v1/download/%s", - "scaladoc.akka.base_url" -> "https://doc.akka.io/api/akka/2.5", - "scaladoc.akka.http.base_url" -> "https://doc.akka.io/api/akka-http/current", - "javadoc.akka.base_url" -> "https://doc.akka.io/japi/akka/2.5", - "javadoc.akka.http.base_url" -> "https://doc.akka.io/japi/akka-http/current", - "scala.version" -> scalaVersion.value, - "scala.binary_version" -> scalaBinaryVersion.value, - "akka.version" -> version.value, - "sigar_loader.version" -> "1.6.6-rev002", - "algolia.docsearch.api_key" -> "543bad5ad786495d9ccd445ed34ed082", - "algolia.docsearch.index_name" -> "akka_io", - "google.analytics.account" -> "UA-21117439-1", - "google.analytics.domain.name" -> "akka.io", - "signature.akka.base_dir" -> (baseDirectory in ThisBuild).value.getAbsolutePath, - "fiddle.code.base_dir" -> (sourceDirectory in Test).value.getAbsolutePath, - "fiddle.akka.base_dir" -> (baseDirectory in ThisBuild).value.getAbsolutePath, - ), + "canonical.base_url" -> "https://doc.akka.io/docs/akka/current", + "github.base_url" -> GitHub + .url(version.value), // for links like this: @github[#1](#1) or @github[83986f9](83986f9) + "extref.akka.http.base_url" -> "https://doc.akka.io/docs/akka-http/current/%s", + "extref.wikipedia.base_url" -> "https://en.wikipedia.org/wiki/%s", + "extref.github.base_url" -> (GitHub.url(version.value) + "/%s"), // for links to our sources + "extref.samples.base_url" -> "https://developer.lightbend.com/start/?group=akka&project=%s", + "extref.ecs.base_url" -> "https://example.lightbend.com/v1/download/%s", + "scaladoc.akka.base_url" -> "https://doc.akka.io/api/akka/2.5", + "scaladoc.akka.http.base_url" -> "https://doc.akka.io/api/akka-http/current", + "javadoc.akka.base_url" -> "https://doc.akka.io/japi/akka/2.5", + "javadoc.akka.http.base_url" -> "https://doc.akka.io/japi/akka-http/current", + "scala.version" -> scalaVersion.value, + "scala.binary_version" -> scalaBinaryVersion.value, + "akka.version" -> version.value, + "sigar_loader.version" -> "1.6.6-rev002", + "algolia.docsearch.api_key" -> "543bad5ad786495d9ccd445ed34ed082", + "algolia.docsearch.index_name" -> "akka_io", + "google.analytics.account" -> "UA-21117439-1", + "google.analytics.domain.name" -> "akka.io", + "signature.akka.base_dir" -> (baseDirectory in ThisBuild).value.getAbsolutePath, + "fiddle.code.base_dir" -> (sourceDirectory in Test).value.getAbsolutePath, + "fiddle.akka.base_dir" -> (baseDirectory in ThisBuild).value.getAbsolutePath), Compile / paradoxGroups := Map("Language" -> Seq("Scala", "Java")), resolvers += Resolver.jcenterRepo, apidocRootPackage := "akka", - deployRsyncArtifact := List((paradox in Compile).value -> s"www/docs/akka/${version.value}") - ) + deployRsyncArtifact := List((paradox in Compile).value -> s"www/docs/akka/${version.value}")) .enablePlugins( - AkkaParadoxPlugin, DeployRsync, NoPublish, ParadoxBrowse, + AkkaParadoxPlugin, + DeployRsync, + NoPublish, + ParadoxBrowse, ScaladocNoVerificationOfDiagrams, StreamOperatorsIndexGenerator) .settings(ParadoxSupport.paradoxWithCustomDirectives) @@ -292,9 +285,7 @@ lazy val osgi = akkaModule("akka-osgi") .settings(Dependencies.osgi) .settings(AutomaticModuleName.settings("akka.osgi")) .settings(OSGi.osgi) - .settings( - parallelExecution in Test := false - ) + .settings(parallelExecution in Test := false) lazy val persistence = akkaModule("akka-persistence") .dependsOn(actor, testkit % "test->test", protobuf) @@ -302,31 +293,21 @@ lazy val persistence = akkaModule("akka-persistence") .settings(AutomaticModuleName.settings("akka.persistence")) .settings(OSGi.persistence) .settings(Protobuf.settings) - .settings( - fork in Test := true - ) + .settings(fork in Test := true) lazy val persistenceQuery = akkaModule("akka-persistence-query") - .dependsOn( - stream, - persistence % "compile->compile;test->test", - streamTestkit % "test" - ) + .dependsOn(stream, persistence % "compile->compile;test->test", streamTestkit % "test") .settings(Dependencies.persistenceQuery) .settings(AutomaticModuleName.settings("akka.persistence.query")) .settings(OSGi.persistenceQuery) - .settings( - fork in Test := true - ) + .settings(fork in Test := true) .enablePlugins(ScaladocNoVerificationOfDiagrams) lazy val persistenceShared = akkaModule("akka-persistence-shared") .dependsOn(persistence % "test->test", testkit % "test->test", remote % "test", protobuf) .settings(Dependencies.persistenceShared) .settings(AutomaticModuleName.settings("akka.persistence.shared")) - .settings( - fork in Test := true - ) + .settings(fork in Test := true) .enablePlugins(NoPublish) .disablePlugins(MimaPlugin, WhiteSourcePlugin) @@ -335,9 +316,7 @@ lazy val persistenceTck = akkaModule("akka-persistence-tck") .settings(Dependencies.persistenceTck) .settings(AutomaticModuleName.settings("akka.persistence.tck")) //.settings(OSGi.persistenceTck) TODO: we do need to export this as OSGi bundle too? - .settings( - fork in Test := true - ) + .settings(fork in Test := true) .disablePlugins(MimaPlugin) lazy val protobuf = akkaModule("akka-protobuf") @@ -352,17 +331,13 @@ lazy val remote = akkaModule("akka-remote") .settings(AutomaticModuleName.settings("akka.remote")) .settings(OSGi.remote) .settings(Protobuf.settings) - .settings( - parallelExecution in Test := false - ) + .settings(parallelExecution in Test := false) lazy val remoteTests = akkaModule("akka-remote-tests") .dependsOn(actorTests % "test->test", remote % "test->test", streamTestkit % "test", multiNodeTestkit) .settings(Dependencies.remoteTests) .settings(Protobuf.settings) - .settings( - parallelExecution in Test := false - ) + .settings(parallelExecution in Test := false) .configs(MultiJvm) .enablePlugins(MultiNodeScalaTest, NoPublish) .disablePlugins(MimaPlugin, WhiteSourcePlugin) @@ -402,8 +377,7 @@ lazy val streamTestsTck = akkaModule("akka-stream-tests-tck") // is causing long GC pauses when running with G1 on // the CI build servers. Therefore we fork these tests // to run with small heap without G1. - fork in Test := true - ) + fork in Test := true) .enablePlugins(NoPublish) .disablePlugins(MimaPlugin, WhiteSourcePlugin) @@ -412,26 +386,22 @@ lazy val testkit = akkaModule("akka-testkit") .settings(Dependencies.testkit) .settings(AutomaticModuleName.settings("akka.actor.testkit")) .settings(OSGi.testkit) - .settings( - initialCommands += "import akka.testkit._" - ) + .settings(initialCommands += "import akka.testkit._") lazy val actorTyped = akkaModule("akka-actor-typed") .dependsOn(actor) .settings(AkkaBuild.mayChangeSettings) .settings(AutomaticModuleName.settings("akka.actor.typed")) // fine for now, eventually new module name to become typed.actor .settings(OSGi.actorTyped) - .settings( - initialCommands := - """ + .settings(initialCommands := + """ import akka.actor.typed._ import akka.actor.typed.scaladsl.Behaviors import scala.concurrent._ import scala.concurrent.duration._ import akka.util.Timeout implicit val timeout = Timeout(5.seconds) - """ - ) + """) .disablePlugins(MimaPlugin) lazy val persistenceTyped = akkaModule("akka-persistence-typed") @@ -440,8 +410,7 @@ lazy val persistenceTyped = akkaModule("akka-persistence-typed") persistence % "compile->compile;test->test", persistenceQuery % "test", actorTypedTests % "test->test", - actorTestkitTyped % "compile->compile;test->test" - ) + actorTestkitTyped % "compile->compile;test->test") .settings(Dependencies.persistenceShared) .settings(AkkaBuild.mayChangeSettings) .settings(AutomaticModuleName.settings("akka.persistence.typed")) @@ -459,8 +428,7 @@ lazy val clusterTyped = akkaModule("akka-cluster-typed") protobuf, actorTestkitTyped % "test->test", actorTypedTests % "test->test", - remoteTests % "test->test" - ) + remoteTests % "test->test") .settings(AkkaBuild.mayChangeSettings) .settings(AutomaticModuleName.settings("akka.cluster.typed")) .disablePlugins(MimaPlugin) @@ -475,8 +443,7 @@ lazy val clusterShardingTyped = akkaModule("akka-cluster-sharding-typed") actorTestkitTyped % "test->test", actorTypedTests % "test->test", persistenceTyped % "test->test", - remoteTests % "test->test" - ) + remoteTests % "test->test") .settings(AkkaBuild.mayChangeSettings) .settings(AutomaticModuleName.settings("akka.cluster.sharding.typed")) // To be able to import ContainerFormats.proto @@ -491,8 +458,7 @@ lazy val streamTyped = akkaModule("akka-stream-typed") stream, streamTestkit % "test->test", actorTestkitTyped % "test->test", - actorTypedTests % "test->test" - ) + actorTypedTests % "test->test") .settings(AkkaBuild.mayChangeSettings) .settings(AutomaticModuleName.settings("akka.stream.typed")) .disablePlugins(MimaPlugin) @@ -505,38 +471,25 @@ lazy val actorTestkitTyped = akkaModule("akka-actor-testkit-typed") .disablePlugins(MimaPlugin) lazy val actorTypedTests = akkaModule("akka-actor-typed-tests") - .dependsOn( - actorTyped, - actorTestkitTyped % "compile->compile;test->test" - ) + .dependsOn(actorTyped, actorTestkitTyped % "compile->compile;test->test") .settings(AkkaBuild.mayChangeSettings) .disablePlugins(MimaPlugin) .enablePlugins(NoPublish) lazy val discovery = akkaModule("akka-discovery") - .dependsOn( - actor, - testkit % "test->test", - actorTests % "test->test" - ) + .dependsOn(actor, testkit % "test->test", actorTests % "test->test") .settings(Dependencies.discovery) .settings(AutomaticModuleName.settings("akka.discovery")) .settings(OSGi.discovery) lazy val coordination = akkaModule("akka-coordination") - .dependsOn( - actor, - testkit % "test->test", - actorTests % "test->test", - cluster % "test->test" - ) + .dependsOn(actor, testkit % "test->test", actorTests % "test->test", cluster % "test->test") .settings(Dependencies.coordination) .settings(AutomaticModuleName.settings("akka.coordination")) .settings(OSGi.coordination) .settings(AkkaBuild.mayChangeSettings) .disablePlugins(MimaPlugin) - def akkaModule(name: String): Project = Project(id = name, base = file(name)) .enablePlugins(ReproducibleBuildsPlugin) @@ -560,20 +513,20 @@ addCommandAlias("allClusterCore", commandValue(cluster)) addCommandAlias("allClusterMetrics", commandValue(clusterMetrics)) addCommandAlias("allClusterSharding", commandValue(clusterSharding)) addCommandAlias("allClusterTools", commandValue(clusterTools)) -addCommandAlias("allCluster", Seq( - commandValue(cluster), - commandValue(distributedData), - commandValue(clusterSharding), - commandValue(clusterTools)).mkString) +addCommandAlias( + "allCluster", + Seq(commandValue(cluster), commandValue(distributedData), commandValue(clusterSharding), commandValue(clusterTools)).mkString) addCommandAlias("allCoordination", commandValue(coordination)) addCommandAlias("allDistributedData", commandValue(distributedData)) addCommandAlias("allPersistence", commandValue(persistence)) addCommandAlias("allStream", commandValue(stream, Some(streamTests))) addCommandAlias("allDiscovery", commandValue(discovery)) -addCommandAlias("allTyped", Seq( - commandValue(actorTyped, Some(actorTypedTests)), - commandValue(actorTestkitTyped), - commandValue(clusterTyped), - commandValue(clusterShardingTyped), - commandValue(persistenceTyped), - commandValue(streamTyped)).mkString) +addCommandAlias( + "allTyped", + Seq( + commandValue(actorTyped, Some(actorTypedTests)), + commandValue(actorTestkitTyped), + commandValue(clusterTyped), + commandValue(clusterShardingTyped), + commandValue(persistenceTyped), + commandValue(streamTyped)).mkString) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 26cb0de5d6..3f62a9adec 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -101,11 +101,8 @@ object AkkaBuild { if (JavaVersion.isJdk8) Seq("-target:jvm-1.8") else - if (scalaBinaryVersion.value == "2.11") - Seq("-target:jvm-1.8", "-javabootclasspath", CrossJava.Keys.fullJavaHomes.value("8") + "/jre/lib/rt.jar") - else - // -release 8 is not enough, for some reason we need the 8 rt.jar explicitly #25330 - Seq("-release", "8", "-javabootclasspath", CrossJava.Keys.fullJavaHomes.value("8") + "/jre/lib/rt.jar")), + // -release 8 is not enough, for some reason we need the 8 rt.jar explicitly #25330 + Seq("-release", "8", "-javabootclasspath", CrossJava.Keys.fullJavaHomes.value("8") + "/jre/lib/rt.jar")), scalacOptions in Compile ++= (if (allWarnings) Seq("-deprecation") else Nil), scalacOptions in Test := (scalacOptions in Test).value.filterNot(opt => opt == "-Xlog-reflective-calls" || opt.contains("genjavadoc")), diff --git a/project/AkkaDisciplinePlugin.scala b/project/AkkaDisciplinePlugin.scala index 3b59df9a31..03ed15f686 100644 --- a/project/AkkaDisciplinePlugin.scala +++ b/project/AkkaDisciplinePlugin.scala @@ -37,10 +37,7 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport { lazy val scalaFixSettings = Seq(Compile / scalacOptions += "-Yrangepos") lazy val scoverageSettings = - Seq(coverageMinimum := 70, coverageFailOnMinimum := false, coverageOutputHTML := true, coverageHighlighting := { - import sbt.librarymanagement.{ SemanticSelector, VersionNumber } - !VersionNumber(scalaVersion.value).matchesSemVer(SemanticSelector("<=2.11.1")) - }) + Seq(coverageMinimum := 70, coverageFailOnMinimum := false, coverageOutputHTML := true, coverageHighlighting := true) lazy val silencerSettings = { val silencerVersion = "1.3.1" @@ -55,7 +52,7 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport { silencerSettings ++ scoverageSettings ++ Seq( Compile / scalacOptions ++= ( - if (!scalaVersion.value.startsWith("2.11") && !nonFatalWarningsFor(name.value)) Seq("-Xfatal-warnings") + if (!nonFatalWarningsFor(name.value)) Seq("-Xfatal-warnings") else Seq.empty ), Test / scalacOptions --= testUndicipline, @@ -74,12 +71,6 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport { case _ => Nil }).toSeq, - Compile / doc / scalacOptions ++= (CrossVersion.partialVersion(scalaVersion.value) match { - case Some((2, 11)) => - Seq("-no-link-warnings") - case _ => - Seq.empty - }), Compile / scalacOptions --= (if (strictProjects.contains(name.value)) Seq.empty else undisciplineScalacOptions.toSeq), diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 80768b22d5..ace0150425 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -24,18 +24,12 @@ object Dependencies { crossScalaVersions := Seq("2.12.8", "2.13.0-M5"), scalaVersion := System.getProperty("akka.build.scalaVersion", crossScalaVersions.value.head), scalaStmVersion := sys.props.get("akka.build.scalaStmVersion").getOrElse("0.9"), - scalaCheckVersion := sys.props - .get("akka.build.scalaCheckVersion") - .getOrElse(CrossVersion.partialVersion(scalaVersion.value) match { - case Some((2, n)) if n >= 12 => "1.14.0" // does not work for 2.11 - case _ => "1.13.2" - }), + scalaCheckVersion := sys.props.get("akka.build.scalaCheckVersion").getOrElse("1.14.0"), scalaTestVersion := "3.0.7", java8CompatVersion := { CrossVersion.partialVersion(scalaVersion.value) match { case Some((2, n)) if n >= 13 => "0.9.0" - case Some((2, n)) if n == 12 => "0.8.0" - case _ => "0.7.0" + case _ => "0.8.0" } }) diff --git a/project/Jdk9.scala b/project/Jdk9.scala index d73078a203..1f84f779de 100644 --- a/project/Jdk9.scala +++ b/project/Jdk9.scala @@ -12,11 +12,6 @@ object Jdk9 extends AutoPlugin { lazy val CompileJdk9 = config("CompileJdk9").extend(Compile) - def notOnScala211[T](scalaBinaryVersion: String, values: Seq[T]): Seq[T] = scalaBinaryVersion match { - case "2.11" => Seq() - case _ => values - } - val SCALA_SOURCE_DIRECTORY = "scala-jdk-9" val SCALA_TEST_SOURCE_DIRECTORY = "scala-jdk9-only" val JAVA_SOURCE_DIRECTORY = "java-jdk-9" @@ -25,15 +20,11 @@ object Jdk9 extends AutoPlugin { val compileJdk9Settings = Seq( // following the scala-2.12, scala-sbt-1.0, ... convention unmanagedSourceDirectories := notOnJdk8( - notOnScala211( - scalaBinaryVersion.value, - Seq( - (Compile / sourceDirectory).value / SCALA_SOURCE_DIRECTORY, - (Compile / sourceDirectory).value / JAVA_SOURCE_DIRECTORY))), - scalacOptions := AkkaBuild.DefaultScalacOptions ++ notOnJdk8( - notOnScala211(scalaBinaryVersion.value, Seq("-release", "11"))), - javacOptions := AkkaBuild.DefaultJavacOptions ++ notOnJdk8( - notOnScala211(scalaBinaryVersion.value, Seq("--release", "11")))) + Seq( + (Compile / sourceDirectory).value / SCALA_SOURCE_DIRECTORY, + (Compile / sourceDirectory).value / JAVA_SOURCE_DIRECTORY)), + scalacOptions := AkkaBuild.DefaultScalacOptions ++ notOnJdk8(Seq("-release", "11")), + javacOptions := AkkaBuild.DefaultJavacOptions ++ notOnJdk8(Seq("--release", "11"))) val compileSettings = Seq( // It might have been more 'neat' to add the jdk9 products to the jar via packageBin/mappings, but that doesn't work with the OSGi plugin, diff --git a/project/MiMa.scala b/project/MiMa.scala index 40c84188eb..71fd3aec40 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -39,15 +39,6 @@ object MiMa extends AutoPlugin { val akka2519NewArtifacts = Seq("akka-discovery") scalaBinaryVersion match { - case "2.11" => - if (akka2519NewArtifacts.contains(projectName)) - akka25DiscoveryVersions - else if (akka250NewArtifacts.contains(projectName)) akka25Versions - else { - if (!akka242NewArtifacts.contains(projectName)) akka24NoStreamVersions - else Seq.empty - } ++ akka24StreamVersions ++ akka24WithScala212 ++ akka25Versions - case "2.12" => if (akka2519NewArtifacts.contains(projectName)) akka25DiscoveryVersions diff --git a/project/scripts/release b/project/scripts/release index 4bc272b5c3..75899abfe8 100755 --- a/project/scripts/release +++ b/project/scripts/release @@ -299,7 +299,7 @@ else RELEASE_OPT="-Dakka.genjavadoc.enabled=true" fi try sbt $RELEASE_OPT +buildRelease -try sbt -Dakka.build.scalaVersion=2.11.12 -D$RELEASE_OPT buildRelease +try sbt -Dakka.build.scalaVersion=2.12.8 -D$RELEASE_OPT buildRelease try sbt $RELEASE_OPT buildDocs echolog "Successfully created local release" diff --git a/scripts/authors.scala b/scripts/authors.scala index 5e79d27561..149cbc41de 100755 --- a/scripts/authors.scala +++ b/scripts/authors.scala @@ -9,7 +9,7 @@ exec scala "$0" "$@" * or if on non unixy os: * scala authors.scala tag1 tag2 * - * requires scala 2.11.x+ and command line git on path + * requires scala 2.12.x+ and command line git on path */ import scala.sys.process._