Skip to content

Commit

Permalink
Cross build fs2.io for Scala.js (#2453)
Browse files Browse the repository at this point in the history
* Extract JVM-specific io implementation

* Add Node.js types dependency

* Move resources to JVM directory

* Fix build

* Implement FileHandle for JS

* Fix scala 3 compile errors

* Formatting

* Disable fail-fast CI

* Regenerate workflow

* Trigger CI

* Implement Socket for Node.js

* Formatting

* Fix 2.12 compile

* Renaming

* More socket implementing

* Renaming

* Implement UDP for Node

* Debugging, tcp now passing tests :)

* Formatting

* Improve ScalablyTyped build

* Uncomment forgotten line

* UdpSuite passing on Node.js

* Formatting

* Bump ST plugin. Scala 3 support!

* Revert "Disable fail-fast CI"

This reverts commit baa0f52.

* Regenerate workflow

* Fix scala 3 compile

* Refactoring

* TLS on Node.js

* Delete duplicated resource

* Fix scala 3 compile

* Update yarn.lock

* Add JS methods to Chunk[Byte]

* Make ST facade internal, new public APIs

* Formatting

* Polish SecureContext API

* Polish TLSParameters API

* Fix Scala 2.12 compile

* Fixup TLS suite

* Refactor UnixSockets

* Fix scala 3 compile

* Annotate/simplify MiMa exclusions
  • Loading branch information
armanbilge authored Jul 11, 2021
1 parent b119f82 commit 670be35
Show file tree
Hide file tree
Showing 101 changed files with 7,104 additions and 565 deletions.
2 changes: 1 addition & 1 deletion .github/workflows/ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ jobs:
run: sbt --client '++${{ matrix.scala }}; microsite/mdoc'

- name: Compress target directories
run: tar cf targets.tar target core/js/target core/jvm/target io/target reactive-streams/target benchmark/target project/target
run: tar cf targets.tar target core/js/target core/jvm/target io/js/target reactive-streams/target io/jvm/target benchmark/target project/target

- name: Upload target directories
uses: actions/upload-artifact@v2
Expand Down
59 changes: 46 additions & 13 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,8 @@ addCommandAlias(
"fmtCheck",
"; compile:scalafmtCheck; test:scalafmtCheck; it:scalafmtCheck; scalafmtSbtCheck"
)
addCommandAlias("testJVM", ";coreJVM/test;io/test;reactiveStreams/test;benchmark/test")
addCommandAlias("testJS", "coreJS/test")
addCommandAlias("testJVM", ";rootJVM/test")
addCommandAlias("testJS", "rootJS/test")

Global / onChangedBuildSource := ReloadOnSourceChanges

Expand Down Expand Up @@ -104,13 +104,36 @@ ThisBuild / mimaBinaryIssueFilters ++= Seq(
ProblemFilters.exclude[ReversedMissingMethodProblem](
"fs2.io.net.tls.TLSContext.dtlsServerBuilder"
),
ProblemFilters.exclude[Problem]("fs2.io.net.tls.TLSEngine*")
ProblemFilters.exclude[Problem]("fs2.io.net.tls.TLSEngine*"),
// start #2453 cross-build fs2.io for scala.js
// private implementation classes
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.Socket$IntCallbackHandler"),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.Socket$BufferedReads"),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.SocketGroup$AsyncSocketGroup"),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.Socket$AsyncSocket"),
ProblemFilters.exclude[MissingClassProblem](
"fs2.io.net.DatagramSocketGroup$AsyncDatagramSocketGroup"
),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.unixsocket.UnixSockets$AsyncSocket"),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.unixsocket.UnixSockets$AsyncUnixSockets"),
ProblemFilters.exclude[MissingClassProblem]("fs2.io.net.tls.TLSContext$Builder$AsyncBuilder"),
// sealed traits
ProblemFilters.exclude[NewMixinForwarderProblem]("fs2.io.net.Network.*"),
ProblemFilters.exclude[NewMixinForwarderProblem]("fs2.io.net.tls.TLSContext.*"),
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("fs2.io.net.tls.TLSContext.*")
// end #2453
)

lazy val root = project
.in(file("."))
.enablePlugins(NoPublishPlugin, SonatypeCiReleasePlugin)
.aggregate(coreJVM, coreJS, io, reactiveStreams, benchmark)
.aggregate(coreJVM, coreJS, io.jvm, io.js, reactiveStreams, benchmark)

lazy val rootJVM = project
.in(file("."))
.enablePlugins(NoPublishPlugin)
.aggregate(coreJVM, io.jvm, reactiveStreams, benchmark)
lazy val rootJS = project.in(file(".")).enablePlugins(NoPublishPlugin).aggregate(coreJS, io.js)

lazy val IntegrationTest = config("it").extend(Test)

Expand Down Expand Up @@ -177,16 +200,13 @@ lazy val coreJS = core.js
scalaJSLinkerConfig ~= (_.withModuleKind(ModuleKind.CommonJSModule))
)

lazy val io = project
lazy val io = crossProject(JVMPlatform, JSPlatform)
.in(file("io"))
.enablePlugins(SbtOsgi)
.jsConfigure(_.enablePlugins(ScalablyTypedConverterGenSourcePlugin))
.settings(
name := "fs2-io",
libraryDependencies ++= Seq(
"com.comcast" %% "ip4s-core" % "3.0.3",
"com.github.jnr" % "jnr-unixsocket" % "0.38.8" % Optional
),
Test / fork := true,
libraryDependencies += "com.comcast" %%% "ip4s-core" % "3.0.3",
OsgiKeys.exportPackage := Seq("fs2.io.*"),
OsgiKeys.privatePackage := Seq(),
OsgiKeys.importPackage := {
Expand All @@ -200,7 +220,20 @@ lazy val io = project
OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"),
osgiSettings
)
.dependsOn(coreJVM % "compile->compile;test->test")
.jvmSettings(
Test / fork := true,
libraryDependencies += "com.github.jnr" % "jnr-unixsocket" % "0.38.8" % Optional
)
.jsSettings(
scalaJSLinkerConfig ~= (_.withModuleKind(ModuleKind.CommonJSModule)),
Compile / npmDependencies += "@types/node" -> "16.0.0",
Test / npmDependencies += "jks-js" -> "1.0.1",
useYarn := true,
stOutputPackage := "fs2.internal.jsdeps",
stStdlib := List("es2020"),
stIgnore += "jks-js"
)
.dependsOn(core % "compile->compile;test->test")

lazy val reactiveStreams = project
.in(file("reactive-streams"))
Expand Down Expand Up @@ -236,7 +269,7 @@ lazy val benchmark = project
Test / run / javaOptions := (Test / run / javaOptions).value
.filterNot(o => o.startsWith("-Xmx") || o.startsWith("-Xms")) ++ Seq("-Xms256m", "-Xmx256m")
)
.dependsOn(io)
.dependsOn(io.jvm)

lazy val microsite = project
.in(file("mdoc"))
Expand All @@ -252,7 +285,7 @@ lazy val microsite = project
githubWorkflowArtifactUpload := false,
fatalWarningsInCI := false
)
.dependsOn(coreJVM, io, reactiveStreams)
.dependsOn(coreJVM, io.jvm, reactiveStreams)
.enablePlugins(MdocPlugin, NoPublishPlugin)

ThisBuild / githubWorkflowBuildPostamble ++= List(
Expand Down
57 changes: 57 additions & 0 deletions core/js/src/main/scala/fs2/ChunkRuntimePlatform.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* Copyright (c) 2013 Functional Streams for Scala
*
* Permission is hereby granted, free of charge, to any person obtaining a copy of
* this software and associated documentation files (the "Software"), to deal in
* the Software without restriction, including without limitation the rights to
* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
* the Software, and to permit persons to whom the Software is furnished to do so,
* subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
* FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
* IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
*/

package fs2

import scala.scalajs.js.typedarray.ArrayBuffer
import scala.scalajs.js.typedarray.TypedArrayBuffer
import scala.scalajs.js.typedarray.Uint8Array
import scala.scalajs.js.typedarray.TypedArrayBufferOps._

trait ChunkRuntimePlatform[+O] { self: Chunk[O] =>

def toJSArrayBuffer[B >: O](implicit ev: B =:= Byte): ArrayBuffer = {
val bb = toByteBuffer[B]
if (bb.hasArrayBuffer())
bb.arrayBuffer()
else {
val ab = new ArrayBuffer(bb.remaining())
TypedArrayBuffer.wrap(ab).put(bb)
ab
}
}

def toUint8Array[B >: O](implicit ev: B =:= Byte): Uint8Array = {
val ab = toJSArrayBuffer[B]
new Uint8Array(ab, 0, ab.byteLength)
}

}

trait ChunkCompanionRuntimePlatform { self: Chunk.type =>

def jsArrayBuffer(buffer: ArrayBuffer): Chunk[Byte] =
byteBuffer(TypedArrayBuffer.wrap(buffer))

def uint8Array(array: Uint8Array): Chunk[Byte] =
jsArrayBuffer(array.buffer)

}
58 changes: 58 additions & 0 deletions core/js/src/test/scala/fs2/ChunkRuntimePlatformSuite.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Copyright (c) 2013 Functional Streams for Scala
*
* Permission is hereby granted, free of charge, to any person obtaining a copy of
* this software and associated documentation files (the "Software"), to deal in
* the Software without restriction, including without limitation the rights to
* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
* the Software, and to permit persons to whom the Software is furnished to do so,
* subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
* FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
* IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
*/

package fs2

import org.scalacheck.Arbitrary
import org.scalacheck.Gen
import org.scalacheck.Prop.forAll

class ChunkRuntimePlatformSuite extends Fs2Suite {
override def scalaCheckTestParameters =
super.scalaCheckTestParameters
.withMinSuccessfulTests(if (isJVM) 100 else 25)
.withWorkers(1)

// Override to remove from implicit scope
override val byteChunkArbitrary = Arbitrary(???)

def testByteChunk(
genChunk: Gen[Chunk[Byte]],
name: String
): Unit =
group(s"$name") {
implicit val implicitChunkArb: Arbitrary[Chunk[Byte]] = Arbitrary(genChunk)
property("JSArrayBuffer conversion is idempotent") {
forAll { (c: Chunk[Byte]) =>
assertEquals(Chunk.jsArrayBuffer(c.toJSArrayBuffer), c)
}
}
property("Uint8Array conversion is idempotent") {
forAll { (c: Chunk[Byte]) =>
assertEquals(Chunk.uint8Array(c.toUint8Array), c)
}
}
}

testByteChunk(byteBufferChunkGenerator, "ByteBuffer")
testByteChunk(byteVectorChunkGenerator, "ByteVector")

}
26 changes: 26 additions & 0 deletions core/jvm/src/main/scala/fs2/ChunkRuntimePlatform.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright (c) 2013 Functional Streams for Scala
*
* Permission is hereby granted, free of charge, to any person obtaining a copy of
* this software and associated documentation files (the "Software"), to deal in
* the Software without restriction, including without limitation the rights to
* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
* the Software, and to permit persons to whom the Software is furnished to do so,
* subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
* FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
* IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
*/

package fs2

trait ChunkRuntimePlatform[+O]

trait ChunkCompanionRuntimePlatform
8 changes: 6 additions & 2 deletions core/shared/src/main/scala/fs2/Chunk.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,8 @@ import cats.syntax.all._
* access to the underlying backing array, along with an offset and length, referring to a slice
* of that array.
*/
abstract class Chunk[+O] extends Serializable with ChunkPlatform[O] { self =>
abstract class Chunk[+O] extends Serializable with ChunkPlatform[O] with ChunkRuntimePlatform[O] {
self =>

/** Returns the number of elements in this chunk. */
def size: Int
Expand Down Expand Up @@ -543,7 +544,10 @@ abstract class Chunk[+O] extends Serializable with ChunkPlatform[O] { self =>
iterator.mkString("Chunk(", ", ", ")")
}

object Chunk extends CollectorK[Chunk] with ChunkCompanionPlatform {
object Chunk
extends CollectorK[Chunk]
with ChunkCompanionPlatform
with ChunkCompanionRuntimePlatform {

private val empty_ : Chunk[Nothing] = new EmptyChunk
private final class EmptyChunk extends Chunk[Nothing] {
Expand Down
63 changes: 63 additions & 0 deletions io/js/src/main/scala/fs2/io/file/FileHandlePlatform.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Copyright (c) 2013 Functional Streams for Scala
*
* Permission is hereby granted, free of charge, to any person obtaining a copy of
* this software and associated documentation files (the "Software"), to deal in
* the Software without restriction, including without limitation the rights to
* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
* the Software, and to permit persons to whom the Software is furnished to do so,
* subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
* FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
* IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
*/

package fs2
package io
package file

import fs2.internal.jsdeps.node.fsPromisesMod
import cats.effect.kernel.Async
import cats.syntax.all._
import scala.scalajs.js.typedarray.ArrayBuffer
import scala.scalajs.js.typedarray.TypedArrayBuffer
import scala.scalajs.js.typedarray.TypedArrayBufferOps._

private[file] trait FileHandlePlatform[F[_]]

private[file] trait FileHandleCompanionPlatform {
private[file] def make[F[_]](
fd: fsPromisesMod.FileHandle
)(implicit F: Async[F]): FileHandle[F] =
new FileHandle[F] {

override def force(metaData: Boolean): F[Unit] =
F.fromPromise(F.delay(fsPromisesMod.fdatasync(fd)))

override def read(numBytes: Int, offset: Long): F[Option[Chunk[Byte]]] =
F.fromPromise(F.delay(fd.read(new ArrayBuffer(numBytes), offset.toDouble, numBytes))).map {
res =>
if (res.bytesRead < 0) None
else if (res.bytesRead == 0) Some(Chunk.empty)
else
Some(Chunk.byteBuffer(TypedArrayBuffer.wrap(res.buffer).limit(res.bytesRead.toInt)))
}

override def size: F[Long] =
F.fromPromise(F.delay(fd.stat())).map(_.size.toLong)

override def truncate(size: Long): F[Unit] =
F.fromPromise(F.delay(fd.truncate(size.toDouble)))

override def write(bytes: Chunk[Byte], offset: Long): F[Int] =
F.fromPromise(F.delay(fd.write(bytes.toByteBuffer.arrayBuffer(), offset.toDouble)))
.map(_.bytesWritten.toInt)
}
}
45 changes: 45 additions & 0 deletions io/js/src/main/scala/fs2/io/file/Path.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Copyright (c) 2013 Functional Streams for Scala
*
* Permission is hereby granted, free of charge, to any person obtaining a copy of
* this software and associated documentation files (the "Software"), to deal in
* the Software without restriction, including without limitation the rights to
* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
* the Software, and to permit persons to whom the Software is furnished to do so,
* subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in all
* copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
* FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
* IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
* CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
*/

package fs2
package io.file

import fs2.internal.jsdeps.node.fsMod.PathLike
import fs2.io.internal.ByteChunkOps._

sealed abstract class Path {
private[file] def toPathLike: PathLike
}

object Path {

def apply(path: String): Path = StringPath(path)
def apply(path: Chunk[Byte]): Path = BufferPath(path)

private final case class StringPath(path: String) extends Path {
override private[file] def toPathLike: PathLike = path.asInstanceOf[PathLike]
}

private final case class BufferPath(path: Chunk[Byte]) extends Path {
override private[file] def toPathLike: PathLike = path.toBuffer.asInstanceOf[PathLike]
}

}
Loading

0 comments on commit 670be35

Please sign in to comment.