From 786abd1857339dcf834d83e065f8f5f80559119e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 30 May 2023 18:45:21 +0200 Subject: [PATCH 001/200] Add unsafe to jvm --- .../fs2/io/uring/unsafe/UringSystem.scala | 38 +++++++++++++++++++ .../main/scala/fs2/io/uring/unsafe/util.scala | 23 +++++++++++ 2 files changed, 61 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala new file mode 100644 index 00000000..97f822ca --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -0,0 +1,38 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring +package unsafe + +import cats.effect.unsafe.PollingSystem + +object UringSystem extends PollingSystem { + + override def makeApi(register: (Poller => Unit) => Unit): Api = ??? + + override def makePoller(): Poller = ??? + + override def closePoller(poller: Poller): Unit = ??? + + override def poll(poller: Poller, nanos: Long, reportFailure: Throwable => Unit): Boolean = ??? + + override def needsPoll(poller: Poller): Boolean = ??? + + override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = ??? + + + final class Poller private[UringSystem] () {} +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala new file mode 100644 index 00000000..d2801c39 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -0,0 +1,23 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring.unsafe + + +private[uring] object util { + + +} From 16e6305b66a227e042d97d30797e87967b50b739 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 30 May 2023 18:46:09 +0200 Subject: [PATCH 002/200] Add uring files from native --- .../fs2/io/uring/IOExceptionHelper.scala | 35 ++++++++++++++++ .../src/main/scala/fs2/io/uring/Uring.scala | 41 +++++++++++++++++++ .../main/scala/fs2/io/uring/UringApp.scala | 29 +++++++++++++ .../main/scala/fs2/io/uring/implicits.scala | 28 +++++++++++++ .../test/scala/fs2/io/uring/UringSuite.scala | 27 ++++++++++++ 5 files changed, 160 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/Uring.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/implicits.scala create mode 100644 uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala b/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala new file mode 100644 index 00000000..34f9f124 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala @@ -0,0 +1,35 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import java.io.IOException +import java.net.ConnectException +import java.net.BindException + +private[uring] object IOExceptionHelper { + + def apply(errno: Int): IOException = errno match { + case 98 => // EADDRINUSE + new BindException("Address already in use") + case 99 => // EADDRNOTAVAIL + new BindException("Cannot assign requested address") + case 111 => // ECONNREFUSED + new ConnectException("Connection refused") + case _ => new IOException(errno.toString) + } + +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala new file mode 100644 index 00000000..e30d9de1 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -0,0 +1,41 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import cats.effect.IO +import cats.effect.LiftIO +import cats.effect.kernel.Resource +import cats.syntax.all._ + + +abstract class Uring private[uring] { + + def call(): IO[Int] + + def bracket()(release: Int => IO[Unit]): Resource[IO, Int] +} + +object Uring { + + def get[F[_]: LiftIO]: F[Uring] = + IO.pollers.flatMap { + _.collectFirst { case ring: Uring => + ring + }.liftTo[IO](new RuntimeException("No UringSystem installed")) + }.to + +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala b/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala new file mode 100644 index 00000000..a4047dd3 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import cats.effect.IOApp + +trait UringApp extends IOApp { + + override protected final def pollingSystem = ??? + +} + +object UringApp { + trait Simple extends IOApp.Simple with UringApp +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/implicits.scala b/uring/jvm/src/main/scala/fs2/io/uring/implicits.scala new file mode 100644 index 00000000..7232a779 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/implicits.scala @@ -0,0 +1,28 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import cats.effect.LiftIO +import cats.effect.kernel.Async +import com.comcast.ip4s.Dns +import fs2.io.net.Network + +object implicits { + + @inline implicit def network[F[_]: Async: Dns: LiftIO]: Network[F] = ??? + +} diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala new file mode 100644 index 00000000..7f24ec9b --- /dev/null +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import cats.effect.unsafe.IORuntime +// import cats.effect.unsafe.IORuntimeConfig +// import fs2.io.uring.unsafe.UringSystem +import munit.CatsEffectSuite + +abstract class UringSuite extends CatsEffectSuite { + + override lazy val munitIORuntime = IORuntime.global +} \ No newline at end of file From 46f3dc822566b4e629fe570c52ff54aef5b1707c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 31 May 2023 19:33:21 +0200 Subject: [PATCH 003/200] Import Netty io_uring --- build.sbt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index ae143f1c..2f1877db 100644 --- a/build.sbt +++ b/build.sbt @@ -53,7 +53,8 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) libraryDependencies ++= Seq( "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, - "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test + "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, + "io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final" classifier "linux-x86_64" ), Test / testOptions += Tests.Argument("+l") ) From fd2c4593a4d68ad52dab61fb4f084cf1ea8e7b6b Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 12:37:55 +0200 Subject: [PATCH 004/200] Expose Native methods and RingBuffer --- .../fs2/io/uring/unsafe/NativeAccess.scala | 154 ++++++++++++++++++ 1 file changed, 154 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala new file mode 100644 index 00000000..197968d6 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala @@ -0,0 +1,154 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.netty.incubator.channel.uring + +import cats.effect.{Resource, Sync} +import io.netty.channel.unix.FileDescriptor + +object NativeAccessEffect { + + val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE + val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD + + def createRingBuffer[F[_]: Sync](): Resource[F, RingBuffer] = + createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer[F[_]: Sync](size: Int): Resource[F, RingBuffer] = + createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer[F[_]: Sync](size: Int, sqeAsyncThreshold: Int): Resource[F, RingBuffer] = + Resource.make { + Sync[F].delay(Native.createRingBuffer(size, sqeAsyncThreshold)) + } { ringBuffer => + Sync[F].delay(ringBuffer.close()) + } + + def checkAllIOSupported[F[_]: Sync](ringFd: Int): F[Unit] = + Sync[F].delay(Native.checkAllIOSupported(ringFd)) + + def checkKernelVersion[F[_]: Sync](kernelVersion: String): F[Unit] = + Sync[F].delay(Native.checkKernelVersion(kernelVersion)) + + def ioUringEnter[F[_]: Sync](ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): F[Int] = + Sync[F].delay(Native.ioUringEnter(ringFd, toSubmit, minComplete, flags)) + + def eventFdWrite[F[_]: Sync](fd: Int, value: Long): F[Unit] = + Sync[F].delay(Native.eventFdWrite(fd, value)) + + def newBlockingEventFd[F[_]: Sync]: F[FileDescriptor] = + Sync[F].delay(Native.newBlockingEventFd()) + + def ioUringExit[F[_]: Sync]( + submissionQueueArrayAddress: Long, + submissionQueueRingEntries: Int, + submissionQueueRingAddress: Long, + submissionQueueRingSize: Int, + completionQueueRingAddress: Long, + completionQueueRingSize: Int, + ringFd: Int + ): F[Unit] = + Sync[F].delay( + Native.ioUringExit( + submissionQueueArrayAddress, + submissionQueueRingEntries, + submissionQueueRingAddress, + submissionQueueRingSize, + completionQueueRingAddress, + completionQueueRingSize, + ringFd + ) + ) + + def createFile[F[_]: Sync](name: String): F[Int] = + Sync[F].delay(Native.createFile(name)) + + def cmsghdrData[F[_]: Sync](hdrAddr: Long): F[Long] = + Sync[F].delay(Native.cmsghdrData(hdrAddr)) + + def kernelVersion[F[_]: Sync]: F[String] = + Sync[F].delay(Native.kernelVersion()) + +} + +object NativeAccess { + val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE + val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD + + def createRingBuffer(): RingBuffer = + createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer(size: Int): RingBuffer = + createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer(size: Int, sqeAsyncThreshold: Int): RingBuffer = + Native.createRingBuffer(size, sqeAsyncThreshold) + + def checkAllIOSupported(ringFd: Int): Unit = + Native.checkAllIOSupported(ringFd) + + def checkKernelVersion(kernelVersion: String): Unit = + Native.checkKernelVersion(kernelVersion) + + def ioUringEnter(ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): Int = + Native.ioUringEnter(ringFd, toSubmit, minComplete, flags) + + def eventFdWrite(fd: Int, value: Long): Unit = + Native.eventFdWrite(fd, value) + + def newBlockingEventFd: FileDescriptor = + Native.newBlockingEventFd() + + def ioUringExit( + submissionQueueArrayAddress: Long, + submissionQueueRingEntries: Int, + submissionQueueRingAddress: Long, + submissionQueueRingSize: Int, + completionQueueRingAddress: Long, + completionQueueRingSize: Int, + ringFd: Int + ): Unit = + Native.ioUringExit( + submissionQueueArrayAddress, + submissionQueueRingEntries, + submissionQueueRingAddress, + submissionQueueRingSize, + completionQueueRingAddress, + completionQueueRingSize, + ringFd + ) +} + +trait RingBuffer { + def ioUringCompletionQueue(): IOUringCompletionQueue + def ioUringSubmissionQueue(): IOUringSubmissionQueue + def fd(): Int + def close(): Unit +} + +final class MyRingBuffer extends RingBuffer { + private[this] val ringBuffer: RingBuffer = NativeAccess.createRingBuffer() + + override def ioUringCompletionQueue(): IOUringCompletionQueue = + ringBuffer.ioUringCompletionQueue() + + override def ioUringSubmissionQueue(): IOUringSubmissionQueue = + ringBuffer.ioUringSubmissionQueue() + + override def fd(): Int = ringBuffer.fd() + + override def close(): Unit = ringBuffer.close() +} From be118ef2bbd25ea2f2b99ea497fa41e33228e6cb Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 16:33:22 +0200 Subject: [PATCH 005/200] Add Completion and Submission Queues --- .../uring/unsafe/UringCompletionQueue.scala | 18 ++++ .../uring/unsafe/UringSubmissionQueue.scala | 84 +++++++++++++++++++ 2 files changed, 102 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala new file mode 100644 index 00000000..1fdf9394 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala @@ -0,0 +1,18 @@ +package io.netty.incubator.channel.uring + +class UringCompletionQueue(ring: RingBuffer) { + + private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() + + def hashCompletitions(): Boolean = completionQueue.hasCompletions() + + def processs(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) + + def ioUringWaitCqe(): Unit = completionQueue.ioUringWaitCqe() + + def ringAddress(): Long = completionQueue.ringAddress + + def ringFd(): Int = completionQueue.ringFd + + def ringSize(): Int = completionQueue.ringSize +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala new file mode 100644 index 00000000..c845b8d9 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -0,0 +1,84 @@ +package io.netty.incubator.channel.uring + +class UringSubmissionQueue(ring: RingBuffer) { + private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() + + def enqueueSqe( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long, + data: Short + ): Boolean = + submissionQueue.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + + def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() + + def decrementHandledFds(): Unit = submissionQueue.decrementHandledFds() + + def addTimeout(nanoSeconds: Long, extraData: Short): Boolean = + submissionQueue.addTimeout(nanoSeconds, extraData) + + def addPollIn(fd: Int): Boolean = submissionQueue.addPollIn(fd) + + def addPollRdHup(fd: Int): Boolean = submissionQueue.addPollRdHup(fd) + + def addPollOut(fd: Int): Boolean = submissionQueue.addPollOut(fd) + + def addRecvmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = + submissionQueue.addRecvmsg(fd, msgHdr, extraData) + + def addSendmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = + submissionQueue.addSendmsg(fd, msgHdr, extraData) + + def addSendmsg(fd: Int, msgHdr: Long, flags: Int, extraData: Short): Boolean = + submissionQueue.addSendmsg(fd, msgHdr, flags, extraData) + + def addRead(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addRead(fd, bufferAddress, pos, limit, extraData) + + def addEventFdRead( + fd: Int, + bufferAddress: Long, + pos: Int, + limit: Int, + extraData: Short + ): Boolean = submissionQueue.addEventFdRead(fd, bufferAddress, pos, limit, extraData) + + def addWrite(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData) + + def addRecv(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addRecv(fd, bufferAddress, pos, limit, extraData) + + def addSend(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addSend(fd, bufferAddress, pos, limit, extraData) + + def addAccept(fd: Int, address: Long, addressLength: Long, extraData: Short): Boolean = + submissionQueue.addAccept(fd, address, addressLength, extraData) + + def addPollRemove(fd: Int, pollMask: Int): Boolean = submissionQueue.addPollRemove(fd, pollMask) + + def addConnect( + fd: Int, + socketAddress: Long, + socketAddressLength: Long, + extraData: Short + ): Boolean = submissionQueue.addConnect(fd, socketAddress, socketAddressLength, extraData) + + def addWritev(fd: Int, iovecArrayAddress: Long, length: Int, extraData: Short): Boolean = + submissionQueue.addWritev(fd, iovecArrayAddress, length, extraData) + + def addClose(fd: Int, extraData: Short): Boolean = submissionQueue.addClose(fd, extraData) + + def submit(): Int = submissionQueue.submit() + + def submitAndWait(): Int = submissionQueue.submitAndWait() + + def count(): Long = submissionQueue.count() + + def release(): Unit = submissionQueue.release() +} From 6e9ea0aa2c2b38b17565206e7377f3f170fc9ac2 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 16:33:58 +0200 Subject: [PATCH 006/200] Add RingBuffer --- .../fs2/io/uring/unsafe/NativeAccess.scala | 21 ------------------- .../scala/fs2/io/uring/unsafe/UringRing.scala | 18 ++++++++++++++++ 2 files changed, 18 insertions(+), 21 deletions(-) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala index 197968d6..57e59154 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala @@ -131,24 +131,3 @@ object NativeAccess { ringFd ) } - -trait RingBuffer { - def ioUringCompletionQueue(): IOUringCompletionQueue - def ioUringSubmissionQueue(): IOUringSubmissionQueue - def fd(): Int - def close(): Unit -} - -final class MyRingBuffer extends RingBuffer { - private[this] val ringBuffer: RingBuffer = NativeAccess.createRingBuffer() - - override def ioUringCompletionQueue(): IOUringCompletionQueue = - ringBuffer.ioUringCompletionQueue() - - override def ioUringSubmissionQueue(): IOUringSubmissionQueue = - ringBuffer.ioUringSubmissionQueue() - - override def fd(): Int = ringBuffer.fd() - - override def close(): Unit = ringBuffer.close() -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala new file mode 100644 index 00000000..319980a4 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -0,0 +1,18 @@ +package io.netty.incubator.channel.uring + +import io.netty.incubator.channel.uring.UringCompletionQueue +import io.netty.incubator.channel.uring.UringSubmissionQueue + +class UringRing { + private val ringBuffer: RingBuffer = NativeAccess.createRingBuffer() + + def ioUringCompletionQueue(): UringCompletionQueue = + new UringCompletionQueue(ringBuffer) + + def ioUringSubmissionQueue(): UringSubmissionQueue = + new UringSubmissionQueue(ringBuffer) + + def fd(): Int = ringBuffer.fd() + + def close(): Unit = ringBuffer.close() +} From 88f02e7a1f7d388f211157543a9b27574dbe2ebd Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 16:35:06 +0200 Subject: [PATCH 007/200] Add header --- .../io/uring/unsafe/UringCompletionQueue.scala | 16 ++++++++++++++++ .../scala/fs2/io/uring/unsafe/UringRing.scala | 16 ++++++++++++++++ .../io/uring/unsafe/UringSubmissionQueue.scala | 16 ++++++++++++++++ 3 files changed, 48 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala index 1fdf9394..ef6d8ed4 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.netty.incubator.channel.uring class UringCompletionQueue(ring: RingBuffer) { diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 319980a4..07b5eb8f 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.netty.incubator.channel.uring import io.netty.incubator.channel.uring.UringCompletionQueue diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index c845b8d9..0dc973f2 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.netty.incubator.channel.uring class UringSubmissionQueue(ring: RingBuffer) { From 7be84f6041a6aedde58f074acc8a916f8d0a6467 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 16:52:56 +0200 Subject: [PATCH 008/200] Replace Ptr with SubmissionQueue --- uring/jvm/src/main/scala/fs2/io/uring/Uring.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index e30d9de1..ec14eea0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -21,12 +21,13 @@ import cats.effect.LiftIO import cats.effect.kernel.Resource import cats.syntax.all._ +import io.netty.incubator.channel.uring.UringSubmissionQueue abstract class Uring private[uring] { - def call(): IO[Int] + def call(prep: UringSubmissionQueue => Unit): IO[Int] - def bracket()(release: Int => IO[Unit]): Resource[IO, Int] + def bracket(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): Resource[IO, Int] } object Uring { From 08eb5068c7b9b13726dcf581b238aa6369eb00da Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 16:53:43 +0200 Subject: [PATCH 009/200] Call UringSystem as pollingSystem --- uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala b/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala index a4047dd3..ddf7903b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala @@ -17,10 +17,11 @@ package fs2.io.uring import cats.effect.IOApp +import fs2.io.uring.unsafe.UringSystem trait UringApp extends IOApp { - override protected final def pollingSystem = ??? + override protected final def pollingSystem = UringSystem } From 486a1648cc667b12c784d68f2e0e9075317e4a23 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 18:21:36 +0200 Subject: [PATCH 010/200] Add values for Completion and Submission queues --- .../main/scala/fs2/io/uring/unsafe/UringRing.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 07b5eb8f..e9b8ef17 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -16,17 +16,18 @@ package io.netty.incubator.channel.uring +import io.netty.incubator.channel.uring.NativeAccess.createRingBuffer import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringSubmissionQueue class UringRing { - private val ringBuffer: RingBuffer = NativeAccess.createRingBuffer() + private val ringBuffer: RingBuffer = createRingBuffer() + private val uringCompletionQueue: UringCompletionQueue = new UringCompletionQueue(ringBuffer) + private val uringSubmissionQueue: UringSubmissionQueue = new UringSubmissionQueue(ringBuffer) - def ioUringCompletionQueue(): UringCompletionQueue = - new UringCompletionQueue(ringBuffer) + def ioUringCompletionQueue(): UringCompletionQueue = uringCompletionQueue - def ioUringSubmissionQueue(): UringSubmissionQueue = - new UringSubmissionQueue(ringBuffer) + def ioUringSubmissionQueue(): UringSubmissionQueue = uringSubmissionQueue def fd(): Int = ringBuffer.fd() From 09b79cb001b48fec0a8e06676dd162f9e2a1d366 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 5 Jun 2023 18:22:23 +0200 Subject: [PATCH 011/200] Start implementation of Poller --- .../fs2/io/uring/unsafe/UringSystem.scala | 31 ++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 97f822ca..00c37e92 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -18,6 +18,13 @@ package fs2.io.uring package unsafe import cats.effect.unsafe.PollingSystem +import io.netty.incubator.channel.uring.UringRing +import io.netty.incubator.channel.uring.UringSubmissionQueue + +import java.util.Collections +import java.util.IdentityHashMap +import java.util.Set + object UringSystem extends PollingSystem { @@ -33,6 +40,28 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = ??? + final class Poller private[UringSystem] (ring: UringRing) { + + private[this] var pendingSubmissions: Boolean = false + private[this] val callbacks: Set[Either[Throwable, Int] => Unit] = + Collections.newSetFromMap(new IdentityHashMap) + + private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): UringSubmissionQueue = { + pendingSubmissions = true + val sqe = ring.ioUringSubmissionQueue() + // TODO: We modify the "data" from the sqe + callbacks.add(cb) + sqe + } + + private[UringSystem] def close(): Unit = ring.close() + + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty() + + private[UringSystem] def poll(nanos: Long): Boolean = ??? + + private[this] def processCqes(_cqes: List[UringSubmissionQueue]): Boolean = ??? + + } - final class Poller private[UringSystem] () {} } From d767b7bb329ebe2c51518b5be9da73e40b0601d9 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 11:05:12 +0200 Subject: [PATCH 012/200] Add Companion Object --- .../uring/unsafe/UringCompletionQueue.scala | 6 ++++- .../scala/fs2/io/uring/unsafe/UringRing.scala | 25 +++++++++++++++---- .../uring/unsafe/UringSubmissionQueue.scala | 8 +++++- 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala index ef6d8ed4..2f2e2ea8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala @@ -16,7 +16,7 @@ package io.netty.incubator.channel.uring -class UringCompletionQueue(ring: RingBuffer) { +class UringCompletionQueue(private val ring: RingBuffer) { private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() @@ -32,3 +32,7 @@ class UringCompletionQueue(ring: RingBuffer) { def ringSize(): Int = completionQueue.ringSize } + +object UringCompletionQueue { + def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) +} \ No newline at end of file diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index e9b8ef17..2c162049 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -16,14 +16,20 @@ package io.netty.incubator.channel.uring -import io.netty.incubator.channel.uring.NativeAccess.createRingBuffer +import io.netty.incubator.channel.uring.NativeAccess._ import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringSubmissionQueue -class UringRing { - private val ringBuffer: RingBuffer = createRingBuffer() - private val uringCompletionQueue: UringCompletionQueue = new UringCompletionQueue(ringBuffer) - private val uringSubmissionQueue: UringSubmissionQueue = new UringSubmissionQueue(ringBuffer) +class UringRing(private val ringBuffer: RingBuffer) { + private val uringCompletionQueue: UringCompletionQueue = UringCompletionQueue(ringBuffer) + private val uringSubmissionQueue: UringSubmissionQueue = UringSubmissionQueue(ringBuffer) + + def this() = this(createRingBuffer()) + + def this(size: Int) = this(createRingBuffer(size)) + + def this(size: Int, sqeAsyncThreshold: Int) = + this(createRingBuffer(size, sqeAsyncThreshold)) def ioUringCompletionQueue(): UringCompletionQueue = uringCompletionQueue @@ -33,3 +39,12 @@ class UringRing { def close(): Unit = ringBuffer.close() } + +object UringRing { + def apply(): UringRing = new UringRing() + + def apply(size: Int): UringRing = new UringRing(size) + + def apply(size: Int, sqeAsyncThreshold: Int): UringRing = new UringRing(size, sqeAsyncThreshold) + +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index 0dc973f2..14926761 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -16,7 +16,7 @@ package io.netty.incubator.channel.uring -class UringSubmissionQueue(ring: RingBuffer) { +class UringSubmissionQueue(private val ring: RingBuffer) { private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() def enqueueSqe( @@ -97,4 +97,10 @@ class UringSubmissionQueue(ring: RingBuffer) { def count(): Long = submissionQueue.count() def release(): Unit = submissionQueue.release() + + def setData[A <: AnyRef](data: A): Unit = ??? +} + +object UringSubmissionQueue { + def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) } From 408252c611c8c2f9353aadb14922e16035c4fa81 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 12:02:01 +0200 Subject: [PATCH 013/200] Sketch additional methods needed --- .../io/uring/unsafe/UringSubmissionQueue.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index 14926761..a9f3e89f 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -98,7 +98,21 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def release(): Unit = submissionQueue.release() - def setData[A <: AnyRef](data: A): Unit = ??? + def setData[A <: AnyRef](data: A): Unit = + // TODO: We need to set data in UringSystem.Poller.getSqe + ??? + + def userData(): Long = + // TODO: We need to access the userData in UringSystem.ApiImpl.exec + ??? + + def prepCancel(userData: Long, flags: Int): Unit = + // TODO: We need to cancel in UringSystem.ApiImpl.cancel + ??? + + def prepPollAdd(fd: Int, pollMask: Int): Unit = // TODO: mask must be unsigned Int + // TODO: We need to poll in UringSystem.ApiImpl.registerFileDescriptor + ??? } object UringSubmissionQueue { From 3cb48c71437b8ac647095f33da5628cc5d0f27db Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 12:05:59 +0200 Subject: [PATCH 014/200] Sketch reimplementation of UringSystem --- .../fs2/io/uring/unsafe/UringSystem.scala | 131 ++++++++++++++++-- 1 file changed, 123 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 00c37e92..5504e446 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -17,7 +17,21 @@ package fs2.io.uring package unsafe +import cats.~> +import cats.syntax.all._ + +import cats.effect.IO +import cats.effect.FileDescriptorPoller +import cats.effect.FileDescriptorPollHandle + +import cats.effect.kernel.Resource +import cats.effect.kernel.MonadCancelThrow +import cats.effect.kernel.Cont +import cats.effect.std.Semaphore + + import cats.effect.unsafe.PollingSystem + import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue @@ -25,20 +39,121 @@ import java.util.Collections import java.util.IdentityHashMap import java.util.Set - object UringSystem extends PollingSystem { - override def makeApi(register: (Poller => Unit) => Unit): Api = ??? + private final val MaxEvents = 64 - override def makePoller(): Poller = ??? + type Api = Uring with FileDescriptorPoller - override def closePoller(poller: Poller): Unit = ??? + type Address = Long - override def poll(poller: Poller, nanos: Long, reportFailure: Throwable => Unit): Boolean = ??? + override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) - override def needsPoll(poller: Poller): Boolean = ??? + override def makePoller(): Poller = { + val ring = UringRing() + // TODO: review potential errors and handle them + new Poller(ring) + } - override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = ??? + override def closePoller(poller: Poller): Unit = poller.close() + + override def poll(poller: Poller, nanos: Long, reportFailure: Throwable => Unit): Boolean = + poller.poll(nanos) + + override def needsPoll(poller: Poller): Boolean = poller.needsPoll() + + override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () + + private final class ApiImpl(register: (Poller => Unit) => Unit) + extends Uring + with FileDescriptorPoller { + private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit + + def call(prep: UringSubmissionQueue => Unit): IO[Int] = + exec(prep)(noopRelease) + + def bracket(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): Resource[IO, Int] = + Resource.makeFull[IO, Int](poll => poll(exec(prep)(release(_))))(release(_)) + + private def exec(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): IO[Int] = + IO.cont { + new Cont[IO, Int, Int] { + def apply[F[_]](implicit + F: MonadCancelThrow[F] + ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => + F.uncancelable { poll => + val submit = IO.async_[Long] { cb => // TODO: We need Unsigned Long here + register { ring => + val sqe = ring.getSqe(resume) + prep(sqe) + cb(Right(sqe.userData())) + } + } + + lift(submit) + .flatMap { addr => + F.onCancel( + poll(get), + lift(cancel(addr)).ifM( + F.unit, + get.flatMap { rtn => + if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) + else lift(release(rtn)) + } + ) + ) + } + .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) + } + } + } + } + + private[this] def cancel(addr: Address): IO[Boolean] = + IO.async_[Int] { cb => + register { ring => + val sqe = ring.getSqe(cb) + sqe.prepCancel(addr, 0) + } + }.map(_ == 0) // true if we canceled + + def registerFileDescriptor( + fd: Int, + reads: Boolean, + writes: Boolean + ): Resource[IO, FileDescriptorPollHandle] = + Resource.eval { + (Semaphore[IO](1), Semaphore[IO](1)).mapN { (readSemaphore, writeSemaphore) => + new FileDescriptorPollHandle { + + def pollReadRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = + readSemaphore.permit.surround { + a.tailRecM { a => + f(a).flatTap { r => + if (r.isRight) + IO.unit + else + call(_.prepPollAdd(fd, 0x001.toInt)) + } + } + } + + def pollWriteRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = + writeSemaphore.permit.surround { + a.tailRecM { a => + f(a).flatTap { r => + if (r.isRight) + IO.unit + else + call(_.prepPollAdd(fd, 0x004.toInt)) + } + } + } + } + + } + } + } final class Poller private[UringSystem] (ring: UringRing) { @@ -49,7 +164,7 @@ object UringSystem extends PollingSystem { private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): UringSubmissionQueue = { pendingSubmissions = true val sqe = ring.ioUringSubmissionQueue() - // TODO: We modify the "data" from the sqe + sqe.setData(cb) callbacks.add(cb) sqe } From d6e7d7667272f7e9a2333ed6fdfac14f9ecbbda3 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 14:59:11 +0200 Subject: [PATCH 015/200] Add POLLIN & POLLOUT --- .../src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala | 3 +++ .../scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala | 4 ---- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 7 ++----- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala index 57e59154..d805f0bc 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala @@ -87,6 +87,9 @@ object NativeAccessEffect { object NativeAccess { val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD + + val POLLIN = Native.POLLIN + val POLLOUT = Native.POLLOUT def createRingBuffer(): RingBuffer = createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index a9f3e89f..13be9c8b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -109,10 +109,6 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def prepCancel(userData: Long, flags: Int): Unit = // TODO: We need to cancel in UringSystem.ApiImpl.cancel ??? - - def prepPollAdd(fd: Int, pollMask: Int): Unit = // TODO: mask must be unsigned Int - // TODO: We need to poll in UringSystem.ApiImpl.registerFileDescriptor - ??? } object UringSubmissionQueue { diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 5504e446..040e259d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -29,7 +29,6 @@ import cats.effect.kernel.MonadCancelThrow import cats.effect.kernel.Cont import cats.effect.std.Semaphore - import cats.effect.unsafe.PollingSystem import io.netty.incubator.channel.uring.UringRing @@ -133,7 +132,7 @@ object UringSystem extends PollingSystem { if (r.isRight) IO.unit else - call(_.prepPollAdd(fd, 0x001.toInt)) + call(_.addPollIn(fd)) } } } @@ -145,7 +144,7 @@ object UringSystem extends PollingSystem { if (r.isRight) IO.unit else - call(_.prepPollAdd(fd, 0x004.toInt)) + call(_.addPollOut(fd)) } } } @@ -175,8 +174,6 @@ object UringSystem extends PollingSystem { private[UringSystem] def poll(nanos: Long): Boolean = ??? - private[this] def processCqes(_cqes: List[UringSubmissionQueue]): Boolean = ??? - } } From 2b0f5aae594ef79ae6cca4edd79ab55cf0cf03f2 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 18:31:56 +0200 Subject: [PATCH 016/200] Remove FileDescriptor Poller and Handle --- .../fs2/io/uring/unsafe/UringSystem.scala | 45 +------------------ 1 file changed, 2 insertions(+), 43 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 040e259d..e39f4226 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -21,13 +21,10 @@ import cats.~> import cats.syntax.all._ import cats.effect.IO -import cats.effect.FileDescriptorPoller -import cats.effect.FileDescriptorPollHandle import cats.effect.kernel.Resource import cats.effect.kernel.MonadCancelThrow import cats.effect.kernel.Cont -import cats.effect.std.Semaphore import cats.effect.unsafe.PollingSystem @@ -42,7 +39,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - type Api = Uring with FileDescriptorPoller + type Api = Uring type Address = Long @@ -64,8 +61,7 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () private final class ApiImpl(register: (Poller => Unit) => Unit) - extends Uring - with FileDescriptorPoller { + extends Uring { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit def call(prep: UringSubmissionQueue => Unit): IO[Int] = @@ -115,43 +111,6 @@ object UringSystem extends PollingSystem { sqe.prepCancel(addr, 0) } }.map(_ == 0) // true if we canceled - - def registerFileDescriptor( - fd: Int, - reads: Boolean, - writes: Boolean - ): Resource[IO, FileDescriptorPollHandle] = - Resource.eval { - (Semaphore[IO](1), Semaphore[IO](1)).mapN { (readSemaphore, writeSemaphore) => - new FileDescriptorPollHandle { - - def pollReadRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = - readSemaphore.permit.surround { - a.tailRecM { a => - f(a).flatTap { r => - if (r.isRight) - IO.unit - else - call(_.addPollIn(fd)) - } - } - } - - def pollWriteRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = - writeSemaphore.permit.surround { - a.tailRecM { a => - f(a).flatTap { r => - if (r.isRight) - IO.unit - else - call(_.addPollOut(fd)) - } - } - } - } - - } - } } final class Poller private[UringSystem] (ring: UringRing) { From 0d1ae856d5c58a34efda14444f593b2dfda02584 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 18:54:35 +0200 Subject: [PATCH 017/200] Formatting --- .../src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala | 2 +- .../scala/fs2/io/uring/unsafe/UringCompletionQueue.scala | 2 +- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 3 +-- uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala | 6 +----- uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala | 6 ++---- 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala index d805f0bc..f167277e 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala @@ -87,7 +87,7 @@ object NativeAccessEffect { object NativeAccess { val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD - + val POLLIN = Native.POLLIN val POLLOUT = Native.POLLOUT diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala index 2f2e2ea8..1617c6c1 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala @@ -35,4 +35,4 @@ class UringCompletionQueue(private val ring: RingBuffer) { object UringCompletionQueue { def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) -} \ No newline at end of file +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index e39f4226..7068a23c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -60,8 +60,7 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () - private final class ApiImpl(register: (Poller => Unit) => Unit) - extends Uring { + private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit def call(prep: UringSubmissionQueue => Unit): IO[Int] = diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index d2801c39..87d2e6c3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -16,8 +16,4 @@ package fs2.io.uring.unsafe - -private[uring] object util { - - -} +private[uring] object util {} diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 7f24ec9b..69bb2cba 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -16,12 +16,10 @@ package fs2.io.uring -import cats.effect.unsafe.IORuntime -// import cats.effect.unsafe.IORuntimeConfig -// import fs2.io.uring.unsafe.UringSystem +import cats.effect.unsafe.IORuntimes import munit.CatsEffectSuite abstract class UringSuite extends CatsEffectSuite { override lazy val munitIORuntime = IORuntime.global -} \ No newline at end of file +} From 18dd9f3acc5ea512d35f6c22d92f14eabe205c09 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 18:59:32 +0200 Subject: [PATCH 018/200] Formatting --- build.sbt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 2f1877db..a648edf5 100644 --- a/build.sbt +++ b/build.sbt @@ -54,7 +54,8 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, - "io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final" classifier "linux-x86_64" + ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") + .classifier("linux-x86_64") ), Test / testOptions += Tests.Argument("+l") ) From 3fdda21e4ef3045cf7861ebc36ceaab2870eb883 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 8 Jun 2023 22:48:12 +0200 Subject: [PATCH 019/200] Add prepCancel --- .../scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala | 9 ++++++--- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 8 +++++--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index 13be9c8b..8ed829bc 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -16,6 +16,8 @@ package io.netty.incubator.channel.uring +import UringSubmissionQueue.IORING_OP_ASYNC_CANCEL + class UringSubmissionQueue(private val ring: RingBuffer) { private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() @@ -106,11 +108,12 @@ class UringSubmissionQueue(private val ring: RingBuffer) { // TODO: We need to access the userData in UringSystem.ApiImpl.exec ??? - def prepCancel(userData: Long, flags: Int): Unit = - // TODO: We need to cancel in UringSystem.ApiImpl.cancel - ??? + def prepCancel(addr: Long, flags: Int): Boolean = + enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) } object UringSubmissionQueue { + final val IORING_OP_ASYNC_CANCEL: Byte = 14.toByte + def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 7068a23c..50a894c0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -76,7 +76,7 @@ object UringSystem extends PollingSystem { F: MonadCancelThrow[F] ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => - val submit = IO.async_[Long] { cb => // TODO: We need Unsigned Long here + val submit = IO.async_[Long] { cb => register { ring => val sqe = ring.getSqe(resume) prep(sqe) @@ -107,9 +107,11 @@ object UringSystem extends PollingSystem { IO.async_[Int] { cb => register { ring => val sqe = ring.getSqe(cb) - sqe.prepCancel(addr, 0) + val wasCancelled = sqe.prepCancel(addr, 0) + cb(Right(if (wasCancelled) 1 else 0)) } - }.map(_ == 0) // true if we canceled + }.map(_ == 1) + } final class Poller private[UringSystem] (ring: UringRing) { From f294e33f9a55ec64ca474ca81fe8d96207e635db Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 10 Jun 2023 19:41:15 +0200 Subject: [PATCH 020/200] Sketch a plan for manipulating the Data --- .../uring/unsafe/UringSubmissionQueue.scala | 27 ++++++++++++++----- .../fs2/io/uring/unsafe/UringSystem.scala | 4 +-- 2 files changed, 23 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index 8ed829bc..1d27ee52 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -16,7 +16,9 @@ package io.netty.incubator.channel.uring -import UringSubmissionQueue.IORING_OP_ASYNC_CANCEL +import io.netty.util.internal.PlatformDependent + +import UringSubmissionQueue._ class UringSubmissionQueue(private val ring: RingBuffer) { private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() @@ -100,20 +102,33 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def release(): Unit = submissionQueue.release() - def setData[A <: AnyRef](data: A): Unit = + def setData[A <: AnyRef](data: Long): Unit = { // TODO: We need to set data in UringSystem.Poller.getSqe - ??? + // Update: Expose the tail go get access to the address and be able to manipulate the data + val ringMask: Int = submissionQueue.ringEntries - 1 + val sqe: Long = submissionQueue.submissionQueueArrayAddress + (tail ++ & ringMask) * SQE_SIZE + PlatformDependent.putLong(sqe + SQE_USER_DATA_FIELD, data) + } - def userData(): Long = + def userData(): Long = { // TODO: We need to access the userData in UringSystem.ApiImpl.exec - ??? + // Update: Expose the tail go get access to the address and be able to manipulate the data + val ringMask: Int = submissionQueue.ringEntries - 1 + val sqe: Long = submissionQueue.submissionQueueArrayAddress + (tail ++ & ringMask) * SQE_SIZE + PlatformDependent.getLong(sqe + SQE_USER_DATA_FIELD) + } def prepCancel(addr: Long, flags: Int): Boolean = enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) + } object UringSubmissionQueue { + final val SQE_SIZE = 64 + final val IORING_OP_ASYNC_CANCEL: Byte = 14.toByte + final val SQE_USER_DATA_FIELD = 32 + def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) -} +} \ No newline at end of file diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 50a894c0..ce28b3c8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -80,7 +80,7 @@ object UringSystem extends PollingSystem { register { ring => val sqe = ring.getSqe(resume) prep(sqe) - cb(Right(sqe.userData())) + cb(Right(sqe.userData())) // TODO: Implement userData() } } @@ -123,7 +123,7 @@ object UringSystem extends PollingSystem { private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): UringSubmissionQueue = { pendingSubmissions = true val sqe = ring.ioUringSubmissionQueue() - sqe.setData(cb) + sqe.setData(cb) // TODO: Implement setData() callbacks.add(cb) sqe } From 1bc768ee9078bd8326592bfd30619f093b978d1c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 20 Jun 2023 14:03:54 +0200 Subject: [PATCH 021/200] Implement userData and setData --- .../uring/unsafe/UringSubmissionQueue.scala | 56 +++++++++++++------ 1 file changed, 40 insertions(+), 16 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala index 1d27ee52..e95082aa 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala @@ -16,9 +16,10 @@ package io.netty.incubator.channel.uring -import io.netty.util.internal.PlatformDependent - import UringSubmissionQueue._ +import NativeAccess._ +import scala.collection.mutable.LongMap +import java.io.IOException class UringSubmissionQueue(private val ring: RingBuffer) { private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() @@ -102,21 +103,44 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def release(): Unit = submissionQueue.release() - def setData[A <: AnyRef](data: Long): Unit = { - // TODO: We need to set data in UringSystem.Poller.getSqe - // Update: Expose the tail go get access to the address and be able to manipulate the data - val ringMask: Int = submissionQueue.ringEntries - 1 - val sqe: Long = submissionQueue.submissionQueueArrayAddress + (tail ++ & ringMask) * SQE_SIZE - PlatformDependent.putLong(sqe + SQE_USER_DATA_FIELD, data) + def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) + + private[this] val callbacks = new LongMap[Either[Throwable, Long] => Unit]() + var counter: Short = 0 + + def setData(cb: Either[Throwable, Long] => Unit): Boolean = { + val op: Byte = IORING_OP_POLL_WRITE + val flags: Int = 0 + val rwFlags: Int = Native.POLLOUT + val fd: Int = 0 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + counter = (counter + 1).toShort + + val success: Boolean = enqueueSqe( + op, + flags, + rwFlags, + fd, + bufferAddress, + length, + offset, + counter + ) + if (success) callbacks.update(encode(fd, op, counter), cb) + else + cb(Left(new IOException("Failed to enqueue"))) + + success } - def userData(): Long = { - // TODO: We need to access the userData in UringSystem.ApiImpl.exec - // Update: Expose the tail go get access to the address and be able to manipulate the data - val ringMask: Int = submissionQueue.ringEntries - 1 - val sqe: Long = submissionQueue.submissionQueueArrayAddress + (tail ++ & ringMask) * SQE_SIZE - PlatformDependent.getLong(sqe + SQE_USER_DATA_FIELD) - } + def userData(data: Long): Either[Throwable, Long] => Unit = + callbacks.getOrElse( + data, + throw new NoSuchElementException(s"Callback not found for data: $data") + ) def prepCancel(addr: Long, flags: Int): Boolean = enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) @@ -131,4 +155,4 @@ object UringSubmissionQueue { final val SQE_USER_DATA_FIELD = 32 def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) -} \ No newline at end of file +} From 57b6ff34e6ea962f1f447531da0923166167c98f Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 20 Jun 2023 14:05:16 +0200 Subject: [PATCH 022/200] Implement poll and adapt get & set Data --- .../unsafe/UringCompletionQueueCallback.scala | 25 ++++++ .../fs2/io/uring/unsafe/UringSystem.scala | 82 +++++++++++++++---- 2 files changed, 90 insertions(+), 17 deletions(-) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala new file mode 100644 index 00000000..bbd4c322 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala @@ -0,0 +1,25 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.netty.incubator.channel.uring + +/** The UringCompletionQueueCallback trait defines a callback interface for handling completion events + * from the io_uring completion queue. It extends the IOUringCompletionQueueCallback trait and provides + * a method handle to process the completion event. + */ +trait UringCompletionQueueCallback extends IOUringCompletionQueueCallback { + def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index ce28b3c8..24d0f834 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -30,10 +30,10 @@ import cats.effect.unsafe.PollingSystem import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue +import io.netty.incubator.channel.uring.UringCompletionQueueCallback -import java.util.Collections -import java.util.IdentityHashMap -import java.util.Set +import java.util.concurrent.ConcurrentHashMap +import java.io.IOException object UringSystem extends PollingSystem { @@ -45,11 +45,8 @@ object UringSystem extends PollingSystem { override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) - override def makePoller(): Poller = { - val ring = UringRing() - // TODO: review potential errors and handle them - new Poller(ring) - } + override def makePoller(): Poller = + new Poller(UringRing()) override def closePoller(poller: Poller): Unit = poller.close() @@ -74,13 +71,18 @@ object UringSystem extends PollingSystem { new Cont[IO, Int, Int] { def apply[F[_]](implicit F: MonadCancelThrow[F] - ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => + ): (Either[Throwable, Long] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => val submit = IO.async_[Long] { cb => register { ring => val sqe = ring.getSqe(resume) prep(sqe) - cb(Right(sqe.userData())) // TODO: Implement userData() + + sqe.setData(cb) + + val userData = sqe.encode(0, 0, sqe.counter) + + cb(Right(userData)) } } @@ -104,7 +106,7 @@ object UringSystem extends PollingSystem { } private[this] def cancel(addr: Address): IO[Boolean] = - IO.async_[Int] { cb => + IO.async_[Long] { cb => register { ring => val sqe = ring.getSqe(cb) val wasCancelled = sqe.prepCancel(addr, 0) @@ -117,14 +119,21 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { private[this] var pendingSubmissions: Boolean = false - private[this] val callbacks: Set[Either[Throwable, Int] => Unit] = - Collections.newSetFromMap(new IdentityHashMap) + private[this] val callbacks: ConcurrentHashMap[Int, Either[Throwable, Long] => Unit] = + new ConcurrentHashMap[Int, Either[Throwable, Long] => Unit]() + + private[this] val usedIds = new java.util.BitSet() - private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): UringSubmissionQueue = { + private[UringSystem] def getSqe(cb: Either[Throwable, Long] => Unit): UringSubmissionQueue = { pendingSubmissions = true + + val id = usedIds.nextClearBit(0) + usedIds.set(id) + callbacks.put(id, cb) + val sqe = ring.ioUringSubmissionQueue() - sqe.setData(cb) // TODO: Implement setData() - callbacks.add(cb) + sqe.setData(cb) + sqe } @@ -132,7 +141,46 @@ object UringSystem extends PollingSystem { private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty() - private[UringSystem] def poll(nanos: Long): Boolean = ??? + private[UringSystem] def poll(nanos: Long): Boolean = { + if (pendingSubmissions) { + ring.submit() + } + + val sqe = ring.ioUringSubmissionQueue() + val cqe = ring.ioUringCompletionQueue() + + val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(res: Int, flags: Int, data: Long): Unit = { + println(s"Completion event flag: $flags") + + val callback = sqe.userData(data).asInstanceOf[Either[IOException, Long] => Unit] + if (res < 0) { + callback(Left(new IOException("Error in completion queue entry"))) + } else { + callback(Right(res.toLong)) + } + } + } + + // Check if there are any completions ready to be processed + if (cqe.hasCompletions()) { + val processedCount = cqe.process(completionQueueCallback) + // Return true if any completion events were processed + processedCount > 0 + } else if (nanos > 0) { + // If a timeout is specified, block until at least one completion is ready + // or the timeout expires + cqe.ioUringWaitCqe() + + // Check again if there are completions after waiting + val processedCount = cqe.process(completionQueueCallback) + // Return true if any completion events were processed + processedCount > 0 + } else { + // No completions and no timeout specified + false + } + } } From 6af5dbcc68e636a649c081c119a97950f8c6af23 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 20 Jun 2023 14:05:55 +0200 Subject: [PATCH 023/200] Add documentation for Ring and Completion Queue --- .../uring/unsafe/UringCompletionQueue.scala | 18 ++++- .../scala/fs2/io/uring/unsafe/UringRing.scala | 70 ++++++++++++++++++- 2 files changed, 84 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala index 1617c6c1..35967476 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala @@ -16,13 +16,21 @@ package io.netty.incubator.channel.uring +/** + * The UringCompletionQueue class represents a completion queue for the io_uring subsystem in the Netty library. + * It provides methods to interact with the completion queue, such as checking for completions, processing completions, + * waiting for completions, and accessing the underlying ring buffer. + * + * @param ring The RingBuffer associated with the completion queue. + */ class UringCompletionQueue(private val ring: RingBuffer) { + // The IOUringCompletionQueue instance associated with the ring. private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() - def hashCompletitions(): Boolean = completionQueue.hasCompletions() + def hasCompletions(): Boolean = completionQueue.hasCompletions() - def processs(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) + def process(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) def ioUringWaitCqe(): Unit = completionQueue.ioUringWaitCqe() @@ -34,5 +42,11 @@ class UringCompletionQueue(private val ring: RingBuffer) { } object UringCompletionQueue { + /** + * Creates a new UringCompletionQueue instance associated with the specified RingBuffer. + * + * @param ring The RingBuffer associated with the completion queue. + * @return A new UringCompletionQueue instance. + */ def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 2c162049..6d3d5932 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -20,31 +20,97 @@ import io.netty.incubator.channel.uring.NativeAccess._ import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringSubmissionQueue +/** + * The UringRing class represents a complete io_uring ring with both submission and completion queues. + * It provides methods to interact with the submission and completion queues, such as submitting operations, + * accessing the file descriptor of the ring, and closing the ring. + * + * @param ringBuffer The RingBuffer associated with the io_uring ring. + */ class UringRing(private val ringBuffer: RingBuffer) { - private val uringCompletionQueue: UringCompletionQueue = UringCompletionQueue(ringBuffer) - private val uringSubmissionQueue: UringSubmissionQueue = UringSubmissionQueue(ringBuffer) + // The completion queue associated with the ring. + private[this] val uringCompletionQueue: UringCompletionQueue = UringCompletionQueue(ringBuffer) + // The submission queue associated with the ring. + private[this] val uringSubmissionQueue: UringSubmissionQueue = UringSubmissionQueue(ringBuffer) + + /** + * Constructs a new UringRing instance with the default ring buffer size. + */ def this() = this(createRingBuffer()) + /** + * Constructs a new UringRing instance with the specified ring buffer size. + * + * @param size of the ring buffer. + */ def this(size: Int) = this(createRingBuffer(size)) + + /** + * Constructs a new UringRing instance with the specified ring buffer size and + * SQE (Submission Queue Entry) async threshold. + * + * @param size of the ring buffer. + * @param sqeAsyncThreshold The threshold value for determining whether an + * SQE should be submitted asynchronously. + */ def this(size: Int, sqeAsyncThreshold: Int) = this(createRingBuffer(size, sqeAsyncThreshold)) + /** + * @return the UringCompletionQueue associated with the ring. + */ def ioUringCompletionQueue(): UringCompletionQueue = uringCompletionQueue + /** + * @return the UringSubmissionQueue associated with the ring. + */ def ioUringSubmissionQueue(): UringSubmissionQueue = uringSubmissionQueue + /** + * Submits pending operations in the queue to the kernel for processing. + * + * @return The number of operations successfully submitted. + */ + def submit(): Int = uringSubmissionQueue.submit() + + /** + * @return The file descriptor of the ring buffer. + */ def fd(): Int = ringBuffer.fd() + /** + * Closes the ring, realising any associated resources. + */ def close(): Unit = ringBuffer.close() } object UringRing { + /** + * Creates a new UringRing instance with the default ring buffer size. + * + * @return a new UringRing instance. + */ def apply(): UringRing = new UringRing() + /** + * Creates a new UringRing instance with the specified ring buffer size. + * + * @param size of the ring buffer. + * @return a new UringRing instance. + */ def apply(size: Int): UringRing = new UringRing(size) + /** + * Creates a new UringRing instance with the specified ring buffer size + * and SQE (Submission Queue Entry) async threshold. + * + * @param size of the ring buffer. + * @param sqeAsyncThreshold The threshold value for determining whether an SQE should be + * submitted asynchronously. + * @return a new UringRing instance. + */ def apply(size: Int, sqeAsyncThreshold: Int): UringRing = new UringRing(size, sqeAsyncThreshold) } From ca5a434c8ce2de7518c9d4ccb9935607a76c2c00 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 20 Jun 2023 14:06:46 +0200 Subject: [PATCH 024/200] Add UringSystemSuite --- .../scala/fs2/io/uring/UringSystemSuite.scala | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala new file mode 100644 index 00000000..6a03c1bd --- /dev/null +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -0,0 +1,57 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring + +import cats.effect.IO +import fs2.io.uring.unsafe.uring._ + +import io.netty.incubator.channel.uring.UringSubmissionQueue + +class UringSystemSuite extends UringSuite { + + test("successful submission") { + val buffer = ByteBuffer.allocate(256) + buffer.put("Hello, Uring!".getBytes) + + var submissionSuccessful = false + + // Perform the operation + val result: IO[Int] = Uring.get[IO].flatMap { ring => + ring.call { submissionQueue => + val fd = 0 // + val bufferAddress = buffer.array() + val pos = 0 + val limit = buffer.remaining() + val extraData: Short = 0 + + if (submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData)) { + submissionQueue.submit() + submissionSuccessful = true + } + } + } + result.unsafeRunSync() + assert(submissionSuccessful) + } + + test("failed submission") {} + + test("polling without completions and no timeout") {} + + test("polling with timeout and completions") {} + +} From 33eeebb4844baf835c573009ae97fb55fea23a7d Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 20 Jun 2023 14:07:17 +0200 Subject: [PATCH 025/200] Add OP for read and write --- .../src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala index f167277e..1f4f0fbb 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala @@ -84,9 +84,15 @@ object NativeAccessEffect { } +/** + * Provides direct access to the native methods and functionalities + * of the io_uring subsystem in Netty. + */ object NativeAccess { val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD + val IORING_OP_POLL_WRITE = Native.IORING_OP_WRITE + val IORING_OP_POLL_READ = Native.IORING_OP_READ val POLLIN = Native.POLLIN val POLLOUT = Native.POLLOUT From 85c86996c285a0492e99000d03b0a9eea0a40065 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 3 Jul 2023 15:02:00 +0200 Subject: [PATCH 026/200] Isolate netty package in one file --- .../fs2/io/uring/unsafe/NativeAccess.scala | 142 --------- .../uring/unsafe/UringCompletionQueue.scala | 52 ---- .../unsafe/UringCompletionQueueCallback.scala | 25 -- .../scala/fs2/io/uring/unsafe/UringRing.scala | 288 ++++++++++++++++-- .../uring/unsafe/UringSubmissionQueue.scala | 158 ---------- 5 files changed, 257 insertions(+), 408 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala deleted file mode 100644 index 1f4f0fbb..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/NativeAccess.scala +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.netty.incubator.channel.uring - -import cats.effect.{Resource, Sync} -import io.netty.channel.unix.FileDescriptor - -object NativeAccessEffect { - - val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE - val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD - - def createRingBuffer[F[_]: Sync](): Resource[F, RingBuffer] = - createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) - - def createRingBuffer[F[_]: Sync](size: Int): Resource[F, RingBuffer] = - createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) - - def createRingBuffer[F[_]: Sync](size: Int, sqeAsyncThreshold: Int): Resource[F, RingBuffer] = - Resource.make { - Sync[F].delay(Native.createRingBuffer(size, sqeAsyncThreshold)) - } { ringBuffer => - Sync[F].delay(ringBuffer.close()) - } - - def checkAllIOSupported[F[_]: Sync](ringFd: Int): F[Unit] = - Sync[F].delay(Native.checkAllIOSupported(ringFd)) - - def checkKernelVersion[F[_]: Sync](kernelVersion: String): F[Unit] = - Sync[F].delay(Native.checkKernelVersion(kernelVersion)) - - def ioUringEnter[F[_]: Sync](ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): F[Int] = - Sync[F].delay(Native.ioUringEnter(ringFd, toSubmit, minComplete, flags)) - - def eventFdWrite[F[_]: Sync](fd: Int, value: Long): F[Unit] = - Sync[F].delay(Native.eventFdWrite(fd, value)) - - def newBlockingEventFd[F[_]: Sync]: F[FileDescriptor] = - Sync[F].delay(Native.newBlockingEventFd()) - - def ioUringExit[F[_]: Sync]( - submissionQueueArrayAddress: Long, - submissionQueueRingEntries: Int, - submissionQueueRingAddress: Long, - submissionQueueRingSize: Int, - completionQueueRingAddress: Long, - completionQueueRingSize: Int, - ringFd: Int - ): F[Unit] = - Sync[F].delay( - Native.ioUringExit( - submissionQueueArrayAddress, - submissionQueueRingEntries, - submissionQueueRingAddress, - submissionQueueRingSize, - completionQueueRingAddress, - completionQueueRingSize, - ringFd - ) - ) - - def createFile[F[_]: Sync](name: String): F[Int] = - Sync[F].delay(Native.createFile(name)) - - def cmsghdrData[F[_]: Sync](hdrAddr: Long): F[Long] = - Sync[F].delay(Native.cmsghdrData(hdrAddr)) - - def kernelVersion[F[_]: Sync]: F[String] = - Sync[F].delay(Native.kernelVersion()) - -} - -/** - * Provides direct access to the native methods and functionalities - * of the io_uring subsystem in Netty. - */ -object NativeAccess { - val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE - val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD - val IORING_OP_POLL_WRITE = Native.IORING_OP_WRITE - val IORING_OP_POLL_READ = Native.IORING_OP_READ - - val POLLIN = Native.POLLIN - val POLLOUT = Native.POLLOUT - - def createRingBuffer(): RingBuffer = - createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) - - def createRingBuffer(size: Int): RingBuffer = - createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) - - def createRingBuffer(size: Int, sqeAsyncThreshold: Int): RingBuffer = - Native.createRingBuffer(size, sqeAsyncThreshold) - - def checkAllIOSupported(ringFd: Int): Unit = - Native.checkAllIOSupported(ringFd) - - def checkKernelVersion(kernelVersion: String): Unit = - Native.checkKernelVersion(kernelVersion) - - def ioUringEnter(ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): Int = - Native.ioUringEnter(ringFd, toSubmit, minComplete, flags) - - def eventFdWrite(fd: Int, value: Long): Unit = - Native.eventFdWrite(fd, value) - - def newBlockingEventFd: FileDescriptor = - Native.newBlockingEventFd() - - def ioUringExit( - submissionQueueArrayAddress: Long, - submissionQueueRingEntries: Int, - submissionQueueRingAddress: Long, - submissionQueueRingSize: Int, - completionQueueRingAddress: Long, - completionQueueRingSize: Int, - ringFd: Int - ): Unit = - Native.ioUringExit( - submissionQueueArrayAddress, - submissionQueueRingEntries, - submissionQueueRingAddress, - submissionQueueRingSize, - completionQueueRingAddress, - completionQueueRingSize, - ringFd - ) -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala deleted file mode 100644 index 35967476..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueue.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.netty.incubator.channel.uring - -/** - * The UringCompletionQueue class represents a completion queue for the io_uring subsystem in the Netty library. - * It provides methods to interact with the completion queue, such as checking for completions, processing completions, - * waiting for completions, and accessing the underlying ring buffer. - * - * @param ring The RingBuffer associated with the completion queue. - */ -class UringCompletionQueue(private val ring: RingBuffer) { - - // The IOUringCompletionQueue instance associated with the ring. - private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() - - def hasCompletions(): Boolean = completionQueue.hasCompletions() - - def process(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) - - def ioUringWaitCqe(): Unit = completionQueue.ioUringWaitCqe() - - def ringAddress(): Long = completionQueue.ringAddress - - def ringFd(): Int = completionQueue.ringFd - - def ringSize(): Int = completionQueue.ringSize -} - -object UringCompletionQueue { - /** - * Creates a new UringCompletionQueue instance associated with the specified RingBuffer. - * - * @param ring The RingBuffer associated with the completion queue. - * @return A new UringCompletionQueue instance. - */ - def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala deleted file mode 100644 index bbd4c322..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringCompletionQueueCallback.scala +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.netty.incubator.channel.uring - -/** The UringCompletionQueueCallback trait defines a callback interface for handling completion events - * from the io_uring completion queue. It extends the IOUringCompletionQueueCallback trait and provides - * a method handle to process the completion event. - */ -trait UringCompletionQueueCallback extends IOUringCompletionQueueCallback { - def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 6d3d5932..b97ad7e6 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -16,15 +16,13 @@ package io.netty.incubator.channel.uring -import io.netty.incubator.channel.uring.NativeAccess._ -import io.netty.incubator.channel.uring.UringCompletionQueue -import io.netty.incubator.channel.uring.UringSubmissionQueue +import io.netty.channel.unix.FileDescriptor +import NativeAccess._ -/** - * The UringRing class represents a complete io_uring ring with both submission and completion queues. +/** The UringRing class represents a complete io_uring ring with both submission and completion queues. * It provides methods to interact with the submission and completion queues, such as submitting operations, * accessing the file descriptor of the ring, and closing the ring. - * + * * @param ringBuffer The RingBuffer associated with the io_uring ring. */ class UringRing(private val ringBuffer: RingBuffer) { @@ -34,83 +32,311 @@ class UringRing(private val ringBuffer: RingBuffer) { // The submission queue associated with the ring. private[this] val uringSubmissionQueue: UringSubmissionQueue = UringSubmissionQueue(ringBuffer) - /** - * Constructs a new UringRing instance with the default ring buffer size. + /** Constructs a new UringRing instance with the default ring buffer size. */ def this() = this(createRingBuffer()) - /** - * Constructs a new UringRing instance with the specified ring buffer size. + /** Constructs a new UringRing instance with the specified ring buffer size. * * @param size of the ring buffer. */ def this(size: Int) = this(createRingBuffer(size)) - - /** - * Constructs a new UringRing instance with the specified ring buffer size and + /** Constructs a new UringRing instance with the specified ring buffer size and * SQE (Submission Queue Entry) async threshold. * * @param size of the ring buffer. - * @param sqeAsyncThreshold The threshold value for determining whether an + * @param sqeAsyncThreshold The threshold value for determining whether an * SQE should be submitted asynchronously. */ def this(size: Int, sqeAsyncThreshold: Int) = this(createRingBuffer(size, sqeAsyncThreshold)) - /** - * @return the UringCompletionQueue associated with the ring. + /** @return the UringCompletionQueue associated with the ring. */ def ioUringCompletionQueue(): UringCompletionQueue = uringCompletionQueue - /** - * @return the UringSubmissionQueue associated with the ring. + /** @return the UringSubmissionQueue associated with the ring. */ def ioUringSubmissionQueue(): UringSubmissionQueue = uringSubmissionQueue - /** - * Submits pending operations in the queue to the kernel for processing. + /** Submits pending operations in the queue to the kernel for processing. * * @return The number of operations successfully submitted. */ def submit(): Int = uringSubmissionQueue.submit() - /** - * @return The file descriptor of the ring buffer. + /** @return The file descriptor of the ring buffer. */ def fd(): Int = ringBuffer.fd() - /** - * Closes the ring, realising any associated resources. + /** Closes the ring, realising any associated resources. */ def close(): Unit = ringBuffer.close() } object UringRing { - /** - * Creates a new UringRing instance with the default ring buffer size. + + /** Creates a new UringRing instance with the default ring buffer size. * * @return a new UringRing instance. */ def apply(): UringRing = new UringRing() - /** - * Creates a new UringRing instance with the specified ring buffer size. + /** Creates a new UringRing instance with the specified ring buffer size. * * @param size of the ring buffer. * @return a new UringRing instance. */ def apply(size: Int): UringRing = new UringRing(size) - /** - * Creates a new UringRing instance with the specified ring buffer size + /** Creates a new UringRing instance with the specified ring buffer size * and SQE (Submission Queue Entry) async threshold. * * @param size of the ring buffer. * @param sqeAsyncThreshold The threshold value for determining whether an SQE should be * submitted asynchronously. - * @return a new UringRing instance. + * @return a new UringRing instance. */ def apply(size: Int, sqeAsyncThreshold: Int): UringRing = new UringRing(size, sqeAsyncThreshold) } + +class UringSubmissionQueue(private val ring: RingBuffer) { + import UringSubmissionQueue._ + + private[this] val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() + + private[this] val callbacks = + scala.collection.mutable.Map[Long, Either[Throwable, Long] => Unit]() + private[this] var id: Short = 0 + + def enqueueSqe( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long, + data: Short + ): Boolean = + submissionQueue.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + + def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() + + def decrementHandledFds(): Unit = submissionQueue.decrementHandledFds() + + def addTimeout(nanoSeconds: Long, extraData: Short): Boolean = + submissionQueue.addTimeout(nanoSeconds, extraData) + + def addPollIn(fd: Int): Boolean = submissionQueue.addPollIn(fd) + + def addPollRdHup(fd: Int): Boolean = submissionQueue.addPollRdHup(fd) + + def addPollOut(fd: Int): Boolean = submissionQueue.addPollOut(fd) + + def addRecvmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = + submissionQueue.addRecvmsg(fd, msgHdr, extraData) + + def addSendmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = + submissionQueue.addSendmsg(fd, msgHdr, extraData) + + def addSendmsg(fd: Int, msgHdr: Long, flags: Int, extraData: Short): Boolean = + submissionQueue.addSendmsg(fd, msgHdr, flags, extraData) + + def addRead(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addRead(fd, bufferAddress, pos, limit, extraData) + + def addEventFdRead( + fd: Int, + bufferAddress: Long, + pos: Int, + limit: Int, + extraData: Short + ): Boolean = submissionQueue.addEventFdRead(fd, bufferAddress, pos, limit, extraData) + + def addWrite(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData) + + def addRecv(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addRecv(fd, bufferAddress, pos, limit, extraData) + + def addSend(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = + submissionQueue.addSend(fd, bufferAddress, pos, limit, extraData) + + def addAccept(fd: Int, address: Long, addressLength: Long, extraData: Short): Boolean = + submissionQueue.addAccept(fd, address, addressLength, extraData) + + def addPollRemove(fd: Int, pollMask: Int): Boolean = submissionQueue.addPollRemove(fd, pollMask) + + def addConnect( + fd: Int, + socketAddress: Long, + socketAddressLength: Long, + extraData: Short + ): Boolean = submissionQueue.addConnect(fd, socketAddress, socketAddressLength, extraData) + + def addWritev(fd: Int, iovecArrayAddress: Long, length: Int, extraData: Short): Boolean = + submissionQueue.addWritev(fd, iovecArrayAddress, length, extraData) + + def addClose(fd: Int, extraData: Short): Boolean = submissionQueue.addClose(fd, extraData) + + def submit(): Int = submissionQueue.submit() + + def submitAndWait(): Int = submissionQueue.submitAndWait() + + def count(): Long = submissionQueue.count() + + def release(): Unit = submissionQueue.release() + + def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) + + def setData(cb: Either[Throwable, Long] => Unit): Boolean = { + val op: Byte = IORING_OP_POLL_WRITE + val flags: Int = 0 + val rwFlags: Int = Native.POLLOUT + val fd: Int = 0 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + val wasEnqueue: Boolean = enqueueSqe( + op, + flags, + rwFlags, + fd, + bufferAddress, + length, + offset, + id + ) + + callbacks += (encode(0, 0, id) -> cb) + id = (id + 1).toShort + + wasEnqueue + } + + def getData(): Short = { + (id - 1).toShort + } + + def removeCallback(data: Short): Option[Either[Throwable,Long] => Unit] = { + callbacks.remove(encode(0, 0, data)) + } + + def callbacksIsEmpty(): Boolean = callbacks.isEmpty + + def prepCancel(addr: Long, flags: Int): Boolean = + enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) + +} + +object UringSubmissionQueue { + final val SQE_SIZE = 64 + + final val IORING_OP_ASYNC_CANCEL: Byte = 14.toByte + + final val SQE_USER_DATA_FIELD = 32 + + def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) +} + +/** The UringCompletionQueue class represents a completion queue for the io_uring subsystem in the Netty library. + * It provides methods to interact with the completion queue, such as checking for completions, processing completions, + * waiting for completions, and accessing the underlying ring buffer. + * + * @param ring The RingBuffer associated with the completion queue. + */ +class UringCompletionQueue(private val ring: RingBuffer) { + + // The IOUringCompletionQueue instance associated with the ring. + private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() + + def hasCompletions(): Boolean = completionQueue.hasCompletions() + + def process(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) + + def ioUringWaitCqe(): Unit = completionQueue.ioUringWaitCqe() + + def ringAddress(): Long = completionQueue.ringAddress + + def ringFd(): Int = completionQueue.ringFd + + def ringSize(): Int = completionQueue.ringSize +} + +object UringCompletionQueue { + + /** Creates a new UringCompletionQueue instance associated with the specified RingBuffer. + * + * @param ring The RingBuffer associated with the completion queue. + * @return A new UringCompletionQueue instance. + */ + def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) +} + +/** The UringCompletionQueueCallback trait defines a callback interface for handling completion events + * from the io_uring completion queue. It extends the IOUringCompletionQueueCallback trait and provides + * a method handle to process the completion event. + */ +trait UringCompletionQueueCallback extends IOUringCompletionQueueCallback { + def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit +} + +/** Provides direct access to the native methods and functionalities + * of the io_uring subsystem in Netty. + */ +object NativeAccess { + val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE + val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD + val IORING_OP_POLL_WRITE = Native.IORING_OP_WRITE + val IORING_OP_POLL_READ = Native.IORING_OP_READ + + val POLLIN = Native.POLLIN + val POLLOUT = Native.POLLOUT + + def createRingBuffer(): RingBuffer = + createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer(size: Int): RingBuffer = + createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + + def createRingBuffer(size: Int, sqeAsyncThreshold: Int): RingBuffer = + Native.createRingBuffer(size, sqeAsyncThreshold) + + def checkAllIOSupported(ringFd: Int): Unit = + Native.checkAllIOSupported(ringFd) + + def checkKernelVersion(kernelVersion: String): Unit = + Native.checkKernelVersion(kernelVersion) + + def ioUringEnter(ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): Int = + Native.ioUringEnter(ringFd, toSubmit, minComplete, flags) + + def eventFdWrite(fd: Int, value: Long): Unit = + Native.eventFdWrite(fd, value) + + def newBlockingEventFd: FileDescriptor = + Native.newBlockingEventFd() + + def ioUringExit( + submissionQueueArrayAddress: Long, + submissionQueueRingEntries: Int, + submissionQueueRingAddress: Long, + submissionQueueRingSize: Int, + completionQueueRingAddress: Long, + completionQueueRingSize: Int, + ringFd: Int + ): Unit = + Native.ioUringExit( + submissionQueueArrayAddress, + submissionQueueRingEntries, + submissionQueueRingAddress, + submissionQueueRingSize, + completionQueueRingAddress, + completionQueueRingSize, + ringFd + ) +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala deleted file mode 100644 index e95082aa..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSubmissionQueue.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.netty.incubator.channel.uring - -import UringSubmissionQueue._ -import NativeAccess._ -import scala.collection.mutable.LongMap -import java.io.IOException - -class UringSubmissionQueue(private val ring: RingBuffer) { - private val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() - - def enqueueSqe( - op: Byte, - flags: Int, - rwFlags: Int, - fd: Int, - bufferAddress: Long, - length: Int, - offset: Long, - data: Short - ): Boolean = - submissionQueue.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) - - def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() - - def decrementHandledFds(): Unit = submissionQueue.decrementHandledFds() - - def addTimeout(nanoSeconds: Long, extraData: Short): Boolean = - submissionQueue.addTimeout(nanoSeconds, extraData) - - def addPollIn(fd: Int): Boolean = submissionQueue.addPollIn(fd) - - def addPollRdHup(fd: Int): Boolean = submissionQueue.addPollRdHup(fd) - - def addPollOut(fd: Int): Boolean = submissionQueue.addPollOut(fd) - - def addRecvmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = - submissionQueue.addRecvmsg(fd, msgHdr, extraData) - - def addSendmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = - submissionQueue.addSendmsg(fd, msgHdr, extraData) - - def addSendmsg(fd: Int, msgHdr: Long, flags: Int, extraData: Short): Boolean = - submissionQueue.addSendmsg(fd, msgHdr, flags, extraData) - - def addRead(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = - submissionQueue.addRead(fd, bufferAddress, pos, limit, extraData) - - def addEventFdRead( - fd: Int, - bufferAddress: Long, - pos: Int, - limit: Int, - extraData: Short - ): Boolean = submissionQueue.addEventFdRead(fd, bufferAddress, pos, limit, extraData) - - def addWrite(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = - submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData) - - def addRecv(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = - submissionQueue.addRecv(fd, bufferAddress, pos, limit, extraData) - - def addSend(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = - submissionQueue.addSend(fd, bufferAddress, pos, limit, extraData) - - def addAccept(fd: Int, address: Long, addressLength: Long, extraData: Short): Boolean = - submissionQueue.addAccept(fd, address, addressLength, extraData) - - def addPollRemove(fd: Int, pollMask: Int): Boolean = submissionQueue.addPollRemove(fd, pollMask) - - def addConnect( - fd: Int, - socketAddress: Long, - socketAddressLength: Long, - extraData: Short - ): Boolean = submissionQueue.addConnect(fd, socketAddress, socketAddressLength, extraData) - - def addWritev(fd: Int, iovecArrayAddress: Long, length: Int, extraData: Short): Boolean = - submissionQueue.addWritev(fd, iovecArrayAddress, length, extraData) - - def addClose(fd: Int, extraData: Short): Boolean = submissionQueue.addClose(fd, extraData) - - def submit(): Int = submissionQueue.submit() - - def submitAndWait(): Int = submissionQueue.submitAndWait() - - def count(): Long = submissionQueue.count() - - def release(): Unit = submissionQueue.release() - - def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) - - private[this] val callbacks = new LongMap[Either[Throwable, Long] => Unit]() - var counter: Short = 0 - - def setData(cb: Either[Throwable, Long] => Unit): Boolean = { - val op: Byte = IORING_OP_POLL_WRITE - val flags: Int = 0 - val rwFlags: Int = Native.POLLOUT - val fd: Int = 0 - val bufferAddress: Long = 0 - val length: Int = 0 - val offset: Long = 0 - - counter = (counter + 1).toShort - - val success: Boolean = enqueueSqe( - op, - flags, - rwFlags, - fd, - bufferAddress, - length, - offset, - counter - ) - if (success) callbacks.update(encode(fd, op, counter), cb) - else - cb(Left(new IOException("Failed to enqueue"))) - - success - } - - def userData(data: Long): Either[Throwable, Long] => Unit = - callbacks.getOrElse( - data, - throw new NoSuchElementException(s"Callback not found for data: $data") - ) - - def prepCancel(addr: Long, flags: Int): Boolean = - enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) - -} - -object UringSubmissionQueue { - final val SQE_SIZE = 64 - - final val IORING_OP_ASYNC_CANCEL: Byte = 14.toByte - - final val SQE_USER_DATA_FIELD = 32 - - def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) -} From 8445b0147daee57f38240880e0e246ed0ee565d1 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 3 Jul 2023 15:03:34 +0200 Subject: [PATCH 027/200] Update Callbacks Map --- .../src/main/scala/fs2/io/uring/Uring.scala | 4 +- .../fs2/io/uring/unsafe/UringSystem.scala | 128 ++++++++---------- 2 files changed, 58 insertions(+), 74 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index ec14eea0..27c9db80 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -25,9 +25,9 @@ import io.netty.incubator.channel.uring.UringSubmissionQueue abstract class Uring private[uring] { - def call(prep: UringSubmissionQueue => Unit): IO[Int] + def call(prep: UringSubmissionQueue => Unit): IO[Long] - def bracket(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): Resource[IO, Int] + def bracket(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): Resource[IO, Long] } object Uring { diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 24d0f834..1d29e856 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -30,9 +30,9 @@ import cats.effect.unsafe.PollingSystem import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue +import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringCompletionQueueCallback -import java.util.concurrent.ConcurrentHashMap import java.io.IOException object UringSystem extends PollingSystem { @@ -41,12 +41,13 @@ object UringSystem extends PollingSystem { type Api = Uring - type Address = Long - override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) - override def makePoller(): Poller = - new Poller(UringRing()) + override def makePoller(): Poller = { + val ring = UringRing() + + new Poller(ring) + } override def closePoller(poller: Poller): Unit = poller.close() @@ -58,58 +59,55 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { - private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit + private[this] val noopRelease: Long => IO[Unit] = _ => IO.unit - def call(prep: UringSubmissionQueue => Unit): IO[Int] = + def call(prep: UringSubmissionQueue => Unit): IO[Long] = exec(prep)(noopRelease) - def bracket(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): Resource[IO, Int] = - Resource.makeFull[IO, Int](poll => poll(exec(prep)(release(_))))(release(_)) + def bracket(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): Resource[IO, Long] = + Resource.makeFull[IO, Long](poll => poll(exec(prep)(release(_))))(release(_)) - private def exec(prep: UringSubmissionQueue => Unit)(release: Int => IO[Unit]): IO[Int] = + private def exec(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): IO[Long] = IO.cont { - new Cont[IO, Int, Int] { + new Cont[IO, Long, Long] { def apply[F[_]](implicit F: MonadCancelThrow[F] - ): (Either[Throwable, Long] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => - F.uncancelable { poll => - val submit = IO.async_[Long] { cb => - register { ring => - val sqe = ring.getSqe(resume) - prep(sqe) - - sqe.setData(cb) - - val userData = sqe.encode(0, 0, sqe.counter) - - cb(Right(userData)) + ): (Either[Throwable, Long] => Unit, F[Long], IO ~> F) => F[Long] = { + (resume, get, lift) => + F.uncancelable { poll => + val submit = IO.async_[Long] { cb => + register { ring => + val sqe = ring.getSqe(resume) + prep(sqe) + val userData: Long = sqe.encode(0, 0, sqe.getData()) + cb(Right(userData)) + } } - } - lift(submit) - .flatMap { addr => - F.onCancel( - poll(get), - lift(cancel(addr)).ifM( - F.unit, - get.flatMap { rtn => - if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) - else lift(release(rtn)) - } + lift(submit) + .flatMap { addr => + F.onCancel( + poll(get), + lift(cancel(addr)).ifM( + F.unit, + get.flatMap { rtn => + if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn.toInt)) + else lift(release(rtn)) + } + ) ) - ) - } - .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) - } + } + .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e.toInt))) + } } } } - private[this] def cancel(addr: Address): IO[Boolean] = + private[this] def cancel(addr: Long): IO[Boolean] = IO.async_[Long] { cb => register { ring => val sqe = ring.getSqe(cb) - val wasCancelled = sqe.prepCancel(addr, 0) + val wasCancelled: Boolean = sqe.prepCancel(addr, 0) cb(Right(if (wasCancelled) 1 else 0)) } }.map(_ == 1) @@ -119,63 +117,49 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { private[this] var pendingSubmissions: Boolean = false - private[this] val callbacks: ConcurrentHashMap[Int, Either[Throwable, Long] => Unit] = - new ConcurrentHashMap[Int, Either[Throwable, Long] => Unit]() - - private[this] val usedIds = new java.util.BitSet() + private[this] val sqe: UringSubmissionQueue = ring.ioUringSubmissionQueue() + private[this] val cqe: UringCompletionQueue = ring.ioUringCompletionQueue() private[UringSystem] def getSqe(cb: Either[Throwable, Long] => Unit): UringSubmissionQueue = { pendingSubmissions = true - - val id = usedIds.nextClearBit(0) - usedIds.set(id) - callbacks.put(id, cb) - - val sqe = ring.ioUringSubmissionQueue() sqe.setData(cb) - sqe } private[UringSystem] def close(): Unit = ring.close() - private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty() + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !sqe.callbacksIsEmpty() + + private[UringSystem] def process( + completionQueueCallback: UringCompletionQueueCallback + ): Boolean = + cqe.process(completionQueueCallback) > 0 // True if any completion events were processed private[UringSystem] def poll(nanos: Long): Boolean = { if (pendingSubmissions) { ring.submit() + pendingSubmissions = false } - val sqe = ring.ioUringSubmissionQueue() - val cqe = ring.ioUringCompletionQueue() - val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(res: Int, flags: Int, data: Long): Unit = { - println(s"Completion event flag: $flags") - - val callback = sqe.userData(data).asInstanceOf[Either[IOException, Long] => Unit] - if (res < 0) { - callback(Left(new IOException("Error in completion queue entry"))) - } else { - callback(Right(res.toLong)) + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { + val removedCallback = sqe.removeCallback(data) + + removedCallback.foreach { cb => + if (res < 0) cb(Left(new IOException("Error in completion queue entry"))) + else cb(Right(res.toLong)) } } } - // Check if there are any completions ready to be processed if (cqe.hasCompletions()) { - val processedCount = cqe.process(completionQueueCallback) - // Return true if any completion events were processed - processedCount > 0 + process(completionQueueCallback) } else if (nanos > 0) { - // If a timeout is specified, block until at least one completion is ready - // or the timeout expires + // TODO sqe.addTimeout() and then: cqe.ioUringWaitCqe() // Check again if there are completions after waiting - val processedCount = cqe.process(completionQueueCallback) - // Return true if any completion events were processed - processedCount > 0 + process(completionQueueCallback) } else { // No completions and no timeout specified false From 21ca1e770fd345d25acf37c668a24794eadcf73e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 4 Jul 2023 15:42:07 +0200 Subject: [PATCH 028/200] Create IORuntime with Uring polling System --- .../src/test/scala/fs2/io/uring/UringSuite.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 69bb2cba..0d5dfb69 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -16,10 +16,19 @@ package fs2.io.uring -import cats.effect.unsafe.IORuntimes import munit.CatsEffectSuite +import fs2.io.uring.unsafe.UringSystem +import cats.effect.unsafe.IORuntimeBuilder + abstract class UringSuite extends CatsEffectSuite { - override lazy val munitIORuntime = IORuntime.global + override lazy val munitIORuntime = { + val builder = IORuntimeBuilder() + + builder.setPollingSystem(UringSystem) + // We can set other components as well: Compute Execution Context, Scheduler, Config, ... + + builder.build() + } } From 4d432c40e779d64007d1b0601de1c326282f018f Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 4 Jul 2023 15:42:56 +0200 Subject: [PATCH 029/200] Update the poll processing and timeout --- .../main/scala/fs2/io/uring/unsafe/UringSystem.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 1d29e856..b440ce59 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -155,14 +155,12 @@ object UringSystem extends PollingSystem { if (cqe.hasCompletions()) { process(completionQueueCallback) } else if (nanos > 0) { - // TODO sqe.addTimeout() and then: + sqe.addTimeout(nanos, 0) + ring.submit() cqe.ioUringWaitCqe() - - // Check again if there are completions after waiting - process(completionQueueCallback) + process(completionQueueCallback) } else { - // No completions and no timeout specified - false + false } } From 50514ee6694ccfa9e49ca7ad91150946eace860a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 5 Jul 2023 13:55:28 +0200 Subject: [PATCH 030/200] Add submission test --- .../scala/fs2/io/uring/UringSystemSuite.scala | 42 +++++++------------ 1 file changed, 15 insertions(+), 27 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 6a03c1bd..6cd45bc5 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -17,37 +17,25 @@ package fs2.io.uring import cats.effect.IO -import fs2.io.uring.unsafe.uring._ - -import io.netty.incubator.channel.uring.UringSubmissionQueue - -class UringSystemSuite extends UringSuite { - - test("successful submission") { - val buffer = ByteBuffer.allocate(256) - buffer.put("Hello, Uring!".getBytes) - - var submissionSuccessful = false - - // Perform the operation - val result: IO[Int] = Uring.get[IO].flatMap { ring => - ring.call { submissionQueue => - val fd = 0 // - val bufferAddress = buffer.array() - val pos = 0 - val limit = buffer.remaining() - val extraData: Short = 0 - - if (submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData)) { - submissionQueue.submit() - submissionSuccessful = true + +import fs2.io.uring.UringSuite + +class UringSystemSuit extends UringSuite { + + test("submission") { + Uring + .get[IO] + .flatMap { ring => + ring.call { sqe => + sqe.submit() + () } } - } - result.unsafeRunSync() - assert(submissionSuccessful) + .assertEquals(0.toLong) } + test("successful submission") {} + test("failed submission") {} test("polling without completions and no timeout") {} From 118ad86f2634af656eae856864cb09a4b249b29b Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 10 Jul 2023 21:19:43 +0200 Subject: [PATCH 031/200] Reimplement UringSystem using netty api --- .../src/main/scala/fs2/io/uring/Uring.scala | 24 +- .../scala/fs2/io/uring/net/UringNetwork.scala | 61 +++++ .../scala/fs2/io/uring/net/UringSocket.scala | 21 ++ .../fs2/io/uring/unsafe/UringSystem.scala | 217 +++++++++++------- .../test/scala/fs2/io/uring/UringSuite.scala | 3 +- 5 files changed, 235 insertions(+), 91 deletions(-) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index 27c9db80..2bb3705e 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -21,13 +21,27 @@ import cats.effect.LiftIO import cats.effect.kernel.Resource import cats.syntax.all._ -import io.netty.incubator.channel.uring.UringSubmissionQueue - abstract class Uring private[uring] { - def call(prep: UringSubmissionQueue => Unit): IO[Long] - - def bracket(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): Resource[IO, Long] + def call( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long + ): IO[Int] + + def bracket( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long + )(release: Int => IO[Unit]): Resource[IO, Int] } object Uring { diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala new file mode 100644 index 00000000..c3cb8748 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -0,0 +1,61 @@ +// package fs2.io.uring.net + +// import fs2.io.net.Network +// import fs2.io.net.tls.TLSContext +// import cats.effect.kernel.Resource +// import com.comcast.ip4s.{Host, Port} +// import fs2.io.net.{DatagramSocket, DatagramSocketGroup} +// import fs2.io.net.DatagramSocketOption +// import com.comcast.ip4s.{Host, Port} +// import fs2.io.net.{Socket, SocketOption} +// import cats.effect.kernel.Resource +// import com.comcast.ip4s.{Host, IpAddress, Port, SocketAddress} +// import fs2.io.net.{Socket, SocketOption} +// import cats.effect.kernel.Resource +// import fs2.io.net.SocketGroup +// import java.util.concurrent.ThreadFactory +// import cats.effect.kernel.Resource +// import fs2.io.net.DatagramSocketGroup +// import java.util.concurrent.ThreadFactory +// import cats.effect.kernel.Resource +// import com.comcast.ip4s.{Host, SocketAddress} +// import fs2.io.net.{Socket, SocketOption} + +// private[net] final class UringNetwork[F[_]]() extends Network.UnsealedNetwork[F] { + +// override def socketGroup( +// threadCount: Int, +// threadFactory: ThreadFactory +// ): Resource[F, SocketGroup[F]] = ??? + +// override def datagramSocketGroup( +// threadFactory: ThreadFactory +// ): Resource[F, DatagramSocketGroup[F]] = ??? + +// override def client( +// to: SocketAddress[Host], +// options: List[SocketOption] +// ): Resource[F, Socket[F]] = ??? + +// override def server( +// address: Option[Host], +// port: Option[Port], +// options: List[SocketOption] +// ): fs2.Stream[F, Socket[F]] = ??? + +// override def serverResource( +// address: Option[Host], +// port: Option[Port], +// options: List[SocketOption] +// ): Resource[F, (SocketAddress[IpAddress], fs2.Stream[F, Socket[F]])] = ??? + +// override def openDatagramSocket( +// address: Option[Host], +// port: Option[Port], +// options: List[DatagramSocketOption], +// protocolFamily: Option[DatagramSocketGroup.ProtocolFamily] +// ): Resource[F, DatagramSocket[F]] = ??? + +// override def tlsContext: TLSContext.Builder[F] = ??? + +// } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala new file mode 100644 index 00000000..110df2b3 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -0,0 +1,21 @@ +// package fs2.io.uring.net + +// import cats.effect.LiftIO +// import cats.effect.std.Mutex +// import cats.effect.kernel.Async + +// import fs2.io.uring.Uring + +// import com.comcast.ip4s.SocketAddress +// import com.comcast.ip4s.IpAddress +// import fs2.io.net.Socket + +// private[net] final class UringSocket[F[_]: LiftIO]( +// ring: Uring, +// fd: Int, +// remoteAddress: SocketAddress[IpAddress], +// buffer: Array[F[_]], +// readMutex: Mutex[F], +// writeMutex: Mutex[F] +// )(implicit F: Async[F]) +// extends Socket[F] {} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index b440ce59..ef5c6ab6 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -35,6 +35,9 @@ import io.netty.incubator.channel.uring.UringCompletionQueueCallback import java.io.IOException +import scala.collection.mutable.Map +import scala.collection.mutable.BitSet + object UringSystem extends PollingSystem { private final val MaxEvents = 64 @@ -43,15 +46,16 @@ object UringSystem extends PollingSystem { override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) - override def makePoller(): Poller = { - val ring = UringRing() - - new Poller(ring) - } + override def makePoller(): Poller = + new Poller(UringRing()) override def closePoller(poller: Poller): Unit = poller.close() - override def poll(poller: Poller, nanos: Long, reportFailure: Throwable => Unit): Boolean = + override def poll( + poller: Poller, + nanos: Long, + reportFailure: Throwable => Unit + ): Boolean = poller.poll(nanos) override def needsPoll(poller: Poller): Boolean = poller.needsPoll() @@ -59,109 +63,154 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { - private[this] val noopRelease: Long => IO[Unit] = _ => IO.unit - - def call(prep: UringSubmissionQueue => Unit): IO[Long] = - exec(prep)(noopRelease) - - def bracket(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): Resource[IO, Long] = - Resource.makeFull[IO, Long](poll => poll(exec(prep)(release(_))))(release(_)) - - private def exec(prep: UringSubmissionQueue => Unit)(release: Long => IO[Unit]): IO[Long] = - IO.cont { - new Cont[IO, Long, Long] { - def apply[F[_]](implicit - F: MonadCancelThrow[F] - ): (Either[Throwable, Long] => Unit, F[Long], IO ~> F) => F[Long] = { - (resume, get, lift) => - F.uncancelable { poll => - val submit = IO.async_[Long] { cb => - register { ring => - val sqe = ring.getSqe(resume) - prep(sqe) - val userData: Long = sqe.encode(0, 0, sqe.getData()) - cb(Right(userData)) - } - } - - lift(submit) - .flatMap { addr => - F.onCancel( - poll(get), - lift(cancel(addr)).ifM( - F.unit, - get.flatMap { rtn => - if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn.toInt)) - else lift(release(rtn)) - } - ) - ) - } - .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e.toInt))) + private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit + + def call( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long + ): IO[Int] = + exec(op, flags, rwFlags, fd, bufferAddress, length, offset)(noopRelease) + + def bracket( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long + )(release: Int => IO[Unit]): Resource[IO, Int] = + Resource.makeFull[IO, Int](poll => + poll(exec(op, flags, rwFlags, fd, bufferAddress, length, offset)(release(_))) + )(release) + + private def exec( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long + )(release: Int => IO[Unit]): IO[Int] = IO.cont { + new Cont[IO, Int, Int] { + def apply[F[_]](implicit + F: MonadCancelThrow[F] + ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => + F.uncancelable { poll => + val submit: IO[Short] = IO.async_[Short] { cb => + register { ring => + val id = ring.getSqe(resume) + val sq: UringSubmissionQueue = ring.getSq() + sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) + sq.submit() + cb(Right(id)) } + } + + lift(submit) + .flatMap { id => + F.onCancel( + poll(get), + lift(cancel(id)).ifM( + F.unit, + // if cannot cancel, fallback to get + get.flatMap { rtn => + if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) + else lift(release(rtn)) + } + ) + ) + } + .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) } + } } + } - private[this] def cancel(addr: Long): IO[Boolean] = - IO.async_[Long] { cb => + private[this] def cancel(id: Short): IO[Boolean] = + IO.async_[Boolean] { cb => register { ring => - val sqe = ring.getSqe(cb) - val wasCancelled: Boolean = sqe.prepCancel(addr, 0) - cb(Right(if (wasCancelled) 1 else 0)) + val wasCancel = ring.removeCallback(id) + cb(Right(wasCancel)) } - }.map(_ == 1) - + } } final class Poller private[UringSystem] (ring: UringRing) { + private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() + private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() + private[this] var pendingSubmissions: Boolean = false - private[this] val sqe: UringSubmissionQueue = ring.ioUringSubmissionQueue() - private[this] val cqe: UringCompletionQueue = ring.ioUringCompletionQueue() + private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = + Map.empty[Short, Either[Throwable, Int] => Unit] + private[this] val ids = BitSet(Short.MaxValue + 1) + private[this] var lastUsedId: Int = -1 + + private[this] def getUniqueId(): Short = { + val id = (lastUsedId + 1 until Short.MaxValue) + .find(!ids.contains(_)) + .getOrElse( + (0 until lastUsedId) + .find(!ids.contains(_)) + .getOrElse( + throw new RuntimeException("No available IDs") + ) + ) + lastUsedId = id + ids.add(id) + id.toShort + } + + private[UringSystem] def releaseId(id: Short): Boolean = ids.remove(id.toInt) - private[UringSystem] def getSqe(cb: Either[Throwable, Long] => Unit): UringSubmissionQueue = { + private[UringSystem] def removeCallback(id: Short): Boolean = { + val removed = callbacks.remove(id).isDefined + if (removed) releaseId(id) + removed + } + + private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): Short = { pendingSubmissions = true - sqe.setData(cb) - sqe + val id: Short = getUniqueId() + if (sq.setData(id)) { + callbacks.put(id, cb) + } + id } - private[UringSystem] def close(): Unit = ring.close() + private[UringSystem] def getSq(): UringSubmissionQueue = sq - private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !sqe.callbacksIsEmpty() + private[UringSystem] def close(): Unit = ring.close() - private[UringSystem] def process( - completionQueueCallback: UringCompletionQueueCallback - ): Boolean = - cqe.process(completionQueueCallback) > 0 // True if any completion events were processed + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty private[UringSystem] def poll(nanos: Long): Boolean = { - if (pendingSubmissions) { - ring.submit() - pendingSubmissions = false - } - val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - val removedCallback = sqe.removeCallback(data) + def processCqes( + completionQueueCallback: UringCompletionQueueCallback + ): Boolean = + cq.process(completionQueueCallback) > 0 // True if any completion events were processed - removedCallback.foreach { cb => - if (res < 0) cb(Left(new IOException("Error in completion queue entry"))) - else cb(Right(res.toLong)) + val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = + callbacks.get(data).foreach { cb => + if (res < 0) { + cb(Left(new IOException(s"Error in completion queue entry"))) + } else { + cb(Right(res)) + } } - } } - if (cqe.hasCompletions()) { - process(completionQueueCallback) - } else if (nanos > 0) { - sqe.addTimeout(nanos, 0) - ring.submit() - cqe.ioUringWaitCqe() - process(completionQueueCallback) - } else { - false - } + ??? } } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 0d5dfb69..915d4107 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -27,8 +27,7 @@ abstract class UringSuite extends CatsEffectSuite { val builder = IORuntimeBuilder() builder.setPollingSystem(UringSystem) - // We can set other components as well: Compute Execution Context, Scheduler, Config, ... - + builder.build() } } From 5f7445205870f09a28f2215131fa4eaf2e371d4c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 13:40:56 +0200 Subject: [PATCH 032/200] Remove Cb from SQ --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 36 ++++++------------- 1 file changed, 10 insertions(+), 26 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index b97ad7e6..68580c2c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -25,7 +25,7 @@ import NativeAccess._ * * @param ringBuffer The RingBuffer associated with the io_uring ring. */ -class UringRing(private val ringBuffer: RingBuffer) { +final class UringRing(private val ringBuffer: RingBuffer) { // The completion queue associated with the ring. private[this] val uringCompletionQueue: UringCompletionQueue = UringCompletionQueue(ringBuffer) @@ -103,14 +103,9 @@ object UringRing { } class UringSubmissionQueue(private val ring: RingBuffer) { - import UringSubmissionQueue._ private[this] val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() - private[this] val callbacks = - scala.collection.mutable.Map[Long, Either[Throwable, Long] => Unit]() - private[this] var id: Short = 0 - def enqueueSqe( op: Byte, flags: Int, @@ -190,9 +185,7 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def release(): Unit = submissionQueue.release() - def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) - - def setData(cb: Either[Throwable, Long] => Unit): Boolean = { + def setData(id: Short): Boolean = { val op: Byte = IORING_OP_POLL_WRITE val flags: Int = 0 val rwFlags: Int = Native.POLLOUT @@ -212,25 +205,8 @@ class UringSubmissionQueue(private val ring: RingBuffer) { id ) - callbacks += (encode(0, 0, id) -> cb) - id = (id + 1).toShort - wasEnqueue } - - def getData(): Short = { - (id - 1).toShort - } - - def removeCallback(data: Short): Option[Either[Throwable,Long] => Unit] = { - callbacks.remove(encode(0, 0, data)) - } - - def callbacksIsEmpty(): Boolean = callbacks.isEmpty - - def prepCancel(addr: Long, flags: Int): Boolean = - enqueueSqe(IORING_OP_ASYNC_CANCEL, flags, 0, -1, addr, 0, 0, 0) - } object UringSubmissionQueue { @@ -340,3 +316,11 @@ object NativeAccess { ringFd ) } + +object Encoder { + def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) + + def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = + UserData.decode(res, flags, udata, callback) + +} From 54dde73faf695d08734c00388223c81f862d68ca Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 13:41:10 +0200 Subject: [PATCH 033/200] Add NOP test --- .../scala/fs2/io/uring/UringSystemSuite.scala | 42 ++++++++++++++++--- 1 file changed, 37 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 6cd45bc5..65533a42 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -26,14 +26,46 @@ class UringSystemSuit extends UringSuite { Uring .get[IO] .flatMap { ring => - ring.call { sqe => - sqe.submit() - () - } + val IORING_OP_NOP: Byte = 0 + + val op: Byte = IORING_OP_NOP + val flags: Int = 0 + val rwFlags: Int = 0 + val fd: Int = -1 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) } - .assertEquals(0.toLong) + .assertEquals(0) + } +// IOURINGINLINE void io_uring_prep_nop(struct io_uring_sqe *sqe) +// { +// io_uring_prep_rw(IORING_OP_NOP, sqe, -1, NULL, 0, 0); +// } + +// IOURINGINLINE void io_uring_prep_rw(int op, struct io_uring_sqe *sqe, int fd, +// const void *addr, unsigned len, +// __u64 offset) +// { +// sqe->opcode = (__u8) op; +// sqe->flags = 0; +// sqe->ioprio = 0; +// sqe->fd = fd; +// sqe->off = offset; +// sqe->addr = (unsigned long) addr; +// sqe->len = len; +// sqe->rw_flags = 0; +// sqe->buf_index = 0; +// sqe->personality = 0; +// sqe->file_index = 0; +// sqe->addr3 = 0; +// sqe->__pad2[0] = 0; +// } + test("successful submission") {} test("failed submission") {} From 42fe04b5651cca0d0e8962e106bba413607864b8 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 13:41:49 +0200 Subject: [PATCH 034/200] Add poll --- .../fs2/io/uring/unsafe/UringSystem.scala | 24 ++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index ef5c6ab6..4783bf3a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -108,7 +108,7 @@ object UringSystem extends PollingSystem { val id = ring.getSqe(resume) val sq: UringSubmissionQueue = ring.getSq() sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - sq.submit() + // sq.submit() cb(Right(id)) } } @@ -169,7 +169,7 @@ object UringSystem extends PollingSystem { id.toShort } - private[UringSystem] def releaseId(id: Short): Boolean = ids.remove(id.toInt) + private[this] def releaseId(id: Short): Boolean = ids.remove(id.toInt) private[UringSystem] def removeCallback(id: Short): Boolean = { val removed = callbacks.remove(id).isDefined @@ -210,7 +210,25 @@ object UringSystem extends PollingSystem { } } - ??? + if (nanos != 0) { + sq.addTimeout(nanos, getUniqueId()) + } + + if (pendingSubmissions) { + sq.submit() + } + + val invokedCbs = processCqes(completionQueueCallback) + + // if no completion events were processed, block until one is ready + if (!invokedCbs) { + cq.ioUringWaitCqe() + processCqes(completionQueueCallback) + } + + pendingSubmissions = false + + invokedCbs } } From 18e2466d1071038a55c142405d70f2bae495eaa2 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 14:06:19 +0200 Subject: [PATCH 035/200] Remove Socket --- .../scala/fs2/io/uring/net/UringNetwork.scala | 61 ------------------- .../scala/fs2/io/uring/net/UringSocket.scala | 21 ------- 2 files changed, 82 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala deleted file mode 100644 index c3cb8748..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ /dev/null @@ -1,61 +0,0 @@ -// package fs2.io.uring.net - -// import fs2.io.net.Network -// import fs2.io.net.tls.TLSContext -// import cats.effect.kernel.Resource -// import com.comcast.ip4s.{Host, Port} -// import fs2.io.net.{DatagramSocket, DatagramSocketGroup} -// import fs2.io.net.DatagramSocketOption -// import com.comcast.ip4s.{Host, Port} -// import fs2.io.net.{Socket, SocketOption} -// import cats.effect.kernel.Resource -// import com.comcast.ip4s.{Host, IpAddress, Port, SocketAddress} -// import fs2.io.net.{Socket, SocketOption} -// import cats.effect.kernel.Resource -// import fs2.io.net.SocketGroup -// import java.util.concurrent.ThreadFactory -// import cats.effect.kernel.Resource -// import fs2.io.net.DatagramSocketGroup -// import java.util.concurrent.ThreadFactory -// import cats.effect.kernel.Resource -// import com.comcast.ip4s.{Host, SocketAddress} -// import fs2.io.net.{Socket, SocketOption} - -// private[net] final class UringNetwork[F[_]]() extends Network.UnsealedNetwork[F] { - -// override def socketGroup( -// threadCount: Int, -// threadFactory: ThreadFactory -// ): Resource[F, SocketGroup[F]] = ??? - -// override def datagramSocketGroup( -// threadFactory: ThreadFactory -// ): Resource[F, DatagramSocketGroup[F]] = ??? - -// override def client( -// to: SocketAddress[Host], -// options: List[SocketOption] -// ): Resource[F, Socket[F]] = ??? - -// override def server( -// address: Option[Host], -// port: Option[Port], -// options: List[SocketOption] -// ): fs2.Stream[F, Socket[F]] = ??? - -// override def serverResource( -// address: Option[Host], -// port: Option[Port], -// options: List[SocketOption] -// ): Resource[F, (SocketAddress[IpAddress], fs2.Stream[F, Socket[F]])] = ??? - -// override def openDatagramSocket( -// address: Option[Host], -// port: Option[Port], -// options: List[DatagramSocketOption], -// protocolFamily: Option[DatagramSocketGroup.ProtocolFamily] -// ): Resource[F, DatagramSocket[F]] = ??? - -// override def tlsContext: TLSContext.Builder[F] = ??? - -// } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala deleted file mode 100644 index 110df2b3..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ /dev/null @@ -1,21 +0,0 @@ -// package fs2.io.uring.net - -// import cats.effect.LiftIO -// import cats.effect.std.Mutex -// import cats.effect.kernel.Async - -// import fs2.io.uring.Uring - -// import com.comcast.ip4s.SocketAddress -// import com.comcast.ip4s.IpAddress -// import fs2.io.net.Socket - -// private[net] final class UringSocket[F[_]: LiftIO]( -// ring: Uring, -// fd: Int, -// remoteAddress: SocketAddress[IpAddress], -// buffer: Array[F[_]], -// readMutex: Mutex[F], -// writeMutex: Mutex[F] -// )(implicit F: Async[F]) -// extends Socket[F] {} From 6a243be11c14b036a9845010f648fd913cd56397 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 18:24:32 +0200 Subject: [PATCH 036/200] Format file & add print for debugging --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 10 ++- .../fs2/io/uring/unsafe/UringSystem.scala | 62 ++++++++++--------- .../scala/fs2/io/uring/UringSystemSuite.scala | 26 +------- 3 files changed, 43 insertions(+), 55 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 68580c2c..907da505 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -115,9 +115,14 @@ class UringSubmissionQueue(private val ring: RingBuffer) { length: Int, offset: Long, data: Short - ): Boolean = + ): Boolean = { + println( + s"[SQ] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + ) submissionQueue.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + } + def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() def decrementHandledFds(): Unit = submissionQueue.decrementHandledFds() @@ -194,7 +199,7 @@ class UringSubmissionQueue(private val ring: RingBuffer) { val length: Int = 0 val offset: Long = 0 - val wasEnqueue: Boolean = enqueueSqe( + val wasEnqueue: Boolean = !enqueueSqe( op, flags, rwFlags, @@ -205,6 +210,7 @@ class UringSubmissionQueue(private val ring: RingBuffer) { id ) + println(s"We enqueued for the id: $id ? $wasEnqueue") wasEnqueue } } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 4783bf3a..fff91090 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -103,6 +103,7 @@ object UringSystem extends PollingSystem { F: MonadCancelThrow[F] ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => + println("[EXEC]: Entering exec") val submit: IO[Short] = IO.async_[Short] { cb => register { ring => val id = ring.getSqe(resume) @@ -110,6 +111,7 @@ object UringSystem extends PollingSystem { sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) // sq.submit() cb(Right(id)) + println("[EXEC]: Leaving exec") } } @@ -173,16 +175,21 @@ object UringSystem extends PollingSystem { private[UringSystem] def removeCallback(id: Short): Boolean = { val removed = callbacks.remove(id).isDefined - if (removed) releaseId(id) + if (removed) { + println(s"REMOVED CB WITH ID: $id") + println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") + releaseId(id) + } removed } private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): Short = { + println("GETTING SQE") pendingSubmissions = true val id: Short = getUniqueId() - if (sq.setData(id)) { - callbacks.put(id, cb) - } + callbacks.put(id, cb) + println(s"CALLBACK MAP UPDATED: $callbacks") + id } @@ -194,41 +201,40 @@ object UringSystem extends PollingSystem { private[UringSystem] def poll(nanos: Long): Boolean = { - def processCqes( + def process( completionQueueCallback: UringCompletionQueueCallback ): Boolean = cq.process(completionQueueCallback) > 0 // True if any completion events were processed - val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = - callbacks.get(data).foreach { cb => - if (res < 0) { - cb(Left(new IOException(s"Error in completion queue entry"))) - } else { - cb(Right(res)) - } - } + if (pendingSubmissions) { + ring.submit() + pendingSubmissions = false } - if (nanos != 0) { - sq.addTimeout(nanos, getUniqueId()) - } + val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { + val removedCallback = callbacks.get(data) - if (pendingSubmissions) { - sq.submit() - } + println(s"[HANDLE CQCB]: fd: $fd, res: $res, falgs: $flags, op: $op, data: $data") - val invokedCbs = processCqes(completionQueueCallback) + removedCallback.foreach { cb => + if (res < 0) cb(Left(new IOException("Error in completion queue entry"))) + else cb(Right(res)) + removeCallback(data) + } + } + } - // if no completion events were processed, block until one is ready - if (!invokedCbs) { + if (cq.hasCompletions()) { + process(completionQueueCallback) + } else if (nanos > 0) { + sq.addTimeout(nanos, 0) + ring.submit() cq.ioUringWaitCqe() - processCqes(completionQueueCallback) + process(completionQueueCallback) + } else { + false } - - pendingSubmissions = false - - invokedCbs } } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 65533a42..635d2501 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -26,7 +26,7 @@ class UringSystemSuit extends UringSuite { Uring .get[IO] .flatMap { ring => - val IORING_OP_NOP: Byte = 0 + val IORING_OP_NOP: Byte = 0 val op: Byte = IORING_OP_NOP val flags: Int = 0 @@ -42,30 +42,6 @@ class UringSystemSuit extends UringSuite { } -// IOURINGINLINE void io_uring_prep_nop(struct io_uring_sqe *sqe) -// { -// io_uring_prep_rw(IORING_OP_NOP, sqe, -1, NULL, 0, 0); -// } - -// IOURINGINLINE void io_uring_prep_rw(int op, struct io_uring_sqe *sqe, int fd, -// const void *addr, unsigned len, -// __u64 offset) -// { -// sqe->opcode = (__u8) op; -// sqe->flags = 0; -// sqe->ioprio = 0; -// sqe->fd = fd; -// sqe->off = offset; -// sqe->addr = (unsigned long) addr; -// sqe->len = len; -// sqe->rw_flags = 0; -// sqe->buf_index = 0; -// sqe->personality = 0; -// sqe->file_index = 0; -// sqe->addr3 = 0; -// sqe->__pad2[0] = 0; -// } - test("successful submission") {} test("failed submission") {} From 2f37b741c4bcb18b2853e84de04643c9c51cd083 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 22:32:00 +0200 Subject: [PATCH 037/200] Add parallel submission tests --- .../scala/fs2/io/uring/UringSystemSuite.scala | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 635d2501..188ed8b0 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -17,6 +17,7 @@ package fs2.io.uring import cats.effect.IO +import cats.syntax.parallel._ import fs2.io.uring.UringSuite @@ -39,7 +40,51 @@ class UringSystemSuit extends UringSuite { ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) } .assertEquals(0) + } + + test("Parallel submission") { + val IORING_OP_NOP: Byte = 0 + + val op: Byte = IORING_OP_NOP + val flags: Int = 0 + val rwFlags: Int = 0 + val fd: Int = -1 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + val calls: List[IO[Int]] = List.fill(300)( + Uring + .get[IO] + .flatMap { ring => + ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) + } + ) + + val test: IO[List[Int]] = calls.parSequence + + test.map(results => assert(results.forall(_ == 0))) + } + + test("Multiple parallel submission") { + val IORING_OP_NOP: Byte = 0 + val op: Byte = IORING_OP_NOP + val flags: Int = 0 + val rwFlags: Int = 0 + val fd: Int = -1 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + val calls: List[IO[List[Int]]] = List.fill(100)( + Uring.get[IO].flatMap { ring => + ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset).replicateA(50) + } + ) + + val test: IO[List[List[Int]]] = calls.parSequence + test.map(listOfList => assert(listOfList.flatten.forall(_ == 0))) } test("successful submission") {} From 79e9f0366415e764dffd62081b80017073208b54 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 11 Jul 2023 22:32:38 +0200 Subject: [PATCH 038/200] Add feedback comments --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index fff91090..89c5f23c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -104,12 +104,12 @@ object UringSystem extends PollingSystem { ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => println("[EXEC]: Entering exec") + println("THREAD:" + Thread.currentThread().getName) val submit: IO[Short] = IO.async_[Short] { cb => register { ring => val id = ring.getSqe(resume) val sq: UringSubmissionQueue = ring.getSq() sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - // sq.submit() cb(Right(id)) println("[EXEC]: Leaving exec") } @@ -136,7 +136,9 @@ object UringSystem extends PollingSystem { } } - private[this] def cancel(id: Short): IO[Boolean] = + private[this] def cancel( + id: Short + ): IO[Boolean] = // TODO: EnqueueSqe with CANCEL ASYNC instead of just removing the callback from the map (What if it is already in the kernel) IO.async_[Boolean] { cb => register { ring => val wasCancel = ring.removeCallback(id) @@ -153,7 +155,7 @@ object UringSystem extends PollingSystem { private[this] var pendingSubmissions: Boolean = false private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = Map.empty[Short, Either[Throwable, Int] => Unit] - private[this] val ids = BitSet(Short.MaxValue + 1) + private[this] val ids = BitSet(Short.MaxValue + 1) // TODO: remove BitSet and use a simply short private[this] var lastUsedId: Int = -1 private[this] def getUniqueId(): Short = { @@ -228,7 +230,10 @@ object UringSystem extends PollingSystem { if (cq.hasCompletions()) { process(completionQueueCallback) } else if (nanos > 0) { - sq.addTimeout(nanos, 0) + sq.addTimeout( + nanos, + 0 + ) // TODO: Check why they do it in this way instead of Scala Native way ring.submit() cq.ioUringWaitCqe() process(completionQueueCallback) From 19249e45128b1fffb5f4b6adaff262bb3bb42c8d Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 14:42:15 +0200 Subject: [PATCH 039/200] Update the Cancel method --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 89c5f23c..53b45ae8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -138,10 +138,13 @@ object UringSystem extends PollingSystem { private[this] def cancel( id: Short - ): IO[Boolean] = // TODO: EnqueueSqe with CANCEL ASYNC instead of just removing the callback from the map (What if it is already in the kernel) + ): IO[Boolean] = IO.async_[Boolean] { cb => register { ring => - val wasCancel = ring.removeCallback(id) + val IORING_OP_ASYNC_CANCEL: Byte = 13 + + val wasCancel: Boolean = + !ring.getSq().enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) cb(Right(wasCancel)) } } @@ -155,7 +158,7 @@ object UringSystem extends PollingSystem { private[this] var pendingSubmissions: Boolean = false private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = Map.empty[Short, Either[Throwable, Int] => Unit] - private[this] val ids = BitSet(Short.MaxValue + 1) // TODO: remove BitSet and use a simply short + private[this] val ids = BitSet(Short.MaxValue + 1) private[this] var lastUsedId: Int = -1 private[this] def getUniqueId(): Short = { @@ -206,7 +209,7 @@ object UringSystem extends PollingSystem { def process( completionQueueCallback: UringCompletionQueueCallback ): Boolean = - cq.process(completionQueueCallback) > 0 // True if any completion events were processed + cq.process(completionQueueCallback) > 0 if (pendingSubmissions) { ring.submit() @@ -232,7 +235,7 @@ object UringSystem extends PollingSystem { } else if (nanos > 0) { sq.addTimeout( nanos, - 0 + getUniqueId() ) // TODO: Check why they do it in this way instead of Scala Native way ring.submit() cq.ioUringWaitCqe() From 285782f0a141eca7212a795d90f0b4b6017b71da Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 19:31:22 +0200 Subject: [PATCH 040/200] add send_msg_ring and all the OP availables --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 78 +++++++++++++------ 1 file changed, 55 insertions(+), 23 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 907da505..75c950d6 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -190,29 +190,8 @@ class UringSubmissionQueue(private val ring: RingBuffer) { def release(): Unit = submissionQueue.release() - def setData(id: Short): Boolean = { - val op: Byte = IORING_OP_POLL_WRITE - val flags: Int = 0 - val rwFlags: Int = Native.POLLOUT - val fd: Int = 0 - val bufferAddress: Long = 0 - val length: Int = 0 - val offset: Long = 0 - - val wasEnqueue: Boolean = !enqueueSqe( - op, - flags, - rwFlags, - fd, - bufferAddress, - length, - offset, - id - ) - - println(s"We enqueued for the id: $id ? $wasEnqueue") - wasEnqueue - } + def sendMsgRing(flags: Int, fd: Int, length: Int, data: Short): Boolean = + submissionQueue.enqueueSqe(OP.IORING_OP_MSG_RING, flags, 0, fd, 0, length, 0, data) } object UringSubmissionQueue { @@ -330,3 +309,56 @@ object Encoder { UserData.decode(res, flags, udata, callback) } + +object OP { + val IORING_OP_NOP: Byte = 0 + val IORING_OP_READV: Byte = 1 + val IORING_OP_WRITEV: Byte = 2 + val IORING_OP_FSYNC: Byte = 3 + val IORING_OP_READ_FIXED: Byte = 4 + val IORING_OP_WRITE_FIXED: Byte = 5 + val IORING_OP_POLL_ADD: Byte = 6 + val IORING_OP_POLL_REMOVE: Byte = 7 + val IORING_OP_SYNC_FILE_RANGE: Byte = 8 + val IORING_OP_SENDMSG: Byte = 9 + val IORING_OP_RECVMSG: Byte = 10 + val IORING_OP_TIMEOUT: Byte = 11 + val IORING_OP_TIMEOUT_REMOVE: Byte = 12 + val IORING_OP_ACCEPT: Byte = 13 + val IORING_OP_ASYNC_CANCEL: Byte = 14 + val IORING_OP_LINK_TIMEOUT: Byte = 15 + val IORING_OP_CONNECT: Byte = 16 + val IORING_OP_FALLOCATE: Byte = 17 + val IORING_OP_OPENAT: Byte = 18 + val IORING_OP_CLOSE: Byte = 19 + val IORING_OP_FILES_UPDATE: Byte = 20 + val IORING_OP_STATX: Byte = 21 + val IORING_OP_READ: Byte = 22 + val IORING_OP_WRITE: Byte = 23 + val IORING_OP_FADVISE: Byte = 24 + val IORING_OP_MADVISE: Byte = 25 + val IORING_OP_SEND: Byte = 26 + val IORING_OP_RECV: Byte = 27 + val IORING_OP_OPENAT2: Byte = 28 + val IORING_OP_EPOLL_CTL: Byte = 29 + val IORING_OP_SPLICE: Byte = 30 + val IORING_OP_PROVIDE_BUFFERS: Byte = 31 + val IORING_OP_REMOVE_BUFFERS: Byte = 32 + val IORING_OP_TEE: Byte = 33 + val IORING_OP_SHUTDOWN: Byte = 34 + val IORING_OP_RENAMEAT: Byte = 35 + val IORING_OP_UNLINKAT: Byte = 36 + val IORING_OP_MKDIRAT: Byte = 37 + val IORING_OP_SYMLINKAT: Byte = 38 + val IORING_OP_LINKAT: Byte = 39 + val IORING_OP_MSG_RING: Byte = 40 + val IORING_OP_FSETXATTR: Byte = 41 + val IORING_OP_SETXATTR: Byte = 42 + val IORING_OP_FGETXATTR: Byte = 43 + val IORING_OP_GETXATTR: Byte = 44 + val IORING_OP_SOCKET: Byte = 45 + val IORING_OP_URING_CMD: Byte = 46 + val IORING_OP_SEND_ZC: Byte = 47 + val IORING_OP_SENDMSG_ZC: Byte = 48 + val IORING_OP_LAST: Byte = 49 +} From 07629a9c6486cea4a061338c1d99ad24c8f0131f Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 19:32:24 +0200 Subject: [PATCH 041/200] Use the OP object to select an operation --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 53b45ae8..78f9d1c3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -32,6 +32,7 @@ import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringCompletionQueueCallback +import io.netty.incubator.channel.uring.OP import java.io.IOException @@ -108,8 +109,7 @@ object UringSystem extends PollingSystem { val submit: IO[Short] = IO.async_[Short] { cb => register { ring => val id = ring.getSqe(resume) - val sq: UringSubmissionQueue = ring.getSq() - sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) + ring.getSq().enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right(id)) println("[EXEC]: Leaving exec") } @@ -141,10 +141,8 @@ object UringSystem extends PollingSystem { ): IO[Boolean] = IO.async_[Boolean] { cb => register { ring => - val IORING_OP_ASYNC_CANCEL: Byte = 13 - val wasCancel: Boolean = - !ring.getSq().enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) + !ring.getSq().enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) cb(Right(wasCancel)) } } From 27cb02b3b7a909b8ba303bd94f8dc6c427592fdd Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 19:32:43 +0200 Subject: [PATCH 042/200] Remove encoder --- .../src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 75c950d6..14b01c8b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -302,14 +302,6 @@ object NativeAccess { ) } -object Encoder { - def encode(fd: Int, op: Byte, data: Short) = UserData.encode(fd, op, data) - - def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = - UserData.decode(res, flags, udata, callback) - -} - object OP { val IORING_OP_NOP: Byte = 0 val IORING_OP_READV: Byte = 1 From 4d5e8dce231f40e61327bce21043b8a13d9b7d86 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 19:46:31 +0200 Subject: [PATCH 043/200] Encapsulate the SubmissionQueue in poller --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 78f9d1c3..667fbdb5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -109,7 +109,7 @@ object UringSystem extends PollingSystem { val submit: IO[Short] = IO.async_[Short] { cb => register { ring => val id = ring.getSqe(resume) - ring.getSq().enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) + ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right(id)) println("[EXEC]: Leaving exec") } @@ -142,7 +142,7 @@ object UringSystem extends PollingSystem { IO.async_[Boolean] { cb => register { ring => val wasCancel: Boolean = - !ring.getSq().enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) + !ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) cb(Right(wasCancel)) } } @@ -196,7 +196,16 @@ object UringSystem extends PollingSystem { id } - private[UringSystem] def getSq(): UringSubmissionQueue = sq + private[UringSystem] def enqueueSqe( + op: Byte, + flags: Int, + rwFlags: Int, + fd: Int, + bufferAddress: Long, + length: Int, + offset: Long, + data: Short + ): Boolean = sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) private[UringSystem] def close(): Unit = ring.close() From 51cfed98848867b094fd50cd84adaf53be6b5af9 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 12 Jul 2023 20:03:17 +0200 Subject: [PATCH 044/200] Reorder poll flow --- .../main/scala/fs2/io/uring/unsafe/UringSystem.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 667fbdb5..e7bffdf8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -218,11 +218,6 @@ object UringSystem extends PollingSystem { ): Boolean = cq.process(completionQueueCallback) > 0 - if (pendingSubmissions) { - ring.submit() - pendingSubmissions = false - } - val completionQueueCallback = new UringCompletionQueueCallback { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { val removedCallback = callbacks.get(data) @@ -237,6 +232,11 @@ object UringSystem extends PollingSystem { } } + if (pendingSubmissions) { + ring.submit() + pendingSubmissions = false + } + if (cq.hasCompletions()) { process(completionQueueCallback) } else if (nanos > 0) { From a842e887f864a49a3e5ad951e61dc619f2c2f1a4 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 13 Jul 2023 18:06:19 +0200 Subject: [PATCH 045/200] Use java BitSet, handle nanos == -1 and add Cancel --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 7 ++ .../fs2/io/uring/unsafe/UringSystem.scala | 114 +++++++++--------- 2 files changed, 65 insertions(+), 56 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 14b01c8b..0d3480a0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -302,6 +302,13 @@ object NativeAccess { ) } +object Encoder { + def encode(fd: Int, op: Byte, data: Short): Long = UserData.encode(fd, op, data) + + def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = + UserData.decode(res, flags, udata, callback) +} + object OP { val IORING_OP_NOP: Byte = 0 val IORING_OP_READV: Byte = 1 diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index e7bffdf8..1a785bfa 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -33,11 +33,12 @@ import io.netty.incubator.channel.uring.UringSubmissionQueue import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringCompletionQueueCallback import io.netty.incubator.channel.uring.OP +import io.netty.incubator.channel.uring.Encoder import java.io.IOException import scala.collection.mutable.Map -import scala.collection.mutable.BitSet +import java.util.BitSet object UringSystem extends PollingSystem { @@ -98,54 +99,61 @@ object UringSystem extends PollingSystem { bufferAddress: Long, length: Int, offset: Long - )(release: Int => IO[Unit]): IO[Int] = IO.cont { - new Cont[IO, Int, Int] { - def apply[F[_]](implicit - F: MonadCancelThrow[F] - ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => - F.uncancelable { poll => - println("[EXEC]: Entering exec") - println("THREAD:" + Thread.currentThread().getName) - val submit: IO[Short] = IO.async_[Short] { cb => - register { ring => - val id = ring.getSqe(resume) - ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - cb(Right(id)) - println("[EXEC]: Leaving exec") - } + )(release: Int => IO[Unit]): IO[Int] = { + + def cancel(id: Short): IO[Boolean] = + IO.uncancelable { _ => + IO.async_[Int] { cb => + register { ring => + val cancelId = ring.getSqe(cb) + val encodedAddress = Encoder.encode(fd, op, id) + ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, encodedAddress, 0, 0, cancelId) + + () } + } + }.map(_ == 0) + + IO.cont { + new Cont[IO, Int, Int] { + def apply[F[_]](implicit + F: MonadCancelThrow[F] + ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => + F.uncancelable { poll => + println("[EXEC]: Entering exec") + println("THREAD:" + Thread.currentThread().getName) + val submit: IO[Short] = IO.async_[Short] { cb => + register { ring => + val id = ring.getSqe(resume) + ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) + cb(Right(id)) + println("[EXEC]: Leaving exec") + } + } - lift(submit) - .flatMap { id => - F.onCancel( - poll(get), - lift(cancel(id)).ifM( - F.unit, - // if cannot cancel, fallback to get - get.flatMap { rtn => - if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) - else lift(release(rtn)) - } + lift(submit) + .flatMap { id => + F.onCancel( + poll(get), + lift(cancel(id)).ifM( + F.unit, + // if cannot cancel, fallback to get + get.flatMap { rtn => + if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) + else lift(release(rtn)) + } + ) ) - ) - } - .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) - } + } + .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) + } + } } } + } - private[this] def cancel( - id: Short - ): IO[Boolean] = - IO.async_[Boolean] { cb => - register { ring => - val wasCancel: Boolean = - !ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, 0, 0, 0, id) - cb(Right(wasCancel)) - } - } } final class Poller private[UringSystem] (ring: UringRing) { @@ -156,25 +164,16 @@ object UringSystem extends PollingSystem { private[this] var pendingSubmissions: Boolean = false private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = Map.empty[Short, Either[Throwable, Int] => Unit] - private[this] val ids = BitSet(Short.MaxValue + 1) + private[this] val ids = new BitSet(Short.MaxValue + 1) private[this] var lastUsedId: Int = -1 private[this] def getUniqueId(): Short = { - val id = (lastUsedId + 1 until Short.MaxValue) - .find(!ids.contains(_)) - .getOrElse( - (0 until lastUsedId) - .find(!ids.contains(_)) - .getOrElse( - throw new RuntimeException("No available IDs") - ) - ) - lastUsedId = id - ids.add(id) - id.toShort + val newId = ids.nextClearBit(lastUsedId) + lastUsedId = newId + newId.toShort } - private[this] def releaseId(id: Short): Boolean = ids.remove(id.toInt) + private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) private[UringSystem] def removeCallback(id: Short): Boolean = { val removed = callbacks.remove(id).isDefined @@ -243,10 +242,13 @@ object UringSystem extends PollingSystem { sq.addTimeout( nanos, getUniqueId() - ) // TODO: Check why they do it in this way instead of Scala Native way + ) ring.submit() cq.ioUringWaitCqe() process(completionQueueCallback) + } else if (nanos == -1) { + cq.ioUringWaitCqe() + process(completionQueueCallback) } else { false } From 348f684e4cff2eb23cd00e5fb96ff6a30853dc93 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 13 Jul 2023 18:17:18 +0200 Subject: [PATCH 046/200] Use OP from netty and add cancellation test --- .../scala/fs2/io/uring/UringSystemSuite.scala | 41 +++++++++++++++---- 1 file changed, 33 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 188ed8b0..31e32bab 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -18,18 +18,19 @@ package fs2.io.uring import cats.effect.IO import cats.syntax.parallel._ +import scala.concurrent.duration._ import fs2.io.uring.UringSuite -class UringSystemSuit extends UringSuite { +import io.netty.incubator.channel.uring.OP + +class UringSystem extends UringSuite { test("submission") { Uring .get[IO] .flatMap { ring => - val IORING_OP_NOP: Byte = 0 - - val op: Byte = IORING_OP_NOP + val op: Byte = OP.IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 @@ -43,9 +44,8 @@ class UringSystemSuit extends UringSuite { } test("Parallel submission") { - val IORING_OP_NOP: Byte = 0 - val op: Byte = IORING_OP_NOP + val op: Byte = OP.IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 @@ -67,8 +67,7 @@ class UringSystemSuit extends UringSuite { } test("Multiple parallel submission") { - val IORING_OP_NOP: Byte = 0 - val op: Byte = IORING_OP_NOP + val op: Byte = OP.IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 @@ -87,6 +86,32 @@ class UringSystemSuit extends UringSuite { test.map(listOfList => assert(listOfList.flatten.forall(_ == 0))) } + test("cancellation") { + Uring + .get[IO] + .flatMap { ring => + val IORING_OP_NOP: Byte = 0 + + val op: Byte = IORING_OP_NOP + val flags: Int = 0 + val rwFlags: Int = 0 + val fd: Int = -1 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 + + val operation = ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) + + val cancellation = for { + fiber <- operation.start + _ <- IO.sleep(1.second) + cancelled <- fiber.cancel + } yield cancelled + + cancellation + } + } + test("successful submission") {} test("failed submission") {} From dfbc637deee4700c3342b6eca05fbb08589c7b5e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 13 Jul 2023 21:24:46 +0200 Subject: [PATCH 047/200] Remove cancellation test --- .../scala/fs2/io/uring/UringSystemSuite.scala | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 31e32bab..1834548f 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -18,7 +18,6 @@ package fs2.io.uring import cats.effect.IO import cats.syntax.parallel._ -import scala.concurrent.duration._ import fs2.io.uring.UringSuite @@ -86,32 +85,6 @@ class UringSystem extends UringSuite { test.map(listOfList => assert(listOfList.flatten.forall(_ == 0))) } - test("cancellation") { - Uring - .get[IO] - .flatMap { ring => - val IORING_OP_NOP: Byte = 0 - - val op: Byte = IORING_OP_NOP - val flags: Int = 0 - val rwFlags: Int = 0 - val fd: Int = -1 - val bufferAddress: Long = 0 - val length: Int = 0 - val offset: Long = 0 - - val operation = ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) - - val cancellation = for { - fiber <- operation.start - _ <- IO.sleep(1.second) - cancelled <- fiber.cancel - } yield cancelled - - cancellation - } - } - test("successful submission") {} test("failed submission") {} From 9c23c2b3e40114dc0a8c6663cde4cb76eb960b48 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 13 Jul 2023 21:25:24 +0200 Subject: [PATCH 048/200] Comment the problem with nanos == -1 --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 1a785bfa..4536bd2e 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -108,7 +108,6 @@ object UringSystem extends PollingSystem { val cancelId = ring.getSqe(cb) val encodedAddress = Encoder.encode(fd, op, id) ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, encodedAddress, 0, 0, cancelId) - () } } @@ -165,11 +164,10 @@ object UringSystem extends PollingSystem { private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = Map.empty[Short, Either[Throwable, Int] => Unit] private[this] val ids = new BitSet(Short.MaxValue + 1) - private[this] var lastUsedId: Int = -1 private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(lastUsedId) - lastUsedId = newId + val newId = ids.nextClearBit(0) + ids.set(newId) newId.toShort } @@ -246,10 +244,12 @@ object UringSystem extends PollingSystem { ring.submit() cq.ioUringWaitCqe() process(completionQueueCallback) - } else if (nanos == -1) { - cq.ioUringWaitCqe() - process(completionQueueCallback) - } else { + } + // else if (nanos == -1) { // TODO: Tests run forever from the begining due to the cq.ioUringWaitCqe() + // cq.ioUringWaitCqe() + // process(completionQueueCallback) + // } + else { false } } From 070655b9b3669a658a9b04db5b63b457a9ee37e1 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 14 Jul 2023 18:04:02 +0200 Subject: [PATCH 049/200] Abstract the handleCallback --- .../fs2/io/uring/unsafe/UringSystem.scala | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 4536bd2e..9fd42407 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -105,9 +105,9 @@ object UringSystem extends PollingSystem { IO.uncancelable { _ => IO.async_[Int] { cb => register { ring => - val cancelId = ring.getSqe(cb) - val encodedAddress = Encoder.encode(fd, op, id) - ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, encodedAddress, 0, 0, cancelId) + val cancelId = ring.getId(cb) + val opToCancel = Encoder.encode(fd, op, id) + ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, cancelId) () } } @@ -123,7 +123,7 @@ object UringSystem extends PollingSystem { println("THREAD:" + Thread.currentThread().getName) val submit: IO[Short] = IO.async_[Short] { cb => register { ring => - val id = ring.getSqe(resume) + val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right(id)) println("[EXEC]: Leaving exec") @@ -173,23 +173,23 @@ object UringSystem extends PollingSystem { private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - private[UringSystem] def removeCallback(id: Short): Boolean = { - val removed = callbacks.remove(id).isDefined - if (removed) { - println(s"REMOVED CB WITH ID: $id") - println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") - releaseId(id) - } - removed - } + private[this] def removeCallback(id: Short): Boolean = + callbacks + .remove(id) + .map { _ => + println(s"REMOVED CB WITH ID: $id") + println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") + releaseId(id) + } + .isDefined - private[UringSystem] def getSqe(cb: Either[Throwable, Int] => Unit): Short = { - println("GETTING SQE") - pendingSubmissions = true + private[UringSystem] def getId(cb: Either[Throwable, Int] => Unit): Short = { val id: Short = getUniqueId() + + pendingSubmissions = true callbacks.put(id, cb) + println("GETTING ID") println(s"CALLBACK MAP UPDATED: $callbacks") - id } @@ -217,13 +217,15 @@ object UringSystem extends PollingSystem { val completionQueueCallback = new UringCompletionQueueCallback { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - val removedCallback = callbacks.get(data) + def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = + if (res < 0) + cb(Left(new IOException(s"Error in completion queue entry: $res"))) + else cb(Right(res)) - println(s"[HANDLE CQCB]: fd: $fd, res: $res, falgs: $flags, op: $op, data: $data") + println(s"[HANDLE CQCB]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data") - removedCallback.foreach { cb => - if (res < 0) cb(Left(new IOException("Error in completion queue entry"))) - else cb(Right(res)) + callbacks.get(data).foreach { cb => + handleCallback(res, cb) removeCallback(data) } } From 6c9f3016ea5e60b982c8ba156619bf1c6e8e266a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 14 Jul 2023 18:04:17 +0200 Subject: [PATCH 050/200] Add documentation --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 329 +++++++++++++++--- 1 file changed, 277 insertions(+), 52 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 0d3480a0..08a51abf 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -19,44 +19,45 @@ package io.netty.incubator.channel.uring import io.netty.channel.unix.FileDescriptor import NativeAccess._ -/** The UringRing class represents a complete io_uring ring with both submission and completion queues. - * It provides methods to interact with the submission and completion queues, such as submitting operations, +/** Represents an io_uring Ring with both Submission Queue (SQ) and Completion Queue (CQ). + * + * It provides methods to interact with the queues, such as submitting operations, * accessing the file descriptor of the ring, and closing the ring. * - * @param ringBuffer The RingBuffer associated with the io_uring ring. + * @param ringBuffer The RingBuffer associated with the Ring. */ -final class UringRing(private val ringBuffer: RingBuffer) { - // The completion queue associated with the ring. +final class UringRing(private[this] val ringBuffer: RingBuffer) { + // The Completion Queue associated with the Ring. private[this] val uringCompletionQueue: UringCompletionQueue = UringCompletionQueue(ringBuffer) - // The submission queue associated with the ring. + // The Submission Queue associated with the Ring. private[this] val uringSubmissionQueue: UringSubmissionQueue = UringSubmissionQueue(ringBuffer) - /** Constructs a new UringRing instance with the default ring buffer size. + /** Constructs a new Ring instance with the default Ring buffer size. */ def this() = this(createRingBuffer()) - /** Constructs a new UringRing instance with the specified ring buffer size. + /** Constructs a new Ring instance with the specified Ring buffer size. * - * @param size of the ring buffer. + * @param size of the new Ring buffer. */ def this(size: Int) = this(createRingBuffer(size)) - /** Constructs a new UringRing instance with the specified ring buffer size and + /** Constructs a new Ring instance with the specified Ring buffer size and * SQE (Submission Queue Entry) async threshold. * - * @param size of the ring buffer. + * @param size of the Ring buffer. * @param sqeAsyncThreshold The threshold value for determining whether an * SQE should be submitted asynchronously. */ def this(size: Int, sqeAsyncThreshold: Int) = this(createRingBuffer(size, sqeAsyncThreshold)) - /** @return the UringCompletionQueue associated with the ring. + /** @return the Completion Queue (CQ) associated with the Ring. */ def ioUringCompletionQueue(): UringCompletionQueue = uringCompletionQueue - /** @return the UringSubmissionQueue associated with the ring. + /** @return the Submission Queue (SQ) associated with the Ring. */ def ioUringSubmissionQueue(): UringSubmissionQueue = uringSubmissionQueue @@ -66,31 +67,31 @@ final class UringRing(private val ringBuffer: RingBuffer) { */ def submit(): Int = uringSubmissionQueue.submit() - /** @return The file descriptor of the ring buffer. + /** @return The file descriptor of the Ring buffer. */ def fd(): Int = ringBuffer.fd() - /** Closes the ring, realising any associated resources. + /** Closes the Ring, realising any associated resources. */ def close(): Unit = ringBuffer.close() } object UringRing { - /** Creates a new UringRing instance with the default ring buffer size. + /** Creates a new UringRing instance with the default Ring buffer size. * - * @return a new UringRing instance. + * @return a new Ring instance. */ def apply(): UringRing = new UringRing() - /** Creates a new UringRing instance with the specified ring buffer size. + /** Creates a new UringRing instance with the specified Ring buffer size. * * @param size of the ring buffer. - * @return a new UringRing instance. + * @return a new Ring instance. */ def apply(size: Int): UringRing = new UringRing(size) - /** Creates a new UringRing instance with the specified ring buffer size + /** Creates a new Ring instance with the specified ring buffer size * and SQE (Submission Queue Entry) async threshold. * * @param size of the ring buffer. @@ -102,10 +103,28 @@ object UringRing { } -class UringSubmissionQueue(private val ring: RingBuffer) { +/** Represents an io_uring Submission Queue (SQ). + * + * It provides methods for enqueuing different types of IO operations and controlling their execution. + * + * @param ring The RingBuffer used to queue IO operations. + */ +final class UringSubmissionQueue(private[this] val ring: RingBuffer) { + // The Submission Queue instance associated with the Ring. private[this] val submissionQueue: IOUringSubmissionQueue = ring.ioUringSubmissionQueue() + /** Creates a Submission Queue Entry (SQE) associates an IO operation and enqueues it to the Submission Queue. + * @param op The type of IO operation to enqueue. + * @param flags The flags for the IO operation. + * @param rwFlags The flags for read/write operations. + * @param fd The file descriptor associated with the IO operation. + * @param bufferAddress The address of the buffer for read/write operations. + * @param length The length of the buffer for read/write operations. + * @param offset The offset at which to start read/write operations. + * @param data Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def enqueueSqe( op: Byte, flags: Int, @@ -123,31 +142,85 @@ class UringSubmissionQueue(private val ring: RingBuffer) { } + /** Increment the number of handled file descriptors. */ def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() + /** Decrement the number of handled file descriptors. */ def decrementHandledFds(): Unit = submissionQueue.decrementHandledFds() + /** Add a timeout operation to the Submission Queue. + * @param nanoSeconds The timeout duration in nanoseconds. + * @param extraData Extra data (id) for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addTimeout(nanoSeconds: Long, extraData: Short): Boolean = submissionQueue.addTimeout(nanoSeconds, extraData) + /** Enqueues an operation to the Submission Queue to add a poll on the input availability of a file descriptor. + * @param fd The file descriptor to poll. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addPollIn(fd: Int): Boolean = submissionQueue.addPollIn(fd) + /** Add a poll operation on the hang-up event of a file descriptor. + * @param fd The file descriptor to poll. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addPollRdHup(fd: Int): Boolean = submissionQueue.addPollRdHup(fd) + /** Add a poll operation on the output availability of a file descriptor. + * @param fd The file descriptor to poll. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addPollOut(fd: Int): Boolean = submissionQueue.addPollOut(fd) + /** Add a receive message operation from a file descriptor. + * @param fd The file descriptor. + * @param msgHdr The address of the message header. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addRecvmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = submissionQueue.addRecvmsg(fd, msgHdr, extraData) + /** Enqueues a send message operation to a file descriptor. + * @param fd The file descriptor. + * @param msgHdr The address of the message header. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addSendmsg(fd: Int, msgHdr: Long, extraData: Short): Boolean = submissionQueue.addSendmsg(fd, msgHdr, extraData) + /** Enqueues a send message operation to a file descriptor, with specific flags. + * @param fd The file descriptor. + * @param msgHdr The address of the message header. + * @param flags The flags for the send message operation. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addSendmsg(fd: Int, msgHdr: Long, flags: Int, extraData: Short): Boolean = submissionQueue.addSendmsg(fd, msgHdr, flags, extraData) + /** Add a read operation from a file descriptor. + * @param fd The file descriptor. + * @param bufferAddress The address of the buffer where to store the data. + * @param pos The position in the buffer to start storing data. + * @param limit The maximum number of bytes to read. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addRead(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = submissionQueue.addRead(fd, bufferAddress, pos, limit, extraData) + /** Enqueues an operation to read data from an event file descriptor. + * @param fd The file descriptor. + * @param bufferAddress The address of the buffer where the read data should be placed. + * @param pos The position in the buffer to start placing data. + * @param limit The maximum number of bytes to read. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addEventFdRead( fd: Int, bufferAddress: Long, @@ -156,20 +229,63 @@ class UringSubmissionQueue(private val ring: RingBuffer) { extraData: Short ): Boolean = submissionQueue.addEventFdRead(fd, bufferAddress, pos, limit, extraData) + /** Enqueues a write operation to a file descriptor. + * @param fd The file descriptor. + * @param bufferAddress The address of the buffer containing the data to write. + * @param pos The position in the buffer to start writing data. + * @param limit The maximum number of bytes to write. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addWrite(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = submissionQueue.addWrite(fd, bufferAddress, pos, limit, extraData) + /** Enqueues a receive operation from a file descriptor. + * @param fd The file descriptor. + * @param bufferAddress The address of the buffer where the received data should be placed. + * @param pos The position in the buffer to start placing data. + * @param limit The maximum number of bytes to receive. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addRecv(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = submissionQueue.addRecv(fd, bufferAddress, pos, limit, extraData) + /** Enqueues a send operation to a file descriptor. + * @param fd The file descriptor. + * @param bufferAddress The address of the buffer containing the data to send. + * @param pos The position in the buffer to start sending data. + * @param limit The maximum number of bytes to send. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addSend(fd: Int, bufferAddress: Long, pos: Int, limit: Int, extraData: Short): Boolean = submissionQueue.addSend(fd, bufferAddress, pos, limit, extraData) + /** Enqueues an accept operation for a file descriptor. + * @param fd The file descriptor. + * @param address The address where the details of the incoming connection will be stored. + * @param addressLength The length of the address structure. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addAccept(fd: Int, address: Long, addressLength: Long, extraData: Short): Boolean = submissionQueue.addAccept(fd, address, addressLength, extraData) + /** Enqueues an operation to remove a poll event from the monitoring of a file descriptor. + * @param fd The file descriptor. + * @param pollMask The mask for the poll events to be removed. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addPollRemove(fd: Int, pollMask: Int): Boolean = submissionQueue.addPollRemove(fd, pollMask) + /** Enqueues a connection operation for a file descriptor to a socket address. + * @param fd The file descriptor. + * @param socketAddress The address of the socket to connect to. + * @param socketAddressLength The length of the socket address. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addConnect( fd: Int, socketAddress: Long, @@ -177,111 +293,197 @@ class UringSubmissionQueue(private val ring: RingBuffer) { extraData: Short ): Boolean = submissionQueue.addConnect(fd, socketAddress, socketAddressLength, extraData) + /** Enqueues an operation to write data to a file descriptor from multiple buffers. + * @param fd The file descriptor. + * @param iovecArrayAddress The address of an array of iovec structures, each specifying a buffer. + * @param length The total length of the data to write. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addWritev(fd: Int, iovecArrayAddress: Long, length: Int, extraData: Short): Boolean = submissionQueue.addWritev(fd, iovecArrayAddress, length, extraData) + /** Enqueues a close operation for a file descriptor. + * @param fd The file descriptor to close. + * @param extraData Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def addClose(fd: Int, extraData: Short): Boolean = submissionQueue.addClose(fd, extraData) + /** Submit all enqueued operations in the Submission Queue to the kernel for execution. + * @return The number of submitted operations. + */ def submit(): Int = submissionQueue.submit() + /** Submit all enqueued operations in the Submission Queue to the kernel for execution and wait for them to complete. + * @return The number of submitted operations. + */ def submitAndWait(): Int = submissionQueue.submitAndWait() + /** Get the number of operations in the Submission Queue. + * @return The number of operations in the Submission Queue. + */ def count(): Long = submissionQueue.count() + /** Release resources associated with the submission queue. */ def release(): Unit = submissionQueue.release() + /** Enqueues an IO operation to send a message to a Ring. + * @param flags The flags for the IO operation. + * @param fd The file descriptor associated with the IO operation. + * @param length The length of the message to send. + * @param data Extra data for the IO operation. + * @return true if the operation is successfully enqueued, false otherwise. + */ def sendMsgRing(flags: Int, fd: Int, length: Int, data: Short): Boolean = submissionQueue.enqueueSqe(OP.IORING_OP_MSG_RING, flags, 0, fd, 0, length, 0, data) } -object UringSubmissionQueue { - final val SQE_SIZE = 64 - - final val IORING_OP_ASYNC_CANCEL: Byte = 14.toByte - - final val SQE_USER_DATA_FIELD = 32 +private[this] object UringSubmissionQueue { + /** Creates a new Submission Queue (SQ) instance associated with the specified RingBuffer. + * + * @param ring The RingBuffer associated with the Submission Queue. + * @return A new Submission Queue instance. + */ def apply(ring: RingBuffer): UringSubmissionQueue = new UringSubmissionQueue(ring) } -/** The UringCompletionQueue class represents a completion queue for the io_uring subsystem in the Netty library. - * It provides methods to interact with the completion queue, such as checking for completions, processing completions, - * waiting for completions, and accessing the underlying ring buffer. +/** Represents a io_uring Completion Queue (CQ). * - * @param ring The RingBuffer associated with the completion queue. + * It provides methods to interact with the Completion Queue, such as checking for completions, processing completions, + * waiting for completions, and accessing the underlying RingBuffer. + * + * @param ring The RingBuffer associated with the Completion Queue. */ -class UringCompletionQueue(private val ring: RingBuffer) { +final class UringCompletionQueue(private[this] val ring: RingBuffer) { - // The IOUringCompletionQueue instance associated with the ring. - private val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() + // The Completion Queue instance associated with the Ring. + private[this] val completionQueue: IOUringCompletionQueue = ring.ioUringCompletionQueue() + /** Checks if there are any completions in the Completion Queue. + * + * @return `true` if there are completions, `false` otherwise. + */ def hasCompletions(): Boolean = completionQueue.hasCompletions() + /** Processes the Completion Queue entries (CQE) with the provided callback. + * + * @param cb Callback function to process each entry. + * @return The number of entries processed. + */ def process(cb: IOUringCompletionQueueCallback): Int = completionQueue.process(cb) + /** Waits for at least one completion entry in the Completion Queue. + */ def ioUringWaitCqe(): Unit = completionQueue.ioUringWaitCqe() + /** Fetches the ring address of the associated ring. + * + * @return The address of the ring. + */ def ringAddress(): Long = completionQueue.ringAddress + /** Fetches the file descriptor of the associated ring. + * + * @return The file descriptor of the ring. + */ def ringFd(): Int = completionQueue.ringFd + /** Fetches the size of the associated ring. + * + * @return The size of the ring. + */ def ringSize(): Int = completionQueue.ringSize } -object UringCompletionQueue { +private[this] object UringCompletionQueue { - /** Creates a new UringCompletionQueue instance associated with the specified RingBuffer. + /** Creates a new Completion Queue (CQ) instance associated with the specified RingBuffer. * - * @param ring The RingBuffer associated with the completion queue. - * @return A new UringCompletionQueue instance. + * @param ring The RingBuffer associated with the Completion Queue. + * @return A new Completion Queue instance. */ def apply(ring: RingBuffer): UringCompletionQueue = new UringCompletionQueue(ring) } -/** The UringCompletionQueueCallback trait defines a callback interface for handling completion events - * from the io_uring completion queue. It extends the IOUringCompletionQueueCallback trait and provides - * a method handle to process the completion event. +/** The UringCompletionQueueCallback trait defines a callback interface for handling completion event from the io_uring Completion Queue. */ trait UringCompletionQueueCallback extends IOUringCompletionQueueCallback { def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit } -/** Provides direct access to the native methods and functionalities - * of the io_uring subsystem in Netty. +/** Provides a bridge to the native io_uring functionalities provided by the Netty library. + * It provides methods to create RingBuffers with varying sizes and thresholds, check IO support, + * verify kernel version, manipulate event file descriptors, and interact directly with the underlying io_uring. */ object NativeAccess { - val DEFAULT_RING_SIZE = Native.DEFAULT_RING_SIZE - val DEFAULT_IOSEQ_ASYNC_THRESHOLD = Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD - val IORING_OP_POLL_WRITE = Native.IORING_OP_WRITE - val IORING_OP_POLL_READ = Native.IORING_OP_READ - - val POLLIN = Native.POLLIN - val POLLOUT = Native.POLLOUT + /** Creates a RingBuffer with the default size and IO sequence async threshold. + * @return A new RingBuffer instance. + */ def createRingBuffer(): RingBuffer = - createRingBuffer(DEFAULT_RING_SIZE, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + createRingBuffer(Native.DEFAULT_RING_SIZE, Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD) + /** Creates a RingBuffer with the specified size and default IO sequence async threshold. + * @param size The desired size for the RingBuffer. + * @return A new RingBuffer instance. + */ def createRingBuffer(size: Int): RingBuffer = - createRingBuffer(size, DEFAULT_IOSEQ_ASYNC_THRESHOLD) + createRingBuffer(size, Native.DEFAULT_IOSEQ_ASYNC_THRESHOLD) + /** Creates a RingBuffer with the specified size and IO sequence async threshold. + * @param size The desired size for the RingBuffer. + * @param sqeAsyncThreshold The desired IO sequence async threshold for the RingBuffer. + * @return A new RingBuffer instance. + */ def createRingBuffer(size: Int, sqeAsyncThreshold: Int): RingBuffer = Native.createRingBuffer(size, sqeAsyncThreshold) + /** Checks if all IO operations are supported for the given ring file descriptor. + * @param ringFd The file descriptor of the ring. + */ def checkAllIOSupported(ringFd: Int): Unit = Native.checkAllIOSupported(ringFd) + /** Checks if the given kernel version is compatible with the io_uring. + * @param kernelVersion The version of the kernel to check. + */ def checkKernelVersion(kernelVersion: String): Unit = Native.checkKernelVersion(kernelVersion) + /** Submits requests to the kernel and waits for a minimum number of completions. + * @param ringFd The file descriptor of the ring. + * @param toSubmit The number of submissions to make. + * @param minComplete The minimum number of completions to wait for. + * @param flags The flags for the operation. + * @return The number of IO events retrieved. + */ def ioUringEnter(ringFd: Int, toSubmit: Int, minComplete: Int, flags: Int): Int = Native.ioUringEnter(ringFd, toSubmit, minComplete, flags) + /** Writes a value to the specified event file descriptor. + * @param fd The file descriptor to write to. + * @param value The value to write. + */ def eventFdWrite(fd: Int, value: Long): Unit = Native.eventFdWrite(fd, value) + /** Creates a new blocking event file descriptor. + * @return A new FileDescriptor instance. + */ def newBlockingEventFd: FileDescriptor = Native.newBlockingEventFd() + /** Closes the Ring, releasing the memory associated with it. + * @param submissionQueueArrayAddress The address of the Submission Queue array. + * @param submissionQueueRingEntries The number of entries in the Submission Queue Ring. + * @param submissionQueueRingAddress The address of the Submission Queue Ring. + * @param submissionQueueRingSize The size of the Submission Queue Ring. + * @param completionQueueRingAddress The address of the Completion Queue Ring. + * @param completionQueueRingSize The size of the Completion Queue Ring. + * @param ringFd The file descriptor of the Ring. + */ def ioUringExit( submissionQueueArrayAddress: Long, submissionQueueRingEntries: Int, @@ -302,13 +504,36 @@ object NativeAccess { ) } +/** Provides utility methods for encoding and decoding user data in IO operations. + * It uses the same encoding scheme as the Netty API. This is used to store and retrieve information about the operations + * being submitted to or completed by the IOUring interface. + */ object Encoder { + + /** Encodes the file descriptor, operation type, and data into a Long value for use with the IOUring interface. + * This encoding is performed in the same manner as in the Netty API. + * + * @param fd The file descriptor for the IO operation. + * @param op The operation type. + * @param data The data for the operation. + * @return Encoded Long value that represents the given parameters. + */ def encode(fd: Int, op: Byte, data: Short): Long = UserData.encode(fd, op, data) + /** Decodes the result, flags and user data from a completed IO operation, and passes this information to the provided callback. + * This decoding is performed in the same manner as in the Netty API. + * + * @param res The result of the operation. + * @param flags Any flags associated with the operation. + * @param udata The user data for the operation. + * @param callback The callback to be invoked with the decoded information. + */ def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = UserData.decode(res, flags, udata, callback) } +/** Defines constants for various operation types supported by the io_uring interface. + */ object OP { val IORING_OP_NOP: Byte = 0 val IORING_OP_READV: Byte = 1 From 28ffbca7b98e860aac7fa587863575096c598219 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 14 Jul 2023 21:04:10 +0200 Subject: [PATCH 051/200] Rename UringSystemSuite & Reformat IORuntime --- .../jvm/src/test/scala/fs2/io/uring/UringSuite.scala | 11 ++++------- .../test/scala/fs2/io/uring/UringSystemSuite.scala | 2 +- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 915d4107..6c60cfc8 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -23,11 +23,8 @@ import cats.effect.unsafe.IORuntimeBuilder abstract class UringSuite extends CatsEffectSuite { - override lazy val munitIORuntime = { - val builder = IORuntimeBuilder() - - builder.setPollingSystem(UringSystem) - - builder.build() - } + override lazy val munitIORuntime = + IORuntimeBuilder() + .setPollingSystem(UringSystem) + .build() } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 1834548f..9d1f45dc 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -23,7 +23,7 @@ import fs2.io.uring.UringSuite import io.netty.incubator.channel.uring.OP -class UringSystem extends UringSuite { +class UringSystemSuite extends UringSuite { test("submission") { Uring From 46672976c50fd542b4845622284ac4bff6b1ab6e Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sun, 16 Jul 2023 12:37:59 +0000 Subject: [PATCH 052/200] Add cancel and sendMsgRing to Poller --- .../fs2/io/uring/unsafe/UringSystem.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 9fd42407..dbb5d0aa 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -107,7 +107,7 @@ object UringSystem extends PollingSystem { register { ring => val cancelId = ring.getId(cb) val opToCancel = Encoder.encode(fd, op, id) - ring.enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, cancelId) + ring.cancel(opToCancel, cancelId) () } } @@ -204,6 +204,12 @@ object UringSystem extends PollingSystem { data: Short ): Boolean = sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + private[UringSystem] def cancel(opToCancel: Long, id: Short) = + enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) + + private[UringSystem] def sendMsgRing(flags: Int, fd: Int, length: Int, data: Short): Boolean = + sq.sendMsgRing(flags, fd, length, data) + private[UringSystem] def close(): Unit = ring.close() private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty @@ -246,12 +252,14 @@ object UringSystem extends PollingSystem { ring.submit() cq.ioUringWaitCqe() process(completionQueueCallback) - } - // else if (nanos == -1) { // TODO: Tests run forever from the begining due to the cq.ioUringWaitCqe() - // cq.ioUringWaitCqe() - // process(completionQueueCallback) - // } - else { + } else if (nanos == -1) { // TODO: Tests run forever due to the cq.ioUringWaitCqe() + if (pendingSubmissions) { + ring.ioUringSubmissionQueue().submitAndWait() + pendingSubmissions = false + } + // cq.ioUringWaitCqe() + process(completionQueueCallback) + } else { false } } From 032a279528322cf3bae7173a5ee7b6a6091b75d6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 17 Jul 2023 17:24:59 +0000 Subject: [PATCH 053/200] Remove unused tests --- .../src/test/scala/fs2/io/uring/UringSystemSuite.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 9d1f45dc..32555c7b 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -84,13 +84,4 @@ class UringSystemSuite extends UringSuite { test.map(listOfList => assert(listOfList.flatten.forall(_ == 0))) } - - test("successful submission") {} - - test("failed submission") {} - - test("polling without completions and no timeout") {} - - test("polling with timeout and completions") {} - } From 0e7365cc0c0cd1652b517c2db6c3e78ab191ebc1 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 17 Jul 2023 17:25:35 +0000 Subject: [PATCH 054/200] Rewrite poll system and add debug mode --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 7 +- .../fs2/io/uring/unsafe/UringSystem.scala | 69 +++++++++++-------- 2 files changed, 41 insertions(+), 35 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 08a51abf..fdec0ea2 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -134,14 +134,9 @@ final class UringSubmissionQueue(private[this] val ring: RingBuffer) { length: Int, offset: Long, data: Short - ): Boolean = { - println( - s"[SQ] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" - ) + ): Boolean = submissionQueue.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) - } - /** Increment the number of handled file descriptors. */ def incrementHandledFds(): Unit = submissionQueue.incrementHandledFds() diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index dbb5d0aa..8ebb79b9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -44,6 +44,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 + private val debug = false // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -64,6 +65,8 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () + def interrupt(poller: Poller, targetThread: Thread, targetPoller: Poller): Unit = ??? + private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit @@ -119,14 +122,11 @@ object UringSystem extends PollingSystem { F: MonadCancelThrow[F] ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => - println("[EXEC]: Entering exec") - println("THREAD:" + Thread.currentThread().getName) val submit: IO[Short] = IO.async_[Short] { cb => register { ring => val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right(id)) - println("[EXEC]: Leaving exec") } } @@ -177,8 +177,10 @@ object UringSystem extends PollingSystem { callbacks .remove(id) .map { _ => - println(s"REMOVED CB WITH ID: $id") - println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") + if (debug) { + println(s"REMOVED CB WITH ID: $id") + println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") + } releaseId(id) } .isDefined @@ -188,8 +190,10 @@ object UringSystem extends PollingSystem { pendingSubmissions = true callbacks.put(id, cb) - println("GETTING ID") - println(s"CALLBACK MAP UPDATED: $callbacks") + if (debug) { + println("GETTING ID") + println(s"CALLBACK MAP UPDATED: $callbacks") + } id } @@ -202,7 +206,13 @@ object UringSystem extends PollingSystem { length: Int, offset: Long, data: Short - ): Boolean = sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + ): Boolean = { + if (debug) + println( + s"[SQ] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + ) + sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + } private[UringSystem] def cancel(opToCancel: Long, id: Short) = enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) @@ -228,7 +238,10 @@ object UringSystem extends PollingSystem { cb(Left(new IOException(s"Error in completion queue entry: $res"))) else cb(Right(res)) - println(s"[HANDLE CQCB]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data") + if ( + op != 11 && debug + ) // To prevent the constant printouts of timeout operation when NANOS == -1 + println(s"[HANDLE CQCB]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data") callbacks.get(data).foreach { cb => handleCallback(res, cb) @@ -237,30 +250,28 @@ object UringSystem extends PollingSystem { } } - if (pendingSubmissions) { - ring.submit() - pendingSubmissions = false + def handlePendingSubmissions(submitAndWait: Boolean): Boolean = { + val submitted = if (submitAndWait) sq.submitAndWait() > 0 else sq.submit() > 0 + if (submitted) pendingSubmissions = false + submitted } - if (cq.hasCompletions()) { - process(completionQueueCallback) - } else if (nanos > 0) { - sq.addTimeout( - nanos, - getUniqueId() - ) - ring.submit() + def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { + sq.addTimeout(nanos, getUniqueId()) + val submitted = handlePendingSubmissions(submitAndWait) cq.ioUringWaitCqe() process(completionQueueCallback) - } else if (nanos == -1) { // TODO: Tests run forever due to the cq.ioUringWaitCqe() - if (pendingSubmissions) { - ring.ioUringSubmissionQueue().submitAndWait() - pendingSubmissions = false - } - // cq.ioUringWaitCqe() - process(completionQueueCallback) - } else { - false + submitted + } + + nanos match { + case -1 => + if (pendingSubmissions) handlePendingSubmissions(true) + else handleTimeoutAndQueue(-1, true) + case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false + case _ => + if (pendingSubmissions) handlePendingSubmissions(true) + else handleTimeoutAndQueue(nanos, false) } } From d42665d6c7fb5489a89d400c885ecb63ef326809 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 20:04:46 +0200 Subject: [PATCH 055/200] Move OP to util and rework on SendMsgRing --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 65 ------------------- .../fs2/io/uring/unsafe/UringSystem.scala | 15 +++-- .../main/scala/fs2/io/uring/unsafe/util.scala | 59 ++++++++++++++++- 3 files changed, 68 insertions(+), 71 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index fdec0ea2..2db298c8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -322,16 +322,6 @@ final class UringSubmissionQueue(private[this] val ring: RingBuffer) { /** Release resources associated with the submission queue. */ def release(): Unit = submissionQueue.release() - - /** Enqueues an IO operation to send a message to a Ring. - * @param flags The flags for the IO operation. - * @param fd The file descriptor associated with the IO operation. - * @param length The length of the message to send. - * @param data Extra data for the IO operation. - * @return true if the operation is successfully enqueued, false otherwise. - */ - def sendMsgRing(flags: Int, fd: Int, length: Int, data: Short): Boolean = - submissionQueue.enqueueSqe(OP.IORING_OP_MSG_RING, flags, 0, fd, 0, length, 0, data) } private[this] object UringSubmissionQueue { @@ -526,58 +516,3 @@ object Encoder { def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = UserData.decode(res, flags, udata, callback) } - -/** Defines constants for various operation types supported by the io_uring interface. - */ -object OP { - val IORING_OP_NOP: Byte = 0 - val IORING_OP_READV: Byte = 1 - val IORING_OP_WRITEV: Byte = 2 - val IORING_OP_FSYNC: Byte = 3 - val IORING_OP_READ_FIXED: Byte = 4 - val IORING_OP_WRITE_FIXED: Byte = 5 - val IORING_OP_POLL_ADD: Byte = 6 - val IORING_OP_POLL_REMOVE: Byte = 7 - val IORING_OP_SYNC_FILE_RANGE: Byte = 8 - val IORING_OP_SENDMSG: Byte = 9 - val IORING_OP_RECVMSG: Byte = 10 - val IORING_OP_TIMEOUT: Byte = 11 - val IORING_OP_TIMEOUT_REMOVE: Byte = 12 - val IORING_OP_ACCEPT: Byte = 13 - val IORING_OP_ASYNC_CANCEL: Byte = 14 - val IORING_OP_LINK_TIMEOUT: Byte = 15 - val IORING_OP_CONNECT: Byte = 16 - val IORING_OP_FALLOCATE: Byte = 17 - val IORING_OP_OPENAT: Byte = 18 - val IORING_OP_CLOSE: Byte = 19 - val IORING_OP_FILES_UPDATE: Byte = 20 - val IORING_OP_STATX: Byte = 21 - val IORING_OP_READ: Byte = 22 - val IORING_OP_WRITE: Byte = 23 - val IORING_OP_FADVISE: Byte = 24 - val IORING_OP_MADVISE: Byte = 25 - val IORING_OP_SEND: Byte = 26 - val IORING_OP_RECV: Byte = 27 - val IORING_OP_OPENAT2: Byte = 28 - val IORING_OP_EPOLL_CTL: Byte = 29 - val IORING_OP_SPLICE: Byte = 30 - val IORING_OP_PROVIDE_BUFFERS: Byte = 31 - val IORING_OP_REMOVE_BUFFERS: Byte = 32 - val IORING_OP_TEE: Byte = 33 - val IORING_OP_SHUTDOWN: Byte = 34 - val IORING_OP_RENAMEAT: Byte = 35 - val IORING_OP_UNLINKAT: Byte = 36 - val IORING_OP_MKDIRAT: Byte = 37 - val IORING_OP_SYMLINKAT: Byte = 38 - val IORING_OP_LINKAT: Byte = 39 - val IORING_OP_MSG_RING: Byte = 40 - val IORING_OP_FSETXATTR: Byte = 41 - val IORING_OP_SETXATTR: Byte = 42 - val IORING_OP_FGETXATTR: Byte = 43 - val IORING_OP_GETXATTR: Byte = 44 - val IORING_OP_SOCKET: Byte = 45 - val IORING_OP_URING_CMD: Byte = 46 - val IORING_OP_SEND_ZC: Byte = 47 - val IORING_OP_SENDMSG_ZC: Byte = 48 - val IORING_OP_LAST: Byte = 49 -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 8ebb79b9..27d75ce3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -32,14 +32,17 @@ import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringCompletionQueueCallback -import io.netty.incubator.channel.uring.OP import io.netty.incubator.channel.uring.Encoder +import fs2.io.uring.unsafe.util.OP._ + import java.io.IOException import scala.collection.mutable.Map import java.util.BitSet +// import org.slf4j.LoggerFactory + object UringSystem extends PollingSystem { private final val MaxEvents = 64 @@ -214,11 +217,13 @@ object UringSystem extends PollingSystem { sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } - private[UringSystem] def cancel(opToCancel: Long, id: Short) = - enqueueSqe(OP.IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) + private[UringSystem] def cancel(opToCancel: Long, id: Short): Boolean = + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) + + private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = + enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) - private[UringSystem] def sendMsgRing(flags: Int, fd: Int, length: Int, data: Short): Boolean = - sq.sendMsgRing(flags, fd, length, data) + private[UringSystem] def getFd(): Int = ring.fd() private[UringSystem] def close(): Unit = ring.close() diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index 87d2e6c3..4c6b164a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -16,4 +16,61 @@ package fs2.io.uring.unsafe -private[uring] object util {} +private[uring] object util { + + /** Defines constants for various operation types supported by the io_uring interface. + */ + object OP { + val IORING_OP_NOP: Byte = 0 + val IORING_OP_READV: Byte = 1 + val IORING_OP_WRITEV: Byte = 2 + val IORING_OP_FSYNC: Byte = 3 + val IORING_OP_READ_FIXED: Byte = 4 + val IORING_OP_WRITE_FIXED: Byte = 5 + val IORING_OP_POLL_ADD: Byte = 6 + val IORING_OP_POLL_REMOVE: Byte = 7 + val IORING_OP_SYNC_FILE_RANGE: Byte = 8 + val IORING_OP_SENDMSG: Byte = 9 + val IORING_OP_RECVMSG: Byte = 10 + val IORING_OP_TIMEOUT: Byte = 11 + val IORING_OP_TIMEOUT_REMOVE: Byte = 12 + val IORING_OP_ACCEPT: Byte = 13 + val IORING_OP_ASYNC_CANCEL: Byte = 14 + val IORING_OP_LINK_TIMEOUT: Byte = 15 + val IORING_OP_CONNECT: Byte = 16 + val IORING_OP_FALLOCATE: Byte = 17 + val IORING_OP_OPENAT: Byte = 18 + val IORING_OP_CLOSE: Byte = 19 + val IORING_OP_FILES_UPDATE: Byte = 20 + val IORING_OP_STATX: Byte = 21 + val IORING_OP_READ: Byte = 22 + val IORING_OP_WRITE: Byte = 23 + val IORING_OP_FADVISE: Byte = 24 + val IORING_OP_MADVISE: Byte = 25 + val IORING_OP_SEND: Byte = 26 + val IORING_OP_RECV: Byte = 27 + val IORING_OP_OPENAT2: Byte = 28 + val IORING_OP_EPOLL_CTL: Byte = 29 + val IORING_OP_SPLICE: Byte = 30 + val IORING_OP_PROVIDE_BUFFERS: Byte = 31 + val IORING_OP_REMOVE_BUFFERS: Byte = 32 + val IORING_OP_TEE: Byte = 33 + val IORING_OP_SHUTDOWN: Byte = 34 + val IORING_OP_RENAMEAT: Byte = 35 + val IORING_OP_UNLINKAT: Byte = 36 + val IORING_OP_MKDIRAT: Byte = 37 + val IORING_OP_SYMLINKAT: Byte = 38 + val IORING_OP_LINKAT: Byte = 39 + val IORING_OP_MSG_RING: Byte = 40 + val IORING_OP_FSETXATTR: Byte = 41 + val IORING_OP_SETXATTR: Byte = 42 + val IORING_OP_FGETXATTR: Byte = 43 + val IORING_OP_GETXATTR: Byte = 44 + val IORING_OP_SOCKET: Byte = 45 + val IORING_OP_URING_CMD: Byte = 46 + val IORING_OP_SEND_ZC: Byte = 47 + val IORING_OP_SENDMSG_ZC: Byte = 48 + val IORING_OP_LAST: Byte = 49 + } + +} From d273a5e01d0d0669316efe17f7b01295bda79d1a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 20:04:58 +0200 Subject: [PATCH 056/200] Add default values to call and bracket --- .../src/main/scala/fs2/io/uring/Uring.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index 2bb3705e..a0bfcb88 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -25,22 +25,22 @@ abstract class Uring private[uring] { def call( op: Byte, - flags: Int, - rwFlags: Int, + flags: Int = 0, + rwFlags: Int = 0, fd: Int, - bufferAddress: Long, - length: Int, - offset: Long + bufferAddress: Long = 0, + length: Int = 0, + offset: Long = 0 ): IO[Int] def bracket( op: Byte, - flags: Int, - rwFlags: Int, + flags: Int = 0, + rwFlags: Int = 0, fd: Int, - bufferAddress: Long, - length: Int, - offset: Long + bufferAddress: Long = 0, + length: Int = 0, + offset: Long = 0 )(release: Int => IO[Unit]): Resource[IO, Int] } From c96fc7d44a4de4939851fefdf0b13f0111e5b361 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 20:05:12 +0200 Subject: [PATCH 057/200] Implement UringSocket --- .../scala/fs2/io/uring/net/UringSocket.scala | 134 ++++++++++++++++++ 1 file changed, 134 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala new file mode 100644 index 00000000..6fd16703 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -0,0 +1,134 @@ +package fs2.io.uring.net + +import cats.effect.LiftIO +import cats.effect.kernel.Async +import cats.effect.kernel.Resource +import cats.effect.kernel.Sync +import cats.effect.std.Mutex +import cats.syntax.all._ + +import com.comcast.ip4s.IpAddress +import com.comcast.ip4s.SocketAddress + +import fs2.Chunk +import fs2.Pipe +import fs2.Stream +import fs2.io.net.Socket + +import fs2.io.uring.Uring +import fs2.io.uring.unsafe.util.OP._ + +import io.netty.buffer.ByteBuf +import io.netty.buffer.UnpooledByteBufAllocator + +private[net] final class UringSocket[F[_]: LiftIO]( + ring: Uring, + sockfd: Int, + remoteAddress: SocketAddress[IpAddress], + buffer: ByteBuf, + defaultReadSize: Int, + readMutex: Mutex[F], + writeMutex: Mutex[F] +)(implicit F: Async[F]) + extends Socket[F] { + + private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = + ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + + def read(maxBytes: Int): F[Option[Chunk[Byte]]] = + readMutex.lock.surround { + for { + _ <- F.delay(buffer.clear()) + readed <- recv(buffer.memoryAddress(), 0, maxBytes, 0) + bytes <- F.delay { + val arr = new Array[Byte](readed) + buffer.getBytes(0, arr) + arr + } + } yield Option.when(readed > 0)(Chunk.array(bytes)) + } + + def readN(numBytes: Int): F[Chunk[Byte]] = + readMutex.lock.surround { + for { + _ <- F.delay(buffer.clear()) + readed <- recv( + buffer.memoryAddress(), + 0, + numBytes, + -1 + ) // TODO: Replace -1 with MSG_WAITALL + bytes <- F.delay { + val arr = new Array[Byte](readed) + buffer.getBytes(0, arr) + arr + } + } yield if (readed > 0) Chunk.array(bytes) else Chunk.empty + } + + def reads: Stream[F, Byte] = Stream.repeatEval(read(defaultReadSize)).unNoneTerminate.unchunks + + def endOfInput: F[Unit] = ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 0).void.to + + def endOfOutput: F[Unit] = ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 1).void.to + + def isOpen: F[Boolean] = F.pure(true) + + def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(remoteAddress) + + def localAddress: F[SocketAddress[IpAddress]] = UringSocket.getLocalAddress(sockfd) + + private[this] def send(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = + ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + def write(bytes: Chunk[Byte]): F[Unit] = + writeMutex.lock + .surround { + for { + _ <- F.delay { + buffer.clear() + buffer.writeBytes(bytes.toArray) + } + _ <- send( + buffer.memoryAddress(), + 0, + bytes.size, + -1 + ) // TODO: Replace -1 with MSG_NOSIGNAL + } yield () + } + .unlessA(bytes.isEmpty) + + def writes: Pipe[F, Byte, Nothing] = _.chunks.foreach(write) +} + +private[net] object UringSocket { + + def apply[F[_]: LiftIO](ring: Uring, fd: Int, remote: SocketAddress[IpAddress])(implicit + F: Async[F] + ): Resource[F, UringSocket[F]] = + for { + buffer <- createBuffer() + readMutex <- Resource.eval(Mutex[F]) + writeMutex <- Resource.eval(Mutex[F]) + socket = new UringSocket(ring, fd, remote, buffer, 8192, readMutex, writeMutex) + } yield socket + + /** TODO: We need to choose between heap or direct buffer and pooled or unpooled buffer: (I feel that Direct/Unpooled is the right combination) + * + * - Heap Buffer: Buffer is backed by a byte array located in the JVM's heap. Convenient if we work with API's that requires byte arrays. + * However, reading/writing from I/O channels requires copying data between the JVM heap and the Native heap which is slow. + * + * - Direct Buffer: Buffer is allocated on the Native heap. Read and writes from I/O channels can occur without copying any data which is faster. + * However, interacting with other Java APIs will require additional data copy. (REMEMBER: They are not subject to the JVM garbage collector, we have to free the memory) + * + * - Pooled Buffer: pre-allocated in memory and reused as needed. It is faster but consumes a lot of memory (we need to conserve a pool of buffers). + * + * - Unpooled Buffer: Allocated when we need them and deallocated when we are done. It may be slower but consume only the memory of the buffer that we are using. + */ + def createBuffer[F[_]: Sync](defaultReadSize: Int = 8192): Resource[F, ByteBuf] = + Resource.make( + Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(defaultReadSize)) + )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) + + def getLocalAddress[F[_]](fd: Int)(implicit F: Sync[F]): F[SocketAddress[IpAddress]] = ??? +} From 4cce052716ab8508e7184f9c4b8bab7d53eab37c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 20:19:20 +0200 Subject: [PATCH 058/200] Add TODO --- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 6fd16703..5c65990b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -130,5 +130,7 @@ private[net] object UringSocket { Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(defaultReadSize)) )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) - def getLocalAddress[F[_]](fd: Int)(implicit F: Sync[F]): F[SocketAddress[IpAddress]] = ??? + def getLocalAddress[F[_]](fd: Int)(implicit F: Sync[F]): F[SocketAddress[IpAddress]] = + /* TODO: Work on SocketAddressHelper before implementing this method */ + ??? } From c0281fa95efaba661f61dd5207f4b7a8828a7a82 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 22:08:15 +0200 Subject: [PATCH 059/200] Move the OP to util --- .../src/test/scala/fs2/io/uring/UringSystemSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 32555c7b..2da8958d 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -21,7 +21,7 @@ import cats.syntax.parallel._ import fs2.io.uring.UringSuite -import io.netty.incubator.channel.uring.OP +import fs2.io.uring.unsafe.util.OP._ class UringSystemSuite extends UringSuite { @@ -29,7 +29,7 @@ class UringSystemSuite extends UringSuite { Uring .get[IO] .flatMap { ring => - val op: Byte = OP.IORING_OP_NOP + val op: Byte = IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 @@ -44,7 +44,7 @@ class UringSystemSuite extends UringSuite { test("Parallel submission") { - val op: Byte = OP.IORING_OP_NOP + val op: Byte = IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 @@ -66,7 +66,7 @@ class UringSystemSuite extends UringSuite { } test("Multiple parallel submission") { - val op: Byte = OP.IORING_OP_NOP + val op: Byte = IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 val fd: Int = -1 From be3bf0fa009160235e1a2d76163ed36343a4ae5a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 22 Jul 2023 22:18:01 +0200 Subject: [PATCH 060/200] Add default value --- uring/jvm/src/main/scala/fs2/io/uring/Uring.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index a0bfcb88..6f28c22b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -27,7 +27,7 @@ abstract class Uring private[uring] { op: Byte, flags: Int = 0, rwFlags: Int = 0, - fd: Int, + fd: Int = 0, bufferAddress: Long = 0, length: Int = 0, offset: Long = 0 @@ -37,7 +37,7 @@ abstract class Uring private[uring] { op: Byte, flags: Int = 0, rwFlags: Int = 0, - fd: Int, + fd: Int = 0, bufferAddress: Long = 0, length: Int = 0, offset: Long = 0 From ecb3e5856ec4c21a4d7d3f67d4965e4fee8dc3ee Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 24 Jul 2023 19:00:25 +0200 Subject: [PATCH 061/200] Add MsgHdr and MsgHdrArray classes --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 2db298c8..4a2ad876 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -516,3 +516,43 @@ object Encoder { def decode(res: Int, flags: Int, udata: Long, callback: IOUringCompletionQueueCallback) = UserData.decode(res, flags, udata, callback) } + +final class UringMsgHdr { + def write( + memoryAddress: Long, + address: Long, + addressSize: Int, + iovAddress: Long, + iovLength: Int, + msgControlAddr: Long, + cmsgHdrDataAddress: Long, + segmentSize: Short + ): Unit = + MsgHdr.write( + memoryAddress, + address, + addressSize, + iovAddress, + iovLength, + msgControlAddr, + cmsgHdrDataAddress, + segmentSize + ) +} + +final class UringMsgHdrMemoryArray(capacity: Int) { + private[this] val msgHdrMemoryArray: MsgHdrMemoryArray = new MsgHdrMemoryArray(capacity) + + def clear(): Unit = msgHdrMemoryArray.clear() + + def capacity(): Int = msgHdrMemoryArray.capacity() + + def length(): Int = msgHdrMemoryArray.length() + + def release(): Unit = msgHdrMemoryArray.release() + + def hdr(idx: Int): MsgHdrMemory = msgHdrMemoryArray.hdr(idx) + + def nextHdr(): MsgHdrMemory = msgHdrMemoryArray.nextHdr() + +} From d52b7e7b5fa0cdd7672f2fb1cf9b75cbe4207857 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 24 Jul 2023 23:01:32 +0200 Subject: [PATCH 062/200] Add Iov class --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 4a2ad876..fcadfab5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -541,18 +541,27 @@ final class UringMsgHdr { } final class UringMsgHdrMemoryArray(capacity: Int) { - private[this] val msgHdrMemoryArray: MsgHdrMemoryArray = new MsgHdrMemoryArray(capacity) + private[this] val msgHdrMemoryArray: MsgHdrMemoryArray = new MsgHdrMemoryArray(capacity) - def clear(): Unit = msgHdrMemoryArray.clear() + def clear(): Unit = msgHdrMemoryArray.clear() - def capacity(): Int = msgHdrMemoryArray.capacity() + def capacity(): Int = msgHdrMemoryArray.capacity() - def length(): Int = msgHdrMemoryArray.length() + def length(): Int = msgHdrMemoryArray.length() - def release(): Unit = msgHdrMemoryArray.release() + def release(): Unit = msgHdrMemoryArray.release() - def hdr(idx: Int): MsgHdrMemory = msgHdrMemoryArray.hdr(idx) + def hdr(idx: Int): MsgHdrMemory = msgHdrMemoryArray.hdr(idx) - def nextHdr(): MsgHdrMemory = msgHdrMemoryArray.nextHdr() + def nextHdr(): MsgHdrMemory = msgHdrMemoryArray.nextHdr() } + +final class UringIov() { + def write(iovAddress: Long, bufferAddress: Long, length: Int): Unit = + Iov.write(iovAddress, bufferAddress, length) + + def readBufferAddress(iovAddress: Long): Long = Iov.readBufferAddress(iovAddress) + + def readBufferLength(iovAddress: Long): Int = Iov.readBufferLength(iovAddress) +} From 7fa8b053cd5de02dc6459510b6f9774c38ad9cd6 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 15:56:32 +0200 Subject: [PATCH 063/200] Add LinuxSocket --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index fcadfab5..673fdd9c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -18,6 +18,8 @@ package io.netty.incubator.channel.uring import io.netty.channel.unix.FileDescriptor import NativeAccess._ +import java.net.InetSocketAddress +import io.netty.channel.socket.InternetProtocolFamily /** Represents an io_uring Ring with both Submission Queue (SQ) and Completion Queue (CQ). * @@ -565,3 +567,16 @@ final class UringIov() { def readBufferLength(iovAddress: Long): Int = Iov.readBufferLength(iovAddress) } + +final class UringLinuxSocket(fd: Int) { + val socket: LinuxSocket = new LinuxSocket(fd) + + def getLocalAddress(): InetSocketAddress = socket.localAddress() + + def getRemoteAddress(): InetSocketAddress = socket.remoteAddress() + + def family(): InternetProtocolFamily = socket.family() + + def isIpv6(): Boolean = socket.isIpv6() + +} From 37e713a7211cea6b77e077223d4f1e7862bb18f1 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 16:16:40 +0200 Subject: [PATCH 064/200] Implement localAddress --- .../scala/fs2/io/uring/net/UringSocket.scala | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 5c65990b..24c81557 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package fs2.io.uring.net import cats.effect.LiftIO @@ -20,6 +36,7 @@ import fs2.io.uring.unsafe.util.OP._ import io.netty.buffer.ByteBuf import io.netty.buffer.UnpooledByteBufAllocator +import io.netty.incubator.channel.uring.UringLinuxSocket private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, @@ -32,6 +49,8 @@ private[net] final class UringSocket[F[_]: LiftIO]( )(implicit F: Async[F]) extends Socket[F] { + private[this] val socket: UringLinuxSocket = new UringLinuxSocket(sockfd) + private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to @@ -76,7 +95,8 @@ private[net] final class UringSocket[F[_]: LiftIO]( def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(remoteAddress) - def localAddress: F[SocketAddress[IpAddress]] = UringSocket.getLocalAddress(sockfd) + def localAddress: F[SocketAddress[IpAddress]] = + F.delay(SocketAddress.fromInetSocketAddress(socket.getLocalAddress())) private[this] def send(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to @@ -129,8 +149,4 @@ private[net] object UringSocket { Resource.make( Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(defaultReadSize)) )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) - - def getLocalAddress[F[_]](fd: Int)(implicit F: Sync[F]): F[SocketAddress[IpAddress]] = - /* TODO: Work on SocketAddressHelper before implementing this method */ - ??? } From 49ba2b148bd13d1007f3b6a5df8d41d5d8dbe9fa Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 17:27:15 +0200 Subject: [PATCH 065/200] Add Companion Object for LinuxSocket --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 673fdd9c..a70b8be1 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -568,8 +568,7 @@ final class UringIov() { def readBufferLength(iovAddress: Long): Int = Iov.readBufferLength(iovAddress) } -final class UringLinuxSocket(fd: Int) { - val socket: LinuxSocket = new LinuxSocket(fd) +final class UringLinuxSocket(private[this] val socket: LinuxSocket) { def getLocalAddress(): InetSocketAddress = socket.localAddress() @@ -580,3 +579,18 @@ final class UringLinuxSocket(fd: Int) { def isIpv6(): Boolean = socket.isIpv6() } + +object UringLinuxSocket { + def newSocketStream(): UringLinuxSocket = new UringLinuxSocket(LinuxSocket.newSocketStream()) + + def newSocketStream(ipv6: Boolean): UringLinuxSocket = new UringLinuxSocket( + LinuxSocket.newSocketStream(ipv6) + ) + + def newSocketDatagram(): UringLinuxSocket = new UringLinuxSocket(LinuxSocket.newSocketDgram()) + + def newSocketDatagram(ipv6: Boolean): UringLinuxSocket = new UringLinuxSocket( + LinuxSocket.newSocketDgram(ipv6) + ) + +} From 2215a4d0c23a57a1a1aa77194145bc0388818c4a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 18:51:48 +0200 Subject: [PATCH 066/200] Add fd method in LinuxSocket --- .../main/scala/fs2/io/uring/net/UringSocket.scala | 12 +++++------- .../main/scala/fs2/io/uring/unsafe/UringRing.scala | 2 ++ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 24c81557..9470e44a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -40,6 +40,7 @@ import io.netty.incubator.channel.uring.UringLinuxSocket private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, + linuxSocket: UringLinuxSocket, sockfd: Int, remoteAddress: SocketAddress[IpAddress], buffer: ByteBuf, @@ -48,12 +49,9 @@ private[net] final class UringSocket[F[_]: LiftIO]( writeMutex: Mutex[F] )(implicit F: Async[F]) extends Socket[F] { - - private[this] val socket: UringLinuxSocket = new UringLinuxSocket(sockfd) - + private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to - def read(maxBytes: Int): F[Option[Chunk[Byte]]] = readMutex.lock.surround { for { @@ -96,7 +94,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(remoteAddress) def localAddress: F[SocketAddress[IpAddress]] = - F.delay(SocketAddress.fromInetSocketAddress(socket.getLocalAddress())) + F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) private[this] def send(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to @@ -123,14 +121,14 @@ private[net] final class UringSocket[F[_]: LiftIO]( private[net] object UringSocket { - def apply[F[_]: LiftIO](ring: Uring, fd: Int, remote: SocketAddress[IpAddress])(implicit + def apply[F[_]: LiftIO](ring: Uring, linuxSocket: UringLinuxSocket, fd: Int, remote: SocketAddress[IpAddress])(implicit F: Async[F] ): Resource[F, UringSocket[F]] = for { buffer <- createBuffer() readMutex <- Resource.eval(Mutex[F]) writeMutex <- Resource.eval(Mutex[F]) - socket = new UringSocket(ring, fd, remote, buffer, 8192, readMutex, writeMutex) + socket = new UringSocket(ring, linuxSocket, fd, remote, buffer, 8192, readMutex, writeMutex) } yield socket /** TODO: We need to choose between heap or direct buffer and pooled or unpooled buffer: (I feel that Direct/Unpooled is the right combination) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index a70b8be1..209c3c41 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -578,6 +578,8 @@ final class UringLinuxSocket(private[this] val socket: LinuxSocket) { def isIpv6(): Boolean = socket.isIpv6() + def fd(): Int = socket.intValue() + } object UringLinuxSocket { From a518d30aed0eacb6faba387f5adfd35c405bdb4c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 18:52:20 +0200 Subject: [PATCH 067/200] Add UringNetwork --- .../scala/fs2/io/uring/net/UringNetwork.scala | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala new file mode 100644 index 00000000..1740a337 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -0,0 +1,87 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring.net + +import cats.effect.LiftIO +import cats.effect.kernel.Async +import cats.effect.kernel.Resource + +import com.comcast.ip4s.Dns +import com.comcast.ip4s.Host +import com.comcast.ip4s.Port +import com.comcast.ip4s.SocketAddress +import com.comcast.ip4s.IpAddress + +import fs2.Stream +import fs2.io.net.Network +import fs2.io.net.SocketOption +import fs2.io.net.tls.TLSContext +import fs2.io.net.SocketGroup +import fs2.io.net.Socket +import fs2.io.net.DatagramSocket +import fs2.io.net.DatagramSocketGroup +import fs2.io.net.DatagramSocketOption + +import java.net.ProtocolFamily + +import java.util.concurrent.ThreadFactory + +private[net] final class UringNetwork[F[_]]( + sg: UringSocketGroup[F], + dsg: UringDatagramSocketGroup[F], + val tlsContext: TLSContext.Builder[F] +) extends Network.UnsealedNetwork[F] { + + def socketGroup(threadCount: Int, threadFactory: ThreadFactory): Resource[F, SocketGroup[F]] = + Resource.pure[F, SocketGroup[F]](sg) + + def datagramSocketGroup(threadFactory: ThreadFactory): Resource[F, DatagramSocketGroup[F]] = + Resource.pure[F, DatagramSocketGroup[F]](dsg) + + def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = + sg.client(to, options) + + def server( + address: Option[Host], + port: Option[Port], + options: List[SocketOption] + ): Stream[F, Socket[F]] = sg.server(address, port, options) + + def serverResource( + address: Option[Host], + port: Option[Port], + options: List[SocketOption] + ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = + sg.serverResource(address, port, options) + + def openDatagramSocket( + address: Option[Host], + port: Option[Port], + options: List[DatagramSocketOption], + protocolFamily: Option[ProtocolFamily] + ): Resource[F, DatagramSocket[F]] = + dsg.openDatagramSocket(address, port, options, protocolFamily) +} + +object UringNetwork { + def apply[F[_]: Async: Dns: LiftIO]: Network[F] = + new UringNetwork( + new UringSocketGroup[F], + new UringDatagramSocketGroup[F], + TLSContext.Builder.forAsync[F] + ) +} From 59fa37ca476130321af08f1b1e0c13ee07f9e038 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 18:52:37 +0200 Subject: [PATCH 068/200] Add TcpSocket Suite --- .../fs2/io/uring/net/TcpSocketSuite.scala | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala new file mode 100644 index 00000000..3d9e9cfb --- /dev/null +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -0,0 +1,54 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring.net + +import cats.effect.IO + +import com.comcast.ip4s._ + +import fs2.Stream +import fs2.text._ + +import fs2.io.uring.UringSuite + +class TcpSocketSuite extends UringSuite { + + val sg = UringSocketGroup[IO] + + test("postman echo") { + sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => + val msg = + """|GET /get HTTP/1.1 + |Host: postman-echo.com + | + |""".stripMargin + + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .head + + writeRead.compile.lastOrError + .assertEquals("HTTP/1.1 200 OK") + } + } + +} From ca46d74d56f9153c27ac1c5d9b0476963f2c557d Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 18:53:18 +0200 Subject: [PATCH 069/200] Reformat UringSocket --- .../src/main/scala/fs2/io/uring/net/UringSocket.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 9470e44a..684bf155 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -49,7 +49,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( writeMutex: Mutex[F] )(implicit F: Async[F]) extends Socket[F] { - + private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to def read(maxBytes: Int): F[Option[Chunk[Byte]]] = @@ -121,7 +121,12 @@ private[net] final class UringSocket[F[_]: LiftIO]( private[net] object UringSocket { - def apply[F[_]: LiftIO](ring: Uring, linuxSocket: UringLinuxSocket, fd: Int, remote: SocketAddress[IpAddress])(implicit + def apply[F[_]: LiftIO]( + ring: Uring, + linuxSocket: UringLinuxSocket, + fd: Int, + remote: SocketAddress[IpAddress] + )(implicit F: Async[F] ): Resource[F, UringSocket[F]] = for { From ff89a44c10a61ea48f05c9e92a2d9d4dbc195952 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 20:39:36 +0200 Subject: [PATCH 070/200] Add SockaddrIn --- .../jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 209c3c41..ece6bdac 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -406,6 +406,8 @@ trait UringCompletionQueueCallback extends IOUringCompletionQueueCallback { */ object NativeAccess { + val SIZEOF_SOCKADDR_IN6 = Native.SIZEOF_SOCKADDR_IN6 + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ @@ -596,3 +598,7 @@ object UringLinuxSocket { ) } + +object UringSockaddrIn { + def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = SockaddrIn.write(ipv6, memory, address) +} From dd1f553ab83b665d7ceec069b3fbd43002e11a87 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 20:41:10 +0200 Subject: [PATCH 071/200] Move createBuffer to util --- .../scala/fs2/io/uring/net/UringSocket.scala | 22 +++---------------- .../main/scala/fs2/io/uring/unsafe/util.scala | 22 +++++++++++++++++++ 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 684bf155..337685c3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -19,7 +19,6 @@ package fs2.io.uring.net import cats.effect.LiftIO import cats.effect.kernel.Async import cats.effect.kernel.Resource -import cats.effect.kernel.Sync import cats.effect.std.Mutex import cats.syntax.all._ @@ -32,10 +31,10 @@ import fs2.Stream import fs2.io.net.Socket import fs2.io.uring.Uring +import fs2.io.uring.unsafe.util.createBuffer import fs2.io.uring.unsafe.util.OP._ import io.netty.buffer.ByteBuf -import io.netty.buffer.UnpooledByteBufAllocator import io.netty.incubator.channel.uring.UringLinuxSocket private[net] final class UringSocket[F[_]: LiftIO]( @@ -120,6 +119,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( } private[net] object UringSocket { + private[this] val defaultReadSize = 8192 def apply[F[_]: LiftIO]( ring: Uring, @@ -130,26 +130,10 @@ private[net] object UringSocket { F: Async[F] ): Resource[F, UringSocket[F]] = for { - buffer <- createBuffer() + buffer <- createBuffer(defaultReadSize) readMutex <- Resource.eval(Mutex[F]) writeMutex <- Resource.eval(Mutex[F]) socket = new UringSocket(ring, linuxSocket, fd, remote, buffer, 8192, readMutex, writeMutex) } yield socket - /** TODO: We need to choose between heap or direct buffer and pooled or unpooled buffer: (I feel that Direct/Unpooled is the right combination) - * - * - Heap Buffer: Buffer is backed by a byte array located in the JVM's heap. Convenient if we work with API's that requires byte arrays. - * However, reading/writing from I/O channels requires copying data between the JVM heap and the Native heap which is slow. - * - * - Direct Buffer: Buffer is allocated on the Native heap. Read and writes from I/O channels can occur without copying any data which is faster. - * However, interacting with other Java APIs will require additional data copy. (REMEMBER: They are not subject to the JVM garbage collector, we have to free the memory) - * - * - Pooled Buffer: pre-allocated in memory and reused as needed. It is faster but consumes a lot of memory (we need to conserve a pool of buffers). - * - * - Unpooled Buffer: Allocated when we need them and deallocated when we are done. It may be slower but consume only the memory of the buffer that we are using. - */ - def createBuffer[F[_]: Sync](defaultReadSize: Int = 8192): Resource[F, ByteBuf] = - Resource.make( - Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(defaultReadSize)) - )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index 4c6b164a..f355d07b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -16,8 +16,30 @@ package fs2.io.uring.unsafe +import cats.effect.kernel.Sync +import cats.effect.kernel.Resource +import io.netty.buffer.UnpooledByteBufAllocator +import io.netty.buffer.ByteBuf + private[uring] object util { + /** TODO: We need to choose between heap or direct buffer and pooled or unpooled buffer: (I feel that Direct/Unpooled is the right combination) + * + * - Heap Buffer: Buffer is backed by a byte array located in the JVM's heap. Convenient if we work with API's that requires byte arrays. + * However, reading/writing from I/O channels requires copying data between the JVM heap and the Native heap which is slow. + * + * - Direct Buffer: Buffer is allocated on the Native heap. Read and writes from I/O channels can occur without copying any data which is faster. + * However, interacting with other Java APIs will require additional data copy. (REMEMBER: They are not subject to the JVM garbage collector, we have to free the memory) + * + * - Pooled Buffer: pre-allocated in memory and reused as needed. It is faster but consumes a lot of memory (we need to conserve a pool of buffers). + * + * - Unpooled Buffer: Allocated when we need them and deallocated when we are done. It may be slower but consume only the memory of the buffer that we are using. + */ + def createBuffer[F[_]: Sync](size: Int): Resource[F, ByteBuf] = + Resource.make( + Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(size)) + )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) + /** Defines constants for various operation types supported by the io_uring interface. */ object OP { From f8f331b03482c001087a676b72e7a0a8c1564eae Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 20:41:52 +0200 Subject: [PATCH 072/200] Add SocketGroup and Implement client and server --- .../fs2/io/uring/net/UringSocketGroup.scala | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala new file mode 100644 index 00000000..e1574caf --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -0,0 +1,93 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fs2.io.uring.net + +import cats.effect.IO +import cats.effect.LiftIO +import cats.effect.kernel.Async +import cats.effect.kernel.Resource +import cats.syntax.all._ + +import com.comcast.ip4s._ + +import fs2.Stream +import fs2.io.net.Socket +import fs2.io.net.SocketGroup +import fs2.io.net.SocketOption + +import fs2.io.uring.Uring +import fs2.io.uring.unsafe.util.createBuffer +import fs2.io.uring.unsafe.util.OP._ + +import io.netty.incubator.channel.uring.UringSockaddrIn +import io.netty.incubator.channel.uring.UringLinuxSocket +import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 + +import java.net.Inet6Address + +private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) + extends SocketGroup[F] { + def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = + Resource.eval(Uring.get[F]).flatMap { ring => + Resource.eval(to.resolve).flatMap { address => + openSocket(ring, address.host.isInstanceOf[Ipv6Address]).flatMap { linuxSocket => + Resource.eval { + createBuffer(SIZEOF_SOCKADDR_IN6).use { buf => + val length: Int = UringSockaddrIn.write( + address.toInetSocketAddress.getAddress.isInstanceOf[Inet6Address], + buf.memoryAddress(), + address.toInetSocketAddress + ) + ring + .call(IORING_OP_CONNECT, 0, 0, linuxSocket.fd(), buf.memoryAddress(), length, 0) + .to + + } + } *> UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + } + } + } + + def server( + address: Option[Host], + port: Option[Port], + options: List[SocketOption] + ): Stream[F, Socket[F]] = Stream.resource(serverResource(address, port, options)).flatMap(_._2) + + def serverResource( + address: Option[Host], + port: Option[Port], + options: List[SocketOption] + ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = + ??? + + private def openSocket( + ring: Uring, + ipv6: Boolean + ): Resource[F, UringLinuxSocket] = + Resource.make[F, UringLinuxSocket](F.delay(UringLinuxSocket.newSocketStream(ipv6)))( + linuxSocket => closeSocket(ring, linuxSocket.fd()).to + ) + + private def closeSocket(ring: Uring, fd: Int): IO[Unit] = + ring.call(op = IORING_OP_CLOSE, fd = fd).void + +} + +object UringSocketGroup { + def apply[F[_]: Async: Dns: LiftIO]: SocketGroup[F] = new UringSocketGroup +} From f19997f5c665adfe6e950f1cddb7b64d7ef86a2c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 25 Jul 2023 20:51:26 +0200 Subject: [PATCH 073/200] Add DatagramSocketGroup --- .../uring/net/UringDatagramSocketGroup.scala | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala new file mode 100644 index 00000000..1441f2b3 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala @@ -0,0 +1,38 @@ +package fs2.io.uring.net + +import cats.effect.IO +import cats.effect.LiftIO +import cats.effect.kernel.Async +import cats.effect.kernel.Resource +import cats.syntax.all._ + +import com.comcast.ip4s._ + +import fs2.Stream +import fs2.io.net.Socket +import fs2.io.net.SocketGroup +import fs2.io.net.SocketOption +import fs2.io.net.DatagramSocketGroup +import fs2.io.net.DatagramSocket +import fs2.io.net._ + +import fs2.io.uring.Uring +import fs2.io.uring.unsafe.util.OP._ + +import java.net.ProtocolFamily + +private final class UringDatagramSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) + extends DatagramSocketGroup[F] { + + override def openDatagramSocket( + address: Option[Host], + port: Option[Port], + options: List[DatagramSocketOption], + protocolFamily: Option[ProtocolFamily] + ): Resource[F, DatagramSocket[F]] = ??? + +} + +object UringDatagramSocketGroup { + def apply[F[_]: Async: Dns: LiftIO]: DatagramSocketGroup[F] = new UringDatagramSocketGroup +} From 1714e86c5e85e073a32e34264c3093a564db1dc0 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 25 Jul 2023 20:52:26 +0000 Subject: [PATCH 074/200] Echo test working --- .../scala/fs2/io/uring/net/UringSocket.scala | 8 +++--- .../fs2/io/uring/net/UringSocketGroup.scala | 28 +++++++++++++++---- .../scala/fs2/io/uring/unsafe/UringRing.scala | 5 +++- 3 files changed, 30 insertions(+), 11 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 337685c3..fe8bdc49 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -72,8 +72,8 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.memoryAddress(), 0, numBytes, - -1 - ) // TODO: Replace -1 with MSG_WAITALL + 0 // TODO: Replace this with MSG_WAITALL + ) bytes <- F.delay { val arr = new Array[Byte](readed) buffer.getBytes(0, arr) @@ -109,8 +109,8 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.memoryAddress(), 0, bytes.size, - -1 - ) // TODO: Replace -1 with MSG_NOSIGNAL + 0 // TODO Replace this with MSG_NOSIGNAL + ) } yield () } .unlessA(bytes.isEmpty) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index e1574caf..6af8f577 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -35,27 +35,43 @@ import fs2.io.uring.unsafe.util.OP._ import io.netty.incubator.channel.uring.UringSockaddrIn import io.netty.incubator.channel.uring.UringLinuxSocket +import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 -import java.net.Inet6Address +import io.netty.buffer.ByteBuf private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { + + private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = + if (isIpv6) createBuffer(SIZEOF_SOCKADDR_IN6) else createBuffer(SIZEOF_SOCKADDR_IN) def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = Resource.eval(Uring.get[F]).flatMap { ring => Resource.eval(to.resolve).flatMap { address => - openSocket(ring, address.host.isInstanceOf[Ipv6Address]).flatMap { linuxSocket => + val isIpv6: Boolean = address.host.isInstanceOf[Ipv6Address] + openSocket(ring, isIpv6).flatMap { linuxSocket => Resource.eval { - createBuffer(SIZEOF_SOCKADDR_IN6).use { buf => + createBufferAux(isIpv6).use { buf => val length: Int = UringSockaddrIn.write( - address.toInetSocketAddress.getAddress.isInstanceOf[Inet6Address], + isIpv6, buf.memoryAddress(), address.toInetSocketAddress ) + + println( + s"[CLIENT] address: ${address.toString()}, buffer: ${buf.toString()}, length: $length" + ) ring - .call(IORING_OP_CONNECT, 0, 0, linuxSocket.fd(), buf.memoryAddress(), length, 0) + .call( + IORING_OP_CONNECT, + 0, + 0, + linuxSocket.fd(), + buf.memoryAddress(), + 0, + length.toLong + ) .to - } } *> UringSocket(ring, linuxSocket, linuxSocket.fd(), address) } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index ece6bdac..ce33a5db 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -408,6 +408,8 @@ object NativeAccess { val SIZEOF_SOCKADDR_IN6 = Native.SIZEOF_SOCKADDR_IN6 + val SIZEOF_SOCKADDR_IN = Native.SIZEOF_SOCKADDR_IN + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ @@ -600,5 +602,6 @@ object UringLinuxSocket { } object UringSockaddrIn { - def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = SockaddrIn.write(ipv6, memory, address) + def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = + SockaddrIn.write(ipv6, memory, address) } From 3abe875f4032c4d620d8dcee12a5ee3461891824 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 26 Jul 2023 15:14:14 +0200 Subject: [PATCH 075/200] Add bind, listen, accept and apply for LinuxSocket --- .../src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index ce33a5db..9ed32235 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -19,6 +19,7 @@ package io.netty.incubator.channel.uring import io.netty.channel.unix.FileDescriptor import NativeAccess._ import java.net.InetSocketAddress +import java.net.SocketAddress import io.netty.channel.socket.InternetProtocolFamily /** Represents an io_uring Ring with both Submission Queue (SQ) and Completion Queue (CQ). @@ -584,9 +585,17 @@ final class UringLinuxSocket(private[this] val socket: LinuxSocket) { def fd(): Int = socket.intValue() + def bind(socketAddress: SocketAddress): Unit = socket.bind(socketAddress) + + def listen(backlog: Int): Unit = socket.listen(backlog) + + def accept(addr: Array[Byte]): Int = socket.accept(addr) + } object UringLinuxSocket { + def apply(fd: Int): UringLinuxSocket = new UringLinuxSocket(new LinuxSocket(fd)) + def newSocketStream(): UringLinuxSocket = new UringLinuxSocket(LinuxSocket.newSocketStream()) def newSocketStream(ipv6: Boolean): UringLinuxSocket = new UringLinuxSocket( From 7f739ebae83f82c756176023c991d0f5efc96f9a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 27 Jul 2023 13:35:40 +0200 Subject: [PATCH 076/200] Implement serverResource --- .../fs2/io/uring/net/UringSocketGroup.scala | 70 ++++++++++++++++++- 1 file changed, 68 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 6af8f577..56d821d9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -39,6 +39,7 @@ import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 import io.netty.buffer.ByteBuf +import java.net.InetSocketAddress private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { @@ -73,7 +74,12 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) .to } - } *> UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + } *> UringSocket( + ring, + linuxSocket, + linuxSocket.fd(), + address + ) } } } @@ -89,7 +95,67 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn port: Option[Port], options: List[SocketOption] ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = - ??? + Resource.eval(Uring.get[F]).flatMap { ring => + for { + resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) + + isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] + + linuxSocket <- openSocket(ring, isIpv6) + + localAddress <- Resource.eval { + val bindF = F.delay { + val socketAddress = + new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) + linuxSocket.bind(socketAddress) + } + + val listenF = F.delay(linuxSocket.listen(65535)) + + bindF *> listenF *> F + .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) + } + + sockets = Stream + .resource(createBufferAux(isIpv6)) + .flatMap { buf => + Stream.resource { + val accept = + ring + .bracket( + IORING_OP_ACCEPT, + 0, + 0, + linuxSocket.fd(), + buf.memoryAddress(), + 0, + buf.capacity().toLong + )(closeSocket(ring, _)) + .mapK(LiftIO.liftK) + + val hostAndPort = buf.toString().split(":") + val host = hostAndPort(0) + val port = hostAndPort(1).toInt + val socketAddress = new InetSocketAddress(host, port) + + val convert: F[SocketAddress[IpAddress]] = F + .delay( + SocketAddress.fromInetSocketAddress(socketAddress) + ) + + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } + } + .attempt + .map(_.toOption) + }.repeat + } + + } yield (localAddress, sockets.unNone) + } private def openSocket( ring: Uring, From 6b4940ae7184495f3fcf57f4549633aff96608c5 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 27 Jul 2023 13:36:18 +0200 Subject: [PATCH 077/200] Add tests for serverResource --- .../fs2/io/uring/net/TcpSocketSuite.scala | 122 ++++++++++++++++++ 1 file changed, 122 insertions(+) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 3d9e9cfb..1ad168da 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -21,6 +21,7 @@ import cats.effect.IO import com.comcast.ip4s._ import fs2.Stream +import fs2.Chunk import fs2.text._ import fs2.io.uring.UringSuite @@ -51,4 +52,125 @@ class TcpSocketSuite extends UringSuite { } } + val setup = for { + serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) + (bindAddress, server) = serverSetup + clients = Stream.resource(sg.client(bindAddress)).repeat + } yield server -> clients + + test("echo requests - each concurrent client gets back what it sent") { + val message = Chunk.array("fs2.rocks".getBytes) + val clientCount = 20L + + Stream + .resource(setup) + .flatMap { case (server, clients) => + val echoServer = server.map { socket => + socket.reads + .through(socket.writes) + .onFinalize(socket.endOfOutput) + }.parJoinUnbounded + + val msgClients = clients + .take(clientCount) + .map { socket => + Stream + .chunk(message) + .through(socket.writes) + .onFinalize(socket.endOfOutput) ++ + socket.reads.chunks + .map(bytes => new String(bytes.toArray)) + } + .parJoin(10) + .take(clientCount) + + msgClients.concurrently(echoServer) + } + .compile + .toVector + .map { it => + assertEquals(it.size.toLong, clientCount) + assert(it.forall(_ == "fs2.rocks")) + } + } + + test("readN yields chunks of the requested size") { + val message = Chunk.array("123456789012345678901234567890".getBytes) + val sizes = Vector(1, 2, 3, 4, 3, 2, 1) + + Stream + .resource(setup) + .flatMap { case (server, clients) => + val junkServer = server.map { socket => + Stream + .chunk(message) + .through(socket.writes) + .onFinalize(socket.endOfOutput) + }.parJoinUnbounded + + val client = + clients + .take(1) + .flatMap { socket => + Stream + .emits(sizes) + .evalMap(socket.readN(_)) + .map(_.size) + } + .take(sizes.length.toLong) + + client.concurrently(junkServer) + } + .compile + .toVector + .assertEquals(sizes) + } + + test("write - concurrent calls do not cause a WritePendingException") { + val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) + + Stream + .resource(setup) + .flatMap { case (server, clients) => + val readOnlyServer = server.map(_.reads).parJoinUnbounded + val client = + clients.take(1).flatMap { socket => + // concurrent writes + Stream { + Stream.eval(socket.write(message)).repeatN(10L) + }.repeatN(2L).parJoinUnbounded + } + + client.concurrently(readOnlyServer) + } + .compile + .drain + } + + test("addresses - should match across client and server sockets") { + Stream + .resource(setup) + .flatMap { case (server, clients) => + val serverSocketAddresses = server.evalMap { socket => + socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) + } + + val clientSocketAddresses = + clients + .take(1) + .evalMap { socket => + socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) + } + + serverSocketAddresses.parZip(clientSocketAddresses).map { + case ((serverLocal, serverRemote), (clientLocal, clientRemote)) => + assertEquals(clientRemote, serverLocal) + assertEquals(clientLocal, serverRemote) + } + + } + .compile + .drain + } + } From c73845929b9b8504920de46236e253a600a9ec09 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 27 Jul 2023 16:25:39 +0000 Subject: [PATCH 078/200] Add readIPv4 --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 9ed32235..b515df81 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -411,6 +411,8 @@ object NativeAccess { val SIZEOF_SOCKADDR_IN = Native.SIZEOF_SOCKADDR_IN + val SOCK_NONBLOCK = Native.SOCK_NONBLOCK + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ @@ -613,4 +615,7 @@ object UringLinuxSocket { object UringSockaddrIn { def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = SockaddrIn.write(ipv6, memory, address) + + def readIPv4(memory: Long, tmpArray: Array[Byte]): InetSocketAddress = + SockaddrIn.readIPv4(memory, tmpArray) } From 1df8f92353d74bcfb703a19852e575e2a10cf9b7 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 27 Jul 2023 16:26:43 +0000 Subject: [PATCH 079/200] Correct serverResource --- .../fs2/io/uring/net/UringSocketGroup.scala | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 56d821d9..85abbab1 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -40,6 +40,7 @@ import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 import io.netty.buffer.ByteBuf import java.net.InetSocketAddress +import io.netty.incubator.channel.uring.NativeAccess private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { @@ -99,6 +100,8 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn for { resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) + _ <- Resource.eval(F.delay(println(s"Resolved Address: $resolvedAddress"))) + isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] linuxSocket <- openSocket(ring, isIpv6) @@ -116,16 +119,19 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) } + _ <- Resource.eval(F.delay(println(s"Local Address: $localAddress"))) + sockets = Stream .resource(createBufferAux(isIpv6)) .flatMap { buf => Stream.resource { + val accept = ring .bracket( IORING_OP_ACCEPT, 0, - 0, + NativeAccess.SOCK_NONBLOCK, linuxSocket.fd(), buf.memoryAddress(), 0, @@ -133,24 +139,27 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn )(closeSocket(ring, _)) .mapK(LiftIO.liftK) - val hostAndPort = buf.toString().split(":") - val host = hostAndPort(0) - val port = hostAndPort(1).toInt - val socketAddress = new InetSocketAddress(host, port) + val test = UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) + val socketAddress = new InetSocketAddress(test.getHostString(), test.getPort()) val convert: F[SocketAddress[IpAddress]] = F .delay( SocketAddress.fromInetSocketAddress(socketAddress) ) - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + Resource.eval(F.delay(println(s"Socket Address: $socketAddress"))) *> + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + Resource + .eval(F.delay(println(s"NOW WE SHOULD HAVE THE ADDRESS: $remoteAddress"))) + .flatMap { _ => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } + } } - } - .attempt - .map(_.toOption) + .attempt + .map(_.toOption) }.repeat } From c9d9fe4a2ad2357a7131bcf9748927ff67df3f03 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 27 Jul 2023 16:27:27 +0000 Subject: [PATCH 080/200] Reserve bit 0 to timeout --- .../main/scala/fs2/io/uring/unsafe/UringSystem.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 27d75ce3..305e24a5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -47,7 +47,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false // True to printout operations + private val debug = true // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -166,10 +166,10 @@ object UringSystem extends PollingSystem { private[this] var pendingSubmissions: Boolean = false private[this] val callbacks: Map[Short, Either[Throwable, Int] => Unit] = Map.empty[Short, Either[Throwable, Int] => Unit] - private[this] val ids = new BitSet(Short.MaxValue + 1) + private[this] val ids = new BitSet(Short.MaxValue) private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(0) + val newId = ids.nextClearBit(1) ids.set(newId) newId.toShort } @@ -262,7 +262,7 @@ object UringSystem extends PollingSystem { } def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { - sq.addTimeout(nanos, getUniqueId()) + sq.addTimeout(nanos, 0) val submitted = handlePendingSubmissions(submitAndWait) cq.ioUringWaitCqe() process(completionQueueCallback) @@ -273,7 +273,7 @@ object UringSystem extends PollingSystem { case -1 => if (pendingSubmissions) handlePendingSubmissions(true) else handleTimeoutAndQueue(-1, true) - case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false + case 0 | -1 => if (pendingSubmissions) handlePendingSubmissions(false) else false case _ => if (pendingSubmissions) handlePendingSubmissions(true) else handleTimeoutAndQueue(nanos, false) From fe438f78adcb42a5a08ae3aaacdcd6d2a5c8fcfd Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 27 Jul 2023 16:28:16 +0000 Subject: [PATCH 081/200] Increase timeout for testing --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 2 +- uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 305e24a5..3e52c96d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -273,7 +273,7 @@ object UringSystem extends PollingSystem { case -1 => if (pendingSubmissions) handlePendingSubmissions(true) else handleTimeoutAndQueue(-1, true) - case 0 | -1 => if (pendingSubmissions) handlePendingSubmissions(false) else false + case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false case _ => if (pendingSubmissions) handlePendingSubmissions(true) else handleTimeoutAndQueue(nanos, false) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 6c60cfc8..f0bada18 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -20,6 +20,7 @@ import munit.CatsEffectSuite import fs2.io.uring.unsafe.UringSystem import cats.effect.unsafe.IORuntimeBuilder +import scala.concurrent.duration._ abstract class UringSuite extends CatsEffectSuite { @@ -27,4 +28,6 @@ abstract class UringSuite extends CatsEffectSuite { IORuntimeBuilder() .setPollingSystem(UringSystem) .build() + + override def munitIOTimeout: Duration = 1000000.second } From f152bd324a9f7576084b93d34027726269883978 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 27 Jul 2023 20:42:49 +0000 Subject: [PATCH 082/200] Add simple test --- .../fs2/io/uring/net/UringSocketGroup.scala | 33 +++++++++++-------- .../fs2/io/uring/unsafe/UringSystem.scala | 4 +-- .../fs2/io/uring/net/TcpSocketSuite.scala | 19 ++++++++++- 3 files changed, 40 insertions(+), 16 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 85abbab1..dc2b2161 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -63,6 +63,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn println( s"[CLIENT] address: ${address.toString()}, buffer: ${buf.toString()}, length: $length" ) + + println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}") + ring .call( IORING_OP_CONNECT, @@ -100,12 +103,14 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn for { resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) - _ <- Resource.eval(F.delay(println(s"Resolved Address: $resolvedAddress"))) + _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] linuxSocket <- openSocket(ring, isIpv6) + _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) + localAddress <- Resource.eval { val bindF = F.delay { val socketAddress = @@ -119,7 +124,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) } - _ <- Resource.eval(F.delay(println(s"Local Address: $localAddress"))) + _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) sockets = Stream .resource(createBufferAux(isIpv6)) @@ -139,27 +144,29 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn )(closeSocket(ring, _)) .mapK(LiftIO.liftK) - val test = UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) - val socketAddress = new InetSocketAddress(test.getHostString(), test.getPort()) - val convert: F[SocketAddress[IpAddress]] = F - .delay( + .delay { + val test = UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) + val socketAddress = new InetSocketAddress(test.getHostString(), test.getPort()) SocketAddress.fromInetSocketAddress(socketAddress) - ) + } - Resource.eval(F.delay(println(s"Socket Address: $socketAddress"))) *> - accept - .flatMap { clientFd => + accept + .flatMap { clientFd => + Resource.eval(F.delay(println(s"SOCKETADDRESS: $clientFd"))).flatMap { _ => Resource.eval(convert).flatMap { remoteAddress => Resource - .eval(F.delay(println(s"NOW WE SHOULD HAVE THE ADDRESS: $remoteAddress"))) + .eval( + F.delay(println(s"NOW WE SHOULD HAVE THE ADDRESS: $remoteAddress")) + ) .flatMap { _ => UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) } } } - .attempt - .map(_.toOption) + } + .attempt + .map(_.toOption) }.repeat } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 3e52c96d..22005884 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -47,7 +47,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = true // True to printout operations + private val debug = false // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -272,7 +272,7 @@ object UringSystem extends PollingSystem { nanos match { case -1 => if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(-1, true) + else handleTimeoutAndQueue(-1, false) case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false case _ => if (pendingSubmissions) handlePendingSubmissions(true) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 1ad168da..95725791 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -25,8 +25,9 @@ import fs2.Chunk import fs2.text._ import fs2.io.uring.UringSuite +import cats.effect.kernel.Resource -class TcpSocketSuite extends UringSuite { +class TcpSocketSuit extends UringSuite { val sg = UringSocketGroup[IO] @@ -55,6 +56,7 @@ class TcpSocketSuite extends UringSuite { val setup = for { serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) (bindAddress, server) = serverSetup + _ <- Resource.eval(IO.delay(println(s"Bind address: $bindAddress"))) clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients @@ -94,6 +96,21 @@ class TcpSocketSuite extends UringSuite { } } + test("simple test") { + val message = Chunk.array("fs2.rocks".getBytes) + + val test = Stream.resource(setup).flatMap { case ((server, cleints)) => + + val testServer = server.map { socket => + Stream.chunk(message).through(socket.writes).onFinalize(socket.endOfInput) + } + + testServer.flatten.drain + } + + test.compile.resource.drain.useForever + } + test("readN yields chunks of the requested size") { val message = Chunk.array("123456789012345678901234567890".getBytes) val sizes = Vector(1, 2, 3, 4, 3, 2, 1) From 9562db65ea1e24bbf6c210fdd30137662b38e3e8 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 28 Jul 2023 12:05:12 +0200 Subject: [PATCH 083/200] Add more tests --- .../fs2/io/uring/net/TcpSocketSuite.scala | 126 ++++++++++++++++-- 1 file changed, 116 insertions(+), 10 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 95725791..0c408e27 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -27,7 +27,11 @@ import fs2.text._ import fs2.io.uring.UringSuite import cats.effect.kernel.Resource -class TcpSocketSuit extends UringSuite { +import scala.concurrent.duration._ +import java.io.IOException +import fs2.io.net.Socket + +class TcpSocketSuite extends UringSuite { val sg = UringSocketGroup[IO] @@ -53,6 +57,73 @@ class TcpSocketSuit extends UringSuite { } } + test("jsonplaceholder get") { + sg.client(SocketAddress(host"jsonplaceholder.typicode.com", port"80")).use { socket => + val msg = + """|GET /todos/1 HTTP/1.1 + |Host: jsonplaceholder.typicode.com + | + |""".stripMargin + + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .take(1) + + writeRead.compile.lastOrError + .assertEquals("HTTP/1.1 200 OK") + } +} + +test("jsonplaceholder post") { + sg.client(SocketAddress(host"jsonplaceholder.typicode.com", port"80")).use { socket => + val msg = + """|POST /posts HTTP/1.1 + |Host: jsonplaceholder.typicode.com + |Content-Type: application/json + |Content-Length: 69 + | + |{"title": "foo", "body": "bar", "userId": 1} + |""".stripMargin + + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .take(1) + + writeRead.compile.lastOrError + .assertEquals("HTTP/1.1 201 Created") + } +} + + test("invalid address") { + sg.client(SocketAddress(host"invalid-address", port"80")).use(_ => + IO.unit + ).intercept[IOException] + } + + test("write after close") { + sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => + val msg = "GET /get HTTP/1.1\nHost: postman-echo.com\n\n" + + socket.endOfOutput >> + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) + .compile + .drain + .intercept[IOException] + } +} + val setup = for { serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) (bindAddress, server) = serverSetup @@ -60,6 +131,42 @@ class TcpSocketSuit extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients + val echoServerResource: Resource[IO, (SocketAddress[IpAddress], Stream[IO, Socket[IO]])] = + UringSocketGroup[IO].serverResource(Some(Host.fromString("localhost").get), Some(port"51343"), Nil) + + test("local echo server") { + echoServerResource.use { case (_, serverStream) => + sg.client(SocketAddress(host"localhost", port"51343")).use { socket => + val msg = "Hello, echo server!\n" + + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .head + + val echoServer = serverStream.flatMap { socket => + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .map(line => s"$line\n") + .through(utf8.encode[IO]) + .through(socket.writes) + }.compile.drain + + echoServer.background.use(_ => + IO.sleep(1.second) *> // Ensures that the server is ready before the client tries to connect + writeRead.compile.lastOrError + .assertEquals("Hello, echo server!") + ) + } + } + } + + test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) val clientCount = 20L @@ -96,19 +203,18 @@ class TcpSocketSuit extends UringSuite { } } - test("simple test") { - val message = Chunk.array("fs2.rocks".getBytes) + val socketGroup = UringSocketGroup[IO] + test("Start server and waot fpr a connection") { + val serverResource = socketGroup.serverResource(Some(Host.fromString("localhost").get), Some(Port.fromInt(0).get), List()) - val test = Stream.resource(setup).flatMap { case ((server, cleints)) => - - val testServer = server.map { socket => - Stream.chunk(message).through(socket.writes).onFinalize(socket.endOfInput) - } + serverResource.use { case (localAddress, _) => + IO { + println(s"Server started at $localAddress") + println(s"You can now connect to this server") + } *> IO.sleep(1.minute) - testServer.flatten.drain } - test.compile.resource.drain.useForever } test("readN yields chunks of the requested size") { From 3b7b38c17a769f09b38e8372a0ae2c0ca87adcfe Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 28 Jul 2023 12:37:30 +0200 Subject: [PATCH 084/200] Formatting --- .../fs2/io/uring/net/TcpSocketSuite.scala | 100 ++++++++++-------- 1 file changed, 56 insertions(+), 44 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 0c408e27..f199d7d5 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -77,52 +77,52 @@ class TcpSocketSuite extends UringSuite { writeRead.compile.lastOrError .assertEquals("HTTP/1.1 200 OK") } -} + } -test("jsonplaceholder post") { - sg.client(SocketAddress(host"jsonplaceholder.typicode.com", port"80")).use { socket => - val msg = - """|POST /posts HTTP/1.1 + test("jsonplaceholder post") { + sg.client(SocketAddress(host"jsonplaceholder.typicode.com", port"80")).use { socket => + val msg = + """|POST /posts HTTP/1.1 |Host: jsonplaceholder.typicode.com |Content-Type: application/json - |Content-Length: 69 + |Content-Length: 44 | |{"title": "foo", "body": "bar", "userId": 1} |""".stripMargin - val writeRead = - Stream(msg) - .through(utf8.encode[IO]) - .through(socket.writes) ++ - socket.reads - .through(utf8.decode[IO]) - .through(lines) - .take(1) - - writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 201 Created") + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .take(1) + + writeRead.compile.lastOrError + .assertEquals("HTTP/1.1 201 Created") + } } -} test("invalid address") { - sg.client(SocketAddress(host"invalid-address", port"80")).use(_ => - IO.unit - ).intercept[IOException] + sg.client(SocketAddress(host"invalid-address", port"80")) + .use(_ => IO.unit) + .intercept[IOException] } test("write after close") { sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => val msg = "GET /get HTTP/1.1\nHost: postman-echo.com\n\n" - socket.endOfOutput >> - Stream(msg) - .through(utf8.encode[IO]) - .through(socket.writes) - .compile - .drain - .intercept[IOException] + socket.endOfOutput >> + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) + .compile + .drain + .intercept[IOException] } -} + } val setup = for { serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) @@ -132,7 +132,11 @@ test("jsonplaceholder post") { } yield server -> clients val echoServerResource: Resource[IO, (SocketAddress[IpAddress], Stream[IO, Socket[IO]])] = - UringSocketGroup[IO].serverResource(Some(Host.fromString("localhost").get), Some(port"51343"), Nil) + UringSocketGroup[IO].serverResource( + Some(Host.fromString("localhost").get), + Some(port"51343"), + Nil + ) test("local echo server") { echoServerResource.use { case (_, serverStream) => @@ -148,25 +152,29 @@ test("jsonplaceholder post") { .through(lines) .head - val echoServer = serverStream.flatMap { socket => - socket.reads - .through(utf8.decode[IO]) - .through(lines) - .map(line => s"$line\n") - .through(utf8.encode[IO]) - .through(socket.writes) - }.compile.drain + val echoServer = serverStream + .flatMap { socket => + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .map(line => s"$line\n") + .through(utf8.encode[IO]) + .through(socket.writes) + } + .compile + .drain echoServer.background.use(_ => - IO.sleep(1.second) *> // Ensures that the server is ready before the client tries to connect - writeRead.compile.lastOrError - .assertEquals("Hello, echo server!") + IO.sleep( + 1.second + ) *> // Ensures that the server is ready before the client tries to connect + writeRead.compile.lastOrError + .assertEquals("Hello, echo server!") ) } } } - test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) val clientCount = 20L @@ -204,8 +212,12 @@ test("jsonplaceholder post") { } val socketGroup = UringSocketGroup[IO] - test("Start server and waot fpr a connection") { - val serverResource = socketGroup.serverResource(Some(Host.fromString("localhost").get), Some(Port.fromInt(0).get), List()) + test("Start server and wait for a connection") { + val serverResource = socketGroup.serverResource( + Some(Host.fromString("localhost").get), + Some(Port.fromInt(0).get), + List() + ) serverResource.use { case (localAddress, _) => IO { From 059ff22f52699154da6f2b3cbd148be71164343f Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 28 Jul 2023 16:38:29 +0200 Subject: [PATCH 085/200] Isolate server bug in tests --- .../fs2/io/uring/net/TcpSocketSuite.scala | 99 +++++++++++++++---- 1 file changed, 81 insertions(+), 18 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index f199d7d5..81ae1586 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -138,6 +138,87 @@ class TcpSocketSuite extends UringSuite { Nil ) + test("Start server and wait for a connection during 20 sec") { + val serverResource = sg.serverResource( + Some(Host.fromString("localhost").get), + Some(Port.fromInt(0).get), + List() + ) + + serverResource.use { case (localAddress, _) => + IO { + println(s"Server started at $localAddress") + println(s"You can now connect to this server") + } *> IO.sleep(20.second) + + } + + } + + test("Start server and connect client") { + val serverResource = sg.serverResource( + Some(Host.fromString("localhost").get), + Some(Port.fromInt(0).get), + List() + ) + + serverResource.use { case (localAddress, _) => + IO { + println(s"Server started at $localAddress") + } *> IO.sleep(1.second) *> + sg.client(localAddress).use { _ => + IO { + println(s"Client connected to $localAddress") + /// Connection has been established! + } + } *> IO.sleep(3.second) + } + } + + test("Start server, connect client and echo") { + val serverResource = sg.serverResource( + Some(Host.fromString("localhost").get), + Some(Port.fromInt(0).get), + List() + ) + + serverResource.use { case (localAddress, serverStream) => + val serverEcho = serverStream + .map { clientSocket => + clientSocket.reads + .through(utf8.decode[IO]) + .through(lines) + .flatMap(line => + Stream.emit(line).through(utf8.encode[IO]).through(clientSocket.writes) + ) + .compile + .drain + .start + } + .compile + .drain + .start + + serverEcho *> // ServerStream throws the error! + IO.sleep(1.second) *> + sg.client(localAddress).use { socket => + val msg = "Hello, server!" + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .head + IO { + println(s"Client connected to $localAddress") + writeRead.compile.lastOrError.assertEquals(msg) + } + } *> IO.sleep(10.second) + } + } + test("local echo server") { echoServerResource.use { case (_, serverStream) => sg.client(SocketAddress(host"localhost", port"51343")).use { socket => @@ -211,24 +292,6 @@ class TcpSocketSuite extends UringSuite { } } - val socketGroup = UringSocketGroup[IO] - test("Start server and wait for a connection") { - val serverResource = socketGroup.serverResource( - Some(Host.fromString("localhost").get), - Some(Port.fromInt(0).get), - List() - ) - - serverResource.use { case (localAddress, _) => - IO { - println(s"Server started at $localAddress") - println(s"You can now connect to this server") - } *> IO.sleep(1.minute) - - } - - } - test("readN yields chunks of the requested size") { val message = Chunk.array("123456789012345678901234567890".getBytes) val sizes = Vector(1, 2, 3, 4, 3, 2, 1) From 5443096cb14390982f5f7d5eeb9e44c674636445 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 29 Jul 2023 13:13:07 +0200 Subject: [PATCH 086/200] Add prints in server --- .../fs2/io/uring/net/UringSocketGroup.scala | 60 ++++++++++--------- 1 file changed, 32 insertions(+), 28 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index dc2b2161..96c62669 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -132,37 +132,41 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn Stream.resource { val accept = - ring - .bracket( - IORING_OP_ACCEPT, - 0, - NativeAccess.SOCK_NONBLOCK, - linuxSocket.fd(), - buf.memoryAddress(), - 0, - buf.capacity().toLong - )(closeSocket(ring, _)) - .mapK(LiftIO.liftK) - - val convert: F[SocketAddress[IpAddress]] = F - .delay { - val test = UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) - val socketAddress = new InetSocketAddress(test.getHostString(), test.getPort()) - SocketAddress.fromInetSocketAddress(socketAddress) - } + Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> + ring + .bracket( + IORING_OP_ACCEPT, + 0, + NativeAccess.SOCK_NONBLOCK, + linuxSocket.fd(), + buf.memoryAddress(), + 0, + buf.capacity().toLong + )(closeSocket(ring, _)) + .mapK(LiftIO.liftK) + + val convert: F[SocketAddress[IpAddress]] = + F.delay( + println( + "[SERVER] getting the address in memory and converting it to SocketAddress..." + ) + ) *> + F.delay { + val inetAddress = + UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) + println(s"[SERVER] Read IP address from buffer: ${inetAddress.getHostString()}") + new InetSocketAddress(inetAddress.getHostString(), inetAddress.getPort()) + }.flatMap { inetSocketAddress => + F.delay { + println(s"[SERVER] converted and found inetSocketAddress: $inetSocketAddress") + SocketAddress.fromInetSocketAddress(inetSocketAddress) + } + } accept .flatMap { clientFd => - Resource.eval(F.delay(println(s"SOCKETADDRESS: $clientFd"))).flatMap { _ => - Resource.eval(convert).flatMap { remoteAddress => - Resource - .eval( - F.delay(println(s"NOW WE SHOULD HAVE THE ADDRESS: $remoteAddress")) - ) - .flatMap { _ => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) - } - } + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) } } .attempt From 23205a8fc02b152dbd2f2541b7be75e134abf1e8 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 29 Jul 2023 20:03:55 +0200 Subject: [PATCH 087/200] Review server --- .../scala/fs2/io/uring/net/UringSocket.scala | 17 +- .../fs2/io/uring/net/UringSocketGroup.scala | 27 ++- .../scala/fs2/io/uring/unsafe/UringRing.scala | 4 + .../fs2/io/uring/net/TcpSocketSuite.scala | 192 ++++++++++-------- 4 files changed, 147 insertions(+), 93 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index fe8bdc49..cad67c44 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -55,12 +55,17 @@ private[net] final class UringSocket[F[_]: LiftIO]( readMutex.lock.surround { for { _ <- F.delay(buffer.clear()) + readed <- recv(buffer.memoryAddress(), 0, maxBytes, 0) + bytes <- F.delay { val arr = new Array[Byte](readed) buffer.getBytes(0, arr) arr } + + _ <- F.delay(println(s"[SOCKET] reading the array: ${bytes}")) + } yield Option.when(readed > 0)(Chunk.array(bytes)) } @@ -68,12 +73,14 @@ private[net] final class UringSocket[F[_]: LiftIO]( readMutex.lock.surround { for { _ <- F.delay(buffer.clear()) + readed <- recv( buffer.memoryAddress(), 0, numBytes, 0 // TODO: Replace this with MSG_WAITALL - ) + ) + bytes <- F.delay { val arr = new Array[Byte](readed) buffer.getBytes(0, arr) @@ -105,12 +112,18 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.clear() buffer.writeBytes(bytes.toArray) } + + _ <- F.delay(println(s"[SOCKET] writing in array: $bytes")) + _ <- send( buffer.memoryAddress(), 0, bytes.size, 0 // TODO Replace this with MSG_NOSIGNAL - ) + ) + + _ <- F.delay(println(s"[SOCKET] Message sent!")) + } yield () } .unlessA(bytes.isEmpty) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 96c62669..99e015a9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -78,12 +78,15 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) .to } - } *> UringSocket( - ring, - linuxSocket, - linuxSocket.fd(), - address - ) + } *> + Resource.eval(F.delay(println("[CLIENT] connecting..."))).flatMap { _ => + UringSocket( + ring, + linuxSocket, + linuxSocket.fd(), + address + ) + } } } } @@ -137,7 +140,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .bracket( IORING_OP_ACCEPT, 0, - NativeAccess.SOCK_NONBLOCK, + NativeAccess.SOCK_CLOEXEC, linuxSocket.fd(), buf.memoryAddress(), 0, @@ -152,8 +155,12 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) ) *> F.delay { - val inetAddress = - UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](16)) + val inetAddress = if (isIpv6) { + UringSockaddrIn + .readIPv6(buf.memoryAddress(), new Array[Byte](16), new Array[Byte](4)) + } else { + UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](4)) + } println(s"[SERVER] Read IP address from buffer: ${inetAddress.getHostString()}") new InetSocketAddress(inetAddress.getHostString(), inetAddress.getPort()) }.flatMap { inetSocketAddress => @@ -171,7 +178,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } .attempt .map(_.toOption) - }.repeat + }.repeat } } yield (localAddress, sockets.unNone) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index b515df81..ace3412b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -413,6 +413,8 @@ object NativeAccess { val SOCK_NONBLOCK = Native.SOCK_NONBLOCK + val SOCK_CLOEXEC = Native.SOCK_CLOEXEC + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ @@ -618,4 +620,6 @@ object UringSockaddrIn { def readIPv4(memory: Long, tmpArray: Array[Byte]): InetSocketAddress = SockaddrIn.readIPv4(memory, tmpArray) + + def readIPv6(memory: Long, ipv6Array: Array[Byte], ipv4Array: Array[Byte]) = SockaddrIn.readIPv6(memory, ipv6Array, ipv4Array) } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 81ae1586..be3f431b 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -35,6 +35,8 @@ class TcpSocketSuite extends UringSuite { val sg = UringSocketGroup[IO] + // Client test: + test("postman echo") { sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => val msg = @@ -124,104 +126,104 @@ class TcpSocketSuite extends UringSuite { } } - val setup = for { - serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) - (bindAddress, server) = serverSetup - _ <- Resource.eval(IO.delay(println(s"Bind address: $bindAddress"))) - clients = Stream.resource(sg.client(bindAddress)).repeat - } yield server -> clients - - val echoServerResource: Resource[IO, (SocketAddress[IpAddress], Stream[IO, Socket[IO]])] = - UringSocketGroup[IO].serverResource( - Some(Host.fromString("localhost").get), - Some(port"51343"), - Nil - ) + // Server tests: - test("Start server and wait for a connection during 20 sec") { - val serverResource = sg.serverResource( + val serverResource: Resource[IO, (SocketAddress[IpAddress], Stream[IO, Socket[IO]])] = + sg.serverResource( Some(Host.fromString("localhost").get), Some(Port.fromInt(0).get), - List() + Nil ) + test("Start server and wait for a connection during 10 sec") { serverResource.use { case (localAddress, _) => IO { println(s"Server started at $localAddress") println(s"You can now connect to this server") } *> IO.sleep(20.second) - + // Use telnet localhost "port" to connect } - } - test("Start server and connect client") { - val serverResource = sg.serverResource( - Some(Host.fromString("localhost").get), - Some(Port.fromInt(0).get), - List() - ) - + test("Start server and connect external client") { serverResource.use { case (localAddress, _) => IO { println(s"Server started at $localAddress") - } *> IO.sleep(1.second) *> - sg.client(localAddress).use { _ => - IO { - println(s"Client connected to $localAddress") - /// Connection has been established! + } *> + sg.client(localAddress).use { socket => + val info = IO { + println("Socket created and connection established!") + println(s"remote address connected: ${socket.remoteAddress}") } - } *> IO.sleep(3.second) + + val assert = socket.remoteAddress.map(assertEquals(_, localAddress)) + + info *> assert + } } } - test("Start server, connect client and echo") { - val serverResource = sg.serverResource( - Some(Host.fromString("localhost").get), - Some(Port.fromInt(0).get), - List() - ) + // We start using the serverStream + test("Create server and connect external client 2") { + serverResource.use { case (localAddress, serverStream) => + sg.client(localAddress).use { _ => + val echoServer = + serverStream.compile.drain // If we modify the resource server to just take(1) works. I guess we are trying to bind multiple sockets to the same ip/port ? + + IO.println("socket created and connection established!") *> + echoServer.background.use(_ => IO.sleep(1.second)) + } + } + } + test("Create server connect with external client and writes") { serverResource.use { case (localAddress, serverStream) => - val serverEcho = serverStream - .map { clientSocket => - clientSocket.reads - .through(utf8.decode[IO]) - .through(lines) - .flatMap(line => - Stream.emit(line).through(utf8.encode[IO]).through(clientSocket.writes) - ) - .compile - .drain - .start - } - .compile - .drain - .start + sg.client(localAddress).use { socket => + val msg = "Hello, echo server!\n" - serverEcho *> // ServerStream throws the error! - IO.sleep(1.second) *> - sg.client(localAddress).use { socket => - val msg = "Hello, server!" - val writeRead = - Stream(msg) + val write = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) + + val echoServer = serverStream.compile.drain + + IO.println("socket created and connection established!") *> + echoServer.background.use(_ => + IO.sleep(5.second) *> // TODO server waits for connection but we never connect to it. + write.compile.drain + *> IO.println("message written!") + ) + } + } + } + + test("Create server connect with external client and server writes 3") { + serverResource.use { case (localAddress, serverStream) => + sg.client(localAddress).use { socket => + val serverMsg = "Hello, client!\n" + + val echoServer = serverStream + .map { clientSocket => + Stream(serverMsg) .through(utf8.encode[IO]) - .through(socket.writes) ++ - socket.reads - .through(utf8.decode[IO]) - .through(lines) - .head - IO { - println(s"Client connected to $localAddress") - writeRead.compile.lastOrError.assertEquals(msg) + .through(clientSocket.writes) } - } *> IO.sleep(10.second) + .parJoinUnbounded + .compile + .drain + + IO.println("socket created and connection established!") *> + echoServer *> + IO.sleep(20.second) *> // Wait for server to send a message + IO.println("server message sent!") + } } } - test("local echo server") { - echoServerResource.use { case (_, serverStream) => - sg.client(SocketAddress(host"localhost", port"51343")).use { socket => + test("local echo server with read and write") { + serverResource.use { case (localAddress, serverStream) => + sg.client(localAddress).use { socket => val msg = "Hello, echo server!\n" val writeRead = @@ -245,20 +247,48 @@ class TcpSocketSuite extends UringSuite { .compile .drain - echoServer.background.use(_ => - IO.sleep( - 1.second - ) *> // Ensures that the server is ready before the client tries to connect - writeRead.compile.lastOrError - .assertEquals("Hello, echo server!") - ) + IO.println("socket created and connection established!") *> + echoServer.background.use(_ => + IO.sleep( + 1.second + ) *> // Ensures that the server is ready before the client tries to connect + writeRead.compile.lastOrError + .assertEquals("Hello, echo server!") + ) } } } + test("Create server connect and write") { + serverResource.use { case (_, clientsStream) => + val msg = "Hello, echo server!\n" + + val write = clientsStream.take(1).flatMap { socket => + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) + .onFinalize(socket.endOfOutput) + } + + IO.println("Starting operation...") *> + IO.sleep(1.second) *> // Ensures that the server is ready before the client tries to connect + write.compile.drain + } + } + + // Server and client tests: + + val setup = for { + serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) + (bindAddress, server) = serverSetup + _ <- Resource.eval(IO.delay(println(s"Bind address: $bindAddress"))) + clients = Stream.resource(sg.client(bindAddress)).repeat + } yield server -> clients + + test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) - val clientCount = 20L + val clientCount = 1L Stream .resource(setup) @@ -267,9 +297,9 @@ class TcpSocketSuite extends UringSuite { socket.reads .through(socket.writes) .onFinalize(socket.endOfOutput) - }.parJoinUnbounded + }.parJoin(1) - val msgClients = clients + val msgClients = Stream.sleep_[IO](1.second) ++ clients .take(clientCount) .map { socket => Stream @@ -279,7 +309,7 @@ class TcpSocketSuite extends UringSuite { socket.reads.chunks .map(bytes => new String(bytes.toArray)) } - .parJoin(10) + .parJoin(1) .take(clientCount) msgClients.concurrently(echoServer) From 4f522f0d1ec230b72e1220ea292cf5fe5d40aec6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sun, 30 Jul 2023 22:27:16 +0200 Subject: [PATCH 088/200] Fix server bug --- .../fs2/io/uring/net/UringSocketGroup.scala | 108 ++++++++++-------- .../fs2/io/uring/unsafe/UringSystem.scala | 8 +- 2 files changed, 69 insertions(+), 47 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 99e015a9..89a2912b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -132,53 +132,69 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn sockets = Stream .resource(createBufferAux(isIpv6)) .flatMap { buf => - Stream.resource { - - val accept = - Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> - ring - .bracket( - IORING_OP_ACCEPT, - 0, - NativeAccess.SOCK_CLOEXEC, - linuxSocket.fd(), - buf.memoryAddress(), - 0, - buf.capacity().toLong - )(closeSocket(ring, _)) - .mapK(LiftIO.liftK) - - val convert: F[SocketAddress[IpAddress]] = - F.delay( - println( - "[SERVER] getting the address in memory and converting it to SocketAddress..." - ) - ) *> - F.delay { - val inetAddress = if (isIpv6) { - UringSockaddrIn - .readIPv6(buf.memoryAddress(), new Array[Byte](16), new Array[Byte](4)) - } else { - UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](4)) + Stream + .resource(createBuffer(4)) + .flatMap { buf2 => + Stream.resource { + + buf2.writeInt(buf.capacity()) + + val accept = + Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> + ring + .bracket( + IORING_OP_ACCEPT, + 0, + NativeAccess.SOCK_NONBLOCK, + linuxSocket.fd(), + buf.memoryAddress(), + 0, + buf2.memoryAddress() + )(closeSocket(ring, _)) + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = ioa.debug() + } + } + .mapK(LiftIO.liftK) + + val convert: F[SocketAddress[IpAddress]] = + F.delay( + println( + "[SERVER] getting the address in memory and converting it to SocketAddress..." + ) + ) *> + F.delay { + val inetAddress = if (isIpv6) { + UringSockaddrIn + .readIPv6(buf.memoryAddress(), new Array[Byte](16), new Array[Byte](4)) + } else { + UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](4)) + } + println( + s"[SERVER] Read IP address from buffer: ${inetAddress.getHostString()}" + ) + new InetSocketAddress(inetAddress.getHostString(), inetAddress.getPort()) + }.flatMap { inetSocketAddress => + F.delay { + println( + s"[SERVER] converted and found inetSocketAddress: $inetSocketAddress" + ) + SocketAddress.fromInetSocketAddress(inetSocketAddress) + } + } + + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } } - println(s"[SERVER] Read IP address from buffer: ${inetAddress.getHostString()}") - new InetSocketAddress(inetAddress.getHostString(), inetAddress.getPort()) - }.flatMap { inetSocketAddress => - F.delay { - println(s"[SERVER] converted and found inetSocketAddress: $inetSocketAddress") - SocketAddress.fromInetSocketAddress(inetSocketAddress) - } - } - - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) - } - } - .attempt - .map(_.toOption) - }.repeat + .attempt + .map(_.toOption) + }.repeat + } + } } yield (localAddress, sockets.unNone) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 22005884..9cd53ebb 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -240,7 +240,13 @@ object UringSystem extends PollingSystem { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = if (res < 0) - cb(Left(new IOException(s"Error in completion queue entry: $res"))) + cb( + Left( + new IOException( + s"Error in completion queue entry with fd: $fd op: $op res: $res and data: $data" + ) + ) + ) else cb(Right(res)) if ( From 393c98c03bbd35f18d9105d888335a119fe81a62 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sun, 30 Jul 2023 22:27:40 +0200 Subject: [PATCH 089/200] Add test for server --- .../fs2/io/uring/net/TcpSocketSuite.scala | 92 +++++++++---------- 1 file changed, 43 insertions(+), 49 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index be3f431b..69aa5d35 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -30,6 +30,7 @@ import cats.effect.kernel.Resource import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket +import java.util.concurrent.TimeoutException class TcpSocketSuite extends UringSuite { @@ -190,7 +191,7 @@ class TcpSocketSuite extends UringSuite { IO.println("socket created and connection established!") *> echoServer.background.use(_ => - IO.sleep(5.second) *> // TODO server waits for connection but we never connect to it. + IO.sleep(10.second) *> // TODO server waits for connection but we never connect to it. write.compile.drain *> IO.println("message written!") ) @@ -198,29 +199,6 @@ class TcpSocketSuite extends UringSuite { } } - test("Create server connect with external client and server writes 3") { - serverResource.use { case (localAddress, serverStream) => - sg.client(localAddress).use { socket => - val serverMsg = "Hello, client!\n" - - val echoServer = serverStream - .map { clientSocket => - Stream(serverMsg) - .through(utf8.encode[IO]) - .through(clientSocket.writes) - } - .parJoinUnbounded - .compile - .drain - - IO.println("socket created and connection established!") *> - echoServer *> - IO.sleep(20.second) *> // Wait for server to send a message - IO.println("server message sent!") - } - } - } - test("local echo server with read and write") { serverResource.use { case (localAddress, serverStream) => sg.client(localAddress).use { socket => @@ -259,23 +237,6 @@ class TcpSocketSuite extends UringSuite { } } - test("Create server connect and write") { - serverResource.use { case (_, clientsStream) => - val msg = "Hello, echo server!\n" - - val write = clientsStream.take(1).flatMap { socket => - Stream(msg) - .through(utf8.encode[IO]) - .through(socket.writes) - .onFinalize(socket.endOfOutput) - } - - IO.println("Starting operation...") *> - IO.sleep(1.second) *> // Ensures that the server is ready before the client tries to connect - write.compile.drain - } - } - // Server and client tests: val setup = for { @@ -285,19 +246,20 @@ class TcpSocketSuite extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) - val clientCount = 1L + val clientCount = 20L Stream .resource(setup) .flatMap { case (server, clients) => - val echoServer = server.map { socket => - socket.reads - .through(socket.writes) - .onFinalize(socket.endOfOutput) - }.parJoin(1) + val echoServer = server + .map { socket => + socket.reads + .through(socket.writes) + .onFinalize(socket.endOfOutput) + } + .parJoin(1) val msgClients = Stream.sleep_[IO](1.second) ++ clients .take(clientCount) @@ -309,7 +271,7 @@ class TcpSocketSuite extends UringSuite { socket.reads.chunks .map(bytes => new String(bytes.toArray)) } - .parJoin(1) + .parJoin(10) .take(clientCount) msgClients.concurrently(echoServer) @@ -401,4 +363,36 @@ class TcpSocketSuite extends UringSuite { .drain } + // TODO options test + + // TODO decide about "read after timed out read not allowed" + + test("can shutdown a socket that's pending a read") { + val timeout = 2.seconds + val test = sg.serverResource().use { case (bindAddress, clients) => + sg.client(bindAddress).use { _ => + clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { + case _: TimeoutException => () + } + } + } + + // also test that timeouts are working correctly + test.timed.flatMap { case (duration, _) => + IO(assert(clue(duration) < (timeout + 100.millis))) + } + } + + test("accept is cancelable") { + sg.serverResource().use { case (_, clients) => + clients.compile.drain.timeoutTo(100.millis, IO.unit) + } + } + + test("empty write") { + setup.use { case (_, clients) => + clients.take(1).foreach(_.write(Chunk.empty)).compile.drain + } + } + } From 79ebcf7972134e8415beb9c7af2507d50cc3c3ee Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 31 Jul 2023 01:18:22 +0200 Subject: [PATCH 090/200] Add comments and TODO --- .../fs2/io/uring/net/UringSocketGroup.scala | 127 +++++++++--------- .../scala/fs2/io/uring/unsafe/UringRing.scala | 3 +- 2 files changed, 66 insertions(+), 64 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 89a2912b..bfeb6dd0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -33,20 +33,21 @@ import fs2.io.uring.Uring import fs2.io.uring.unsafe.util.createBuffer import fs2.io.uring.unsafe.util.OP._ +import io.netty.buffer.ByteBuf import io.netty.incubator.channel.uring.UringSockaddrIn import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 +import io.netty.incubator.channel.uring.NativeAccess.SOCK_NONBLOCK -import io.netty.buffer.ByteBuf import java.net.InetSocketAddress -import io.netty.incubator.channel.uring.NativeAccess private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = if (isIpv6) createBuffer(SIZEOF_SOCKADDR_IN6) else createBuffer(SIZEOF_SOCKADDR_IN) + def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = Resource.eval(Uring.get[F]).flatMap { ring => Resource.eval(to.resolve).flatMap { address => @@ -68,13 +69,10 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ring .call( - IORING_OP_CONNECT, - 0, - 0, - linuxSocket.fd(), - buf.memoryAddress(), - 0, - length.toLong + op = IORING_OP_CONNECT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = length.toLong ) .to } @@ -97,6 +95,11 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn options: List[SocketOption] ): Stream[F, Socket[F]] = Stream.resource(serverResource(address, port, options)).flatMap(_._2) + // TODO: Replace 4 with SIZEOF_SOCKADDR_IN and 16 with SIZEOF_SOCKADDR_IN6 + private[this] def readIpv(memory: Long, isIpv6: Boolean): InetSocketAddress = + if (isIpv6) UringSockaddrIn.readIPv6(memory, new Array[Byte](16), new Array[Byte](4)) + else UringSockaddrIn.readIPv4(memory, new Array[Byte](4)) + def serverResource( address: Option[Host], port: Option[Port], @@ -131,68 +134,66 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn sockets = Stream .resource(createBufferAux(isIpv6)) - .flatMap { buf => + .flatMap { buf => // Buffer that will contain the remote address Stream - .resource(createBuffer(4)) - .flatMap { buf2 => - Stream.resource { - - buf2.writeInt(buf.capacity()) - - val accept = - Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> - ring - .bracket( - IORING_OP_ACCEPT, - 0, - NativeAccess.SOCK_NONBLOCK, - linuxSocket.fd(), - buf.memoryAddress(), - 0, - buf2.memoryAddress() - )(closeSocket(ring, _)) - .mapK { - new cats.~>[IO, IO] { - def apply[A](ioa: IO[A]) = ioa.debug() + .resource(createBuffer(4)) // TODO: Replace 4 with INT_SIZE ? + .flatMap { + bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer + Stream.resource { + + bufLength.writeInt(buf.capacity()) + + // We accept a connection, we write the remote address on the buf and we get the clientFd + val accept = + Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> + ring + .bracket( + op = IORING_OP_ACCEPT, + rwFlags = SOCK_NONBLOCK, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = bufLength.memoryAddress() + )(closeSocket(ring, _)) + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = ioa.debug() + } } - } - .mapK(LiftIO.liftK) - - val convert: F[SocketAddress[IpAddress]] = - F.delay( - println( - "[SERVER] getting the address in memory and converting it to SocketAddress..." - ) - ) *> - F.delay { - val inetAddress = if (isIpv6) { - UringSockaddrIn - .readIPv6(buf.memoryAddress(), new Array[Byte](16), new Array[Byte](4)) - } else { - UringSockaddrIn.readIPv4(buf.memoryAddress(), new Array[Byte](4)) - } + .mapK(LiftIO.liftK) + + // We read the address from the buf and we convert it to SocketAddress + val convert: F[SocketAddress[IpAddress]] = + F.delay( println( - s"[SERVER] Read IP address from buffer: ${inetAddress.getHostString()}" + "[SERVER] getting the address in memory and converting it to SocketAddress..." ) - new InetSocketAddress(inetAddress.getHostString(), inetAddress.getPort()) - }.flatMap { inetSocketAddress => + ) *> + /* TODO: Merge the next two steps in one: F.delay (SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6))) */ F.delay { + val netRemoteAddress: InetSocketAddress = + readIpv(buf.memoryAddress(), isIpv6) println( - s"[SERVER] converted and found inetSocketAddress: $inetSocketAddress" + s"[SERVER] Read IP address from buffer: ${netRemoteAddress.getHostString()}" ) - SocketAddress.fromInetSocketAddress(inetSocketAddress) + netRemoteAddress + }.flatMap { netRemoteAddress => + F.delay { + println( + s"[SERVER] converted and found inetSocketAddress: $netRemoteAddress" + ) + SocketAddress.fromInetSocketAddress(netRemoteAddress) + } } - } - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } } - } - .attempt - .map(_.toOption) - }.repeat + .attempt + .map(_.toOption) + }.repeat } } @@ -200,7 +201,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } yield (localAddress, sockets.unNone) } - private def openSocket( + private[this] def openSocket( ring: Uring, ipv6: Boolean ): Resource[F, UringLinuxSocket] = @@ -208,7 +209,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn linuxSocket => closeSocket(ring, linuxSocket.fd()).to ) - private def closeSocket(ring: Uring, fd: Int): IO[Unit] = + private[this] def closeSocket(ring: Uring, fd: Int): IO[Unit] = ring.call(op = IORING_OP_CLOSE, fd = fd).void } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index ace3412b..cd04540c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -621,5 +621,6 @@ object UringSockaddrIn { def readIPv4(memory: Long, tmpArray: Array[Byte]): InetSocketAddress = SockaddrIn.readIPv4(memory, tmpArray) - def readIPv6(memory: Long, ipv6Array: Array[Byte], ipv4Array: Array[Byte]) = SockaddrIn.readIPv6(memory, ipv6Array, ipv4Array) + def readIPv6(memory: Long, ipv6Array: Array[Byte], ipv4Array: Array[Byte]): InetSocketAddress = + SockaddrIn.readIPv6(memory, ipv6Array, ipv4Array) } From a2e14ef2710d7fa63d90cc742a49974c4449f76b Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 31 Jul 2023 02:25:08 +0200 Subject: [PATCH 091/200] Remove rwFlags in OP_ACCEPT --- .../jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index bfeb6dd0..808a4f2a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -38,7 +38,6 @@ import io.netty.incubator.channel.uring.UringSockaddrIn import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 -import io.netty.incubator.channel.uring.NativeAccess.SOCK_NONBLOCK import java.net.InetSocketAddress @@ -149,7 +148,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ring .bracket( op = IORING_OP_ACCEPT, - rwFlags = SOCK_NONBLOCK, fd = linuxSocket.fd(), bufferAddress = buf.memoryAddress(), offset = bufLength.memoryAddress() From d0b673bbde2080d205eb42eddad9643e0f8a093e Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 31 Jul 2023 11:56:46 +0200 Subject: [PATCH 092/200] Add prints in Poll --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 6 ++++-- .../test/scala/fs2/io/uring/UringSuite.scala | 17 ++++++++++++----- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 9cd53ebb..7c719fbf 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -47,7 +47,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false // True to printout operations + private val debug = true // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -275,10 +275,12 @@ object UringSystem extends PollingSystem { submitted } + println(s"POLLING! with nanos: $nanos") + nanos match { case -1 => if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(-1, false) + else handleTimeoutAndQueue(-1, true) case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false case _ => if (pendingSubmissions) handlePendingSubmissions(true) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index f0bada18..ed7e3739 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -17,17 +17,24 @@ package fs2.io.uring import munit.CatsEffectSuite - +import cats.effect.unsafe.IORuntime import fs2.io.uring.unsafe.UringSystem import cats.effect.unsafe.IORuntimeBuilder import scala.concurrent.duration._ + abstract class UringSuite extends CatsEffectSuite { - override lazy val munitIORuntime = - IORuntimeBuilder() - .setPollingSystem(UringSystem) - .build() + override lazy val munitIORuntime = { + val (pool, poller, shutdown) = + IORuntime.createWorkStealingComputeThreadPool(threads = 1, pollingSystem = UringSystem) + IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() + } + + // override lazy val munitIORuntime = + // IORuntimeBuilder() + // .setPollingSystem(UringSystem) + // .build() override def munitIOTimeout: Duration = 1000000.second } From 20b3ff27d4410ad65098edd17ec8723bbb2c9850 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 31 Jul 2023 12:16:49 +0200 Subject: [PATCH 093/200] Add ringFd on prints --- .../jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 7c719fbf..8eb289b5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -243,7 +243,7 @@ object UringSystem extends PollingSystem { cb( Left( new IOException( - s"Error in completion queue entry with fd: $fd op: $op res: $res and data: $data" + s"Error in completion queue entry of the ring with fd: ${ring.fd()} with fd: $fd op: $op res: $res and data: $data" ) ) ) @@ -252,7 +252,7 @@ object UringSystem extends PollingSystem { if ( op != 11 && debug ) // To prevent the constant printouts of timeout operation when NANOS == -1 - println(s"[HANDLE CQCB]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data") + println(s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data") callbacks.get(data).foreach { cb => handleCallback(res, cb) From a342b06a37c82c7f97060584b3d6dcf5ab246044 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 31 Jul 2023 12:19:58 +0200 Subject: [PATCH 094/200] Add ringFd in enqueue --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 8eb289b5..289df2c9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -212,7 +212,7 @@ object UringSystem extends PollingSystem { ): Boolean = { if (debug) println( - s"[SQ] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + s"[SQ] Enqueuing a new Sqe in ringFd: ${ring.fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" ) sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } From 1b468d9eec480bb8b78d107b3dc571398c94e1e6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 31 Jul 2023 13:25:53 +0200 Subject: [PATCH 095/200] Remove TODO --- .../jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 808a4f2a..832ccb0a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -94,7 +94,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn options: List[SocketOption] ): Stream[F, Socket[F]] = Stream.resource(serverResource(address, port, options)).flatMap(_._2) - // TODO: Replace 4 with SIZEOF_SOCKADDR_IN and 16 with SIZEOF_SOCKADDR_IN6 private[this] def readIpv(memory: Long, isIpv6: Boolean): InetSocketAddress = if (isIpv6) UringSockaddrIn.readIPv6(memory, new Array[Byte](16), new Array[Byte](4)) else UringSockaddrIn.readIPv4(memory, new Array[Byte](4)) @@ -135,7 +134,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .resource(createBufferAux(isIpv6)) .flatMap { buf => // Buffer that will contain the remote address Stream - .resource(createBuffer(4)) // TODO: Replace 4 with INT_SIZE ? + .resource(createBuffer(4)) .flatMap { bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer Stream.resource { From 5ee436f1ee6d33c20f7ad64066f3d8f16e58f8c5 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 31 Jul 2023 14:00:02 +0200 Subject: [PATCH 096/200] Add uringOpenSocket --- .../scala/fs2/io/uring/net/UringSocketGroup.scala | 14 +++++++++++--- .../main/scala/fs2/io/uring/unsafe/UringRing.scala | 6 ++++++ 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 832ccb0a..68dacd83 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -36,8 +36,7 @@ import fs2.io.uring.unsafe.util.OP._ import io.netty.buffer.ByteBuf import io.netty.incubator.channel.uring.UringSockaddrIn import io.netty.incubator.channel.uring.UringLinuxSocket -import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN -import io.netty.incubator.channel.uring.NativeAccess.SIZEOF_SOCKADDR_IN6 +import io.netty.incubator.channel.uring.NativeAccess._ import java.net.InetSocketAddress @@ -134,7 +133,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .resource(createBufferAux(isIpv6)) .flatMap { buf => // Buffer that will contain the remote address Stream - .resource(createBuffer(4)) + .resource(createBuffer(4)) .flatMap { bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer Stream.resource { @@ -206,6 +205,15 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn linuxSocket => closeSocket(ring, linuxSocket.fd()).to ) + private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { + val domain = if (ipv6) AF_INET else AF_INET6 + ring + .bracket(op = IORING_OP_SOCKET, fd = domain, length = 0, offset = SOCK_STREAM)( + closeSocket(ring, _) + ) + .mapK(LiftIO.liftK) + } + private[this] def closeSocket(ring: Uring, fd: Int): IO[Unit] = ring.call(op = IORING_OP_CLOSE, fd = fd).void diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index cd04540c..57c0ed5a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -415,6 +415,12 @@ object NativeAccess { val SOCK_CLOEXEC = Native.SOCK_CLOEXEC + val AF_INET = Native.AF_INET + + val AF_INET6 = Native.AF_INET6 + + val SOCK_STREAM = 1 + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ From 110958d81f4176fbe91f457518b0506700920f53 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 31 Jul 2023 14:01:15 +0200 Subject: [PATCH 097/200] Correct domain --- .../jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 68dacd83..6d931da9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -206,7 +206,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { - val domain = if (ipv6) AF_INET else AF_INET6 + val domain = if (ipv6) AF_INET6 else AF_INET ring .bracket(op = IORING_OP_SOCKET, fd = domain, length = 0, offset = SOCK_STREAM)( closeSocket(ring, _) From 4422d8704c6ba64214657f474ec5629b3c2f6d46 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 6 Aug 2023 19:28:28 +0200 Subject: [PATCH 098/200] Replace Mutex with Semaphore in Native --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 48498569..042dd5bc 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -24,8 +24,8 @@ import cats.effect.IO import cats.effect.kernel.Cont import cats.effect.kernel.MonadCancelThrow import cats.effect.kernel.Resource -import cats.effect.std.Semaphore import cats.effect.unsafe.PollingSystem +import cats.effect.std.Mutex import cats.syntax.all._ import java.util.Collections @@ -130,11 +130,11 @@ object UringSystem extends PollingSystem { writes: Boolean ): Resource[IO, FileDescriptorPollHandle] = Resource.eval { - (Semaphore[IO](1), Semaphore[IO](1)).mapN { (readSemaphore, writeSemaphore) => + (Mutex[IO], Mutex[IO]).mapN { (readMutex, writeMutex) => new FileDescriptorPollHandle { def pollReadRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = - readSemaphore.permit.surround { + readMutex.lock.surround { a.tailRecM { a => f(a).flatTap { r => if (r.isRight) @@ -146,7 +146,7 @@ object UringSystem extends PollingSystem { } def pollWriteRec[A, B](a: A)(f: A => IO[Either[A, B]]): IO[B] = - writeSemaphore.permit.surround { + writeMutex.lock.surround { a.tailRecM { a => f(a).flatTap { r => if (r.isRight) From 8273561d8522792f12d0d14661ae7e9225da9b08 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 6 Aug 2023 19:49:29 +0200 Subject: [PATCH 099/200] Add DatagramSocket --- .../io/uring/net/UringDatagramSocket.scala | 118 ++++++++++++++++++ 1 file changed, 118 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala new file mode 100644 index 00000000..04e86db8 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala @@ -0,0 +1,118 @@ +package fs2.io.uring.net +import cats.effect.LiftIO +import cats.effect.kernel.Async +import cats.effect.kernel.Resource +import cats.effect.kernel.Sync +import cats.effect.std.Mutex +import cats.syntax.all._ + +import com.comcast.ip4s.IpAddress +import com.comcast.ip4s.SocketAddress +import com.comcast.ip4s.MulticastJoin + +import fs2.Chunk +import fs2.Pipe +import fs2.Stream +import fs2.io.net.Datagram +import fs2.io.net.DatagramSocket + +import fs2.io.uring.Uring +import fs2.io.uring.unsafe.util.createBuffer +import fs2.io.uring.unsafe.util.OP._ + +import io.netty.buffer.ByteBuf +import io.netty.incubator.channel.uring.UringLinuxSocket + +private[net] final class UringDatagramSocket[F[_]: LiftIO]( + ring: Uring, + linuxSocket: UringLinuxSocket, + sockfd: Int, + buffer: ByteBuf, + defaultReadSize: Int, + readMutex: Mutex[F], + writeMutex: Mutex[F] +)(implicit F: Async[F]) + extends DatagramSocket[F] { + + private[this] def recvfrom( + bufferAddress: Long, + len: Int + ): F[(SocketAddress[IpAddress], Int)] = // TODO: Work around this + // ring.call(IORING_OP_RECVFROM, fd = sockfd, bufferAddress = bufferAddress, length = len).to + ??? + + private[this] def recvMsg(msgHdr: Long) = ??? + + def read: F[Datagram] = + readMutex.lock.surround { + for { + _ <- F.delay(buffer.clear()) + (srcAddress, len) <- recvfrom(buffer.memoryAddress(), defaultReadSize) + bytes <- F.delay { + val arr = new Array[Byte](len) + buffer.getBytes(0, arr) + arr + } + } yield Datagram(srcAddress, Chunk.array(bytes)) + } + + def reads: Stream[F, Datagram] = Stream.repeatEval(read) + + private[this] def sendto( + bufferAddress: Long, + len: Int, + address: SocketAddress[IpAddress] + ): F[Int] = + // ring + // .call(IORING_OP_SEND, fd = sockfd, bufferAddress = bufferAddress, length = len) + // .to + ??? + + private[this] def sendMsg(msgHdr: Long, flags: Int) = ??? + + def write(datagram: Datagram): F[Unit] = + writeMutex.lock + .surround { + for { + _ <- F.delay { + buffer.clear() + buffer.writeBytes(datagram.bytes.toArray) + } + _ <- sendto(buffer.memoryAddress(), datagram.bytes.size, datagram.remote) + } yield () + } + .unlessA(datagram.bytes.isEmpty) + + def writes: Pipe[F, Datagram, Nothing] = _.evalMap(datagram => write(datagram)).drain + + def localAddress: F[SocketAddress[IpAddress]] = + F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) + + def join( + join: MulticastJoin[IpAddress], + interface: DatagramSocket.NetworkInterface + ): F[GroupMembership] = + F.raiseError(new UnsupportedOperationException("Not supported in DatagramSocket")) +} + +object UringDatagramSocket { + private[this] val defaultReadSize = 65535 + + def apply[F[_]: LiftIO](ring: Uring, linuxSocket: UringLinuxSocket, fd: Int)(implicit + F: Async[F] + ): Resource[F, UringDatagramSocket[F]] = + for { + buffer <- createBuffer(defaultReadSize) + readMutex <- Resource.eval(Mutex[F]) + writeMutex <- Resource.eval(Mutex[F]) + socket = new UringDatagramSocket( + ring, + linuxSocket, + fd, + buffer, + defaultReadSize, + readMutex, + writeMutex + ) + } yield socket +} From 7f2d0ba6745f1613f3e2bfae87ed8cabca1d4384 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 6 Aug 2023 19:50:08 +0200 Subject: [PATCH 100/200] Format UringSocket --- .../main/scala/fs2/io/uring/net/UringSocket.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index cad67c44..95bda5ce 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -51,6 +51,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + def read(maxBytes: Int): F[Option[Chunk[Byte]]] = readMutex.lock.surround { for { @@ -104,6 +105,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( private[this] def send(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + def write(bytes: Chunk[Byte]): F[Unit] = writeMutex.lock .surround { @@ -146,7 +148,16 @@ private[net] object UringSocket { buffer <- createBuffer(defaultReadSize) readMutex <- Resource.eval(Mutex[F]) writeMutex <- Resource.eval(Mutex[F]) - socket = new UringSocket(ring, linuxSocket, fd, remote, buffer, 8192, readMutex, writeMutex) + socket = new UringSocket( + ring, + linuxSocket, + fd, + remote, + buffer, + defaultReadSize, + readMutex, + writeMutex + ) } yield socket } From c8fccc3792da6d4bfddf5d49f817c54c221bddc0 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 7 Aug 2023 19:43:39 +0200 Subject: [PATCH 101/200] Add address length --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 57c0ed5a..55c6609d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -621,6 +621,10 @@ object UringLinuxSocket { } object UringSockaddrIn { + val IPV6_ADDRESS_LENGTH = SockaddrIn.IPV6_ADDRESS_LENGTH + + val IPV4_ADDRESS_LENGTH = SockaddrIn.IPV4_ADDRESS_LENGTH + def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = SockaddrIn.write(ipv6, memory, address) From 776f411e08b8106a2f3d50ac9926015445c13cb8 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 7 Aug 2023 19:44:00 +0200 Subject: [PATCH 102/200] Rewrite client and server using for-comprehension --- .../fs2/io/uring/net/UringSocketGroup.scala | 220 ++++++++---------- 1 file changed, 91 insertions(+), 129 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 6d931da9..eac41322 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -34,7 +34,7 @@ import fs2.io.uring.unsafe.util.createBuffer import fs2.io.uring.unsafe.util.OP._ import io.netty.buffer.ByteBuf -import io.netty.incubator.channel.uring.UringSockaddrIn +import io.netty.incubator.channel.uring.UringSockaddrIn._ import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess._ @@ -47,45 +47,35 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn if (isIpv6) createBuffer(SIZEOF_SOCKADDR_IN6) else createBuffer(SIZEOF_SOCKADDR_IN) def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = - Resource.eval(Uring.get[F]).flatMap { ring => - Resource.eval(to.resolve).flatMap { address => - val isIpv6: Boolean = address.host.isInstanceOf[Ipv6Address] - openSocket(ring, isIpv6).flatMap { linuxSocket => - Resource.eval { - createBufferAux(isIpv6).use { buf => - val length: Int = UringSockaddrIn.write( - isIpv6, - buf.memoryAddress(), - address.toInetSocketAddress - ) - + for { + ring <- Resource.eval(Uring.get[F]) + address <- Resource.eval(to.resolve) + isIpv6 = address.host.isInstanceOf[Ipv6Address] + linuxSocket <- openSocket(ring, isIpv6) + _ <- Resource.eval( + createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect + for { + length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) + _ <- F.delay( println( s"[CLIENT] address: ${address.toString()}, buffer: ${buf.toString()}, length: $length" ) - - println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}") - - ring - .call( - op = IORING_OP_CONNECT, - fd = linuxSocket.fd(), - bufferAddress = buf.memoryAddress(), - offset = length.toLong - ) - .to - } - } *> - Resource.eval(F.delay(println("[CLIENT] connecting..."))).flatMap { _ => - UringSocket( - ring, - linuxSocket, - linuxSocket.fd(), - address + ) + _ <- F.delay(println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}")) + _ <- ring + .call( + op = IORING_OP_CONNECT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = length.toLong ) - } + .to + } yield () } - } - } + ) + _ <- Resource.eval(F.delay(println("[CLIENT] connecting..."))) + socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + } yield socket def server( address: Option[Host], @@ -94,108 +84,80 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ): Stream[F, Socket[F]] = Stream.resource(serverResource(address, port, options)).flatMap(_._2) private[this] def readIpv(memory: Long, isIpv6: Boolean): InetSocketAddress = - if (isIpv6) UringSockaddrIn.readIPv6(memory, new Array[Byte](16), new Array[Byte](4)) - else UringSockaddrIn.readIPv4(memory, new Array[Byte](4)) + if (isIpv6) + readIPv6(memory, new Array[Byte](IPV6_ADDRESS_LENGTH), new Array[Byte](IPV4_ADDRESS_LENGTH)) + else readIPv4(memory, new Array[Byte](IPV4_ADDRESS_LENGTH)) def serverResource( address: Option[Host], port: Option[Port], options: List[SocketOption] - ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = - Resource.eval(Uring.get[F]).flatMap { ring => - for { - resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) - - _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) - - isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] - - linuxSocket <- openSocket(ring, isIpv6) - - _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) - - localAddress <- Resource.eval { - val bindF = F.delay { - val socketAddress = - new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) - linuxSocket.bind(socketAddress) - } - - val listenF = F.delay(linuxSocket.listen(65535)) - - bindF *> listenF *> F - .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) - } - - _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) - - sockets = Stream - .resource(createBufferAux(isIpv6)) - .flatMap { buf => // Buffer that will contain the remote address - Stream - .resource(createBuffer(4)) - .flatMap { - bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer - Stream.resource { - - bufLength.writeInt(buf.capacity()) - - // We accept a connection, we write the remote address on the buf and we get the clientFd - val accept = - Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> - ring - .bracket( - op = IORING_OP_ACCEPT, - fd = linuxSocket.fd(), - bufferAddress = buf.memoryAddress(), - offset = bufLength.memoryAddress() - )(closeSocket(ring, _)) - .mapK { - new cats.~>[IO, IO] { - def apply[A](ioa: IO[A]) = ioa.debug() - } - } - .mapK(LiftIO.liftK) - - // We read the address from the buf and we convert it to SocketAddress - val convert: F[SocketAddress[IpAddress]] = - F.delay( - println( - "[SERVER] getting the address in memory and converting it to SocketAddress..." - ) - ) *> - /* TODO: Merge the next two steps in one: F.delay (SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6))) */ - F.delay { - val netRemoteAddress: InetSocketAddress = - readIpv(buf.memoryAddress(), isIpv6) - println( - s"[SERVER] Read IP address from buffer: ${netRemoteAddress.getHostString()}" - ) - netRemoteAddress - }.flatMap { netRemoteAddress => - F.delay { - println( - s"[SERVER] converted and found inetSocketAddress: $netRemoteAddress" - ) - SocketAddress.fromInetSocketAddress(netRemoteAddress) - } - } - - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) - } - } - .attempt - .map(_.toOption) - }.repeat + ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = for { + ring <- Resource.eval(Uring.get[F]) + resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) + _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) + isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] + linuxSocket <- openSocket(ring, isIpv6) + _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) + _ <- Resource.eval( + F.delay( + linuxSocket.bind( + new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) + ) + ) + ) + _ <- Resource.eval(F.delay(linuxSocket.listen(65535))) + localAddress <- Resource.eval( + F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) + ) + _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) + sockets = for { + buf <- Stream.resource(createBufferAux(isIpv6)) + bufLength <- Stream.resource(createBuffer(4)) + res <- Stream.resource { + bufLength.writeInt(buf.capacity()) + + // Accept a connection, write the remote address on the buf and get the clientFd + val accept: Resource[F, Int] = + Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> + ring + .bracket( + op = IORING_OP_ACCEPT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = bufLength.memoryAddress() + )(closeSocket(ring, _)) + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = ioa.debug() + } } - + .mapK(LiftIO.liftK) + + // Read the address from the buf and convert it to SocketAddress + val convert: F[SocketAddress[IpAddress]] = + F.delay( + println( + "[SERVER] getting the address in memory and converting it to SocketAddress..." + ) + ) *> + F.delay( + SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6)) + ) + + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } } + .attempt + .map(_.toOption) + }.repeat + } yield res + + } yield (localAddress, sockets.unNone) - } yield (localAddress, sockets.unNone) - } private[this] def openSocket( ring: Uring, From 3561b5183f3305f73e91b686f2130f0b2e56cad7 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 7 Aug 2023 20:14:46 +0200 Subject: [PATCH 103/200] Refactor client and serverResource --- .../fs2/io/uring/net/UringSocketGroup.scala | 77 +++++++++++-------- 1 file changed, 47 insertions(+), 30 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index eac41322..f3d13290 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -46,36 +46,42 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = if (isIpv6) createBuffer(SIZEOF_SOCKADDR_IN6) else createBuffer(SIZEOF_SOCKADDR_IN) - def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = - for { - ring <- Resource.eval(Uring.get[F]) - address <- Resource.eval(to.resolve) - isIpv6 = address.host.isInstanceOf[Ipv6Address] - linuxSocket <- openSocket(ring, isIpv6) - _ <- Resource.eval( - createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect - for { - length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) - _ <- F.delay( - println( - s"[CLIENT] address: ${address.toString()}, buffer: ${buf.toString()}, length: $length" - ) + def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = for { + + ring <- Resource.eval(Uring.get[F]) + + address <- Resource.eval(to.resolve) + + isIpv6 = address.host.isInstanceOf[Ipv6Address] + + linuxSocket <- openSocket(ring, isIpv6) + + _ <- Resource.eval( + createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect + for { + length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) + _ <- F.delay( + println( + s"[CLIENT] address: ${address.toString()}, Buffer length: $length" ) - _ <- F.delay(println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}")) - _ <- ring - .call( - op = IORING_OP_CONNECT, - fd = linuxSocket.fd(), - bufferAddress = buf.memoryAddress(), - offset = length.toLong - ) - .to - } yield () - } - ) - _ <- Resource.eval(F.delay(println("[CLIENT] connecting..."))) - socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) - } yield socket + ) + _ <- F.delay(println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}")) + _ <- ring + .call( + op = IORING_OP_CONNECT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = length.toLong + ) + .to + } yield () + } + ) + _ <- Resource.eval(F.delay(println("[CLIENT] connecting..."))) + + socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + + } yield socket def server( address: Option[Host], @@ -93,12 +99,19 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn port: Option[Port], options: List[SocketOption] ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = for { + ring <- Resource.eval(Uring.get[F]) + resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) + _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) + isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] + linuxSocket <- openSocket(ring, isIpv6) + _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) + _ <- Resource.eval( F.delay( linuxSocket.bind( @@ -106,11 +119,15 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) ) ) + _ <- Resource.eval(F.delay(linuxSocket.listen(65535))) + localAddress <- Resource.eval( F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) ) + _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) + sockets = for { buf <- Stream.resource(createBufferAux(isIpv6)) bufLength <- Stream.resource(createBuffer(4)) @@ -154,11 +171,11 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .attempt .map(_.toOption) }.repeat + } yield res } yield (localAddress, sockets.unNone) - private[this] def openSocket( ring: Uring, ipv6: Boolean From a322a731106e52908f498982001da11eafaa1764 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 7 Aug 2023 20:58:12 +0200 Subject: [PATCH 104/200] Wrap write in buffer --- .../jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index f3d13290..325cd621 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -132,10 +132,10 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn buf <- Stream.resource(createBufferAux(isIpv6)) bufLength <- Stream.resource(createBuffer(4)) res <- Stream.resource { - bufLength.writeInt(buf.capacity()) // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = + Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) *> Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> ring .bracket( From 801cb4023a516e2a999582777884d510e54b9aa6 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 7 Aug 2023 20:58:23 +0200 Subject: [PATCH 105/200] add TODO --- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 95bda5ce..13fe1abc 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -79,7 +79,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.memoryAddress(), 0, numBytes, - 0 // TODO: Replace this with MSG_WAITALL + 0 // TODO: Replace with MSG_WAITALL ) bytes <- F.delay { @@ -121,7 +121,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.memoryAddress(), 0, bytes.size, - 0 // TODO Replace this with MSG_NOSIGNAL + 0 // TODO Replace with MSG_NOSIGNAL ) _ <- F.delay(println(s"[SOCKET] Message sent!")) From cfd82f41e1ed32f9098c9a86eeafc39b731e4ec2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 14:14:40 +0200 Subject: [PATCH 106/200] Wrap bufLength write capacity in a Resource[F,A] I wrapped the side effect bufLength.writeInt(buf.capacity()) in Resource.eval(F.delay(...)) and I found some fatal errors running the tests. I think they are due to the interruption. --- hs_err_pid4794.log | 1149 +++++++++++++++++ hs_err_pid8969.log | 1149 +++++++++++++++++ .../fs2/io/uring/net/UringSocketGroup.scala | 138 +- 3 files changed, 2369 insertions(+), 67 deletions(-) create mode 100644 hs_err_pid4794.log create mode 100644 hs_err_pid8969.log diff --git a/hs_err_pid4794.log b/hs_err_pid4794.log new file mode 100644 index 00000000..da28a1c2 --- /dev/null +++ b/hs_err_pid4794.log @@ -0,0 +1,1149 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x00007f22d2f923e4, pid=4794, tid=4837 +# +# JRE version: OpenJDK Runtime Environment (17.0.7+7) (build 17.0.7+7-Ubuntu-0ubuntu123.04) +# Java VM: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, linux-amd64) +# Problematic frame: +# V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 +# +# Core dump will be written. Default location: Core dumps may be processed with "/usr/share/apport/apport -p%p -s%s -c%c -d%d -P%P -u%u -g%g -- %E" (or dumping to /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/core.4794) +# +# If you would like to submit a bug report, please visit: +# Unknown +# + +--------------- S U M M A R Y ------------ + +Command Line: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring sbt.ForkMain 44215 + +Host: Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz, 8 cores, 15G, Ubuntu 23.04 +Time: Tue Aug 8 13:11:46 2023 CEST elapsed time: 12.336484 seconds (0d 0h 0m 12s) + +--------------- T H R E A D --------------- + +Current thread (0x00007f224c545570): JavaThread "io-compute-3" daemon [_thread_in_vm, id=4837, stack(0x00007f229c0f6000,0x00007f229c1f6000)] + +Stack: [0x00007f229c0f6000,0x00007f229c1f6000], sp=0x00007f229c1f3678, free space=1013k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 +V [libjvm.so+0xe881a4] SystemDictionary::resolve_instance_class_or_null(Symbol*, Handle, Handle, JavaThread*)+0x134 +V [libjvm.so+0xe89f46] SystemDictionary::resolve_or_fail(Symbol*, Handle, Handle, bool, JavaThread*)+0x66 +V [libjvm.so+0x602cc9] ConstantPool::klass_at_impl(constantPoolHandle const&, int, JavaThread*)+0x139 +V [libjvm.so+0x603a0a] ConstantPool::klass_ref_at(int, JavaThread*)+0x7a +V [libjvm.so+0xa948f9] LinkInfo::LinkInfo(constantPoolHandle const&, int, JavaThread*)+0x39 +V [libjvm.so+0xa9b19f] LinkResolver::resolve_invoke(CallInfo&, Handle, constantPoolHandle const&, int, Bytecodes::Code, JavaThread*)+0x1df +V [libjvm.so+0x82c947] InterpreterRuntime::resolve_invoke(JavaThread*, Bytecodes::Code)+0x177 +V [libjvm.so+0x82ce67] InterpreterRuntime::resolve_from_cache(JavaThread*, Bytecodes::Code)+0x37 +j fs2.CompositeFailure$.fromResults(Lscala/util/Either;Lscala/util/Either;)Lscala/util/Either;+65 +j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+89 +j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 +j fs2.Stream$NestedStreamOps$$$Lambda$692+0x00000008013703d0.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 +j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 +j cats.effect.IO$$Lambda$299+0x000000080125da28.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 +j cats.effect.IO$$Lambda$300+0x000000080125ddf8.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +J 1728 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f22b57553e4 [0x00007f22b5754020+0x00000000000013c4] +J 1857 c1 cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V (3533 bytes) @ 0x00007f22b57be284 [0x00007f22b57aec40+0x000000000000f644] +j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 +J 1971 c1 cats.effect.IOFiber.run()V (113 bytes) @ 0x00007f22b580256c [0x00007f22b58020c0+0x00000000000004ac] +J 1751% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f22b5766af4 [0x00007f22b5763da0+0x0000000000002d54] +v ~StubRoutines::call_stub +V [libjvm.so+0x831c02] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x302 +V [libjvm.so+0x8332d2] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x1a2 +V [libjvm.so+0x8fb417] thread_entry(JavaThread*, JavaThread*)+0xa7 +V [libjvm.so+0xed302e] JavaThread::thread_main_inner()+0xce +V [libjvm.so+0xed675f] Thread::call_run()+0xbf +V [libjvm.so+0xc2d5c9] thread_native_entry(Thread*)+0xe9 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j fs2.CompositeFailure$.fromResults(Lscala/util/Either;Lscala/util/Either;)Lscala/util/Either;+65 +j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+89 +j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 +j fs2.Stream$NestedStreamOps$$$Lambda$692+0x00000008013703d0.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 +j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 +j cats.effect.IO$$Lambda$299+0x000000080125da28.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 +j cats.effect.IO$$Lambda$300+0x000000080125ddf8.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +J 1728 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f22b57553e4 [0x00007f22b5754020+0x00000000000013c4] +J 1857 c1 cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V (3533 bytes) @ 0x00007f22b57be284 [0x00007f22b57aec40+0x000000000000f644] +j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 +J 1971 c1 cats.effect.IOFiber.run()V (113 bytes) @ 0x00007f22b580256c [0x00007f22b58020c0+0x00000000000004ac] +J 1751% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f22b5766af4 [0x00007f22b5763da0+0x0000000000002d54] +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr: 0x0000000000001000 + +Register to memory mapping: + +RAX=0x00007f2200000010 points into unknown readable memory: 0x000000000007b000 | 00 b0 07 00 00 00 00 00 +RBX=0x00007f224c0fbad0 points into unknown readable memory: 0x00007f226c6bea9e | 9e ea 6b 6c 22 7f 00 00 +RCX=0x0000000000000003 is an unknown value +RDX=0x00007f22d3d12330: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 +RSP=0x00007f229c1f3678 is pointing into the stack for thread: 0x00007f224c545570 +RBP=0x00007f229c1f36a0 is pointing into the stack for thread: 0x00007f224c545570 +RSI=0x0000000000000060 is an unknown value +RDI=0x0000000000001000 is an unknown value +R8 =0x00007f224c03c230 points into unknown readable memory: 0x63730010ea9e0035 | 35 00 9e ea 10 00 73 63 +R9 =0x00007f224c545570 is a thread +R10=0x00007f22d3dcd000 points into unknown readable memory: 0x0100050403020100 | 00 01 02 03 04 05 00 01 +R11=0x0000000000000001 is an unknown value +R12=0x00007f228c00de00 points into unknown readable memory: 0x00007f2200000010 | 10 00 00 00 22 7f 00 00 +R13=0x0000000708616d18 is an oop: java.security.ProtectionDomain +{0x0000000708616d18} - klass: 'java/security/ProtectionDomain' + - ---- fields (total size 5 words): + - private 'hasAllPerm' 'Z' @12 false + - private final 'staticPermissions' 'Z' @13 false + - private 'codesource' 'Ljava/security/CodeSource;' @16 a 'java/security/CodeSource'{0x0000000708616d40} (e10c2da8) + - private 'classloader' 'Ljava/lang/ClassLoader;' @20 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000007ffd59e70} (fffab3ce) + - private 'principals' '[Ljava/security/Principal;' @24 a 'java/security/Principal'[0] {0x0000000708616ee0} (e10c2ddc) + - private 'permissions' 'Ljava/security/PermissionCollection;' @28 a 'sun/security/util/LazyCodeSourcePermissionCollection'{0x0000000708616ef0} (e10c2dde) + - final 'key' 'Ljava/security/ProtectionDomain$Key;' @32 a 'java/security/ProtectionDomain$Key'{0x00000007086170b0} (e10c2e16) +R14=0x00007f22cc193210 points into unknown readable memory: 0x00007f22cc165008 | 08 50 16 cc 22 7f 00 00 +R15=0x00007f224c545b80 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 + + +Registers: +RAX=0x00007f2200000010, RBX=0x00007f224c0fbad0, RCX=0x0000000000000003, RDX=0x00007f22d3d12330 +RSP=0x00007f229c1f3678, RBP=0x00007f229c1f36a0, RSI=0x0000000000000060, RDI=0x0000000000001000 +R8 =0x00007f224c03c230, R9 =0x00007f224c545570, R10=0x00007f22d3dcd000, R11=0x0000000000000001 +R12=0x00007f228c00de00, R13=0x0000000708616d18, R14=0x00007f22cc193210, R15=0x00007f224c545b80 +RIP=0x00007f22d2f923e4, EFLAGS=0x0000000000010206, CSGSFS=0x002b000000000033, ERR=0x0000000000000004 + TRAPNO=0x000000000000000e + +Top of Stack: (sp=0x00007f229c1f3678) +0x00007f229c1f3678: 00007f22d308ffb8 00007f229c1f36a0 +0x00007f229c1f3688: 00007f224c03c230 00007f224c545570 +0x00007f229c1f3698: 00007f224c545b90 00007f229c1f37d0 +0x00007f229c1f36a8: 00007f22d38881a4 00007f229c1f3820 + +Instructions: (pc=0x00007f22d2f923e4) +0x00007f22d2f922e4: 01 eb df 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f922f4: 31 c0 c3 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92304: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92314: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92324: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92334: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92344: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92354: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92364: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92374: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92384: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92394: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923a4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923b4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923c4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923d4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923e4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f923f4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92404: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92414: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92424: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92434: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92444: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92454: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92464: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92474: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92484: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f92494: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f924a4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f +0x00007f22d2f924b4: 48 85 ff 74 05 e8 e2 45 14 00 4c 89 23 5b 41 5c +0x00007f22d2f924c4: 5d c3 66 2e 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f22d2f924d4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x00007f22d308ffb8: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 +stack at sp + 1 slots: 0x00007f229c1f36a0 is pointing into the stack for thread: 0x00007f224c545570 +stack at sp + 2 slots: 0x00007f224c03c230 points into unknown readable memory: 0x63730010ea9e0035 | 35 00 9e ea 10 00 73 63 +stack at sp + 3 slots: 0x00007f224c545570 is a thread +stack at sp + 4 slots: 0x00007f224c545b90 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 +stack at sp + 5 slots: 0x00007f229c1f37d0 is pointing into the stack for thread: 0x00007f224c545570 +stack at sp + 6 slots: 0x00007f22d38881a4: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 +stack at sp + 7 slots: 0x00007f229c1f3820 is pointing into the stack for thread: 0x00007f224c545570 + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x00007f220c3e8a50, length=27, elements={ +0x00007f22cc015d20, 0x00007f22cc179870, 0x00007f22cc17ac50, 0x00007f22cc1801d0, +0x00007f22cc181580, 0x00007f22cc182990, 0x00007f22cc184340, 0x00007f22cc185870, +0x00007f22cc18ecd0, 0x00007f22cc1a76b0, 0x00007f22cc1ab290, 0x00007f2264001140, +0x00007f22cc1ac250, 0x00007f22cc1b1120, 0x00007f22cc373ce0, 0x00007f224c542b10, +0x00007f224c5436b0, 0x00007f224c544660, 0x00007f224c545570, 0x00007f224c546530, +0x00007f224c547440, 0x00007f224c548800, 0x00007f224c549b00, 0x00007f2230006780, +0x00007f227838dfb0, 0x00007f221c029620, 0x00007f220c3e7e00 +} + +Java Threads: ( => current thread ) + 0x00007f22cc015d20 JavaThread "main" [_thread_blocked, id=4797, stack(0x00007f22d2500000,0x00007f22d2600000)] + 0x00007f22cc179870 JavaThread "Reference Handler" daemon [_thread_blocked, id=4804, stack(0x00007f22acc32000,0x00007f22acd32000)] + 0x00007f22cc17ac50 JavaThread "Finalizer" daemon [_thread_blocked, id=4805, stack(0x00007f22acb32000,0x00007f22acc32000)] + 0x00007f22cc1801d0 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=4806, stack(0x00007f22aca32000,0x00007f22acb32000)] + 0x00007f22cc181580 JavaThread "Service Thread" daemon [_thread_blocked, id=4807, stack(0x00007f22ac932000,0x00007f22aca32000)] + 0x00007f22cc182990 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=4808, stack(0x00007f22ac832000,0x00007f22ac932000)] + 0x00007f22cc184340 JavaThread "C2 CompilerThread0" daemon [_thread_in_native, id=4809, stack(0x00007f22ac732000,0x00007f22ac832000)] + 0x00007f22cc185870 JavaThread "C1 CompilerThread0" daemon [_thread_in_native, id=4810, stack(0x00007f22ac632000,0x00007f22ac732000)] + 0x00007f22cc18ecd0 JavaThread "Sweeper thread" daemon [_thread_blocked, id=4811, stack(0x00007f22ac532000,0x00007f22ac632000)] + 0x00007f22cc1a76b0 JavaThread "JDWP Transport Listener: dt_socket" daemon [_thread_blocked, id=4812, stack(0x00007f22ac432000,0x00007f22ac532000)] + 0x00007f22cc1ab290 JavaThread "JDWP Event Helper Thread" daemon [_thread_blocked, id=4813, stack(0x00007f22ac332000,0x00007f22ac432000)] + 0x00007f2264001140 JavaThread "JDWP Command Reader" daemon [_thread_in_native, id=4819, stack(0x00007f22ac232000,0x00007f22ac332000)] + 0x00007f22cc1ac250 JavaThread "Notification Thread" daemon [_thread_blocked, id=4822, stack(0x00007f22ac132000,0x00007f22ac232000)] + 0x00007f22cc1b1120 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=4824, stack(0x00007f229d100000,0x00007f229d200000)] + 0x00007f22cc373ce0 JavaThread "pool-1-thread-1" [_thread_blocked, id=4826, stack(0x00007f229d000000,0x00007f229d100000)] + 0x00007f224c542b10 JavaThread "io-compute-0" daemon [_thread_in_native, id=4834, stack(0x00007f229c3f6000,0x00007f229c4f6000)] + 0x00007f224c5436b0 JavaThread "io-compute-1" daemon [_thread_in_native, id=4835, stack(0x00007f229c2f6000,0x00007f229c3f6000)] + 0x00007f224c544660 JavaThread "io-compute-2" daemon [_thread_in_native, id=4836, stack(0x00007f229c1f6000,0x00007f229c2f6000)] +=>0x00007f224c545570 JavaThread "io-compute-3" daemon [_thread_in_vm, id=4837, stack(0x00007f229c0f6000,0x00007f229c1f6000)] + 0x00007f224c546530 JavaThread "io-compute-7" daemon [_thread_in_native, id=4838, stack(0x00007f222bf00000,0x00007f222c000000)] + 0x00007f224c547440 JavaThread "io-compute-5" daemon [_thread_in_native, id=4839, stack(0x00007f222be00000,0x00007f222bf00000)] + 0x00007f224c548800 JavaThread "io-compute-4" daemon [_thread_in_native, id=4840, stack(0x00007f222bd00000,0x00007f222be00000)] + 0x00007f224c549b00 JavaThread "io-compute-6" daemon [_thread_in_Java, id=4841, stack(0x00007f222bc00000,0x00007f222bd00000)] + 0x00007f2230006780 JavaThread "pool-2-thread-1" [_thread_blocked, id=4845, stack(0x00007f229c9f6000,0x00007f229caf6000)] + 0x00007f227838dfb0 JavaThread "C2 CompilerThread1" daemon [_thread_in_native, id=4851, stack(0x00007f222bb00000,0x00007f222bc00000)] + 0x00007f221c029620 JavaThread "io-compute-blocker-8" daemon [_thread_blocked, id=4853, stack(0x00007f222b7fc000,0x00007f222b8fc000)] + 0x00007f220c3e7e00 JavaThread "C2 CompilerThread2" daemon [_thread_in_native, id=4854, stack(0x00007f222b5fa000,0x00007f222b6fa000)] + +Other Threads: + 0x00007f22cc1758f0 VMThread "VM Thread" [stack: 0x00007f22acd34000,0x00007f22ace34000] [id=4803] + 0x00007f22cc1adaa0 WatcherThread [stack: 0x00007f22ac032000,0x00007f22ac132000] [id=4823] + 0x00007f22cc080fb0 GCTaskThread "GC Thread#0" [stack: 0x00007f22d02e5000,0x00007f22d03e5000] [id=4798] + 0x00007f228c005f90 GCTaskThread "GC Thread#1" [stack: 0x00007f229cf00000,0x00007f229d000000] [id=4827] + 0x00007f228c0069c0 GCTaskThread "GC Thread#2" [stack: 0x00007f229cdfe000,0x00007f229cefe000] [id=4828] + 0x00007f228c0073f0 GCTaskThread "GC Thread#3" [stack: 0x00007f229ccfc000,0x00007f229cdfc000] [id=4829] + 0x00007f228c007e20 GCTaskThread "GC Thread#4" [stack: 0x00007f229cbfa000,0x00007f229ccfa000] [id=4830] + 0x00007f228c008850 GCTaskThread "GC Thread#5" [stack: 0x00007f229caf8000,0x00007f229cbf8000] [id=4831] + 0x00007f228c00c9d0 GCTaskThread "GC Thread#6" [stack: 0x00007f222ba00000,0x00007f222bb00000] [id=4843] + 0x00007f228c00cf50 GCTaskThread "GC Thread#7" [stack: 0x00007f222b8fe000,0x00007f222b9fe000] [id=4844] + 0x00007f22cc091d10 ConcurrentGCThread "G1 Main Marker" [stack: 0x00007f22d01e3000,0x00007f22d02e3000] [id=4799] + 0x00007f22cc092c00 ConcurrentGCThread "G1 Conc#0" [stack: 0x00007f22d00e1000,0x00007f22d01e1000] [id=4800] + 0x00007f22a0000ea0 ConcurrentGCThread "G1 Conc#1" [stack: 0x00007f222b6fc000,0x00007f222b7fc000] [id=4848] + 0x00007f22cc146590 ConcurrentGCThread "G1 Refine#0" [stack: 0x00007f22ad009000,0x00007f22ad109000] [id=4801] + 0x00007f22cc147480 ConcurrentGCThread "G1 Service" [stack: 0x00007f22acf07000,0x00007f22ad007000] [id=4802] + +Threads with active compile tasks: +C2 CompilerThread0 12483 1801 4 java.lang.invoke.MethodType::makeImpl (109 bytes) +C1 CompilerThread0 12483 2118 3 cats.effect.unsafe.FiberMonitor::monitorSuspended (45 bytes) +C2 CompilerThread1 12483 2043 ! 4 cats.effect.unsafe.WorkerThread::run (1765 bytes) +C2 CompilerThread2 12483 2120 4 cats.effect.IO$$anon$5::flatMap (10 bytes) + +VM state: not at safepoint (normal execution) + +VM Mutex/Monitor currently owned by a thread: None + +Heap address: 0x0000000708200000, size: 3966 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x0000000800000000-0x0000000800be2000-0x0000000800be2000), size 12460032, SharedBaseAddress: 0x0000000800000000, ArchiveRelocationMode: 0. +Compressed class space mapped at: 0x0000000801000000-0x0000000841000000, reserved size: 1073741824 +Narrow klass base: 0x0000000800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 8 total, 8 available + Memory: 15860M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 248M + Heap Max Capacity: 3966M + Pre-touch: Disabled + Parallel Workers: 8 + Concurrent Workers: 2 + Concurrent Refinement Workers: 8 + Periodic GC: Disabled + +Heap: + garbage-first heap total 49152K, used 17737K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 9 young (18432K), 3 survivors (6144K) + Metaspace used 29726K, committed 30144K, reserved 1114112K + class space used 3318K, committed 3520K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000708200000, 0x000000070835a7f8, 0x0000000708400000| 67%| S|CS|TAMS 0x0000000708200000, 0x0000000708200000| Complete +| 1|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| S|CS|TAMS 0x0000000708400000, 0x0000000708400000| Complete +| 2|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| S|CS|TAMS 0x0000000708600000, 0x0000000708600000| Complete +| 3|0x0000000708800000, 0x0000000708800000, 0x0000000708a00000| 0%| F| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked +| 4|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked +| 5|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked +| 6|0x0000000708e00000, 0x0000000708e00000, 0x0000000709000000| 0%| F| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked +| 7|0x0000000709000000, 0x0000000709000000, 0x0000000709200000| 0%| F| |TAMS 0x0000000709000000, 0x0000000709000000| Untracked +| 8|0x0000000709200000, 0x0000000709200000, 0x0000000709400000| 0%| F| |TAMS 0x0000000709200000, 0x0000000709200000| Untracked +| 9|0x0000000709400000, 0x0000000709400000, 0x0000000709600000| 0%| F| |TAMS 0x0000000709400000, 0x0000000709400000| Untracked +| 10|0x0000000709600000, 0x0000000709600000, 0x0000000709800000| 0%| F| |TAMS 0x0000000709600000, 0x0000000709600000| Untracked +| 11|0x0000000709800000, 0x0000000709800000, 0x0000000709a00000| 0%| F| |TAMS 0x0000000709800000, 0x0000000709800000| Untracked +| 12|0x0000000709a00000, 0x0000000709a00000, 0x0000000709c00000| 0%| F| |TAMS 0x0000000709a00000, 0x0000000709a00000| Untracked +| 13|0x0000000709c00000, 0x0000000709c00000, 0x0000000709e00000| 0%| F| |TAMS 0x0000000709c00000, 0x0000000709c00000| Untracked +| 14|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked +| 15|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked +| 16|0x000000070a200000, 0x000000070a30efe0, 0x000000070a400000| 52%| E| |TAMS 0x000000070a200000, 0x000000070a200000| Complete +| 109|0x0000000715c00000, 0x0000000715e00000, 0x0000000715e00000|100%| E|CS|TAMS 0x0000000715c00000, 0x0000000715c00000| Complete +| 110|0x0000000715e00000, 0x0000000716000000, 0x0000000716000000|100%| E|CS|TAMS 0x0000000715e00000, 0x0000000715e00000| Complete +| 111|0x0000000716000000, 0x0000000716200000, 0x0000000716200000|100%| E|CS|TAMS 0x0000000716000000, 0x0000000716000000| Complete +| 122|0x0000000717600000, 0x0000000717800000, 0x0000000717800000|100%| E|CS|TAMS 0x0000000717600000, 0x0000000717600000| Complete +| 123|0x0000000717800000, 0x0000000717a00000, 0x0000000717a00000|100%| E|CS|TAMS 0x0000000717800000, 0x0000000717800000| Complete +|1981|0x00000007ffc00000, 0x00000007ffd76000, 0x00000007ffe00000| 73%|OA| |TAMS 0x00000007ffd76000, 0x00000007ffc00000| Untracked +|1982|0x00000007ffe00000, 0x00000007ffe82000, 0x0000000800000000| 25%|CA| |TAMS 0x00000007ffe82000, 0x00000007ffe00000| Untracked + +Card table byte_map: [0x00007f22d0ba4000,0x00007f22d1363000] _byte_map_base: 0x00007f22cd363000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x00007f22cc081a00, (CMBitMap*) 0x00007f22cc0819c0 + Prev Bits: [0x00007f22ad810000, 0x00007f22b1608000) + Next Bits: [0x00007f22b1608000, 0x00007f22b5400000) + +Polling page: 0x00007f22d40a2000 + +Metaspace: + +Usage: + Non-class: 25.79 MB used. + Class: 3.24 MB used. + Both: 29.03 MB used. + +Virtual space: + Non-class space: 64.00 MB reserved, 26.00 MB ( 41%) committed, 1 nodes. + Class space: 1.00 GB reserved, 3.44 MB ( <1%) committed, 1 nodes. + Both: 1.06 GB reserved, 29.44 MB ( 3%) committed. + +Chunk freelists: + Non-Class: 5.60 MB + Class: 12.44 MB + Both: 18.03 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 35.12 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 3. +num_arena_births: 442. +num_arena_deaths: 0. +num_vsnodes_births: 2. +num_vsnodes_deaths: 0. +num_space_committed: 471. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 3. +num_chunks_taken_from_freelist: 1465. +num_chunk_merges: 3. +num_chunk_splits: 1034. +num_chunks_enlarged: 816. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=120028Kb used=1311Kb max_used=1311Kb free=118716Kb + bounds [0x00007f22bcec9000, 0x00007f22bd139000, 0x00007f22c4400000] +CodeHeap 'profiled nmethods': size=120028Kb used=4340Kb max_used=4340Kb free=115687Kb + bounds [0x00007f22b5400000, 0x00007f22b5840000, 0x00007f22bc937000] +CodeHeap 'non-nmethods': size=5704Kb used=1348Kb max_used=1414Kb free=4355Kb + bounds [0x00007f22bc937000, 0x00007f22bcba7000, 0x00007f22bcec9000] + total_blobs=2722 nmethods=2097 adapters=539 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 12.260 Thread 0x00007f22cc185870 2022 1 cats.effect.kernel.Outcome$Succeeded::fa (5 bytes) +Event: 12.260 Thread 0x00007f22cc185870 nmethod 2022 0x00007f22bd006a90 code [0x00007f22bd006c20, 0x00007f22bd006cf0] +Event: 12.260 Thread 0x00007f22cc185870 2016 1 fs2.Chunk$Singleton::value (5 bytes) +Event: 12.260 Thread 0x00007f22cc185870 nmethod 2016 0x00007f22bd006d90 code [0x00007f22bd006f20, 0x00007f22bd006ff0] +Event: 12.260 Thread 0x00007f22cc185870 2034 1 fs2.internal.ScopedResource$State::open (5 bytes) +Event: 12.260 Thread 0x00007f22cc185870 nmethod 2034 0x00007f22bd007090 code [0x00007f22bd007220, 0x00007f22bd0072f0] +Event: 12.260 Thread 0x00007f22cc185870 2042 3 cats.effect.CallbackStack$::apply (9 bytes) +Event: 12.261 Thread 0x00007f22cc185870 nmethod 2042 0x00007f22b581a210 code [0x00007f22b581a3c0, 0x00007f22b581a6b0] +Event: 12.261 Thread 0x00007f22cc185870 2025 1 scala.collection.mutable.ArrayBuffer::size0_$eq (6 bytes) +Event: 12.261 Thread 0x00007f22cc185870 nmethod 2025 0x00007f22bd007390 code [0x00007f22bd007520, 0x00007f22bd0075f0] +Event: 12.261 Thread 0x00007f22cc185870 2026 1 scala.collection.mutable.ArrayBuffer::array_$eq (6 bytes) +Event: 12.262 Thread 0x00007f22cc185870 nmethod 2026 0x00007f22bd007690 code [0x00007f22bd007820, 0x00007f22bd007930] +Event: 12.262 Thread 0x00007f22cc185870 2041 1 fs2.internal.ScopedResource$State::leases (5 bytes) +Event: 12.262 Thread 0x00007f22cc185870 nmethod 2041 0x00007f22bd007990 code [0x00007f22bd007b20, 0x00007f22bd007bf0] +Event: 12.265 Thread 0x00007f22cc185870 2044 3 cats.data.Chain$::nil (4 bytes) +Event: 12.265 Thread 0x00007f22cc185870 nmethod 2044 0x00007f22b581a810 code [0x00007f22b581a9a0, 0x00007f22b581aab0] +Event: 12.265 Thread 0x00007f22cc185870 2045 3 cats.Eval:: (5 bytes) +Event: 12.266 Thread 0x00007f22cc185870 nmethod 2045 0x00007f22b581ab90 code [0x00007f22b581ad40, 0x00007f22b581ae90] +Event: 12.272 Thread 0x00007f227838dfb0 nmethod 2017 0x00007f22bd007c90 code [0x00007f22bd007e40, 0x00007f22bd0080e8] +Event: 12.272 Thread 0x00007f227838dfb0 2043 ! 4 cats.effect.unsafe.WorkerThread::run (1765 bytes) + +GC Heap History (8 events): +Event: 2.511 GC heap before +{Heap before GC invocations=0 (full 0): + garbage-first heap total 258048K, used 22496K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 11 young (22528K), 0 survivors (0K) + Metaspace used 7378K, committed 7488K, reserved 1114112K + class space used 779K, committed 832K, reserved 1048576K +} +Event: 2.520 GC heap after +{Heap after GC invocations=1 (full 0): + garbage-first heap total 258048K, used 4789K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 7378K, committed 7488K, reserved 1114112K + class space used 779K, committed 832K, reserved 1048576K +} +Event: 5.614 GC heap before +{Heap before GC invocations=1 (full 0): + garbage-first heap total 258048K, used 35509K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 18 young (36864K), 2 survivors (4096K) + Metaspace used 16783K, committed 17024K, reserved 1114112K + class space used 1732K, committed 1856K, reserved 1048576K +} +Event: 5.626 GC heap after +{Heap after GC invocations=2 (full 0): + garbage-first heap total 258048K, used 5916K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 16783K, committed 17024K, reserved 1114112K + class space used 1732K, committed 1856K, reserved 1048576K +} +Event: 8.132 GC heap before +{Heap before GC invocations=2 (full 0): + garbage-first heap total 258048K, used 28444K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 14 young (28672K), 2 survivors (4096K) + Metaspace used 21250K, committed 21504K, reserved 1114112K + class space used 2228K, committed 2368K, reserved 1048576K +} +Event: 8.146 GC heap after +{Heap after GC invocations=3 (full 0): + garbage-first heap total 258048K, used 6546K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 21250K, committed 21504K, reserved 1114112K + class space used 2228K, committed 2368K, reserved 1048576K +} +Event: 11.186 GC heap before +{Heap before GC invocations=4 (full 0): + garbage-first heap total 49152K, used 35218K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 19 young (38912K), 3 survivors (6144K) + Metaspace used 27948K, committed 28288K, reserved 1114112K + class space used 2972K, committed 3136K, reserved 1048576K +} +Event: 11.200 GC heap after +{Heap after GC invocations=5 (full 0): + garbage-first heap total 49152K, used 7497K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 27948K, committed 28288K, reserved 1114112K + class space used 2972K, committed 3136K, reserved 1048576K +} + +Dll operation events (11 events): +Event: 0.008 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +Event: 0.046 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +Event: 0.902 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +Event: 0.945 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +Event: 1.071 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +Event: 1.133 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +Event: 1.178 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +Event: 1.457 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +Event: 3.415 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +Event: 3.422 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +Event: 3.512 Loaded shared library /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so + +Deoptimization events (20 events): +Event: 11.983 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa268 mode 2 +Event: 11.983 Thread 0x00007f224c5436b0 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c3f4268 mode 2 +Event: 12.032 Thread 0x00007f221c029620 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcf395b0 relative=0x0000000000000210 +Event: 12.032 Thread 0x00007f221c029620 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcf395b0 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 +Event: 12.032 Thread 0x00007f221c029620 DEOPT PACKING pc=0x00007f22bcf395b0 sp=0x00007f222b8fa2c0 +Event: 12.032 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa200 mode 2 +Event: 12.066 Thread 0x00007f224c545570 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcf33fb0 relative=0x0000000000000190 +Event: 12.066 Thread 0x00007f224c545570 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcf33fb0 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 +Event: 12.066 Thread 0x00007f224c545570 DEOPT PACKING pc=0x00007f22bcf33fb0 sp=0x00007f229c1f4270 +Event: 12.066 Thread 0x00007f224c545570 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c1f4218 mode 2 +Event: 12.154 Thread 0x00007f221c029620 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcfefb6c relative=0x00000000000004ec +Event: 12.154 Thread 0x00007f221c029620 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcfefb6c method=scala.Option.isEmpty()Z @ 4 c2 +Event: 12.154 Thread 0x00007f221c029620 DEOPT PACKING pc=0x00007f22bcfefb6c sp=0x00007f222b8fa4f0 +Event: 12.154 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa078 mode 2 +Event: 12.177 Thread 0x00007f224c5436b0 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcfffc0c relative=0x000000000000048c +Event: 12.177 Thread 0x00007f224c5436b0 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcfffc0c method=scala.Option.isEmpty()Z @ 4 c2 +Event: 12.177 Thread 0x00007f224c5436b0 DEOPT PACKING pc=0x00007f22bcfffc0c sp=0x00007f229c3f44a0 +Event: 12.177 Thread 0x00007f224c5436b0 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c3f40e8 mode 2 +Event: 12.261 Thread 0x00007f224c547440 DEOPT PACKING pc=0x00007f22b577fcd0 sp=0x00007f222befe660 +Event: 12.261 Thread 0x00007f224c547440 DEOPT UNPACKING pc=0x00007f22bc98de2f sp=0x00007f222befdcf0 mode 0 + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 11.555 Thread 0x00007f224c542b10 Exception (0x000000071775de90) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.559 Thread 0x00007f224c5436b0 Exception (0x000000071778e0d0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.585 Thread 0x00007f224c547440 Exception (0x0000000716001210) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.601 Thread 0x00007f221c029620 Exception (0x00000007177ba870) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.647 Thread 0x00007f221c029620 Exception (0x0000000716087af0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.653 Thread 0x00007f221c029620 Exception (0x000000071608c658) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.657 Thread 0x00007f221c029620 Exception (0x0000000716090e10) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.730 Thread 0x00007f224c545570 Exception (0x000000071611fb28) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.868 Thread 0x00007f224c545570 Exception (0x00000007161e4b60) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.877 Thread 0x00007f224c545570 Exception (0x00000007161f8ab8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.880 Thread 0x00007f224c545570 Exception (0x0000000715e00aa8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.914 Thread 0x00007f224c545570 Exception (0x0000000715e52478) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.917 Thread 0x00007f224c545570 Exception (0x0000000715e72f70) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.920 Thread 0x00007f224c545570 Exception (0x0000000715e77ad8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.923 Thread 0x00007f224c545570 Exception (0x0000000715e80c68) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.927 Thread 0x00007f224c545570 Exception (0x0000000715e8d210) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.930 Thread 0x00007f224c545570 Exception (0x0000000715e91d78) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.984 Thread 0x00007f224c546530 Exception (0x0000000715ea07c0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.999 Thread 0x00007f224c546530 Exception (0x0000000715f64a08) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 12.002 Thread 0x00007f224c546530 Exception (0x0000000715f6cbc0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] + +VM Operations (20 events): +Event: 6.626 Executing VM operation: Cleanup +Event: 6.627 Executing VM operation: Cleanup done +Event: 7.345 Executing VM operation: HandshakeAllThreads +Event: 7.345 Executing VM operation: HandshakeAllThreads done +Event: 8.132 Executing VM operation: CollectForMetadataAllocation +Event: 8.146 Executing VM operation: CollectForMetadataAllocation done +Event: 8.169 Executing VM operation: G1Concurrent +Event: 8.176 Executing VM operation: G1Concurrent done +Event: 8.177 Executing VM operation: G1Concurrent +Event: 8.179 Executing VM operation: G1Concurrent done +Event: 9.179 Executing VM operation: Cleanup +Event: 9.179 Executing VM operation: Cleanup done +Event: 10.119 Executing VM operation: HandshakeAllThreads +Event: 10.121 Executing VM operation: HandshakeAllThreads done +Event: 11.121 Executing VM operation: Cleanup +Event: 11.122 Executing VM operation: Cleanup done +Event: 11.185 Executing VM operation: G1CollectForAllocation +Event: 11.200 Executing VM operation: G1CollectForAllocation done +Event: 12.175 Executing VM operation: HandshakeAllThreads +Event: 12.176 Executing VM operation: HandshakeAllThreads done + +Events (20 events): +Event: 10.120 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 +Event: 10.125 loading class javax/security/cert/X509Certificate +Event: 10.125 loading class javax/security/cert/Certificate +Event: 10.125 loading class javax/security/cert/Certificate done +Event: 10.126 loading class javax/security/cert/X509Certificate done +Event: 10.130 loading class java/nio/ReadOnlyBufferException +Event: 10.131 loading class java/nio/ReadOnlyBufferException done +Event: 10.160 loading class java/util/concurrent/atomic/Striped64$1 +Event: 10.160 loading class java/util/concurrent/atomic/Striped64$1 done +Event: 10.992 Thread 0x00007f226c42fc00 Thread exited: 0x00007f226c42fc00 +Event: 10.993 Thread 0x00007f227838dfb0 Thread exited: 0x00007f227838dfb0 +Event: 11.076 Thread 0x00007f227838dfb0 Thread added: 0x00007f227838dfb0 +Event: 11.076 Protecting memory [0x00007f222bb00000,0x00007f222bb04000] with protection modes 0 +Event: 11.076 Thread 0x00007f22781ddc90 Thread added: 0x00007f22781ddc90 +Event: 11.110 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 +Event: 11.301 Thread 0x00007f22781ddc90 Thread exited: 0x00007f22781ddc90 +Event: 11.466 Thread 0x00007f221c029620 Thread added: 0x00007f221c029620 +Event: 11.466 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 +Event: 11.797 Thread 0x00007f220c3e7e00 Thread added: 0x00007f220c3e7e00 +Event: 11.798 Protecting memory [0x00007f222b5fa000,0x00007f222b5fe000] with protection modes 0 + + +Dynamic libraries: +708200000-70a400000 rw-p 00000000 00:00 0 +70a400000-715c00000 ---p 00000000 00:00 0 +715c00000-716200000 rw-p 00000000 00:00 0 +716200000-717600000 ---p 00000000 00:00 0 +717600000-717a00000 rw-p 00000000 00:00 0 +717a00000-7ffc00000 ---p 00000000 00:00 0 +7ffc00000-7ffd00000 rw-p 00000000 00:00 0 +7ffd00000-7ffd76000 rw-p 00c9d000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +7ffd76000-7ffe00000 rw-p 00000000 00:00 0 +7ffe00000-7ffe82000 rw-p 00c1b000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +7ffe82000-800000000 rw-p 00000000 00:00 0 +800000000-800be2000 rw-p 00001000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +800be2000-801000000 ---p 00000000 00:00 0 +801000000-801030000 rw-p 00000000 00:00 0 +801030000-801130000 rw-p 00000000 00:00 0 +801130000-801170000 rw-p 00000000 00:00 0 +801170000-8011b0000 rw-p 00000000 00:00 0 +8011b0000-8011d0000 rw-p 00000000 00:00 0 +8011d0000-801200000 ---p 00000000 00:00 0 +801200000-8013a0000 rw-p 00000000 00:00 0 +8013a0000-841000000 ---p 00000000 00:00 0 +556b97447000-556b97448000 r--p 00000000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +556b97448000-556b97449000 r-xp 00001000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +556b97449000-556b9744a000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +556b9744a000-556b9744b000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +556b9744b000-556b9744c000 rw-p 00003000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +556b98d13000-556b98d5b000 rw-p 00000000 00:00 0 [heap] +7f21f8000000-7f21f8121000 rw-p 00000000 00:00 0 +7f21f8121000-7f21fc000000 ---p 00000000 00:00 0 +7f2200000000-7f220030a000 rw-p 00000000 00:00 0 +7f220030a000-7f2204000000 ---p 00000000 00:00 0 +7f2204000000-7f22040f0000 rw-p 00000000 00:00 0 +7f22040f0000-7f2208000000 ---p 00000000 00:00 0 +7f2208000000-7f2208021000 rw-p 00000000 00:00 0 +7f2208021000-7f220c000000 ---p 00000000 00:00 0 +7f220c000000-7f220c64f000 rw-p 00000000 00:00 0 +7f220c64f000-7f2210000000 ---p 00000000 00:00 0 +7f2210000000-7f2210161000 rw-p 00000000 00:00 0 +7f2210161000-7f2214000000 ---p 00000000 00:00 0 +7f2214000000-7f221419d000 rw-p 00000000 00:00 0 +7f221419d000-7f2218000000 ---p 00000000 00:00 0 +7f2218000000-7f22180ab000 rw-p 00000000 00:00 0 +7f22180ab000-7f221c000000 ---p 00000000 00:00 0 +7f221c000000-7f221c35c000 rw-p 00000000 00:00 0 +7f221c35c000-7f2220000000 ---p 00000000 00:00 0 +7f2220000000-7f22200f3000 rw-p 00000000 00:00 0 +7f22200f3000-7f2224000000 ---p 00000000 00:00 0 +7f2224000000-7f22240f3000 rw-p 00000000 00:00 0 +7f22240f3000-7f2228000000 ---p 00000000 00:00 0 +7f222b214000-7f222b5fa000 rw-p 00000000 00:00 0 +7f222b5fa000-7f222b5fe000 ---p 00000000 00:00 0 +7f222b5fe000-7f222b6fa000 rw-p 00000000 00:00 0 +7f222b6fa000-7f222b6fb000 ---p 00000000 00:00 0 +7f222b6fb000-7f222b7fc000 rw-p 00000000 00:00 0 +7f222b7fc000-7f222b800000 ---p 00000000 00:00 0 +7f222b800000-7f222b8fc000 rw-p 00000000 00:00 0 +7f222b8fc000-7f222b8fd000 ---p 00000000 00:00 0 +7f222b8fd000-7f222b9fe000 rw-p 00000000 00:00 0 +7f222b9fe000-7f222b9ff000 ---p 00000000 00:00 0 +7f222b9ff000-7f222bb00000 rw-p 00000000 00:00 0 +7f222bb00000-7f222bb04000 ---p 00000000 00:00 0 +7f222bb04000-7f222bc00000 rw-p 00000000 00:00 0 +7f222bc00000-7f222bc04000 ---p 00000000 00:00 0 +7f222bc04000-7f222bd00000 rw-p 00000000 00:00 0 +7f222bd00000-7f222bd04000 ---p 00000000 00:00 0 +7f222bd04000-7f222be00000 rw-p 00000000 00:00 0 +7f222be00000-7f222be04000 ---p 00000000 00:00 0 +7f222be04000-7f222bf00000 rw-p 00000000 00:00 0 +7f222bf00000-7f222bf04000 ---p 00000000 00:00 0 +7f222bf04000-7f222c056000 rw-p 00000000 00:00 0 +7f222c056000-7f2230000000 ---p 00000000 00:00 0 +7f2230000000-7f223004f000 rw-p 00000000 00:00 0 +7f223004f000-7f2234000000 ---p 00000000 00:00 0 +7f2234000000-7f2234181000 rw-p 00000000 00:00 0 +7f2234181000-7f2238000000 ---p 00000000 00:00 0 +7f2238000000-7f223807a000 rw-p 00000000 00:00 0 +7f223807a000-7f223c000000 ---p 00000000 00:00 0 +7f223c000000-7f223c021000 rw-p 00000000 00:00 0 +7f223c021000-7f2240000000 ---p 00000000 00:00 0 +7f2240000000-7f2240021000 rw-p 00000000 00:00 0 +7f2240021000-7f2244000000 ---p 00000000 00:00 0 +7f2244000000-7f2244021000 rw-p 00000000 00:00 0 +7f2244021000-7f2248000000 ---p 00000000 00:00 0 +7f2248000000-7f2248021000 rw-p 00000000 00:00 0 +7f2248021000-7f224c000000 ---p 00000000 00:00 0 +7f224c000000-7f224c623000 rw-p 00000000 00:00 0 +7f224c623000-7f2250000000 ---p 00000000 00:00 0 +7f2250000000-7f2250021000 rw-p 00000000 00:00 0 +7f2250021000-7f2254000000 ---p 00000000 00:00 0 +7f2254000000-7f2254021000 rw-p 00000000 00:00 0 +7f2254021000-7f2258000000 ---p 00000000 00:00 0 +7f2258000000-7f2258021000 rw-p 00000000 00:00 0 +7f2258021000-7f225c000000 ---p 00000000 00:00 0 +7f225c000000-7f225c021000 rw-p 00000000 00:00 0 +7f225c021000-7f2260000000 ---p 00000000 00:00 0 +7f2260000000-7f2260021000 rw-p 00000000 00:00 0 +7f2260021000-7f2264000000 ---p 00000000 00:00 0 +7f2264000000-7f2264021000 rw-p 00000000 00:00 0 +7f2264021000-7f2268000000 ---p 00000000 00:00 0 +7f2268000000-7f2268082000 rw-p 00000000 00:00 0 +7f2268082000-7f226c000000 ---p 00000000 00:00 0 +7f226c000000-7f226c6c0000 rw-p 00000000 00:00 0 +7f226c6c0000-7f2270000000 ---p 00000000 00:00 0 +7f2270000000-7f2270021000 rw-p 00000000 00:00 0 +7f2270021000-7f2274000000 ---p 00000000 00:00 0 +7f2274000000-7f2274021000 rw-p 00000000 00:00 0 +7f2274021000-7f2278000000 ---p 00000000 00:00 0 +7f2278000000-7f22789f2000 rw-p 00000000 00:00 0 +7f22789f2000-7f227c000000 ---p 00000000 00:00 0 +7f227c000000-7f227c021000 rw-p 00000000 00:00 0 +7f227c021000-7f2280000000 ---p 00000000 00:00 0 +7f2280000000-7f2280021000 rw-p 00000000 00:00 0 +7f2280021000-7f2284000000 ---p 00000000 00:00 0 +7f2284000000-7f2284021000 rw-p 00000000 00:00 0 +7f2284021000-7f2288000000 ---p 00000000 00:00 0 +7f2288000000-7f2288021000 rw-p 00000000 00:00 0 +7f2288021000-7f228c000000 ---p 00000000 00:00 0 +7f228c000000-7f228c021000 rw-p 00000000 00:00 0 +7f228c021000-7f2290000000 ---p 00000000 00:00 0 +7f2290000000-7f2290021000 rw-p 00000000 00:00 0 +7f2290021000-7f2294000000 ---p 00000000 00:00 0 +7f2294000000-7f22943f0000 rw-p 00000000 00:00 0 +7f22943f0000-7f2294d90000 rw-p 00000000 00:00 0 +7f2294d90000-7f2294db0000 rw-p 00000000 00:00 0 +7f2294db0000-7f2294ef0000 rw-p 00000000 00:00 0 +7f2294ef0000-7f2294f90000 rw-p 00000000 00:00 0 +7f2294f90000-7f2295000000 ---p 00000000 00:00 0 +7f2295000000-7f2295a70000 rw-p 00000000 00:00 0 +7f2295a70000-7f2298000000 ---p 00000000 00:00 0 +7f2298000000-7f2298021000 rw-p 00000000 00:00 0 +7f2298021000-7f229c000000 ---p 00000000 00:00 0 +7f229c0f6000-7f229c0fa000 ---p 00000000 00:00 0 +7f229c0fa000-7f229c1f6000 rw-p 00000000 00:00 0 +7f229c1f6000-7f229c1fa000 ---p 00000000 00:00 0 +7f229c1fa000-7f229c2f6000 rw-p 00000000 00:00 0 +7f229c2f6000-7f229c2fa000 ---p 00000000 00:00 0 +7f229c2fa000-7f229c3f6000 rw-p 00000000 00:00 0 +7f229c3f6000-7f229c3fa000 ---p 00000000 00:00 0 +7f229c3fa000-7f229c4f6000 rw-p 00000000 00:00 0 +7f229c4f6000-7f229c536000 rw-s 10000000 00:0e 48673 anon_inode:[io_uring] +7f229c536000-7f229c55b000 rw-s 00000000 00:0e 48673 anon_inode:[io_uring] +7f229c55b000-7f229c59b000 rw-s 10000000 00:0e 48671 anon_inode:[io_uring] +7f229c59b000-7f229c5c0000 rw-s 00000000 00:0e 48671 anon_inode:[io_uring] +7f229c5c0000-7f229c600000 rw-s 10000000 00:0e 48669 anon_inode:[io_uring] +7f229c600000-7f229c611000 r-xp 00000000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) +7f229c611000-7f229c811000 ---p 00011000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) +7f229c811000-7f229c813000 rw-p 00011000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) +7f229c822000-7f229c847000 rw-s 00000000 00:0e 48669 anon_inode:[io_uring] +7f229c847000-7f229c887000 rw-s 10000000 00:0e 48667 anon_inode:[io_uring] +7f229c887000-7f229c8ac000 rw-s 00000000 00:0e 48667 anon_inode:[io_uring] +7f229c8ac000-7f229c8ec000 rw-s 10000000 00:0e 48665 anon_inode:[io_uring] +7f229c8ec000-7f229c911000 rw-s 00000000 00:0e 48665 anon_inode:[io_uring] +7f229c911000-7f229c951000 rw-s 10000000 00:0e 48663 anon_inode:[io_uring] +7f229c951000-7f229c976000 rw-s 00000000 00:0e 48663 anon_inode:[io_uring] +7f229c976000-7f229c9b6000 rw-s 10000000 00:0e 48661 anon_inode:[io_uring] +7f229c9b6000-7f229c9f6000 rw-s 10000000 00:0e 48659 anon_inode:[io_uring] +7f229c9f6000-7f229c9fa000 ---p 00000000 00:00 0 +7f229c9fa000-7f229caf6000 rw-p 00000000 00:00 0 +7f229caf6000-7f229caf7000 ---p 00000000 00:00 0 +7f229caf7000-7f229cbf8000 rw-p 00000000 00:00 0 +7f229cbf8000-7f229cbf9000 ---p 00000000 00:00 0 +7f229cbf9000-7f229ccfa000 rw-p 00000000 00:00 0 +7f229ccfa000-7f229ccfb000 ---p 00000000 00:00 0 +7f229ccfb000-7f229cdfc000 rw-p 00000000 00:00 0 +7f229cdfc000-7f229cdfd000 ---p 00000000 00:00 0 +7f229cdfd000-7f229cefe000 rw-p 00000000 00:00 0 +7f229cefe000-7f229ceff000 ---p 00000000 00:00 0 +7f229ceff000-7f229d000000 rw-p 00000000 00:00 0 +7f229d000000-7f229d004000 ---p 00000000 00:00 0 +7f229d004000-7f229d100000 rw-p 00000000 00:00 0 +7f229d100000-7f229d104000 ---p 00000000 00:00 0 +7f229d104000-7f229d200000 rw-p 00000000 00:00 0 +7f229d200000-7f229dfea000 r--p 00000000 103:02 1055507 /usr/lib/locale/locale-archive +7f229e000000-7f22a0021000 rw-p 00000000 00:00 0 +7f22a0021000-7f22a4000000 ---p 00000000 00:00 0 +7f22a4000000-7f22a4021000 rw-p 00000000 00:00 0 +7f22a4021000-7f22a8000000 ---p 00000000 00:00 0 +7f22a8000000-7f22a8021000 rw-p 00000000 00:00 0 +7f22a8021000-7f22ac000000 ---p 00000000 00:00 0 +7f22ac00b000-7f22ac030000 rw-s 00000000 00:0e 48661 anon_inode:[io_uring] +7f22ac030000-7f22ac031000 ---p 00000000 00:00 0 +7f22ac031000-7f22ac132000 rw-p 00000000 00:00 0 +7f22ac132000-7f22ac136000 ---p 00000000 00:00 0 +7f22ac136000-7f22ac232000 rw-p 00000000 00:00 0 +7f22ac232000-7f22ac236000 ---p 00000000 00:00 0 +7f22ac236000-7f22ac332000 rw-p 00000000 00:00 0 +7f22ac332000-7f22ac336000 ---p 00000000 00:00 0 +7f22ac336000-7f22ac432000 rw-p 00000000 00:00 0 +7f22ac432000-7f22ac436000 ---p 00000000 00:00 0 +7f22ac436000-7f22ac532000 rw-p 00000000 00:00 0 +7f22ac532000-7f22ac536000 ---p 00000000 00:00 0 +7f22ac536000-7f22ac632000 rw-p 00000000 00:00 0 +7f22ac632000-7f22ac636000 ---p 00000000 00:00 0 +7f22ac636000-7f22ac732000 rw-p 00000000 00:00 0 +7f22ac732000-7f22ac736000 ---p 00000000 00:00 0 +7f22ac736000-7f22ac832000 rw-p 00000000 00:00 0 +7f22ac832000-7f22ac836000 ---p 00000000 00:00 0 +7f22ac836000-7f22ac932000 rw-p 00000000 00:00 0 +7f22ac932000-7f22ac936000 ---p 00000000 00:00 0 +7f22ac936000-7f22aca32000 rw-p 00000000 00:00 0 +7f22aca32000-7f22aca36000 ---p 00000000 00:00 0 +7f22aca36000-7f22acb32000 rw-p 00000000 00:00 0 +7f22acb32000-7f22acb36000 ---p 00000000 00:00 0 +7f22acb36000-7f22acc32000 rw-p 00000000 00:00 0 +7f22acc32000-7f22acc36000 ---p 00000000 00:00 0 +7f22acc36000-7f22acd32000 rw-p 00000000 00:00 0 +7f22acd32000-7f22acd33000 ---p 00000000 00:00 0 +7f22acd33000-7f22ace34000 rw-p 00000000 00:00 0 +7f22ace34000-7f22ace39000 r--p 00000000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f22ace39000-7f22ace7a000 r-xp 00005000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f22ace7a000-7f22acf03000 r--p 00046000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f22acf03000-7f22acf04000 r--p 000ce000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f22acf04000-7f22acf05000 rw-p 000cf000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f22acf05000-7f22acf06000 ---p 00000000 00:00 0 +7f22acf06000-7f22ad007000 rw-p 00000000 00:00 0 +7f22ad007000-7f22ad008000 ---p 00000000 00:00 0 +7f22ad008000-7f22ad898000 rw-p 00000000 00:00 0 +7f22ad898000-7f22adb78000 ---p 00000000 00:00 0 +7f22adb78000-7f22adb90000 rw-p 00000000 00:00 0 +7f22adb90000-7f22adbe0000 ---p 00000000 00:00 0 +7f22adbe0000-7f22adbf0000 rw-p 00000000 00:00 0 +7f22adbf0000-7f22b15f8000 ---p 00000000 00:00 0 +7f22b15f8000-7f22b1690000 rw-p 00000000 00:00 0 +7f22b1690000-7f22b1970000 ---p 00000000 00:00 0 +7f22b1970000-7f22b1988000 rw-p 00000000 00:00 0 +7f22b1988000-7f22b19d8000 ---p 00000000 00:00 0 +7f22b19d8000-7f22b19e8000 rw-p 00000000 00:00 0 +7f22b19e8000-7f22b53f0000 ---p 00000000 00:00 0 +7f22b53f0000-7f22b5400000 rw-p 00000000 00:00 0 +7f22b5400000-7f22b5840000 rwxp 00000000 00:00 0 +7f22b5840000-7f22bc937000 ---p 00000000 00:00 0 +7f22bc937000-7f22bcba7000 rwxp 00000000 00:00 0 +7f22bcba7000-7f22bcec9000 ---p 00000000 00:00 0 +7f22bcec9000-7f22bd139000 rwxp 00000000 00:00 0 +7f22bd139000-7f22c4400000 ---p 00000000 00:00 0 +7f22c4400000-7f22cbe42000 r--s 00000000 103:02 1966437 /usr/lib/jvm/java-17-openjdk-amd64/lib/modules +7f22cbe45000-7f22cbe6a000 rw-s 00000000 00:0e 48659 anon_inode:[io_uring] +7f22cbe6a000-7f22cbe6b000 r--p 00000000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f22cbe6b000-7f22cbe6c000 r-xp 00001000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f22cbe6c000-7f22cbe6d000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f22cbe6d000-7f22cbe6e000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f22cbe6e000-7f22cbe6f000 rw-p 00003000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f22cbe7e000-7f22cc3d1000 rw-p 00000000 00:00 0 +7f22cc3d1000-7f22d0000000 ---p 00000000 00:00 0 +7f22d0003000-7f22d0005000 r--p 00000000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f22d0005000-7f22d0008000 r-xp 00002000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f22d0008000-7f22d000a000 r--p 00005000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f22d000a000-7f22d000b000 r--p 00006000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f22d000b000-7f22d000c000 rw-p 00007000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f22d000c000-7f22d000e000 r--p 00000000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f22d000e000-7f22d000f000 r-xp 00002000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f22d000f000-7f22d0010000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f22d0010000-7f22d0011000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f22d0011000-7f22d0012000 rw-p 00004000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f22d0012000-7f22d00df000 rw-p 00000000 00:00 0 +7f22d00df000-7f22d00e0000 ---p 00000000 00:00 0 +7f22d00e0000-7f22d01e1000 rw-p 00000000 00:00 0 +7f22d01e1000-7f22d01e2000 ---p 00000000 00:00 0 +7f22d01e2000-7f22d02e3000 rw-p 00000000 00:00 0 +7f22d02e3000-7f22d02e4000 ---p 00000000 00:00 0 +7f22d02e4000-7f22d03f6000 rw-p 00000000 00:00 0 +7f22d03f6000-7f22d0452000 ---p 00000000 00:00 0 +7f22d0452000-7f22d0455000 rw-p 00000000 00:00 0 +7f22d0455000-7f22d045f000 ---p 00000000 00:00 0 +7f22d045f000-7f22d0461000 rw-p 00000000 00:00 0 +7f22d0461000-7f22d0ba2000 ---p 00000000 00:00 0 +7f22d0ba2000-7f22d0bb5000 rw-p 00000000 00:00 0 +7f22d0bb5000-7f22d0c11000 ---p 00000000 00:00 0 +7f22d0c11000-7f22d0c14000 rw-p 00000000 00:00 0 +7f22d0c14000-7f22d0c1e000 ---p 00000000 00:00 0 +7f22d0c1e000-7f22d0c20000 rw-p 00000000 00:00 0 +7f22d0c20000-7f22d1361000 ---p 00000000 00:00 0 +7f22d1361000-7f22d1374000 rw-p 00000000 00:00 0 +7f22d1374000-7f22d13d0000 ---p 00000000 00:00 0 +7f22d13d0000-7f22d13d3000 rw-p 00000000 00:00 0 +7f22d13d3000-7f22d13dd000 ---p 00000000 00:00 0 +7f22d13dd000-7f22d13df000 rw-p 00000000 00:00 0 +7f22d13df000-7f22d1b20000 ---p 00000000 00:00 0 +7f22d1b20000-7f22d232f000 rw-p 00000000 00:00 0 +7f22d232f000-7f22d2415000 ---p 00000000 00:00 0 +7f22d2415000-7f22d241e000 rw-p 00000000 00:00 0 +7f22d241e000-7f22d2500000 ---p 00000000 00:00 0 +7f22d2500000-7f22d2504000 ---p 00000000 00:00 0 +7f22d2504000-7f22d2600000 rw-p 00000000 00:00 0 +7f22d2600000-7f22d269c000 r--p 00000000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f22d269c000-7f22d27cb000 r-xp 0009c000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f22d27cb000-7f22d2858000 r--p 001cb000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f22d2858000-7f22d2863000 r--p 00257000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f22d2863000-7f22d2866000 rw-p 00262000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f22d2866000-7f22d286a000 rw-p 00000000 00:00 0 +7f22d286a000-7f22d286b000 r--p 00000000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f22d286b000-7f22d286c000 r-xp 00001000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f22d286c000-7f22d286d000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f22d286d000-7f22d286e000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f22d286e000-7f22d286f000 rw-p 00003000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f22d286f000-7f22d2873000 r--p 00000000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f22d2873000-7f22d287a000 r-xp 00004000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f22d287a000-7f22d287c000 r--p 0000b000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f22d287c000-7f22d287e000 r--p 0000c000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f22d287e000-7f22d287f000 rw-p 0000e000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f22d287f000-7f22d2883000 r--p 00000000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f22d2883000-7f22d2891000 r-xp 00004000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f22d2891000-7f22d2895000 r--p 00012000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f22d2895000-7f22d2896000 r--p 00015000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f22d2896000-7f22d2897000 rw-p 00016000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f22d2897000-7f22d2917000 rw-p 00000000 00:00 0 +7f22d2917000-7f22d2925000 r--p 00000000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f22d2925000-7f22d29a3000 r-xp 0000e000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f22d29a3000-7f22d29fe000 r--p 0008c000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f22d29fe000-7f22d29ff000 r--p 000e6000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f22d29ff000-7f22d2a00000 rw-p 000e7000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f22d2a00000-7f22d2c51000 r--p 00000000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f22d2c51000-7f22d39d5000 r-xp 00251000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f22d39d5000-7f22d3c56000 r--p 00fd5000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f22d3c56000-7f22d3d0e000 r--p 01255000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f22d3d0e000-7f22d3d43000 rw-p 0130d000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f22d3d43000-7f22d3d9d000 rw-p 00000000 00:00 0 +7f22d3da1000-7f22d3da2000 r--p 00000000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f22d3da2000-7f22d3da3000 r-xp 00001000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f22d3da3000-7f22d3da4000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f22d3da4000-7f22d3da5000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f22d3da5000-7f22d3da6000 rw-p 00003000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f22d3da6000-7f22d3da8000 r--p 00000000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f22d3da8000-7f22d3dad000 r-xp 00002000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f22d3dad000-7f22d3daf000 r--p 00007000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f22d3daf000-7f22d3db0000 r--p 00008000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f22d3db0000-7f22d3db1000 rw-p 00009000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f22d3db1000-7f22d3db7000 r--p 00000000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f22d3db7000-7f22d3dc0000 r-xp 00006000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f22d3dc0000-7f22d3dc4000 r--p 0000f000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f22d3dc4000-7f22d3dc5000 r--p 00013000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f22d3dc5000-7f22d3dc6000 rw-p 00014000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f22d3dc6000-7f22d3dc7000 r--p 00000000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f22d3dc7000-7f22d3dca000 r-xp 00001000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f22d3dca000-7f22d3dcb000 r--p 00004000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f22d3dcb000-7f22d3dcc000 r--p 00005000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f22d3dcc000-7f22d3dcd000 rw-p 00006000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f22d3dcd000-7f22d3dd2000 rw-p 00000000 00:00 0 +7f22d3dd2000-7f22d3dd9000 ---p 00000000 00:00 0 +7f22d3dd9000-7f22d3de5000 r--p 00000000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f22d3de5000-7f22d3df7000 r-xp 0000c000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f22d3df7000-7f22d3dfd000 r--p 0001e000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f22d3dfd000-7f22d3dfe000 r--p 00023000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f22d3dfe000-7f22d3dff000 rw-p 00024000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f22d3dff000-7f22d3e00000 rw-p 00000000 00:00 0 +7f22d3e00000-7f22d3e22000 r--p 00000000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f22d3e22000-7f22d3f9a000 r-xp 00022000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f22d3f9a000-7f22d3ff2000 r--p 0019a000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f22d3ff2000-7f22d3ff6000 r--p 001f1000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f22d3ff6000-7f22d3ff8000 rw-p 001f5000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f22d3ff8000-7f22d4005000 rw-p 00000000 00:00 0 +7f22d4008000-7f22d400f000 r--s 00000000 103:02 1074285 /usr/lib/x86_64-linux-gnu/gconv/gconv-modules.cache +7f22d400f000-7f22d4017000 rw-s 00000000 103:02 3670137 /tmp/hsperfdata_antonio/4794 +7f22d4017000-7f22d401a000 r--p 00000000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f22d401a000-7f22d4048000 r-xp 00003000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f22d4048000-7f22d4056000 r--p 00031000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f22d4056000-7f22d4057000 r--p 0003e000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f22d4057000-7f22d4058000 rw-p 0003f000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f22d4058000-7f22d4059000 rw-p 00000000 00:00 0 +7f22d4059000-7f22d405c000 r--p 00000000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f22d405c000-7f22d4077000 r-xp 00003000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f22d4077000-7f22d407b000 r--p 0001e000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f22d407b000-7f22d407c000 r--p 00021000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f22d407c000-7f22d407d000 rw-p 00022000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f22d407d000-7f22d4080000 rw-p 00000000 00:00 0 +7f22d4080000-7f22d4083000 r--p 00000000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f22d4083000-7f22d4095000 r-xp 00003000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f22d4095000-7f22d409c000 r--p 00015000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f22d409c000-7f22d409d000 r--p 0001b000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f22d409d000-7f22d409e000 rw-p 0001c000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f22d40a2000-7f22d40a3000 ---p 00000000 00:00 0 +7f22d40a3000-7f22d40a4000 r--p 00000000 00:00 0 +7f22d40a4000-7f22d40a5000 ---p 00000000 00:00 0 +7f22d40a5000-7f22d40a7000 r--p 00000000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f22d40a7000-7f22d40aa000 r-xp 00002000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f22d40aa000-7f22d40ab000 r--p 00005000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f22d40ab000-7f22d40ac000 r--p 00006000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f22d40ac000-7f22d40ad000 rw-p 00007000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f22d40ad000-7f22d40af000 r--p 00000000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f22d40af000-7f22d40b9000 r-xp 00002000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f22d40b9000-7f22d40bc000 r--p 0000c000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f22d40bc000-7f22d40bd000 r--p 0000e000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f22d40bd000-7f22d40be000 rw-p 0000f000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f22d40be000-7f22d40c0000 rw-p 00000000 00:00 0 +7f22d40c0000-7f22d40c1000 r--p 00000000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f22d40c1000-7f22d40e9000 r-xp 00001000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f22d40e9000-7f22d40f3000 r--p 00029000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f22d40f3000-7f22d40f5000 r--p 00033000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f22d40f5000-7f22d40f7000 rw-p 00035000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7fff14656000-7fff14678000 rw-p 00000000 00:00 0 [stack] +7fff147e5000-7fff147e9000 r--p 00000000 00:00 0 [vvar] +7fff147e9000-7fff147eb000 r-xp 00000000 00:00 0 [vdso] +ffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall] + + +VM Arguments: +jvm_args: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring +java_command: sbt.ForkMain 44215 +java_class_path (initial): /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/test-classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scala-lang/scala-library/2.13.10/scala-library-2.13.10.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect_2.13/3.6-0142603/cats-effect_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-io_2.13/3.7.0/fs2-io_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/munit-cats-effect_2.13/2.0.0-M3/munit-cats-effect_2.13-2.0.0-M3.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-native-io_uring/0.0.21.Final/netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-kernel_2.13/3.6-0142603/cats-effect-kernel_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-std_2.13/3.6-0142603/cats-effect-std_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-core_2.13/3.7.0/fs2-core_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/com/comcast/ip4s-core_2.13/3.3.0/ip4s-core_2.13-3.3.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scalameta/munit_2.13/1.0.0-M6/munit_2.13-1.0.0-M6.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-classes-io_uring/0.0.21.Final/netty-incubator-transport-classes-io_uring-0.0.21.Final.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-core_2.13/2.9.0/cats-core_2.13-2.9.0.ja +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 4 {product} {ergonomic} + uint ConcGCThreads = 2 {product} {ergonomic} + uint G1ConcRefinementThreads = 8 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + size_t InitialHeapSize = 260046848 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4158652416 {product} {ergonomic} + size_t MaxNewSize = 2493513728 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 5839372 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4158652416 {manageable} {ergonomic} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64/ +PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/games:/usr/local/games:/snap/bin:/snap/bin:/home/antonio/.local/share/coursier/bin:/home/antonio/.local/share/coursier/bin +USERNAME=antonio +SHELL=/bin/bash +DISPLAY=:0 +LANG=en_US.UTF-8 + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.so, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + + +--------------- S Y S T E M --------------- + +OS: +DISTRIB_ID=Ubuntu +DISTRIB_RELEASE=23.04 +DISTRIB_CODENAME=lunar +DISTRIB_DESCRIPTION="Ubuntu 23.04" +uname: Linux 6.2.0-26-generic #26-Ubuntu SMP PREEMPT_DYNAMIC Mon Jul 10 23:39:54 UTC 2023 x86_64 +OS uptime: 0 days 0:40 hours +libc: glibc 2.37 NPTL 2.37 +rlimit (soft/hard): STACK 8192k/infinity , CORE 0k/infinity , NPROC 63121/63121 , NOFILE 1048576/1048576 , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK 2030188k/2030188k +load average: 4.99 2.00 1.29 + +/proc/meminfo: +MemTotal: 16241528 kB +MemFree: 10347640 kB +MemAvailable: 12248880 kB +Buffers: 82448 kB +Cached: 2370536 kB +SwapCached: 0 kB +Active: 668164 kB +Inactive: 4415084 kB +Active(anon): 2104 kB +Inactive(anon): 2969548 kB +Active(file): 666060 kB +Inactive(file): 1445536 kB +Unevictable: 342388 kB +Mlocked: 48 kB +SwapTotal: 4194300 kB +SwapFree: 4194300 kB +Zswap: 0 kB +Zswapped: 0 kB +Dirty: 48 kB +Writeback: 0 kB +AnonPages: 2973056 kB +Mapped: 655108 kB +Shmem: 358916 kB +KReclaimable: 91204 kB +Slab: 234964 kB +SReclaimable: 91204 kB +SUnreclaim: 143760 kB +KernelStack: 17904 kB +PageTables: 33876 kB +SecPageTables: 0 kB +NFS_Unstable: 0 kB +Bounce: 0 kB +WritebackTmp: 0 kB +CommitLimit: 12315064 kB +Committed_AS: 11130900 kB +VmallocTotal: 34359738367 kB +VmallocUsed: 114780 kB +VmallocChunk: 0 kB +Percpu: 5952 kB +HardwareCorrupted: 0 kB +AnonHugePages: 2048 kB +ShmemHugePages: 0 kB +ShmemPmdMapped: 0 kB +FileHugePages: 0 kB +FilePmdMapped: 0 kB +HugePages_Total: 0 +HugePages_Free: 0 +HugePages_Rsvd: 0 +HugePages_Surp: 0 +Hugepagesize: 2048 kB +Hugetlb: 0 kB +DirectMap4k: 377332 kB +DirectMap2M: 6844416 kB +DirectMap1G: 9437184 kB + +/sys/kernel/mm/transparent_hugepage/enabled: always [madvise] never +/sys/kernel/mm/transparent_hugepage/defrag (defrag/compaction efforts parameter): always defer defer+madvise [madvise] never + +Process Memory: +Virtual Size: 8613960K (peak: 8675504K) +Resident Set Size: 171000K (peak: 171000K) (anon: 141304K, file: 29696K, shmem: 0K) +Swapped out: 0K +C-Heap outstanding allocations: 60271K, retained: 13540K +glibc malloc tunables: (default) + +/proc/sys/kernel/threads-max (system-wide limit on the number of threads): 126243 +/proc/sys/vm/max_map_count (maximum number of memory map areas a process may have): 65530 +/proc/sys/kernel/pid_max (system-wide limit on number of process identifiers): 4194304 + +container (cgroup) information: +container_type: cgroupv2 +cpu_cpuset_cpus: not supported +cpu_memory_nodes: not supported +active_processor_count: 8 +cpu_quota: not supported +cpu_period: not supported +cpu_shares: not supported +memory_limit_in_bytes: unlimited +memory_and_swap_limit_in_bytes: unlimited +memory_soft_limit_in_bytes: unlimited +memory_usage_in_bytes: 2812744 k +memory_max_usage_in_bytes: not supported +memory_swap_current_in_bytes: unlimited +memory_swap_max_limit_in_bytes: unlimited +maximum number of tasks: 18936 +current number of tasks: 364 + +Steal ticks since vm start: 0 +Steal ticks percentage since vm start: 0.000 + +CPU: total 8 (initial active 8) (4 cores per cpu, 2 threads per core) family 6 model 142 stepping 10 microcode 0xf0, cx8, cmov, fxsr, ht, mmx, 3dnowpref, sse, sse2, sse3, ssse3, sse4.1, sse4.2, popcnt, lzcnt, tsc, tscinvbit, avx, avx2, aes, erms, clmul, bmi1, bmi2, adx, fma, vzeroupper, clflush, clflushopt +CPU Model and flags from /proc/cpuinfo: +model name : Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc art arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc cpuid aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch cpuid_fault epb invpcid_single pti ssbd ibrs ibpb stibp tpr_shadow vnmi flexpriority ept vpid ept_ad fsgsbase tsc_adjust bmi1 avx2 smep bmi2 erms invpcid mpx rdseed adx smap clflushopt intel_pt xsaveopt xsavec xgetbv1 xsaves dtherm ida arat pln pts hwp hwp_notify hwp_act_window hwp_epp md_clear flush_l1d arch_capabilities + +Online cpus: 0-7 +Offline cpus: +BIOS frequency limitation: +Frequency switch latency (ns): 0 +Available cpu frequencies: +Current governor: powersave +Core performance/turbo boost: + +Memory: 4k page, physical 16241528k(10347640k free), swap 4194300k(4194300k free) +Page Sizes: 4k + +vm_info: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04) for linux-amd64 JRE (17.0.7+7-Ubuntu-0ubuntu123.04), built on May 6 2023 14:43:58 by "buildd" with gcc 12.2.0 + +END. diff --git a/hs_err_pid8969.log b/hs_err_pid8969.log new file mode 100644 index 00000000..12fb274c --- /dev/null +++ b/hs_err_pid8969.log @@ -0,0 +1,1149 @@ +# +# A fatal error has been detected by the Java Runtime Environment: +# +# SIGSEGV (0xb) at pc=0x00007f5294b923e4, pid=8969, tid=9009 +# +# JRE version: OpenJDK Runtime Environment (17.0.7+7) (build 17.0.7+7-Ubuntu-0ubuntu123.04) +# Java VM: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, linux-amd64) +# Problematic frame: +# V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 +# +# Core dump will be written. Default location: Core dumps may be processed with "/usr/share/apport/apport -p%p -s%s -c%c -d%d -P%P -u%u -g%g -- %E" (or dumping to /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/core.8969) +# +# If you would like to submit a bug report, please visit: +# Unknown +# + +--------------- S U M M A R Y ------------ + +Command Line: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring sbt.ForkMain 33543 + +Host: Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz, 8 cores, 15G, Ubuntu 23.04 +Time: Tue Aug 8 13:41:27 2023 CEST elapsed time: 11.994800 seconds (0d 0h 0m 11s) + +--------------- T H R E A D --------------- + +Current thread (0x00007f521453bfc0): JavaThread "io-compute-1" daemon [_thread_in_vm, id=9009, stack(0x00007f5249f1b000,0x00007f524a01b000)] + +Stack: [0x00007f5249f1b000,0x00007f524a01b000], sp=0x00007f524a018828, free space=1014k +Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) +V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 +V [libjvm.so+0xe881a4] SystemDictionary::resolve_instance_class_or_null(Symbol*, Handle, Handle, JavaThread*)+0x134 +V [libjvm.so+0xe89f46] SystemDictionary::resolve_or_fail(Symbol*, Handle, Handle, bool, JavaThread*)+0x66 +V [libjvm.so+0x602cc9] ConstantPool::klass_at_impl(constantPoolHandle const&, int, JavaThread*)+0x139 +V [libjvm.so+0x603a0a] ConstantPool::klass_ref_at(int, JavaThread*)+0x7a +V [libjvm.so+0xa948f9] LinkInfo::LinkInfo(constantPoolHandle const&, int, JavaThread*)+0x39 +V [libjvm.so+0xa9b19f] LinkResolver::resolve_invoke(CallInfo&, Handle, constantPoolHandle const&, int, Bytecodes::Code, JavaThread*)+0x1df +V [libjvm.so+0x82c947] InterpreterRuntime::resolve_invoke(JavaThread*, Bytecodes::Code)+0x177 +V [libjvm.so+0x82ce67] InterpreterRuntime::resolve_from_cache(JavaThread*, Bytecodes::Code)+0x37 +j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+85 +j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 +j fs2.Stream$NestedStreamOps$$$Lambda$682+0x000000080136f5d8.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 +j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 +j cats.effect.IO$$Lambda$302+0x000000080125ee68.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 +j cats.effect.IO$$Lambda$303+0x000000080125f238.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +J 1746 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f527975a764 [0x00007f52797593a0+0x00000000000013c4] +j cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V+262 +j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 +j cats.effect.IOFiber.run()V+66 +J 1768% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f5279764f74 [0x00007f5279762220+0x0000000000002d54] +v ~StubRoutines::call_stub +V [libjvm.so+0x831c02] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x302 +V [libjvm.so+0x8332d2] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x1a2 +V [libjvm.so+0x8fb417] thread_entry(JavaThread*, JavaThread*)+0xa7 +V [libjvm.so+0xed302e] JavaThread::thread_main_inner()+0xce +V [libjvm.so+0xed675f] Thread::call_run()+0xbf +V [libjvm.so+0xc2d5c9] thread_native_entry(Thread*)+0xe9 + +Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) +j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+85 +j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 +j fs2.Stream$NestedStreamOps$$$Lambda$682+0x000000080136f5d8.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 +j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 +j cats.effect.IO$$Lambda$302+0x000000080125ee68.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 +j cats.effect.IO$$Lambda$303+0x000000080125f238.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 +J 1746 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f527975a764 [0x00007f52797593a0+0x00000000000013c4] +j cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V+262 +j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 +j cats.effect.IOFiber.run()V+66 +J 1768% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f5279764f74 [0x00007f5279762220+0x0000000000002d54] +v ~StubRoutines::call_stub + +siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr: 0x0000000000001000 + +Register to memory mapping: + +RAX=0x00007f5200000010 points into unknown readable memory: 0x0000000000021000 | 00 10 02 00 00 00 00 00 +RBX=0x00007f52140f3900 points into unknown readable memory: 0x00007f526e73ea9e | 9e ea 73 6e 52 7f 00 00 +RCX=0x0000000000000003 is an unknown value +RDX=0x00007f5295912330: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 +RSP=0x00007f524a018828 is pointing into the stack for thread: 0x00007f521453bfc0 +RBP=0x00007f524a018850 is pointing into the stack for thread: 0x00007f521453bfc0 +RSI=0x0000000000000060 is an unknown value +RDI=0x0000000000001000 is an unknown value +R8 =0x00007f5214034430 points into unknown readable memory: 0x63730010ea9e003b | 3b 00 9e ea 10 00 73 63 +R9 =0x00007f521453bfc0 is a thread +R10=0x00007f5295c10000 points into unknown readable memory: 0x0100050403020100 | 00 01 02 03 04 05 00 01 +R11=0x0000000000000001 is an unknown value +R12=0x00007f524c00c900 points into unknown readable memory: 0x00007f5200000010 | 10 00 00 00 52 7f 00 00 +R13=0x000000070860fac8 is an oop: java.security.ProtectionDomain +{0x000000070860fac8} - klass: 'java/security/ProtectionDomain' + - ---- fields (total size 5 words): + - private 'hasAllPerm' 'Z' @12 false + - private final 'staticPermissions' 'Z' @13 false + - private 'codesource' 'Ljava/security/CodeSource;' @16 a 'java/security/CodeSource'{0x000000070860faf0} (e10c1f5e) + - private 'classloader' 'Ljava/lang/ClassLoader;' @20 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000007ffd59e70} (fffab3ce) + - private 'principals' '[Ljava/security/Principal;' @24 a 'java/security/Principal'[0] {0x000000070860fc90} (e10c1f92) + - private 'permissions' 'Ljava/security/PermissionCollection;' @28 a 'sun/security/util/LazyCodeSourcePermissionCollection'{0x000000070860fca0} (e10c1f94) + - final 'key' 'Ljava/security/ProtectionDomain$Key;' @32 a 'java/security/ProtectionDomain$Key'{0x000000070860fe60} (e10c1fcc) +R14=0x00007f5290193210 points into unknown readable memory: 0x00007f5290165008 | 08 50 16 90 52 7f 00 00 +R15=0x00007f521453c5d0 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 + + +Registers: +RAX=0x00007f5200000010, RBX=0x00007f52140f3900, RCX=0x0000000000000003, RDX=0x00007f5295912330 +RSP=0x00007f524a018828, RBP=0x00007f524a018850, RSI=0x0000000000000060, RDI=0x0000000000001000 +R8 =0x00007f5214034430, R9 =0x00007f521453bfc0, R10=0x00007f5295c10000, R11=0x0000000000000001 +R12=0x00007f524c00c900, R13=0x000000070860fac8, R14=0x00007f5290193210, R15=0x00007f521453c5d0 +RIP=0x00007f5294b923e4, EFLAGS=0x0000000000010206, CSGSFS=0x002b000000000033, ERR=0x0000000000000004 + TRAPNO=0x000000000000000e + +Top of Stack: (sp=0x00007f524a018828) +0x00007f524a018828: 00007f5294c8ffb8 00007f524a018850 +0x00007f524a018838: 00007f5214034430 00007f521453bfc0 +0x00007f524a018848: 00007f521453c5e0 00007f524a018980 +0x00007f524a018858: 00007f52954881a4 00007f524a018b10 + +Instructions: (pc=0x00007f5294b923e4) +0x00007f5294b922e4: 01 eb df 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b922f4: 31 c0 c3 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92304: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92314: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92324: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92334: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92344: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92354: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92364: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92374: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92384: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92394: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923a4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923b4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923c4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923d4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923e4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b923f4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92404: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92414: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92424: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92434: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92444: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92454: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92464: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92474: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92484: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b92494: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b924a4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f +0x00007f5294b924b4: 48 85 ff 74 05 e8 e2 45 14 00 4c 89 23 5b 41 5c +0x00007f5294b924c4: 5d c3 66 2e 0f 1f 84 00 00 00 00 00 f3 0f 1e fa +0x00007f5294b924d4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f + + +Stack slot to memory mapping: +stack at sp + 0 slots: 0x00007f5294c8ffb8: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 +stack at sp + 1 slots: 0x00007f524a018850 is pointing into the stack for thread: 0x00007f521453bfc0 +stack at sp + 2 slots: 0x00007f5214034430 points into unknown readable memory: 0x63730010ea9e003b | 3b 00 9e ea 10 00 73 63 +stack at sp + 3 slots: 0x00007f521453bfc0 is a thread +stack at sp + 4 slots: 0x00007f521453c5e0 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 +stack at sp + 5 slots: 0x00007f524a018980 is pointing into the stack for thread: 0x00007f521453bfc0 +stack at sp + 6 slots: 0x00007f52954881a4: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 +stack at sp + 7 slots: 0x00007f524a018b10 is pointing into the stack for thread: 0x00007f521453bfc0 + + +--------------- P R O C E S S --------------- + +Threads class SMR info: +_java_thread_list=0x00007f5234390650, length=27, elements={ +0x00007f5290015d20, 0x00007f5290179870, 0x00007f529017ac50, 0x00007f52901801d0, +0x00007f5290181580, 0x00007f5290182990, 0x00007f5290184340, 0x00007f5290185870, +0x00007f529018ecd0, 0x00007f52901a76b0, 0x00007f52901ab290, 0x00007f522c001140, +0x00007f52901ac250, 0x00007f52901b10a0, 0x00007f529037c7f0, 0x00007f521453b510, +0x00007f521453bfc0, 0x00007f521453cf70, 0x00007f521453de80, 0x00007f521453f230, +0x00007f5214540530, 0x00007f52145418f0, 0x00007f5214542800, 0x00007f51f0006af0, +0x00007f523020de90, 0x00007f51ec003790, 0x00007f523438f6f0 +} + +Java Threads: ( => current thread ) + 0x00007f5290015d20 JavaThread "main" [_thread_blocked, id=8972, stack(0x00007f5294100000,0x00007f5294200000)] + 0x00007f5290179870 JavaThread "Reference Handler" daemon [_thread_blocked, id=8979, stack(0x00007f526c865000,0x00007f526c965000)] + 0x00007f529017ac50 JavaThread "Finalizer" daemon [_thread_blocked, id=8980, stack(0x00007f526c765000,0x00007f526c865000)] + 0x00007f52901801d0 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=8981, stack(0x00007f526c665000,0x00007f526c765000)] + 0x00007f5290181580 JavaThread "Service Thread" daemon [_thread_blocked, id=8982, stack(0x00007f526c565000,0x00007f526c665000)] + 0x00007f5290182990 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=8983, stack(0x00007f526c465000,0x00007f526c565000)] + 0x00007f5290184340 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=8984, stack(0x00007f526c365000,0x00007f526c465000)] + 0x00007f5290185870 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=8985, stack(0x00007f526c265000,0x00007f526c365000)] + 0x00007f529018ecd0 JavaThread "Sweeper thread" daemon [_thread_blocked, id=8986, stack(0x00007f526c165000,0x00007f526c265000)] + 0x00007f52901a76b0 JavaThread "JDWP Transport Listener: dt_socket" daemon [_thread_blocked, id=8987, stack(0x00007f526c065000,0x00007f526c165000)] + 0x00007f52901ab290 JavaThread "JDWP Event Helper Thread" daemon [_thread_blocked, id=8988, stack(0x00007f524b100000,0x00007f524b200000)] + 0x00007f522c001140 JavaThread "JDWP Command Reader" daemon [_thread_in_native, id=8993, stack(0x00007f524b000000,0x00007f524b100000)] + 0x00007f52901ac250 JavaThread "Notification Thread" daemon [_thread_blocked, id=8996, stack(0x00007f524af00000,0x00007f524b000000)] + 0x00007f52901b10a0 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=8998, stack(0x00007f524acfe000,0x00007f524adfe000)] + 0x00007f529037c7f0 JavaThread "pool-1-thread-1" [_thread_blocked, id=9000, stack(0x00007f524abfe000,0x00007f524acfe000)] + 0x00007f521453b510 JavaThread "io-compute-blocker-0" daemon [_thread_blocked, id=9008, stack(0x00007f524a01b000,0x00007f524a11b000)] +=>0x00007f521453bfc0 JavaThread "io-compute-1" daemon [_thread_in_vm, id=9009, stack(0x00007f5249f1b000,0x00007f524a01b000)] + 0x00007f521453cf70 JavaThread "io-compute-6" daemon [_thread_in_native, id=9010, stack(0x00007f5249e1b000,0x00007f5249f1b000)] + 0x00007f521453de80 JavaThread "io-compute-2" daemon [_thread_blocked, id=9011, stack(0x00007f5249d1b000,0x00007f5249e1b000)] + 0x00007f521453f230 JavaThread "io-compute-4" daemon [_thread_in_native, id=9012, stack(0x00007f5249c1b000,0x00007f5249d1b000)] + 0x00007f5214540530 JavaThread "io-compute-5" daemon [_thread_blocked, id=9013, stack(0x00007f5249b1b000,0x00007f5249c1b000)] + 0x00007f52145418f0 JavaThread "io-compute-0" daemon [_thread_blocked, id=9014, stack(0x00007f5249a1b000,0x00007f5249b1b000)] + 0x00007f5214542800 JavaThread "io-compute-7" daemon [_thread_blocked, id=9015, stack(0x00007f524991b000,0x00007f5249a1b000)] + 0x00007f51f0006af0 JavaThread "pool-2-thread-1" [_thread_blocked, id=9019, stack(0x00007f524a5f4000,0x00007f524a6f4000)] + 0x00007f523020de90 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=9023, stack(0x00007f524981b000,0x00007f524991b000)] + 0x00007f51ec003790 JavaThread "io-compute-3" daemon [_thread_in_native, id=9024, stack(0x00007f5249415000,0x00007f5249515000)] + 0x00007f523438f6f0 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=9025, stack(0x00007f5249315000,0x00007f5249415000)] + +Other Threads: + 0x00007f52901758f0 VMThread "VM Thread" [stack: 0x00007f526c967000,0x00007f526ca67000] [id=8978] _threads_hazard_ptr=0x00007f5234390650 + 0x00007f52901adaa0 WatcherThread [stack: 0x00007f524ae00000,0x00007f524af00000] [id=8997] + 0x00007f5290080fb0 GCTaskThread "GC Thread#0" [stack: 0x00007f526f7cb000,0x00007f526f8cb000] [id=8973] + 0x00007f524c005f90 GCTaskThread "GC Thread#1" [stack: 0x00007f524aafe000,0x00007f524abfe000] [id=9001] + 0x00007f524c0069c0 GCTaskThread "GC Thread#2" [stack: 0x00007f524a9fc000,0x00007f524aafc000] [id=9002] + 0x00007f524c0073f0 GCTaskThread "GC Thread#3" [stack: 0x00007f524a8fa000,0x00007f524a9fa000] [id=9003] + 0x00007f524c007e20 GCTaskThread "GC Thread#4" [stack: 0x00007f524a7f8000,0x00007f524a8f8000] [id=9004] + 0x00007f524c008850 GCTaskThread "GC Thread#5" [stack: 0x00007f524a6f6000,0x00007f524a7f6000] [id=9005] + 0x00007f524c00c9d0 GCTaskThread "GC Thread#6" [stack: 0x00007f524971b000,0x00007f524981b000] [id=9017] + 0x00007f524c00cfa0 GCTaskThread "GC Thread#7" [stack: 0x00007f5249619000,0x00007f5249719000] [id=9018] + 0x00007f5290091d10 ConcurrentGCThread "G1 Main Marker" [stack: 0x00007f526f6c9000,0x00007f526f7c9000] [id=8974] + 0x00007f5290092c00 ConcurrentGCThread "G1 Conc#0" [stack: 0x00007f526f5c7000,0x00007f526f6c7000] [id=8975] + 0x00007f5260000ea0 ConcurrentGCThread "G1 Conc#1" [stack: 0x00007f5249517000,0x00007f5249617000] [id=9021] + 0x00007f5290146590 ConcurrentGCThread "G1 Refine#0" [stack: 0x00007f526ccbd000,0x00007f526cdbd000] [id=8976] + 0x00007f5290147480 ConcurrentGCThread "G1 Service" [stack: 0x00007f526cbbb000,0x00007f526ccbb000] [id=8977] + +Threads with active compile tasks: +C2 CompilerThread0 12047 1809 4 java.lang.invoke.MethodType::makeImpl (109 bytes) +C1 CompilerThread0 12048 1940 ! 3 cats.effect.IOFiber::runLoop (3533 bytes) +C2 CompilerThread1 12048 1783 % ! 4 cats.effect.unsafe.WorkerThread::run @ 44 (1765 bytes) +C2 CompilerThread2 12048 2018 ! 4 java.lang.ref.ReferenceQueue::poll (28 bytes) + +VM state: synchronizing (normal execution) + +VM Mutex/Monitor currently owned by a thread: ([mutex/lock_event]) +[0x00007f5290012a90] Threads_lock - owner thread: 0x00007f52901758f0 + +Heap address: 0x0000000708200000, size: 3966 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 + +CDS archive(s) mapped at: [0x0000000800000000-0x0000000800be2000-0x0000000800be2000), size 12460032, SharedBaseAddress: 0x0000000800000000, ArchiveRelocationMode: 0. +Compressed class space mapped at: 0x0000000801000000-0x0000000841000000, reserved size: 1073741824 +Narrow klass base: 0x0000000800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 + +GC Precious Log: + CPUs: 8 total, 8 available + Memory: 15860M + Large Page Support: Disabled + NUMA Support: Disabled + Compressed Oops: Enabled (Zero based) + Heap Region Size: 2M + Heap Min Capacity: 8M + Heap Initial Capacity: 248M + Heap Max Capacity: 3966M + Pre-touch: Disabled + Parallel Workers: 8 + Concurrent Workers: 2 + Concurrent Refinement Workers: 8 + Periodic GC: Disabled + +Heap: + garbage-first heap total 49152K, used 16110K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 9 young (18432K), 3 survivors (6144K) + Metaspace used 29626K, committed 30016K, reserved 1114112K + class space used 3307K, committed 3456K, reserved 1048576K + +Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) +| 0|0x0000000708200000, 0x00000007083c3838, 0x0000000708400000| 88%| S|CS|TAMS 0x0000000708200000, 0x0000000708200000| Complete +| 1|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| S|CS|TAMS 0x0000000708400000, 0x0000000708400000| Complete +| 2|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| S|CS|TAMS 0x0000000708600000, 0x0000000708600000| Complete +| 3|0x0000000708800000, 0x0000000708800000, 0x0000000708a00000| 0%| F| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked +| 4|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked +| 5|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked +| 6|0x0000000708e00000, 0x0000000708e00000, 0x0000000709000000| 0%| F| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked +| 7|0x0000000709000000, 0x0000000709000000, 0x0000000709200000| 0%| F| |TAMS 0x0000000709000000, 0x0000000709000000| Untracked +| 8|0x0000000709200000, 0x0000000709200000, 0x0000000709400000| 0%| F| |TAMS 0x0000000709200000, 0x0000000709200000| Untracked +| 9|0x0000000709400000, 0x0000000709400000, 0x0000000709600000| 0%| F| |TAMS 0x0000000709400000, 0x0000000709400000| Untracked +| 10|0x0000000709600000, 0x0000000709600000, 0x0000000709800000| 0%| F| |TAMS 0x0000000709600000, 0x0000000709600000| Untracked +| 11|0x0000000709800000, 0x0000000709800000, 0x0000000709a00000| 0%| F| |TAMS 0x0000000709800000, 0x0000000709800000| Untracked +| 12|0x0000000709a00000, 0x0000000709a00000, 0x0000000709c00000| 0%| F| |TAMS 0x0000000709a00000, 0x0000000709a00000| Untracked +| 13|0x0000000709c00000, 0x0000000709c00000, 0x0000000709e00000| 0%| F| |TAMS 0x0000000709c00000, 0x0000000709c00000| Untracked +| 14|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked +| 15|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked +| 16|0x000000070a200000, 0x000000070a35a460, 0x000000070a400000| 67%| E| |TAMS 0x000000070a200000, 0x000000070a200000| Complete +| 110|0x0000000715e00000, 0x0000000716000000, 0x0000000716000000|100%| E|CS|TAMS 0x0000000715e00000, 0x0000000715e00000| Complete +| 111|0x0000000716000000, 0x0000000716200000, 0x0000000716200000|100%| E|CS|TAMS 0x0000000716000000, 0x0000000716000000| Complete +| 112|0x0000000716200000, 0x0000000716400000, 0x0000000716400000|100%| E| |TAMS 0x0000000716200000, 0x0000000716200000| Complete +| 122|0x0000000717600000, 0x0000000717800000, 0x0000000717800000|100%| E|CS|TAMS 0x0000000717600000, 0x0000000717600000| Complete +| 123|0x0000000717800000, 0x0000000717a00000, 0x0000000717a00000|100%| E|CS|TAMS 0x0000000717800000, 0x0000000717800000| Complete +|1981|0x00000007ffc00000, 0x00000007ffd76000, 0x00000007ffe00000| 73%|OA| |TAMS 0x00000007ffd76000, 0x00000007ffc00000| Untracked +|1982|0x00000007ffe00000, 0x00000007ffe82000, 0x0000000800000000| 25%|CA| |TAMS 0x00000007ffe82000, 0x00000007ffe00000| Untracked + +Card table byte_map: [0x00007f5277c7a000,0x00007f5278439000] _byte_map_base: 0x00007f5274439000 + +Marking Bits (Prev, Next): (CMBitMap*) 0x00007f5290081a00, (CMBitMap*) 0x00007f52900819c0 + Prev Bits: [0x00007f526f8cb000, 0x00007f52736c3000) + Next Bits: [0x00007f52736c3000, 0x00007f52774bb000) + +Polling page: 0x00007f5295caf000 + +Metaspace: + +Usage: + Non-class: 25.70 MB used. + Class: 3.23 MB used. + Both: 28.93 MB used. + +Virtual space: + Non-class space: 64.00 MB reserved, 25.94 MB ( 41%) committed, 1 nodes. + Class space: 1.00 GB reserved, 3.38 MB ( <1%) committed, 1 nodes. + Both: 1.06 GB reserved, 29.31 MB ( 3%) committed. + +Chunk freelists: + Non-Class: 5.67 MB + Class: 12.45 MB + Both: 18.12 MB + +MaxMetaspaceSize: unlimited +CompressedClassSpaceSize: 1.00 GB +Initial GC threshold: 21.00 MB +Current GC threshold: 35.12 MB +CDS: on +MetaspaceReclaimPolicy: balanced + - commit_granule_bytes: 65536. + - commit_granule_words: 8192. + - virtual_space_node_default_size: 8388608. + - enlarge_chunks_in_place: 1. + - new_chunks_are_fully_committed: 0. + - uncommit_free_chunks: 1. + - use_allocation_guard: 0. + - handle_deallocations: 1. + + +Internal statistics: + +num_allocs_failed_limit: 3. +num_arena_births: 432. +num_arena_deaths: 0. +num_vsnodes_births: 2. +num_vsnodes_deaths: 0. +num_space_committed: 469. +num_space_uncommitted: 0. +num_chunks_returned_to_freelist: 3. +num_chunks_taken_from_freelist: 1438. +num_chunk_merges: 3. +num_chunk_splits: 1034. +num_chunks_enlarged: 816. +num_inconsistent_stats: 0. + +CodeHeap 'non-profiled nmethods': size=120028Kb used=1078Kb max_used=1078Kb free=118949Kb + bounds [0x00007f5280ec9000, 0x00007f5281139000, 0x00007f5288400000] +CodeHeap 'profiled nmethods': size=120028Kb used=3792Kb max_used=3792Kb free=116235Kb + bounds [0x00007f5279400000, 0x00007f52797c0000, 0x00007f5280937000] +CodeHeap 'non-nmethods': size=5704Kb used=1349Kb max_used=1375Kb free=4354Kb + bounds [0x00007f5280937000, 0x00007f5280ba7000, 0x00007f5280ec9000] + total_blobs=2552 nmethods=1927 adapters=539 + compilation: enabled + stopped_count=0, restarted_count=0 + full_count=0 + +Compilation events (20 events): +Event: 11.759 Thread 0x00007f5290184340 nmethod 1888 0x00007f5280fd1810 code [0x00007f5280fd1a00, 0x00007f5280fd2478] +Event: 11.759 Thread 0x00007f5290184340 1968 4 cats.effect.IOFiber::shouldFinalize (20 bytes) +Event: 11.761 Thread 0x00007f5290184340 nmethod 1968 0x00007f5280fd3010 code [0x00007f5280fd31a0, 0x00007f5280fd3268] +Event: 11.761 Thread 0x00007f5290184340 1809 4 java.lang.invoke.MethodType::makeImpl (109 bytes) +Event: 11.790 Thread 0x00007f523438f6f0 nmethod 1839 0x00007f5280fd3310 code [0x00007f5280fd34e0, 0x00007f5280fd4138] +Event: 11.790 Thread 0x00007f523438f6f0 1981 4 java.lang.Object::equals (11 bytes) +Event: 11.791 Thread 0x00007f523438f6f0 nmethod 1981 0x00007f5280fd4790 code [0x00007f5280fd4900, 0x00007f5280fd4998] +Event: 11.791 Thread 0x00007f523438f6f0 1977 4 scala.Option::foreach (20 bytes) +Event: 11.812 Thread 0x00007f523438f6f0 nmethod 1977 0x00007f5280fd4a90 code [0x00007f5280fd4c20, 0x00007f5280fd4d68] +Event: 11.812 Thread 0x00007f523438f6f0 1988 4 cats.effect.IO:: (5 bytes) +Event: 11.813 Thread 0x00007f523438f6f0 nmethod 1988 0x00007f5280fd4e90 code [0x00007f5280fd5000, 0x00007f5280fd5078] +Event: 11.813 Thread 0x00007f523438f6f0 1854 4 io.netty.incubator.channel.uring.UringCompletionQueue::ioUringWaitCqe (8 bytes) +Event: 11.816 Thread 0x00007f523438f6f0 nmethod 1854 0x00007f5280fd5190 code [0x00007f5280fd5320, 0x00007f5280fd5428] +Event: 11.817 Thread 0x00007f523438f6f0 1855 4 io.netty.incubator.channel.uring.UringSubmissionQueue::addTimeout (10 bytes) +Event: 11.879 Thread 0x00007f523438f6f0 nmethod 1855 0x00007f5280fd5510 code [0x00007f5280fd56a0, 0x00007f5280fd59d8] +Event: 11.879 Thread 0x00007f523438f6f0 2000 4 java.lang.ClassValue::get (31 bytes) +Event: 11.895 Thread 0x00007f523438f6f0 nmethod 2000 0x00007f5280fd5e10 code [0x00007f5280fd5fa0, 0x00007f5280fd61c8] +Event: 11.895 Thread 0x00007f523438f6f0 2010 4 cats.effect.tracing.TracingPlatform::calculateTracingEvent (36 bytes) +Event: 11.925 Thread 0x00007f523438f6f0 nmethod 2010 0x00007f5280fd6410 code [0x00007f5280fd65c0, 0x00007f5280fd6868] +Event: 11.925 Thread 0x00007f523438f6f0 2018 ! 4 java.lang.ref.ReferenceQueue::poll (28 bytes) + +GC Heap History (8 events): +Event: 2.547 GC heap before +{Heap before GC invocations=0 (full 0): + garbage-first heap total 258048K, used 22496K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 11 young (22528K), 0 survivors (0K) + Metaspace used 7389K, committed 7488K, reserved 1114112K + class space used 779K, committed 832K, reserved 1048576K +} +Event: 2.558 GC heap after +{Heap after GC invocations=1 (full 0): + garbage-first heap total 258048K, used 4874K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 7389K, committed 7488K, reserved 1114112K + class space used 779K, committed 832K, reserved 1048576K +} +Event: 5.887 GC heap before +{Heap before GC invocations=1 (full 0): + garbage-first heap total 258048K, used 35594K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 18 young (36864K), 2 survivors (4096K) + Metaspace used 16922K, committed 17088K, reserved 1114112K + class space used 1758K, committed 1856K, reserved 1048576K +} +Event: 5.903 GC heap after +{Heap after GC invocations=2 (full 0): + garbage-first heap total 258048K, used 6026K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 2 young (4096K), 2 survivors (4096K) + Metaspace used 16922K, committed 17088K, reserved 1114112K + class space used 1758K, committed 1856K, reserved 1048576K +} +Event: 8.154 GC heap before +{Heap before GC invocations=2 (full 0): + garbage-first heap total 258048K, used 24458K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 13 young (26624K), 2 survivors (4096K) + Metaspace used 21262K, committed 21504K, reserved 1114112K + class space used 2226K, committed 2304K, reserved 1048576K +} +Event: 8.173 GC heap after +{Heap after GC invocations=3 (full 0): + garbage-first heap total 258048K, used 6642K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 21262K, committed 21504K, reserved 1114112K + class space used 2226K, committed 2304K, reserved 1048576K +} +Event: 10.961 GC heap before +{Heap before GC invocations=4 (full 0): + garbage-first heap total 49152K, used 35314K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 19 young (38912K), 3 survivors (6144K) + Metaspace used 27645K, committed 27968K, reserved 1114112K + class space used 2968K, committed 3072K, reserved 1048576K +} +Event: 10.978 GC heap after +{Heap after GC invocations=5 (full 0): + garbage-first heap total 49152K, used 7918K [0x0000000708200000, 0x0000000800000000) + region size 2048K, 3 young (6144K), 3 survivors (6144K) + Metaspace used 27645K, committed 27968K, reserved 1114112K + class space used 2968K, committed 3072K, reserved 1048576K +} + +Dll operation events (11 events): +Event: 0.006 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +Event: 0.043 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +Event: 0.725 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +Event: 0.768 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +Event: 0.919 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +Event: 0.988 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +Event: 1.046 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +Event: 1.367 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +Event: 3.493 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +Event: 3.503 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +Event: 3.598 Loaded shared library /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so + +Deoptimization events (20 events): +Event: 11.676 Thread 0x00007f521453b510 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fccbcc relative=0x000000000000032c +Event: 11.676 Thread 0x00007f52145418f0 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fccbcc relative=0x000000000000032c +Event: 11.676 Thread 0x00007f521453b510 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fccbcc method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 +Event: 11.676 Thread 0x00007f521453b510 DEOPT PACKING pc=0x00007f5280fccbcc sp=0x00007f524a1194a0 +Event: 11.677 Thread 0x00007f52145418f0 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fccbcc method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 +Event: 11.678 Thread 0x00007f52145418f0 DEOPT PACKING pc=0x00007f5280fccbcc sp=0x00007f5249b194a0 +Event: 11.678 Thread 0x00007f52145418f0 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249b192e8 mode 2 +Event: 11.679 Thread 0x00007f521453b510 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f524a1192e8 mode 2 +Event: 11.745 Thread 0x00007f51ec003790 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280f34730 relative=0x0000000000000210 +Event: 11.745 Thread 0x00007f51ec003790 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280f34730 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 +Event: 11.745 Thread 0x00007f51ec003790 DEOPT PACKING pc=0x00007f5280f34730 sp=0x00007f52495133f0 +Event: 11.746 Thread 0x00007f51ec003790 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249513358 mode 2 +Event: 11.759 Thread 0x00007f5214540530 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fd2008 relative=0x0000000000000608 +Event: 11.759 Thread 0x00007f5214540530 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fd2008 method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 +Event: 11.759 Thread 0x00007f5214540530 DEOPT PACKING pc=0x00007f5280fd2008 sp=0x00007f5249c19520 +Event: 11.759 Thread 0x00007f521453b510 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280f34f30 relative=0x0000000000000190 +Event: 11.759 Thread 0x00007f5214540530 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249c19240 mode 2 +Event: 11.759 Thread 0x00007f521453b510 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280f34f30 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 +Event: 11.759 Thread 0x00007f521453b510 DEOPT PACKING pc=0x00007f5280f34f30 sp=0x00007f524a119330 +Event: 11.759 Thread 0x00007f521453b510 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f524a119308 mode 2 + +Classes unloaded (0 events): +No events + +Classes redefined (0 events): +No events + +Internal exceptions (20 events): +Event: 11.337 Thread 0x00007f521453bfc0 Exception (0x0000000716290440) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.376 Thread 0x00007f521453b510 Exception (0x00000007162b94e0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.377 Thread 0x00007f521453b510 Exception (0x00000007162bded0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.392 Thread 0x00007f52145418f0 Exception (0x00000007162c7948) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.429 Thread 0x00007f52145418f0 Exception (0x0000000716362dd8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.435 Thread 0x00007f52145418f0 Exception (0x0000000716367940) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.436 Thread 0x00007f52145418f0 Exception (0x000000071636c0f8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.495 Thread 0x00007f51ec003790 Exception (0x00000007163fc868) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.560 Thread 0x00007f51ec003790 Exception (0x00000007160b1a30) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.567 Thread 0x00007f51ec003790 Exception (0x00000007160c5998) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.570 Thread 0x00007f51ec003790 Exception (0x00000007160cd300) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.611 Thread 0x00007f521453cf70 Exception (0x00000007161291e8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.614 Thread 0x00007f521453cf70 Exception (0x0000000716135bb0) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.616 Thread 0x00007f521453cf70 Exception (0x000000071613a718) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.620 Thread 0x00007f521453cf70 Exception (0x0000000716143640) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.624 Thread 0x00007f521453cf70 Exception (0x000000071614fe40) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.626 Thread 0x00007f521453cf70 Exception (0x00000007161549a8) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.680 Thread 0x00007f5214542800 Exception (0x00000007161e9d28) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.704 Thread 0x00007f5214542800 Exception (0x00000007161f9e58) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] +Event: 11.722 Thread 0x00007f5214542800 Exception (0x0000000715e67958) +thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] + +VM Operations (20 events): +Event: 6.904 Executing VM operation: Cleanup +Event: 6.906 Executing VM operation: Cleanup done +Event: 7.161 Executing VM operation: HandshakeAllThreads +Event: 7.162 Executing VM operation: HandshakeAllThreads done +Event: 8.154 Executing VM operation: CollectForMetadataAllocation +Event: 8.173 Executing VM operation: CollectForMetadataAllocation done +Event: 8.207 Executing VM operation: G1Concurrent +Event: 8.214 Executing VM operation: G1Concurrent done +Event: 8.214 Executing VM operation: G1Concurrent +Event: 8.215 Executing VM operation: G1Concurrent done +Event: 8.297 Executing VM operation: HandshakeAllThreads +Event: 8.297 Executing VM operation: HandshakeAllThreads done +Event: 9.297 Executing VM operation: Cleanup +Event: 9.298 Executing VM operation: Cleanup done +Event: 9.929 Executing VM operation: HandshakeAllThreads +Event: 9.929 Executing VM operation: HandshakeAllThreads done +Event: 10.929 Executing VM operation: Cleanup +Event: 10.930 Executing VM operation: Cleanup done +Event: 10.961 Executing VM operation: G1CollectForAllocation +Event: 10.978 Executing VM operation: G1CollectForAllocation done + +Events (20 events): +Event: 9.129 Protecting memory [0x00007f524981b000,0x00007f524981f000] with protection modes 0 +Event: 9.928 loading class java/security/cert/X509Certificate +Event: 9.928 loading class java/security/cert/X509Extension +Event: 9.928 loading class java/security/cert/X509Extension done +Event: 9.928 loading class java/security/cert/X509Certificate done +Event: 9.929 loading class javax/security/cert/X509Certificate +Event: 9.930 loading class javax/security/cert/Certificate +Event: 9.930 loading class javax/security/cert/Certificate done +Event: 9.930 loading class javax/security/cert/X509Certificate done +Event: 9.932 loading class java/nio/ReadOnlyBufferException +Event: 9.932 loading class java/nio/ReadOnlyBufferException done +Event: 9.959 loading class java/util/concurrent/atomic/Striped64$1 +Event: 9.959 loading class java/util/concurrent/atomic/Striped64$1 done +Event: 10.831 Thread 0x00007f523020de90 Thread exited: 0x00007f523020de90 +Event: 10.938 Thread 0x00007f523020de90 Thread added: 0x00007f523020de90 +Event: 10.939 Protecting memory [0x00007f524981b000,0x00007f524981f000] with protection modes 0 +Event: 11.294 Thread 0x00007f51ec003790 Thread added: 0x00007f51ec003790 +Event: 11.294 Protecting memory [0x00007f5249415000,0x00007f5249419000] with protection modes 0 +Event: 11.349 Thread 0x00007f523438f6f0 Thread added: 0x00007f523438f6f0 +Event: 11.354 Protecting memory [0x00007f5249315000,0x00007f5249319000] with protection modes 0 + + +Dynamic libraries: +708200000-70a400000 rw-p 00000000 00:00 0 +70a400000-715e00000 ---p 00000000 00:00 0 +715e00000-716400000 rw-p 00000000 00:00 0 +716400000-717600000 ---p 00000000 00:00 0 +717600000-717a00000 rw-p 00000000 00:00 0 +717a00000-7ffc00000 ---p 00000000 00:00 0 +7ffc00000-7ffd00000 rw-p 00000000 00:00 0 +7ffd00000-7ffd76000 rw-p 00c9d000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +7ffd76000-7ffe00000 rw-p 00000000 00:00 0 +7ffe00000-7ffe82000 rw-p 00c1b000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +7ffe82000-800000000 rw-p 00000000 00:00 0 +800000000-800be2000 rw-p 00001000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa +800be2000-801000000 ---p 00000000 00:00 0 +801000000-801030000 rw-p 00000000 00:00 0 +801030000-801130000 rw-p 00000000 00:00 0 +801130000-801170000 rw-p 00000000 00:00 0 +801170000-8011b0000 rw-p 00000000 00:00 0 +8011b0000-8011d0000 rw-p 00000000 00:00 0 +8011d0000-801200000 ---p 00000000 00:00 0 +801200000-801390000 rw-p 00000000 00:00 0 +801390000-841000000 ---p 00000000 00:00 0 +55843ef12000-55843ef13000 r--p 00000000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +55843ef13000-55843ef14000 r-xp 00001000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +55843ef14000-55843ef15000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +55843ef15000-55843ef16000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +55843ef16000-55843ef17000 rw-p 00003000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java +55844037f000-5584403c7000 rw-p 00000000 00:00 0 [heap] +7f51c0000000-7f51c0052000 rw-p 00000000 00:00 0 +7f51c0052000-7f51c4000000 ---p 00000000 00:00 0 +7f51c4000000-7f51c434a000 rw-p 00000000 00:00 0 +7f51c434a000-7f51c8000000 ---p 00000000 00:00 0 +7f51c8000000-7f51c838b000 rw-p 00000000 00:00 0 +7f51c838b000-7f51cc000000 ---p 00000000 00:00 0 +7f51cc000000-7f51cc4ce000 rw-p 00000000 00:00 0 +7f51cc4ce000-7f51d0000000 ---p 00000000 00:00 0 +7f51d0000000-7f51d00ec000 rw-p 00000000 00:00 0 +7f51d00ec000-7f51d4000000 ---p 00000000 00:00 0 +7f51d4000000-7f51d4021000 rw-p 00000000 00:00 0 +7f51d4021000-7f51d8000000 ---p 00000000 00:00 0 +7f51d8000000-7f51d80b0000 rw-p 00000000 00:00 0 +7f51d80b0000-7f51dc000000 ---p 00000000 00:00 0 +7f51dc000000-7f51dc1f1000 rw-p 00000000 00:00 0 +7f51dc1f1000-7f51e0000000 ---p 00000000 00:00 0 +7f51e0000000-7f51e0021000 rw-p 00000000 00:00 0 +7f51e0021000-7f51e4000000 ---p 00000000 00:00 0 +7f51e4000000-7f51e42c3000 rw-p 00000000 00:00 0 +7f51e42c3000-7f51e8000000 ---p 00000000 00:00 0 +7f51e8000000-7f51e80fd000 rw-p 00000000 00:00 0 +7f51e80fd000-7f51ec000000 ---p 00000000 00:00 0 +7f51ec000000-7f51ec03a000 rw-p 00000000 00:00 0 +7f51ec03a000-7f51f0000000 ---p 00000000 00:00 0 +7f51f0000000-7f51f004a000 rw-p 00000000 00:00 0 +7f51f004a000-7f51f4000000 ---p 00000000 00:00 0 +7f51f4000000-7f51f4161000 rw-p 00000000 00:00 0 +7f51f4161000-7f51f8000000 ---p 00000000 00:00 0 +7f51f8000000-7f51f8021000 rw-p 00000000 00:00 0 +7f51f8021000-7f51fc000000 ---p 00000000 00:00 0 +7f51fc000000-7f51fc1b5000 rw-p 00000000 00:00 0 +7f51fc1b5000-7f5200000000 ---p 00000000 00:00 0 +7f5200000000-7f5200021000 rw-p 00000000 00:00 0 +7f5200021000-7f5204000000 ---p 00000000 00:00 0 +7f5204000000-7f5204021000 rw-p 00000000 00:00 0 +7f5204021000-7f5208000000 ---p 00000000 00:00 0 +7f5208000000-7f5208021000 rw-p 00000000 00:00 0 +7f5208021000-7f520c000000 ---p 00000000 00:00 0 +7f520c000000-7f520c021000 rw-p 00000000 00:00 0 +7f520c021000-7f5210000000 ---p 00000000 00:00 0 +7f5210000000-7f5210021000 rw-p 00000000 00:00 0 +7f5210021000-7f5214000000 ---p 00000000 00:00 0 +7f5214000000-7f5214603000 rw-p 00000000 00:00 0 +7f5214603000-7f5218000000 ---p 00000000 00:00 0 +7f5218000000-7f5218021000 rw-p 00000000 00:00 0 +7f5218021000-7f521c000000 ---p 00000000 00:00 0 +7f521c000000-7f521c021000 rw-p 00000000 00:00 0 +7f521c021000-7f5220000000 ---p 00000000 00:00 0 +7f5220000000-7f5220081000 rw-p 00000000 00:00 0 +7f5220081000-7f5224000000 ---p 00000000 00:00 0 +7f5224000000-7f5224021000 rw-p 00000000 00:00 0 +7f5224021000-7f5228000000 ---p 00000000 00:00 0 +7f5228000000-7f5228021000 rw-p 00000000 00:00 0 +7f5228021000-7f522c000000 ---p 00000000 00:00 0 +7f522c000000-7f522c021000 rw-p 00000000 00:00 0 +7f522c021000-7f5230000000 ---p 00000000 00:00 0 +7f5230000000-7f523080e000 rw-p 00000000 00:00 0 +7f523080e000-7f5234000000 ---p 00000000 00:00 0 +7f5234000000-7f52346cf000 rw-p 00000000 00:00 0 +7f52346cf000-7f5238000000 ---p 00000000 00:00 0 +7f5238000000-7f5238021000 rw-p 00000000 00:00 0 +7f5238021000-7f523c000000 ---p 00000000 00:00 0 +7f523c000000-7f523c021000 rw-p 00000000 00:00 0 +7f523c021000-7f5240000000 ---p 00000000 00:00 0 +7f5240000000-7f5240021000 rw-p 00000000 00:00 0 +7f5240021000-7f5244000000 ---p 00000000 00:00 0 +7f5244000000-7f5244021000 rw-p 00000000 00:00 0 +7f5244021000-7f5248000000 ---p 00000000 00:00 0 +7f5248e2b000-7f5249211000 rw-p 00000000 00:00 0 +7f5249315000-7f5249319000 ---p 00000000 00:00 0 +7f5249319000-7f5249415000 rw-p 00000000 00:00 0 +7f5249415000-7f5249419000 ---p 00000000 00:00 0 +7f5249419000-7f5249515000 rw-p 00000000 00:00 0 +7f5249515000-7f5249516000 ---p 00000000 00:00 0 +7f5249516000-7f5249617000 rw-p 00000000 00:00 0 +7f5249617000-7f5249618000 ---p 00000000 00:00 0 +7f5249618000-7f5249719000 rw-p 00000000 00:00 0 +7f5249719000-7f524971a000 ---p 00000000 00:00 0 +7f524971a000-7f524981b000 rw-p 00000000 00:00 0 +7f524981b000-7f524981f000 ---p 00000000 00:00 0 +7f524981f000-7f524991b000 rw-p 00000000 00:00 0 +7f524991b000-7f524991f000 ---p 00000000 00:00 0 +7f524991f000-7f5249a1b000 rw-p 00000000 00:00 0 +7f5249a1b000-7f5249a1f000 ---p 00000000 00:00 0 +7f5249a1f000-7f5249b1b000 rw-p 00000000 00:00 0 +7f5249b1b000-7f5249b1f000 ---p 00000000 00:00 0 +7f5249b1f000-7f5249c1b000 rw-p 00000000 00:00 0 +7f5249c1b000-7f5249c1f000 ---p 00000000 00:00 0 +7f5249c1f000-7f5249d1b000 rw-p 00000000 00:00 0 +7f5249d1b000-7f5249d1f000 ---p 00000000 00:00 0 +7f5249d1f000-7f5249e1b000 rw-p 00000000 00:00 0 +7f5249e1b000-7f5249e1f000 ---p 00000000 00:00 0 +7f5249e1f000-7f5249f1b000 rw-p 00000000 00:00 0 +7f5249f1b000-7f5249f1f000 ---p 00000000 00:00 0 +7f5249f1f000-7f524a01b000 rw-p 00000000 00:00 0 +7f524a01b000-7f524a01f000 ---p 00000000 00:00 0 +7f524a01f000-7f524a11b000 rw-p 00000000 00:00 0 +7f524a11b000-7f524a15b000 rw-s 10000000 00:0e 65771 anon_inode:[io_uring] +7f524a15b000-7f524a180000 rw-s 00000000 00:0e 65771 anon_inode:[io_uring] +7f524a180000-7f524a1c0000 rw-s 10000000 00:0e 65769 anon_inode:[io_uring] +7f524a1c0000-7f524a200000 rw-s 10000000 00:0e 65767 anon_inode:[io_uring] +7f524a200000-7f524a211000 r-xp 00000000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) +7f524a211000-7f524a411000 ---p 00011000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) +7f524a411000-7f524a413000 rw-p 00011000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) +7f524a416000-7f524a43b000 rw-s 00000000 00:0e 65769 anon_inode:[io_uring] +7f524a43b000-7f524a460000 rw-s 00000000 00:0e 65767 anon_inode:[io_uring] +7f524a460000-7f524a4a0000 rw-s 10000000 00:0e 65765 anon_inode:[io_uring] +7f524a4a0000-7f524a4c5000 rw-s 00000000 00:0e 65765 anon_inode:[io_uring] +7f524a4c5000-7f524a505000 rw-s 10000000 00:0e 65763 anon_inode:[io_uring] +7f524a505000-7f524a52a000 rw-s 00000000 00:0e 65763 anon_inode:[io_uring] +7f524a52a000-7f524a56a000 rw-s 10000000 00:0e 65761 anon_inode:[io_uring] +7f524a56a000-7f524a58f000 rw-s 00000000 00:0e 65761 anon_inode:[io_uring] +7f524a58f000-7f524a5cf000 rw-s 10000000 00:0e 65759 anon_inode:[io_uring] +7f524a5cf000-7f524a5f4000 rw-s 00000000 00:0e 65759 anon_inode:[io_uring] +7f524a5f4000-7f524a5f8000 ---p 00000000 00:00 0 +7f524a5f8000-7f524a6f4000 rw-p 00000000 00:00 0 +7f524a6f4000-7f524a6f5000 ---p 00000000 00:00 0 +7f524a6f5000-7f524a7f6000 rw-p 00000000 00:00 0 +7f524a7f6000-7f524a7f7000 ---p 00000000 00:00 0 +7f524a7f7000-7f524a8f8000 rw-p 00000000 00:00 0 +7f524a8f8000-7f524a8f9000 ---p 00000000 00:00 0 +7f524a8f9000-7f524a9fa000 rw-p 00000000 00:00 0 +7f524a9fa000-7f524a9fb000 ---p 00000000 00:00 0 +7f524a9fb000-7f524aafc000 rw-p 00000000 00:00 0 +7f524aafc000-7f524aafd000 ---p 00000000 00:00 0 +7f524aafd000-7f524abfe000 rw-p 00000000 00:00 0 +7f524abfe000-7f524ac02000 ---p 00000000 00:00 0 +7f524ac02000-7f524acfe000 rw-p 00000000 00:00 0 +7f524acfe000-7f524ad02000 ---p 00000000 00:00 0 +7f524ad02000-7f524adfe000 rw-p 00000000 00:00 0 +7f524adfe000-7f524adff000 ---p 00000000 00:00 0 +7f524adff000-7f524af00000 rw-p 00000000 00:00 0 +7f524af00000-7f524af04000 ---p 00000000 00:00 0 +7f524af04000-7f524b000000 rw-p 00000000 00:00 0 +7f524b000000-7f524b004000 ---p 00000000 00:00 0 +7f524b004000-7f524b100000 rw-p 00000000 00:00 0 +7f524b100000-7f524b104000 ---p 00000000 00:00 0 +7f524b104000-7f524b200000 rw-p 00000000 00:00 0 +7f524b200000-7f524bfea000 r--p 00000000 103:02 1055507 /usr/lib/locale/locale-archive +7f524c000000-7f524c021000 rw-p 00000000 00:00 0 +7f524c021000-7f5250000000 ---p 00000000 00:00 0 +7f5250000000-7f5250021000 rw-p 00000000 00:00 0 +7f5250021000-7f5254000000 ---p 00000000 00:00 0 +7f5254000000-7f52543f0000 rw-p 00000000 00:00 0 +7f52543f0000-7f5254d90000 rw-p 00000000 00:00 0 +7f5254d90000-7f5254db0000 rw-p 00000000 00:00 0 +7f5254db0000-7f5254e90000 rw-p 00000000 00:00 0 +7f5254e90000-7f5254f00000 ---p 00000000 00:00 0 +7f5254f00000-7f5255a60000 rw-p 00000000 00:00 0 +7f5255a60000-7f5258000000 ---p 00000000 00:00 0 +7f5258000000-7f5258021000 rw-p 00000000 00:00 0 +7f5258021000-7f525c000000 ---p 00000000 00:00 0 +7f525c000000-7f525c021000 rw-p 00000000 00:00 0 +7f525c021000-7f5260000000 ---p 00000000 00:00 0 +7f5260000000-7f5260021000 rw-p 00000000 00:00 0 +7f5260021000-7f5264000000 ---p 00000000 00:00 0 +7f5264000000-7f5264021000 rw-p 00000000 00:00 0 +7f5264021000-7f5268000000 ---p 00000000 00:00 0 +7f5268000000-7f5268021000 rw-p 00000000 00:00 0 +7f5268021000-7f526c000000 ---p 00000000 00:00 0 +7f526c000000-7f526c040000 rw-s 10000000 00:0e 65757 anon_inode:[io_uring] +7f526c040000-7f526c065000 rw-s 00000000 00:0e 65757 anon_inode:[io_uring] +7f526c065000-7f526c069000 ---p 00000000 00:00 0 +7f526c069000-7f526c165000 rw-p 00000000 00:00 0 +7f526c165000-7f526c169000 ---p 00000000 00:00 0 +7f526c169000-7f526c265000 rw-p 00000000 00:00 0 +7f526c265000-7f526c269000 ---p 00000000 00:00 0 +7f526c269000-7f526c365000 rw-p 00000000 00:00 0 +7f526c365000-7f526c369000 ---p 00000000 00:00 0 +7f526c369000-7f526c465000 rw-p 00000000 00:00 0 +7f526c465000-7f526c469000 ---p 00000000 00:00 0 +7f526c469000-7f526c565000 rw-p 00000000 00:00 0 +7f526c565000-7f526c569000 ---p 00000000 00:00 0 +7f526c569000-7f526c665000 rw-p 00000000 00:00 0 +7f526c665000-7f526c669000 ---p 00000000 00:00 0 +7f526c669000-7f526c765000 rw-p 00000000 00:00 0 +7f526c765000-7f526c769000 ---p 00000000 00:00 0 +7f526c769000-7f526c865000 rw-p 00000000 00:00 0 +7f526c865000-7f526c869000 ---p 00000000 00:00 0 +7f526c869000-7f526c965000 rw-p 00000000 00:00 0 +7f526c965000-7f526c966000 ---p 00000000 00:00 0 +7f526c966000-7f526ca67000 rw-p 00000000 00:00 0 +7f526ca67000-7f526ca6c000 r--p 00000000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f526ca6c000-7f526caad000 r-xp 00005000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f526caad000-7f526cb36000 r--p 00046000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f526cb36000-7f526cb37000 r--p 000ce000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f526cb37000-7f526cb38000 rw-p 000cf000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so +7f526cb38000-7f526cbb9000 rw-p 00000000 00:00 0 +7f526cbb9000-7f526cbba000 ---p 00000000 00:00 0 +7f526cbba000-7f526ccbb000 rw-p 00000000 00:00 0 +7f526ccbb000-7f526ccbc000 ---p 00000000 00:00 0 +7f526ccbc000-7f526f5c5000 rw-p 00000000 00:00 0 +7f526f5c5000-7f526f5c6000 ---p 00000000 00:00 0 +7f526f5c6000-7f526f6c7000 rw-p 00000000 00:00 0 +7f526f6c7000-7f526f6c8000 ---p 00000000 00:00 0 +7f526f6c8000-7f526f7c9000 rw-p 00000000 00:00 0 +7f526f7c9000-7f526f7ca000 ---p 00000000 00:00 0 +7f526f7ca000-7f526f953000 rw-p 00000000 00:00 0 +7f526f953000-7f526fc3b000 ---p 00000000 00:00 0 +7f526fc3b000-7f526fc53000 rw-p 00000000 00:00 0 +7f526fc53000-7f526fc9b000 ---p 00000000 00:00 0 +7f526fc9b000-7f526fcab000 rw-p 00000000 00:00 0 +7f526fcab000-7f52736b3000 ---p 00000000 00:00 0 +7f52736b3000-7f527374b000 rw-p 00000000 00:00 0 +7f527374b000-7f5273a33000 ---p 00000000 00:00 0 +7f5273a33000-7f5273a4b000 rw-p 00000000 00:00 0 +7f5273a4b000-7f5273a93000 ---p 00000000 00:00 0 +7f5273a93000-7f5273aa3000 rw-p 00000000 00:00 0 +7f5273aa3000-7f52774ab000 ---p 00000000 00:00 0 +7f52774ab000-7f52774cc000 rw-p 00000000 00:00 0 +7f52774cc000-7f5277529000 ---p 00000000 00:00 0 +7f5277529000-7f527752c000 rw-p 00000000 00:00 0 +7f527752c000-7f5277535000 ---p 00000000 00:00 0 +7f5277535000-7f5277537000 rw-p 00000000 00:00 0 +7f5277537000-7f5277c78000 ---p 00000000 00:00 0 +7f5277c78000-7f5277c8b000 rw-p 00000000 00:00 0 +7f5277c8b000-7f5277ce8000 ---p 00000000 00:00 0 +7f5277ce8000-7f5277ceb000 rw-p 00000000 00:00 0 +7f5277ceb000-7f5277cf4000 ---p 00000000 00:00 0 +7f5277cf4000-7f5277cf6000 rw-p 00000000 00:00 0 +7f5277cf6000-7f5278437000 ---p 00000000 00:00 0 +7f5278437000-7f527844a000 rw-p 00000000 00:00 0 +7f527844a000-7f52784a7000 ---p 00000000 00:00 0 +7f52784a7000-7f52784aa000 rw-p 00000000 00:00 0 +7f52784aa000-7f52784b3000 ---p 00000000 00:00 0 +7f52784b3000-7f52784b5000 rw-p 00000000 00:00 0 +7f52784b5000-7f5278bf6000 ---p 00000000 00:00 0 +7f5278bf6000-7f5279400000 rw-p 00000000 00:00 0 +7f5279400000-7f52797c0000 rwxp 00000000 00:00 0 +7f52797c0000-7f5280937000 ---p 00000000 00:00 0 +7f5280937000-7f5280ba7000 rwxp 00000000 00:00 0 +7f5280ba7000-7f5280ec9000 ---p 00000000 00:00 0 +7f5280ec9000-7f5281139000 rwxp 00000000 00:00 0 +7f5281139000-7f5288400000 ---p 00000000 00:00 0 +7f5288400000-7f528fe42000 r--s 00000000 103:02 1966437 /usr/lib/jvm/java-17-openjdk-amd64/lib/modules +7f528fe43000-7f528fe44000 r--p 00000000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f528fe44000-7f528fe45000 r-xp 00001000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f528fe45000-7f528fe46000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f528fe46000-7f528fe47000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f528fe47000-7f528fe48000 rw-p 00003000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 +7f528fe48000-7f528ff1a000 rw-p 00000000 00:00 0 +7f528ff1a000-7f5290000000 ---p 00000000 00:00 0 +7f5290000000-7f5290412000 rw-p 00000000 00:00 0 +7f5290412000-7f5294000000 ---p 00000000 00:00 0 +7f5294001000-7f5294002000 r--p 00000000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f5294002000-7f5294003000 r-xp 00001000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f5294003000-7f5294004000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f5294004000-7f5294005000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f5294005000-7f5294006000 rw-p 00003000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 +7f5294015000-7f529401d000 rw-p 00000000 00:00 0 +7f529401d000-7f5294100000 ---p 00000000 00:00 0 +7f5294100000-7f5294104000 ---p 00000000 00:00 0 +7f5294104000-7f5294200000 rw-p 00000000 00:00 0 +7f5294200000-7f529429c000 r--p 00000000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f529429c000-7f52943cb000 r-xp 0009c000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f52943cb000-7f5294458000 r--p 001cb000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f5294458000-7f5294463000 r--p 00257000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f5294463000-7f5294466000 rw-p 00262000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 +7f5294466000-7f529446a000 rw-p 00000000 00:00 0 +7f529446d000-7f529446f000 r--p 00000000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f529446f000-7f5294472000 r-xp 00002000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f5294472000-7f5294474000 r--p 00005000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f5294474000-7f5294475000 r--p 00006000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f5294475000-7f5294476000 rw-p 00007000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so +7f5294476000-7f5294478000 r--p 00000000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f5294478000-7f5294479000 r-xp 00002000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f5294479000-7f529447a000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f529447a000-7f529447b000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f529447b000-7f529447c000 rw-p 00004000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so +7f529447c000-7f5294480000 r--p 00000000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f5294480000-7f5294487000 r-xp 00004000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f5294487000-7f5294489000 r--p 0000b000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f5294489000-7f529448b000 r--p 0000c000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f529448b000-7f529448c000 rw-p 0000e000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so +7f529448c000-7f529448e000 r--p 00000000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f529448e000-7f5294493000 r-xp 00002000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f5294493000-7f5294495000 r--p 00007000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f5294495000-7f5294496000 r--p 00008000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f5294496000-7f5294497000 rw-p 00009000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so +7f5294497000-7f5294517000 rw-p 00000000 00:00 0 +7f5294517000-7f5294525000 r--p 00000000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f5294525000-7f52945a3000 r-xp 0000e000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f52945a3000-7f52945fe000 r--p 0008c000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f52945fe000-7f52945ff000 r--p 000e6000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f52945ff000-7f5294600000 rw-p 000e7000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 +7f5294600000-7f5294851000 r--p 00000000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f5294851000-7f52955d5000 r-xp 00251000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f52955d5000-7f5295856000 r--p 00fd5000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f5295856000-7f529590e000 r--p 01255000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f529590e000-7f5295943000 rw-p 0130d000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so +7f5295943000-7f529599d000 rw-p 00000000 00:00 0 +7f52959a0000-7f52959a1000 r--p 00000000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f52959a1000-7f52959a2000 r-xp 00001000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f52959a2000-7f52959a3000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f52959a3000-7f52959a4000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f52959a4000-7f52959a5000 rw-p 00003000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so +7f52959a5000-7f52959a9000 r--p 00000000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f52959a9000-7f52959b7000 r-xp 00004000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f52959b7000-7f52959bb000 r--p 00012000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f52959bb000-7f52959bc000 r--p 00015000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f52959bc000-7f52959bd000 rw-p 00016000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so +7f52959bd000-7f52959c3000 r--p 00000000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f52959c3000-7f52959cc000 r-xp 00006000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f52959cc000-7f52959d0000 r--p 0000f000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f52959d0000-7f52959d1000 r--p 00013000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f52959d1000-7f52959d2000 rw-p 00014000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so +7f52959d2000-7f52959d3000 r--p 00000000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f52959d3000-7f52959d6000 r-xp 00001000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f52959d6000-7f52959d7000 r--p 00004000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f52959d7000-7f52959d8000 r--p 00005000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f52959d8000-7f52959d9000 rw-p 00006000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so +7f52959d9000-7f52959e5000 r--p 00000000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f52959e5000-7f52959f7000 r-xp 0000c000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f52959f7000-7f52959fd000 r--p 0001e000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f52959fd000-7f52959fe000 r--p 00023000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f52959fe000-7f52959ff000 rw-p 00024000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so +7f52959ff000-7f5295a00000 rw-p 00000000 00:00 0 +7f5295a00000-7f5295a22000 r--p 00000000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f5295a22000-7f5295b9a000 r-xp 00022000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f5295b9a000-7f5295bf2000 r--p 0019a000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f5295bf2000-7f5295bf6000 r--p 001f1000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f5295bf6000-7f5295bf8000 rw-p 001f5000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 +7f5295bf8000-7f5295c05000 rw-p 00000000 00:00 0 +7f5295c09000-7f5295c10000 r--s 00000000 103:02 1074285 /usr/lib/x86_64-linux-gnu/gconv/gconv-modules.cache +7f5295c10000-7f5295c15000 rw-p 00000000 00:00 0 +7f5295c15000-7f5295c1c000 ---p 00000000 00:00 0 +7f5295c1c000-7f5295c24000 rw-s 00000000 103:02 3673026 /tmp/hsperfdata_antonio/8969 +7f5295c24000-7f5295c27000 r--p 00000000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f5295c27000-7f5295c55000 r-xp 00003000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f5295c55000-7f5295c63000 r--p 00031000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f5295c63000-7f5295c64000 r--p 0003e000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f5295c64000-7f5295c65000 rw-p 0003f000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so +7f5295c65000-7f5295c66000 rw-p 00000000 00:00 0 +7f5295c66000-7f5295c69000 r--p 00000000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f5295c69000-7f5295c84000 r-xp 00003000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f5295c84000-7f5295c88000 r--p 0001e000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f5295c88000-7f5295c89000 r--p 00021000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f5295c89000-7f5295c8a000 rw-p 00022000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 +7f5295c8a000-7f5295c8d000 rw-p 00000000 00:00 0 +7f5295c8d000-7f5295c90000 r--p 00000000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f5295c90000-7f5295ca2000 r-xp 00003000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f5295ca2000-7f5295ca9000 r--p 00015000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f5295ca9000-7f5295caa000 r--p 0001b000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f5295caa000-7f5295cab000 rw-p 0001c000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 +7f5295caf000-7f5295cb0000 ---p 00000000 00:00 0 +7f5295cb0000-7f5295cb1000 r--p 00000000 00:00 0 +7f5295cb1000-7f5295cb2000 ---p 00000000 00:00 0 +7f5295cb2000-7f5295cb4000 r--p 00000000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f5295cb4000-7f5295cb7000 r-xp 00002000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f5295cb7000-7f5295cb8000 r--p 00005000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f5295cb8000-7f5295cb9000 r--p 00006000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f5295cb9000-7f5295cba000 rw-p 00007000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so +7f5295cba000-7f5295cbc000 r--p 00000000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f5295cbc000-7f5295cc6000 r-xp 00002000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f5295cc6000-7f5295cc9000 r--p 0000c000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f5295cc9000-7f5295cca000 r--p 0000e000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f5295cca000-7f5295ccb000 rw-p 0000f000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so +7f5295ccb000-7f5295ccd000 rw-p 00000000 00:00 0 +7f5295ccd000-7f5295cce000 r--p 00000000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f5295cce000-7f5295cf6000 r-xp 00001000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f5295cf6000-7f5295d00000 r--p 00029000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f5295d00000-7f5295d02000 r--p 00033000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7f5295d02000-7f5295d04000 rw-p 00035000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 +7ffe4c443000-7ffe4c465000 rw-p 00000000 00:00 0 [stack] +7ffe4c4c6000-7ffe4c4ca000 r--p 00000000 00:00 0 [vvar] +7ffe4c4ca000-7ffe4c4cc000 r-xp 00000000 00:00 0 [vdso] +ffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall] + + +VM Arguments: +jvm_args: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring +java_command: sbt.ForkMain 33543 +java_class_path (initial): /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/test-classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scala-lang/scala-library/2.13.10/scala-library-2.13.10.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect_2.13/3.6-0142603/cats-effect_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-io_2.13/3.7.0/fs2-io_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/munit-cats-effect_2.13/2.0.0-M3/munit-cats-effect_2.13-2.0.0-M3.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-native-io_uring/0.0.21.Final/netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-kernel_2.13/3.6-0142603/cats-effect-kernel_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-std_2.13/3.6-0142603/cats-effect-std_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-core_2.13/3.7.0/fs2-core_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/com/comcast/ip4s-core_2.13/3.3.0/ip4s-core_2.13-3.3.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scalameta/munit_2.13/1.0.0-M6/munit_2.13-1.0.0-M6.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-classes-io_uring/0.0.21.Final/netty-incubator-transport-classes-io_uring-0.0.21.Final.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-core_2.13/2.9.0/cats-core_2.13-2.9.0.ja +Launcher Type: SUN_STANDARD + +[Global flags] + intx CICompilerCount = 4 {product} {ergonomic} + uint ConcGCThreads = 2 {product} {ergonomic} + uint G1ConcRefinementThreads = 8 {product} {ergonomic} + size_t G1HeapRegionSize = 2097152 {product} {ergonomic} + uintx GCDrainStackTargetSize = 64 {product} {ergonomic} + size_t InitialHeapSize = 260046848 {product} {ergonomic} + size_t MarkStackSize = 4194304 {product} {ergonomic} + size_t MaxHeapSize = 4158652416 {product} {ergonomic} + size_t MaxNewSize = 2493513728 {product} {ergonomic} + size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} + size_t MinHeapSize = 8388608 {product} {ergonomic} + uintx NonNMethodCodeHeapSize = 5839372 {pd product} {ergonomic} + uintx NonProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} + uintx ProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} + uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} + bool SegmentedCodeCache = true {product} {ergonomic} + size_t SoftMaxHeapSize = 4158652416 {manageable} {ergonomic} + bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} + bool UseCompressedOops = true {product lp64_product} {ergonomic} + bool UseG1GC = true {product} {ergonomic} + +Logging: +Log output configuration: + #0: stdout all=warning uptime,level,tags + #1: stderr all=off uptime,level,tags + +Environment Variables: +JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64/ +PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/games:/usr/local/games:/snap/bin:/snap/bin:/home/antonio/.local/share/coursier/bin:/home/antonio/.local/share/coursier/bin +USERNAME=antonio +SHELL=/bin/bash +DISPLAY=:0 +LANG=en_US.UTF-8 + +Active Locale: +LC_ALL=en_US.UTF-8 +LC_COLLATE=en_US.UTF-8 +LC_CTYPE=en_US.UTF-8 +LC_MESSAGES=en_US.UTF-8 +LC_MONETARY=en_US.UTF-8 +LC_NUMERIC=en_US.UTF-8 +LC_TIME=en_US.UTF-8 + +Signal Handlers: + SIGSEGV: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGBUS: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGFPE: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGPIPE: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGXFSZ: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGILL: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + SIGUSR2: SR_handler in libjvm.so, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked + SIGHUP: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGINT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTERM: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGQUIT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked + SIGTRAP: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked + + +--------------- S Y S T E M --------------- + +OS: +DISTRIB_ID=Ubuntu +DISTRIB_RELEASE=23.04 +DISTRIB_CODENAME=lunar +DISTRIB_DESCRIPTION="Ubuntu 23.04" +uname: Linux 6.2.0-26-generic #26-Ubuntu SMP PREEMPT_DYNAMIC Mon Jul 10 23:39:54 UTC 2023 x86_64 +OS uptime: 0 days 1:10 hours +libc: glibc 2.37 NPTL 2.37 +rlimit (soft/hard): STACK 8192k/infinity , CORE 0k/infinity , NPROC 63121/63121 , NOFILE 1048576/1048576 , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK 2030188k/2030188k +load average: 4.85 4.29 4.82 + +/proc/meminfo: +MemTotal: 16241528 kB +MemFree: 6777444 kB +MemAvailable: 9738900 kB +Buffers: 107464 kB +Cached: 3577316 kB +SwapCached: 0 kB +Active: 1069368 kB +Inactive: 7386124 kB +Active(anon): 4076 kB +Inactive(anon): 5296692 kB +Active(file): 1065292 kB +Inactive(file): 2089432 kB +Unevictable: 476784 kB +Mlocked: 64 kB +SwapTotal: 4194300 kB +SwapFree: 4194300 kB +Zswap: 0 kB +Zswapped: 0 kB +Dirty: 188 kB +Writeback: 0 kB +AnonPages: 5247632 kB +Mapped: 894504 kB +Shmem: 547584 kB +KReclaimable: 125384 kB +Slab: 285052 kB +SReclaimable: 125384 kB +SUnreclaim: 159668 kB +KernelStack: 21840 kB +PageTables: 51860 kB +SecPageTables: 0 kB +NFS_Unstable: 0 kB +Bounce: 0 kB +WritebackTmp: 0 kB +CommitLimit: 12315064 kB +Committed_AS: 16092828 kB +VmallocTotal: 34359738367 kB +VmallocUsed: 118800 kB +VmallocChunk: 0 kB +Percpu: 6048 kB +HardwareCorrupted: 0 kB +AnonHugePages: 2048 kB +ShmemHugePages: 0 kB +ShmemPmdMapped: 0 kB +FileHugePages: 0 kB +FilePmdMapped: 0 kB +HugePages_Total: 0 +HugePages_Free: 0 +HugePages_Rsvd: 0 +HugePages_Surp: 0 +Hugepagesize: 2048 kB +Hugetlb: 0 kB +DirectMap4k: 379380 kB +DirectMap2M: 7890944 kB +DirectMap1G: 8388608 kB + +/sys/kernel/mm/transparent_hugepage/enabled: always [madvise] never +/sys/kernel/mm/transparent_hugepage/defrag (defrag/compaction efforts parameter): always defer defer+madvise [madvise] never + +Process Memory: +Virtual Size: 8613960K (peak: 8613960K) +Resident Set Size: 172172K (peak: 172172K) (anon: 142092K, file: 30080K, shmem: 0K) +Swapped out: 0K +C-Heap outstanding allocations: 61705K, retained: 12622K +glibc malloc tunables: (default) + +/proc/sys/kernel/threads-max (system-wide limit on the number of threads): 126243 +/proc/sys/vm/max_map_count (maximum number of memory map areas a process may have): 65530 +/proc/sys/kernel/pid_max (system-wide limit on number of process identifiers): 4194304 + +container (cgroup) information: +container_type: cgroupv2 +cpu_cpuset_cpus: not supported +cpu_memory_nodes: not supported +active_processor_count: 8 +cpu_quota: not supported +cpu_period: not supported +cpu_shares: not supported +memory_limit_in_bytes: unlimited +memory_and_swap_limit_in_bytes: unlimited +memory_soft_limit_in_bytes: unlimited +memory_usage_in_bytes: 4508120 k +memory_max_usage_in_bytes: not supported +memory_swap_current_in_bytes: unlimited +memory_swap_max_limit_in_bytes: unlimited +maximum number of tasks: 18936 +current number of tasks: 370 + +Steal ticks since vm start: 0 +Steal ticks percentage since vm start: 0.000 + +CPU: total 8 (initial active 8) (4 cores per cpu, 2 threads per core) family 6 model 142 stepping 10 microcode 0xf0, cx8, cmov, fxsr, ht, mmx, 3dnowpref, sse, sse2, sse3, ssse3, sse4.1, sse4.2, popcnt, lzcnt, tsc, tscinvbit, avx, avx2, aes, erms, clmul, bmi1, bmi2, adx, fma, vzeroupper, clflush, clflushopt +CPU Model and flags from /proc/cpuinfo: +model name : Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz +flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc art arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc cpuid aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch cpuid_fault epb invpcid_single pti ssbd ibrs ibpb stibp tpr_shadow vnmi flexpriority ept vpid ept_ad fsgsbase tsc_adjust bmi1 avx2 smep bmi2 erms invpcid mpx rdseed adx smap clflushopt intel_pt xsaveopt xsavec xgetbv1 xsaves dtherm ida arat pln pts hwp hwp_notify hwp_act_window hwp_epp md_clear flush_l1d arch_capabilities + +Online cpus: 0-7 +Offline cpus: +BIOS frequency limitation: +Frequency switch latency (ns): 0 +Available cpu frequencies: +Current governor: powersave +Core performance/turbo boost: + +Memory: 4k page, physical 16241528k(6777444k free), swap 4194300k(4194300k free) +Page Sizes: 4k + +vm_info: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04) for linux-amd64 JRE (17.0.7+7-Ubuntu-0ubuntu123.04), built on May 6 2023 14:43:58 by "buildd" with gcc 12.2.0 + +END. diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 325cd621..011afc25 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -98,83 +98,87 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn address: Option[Host], port: Option[Port], options: List[SocketOption] - ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = for { + ): Resource[F, (SocketAddress[IpAddress], Stream[F, Socket[F]])] = + Resource.eval(Uring.get[F]).flatMap { ring => + for { + resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) - ring <- Resource.eval(Uring.get[F]) - - resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) - - _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) - - isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] + _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) - linuxSocket <- openSocket(ring, isIpv6) - - _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) + isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] - _ <- Resource.eval( - F.delay( - linuxSocket.bind( - new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) - ) - ) - ) + linuxSocket <- openSocket(ring, isIpv6) - _ <- Resource.eval(F.delay(linuxSocket.listen(65535))) + _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) - localAddress <- Resource.eval( - F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) - ) + localAddress <- Resource.eval { + val bindF = F.delay { + val socketAddress = + new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) + linuxSocket.bind(socketAddress) + } - _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) - - sockets = for { - buf <- Stream.resource(createBufferAux(isIpv6)) - bufLength <- Stream.resource(createBuffer(4)) - res <- Stream.resource { - - // Accept a connection, write the remote address on the buf and get the clientFd - val accept: Resource[F, Int] = - Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) *> - Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> - ring - .bracket( - op = IORING_OP_ACCEPT, - fd = linuxSocket.fd(), - bufferAddress = buf.memoryAddress(), - offset = bufLength.memoryAddress() - )(closeSocket(ring, _)) - .mapK { - new cats.~>[IO, IO] { - def apply[A](ioa: IO[A]) = ioa.debug() - } + val listenF = F.delay(linuxSocket.listen(65535)) + + bindF *> listenF *> F + .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) + } + + _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) + + sockets = Stream + .resource(createBufferAux(isIpv6)) + .flatMap { buf => // Buffer that will contain the remote address + Stream + .resource(createBuffer(4)) + .flatMap { + bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer + Stream.resource { + + // bufLength.writeInt(buf.capacity()) TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt + + // We accept a connection, we write the remote address on the buf and we get the clientFd + val accept = + Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) *> + Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> + ring + .bracket( + op = IORING_OP_ACCEPT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = bufLength.memoryAddress() + )(closeSocket(ring, _)) + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = ioa.debug() + } + } + .mapK(LiftIO.liftK) + + // We read the address from the buf and we convert it to SocketAddress + val convert: F[SocketAddress[IpAddress]] = + F.delay( + println( + "[SERVER] getting the address in memory and converting it to SocketAddress..." + ) + ) *> + F.delay(SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6))) + + accept + .flatMap { clientFd => + Resource.eval(convert).flatMap { remoteAddress => + UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) + } + } + .attempt + .map(_.toOption) + }.repeat } - .mapK(LiftIO.liftK) - - // Read the address from the buf and convert it to SocketAddress - val convert: F[SocketAddress[IpAddress]] = - F.delay( - println( - "[SERVER] getting the address in memory and converting it to SocketAddress..." - ) - ) *> - F.delay( - SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6)) - ) - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) - } } - .attempt - .map(_.toOption) - }.repeat - - } yield res - } yield (localAddress, sockets.unNone) + } yield (localAddress, sockets.unNone) + } private[this] def openSocket( ring: Uring, From bdf22684cf995e5945b9e9e38502244e493a5f87 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 13:58:41 +0000 Subject: [PATCH 107/200] Remove print bytes --- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 13fe1abc..6a4e7cb1 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -115,7 +115,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( buffer.writeBytes(bytes.toArray) } - _ <- F.delay(println(s"[SOCKET] writing in array: $bytes")) + _ <- F.delay(println(s"[SOCKET] writing in array...")) _ <- send( buffer.memoryAddress(), From 0659f5b7c68cd9f9a029d6a5b0c9ff08b18811a0 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 14:34:04 +0000 Subject: [PATCH 108/200] Correct bug and rewrite using for-comprehension --- .../fs2/io/uring/net/UringSocketGroup.scala | 95 ++++++++++--------- 1 file changed, 50 insertions(+), 45 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 011afc25..8014a8e6 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -62,7 +62,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) _ <- F.delay( println( - s"[CLIENT] address: ${address.toString()}, Buffer length: $length" + s"[CLIENT] connecting to address: ${address.toString()}, Buffer length: $length" ) ) _ <- F.delay(println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}")) @@ -109,20 +109,19 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn linuxSocket <- openSocket(ring, isIpv6) - _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFD: ${linuxSocket.fd()}"))) + _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFd: ${linuxSocket.fd()}"))) - localAddress <- Resource.eval { - val bindF = F.delay { - val socketAddress = - new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value) - linuxSocket.bind(socketAddress) - } + _ <- Resource.eval( + F.delay( + linuxSocket + .bind(new InetSocketAddress(resolvedAddress.toString, port.getOrElse(port"0").value)) + ) + ) - val listenF = F.delay(linuxSocket.listen(65535)) + _ <- Resource.eval(F.delay(linuxSocket.listen(65535))) - bindF *> listenF *> F - .delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) - } + localAddress <- Resource + .eval(F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress()))) _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) @@ -135,43 +134,49 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer Stream.resource { - // bufLength.writeInt(buf.capacity()) TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt - - // We accept a connection, we write the remote address on the buf and we get the clientFd - val accept = - Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) *> - Resource.eval(F.delay(println("[SERVER] accepting connection..."))) *> - ring - .bracket( - op = IORING_OP_ACCEPT, - fd = linuxSocket.fd(), - bufferAddress = buf.memoryAddress(), - offset = bufLength.memoryAddress() - )(closeSocket(ring, _)) - .mapK { - new cats.~>[IO, IO] { - def apply[A](ioa: IO[A]) = ioa.debug() - } + bufLength.writeInt( + buf.capacity() + ) // TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt + + // Accept a connection, write the remote address on the buf and get the clientFd + val accept: Resource[F, Int] = for { + // _ <- Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) + _ <- Resource.eval(F.delay(println("[SERVER] accepting connection..."))) + clientFd <- ring + .bracket( + op = IORING_OP_ACCEPT, + fd = linuxSocket.fd(), + bufferAddress = buf.memoryAddress(), + offset = bufLength.memoryAddress() + )(closeSocket(ring, _)) + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = ioa.debug() } - .mapK(LiftIO.liftK) + } + .mapK(LiftIO.liftK) + } yield clientFd - // We read the address from the buf and we convert it to SocketAddress + // Read the address from the buf and convert it to SocketAddress val convert: F[SocketAddress[IpAddress]] = F.delay( - println( - "[SERVER] getting the address in memory and converting it to SocketAddress..." - ) - ) *> - F.delay(SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6))) - - accept - .flatMap { clientFd => - Resource.eval(convert).flatMap { remoteAddress => - UringSocket(ring, UringLinuxSocket(clientFd), clientFd, remoteAddress) - } - } - .attempt - .map(_.toOption) + SocketAddress.fromInetSocketAddress(readIpv(buf.memoryAddress(), isIpv6)) + ) + + val socket: Resource[F, UringSocket[F]] = for { + clientFd <- accept + remoteAddress <- Resource.eval(convert) + _ <- Resource + .eval(F.delay(s"[SERVER] connected to $remoteAddress with fd: $clientFd")) + socket <- UringSocket( + ring, + UringLinuxSocket(clientFd), + clientFd, + remoteAddress + ) + } yield socket + + socket.attempt.map(_.toOption) }.repeat } From 6659989b0404f9dc738a072633359643e6398279 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 14:35:01 +0000 Subject: [PATCH 109/200] Add replicateA to tests --- .../fs2/io/uring/net/TcpSocketSuite.scala | 62 +++++++++---------- 1 file changed, 28 insertions(+), 34 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 69aa5d35..61c748da 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -139,45 +139,29 @@ class TcpSocketSuite extends UringSuite { test("Start server and wait for a connection during 10 sec") { serverResource.use { case (localAddress, _) => IO { - println(s"Server started at $localAddress") - println(s"You can now connect to this server") - } *> IO.sleep(20.second) + println(s"[TEST] Server started at $localAddress") + println(s"[TEST] You can now connect to this server") + } *> IO.sleep(10.second) // Use telnet localhost "port" to connect } } test("Start server and connect external client") { serverResource.use { case (localAddress, _) => - IO { - println(s"Server started at $localAddress") - } *> - sg.client(localAddress).use { socket => - val info = IO { - println("Socket created and connection established!") - println(s"remote address connected: ${socket.remoteAddress}") - } - - val assert = socket.remoteAddress.map(assertEquals(_, localAddress)) - - info *> assert + for { + _ <- IO.println(s"[TEST] Server started at $localAddress") + _ <- sg.client(localAddress).use { socket => + for { + remoteAddress <- socket.remoteAddress + _ <- IO.println(s"[TEST] Socket created and connected to $remoteAddress!") + _ <- socket.remoteAddress.map(assertEquals(_, localAddress)) + } yield () } + } yield () } } - // We start using the serverStream - test("Create server and connect external client 2") { - serverResource.use { case (localAddress, serverStream) => - sg.client(localAddress).use { _ => - val echoServer = - serverStream.compile.drain // If we modify the resource server to just take(1) works. I guess we are trying to bind multiple sockets to the same ip/port ? - - IO.println("socket created and connection established!") *> - echoServer.background.use(_ => IO.sleep(1.second)) - } - } - } - - test("Create server connect with external client and writes") { + test("Create server connect external client and writes") { serverResource.use { case (localAddress, serverStream) => sg.client(localAddress).use { socket => val msg = "Hello, echo server!\n" @@ -229,7 +213,7 @@ class TcpSocketSuite extends UringSuite { echoServer.background.use(_ => IO.sleep( 1.second - ) *> // Ensures that the server is ready before the client tries to connect + ) *> writeRead.compile.lastOrError .assertEquals("Hello, echo server!") ) @@ -246,11 +230,13 @@ class TcpSocketSuite extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients + val repetitions: Int = 5 + test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) val clientCount = 20L - Stream + val test: IO[Unit] = Stream .resource(setup) .flatMap { case (server, clients) => val echoServer = server @@ -282,13 +268,15 @@ class TcpSocketSuite extends UringSuite { assertEquals(it.size.toLong, clientCount) assert(it.forall(_ == "fs2.rocks")) } + + test.replicateA(repetitions).void } test("readN yields chunks of the requested size") { val message = Chunk.array("123456789012345678901234567890".getBytes) val sizes = Vector(1, 2, 3, 4, 3, 2, 1) - Stream + val test: IO[Unit] = Stream .resource(setup) .flatMap { case (server, clients) => val junkServer = server.map { socket => @@ -314,12 +302,14 @@ class TcpSocketSuite extends UringSuite { .compile .toVector .assertEquals(sizes) + + test.replicateA(repetitions).void } test("write - concurrent calls do not cause a WritePendingException") { val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) - Stream + val test: IO[Unit] = Stream .resource(setup) .flatMap { case (server, clients) => val readOnlyServer = server.map(_.reads).parJoinUnbounded @@ -335,10 +325,12 @@ class TcpSocketSuite extends UringSuite { } .compile .drain + + test.replicateA(repetitions).void } test("addresses - should match across client and server sockets") { - Stream + val test: IO[Unit] = Stream .resource(setup) .flatMap { case (server, clients) => val serverSocketAddresses = server.evalMap { socket => @@ -361,6 +353,8 @@ class TcpSocketSuite extends UringSuite { } .compile .drain + + test.replicateA(repetitions).void } // TODO options test From 7e057db6bd16544892e3821ba33b7dd94e78e1e2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 15:55:04 +0000 Subject: [PATCH 110/200] Add better prints for debugging --- .../scala/fs2/io/uring/net/UringSocket.scala | 13 +++++++------ .../fs2/io/uring/net/UringSocketGroup.scala | 18 +++++++++++++++--- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 6a4e7cb1..2b5ee33a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -55,17 +55,18 @@ private[net] final class UringSocket[F[_]: LiftIO]( def read(maxBytes: Int): F[Option[Chunk[Byte]]] = readMutex.lock.surround { for { - _ <- F.delay(buffer.clear()) + _ <- F.delay(buffer.clear()) // Clear the buffer before writing + _ <- F.delay(println(s"[SOCKET][READ] writing the received message in the buffer...")) readed <- recv(buffer.memoryAddress(), 0, maxBytes, 0) + _ <- F.delay(println(s"[SOCKET][READ] transfering the message from the buffer to a new array...")) bytes <- F.delay { val arr = new Array[Byte](readed) buffer.getBytes(0, arr) arr } - - _ <- F.delay(println(s"[SOCKET] reading the array: ${bytes}")) + _ <- F.delay(println(s"[SOCKET][READ] Done reading!")) } yield Option.when(readed > 0)(Chunk.array(bytes)) } @@ -110,13 +111,13 @@ private[net] final class UringSocket[F[_]: LiftIO]( writeMutex.lock .surround { for { + _ <- F.delay(println(s"[SOCKET][WRITE] transfering to the buffer the bytes...")) _ <- F.delay { buffer.clear() buffer.writeBytes(bytes.toArray) } - _ <- F.delay(println(s"[SOCKET] writing in array...")) - + _ <- F.delay(println(s"[SOCKET][WRITE] sending the bytes in the buffer...")) _ <- send( buffer.memoryAddress(), 0, @@ -124,7 +125,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( 0 // TODO Replace with MSG_NOSIGNAL ) - _ <- F.delay(println(s"[SOCKET] Message sent!")) + _ <- F.delay(println(s"[SOCKET][WRITE] message sent!")) } yield () } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 8014a8e6..9f08d06b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -62,10 +62,12 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) _ <- F.delay( println( - s"[CLIENT] connecting to address: ${address.toString()}, Buffer length: $length" + s"[CLIENT] Connecting to address: ${address + .toString()}, Buffer length: $length and LinuxSocket fd: ${linuxSocket.fd()}" ) ) - _ <- F.delay(println(s"[CLIENT] LinuxSocket fd: ${linuxSocket.fd()}")) + + _ <- F.delay(println("[CLIENT] Connecting...")) _ <- ring .call( op = IORING_OP_CONNECT, @@ -77,10 +79,11 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } yield () } ) - _ <- Resource.eval(F.delay(println("[CLIENT] connecting..."))) socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + _ <- Resource.eval(F.delay(println("[CLIENT] Connexion established!"))) + } yield socket def server( @@ -155,6 +158,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } } .mapK(LiftIO.liftK) + + _ <- Resource.eval(F.delay(println("[SERVER] connexion established!"))) + } yield clientFd // Read the address from the buf and convert it to SocketAddress @@ -193,6 +199,12 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn linuxSocket => closeSocket(ring, linuxSocket.fd()).to ) + /* + The bind, listen and getLocalAddress functions are in the LinuxSocket class. + If we open a socket using the ring, we only get a fd => Problem: how to bind and listen ? + - io_uring doesn't have a bind, listen or getLocalAddress operators. + - Is it possible to create a LinuxSocket using the fd created by the ring ? This would solve the problem. + */ private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { val domain = if (ipv6) AF_INET6 else AF_INET ring From 053cb86b4bfb7bd694b6f263e91a88cdee571467 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 17:21:18 +0000 Subject: [PATCH 111/200] Add logger for cats effect --- build.sbt | 5 ++++ .../scala/fs2/io/uring/logging/syntax.scala | 23 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala diff --git a/build.sbt b/build.sbt index a648edf5..56040c65 100644 --- a/build.sbt +++ b/build.sbt @@ -34,6 +34,8 @@ ThisBuild / githubWorkflowBuild += val ceVersion = "3.6-0142603" val fs2Version = "3.7.0" val munitCEVersion = "2.0.0-M3" +val log4catsVersion = "2.4.0" +val slf4jVersion = "2.0.0" ThisBuild / nativeConfig ~= { c => val arch = System.getProperty("os.arch").toLowerCase() @@ -53,6 +55,9 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) libraryDependencies ++= Seq( "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, + "org.typelevel" %% "log4cats-slf4j" % log4catsVersion, + "org.slf4j" % "slf4j-simple" % slf4jVersion, + "org.typelevel" %% "log4cats-noop" % log4catsVersion % Test, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") .classifier("linux-x86_64") diff --git a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala new file mode 100644 index 00000000..92cd90ff --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala @@ -0,0 +1,23 @@ +package fs2.io.uring.logging + +import org.typelevel.log4cats.Logger +import cats._ +import cats.implicits._ + +object syntax { + + implicit class LogSyntax[F[_], E, A](fa: F[A])(implicit me: MonadError[F, E], logger: Logger[F]) { + + def log(success: A => String, error: E => String): F[A] = fa.attemptTap { + case Left(e) => logger.error(error(e)) + case Right(a) => logger.info(success(a)) + } + + def logError(error: E => String): F[A] = fa.attemptTap { + case Left(e) => logger.error(error(e)) + case Right(_) => ().pure[F] + } + + } + +} From 40bc403e75fae8cf8e6579128da8ea305095c215 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 17:42:22 +0000 Subject: [PATCH 112/200] Add logger for cats effect --- .../main/scala/fs2/io/uring/logging/syntax.scala | 15 +++++++++++++++ .../scala/fs2/io/uring/net/UringNetwork.scala | 3 ++- .../scala/fs2/io/uring/net/UringSocketGroup.scala | 14 ++++++++++---- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 8 ++++++-- 4 files changed, 33 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala index 92cd90ff..3c216e31 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala @@ -3,6 +3,7 @@ package fs2.io.uring.logging import org.typelevel.log4cats.Logger import cats._ import cats.implicits._ +import cats.effect.kernel.Resource object syntax { @@ -20,4 +21,18 @@ object syntax { } + implicit class LogSyntaxForResource[F[_], E, A](r: Resource[F, A])(implicit me: MonadError[F, E], logger: Logger[F]) { + + def log(success: A => String, error: E => String): Resource[F, A] = r.attemptTap { + case Left(e) => Resource.eval(logger.error(error(e))) + case Right(a) => Resource.eval(logger.info(success(a))) + } + + def logError(error: E => String): Resource[F, A] = r.attemptTap { + case Left(e) => Resource.eval(logger.error(error(e))) + case Right(a) => Resource.pure[F,A](a) + } + + } + } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala index 1740a337..7ea120f4 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -39,6 +39,7 @@ import fs2.io.net.DatagramSocketOption import java.net.ProtocolFamily import java.util.concurrent.ThreadFactory +import org.typelevel.log4cats.Logger private[net] final class UringNetwork[F[_]]( sg: UringSocketGroup[F], @@ -78,7 +79,7 @@ private[net] final class UringNetwork[F[_]]( } object UringNetwork { - def apply[F[_]: Async: Dns: LiftIO]: Network[F] = + def apply[F[_]: Async: Dns: LiftIO: Logger]: Network[F] = new UringNetwork( new UringSocketGroup[F], new UringDatagramSocketGroup[F], diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 9f08d06b..2fc5f8d5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -38,9 +38,13 @@ import io.netty.incubator.channel.uring.UringSockaddrIn._ import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess._ +import org.typelevel.log4cats.Logger + +import fs2.io.uring.logging.syntax._ + import java.net.InetSocketAddress -private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) +private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = @@ -48,7 +52,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = for { - ring <- Resource.eval(Uring.get[F]) + ring <- Resource.eval(Uring.get[F].logError(e => s"got the error: $e")).log(uring => "WE GOT THE URING!!", e => s"We got an error: $e") + + _ <- Resource.eval(Logger[F].info("Trying to resolve address...")) address <- Resource.eval(to.resolve) @@ -139,7 +145,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufLength.writeInt( buf.capacity() - ) // TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt + ) // TODO: Moved to accept, wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = for { @@ -220,5 +226,5 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } object UringSocketGroup { - def apply[F[_]: Async: Dns: LiftIO]: SocketGroup[F] = new UringSocketGroup + def apply[F[_]: Async: Dns: LiftIO: Logger]: SocketGroup[F] = new UringSocketGroup } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 61c748da..7c6e4e59 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -31,8 +31,12 @@ import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException +import org.typelevel.log4cats.Logger +import org.typelevel.log4cats.slf4j.Slf4jLogger -class TcpSocketSuite extends UringSuite { +class TcpSocketSuit extends UringSuite { + + implicit val logger: Logger[IO] = Slf4jLogger.getLogger[IO] val sg = UringSocketGroup[IO] @@ -230,7 +234,7 @@ class TcpSocketSuite extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - val repetitions: Int = 5 + val repetitions: Int = 1 test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) From 30c1e5f156c7e17497b4b0a1464e53d37d1a1250 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 8 Aug 2023 20:01:31 +0200 Subject: [PATCH 113/200] Revert "Add logger for cats effect" This reverts commit 40bc403e75fae8cf8e6579128da8ea305095c215. --- .../main/scala/fs2/io/uring/logging/syntax.scala | 15 --------------- .../scala/fs2/io/uring/net/UringNetwork.scala | 3 +-- .../scala/fs2/io/uring/net/UringSocketGroup.scala | 14 ++++---------- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 8 ++------ 4 files changed, 7 insertions(+), 33 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala index 3c216e31..92cd90ff 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala @@ -3,7 +3,6 @@ package fs2.io.uring.logging import org.typelevel.log4cats.Logger import cats._ import cats.implicits._ -import cats.effect.kernel.Resource object syntax { @@ -21,18 +20,4 @@ object syntax { } - implicit class LogSyntaxForResource[F[_], E, A](r: Resource[F, A])(implicit me: MonadError[F, E], logger: Logger[F]) { - - def log(success: A => String, error: E => String): Resource[F, A] = r.attemptTap { - case Left(e) => Resource.eval(logger.error(error(e))) - case Right(a) => Resource.eval(logger.info(success(a))) - } - - def logError(error: E => String): Resource[F, A] = r.attemptTap { - case Left(e) => Resource.eval(logger.error(error(e))) - case Right(a) => Resource.pure[F,A](a) - } - - } - } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala index 7ea120f4..1740a337 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -39,7 +39,6 @@ import fs2.io.net.DatagramSocketOption import java.net.ProtocolFamily import java.util.concurrent.ThreadFactory -import org.typelevel.log4cats.Logger private[net] final class UringNetwork[F[_]]( sg: UringSocketGroup[F], @@ -79,7 +78,7 @@ private[net] final class UringNetwork[F[_]]( } object UringNetwork { - def apply[F[_]: Async: Dns: LiftIO: Logger]: Network[F] = + def apply[F[_]: Async: Dns: LiftIO]: Network[F] = new UringNetwork( new UringSocketGroup[F], new UringDatagramSocketGroup[F], diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 2fc5f8d5..9f08d06b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -38,13 +38,9 @@ import io.netty.incubator.channel.uring.UringSockaddrIn._ import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess._ -import org.typelevel.log4cats.Logger - -import fs2.io.uring.logging.syntax._ - import java.net.InetSocketAddress -private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], dns: Dns[F]) +private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = @@ -52,9 +48,7 @@ private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = for { - ring <- Resource.eval(Uring.get[F].logError(e => s"got the error: $e")).log(uring => "WE GOT THE URING!!", e => s"We got an error: $e") - - _ <- Resource.eval(Logger[F].info("Trying to resolve address...")) + ring <- Resource.eval(Uring.get[F]) address <- Resource.eval(to.resolve) @@ -145,7 +139,7 @@ private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], bufLength.writeInt( buf.capacity() - ) // TODO: Moved to accept, wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt + ) // TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = for { @@ -226,5 +220,5 @@ private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], } object UringSocketGroup { - def apply[F[_]: Async: Dns: LiftIO: Logger]: SocketGroup[F] = new UringSocketGroup + def apply[F[_]: Async: Dns: LiftIO]: SocketGroup[F] = new UringSocketGroup } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 7c6e4e59..61c748da 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -31,12 +31,8 @@ import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException -import org.typelevel.log4cats.Logger -import org.typelevel.log4cats.slf4j.Slf4jLogger -class TcpSocketSuit extends UringSuite { - - implicit val logger: Logger[IO] = Slf4jLogger.getLogger[IO] +class TcpSocketSuite extends UringSuite { val sg = UringSocketGroup[IO] @@ -234,7 +230,7 @@ class TcpSocketSuit extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - val repetitions: Int = 1 + val repetitions: Int = 5 test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) From ce99d4dceaf222f5fb5e61930d9ca77d8cf2b7c2 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 8 Aug 2023 20:01:53 +0200 Subject: [PATCH 114/200] Revert "Revert "Add logger for cats effect"" This reverts commit 30c1e5f156c7e17497b4b0a1464e53d37d1a1250. --- .../main/scala/fs2/io/uring/logging/syntax.scala | 15 +++++++++++++++ .../scala/fs2/io/uring/net/UringNetwork.scala | 3 ++- .../scala/fs2/io/uring/net/UringSocketGroup.scala | 14 ++++++++++---- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 8 ++++++-- 4 files changed, 33 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala index 92cd90ff..3c216e31 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala @@ -3,6 +3,7 @@ package fs2.io.uring.logging import org.typelevel.log4cats.Logger import cats._ import cats.implicits._ +import cats.effect.kernel.Resource object syntax { @@ -20,4 +21,18 @@ object syntax { } + implicit class LogSyntaxForResource[F[_], E, A](r: Resource[F, A])(implicit me: MonadError[F, E], logger: Logger[F]) { + + def log(success: A => String, error: E => String): Resource[F, A] = r.attemptTap { + case Left(e) => Resource.eval(logger.error(error(e))) + case Right(a) => Resource.eval(logger.info(success(a))) + } + + def logError(error: E => String): Resource[F, A] = r.attemptTap { + case Left(e) => Resource.eval(logger.error(error(e))) + case Right(a) => Resource.pure[F,A](a) + } + + } + } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala index 1740a337..7ea120f4 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -39,6 +39,7 @@ import fs2.io.net.DatagramSocketOption import java.net.ProtocolFamily import java.util.concurrent.ThreadFactory +import org.typelevel.log4cats.Logger private[net] final class UringNetwork[F[_]]( sg: UringSocketGroup[F], @@ -78,7 +79,7 @@ private[net] final class UringNetwork[F[_]]( } object UringNetwork { - def apply[F[_]: Async: Dns: LiftIO]: Network[F] = + def apply[F[_]: Async: Dns: LiftIO: Logger]: Network[F] = new UringNetwork( new UringSocketGroup[F], new UringDatagramSocketGroup[F], diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 9f08d06b..2fc5f8d5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -38,9 +38,13 @@ import io.netty.incubator.channel.uring.UringSockaddrIn._ import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess._ +import org.typelevel.log4cats.Logger + +import fs2.io.uring.logging.syntax._ + import java.net.InetSocketAddress -private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) +private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = @@ -48,7 +52,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = for { - ring <- Resource.eval(Uring.get[F]) + ring <- Resource.eval(Uring.get[F].logError(e => s"got the error: $e")).log(uring => "WE GOT THE URING!!", e => s"We got an error: $e") + + _ <- Resource.eval(Logger[F].info("Trying to resolve address...")) address <- Resource.eval(to.resolve) @@ -139,7 +145,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufLength.writeInt( buf.capacity() - ) // TODO: Moved to accept wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt + ) // TODO: Moved to accept, wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = for { @@ -220,5 +226,5 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } object UringSocketGroup { - def apply[F[_]: Async: Dns: LiftIO]: SocketGroup[F] = new UringSocketGroup + def apply[F[_]: Async: Dns: LiftIO: Logger]: SocketGroup[F] = new UringSocketGroup } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 61c748da..7c6e4e59 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -31,8 +31,12 @@ import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException +import org.typelevel.log4cats.Logger +import org.typelevel.log4cats.slf4j.Slf4jLogger -class TcpSocketSuite extends UringSuite { +class TcpSocketSuit extends UringSuite { + + implicit val logger: Logger[IO] = Slf4jLogger.getLogger[IO] val sg = UringSocketGroup[IO] @@ -230,7 +234,7 @@ class TcpSocketSuite extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - val repetitions: Int = 5 + val repetitions: Int = 1 test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) From 89cac943f79868535ce5620be7acaec01116265c Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 8 Aug 2023 18:10:31 +0000 Subject: [PATCH 115/200] Remove logger --- build.sbt | 5 --- .../scala/fs2/io/uring/logging/syntax.scala | 38 ------------------- .../scala/fs2/io/uring/net/UringNetwork.scala | 3 +- .../fs2/io/uring/net/UringSocketGroup.scala | 12 ++---- .../fs2/io/uring/net/TcpSocketSuite.scala | 4 -- 5 files changed, 4 insertions(+), 58 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala diff --git a/build.sbt b/build.sbt index 56040c65..a648edf5 100644 --- a/build.sbt +++ b/build.sbt @@ -34,8 +34,6 @@ ThisBuild / githubWorkflowBuild += val ceVersion = "3.6-0142603" val fs2Version = "3.7.0" val munitCEVersion = "2.0.0-M3" -val log4catsVersion = "2.4.0" -val slf4jVersion = "2.0.0" ThisBuild / nativeConfig ~= { c => val arch = System.getProperty("os.arch").toLowerCase() @@ -55,9 +53,6 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) libraryDependencies ++= Seq( "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, - "org.typelevel" %% "log4cats-slf4j" % log4catsVersion, - "org.slf4j" % "slf4j-simple" % slf4jVersion, - "org.typelevel" %% "log4cats-noop" % log4catsVersion % Test, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") .classifier("linux-x86_64") diff --git a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala b/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala deleted file mode 100644 index 3c216e31..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/logging/syntax.scala +++ /dev/null @@ -1,38 +0,0 @@ -package fs2.io.uring.logging - -import org.typelevel.log4cats.Logger -import cats._ -import cats.implicits._ -import cats.effect.kernel.Resource - -object syntax { - - implicit class LogSyntax[F[_], E, A](fa: F[A])(implicit me: MonadError[F, E], logger: Logger[F]) { - - def log(success: A => String, error: E => String): F[A] = fa.attemptTap { - case Left(e) => logger.error(error(e)) - case Right(a) => logger.info(success(a)) - } - - def logError(error: E => String): F[A] = fa.attemptTap { - case Left(e) => logger.error(error(e)) - case Right(_) => ().pure[F] - } - - } - - implicit class LogSyntaxForResource[F[_], E, A](r: Resource[F, A])(implicit me: MonadError[F, E], logger: Logger[F]) { - - def log(success: A => String, error: E => String): Resource[F, A] = r.attemptTap { - case Left(e) => Resource.eval(logger.error(error(e))) - case Right(a) => Resource.eval(logger.info(success(a))) - } - - def logError(error: E => String): Resource[F, A] = r.attemptTap { - case Left(e) => Resource.eval(logger.error(error(e))) - case Right(a) => Resource.pure[F,A](a) - } - - } - -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala index 7ea120f4..1740a337 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -39,7 +39,6 @@ import fs2.io.net.DatagramSocketOption import java.net.ProtocolFamily import java.util.concurrent.ThreadFactory -import org.typelevel.log4cats.Logger private[net] final class UringNetwork[F[_]]( sg: UringSocketGroup[F], @@ -79,7 +78,7 @@ private[net] final class UringNetwork[F[_]]( } object UringNetwork { - def apply[F[_]: Async: Dns: LiftIO: Logger]: Network[F] = + def apply[F[_]: Async: Dns: LiftIO]: Network[F] = new UringNetwork( new UringSocketGroup[F], new UringDatagramSocketGroup[F], diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 2fc5f8d5..b800a08c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -38,13 +38,9 @@ import io.netty.incubator.channel.uring.UringSockaddrIn._ import io.netty.incubator.channel.uring.UringLinuxSocket import io.netty.incubator.channel.uring.NativeAccess._ -import org.typelevel.log4cats.Logger - -import fs2.io.uring.logging.syntax._ - import java.net.InetSocketAddress -private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], dns: Dns[F]) +private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = @@ -52,9 +48,7 @@ private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = for { - ring <- Resource.eval(Uring.get[F].logError(e => s"got the error: $e")).log(uring => "WE GOT THE URING!!", e => s"We got an error: $e") - - _ <- Resource.eval(Logger[F].info("Trying to resolve address...")) + ring <- Resource.eval(Uring.get[F]) address <- Resource.eval(to.resolve) @@ -226,5 +220,5 @@ private final class UringSocketGroup[F[_]: LiftIO: Logger](implicit F: Async[F], } object UringSocketGroup { - def apply[F[_]: Async: Dns: LiftIO: Logger]: SocketGroup[F] = new UringSocketGroup + def apply[F[_]: Async: Dns: LiftIO]: SocketGroup[F] = new UringSocketGroup } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 7c6e4e59..d111d593 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -31,13 +31,9 @@ import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException -import org.typelevel.log4cats.Logger -import org.typelevel.log4cats.slf4j.Slf4jLogger class TcpSocketSuit extends UringSuite { - implicit val logger: Logger[IO] = Slf4jLogger.getLogger[IO] - val sg = UringSocketGroup[IO] // Client test: From b3e83b2f13df5d4258e47c6334a0d06d915f1814 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 9 Aug 2023 23:51:59 +0200 Subject: [PATCH 116/200] Add pipe for testing --- .../fs2/io/uring/unsafe/UringSystem.scala | 62 +++++++++++++------ 1 file changed, 44 insertions(+), 18 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 289df2c9..9d9597a8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -41,7 +41,8 @@ import java.io.IOException import scala.collection.mutable.Map import java.util.BitSet -// import org.slf4j.LoggerFactory +import java.nio.ByteBuffer +import java.nio.channels.Pipe object UringSystem extends PollingSystem { @@ -66,9 +67,13 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() - override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () - - def interrupt(poller: Poller, targetThread: Thread, targetPoller: Poller): Unit = ??? + override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { + val buffer: ByteBuffer = ByteBuffer.allocate(1) + buffer.put(0.toByte) + buffer.flip() + targetPoller.writeSink(buffer) + () + } private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit @@ -160,6 +165,10 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { + private[this] val interruptPipe: Pipe = Pipe.open() + private[this] val interruptSource: Pipe.SourceChannel = interruptPipe.source() + private[this] val interruptSink: Pipe.SinkChannel = interruptPipe.sink() + private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() @@ -188,6 +197,8 @@ object UringSystem extends PollingSystem { } .isDefined + private[UringSystem] def writeSink(buffer: ByteBuffer): Int = interruptSink.write(buffer) + private[UringSystem] def getId(cb: Either[Throwable, Int] => Unit): Short = { val id: Short = getUniqueId() @@ -212,7 +223,8 @@ object UringSystem extends PollingSystem { ): Boolean = { if (debug) println( - s"[SQ] Enqueuing a new Sqe in ringFd: ${ring.fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + s"[SQ] Enqueuing a new Sqe in ringFd: ${ring + .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" ) sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } @@ -225,7 +237,11 @@ object UringSystem extends PollingSystem { private[UringSystem] def getFd(): Int = ring.fd() - private[UringSystem] def close(): Unit = ring.close() + private[UringSystem] def close(): Unit = { + interruptSource.close() + interruptSink.close() + ring.close() + } private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty @@ -243,7 +259,8 @@ object UringSystem extends PollingSystem { cb( Left( new IOException( - s"Error in completion queue entry of the ring with fd: ${ring.fd()} with fd: $fd op: $op res: $res and data: $data" + s"Error in completion queue entry of the ring with fd: ${ring + .fd()} with fd: $fd op: $op res: $res and data: $data" ) ) ) @@ -252,7 +269,9 @@ object UringSystem extends PollingSystem { if ( op != 11 && debug ) // To prevent the constant printouts of timeout operation when NANOS == -1 - println(s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data") + println( + s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + ) callbacks.get(data).foreach { cb => handleCallback(res, cb) @@ -275,16 +294,23 @@ object UringSystem extends PollingSystem { submitted } - println(s"POLLING! with nanos: $nanos") - - nanos match { - case -1 => - if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(-1, true) - case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false - case _ => - if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(nanos, false) + if (debug) println(s"POLLING! with nanos: $nanos") + + val interruptBuffer: ByteBuffer = ByteBuffer.allocate(1) + if (interruptSource.read(interruptBuffer) > 0) { + // Data is available, it means an interrupt signal was sent + // Clear the data and return from poll or handle interrupt as needed + false + } else { + nanos match { + case -1 => + if (pendingSubmissions) handlePendingSubmissions(true) + else handleTimeoutAndQueue(-1, true) + case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false + case _ => + if (pendingSubmissions) handlePendingSubmissions(true) + else handleTimeoutAndQueue(nanos, false) + } } } From 95eacc8263cc2372fc322be79e55eaabfb16b93e Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Wed, 9 Aug 2023 22:51:53 +0000 Subject: [PATCH 117/200] Add prints for debugging interrupt --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 9d9597a8..ab6623aa 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -68,6 +68,7 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { + println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") val buffer: ByteBuffer = ByteBuffer.allocate(1) buffer.put(0.toByte) buffer.flip() @@ -281,25 +282,30 @@ object UringSystem extends PollingSystem { } def handlePendingSubmissions(submitAndWait: Boolean): Boolean = { + if (submitAndWait) println("[HANDLE PENDING SUMBISSION] Submiting and waiting...") + else println("[HANDLE PENDING SUMBISSION] Submiting...") val submitted = if (submitAndWait) sq.submitAndWait() > 0 else sq.submit() > 0 if (submitted) pendingSubmissions = false + println(s"[HANDLE PENDING SUBMISSION] submitted a positive number of operations: $submitted") submitted } def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { + println(s"[HANDLE TIMEOUT AND QUEUE] adding timeout: $nanos") sq.addTimeout(nanos, 0) val submitted = handlePendingSubmissions(submitAndWait) + println(s"[HANDLE TIMEOUT AND QUEUE] waiting CQE") cq.ioUringWaitCqe() + println(s"[HANDLE TIMEOUT AND QUEUE] processing CQ") process(completionQueueCallback) + println(s"[HANDLE TIMEOUT AND QUEUE] submitted a positive number of operations: $submitted") submitted } - if (debug) println(s"POLLING! with nanos: $nanos") - val interruptBuffer: ByteBuffer = ByteBuffer.allocate(1) if (interruptSource.read(interruptBuffer) > 0) { - // Data is available, it means an interrupt signal was sent - // Clear the data and return from poll or handle interrupt as needed + println("INTERRUPTED!") + interruptBuffer.clear() false } else { nanos match { From 80a39a91800163d776da5dd5e5d93917e57d6a0e Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 11 Aug 2023 18:04:24 +0000 Subject: [PATCH 118/200] Simplify Poll --- .../fs2/io/uring/unsafe/UringSystem.scala | 83 ++++++------------- .../test/scala/fs2/io/uring/UringSuite.scala | 20 ++--- 2 files changed, 37 insertions(+), 66 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index ab6623aa..743e9121 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -41,11 +41,10 @@ import java.io.IOException import scala.collection.mutable.Map import java.util.BitSet -import java.nio.ByteBuffer -import java.nio.channels.Pipe - object UringSystem extends PollingSystem { + private val extraPoller: Poller = new Poller(UringRing()) + private final val MaxEvents = 64 private val debug = true // True to printout operations @@ -69,10 +68,8 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") - val buffer: ByteBuffer = ByteBuffer.allocate(1) - buffer.put(0.toByte) - buffer.flip() - targetPoller.writeSink(buffer) + println(s"[INTERRUPT current thread: ${Thread.currentThread().getName()}]") + extraPoller.sendMsgRing(0, targetPoller.getFd()) () } @@ -166,10 +163,6 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { - private[this] val interruptPipe: Pipe = Pipe.open() - private[this] val interruptSource: Pipe.SourceChannel = interruptPipe.source() - private[this] val interruptSink: Pipe.SinkChannel = interruptPipe.sink() - private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() @@ -179,7 +172,7 @@ object UringSystem extends PollingSystem { private[this] val ids = new BitSet(Short.MaxValue) private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(1) + val newId = ids.nextClearBit(0) ids.set(newId) newId.toShort } @@ -198,8 +191,6 @@ object UringSystem extends PollingSystem { } .isDefined - private[UringSystem] def writeSink(buffer: ByteBuffer): Int = interruptSink.write(buffer) - private[UringSystem] def getId(cb: Either[Throwable, Int] => Unit): Short = { val id: Short = getUniqueId() @@ -233,16 +224,16 @@ object UringSystem extends PollingSystem { private[UringSystem] def cancel(opToCancel: Long, id: Short): Boolean = enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) - private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = - enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) + private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { + println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") + enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, getUniqueId()) + sq.submit() > 0 + } private[UringSystem] def getFd(): Int = ring.fd() - private[UringSystem] def close(): Unit = { - interruptSource.close() - interruptSink.close() + private[UringSystem] def close(): Unit = ring.close() - } private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty @@ -267,9 +258,7 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - if ( - op != 11 && debug - ) // To prevent the constant printouts of timeout operation when NANOS == -1 + if (debug) // To prevent the constant printouts of timeout operation when NANOS == -1 println( s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) @@ -281,43 +270,25 @@ object UringSystem extends PollingSystem { } } - def handlePendingSubmissions(submitAndWait: Boolean): Boolean = { - if (submitAndWait) println("[HANDLE PENDING SUMBISSION] Submiting and waiting...") - else println("[HANDLE PENDING SUMBISSION] Submiting...") - val submitted = if (submitAndWait) sq.submitAndWait() > 0 else sq.submit() > 0 - if (submitted) pendingSubmissions = false - println(s"[HANDLE PENDING SUBMISSION] submitted a positive number of operations: $submitted") - submitted + // 1. Submit pending operations if any + if (pendingSubmissions) { + sq.submit() + pendingSubmissions = false } - def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { - println(s"[HANDLE TIMEOUT AND QUEUE] adding timeout: $nanos") - sq.addTimeout(nanos, 0) - val submitted = handlePendingSubmissions(submitAndWait) - println(s"[HANDLE TIMEOUT AND QUEUE] waiting CQE") - cq.ioUringWaitCqe() - println(s"[HANDLE TIMEOUT AND QUEUE] processing CQ") - process(completionQueueCallback) - println(s"[HANDLE TIMEOUT AND QUEUE] submitted a positive number of operations: $submitted") - submitted + // 2. Check for events based on nanos value + nanos match { + case -1 => + cq.ioUringWaitCqe() + case 0 => + // do nothing, just check without waiting + case _ => + sq.addTimeout(nanos, getUniqueId()) + cq.ioUringWaitCqe() } - val interruptBuffer: ByteBuffer = ByteBuffer.allocate(1) - if (interruptSource.read(interruptBuffer) > 0) { - println("INTERRUPTED!") - interruptBuffer.clear() - false - } else { - nanos match { - case -1 => - if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(-1, true) - case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false - case _ => - if (pendingSubmissions) handlePendingSubmissions(true) - else handleTimeoutAndQueue(nanos, false) - } - } + // 3. Process the events + process(completionQueueCallback) } } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index ed7e3739..55fb008b 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -25,16 +25,16 @@ import scala.concurrent.duration._ abstract class UringSuite extends CatsEffectSuite { - override lazy val munitIORuntime = { - val (pool, poller, shutdown) = - IORuntime.createWorkStealingComputeThreadPool(threads = 1, pollingSystem = UringSystem) - IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() - } + // override lazy val munitIORuntime = { + // val (pool, poller, shutdown) = + // IORuntime.createWorkStealingComputeThreadPool(threads = 1, pollingSystem = UringSystem) + // IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() + // } - // override lazy val munitIORuntime = - // IORuntimeBuilder() - // .setPollingSystem(UringSystem) - // .build() + override lazy val munitIORuntime = + IORuntimeBuilder() + .setPollingSystem(UringSystem) + .build() - override def munitIOTimeout: Duration = 1000000.second + override def munitIOTimeout: Duration = 6.second } From b8433bb24edf9cbcf6949d5405edcd5c17490311 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 11 Aug 2023 18:04:40 +0000 Subject: [PATCH 119/200] Add print in submission test --- .../scala/fs2/io/uring/UringSystemSuite.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 2da8958d..f29014b5 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -23,12 +23,13 @@ import fs2.io.uring.UringSuite import fs2.io.uring.unsafe.util.OP._ -class UringSystemSuite extends UringSuite { +class UringSystemSuit extends UringSuite { test("submission") { - Uring - .get[IO] - .flatMap { ring => + val test = for { + ring <- Uring.get[IO] + _ <- IO.println("[TEST] We got the ring!") + res <- { val op: Byte = IORING_OP_NOP val flags: Int = 0 val rwFlags: Int = 0 @@ -39,7 +40,9 @@ class UringSystemSuite extends UringSuite { ring.call(op, flags, rwFlags, fd, bufferAddress, length, offset) } - .assertEquals(0) + } yield res + + test.assertEquals(0) } test("Parallel submission") { @@ -52,7 +55,7 @@ class UringSystemSuite extends UringSuite { val length: Int = 0 val offset: Long = 0 - val calls: List[IO[Int]] = List.fill(300)( + val calls: List[IO[Int]] = List.fill(100)( Uring .get[IO] .flatMap { ring => From 1d9bfaf27f92f9b31c891a2e15ab75d749ce662e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 11 Aug 2023 20:07:06 +0200 Subject: [PATCH 120/200] Add JNA --- build.sbt | 2 + .../scala/fs2/io/uring/JnaPlayground.scala | 56 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala diff --git a/build.sbt b/build.sbt index a648edf5..a1903c5b 100644 --- a/build.sbt +++ b/build.sbt @@ -34,6 +34,7 @@ ThisBuild / githubWorkflowBuild += val ceVersion = "3.6-0142603" val fs2Version = "3.7.0" val munitCEVersion = "2.0.0-M3" +val jnaVersion = "5.8.0" ThisBuild / nativeConfig ~= { c => val arch = System.getProperty("os.arch").toLowerCase() @@ -54,6 +55,7 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, + "net.java.dev.jna" % "jna" % jnaVersion, // JNA to import C code to java ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") .classifier("linux-x86_64") ), diff --git a/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala b/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala new file mode 100644 index 00000000..c5e058b7 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala @@ -0,0 +1,56 @@ +package fs2.io.uring + +import com.sun.jna.Library +import com.sun.jna.Native +import com.sun.jna.Pointer +import com.sun.jna.Memory + +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ + +trait CLibrary extends Library { + def printf(format: String): Int +} + +trait CPipe extends Library { + def pipe(pipefd: Array[Int]): Int + def read(fd: Int, buf: Pointer, count: Int): Int + def write(fd: Int, buf: Pointer, count: Int): Int + def close(fd: Int): Int +} + +object JnaPlayground extends App { + val printer: CLibrary = Native.load("c", classOf[CLibrary]) + printer.printf("Hello, JNA from Scala!") + + val libc: CPipe = Native.load("c", classOf[CPipe]) + val BUFFER_SIZE = 100 + val pipefd = new Array[Int](2) + + if (libc.pipe(pipefd) != 0) { + throw new RuntimeException("Pipe creation failed!") + } + + val reader = Future { + val buffer: Pointer = new Memory(BUFFER_SIZE) + libc.read(pipefd(0), buffer, BUFFER_SIZE) + val message = buffer.getString(0) + libc.close(pipefd(0)) + message + } + + val writer = Future { + val message = "Hello from writer thread!" + val buffer: Pointer = new Memory(message.length + 1) + buffer.setString(0, message) + println("waiting 2 sec before writing...") + Thread.sleep(2000) + libc.write(pipefd(1), buffer, message.length) + libc.close(pipefd(1)) + } + + Await.ready(writer, 10.seconds) + println(s"Reader received: '${Await.result(reader, 10.seconds)}'") + +} From 6df58b53dc6da78ef2ce27c7aebe5c20587c526e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 11 Aug 2023 20:07:16 +0200 Subject: [PATCH 121/200] Create Pipe class --- .../main/scala/fs2/io/uring/unsafe/Pipe.scala | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala new file mode 100644 index 00000000..e6aec0c5 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala @@ -0,0 +1,36 @@ +package fs2.io.uring.unsafe + +import com.sun.jna.{Library, Native, Pointer} +import com.sun.jna.Memory + +class Pipe { + private val libc: CLibrary = Native.load("c", classOf[CLibrary]) + private val pipefd = Array.ofDim[Int](2) + + if (libc.pipe(pipefd) != 0) { + throw new RuntimeException("Pipe creation failed!") + } + + def write(message: String): Int = { + val buffer: Pointer = new Memory(message.length + 1) + buffer.setString(0, message) + val bytesWritten = libc.write(pipefd(1), buffer, message.length) + libc.close(pipefd(1)) + bytesWritten + } + + def read(bufferSize: Int = 100): String = { + val buffer: Pointer = new Memory(bufferSize) + libc.read(pipefd(0), buffer, bufferSize) + val message = buffer.getString(0) + libc.close(pipefd(0)) + message + } + + trait CLibrary extends Library { + def pipe(pipefd: Array[Int]): Int + def read(fd: Int, buf: Pointer, count: Int): Int + def write(fd: Int, buf: Pointer, count: Int): Int + def close(fd: Int): Int + } +} From 07891248de434e396a7b1dcbac9cb455e228f80f Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 11 Aug 2023 23:31:50 +0000 Subject: [PATCH 122/200] Add prints for cancel --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 9 ++++++- .../fs2/io/uring/unsafe/UringSystem.scala | 26 ++++++++++++++----- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 55c6609d..6c2ab7d5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -74,6 +74,13 @@ final class UringRing(private[this] val ringBuffer: RingBuffer) { */ def fd(): Int = ringBuffer.fd() + def sendMsgRing(flags: Int, fd: Int): Boolean = { + println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") + uringSubmissionQueue.enqueueSqe(40, flags, 0, fd, 0, 0, 0, 0) + uringSubmissionQueue.submit() + uringCompletionQueue.hasCompletions() + } + /** Closes the Ring, realising any associated resources. */ def close(): Unit = ringBuffer.close() @@ -622,7 +629,7 @@ object UringLinuxSocket { object UringSockaddrIn { val IPV6_ADDRESS_LENGTH = SockaddrIn.IPV6_ADDRESS_LENGTH - + val IPV4_ADDRESS_LENGTH = SockaddrIn.IPV4_ADDRESS_LENGTH def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 743e9121..cd21a4c7 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -43,7 +43,9 @@ import java.util.BitSet object UringSystem extends PollingSystem { - private val extraPoller: Poller = new Poller(UringRing()) + private val extraRing: UringRing = UringRing() + + private val extraPoller: Poller = makePoller() private final val MaxEvents = 64 @@ -68,9 +70,15 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") - println(s"[INTERRUPT current thread: ${Thread.currentThread().getName()}]") - extraPoller.sendMsgRing(0, targetPoller.getFd()) - () + println(s"[INTERRUPT] current thread: ${Thread.currentThread().getName()}]") + if (targetThread == Thread.currentThread()) { + println("WE ARE IN THE SAME THREAD AS THE POLLER!!!!!!!!!!!") + targetPoller.sendMsgRing(0, targetPoller.getFd()) + () + } else { + extraRing.sendMsgRing(0, targetPoller.getFd()) + () + } } private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { @@ -115,7 +123,8 @@ object UringSystem extends PollingSystem { IO.async_[Int] { cb => register { ring => val cancelId = ring.getId(cb) - val opToCancel = Encoder.encode(fd, op, id) + val opToCancel = Encoder.encode(fd, op, id - 1) + println(s"[CANCEL] cancel id: $cancelId and op to cancel: $opToCancel") ring.cancel(opToCancel, cancelId) () } @@ -172,7 +181,7 @@ object UringSystem extends PollingSystem { private[this] val ids = new BitSet(Short.MaxValue) private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(0) + val newId = ids.nextClearBit(1) ids.set(newId) newId.toShort } @@ -267,6 +276,7 @@ object UringSystem extends PollingSystem { handleCallback(res, cb) removeCallback(data) } + } } @@ -279,12 +289,14 @@ object UringSystem extends PollingSystem { // 2. Check for events based on nanos value nanos match { case -1 => + println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") cq.ioUringWaitCqe() case 0 => // do nothing, just check without waiting case _ => + println(s"[POLL] we are polling with nanos = $nanos") sq.addTimeout(nanos, getUniqueId()) - cq.ioUringWaitCqe() + sq.submitAndWait() } // 3. Process the events From 730ad950bce9552393a0496d7b779ce1a362046f Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sun, 13 Aug 2023 01:02:14 +0000 Subject: [PATCH 123/200] Debug PollingSystem with one thread --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 2 +- .../fs2/io/uring/unsafe/UringSystem.scala | 138 +++++++++++------- .../test/scala/fs2/io/uring/UringSuite.scala | 20 +-- 3 files changed, 93 insertions(+), 67 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 6c2ab7d5..b403b7fd 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -74,7 +74,7 @@ final class UringRing(private[this] val ringBuffer: RingBuffer) { */ def fd(): Int = ringBuffer.fd() - def sendMsgRing(flags: Int, fd: Int): Boolean = { + def sendMsgRing(flags: Int, fd: Int): Boolean = { // TODO bypass the submission queue println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") uringSubmissionQueue.enqueueSqe(40, flags, 0, fd, 0, 0, 0, 0) uringSubmissionQueue.submit() diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index cd21a4c7..4faaaac3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -45,8 +45,6 @@ object UringSystem extends PollingSystem { private val extraRing: UringRing = UringRing() - private val extraPoller: Poller = makePoller() - private final val MaxEvents = 64 private val debug = true // True to printout operations @@ -71,14 +69,8 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") println(s"[INTERRUPT] current thread: ${Thread.currentThread().getName()}]") - if (targetThread == Thread.currentThread()) { - println("WE ARE IN THE SAME THREAD AS THE POLLER!!!!!!!!!!!") - targetPoller.sendMsgRing(0, targetPoller.getFd()) - () - } else { - extraRing.sendMsgRing(0, targetPoller.getFd()) - () - } + extraRing.sendMsgRing(0, targetPoller.getFd()) // comment this line and change the number of threads in UringSuite to test 1 thread + () } private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { @@ -123,9 +115,11 @@ object UringSystem extends PollingSystem { IO.async_[Int] { cb => register { ring => val cancelId = ring.getId(cb) - val opToCancel = Encoder.encode(fd, op, id - 1) - println(s"[CANCEL] cancel id: $cancelId and op to cancel: $opToCancel") - ring.cancel(opToCancel, cancelId) + val opAddressToCancel = Encoder.encode(fd, op, id) + println( + s"[CANCEL] cancel id: $cancelId and op to cancel is in address: $opAddressToCancel" + ) + ring.cancel(opAddressToCancel, cancelId) () } } @@ -172,7 +166,7 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { - private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() + private[UringSystem] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() private[this] var pendingSubmissions: Boolean = false @@ -180,10 +174,15 @@ object UringSystem extends PollingSystem { Map.empty[Short, Either[Throwable, Int] => Unit] private[this] val ids = new BitSet(Short.MaxValue) + // var counter = 1 private[this] def getUniqueId(): Short = { val newId = ids.nextClearBit(1) ids.set(newId) newId.toShort + // val newId = counter + // counter = counter + 1 + // newId.toShort + } private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) @@ -230,77 +229,104 @@ object UringSystem extends PollingSystem { sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } - private[UringSystem] def cancel(opToCancel: Long, id: Short): Boolean = - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opToCancel, 0, 0, id) + private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { - println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") - enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, getUniqueId()) - sq.submit() > 0 - } + // private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { + // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") + // enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) + // submit() + // cq.ioUringWaitCqe() + // cq.process(completionQueueCallback) + // sq.submit() > 0 + // } private[UringSystem] def getFd(): Int = ring.fd() private[UringSystem] def close(): Unit = ring.close() - private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty + private[UringSystem] def submit(): Boolean = { + val submitted = sq.submit() + pendingSubmissions = false - private[UringSystem] def poll(nanos: Long): Boolean = { + submitted > 0 + } + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty - def process( - completionQueueCallback: UringCompletionQueueCallback - ): Boolean = - cq.process(completionQueueCallback) > 0 - - val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if (res < 0) - cb( - Left( - new IOException( - s"Error in completion queue entry of the ring with fd: ${ring - .fd()} with fd: $fd op: $op res: $res and data: $data" - ) + private[this] val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { + def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = + if (res < 0 ) + cb( + Left( + new IOException( + s"Error in completion queue entry of the ring with fd: ${ring + .fd()} with fd: $fd op: $op res: $res and data: $data" ) ) - else cb(Right(res)) - - if (debug) // To prevent the constant printouts of timeout operation when NANOS == -1 - println( - s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) + else cb(Right(res)) - callbacks.get(data).foreach { cb => - handleCallback(res, cb) - removeCallback(data) - } + if (debug) + println( + s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + ) + callbacks.get(data).foreach { cb => + handleCallback(res, cb) + removeCallback(data) } + } + } + + private[this] def process( + completionQueueCallback: UringCompletionQueueCallback + ): Boolean = + cq.process(completionQueueCallback) > 0 + + private[UringSystem] def poll(nanos: Long): Boolean = { // 1. Submit pending operations if any - if (pendingSubmissions) { - sq.submit() - pendingSubmissions = false - } + val submitted = submit() // 2. Check for events based on nanos value nanos match { case -1 => - println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") - cq.ioUringWaitCqe() + // println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") + if (submitted && !cq.hasCompletions()) { + cq.ioUringWaitCqe() + } else { + sq.addTimeout(1000000000, 0) + false + } case 0 => // do nothing, just check without waiting case _ => println(s"[POLL] we are polling with nanos = $nanos") - sq.addTimeout(nanos, getUniqueId()) - sq.submitAndWait() + + if (submitted && !cq.hasCompletions()) { + cq.ioUringWaitCqe() + } else { + println("[POLL] NOTHING WAS SUBMITTED") + sq.addTimeout(1000000000, 0) // + false + } } + // println(s"are you here? $nanos") + // if (sq.submit() <= 0) { + // sq.addTimeout(1000000000, 0) + // return false + // } + // println("WE SUBMITTED SOMETHING!!!") + // cq.ioUringWaitCqe() // 3. Process the events - process(completionQueueCallback) + val proc = process(completionQueueCallback) + println(s"[POLL] We processed cqe ? : $proc") + + proc } } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 55fb008b..7c929541 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -25,16 +25,16 @@ import scala.concurrent.duration._ abstract class UringSuite extends CatsEffectSuite { - // override lazy val munitIORuntime = { - // val (pool, poller, shutdown) = - // IORuntime.createWorkStealingComputeThreadPool(threads = 1, pollingSystem = UringSystem) - // IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() - // } + override lazy val munitIORuntime = { + val (pool, poller, shutdown) = + IORuntime.createWorkStealingComputeThreadPool(threads = 2, pollingSystem = UringSystem) + IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() + } - override lazy val munitIORuntime = - IORuntimeBuilder() - .setPollingSystem(UringSystem) - .build() + // override lazy val munitIORuntime = + // IORuntimeBuilder() + // .setPollingSystem(UringSystem) + // .build() - override def munitIOTimeout: Duration = 6.second + override def munitIOTimeout: Duration = 20.second } From 51367ce34ee7c0a29ed96f88d6035da427ba3b47 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 14 Aug 2023 01:47:00 +0200 Subject: [PATCH 124/200] Sketch interrupt using FileDescriptor --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 4 +- .../fs2/io/uring/unsafe/UringSystem.scala | 75 ++++++++++++------- 2 files changed, 51 insertions(+), 28 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index b403b7fd..8c58c1f9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -75,7 +75,7 @@ final class UringRing(private[this] val ringBuffer: RingBuffer) { def fd(): Int = ringBuffer.fd() def sendMsgRing(flags: Int, fd: Int): Boolean = { // TODO bypass the submission queue - println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") + // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") uringSubmissionQueue.enqueueSqe(40, flags, 0, fd, 0, 0, 0, 0) uringSubmissionQueue.submit() uringCompletionQueue.hasCompletions() @@ -428,6 +428,8 @@ object NativeAccess { val SOCK_STREAM = 1 + val POLLIN = Native.POLLIN + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 4faaaac3..270786cd 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -40,6 +40,9 @@ import java.io.IOException import scala.collection.mutable.Map import java.util.BitSet +import io.netty.channel.unix.FileDescriptor +import java.nio.ByteBuffer +import io.netty.incubator.channel.uring.NativeAccess object UringSystem extends PollingSystem { @@ -47,7 +50,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = true // True to printout operations + private val debug = false // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -67,9 +70,13 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { - println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") - println(s"[INTERRUPT] current thread: ${Thread.currentThread().getName()}]") - extraRing.sendMsgRing(0, targetPoller.getFd()) // comment this line and change the number of threads in UringSuite to test 1 thread + if (debug) println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") + if (debug) println(s"[INTERRUPT] current thread: ${Thread.currentThread().getName()}]") + // extraRing.sendMsgRing( + // 0, + // targetPoller.getFd() + // ) // comment this line and change the number of threads in UringSuite to test 1 thread + targetPoller.writeFd() () } @@ -166,7 +173,11 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { - private[UringSystem] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() + val interruptFd = FileDescriptor.pipe() + val readEnd = interruptFd(0) + val writeEnd = interruptFd(1) + + private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() private[this] var pendingSubmissions: Boolean = false @@ -174,15 +185,10 @@ object UringSystem extends PollingSystem { Map.empty[Short, Either[Throwable, Int] => Unit] private[this] val ids = new BitSet(Short.MaxValue) - // var counter = 1 private[this] def getUniqueId(): Short = { val newId = ids.nextClearBit(1) ids.set(newId) newId.toShort - // val newId = counter - // counter = counter + 1 - // newId.toShort - } private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) @@ -221,7 +227,7 @@ object UringSystem extends PollingSystem { offset: Long, data: Short ): Boolean = { - if (debug) + if (debug || op == 14) println( s"[SQ] Enqueuing a new Sqe in ringFd: ${ring .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" @@ -257,7 +263,7 @@ object UringSystem extends PollingSystem { private[this] val completionQueueCallback = new UringCompletionQueueCallback { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if (res < 0 ) + if (res < 0) cb( Left( new IOException( @@ -286,6 +292,12 @@ object UringSystem extends PollingSystem { ): Boolean = cq.process(completionQueueCallback) > 0 + private[UringSystem] def writeFd(): Int = { + val buf = ByteBuffer.allocateDirect(1) + buf.put(0.toByte) + buf.flip() + writeEnd.write(buf, 0, 1) + } private[UringSystem] def poll(nanos: Long): Boolean = { // 1. Submit pending operations if any @@ -294,37 +306,46 @@ object UringSystem extends PollingSystem { // 2. Check for events based on nanos value nanos match { case -1 => - // println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") + if (debug) println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") if (submitted && !cq.hasCompletions()) { + if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") cq.ioUringWaitCqe() } else { - sq.addTimeout(1000000000, 0) - false + // sq.addTimeout(0, 0)// replace 1 sec with 0 + enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + submit() + cq.ioUringWaitCqe() + + val buf = ByteBuffer.allocateDirect(1) + readEnd.read(buf, 0, 1) } case 0 => // do nothing, just check without waiting case _ => - println(s"[POLL] we are polling with nanos = $nanos") + if (debug) println(s"[POLL] we are polling with nanos = $nanos") if (submitted && !cq.hasCompletions()) { + if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") cq.ioUringWaitCqe() + if (sq.count() > 0) sq.submit() + if (cq.hasCompletions()) { + process(completionQueueCallback) + } } else { - println("[POLL] NOTHING WAS SUBMITTED") - sq.addTimeout(1000000000, 0) // - false + // sq.addTimeout(0, 0)// replace 1 sec with 0 + enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + submit() + cq.ioUringWaitCqe() + + val buf = ByteBuffer.allocateDirect(1) + readEnd.read(buf, 0, 1) + // sq.addTimeout(nanos, 0) // } } - // println(s"are you here? $nanos") - // if (sq.submit() <= 0) { - // sq.addTimeout(1000000000, 0) - // return false - // } - // println("WE SUBMITTED SOMETHING!!!") - // cq.ioUringWaitCqe() // 3. Process the events val proc = process(completionQueueCallback) - println(s"[POLL] We processed cqe ? : $proc") + if (debug) println(s"[POLL] We processed cqe ? : $proc") proc } From 196753621328df98207690f8b30d5138137da56b Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 14 Aug 2023 02:16:46 +0200 Subject: [PATCH 125/200] Handle interrupt --- .../fs2/io/uring/unsafe/UringSystem.scala | 37 ++++++++++++++----- .../test/scala/fs2/io/uring/UringSuite.scala | 2 +- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 270786cd..40f3413b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -176,6 +176,7 @@ object UringSystem extends PollingSystem { val interruptFd = FileDescriptor.pipe() val readEnd = interruptFd(0) val writeEnd = interruptFd(1) + var wakingUp: Boolean = false private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() @@ -249,8 +250,11 @@ object UringSystem extends PollingSystem { private[UringSystem] def getFd(): Int = ring.fd() - private[UringSystem] def close(): Unit = + private[UringSystem] def close(): Unit = { + readEnd.close() + writeEnd.close() ring.close() + } private[UringSystem] def submit(): Boolean = { val submitted = sq.submit() @@ -279,6 +283,16 @@ object UringSystem extends PollingSystem { s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) + if (fd == readEnd.intValue()) { + val buf = ByteBuffer.allocateDirect(1) + readEnd.read(buf, 0, 1) // we consume the fd + wakingUp = false // now we are not listening anymore to the fd + } + // if it is the operation handling the POLL_ADD (there was a write (interrupted)) + // val buf = ByteBuffer.allocateDirect(1) + // readEnd.read(buf, 0, 1) // we consume the fd + // wakingUp = false // now we are not listening anymore to the fd + callbacks.get(data).foreach { cb => handleCallback(res, cb) removeCallback(data) @@ -300,6 +314,11 @@ object UringSystem extends PollingSystem { } private[UringSystem] def poll(nanos: Long): Boolean = { + if (!wakingUp) { + enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + wakingUp = true + } + // 1. Submit pending operations if any val submitted = submit() @@ -312,12 +331,12 @@ object UringSystem extends PollingSystem { cq.ioUringWaitCqe() } else { // sq.addTimeout(0, 0)// replace 1 sec with 0 - enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - submit() + // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + // submit() cq.ioUringWaitCqe() - val buf = ByteBuffer.allocateDirect(1) - readEnd.read(buf, 0, 1) + // val buf = ByteBuffer.allocateDirect(1) + // readEnd.read(buf, 0, 1) } case 0 => // do nothing, just check without waiting @@ -333,12 +352,12 @@ object UringSystem extends PollingSystem { } } else { // sq.addTimeout(0, 0)// replace 1 sec with 0 - enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - submit() + // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + // submit() cq.ioUringWaitCqe() - val buf = ByteBuffer.allocateDirect(1) - readEnd.read(buf, 0, 1) + // val buf = ByteBuffer.allocateDirect(1) + // readEnd.read(buf, 0, 1) // sq.addTimeout(nanos, 0) // } } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 7c929541..8e159831 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -36,5 +36,5 @@ abstract class UringSuite extends CatsEffectSuite { // .setPollingSystem(UringSystem) // .build() - override def munitIOTimeout: Duration = 20.second + override def munitIOTimeout: Duration = 3.second } From 1a32946f567c9059e1cba48cd9d7f997d900b554 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 15 Aug 2023 00:14:07 +0000 Subject: [PATCH 126/200] Poll handles correctly UringSystemSuite --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 2 + .../fs2/io/uring/unsafe/UringSystem.scala | 230 +++++++++++++----- .../scala/fs2/io/uring/UringSystemSuite.scala | 30 ++- 3 files changed, 201 insertions(+), 61 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 8c58c1f9..7f98fb1a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -430,6 +430,8 @@ object NativeAccess { val POLLIN = Native.POLLIN + val POLLOUT = Native.POLLOUT + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 40f3413b..b5e4d883 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -70,8 +70,10 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { - if (debug) println(s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread") - if (debug) println(s"[INTERRUPT] current thread: ${Thread.currentThread().getName()}]") + if (debug) + println( + s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread current thread: ${Thread.currentThread().getName()}" + ) // extraRing.sendMsgRing( // 0, // targetPoller.getFd() @@ -173,6 +175,8 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { + val test: FileDescriptor = NativeAccess.newBlockingEventFd + val interruptFd = FileDescriptor.pipe() val readEnd = interruptFd(0) val writeEnd = interruptFd(1) @@ -227,14 +231,13 @@ object UringSystem extends PollingSystem { length: Int, offset: Long, data: Short - ): Boolean = { - if (debug || op == 14) - println( - s"[SQ] Enqueuing a new Sqe in ringFd: ${ring - .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" - ) + ): Boolean = + // if (debug || op == 14) + // println( + // s"[SQ] Enqueuing a new Sqe in ringFd: ${ring + // .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + // ) sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) - } private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) @@ -251,6 +254,7 @@ object UringSystem extends PollingSystem { private[UringSystem] def getFd(): Int = ring.fd() private[UringSystem] def close(): Unit = { + println("WE CLOSE EVERYTHING") readEnd.close() writeEnd.close() ring.close() @@ -278,21 +282,21 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - if (debug) - println( - s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" - ) + // if (debug) { + // println( + // s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + // ) + // } if (fd == readEnd.intValue()) { val buf = ByteBuffer.allocateDirect(1) - readEnd.read(buf, 0, 1) // we consume the fd - wakingUp = false // now we are not listening anymore to the fd + val readed = readEnd.read(buf, 0, 1) // we consume the fd + if (readed > 0) { + buf.clear() + wakingUp = false // We are not listening to the FD anymore + } } - // if it is the operation handling the POLL_ADD (there was a write (interrupted)) - // val buf = ByteBuffer.allocateDirect(1) - // readEnd.read(buf, 0, 1) // we consume the fd - // wakingUp = false // now we are not listening anymore to the fd - + callbacks.get(data).foreach { cb => handleCallback(res, cb) removeCallback(data) @@ -314,61 +318,171 @@ object UringSystem extends PollingSystem { } private[UringSystem] def poll(nanos: Long): Boolean = { + // Check if we are listening to the FD. If not, start listening if (!wakingUp) { - enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - wakingUp = true + enqueueSqe( + IORING_OP_POLL_ADD, + 0, + NativeAccess.POLLIN, + readEnd.intValue(), + 0, + 0, + 0, + NativeAccess.POLLIN.toShort + ) + sq.submit() + wakingUp = true // Set the flag indicating we're now listening } - // 1. Submit pending operations if any - val submitted = submit() - - // 2. Check for events based on nanos value + var rtn = -1 nanos match { case -1 => - if (debug) println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") - if (submitted && !cq.hasCompletions()) { - if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") - cq.ioUringWaitCqe() + if (pendingSubmissions) { + rtn = sq.submitAndWait() } else { - // sq.addTimeout(0, 0)// replace 1 sec with 0 - // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - // submit() cq.ioUringWaitCqe() - - // val buf = ByteBuffer.allocateDirect(1) - // readEnd.read(buf, 0, 1) } + case 0 => - // do nothing, just check without waiting - case _ => - if (debug) println(s"[POLL] we are polling with nanos = $nanos") + if (pendingSubmissions) { + rtn = sq.submit() + } - if (submitted && !cq.hasCompletions()) { - if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") - cq.ioUringWaitCqe() - if (sq.count() > 0) sq.submit() - if (cq.hasCompletions()) { - process(completionQueueCallback) - } + case _ => + if (pendingSubmissions) { + sq.addTimeout(nanos, 0) + rtn = sq.submitAndWait() } else { - // sq.addTimeout(0, 0)// replace 1 sec with 0 - // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - // submit() + sq.addTimeout(nanos, 0) + sq.submit() cq.ioUringWaitCqe() - - // val buf = ByteBuffer.allocateDirect(1) - // readEnd.read(buf, 0, 1) - // sq.addTimeout(nanos, 0) // } } - // 3. Process the events - val proc = process(completionQueueCallback) - if (debug) println(s"[POLL] We processed cqe ? : $proc") + val invokedCbs = process(completionQueueCallback) + + // If pending submissions exist and we encountered an error, retry submission. + while (pendingSubmissions && rtn < 0) + rtn = sq.submit() + + pendingSubmissions = false + + invokedCbs + + // if (!wakingUp) { + // enqueueSqe( + // IORING_OP_POLL_ADD, + // 0, + // NativeAccess.POLLIN, + // readEnd.intValue(), + // 0, + // 0, + // 0, + // NativeAccess.POLLIN.toShort + // ) + // sq.submit() + // wakingUp = true + // } + + // // 1. Submit pending operations if any + // val submitted = submit() + + // // 2. Check for events based on nanos value + // nanos match { + // case -1 => + // if (debug) println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") + // if (submitted && !cq.hasCompletions()) { + // // if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") + // cq.ioUringWaitCqe() + // } else { + // // sq.addTimeout(0, 0)// replace 1 sec with 0 + // // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + // // submit() + // cq.ioUringWaitCqe() + + // // val buf = ByteBuffer.allocateDirect(1) + // // readEnd.read(buf, 0, 1) + // } + // case 0 => + // // do nothing, just check without waiting + // case _ => + // if (debug) println(s"[POLL] we are polling with nanos = $nanos") + + // if (submitted) { + // // if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") + // cq.ioUringWaitCqe() + // // if (sq.count() > 0) sq.submit() + // // if (cq.hasCompletions()) { + // // process(completionQueueCallback) + // // } + // } else { + // // sq.addTimeout(0, 0)// replace 1 sec with 0 + // // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) + // // submit() + + // cq.ioUringWaitCqe() + + // // val buf = ByteBuffer.allocateDirect(1) + // // readEnd.read(buf, 0, 1) + // // sq.addTimeout(nanos, 0) // + // } + // } + + // // 3. Process the events + // val proc = process(completionQueueCallback) + // // if (debug) println(s"[POLL] We processed cqe ? : $proc") + + // proc + + // if (!wakingUp) { + // enqueueSqe( + // IORING_OP_POLL_ADD, + // 0, + // NativeAccess.POLLIN, + // readEnd.intValue(), + // 0, + // 0, + // 0, + // NativeAccess.POLLIN.toShort + // ) + // sq.submit() // we start listening, it will be completed only when we call interrupt, therefore we don't want to submit and wait + // wakingUp = true // Now we are listening + // } + + // def handlePendingSubmissions(submitAndWait: Boolean): Boolean = { + // // if (submitAndWait) println("[HANDLE PENDING SUMBISSION] Submiting and waiting...") + // // else println("[HANDLE PENDING SUMBISSION] Submiting...") + // val submitted = if (submitAndWait) sq.submitAndWait() > 0 else sq.submit() > 0 + // if (submitted) pendingSubmissions = false + // // println( + // // s"[HANDLE PENDING SUBMISSION] submitted a positive number of operations: $submitted" + // // ) + // submitted + // } + + // def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { + // // println(s"[HANDLE TIMEOUT AND QUEUE] adding timeout: $nanos") + // sq.addTimeout(nanos, 0) + // val submitted = handlePendingSubmissions(submitAndWait) + // // println(s"[HANDLE TIMEOUT AND QUEUE] waiting CQE") + // cq.ioUringWaitCqe() + // // println(s"[HANDLE TIMEOUT AND QUEUE] processing CQ") + // process(completionQueueCallback) + // // println(s"[HANDLE TIMEOUT AND QUEUE] submitted a positive number of operations: $submitted") + // submitted + // } + + // nanos match { + // case -1 => + // if (pendingSubmissions) handlePendingSubmissions(true) + // else handleTimeoutAndQueue(-1, true) + // case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false + // case _ => + // if (pendingSubmissions) handlePendingSubmissions(true) + // else handleTimeoutAndQueue(nanos, false) + // } - proc } } - } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index f29014b5..818a5197 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -22,8 +22,18 @@ import cats.syntax.parallel._ import fs2.io.uring.UringSuite import fs2.io.uring.unsafe.util.OP._ +import scala.io.StdIn -class UringSystemSuit extends UringSuite { +class UringSystemSuitd extends UringSuite { + + test("Create a ring") { + val test = for { + _ <- Uring.get[IO] + _ <- IO.println("[TEST] We got the ring!") + } yield () + + test.assertEquals(()) + } test("submission") { val test = for { @@ -65,7 +75,14 @@ class UringSystemSuit extends UringSuite { val test: IO[List[Int]] = calls.parSequence - test.map(results => assert(results.forall(_ == 0))) + val list = for { + results <- test + _ <- IO.println(results) + _ <- IO.println(results.size) + } yield results + + + list.map(results => assert(results.forall(_ >= 0))) } test("Multiple parallel submission") { @@ -85,6 +102,13 @@ class UringSystemSuit extends UringSuite { val test: IO[List[List[Int]]] = calls.parSequence - test.map(listOfList => assert(listOfList.flatten.forall(_ == 0))) + val list = for { + listOfList <- test + listFlatten <- IO(listOfList.flatten) + _ <- IO.println(listFlatten) + _ <- IO.println(listFlatten.size) + } yield listFlatten + + list.map(results => assert(results.forall(_ >= 0))) } } From e91ab6fba1d192acfc4bf8109571a0c19ddda0a5 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 15 Aug 2023 18:27:51 +0200 Subject: [PATCH 127/200] Debug poll and TcpSocketSuite work --- .../fs2/io/uring/unsafe/UringSystem.scala | 95 ++++++++++++------- 1 file changed, 59 insertions(+), 36 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index b5e4d883..c331e595 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -50,7 +50,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false // True to printout operations + private val debug = true // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -74,10 +74,7 @@ object UringSystem extends PollingSystem { println( s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread current thread: ${Thread.currentThread().getName()}" ) - // extraRing.sendMsgRing( - // 0, - // targetPoller.getFd() - // ) // comment this line and change the number of threads in UringSuite to test 1 thread + // extraRing.sendMsgRing(0, targetPoller.getFd()) targetPoller.writeFd() () } @@ -180,7 +177,7 @@ object UringSystem extends PollingSystem { val interruptFd = FileDescriptor.pipe() val readEnd = interruptFd(0) val writeEnd = interruptFd(1) - var wakingUp: Boolean = false + var listenFd: Boolean = false private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() @@ -191,7 +188,7 @@ object UringSystem extends PollingSystem { private[this] val ids = new BitSet(Short.MaxValue) private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(1) + val newId = ids.nextClearBit(10) ids.set(newId) newId.toShort } @@ -202,10 +199,10 @@ object UringSystem extends PollingSystem { callbacks .remove(id) .map { _ => - if (debug) { - println(s"REMOVED CB WITH ID: $id") - println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") - } + // if (debug) { + // println(s"REMOVED CB WITH ID: $id") + // println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") + // } releaseId(id) } .isDefined @@ -215,10 +212,10 @@ object UringSystem extends PollingSystem { pendingSubmissions = true callbacks.put(id, cb) - if (debug) { - println("GETTING ID") - println(s"CALLBACK MAP UPDATED: $callbacks") - } + // if (debug) { + // println("GETTING ID") + // println(s"CALLBACK MAP UPDATED: $callbacks") + // } id } @@ -231,16 +228,23 @@ object UringSystem extends PollingSystem { length: Int, offset: Long, data: Short - ): Boolean = - // if (debug || op == 14) - // println( - // s"[SQ] Enqueuing a new Sqe in ringFd: ${ring - // .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" - // ) + ): Boolean = { + if (debug || op == 14) + println( + s"[SQ] Enqueuing a new Sqe in ringFd: ${ring + .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + ) + sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) + } private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + if (callbacks.contains(id)) { + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + } else { + println("[CANCEL] the cb has already been handled") + false + } // private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") @@ -282,24 +286,41 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - // if (debug) { - // println( - // s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" - // ) - // } + if (op == 14 || op == 19) { + println( + s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + ) + } if (fd == readEnd.intValue()) { val buf = ByteBuffer.allocateDirect(1) val readed = readEnd.read(buf, 0, 1) // we consume the fd if (readed > 0) { buf.clear() - wakingUp = false // We are not listening to the FD anymore + listenFd = false // We are not listening to the FD anymore } } - - callbacks.get(data).foreach { cb => - handleCallback(res, cb) - removeCallback(data) + + if (op == 14) { + if (callbacks.contains(data)) { + println("[CANCEL] it hasn't been processed yet!") + callbacks.get(data).foreach { cb => + handleCallback(res, cb) + removeCallback(data) + } + } else { + println("[CANCEL] it has already been processed!") + callbacks.get(data).foreach { cb => + handleCallback(res, cb) + removeCallback(data) + } + } + } else { + + callbacks.get(data).foreach { cb => + handleCallback(res, cb) + removeCallback(data) + } } } @@ -319,7 +340,7 @@ object UringSystem extends PollingSystem { private[UringSystem] def poll(nanos: Long): Boolean = { // Check if we are listening to the FD. If not, start listening - if (!wakingUp) { + if (!listenFd) { enqueueSqe( IORING_OP_POLL_ADD, 0, @@ -330,10 +351,12 @@ object UringSystem extends PollingSystem { 0, NativeAccess.POLLIN.toShort ) - sq.submit() - wakingUp = true // Set the flag indicating we're now listening + pendingSubmissions = true + listenFd = true // Set the flag indicating we're now listening } + if (debug) println(s"[POLL ${Thread.currentThread().getName()}]Polling with nanos = $nanos") + var rtn = -1 nanos match { case -1 => @@ -362,7 +385,7 @@ object UringSystem extends PollingSystem { val invokedCbs = process(completionQueueCallback) // If pending submissions exist and we encountered an error, retry submission. - while (pendingSubmissions && rtn < 0) + while (pendingSubmissions && rtn <= 0) rtn = sq.submit() pendingSubmissions = false From 5f6c476850312d2a2cfb1821365be59b1bcfb465 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Wed, 16 Aug 2023 12:48:55 +0000 Subject: [PATCH 128/200] Add concurrent Map to track op and poller --- .../fs2/io/uring/net/UringSocketGroup.scala | 16 +- .../fs2/io/uring/unsafe/UringSystem.scala | 157 ++++++++---------- 2 files changed, 80 insertions(+), 93 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index b800a08c..33ac36b8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -205,14 +205,14 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn - io_uring doesn't have a bind, listen or getLocalAddress operators. - Is it possible to create a LinuxSocket using the fd created by the ring ? This would solve the problem. */ - private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { - val domain = if (ipv6) AF_INET6 else AF_INET - ring - .bracket(op = IORING_OP_SOCKET, fd = domain, length = 0, offset = SOCK_STREAM)( - closeSocket(ring, _) - ) - .mapK(LiftIO.liftK) - } + // private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { + // val domain = if (ipv6) AF_INET6 else AF_INET + // ring + // .bracket(op = IORING_OP_SOCKET, fd = domain, length = 0, offset = SOCK_STREAM)( + // closeSocket(ring, _) + // ) + // .mapK(LiftIO.liftK) + // } private[this] def closeSocket(ring: Uring, fd: Int): IO[Unit] = ring.call(op = IORING_OP_CLOSE, fd = fd).void diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index c331e595..efa71cae 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -48,9 +48,12 @@ object UringSystem extends PollingSystem { private val extraRing: UringRing = UringRing() + private[this] val rings: scala.collection.concurrent.Map[Short, Poller] = + scala.collection.concurrent.TrieMap.empty + private final val MaxEvents = 64 - private val debug = true // True to printout operations + private val debug = false // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -65,17 +68,26 @@ object UringSystem extends PollingSystem { nanos: Long, reportFailure: Throwable => Unit ): Boolean = - poller.poll(nanos) + poller.poll(nanos, rings) override def needsPoll(poller: Poller): Boolean = poller.needsPoll() override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { if (debug) println( - s"[INTERRUPT] waking up poller: $targetPoller in thread: $targetThread current thread: ${Thread.currentThread().getName()}" + s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: $targetPoller in thread: $targetThread" ) + // Interrupt using an extra ring // extraRing.sendMsgRing(0, targetPoller.getFd()) + + // Interrupt using a pipe targetPoller.writeFd() + + /* + Interrupt for single-thread + It consistently calls the interrupt for a single-threaded context. + A suitable implementation might be required to manage this. (?) + */ () } @@ -120,12 +132,14 @@ object UringSystem extends PollingSystem { IO.uncancelable { _ => IO.async_[Int] { cb => register { ring => - val cancelId = ring.getId(cb) + val cancelId = ring.getId(rings, cb) val opAddressToCancel = Encoder.encode(fd, op, id) println( - s"[CANCEL] cancel id: $cancelId and op to cancel is in address: $opAddressToCancel" + s"[CANCEL] from fd: $fd cancel id: $cancelId and op to cancel is in address: $opAddressToCancel" ) - ring.cancel(opAddressToCancel, cancelId) + val correctRing = rings.getOrElse(id, ring) + println(s"The right ring is $correctRing") + correctRing.cancel(opAddressToCancel, cancelId) () } } @@ -139,7 +153,7 @@ object UringSystem extends PollingSystem { F.uncancelable { poll => val submit: IO[Short] = IO.async_[Short] { cb => register { ring => - val id = ring.getId(resume) + val id = ring.getId(rings, resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right(id)) } @@ -172,8 +186,6 @@ object UringSystem extends PollingSystem { final class Poller private[UringSystem] (ring: UringRing) { - val test: FileDescriptor = NativeAccess.newBlockingEventFd - val interruptFd = FileDescriptor.pipe() val readEnd = interruptFd(0) val writeEnd = interruptFd(1) @@ -195,7 +207,7 @@ object UringSystem extends PollingSystem { private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - private[this] def removeCallback(id: Short): Boolean = + private[this] def removeCallback(id: Short, rings: scala.collection.concurrent.Map[Short, Poller]): Boolean = callbacks .remove(id) .map { _ => @@ -204,14 +216,19 @@ object UringSystem extends PollingSystem { // println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") // } releaseId(id) + rings.remove(id) } .isDefined - private[UringSystem] def getId(cb: Either[Throwable, Int] => Unit): Short = { + private[UringSystem] def getId( + rings: scala.collection.concurrent.Map[Short, Poller], + cb: Either[Throwable, Int] => Unit + ): Short = { val id: Short = getUniqueId() pendingSubmissions = true callbacks.put(id, cb) + rings.put(id, this) // if (debug) { // println("GETTING ID") // println(s"CALLBACK MAP UPDATED: $callbacks") @@ -229,7 +246,7 @@ object UringSystem extends PollingSystem { offset: Long, data: Short ): Boolean = { - if (debug || op == 14) + if (debug) println( s"[SQ] Enqueuing a new Sqe in ringFd: ${ring .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" @@ -239,105 +256,75 @@ object UringSystem extends PollingSystem { } private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = - if (callbacks.contains(id)) { - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - } else { - println("[CANCEL] the cb has already been handled") - false - } - - // private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { - // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") - // enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) - // submit() - // cq.ioUringWaitCqe() - // cq.process(completionQueueCallback) - // sq.submit() > 0 - // } + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) private[UringSystem] def getFd(): Int = ring.fd() private[UringSystem] def close(): Unit = { - println("WE CLOSE EVERYTHING") readEnd.close() writeEnd.close() ring.close() } - private[UringSystem] def submit(): Boolean = { - val submitted = sq.submit() - pendingSubmissions = false + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty - submitted > 0 + private[UringSystem] def writeFd(): Int = { + val buf = ByteBuffer.allocateDirect(1) + buf.put(0.toByte) + buf.flip() + writeEnd.write(buf, 0, 1) } - private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty + private[UringSystem] def poll( + nanos: Long, + rings: scala.collection.concurrent.Map[Short, Poller] + ): Boolean = { - private[this] val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if (res < 0) - cb( - Left( - new IOException( - s"Error in completion queue entry of the ring with fd: ${ring - .fd()} with fd: $fd op: $op res: $res and data: $data" + val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { + def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = + if (res < 0) + cb( + Left( + new IOException( + s"Error in completion queue entry of the ring with fd: ${ring + .fd()} with fd: $fd op: $op res: $res and data: $data" + ) ) ) - ) - else cb(Right(res)) + else cb(Right(res)) - if (op == 14 || op == 19) { - println( - s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" - ) - } - - if (fd == readEnd.intValue()) { - val buf = ByteBuffer.allocateDirect(1) - val readed = readEnd.read(buf, 0, 1) // we consume the fd - if (readed > 0) { - buf.clear() - listenFd = false // We are not listening to the FD anymore + if (op == 14 || op == 19) { + println( + s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + ) } - } - if (op == 14) { - if (callbacks.contains(data)) { - println("[CANCEL] it hasn't been processed yet!") - callbacks.get(data).foreach { cb => - handleCallback(res, cb) - removeCallback(data) - } - } else { - println("[CANCEL] it has already been processed!") - callbacks.get(data).foreach { cb => - handleCallback(res, cb) - removeCallback(data) + /* + Instead of using a callback for interrupt handling, we manage the interrupt directly within this block. + Checks for an interrupt by determining if the FileDescriptor (fd) has been written to. + */ + if (fd == readEnd.intValue()) { + val buf = ByteBuffer.allocateDirect(1) + val readed = readEnd.read(buf, 0, 1) // Consume the listener + if (readed > 0) { + buf.clear() + listenFd = false // Sets the flag to stop listening to the FileDescriptor } } - } else { + // Handle the callback callbacks.get(data).foreach { cb => handleCallback(res, cb) - removeCallback(data) + removeCallback(data, rings) } - } + } } - } - - private[this] def process( - completionQueueCallback: UringCompletionQueueCallback - ): Boolean = - cq.process(completionQueueCallback) > 0 - private[UringSystem] def writeFd(): Int = { - val buf = ByteBuffer.allocateDirect(1) - buf.put(0.toByte) - buf.flip() - writeEnd.write(buf, 0, 1) - } - private[UringSystem] def poll(nanos: Long): Boolean = { + def process( + completionQueueCallback: UringCompletionQueueCallback + ): Boolean = + cq.process(completionQueueCallback) > 0 // Check if we are listening to the FD. If not, start listening if (!listenFd) { From 190d060d76bf9743e382b3ff5d60feb20f278027 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 17 Aug 2023 01:47:45 +0200 Subject: [PATCH 129/200] Add TODO for handling cancellation on the right poller --- .../fs2/io/uring/unsafe/UringSystem.scala | 71 ++++++++++++------- 1 file changed, 47 insertions(+), 24 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index efa71cae..b9ce808e 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -43,14 +43,10 @@ import java.util.BitSet import io.netty.channel.unix.FileDescriptor import java.nio.ByteBuffer import io.netty.incubator.channel.uring.NativeAccess +import java.util.concurrent.ConcurrentLinkedDeque object UringSystem extends PollingSystem { - private val extraRing: UringRing = UringRing() - - private[this] val rings: scala.collection.concurrent.Map[Short, Poller] = - scala.collection.concurrent.TrieMap.empty - private final val MaxEvents = 64 private val debug = false // True to printout operations @@ -68,7 +64,7 @@ object UringSystem extends PollingSystem { nanos: Long, reportFailure: Throwable => Unit ): Boolean = - poller.poll(nanos, rings) + poller.poll(nanos) override def needsPoll(poller: Poller): Boolean = poller.needsPoll() @@ -78,7 +74,7 @@ object UringSystem extends PollingSystem { s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: $targetPoller in thread: $targetThread" ) // Interrupt using an extra ring - // extraRing.sendMsgRing(0, targetPoller.getFd()) + // targetPoller.wakeup() // Interrupt using a pipe targetPoller.writeFd() @@ -128,18 +124,21 @@ object UringSystem extends PollingSystem { offset: Long )(release: Int => IO[Unit]): IO[Int] = { - def cancel(id: Short): IO[Boolean] = + /* + TODO: + - If the ring for current thrad matches the ring we need to send cancel to, submit the cancel right away + - otherwise we access the cancel queue for that ring, we add the op and we interrupt the ring so it process the cancel + */ + def cancel(id: Short): IO[Boolean] = // We need access to the correct ring IO.uncancelable { _ => IO.async_[Int] { cb => register { ring => - val cancelId = ring.getId(rings, cb) + val cancelId = ring.getId(cb) val opAddressToCancel = Encoder.encode(fd, op, id) println( s"[CANCEL] from fd: $fd cancel id: $cancelId and op to cancel is in address: $opAddressToCancel" ) - val correctRing = rings.getOrElse(id, ring) - println(s"The right ring is $correctRing") - correctRing.cancel(opAddressToCancel, cancelId) + ring.cancel(opAddressToCancel, cancelId) () } } @@ -153,17 +152,17 @@ object UringSystem extends PollingSystem { F.uncancelable { poll => val submit: IO[Short] = IO.async_[Short] { cb => register { ring => - val id = ring.getId(rings, resume) + val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - cb(Right(id)) + cb(Right(id)) // pass the pair (id, ring) } } lift(submit) - .flatMap { id => + .flatMap { id => // (id, ring) F.onCancel( poll(get), - lift(cancel(id)).ifM( + lift(cancel(id)).ifM( // pass cancel(id, ring) F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => @@ -184,6 +183,11 @@ object UringSystem extends PollingSystem { } + /* + TODO: + If the ring was woke up by interruption, check the cancel queue + Replace the Pipe method with send_ring_msg + */ final class Poller private[UringSystem] (ring: UringRing) { val interruptFd = FileDescriptor.pipe() @@ -191,6 +195,11 @@ object UringSystem extends PollingSystem { val writeEnd = interruptFd(1) var listenFd: Boolean = false + private[this] val extraRing: UringRing = UringRing() + + private[this] val cancelOperations: ConcurrentLinkedDeque[(Long, Short)] = + new ConcurrentLinkedDeque() + private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() private[this] val cq: UringCompletionQueue = ring.ioUringCompletionQueue() @@ -207,7 +216,7 @@ object UringSystem extends PollingSystem { private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - private[this] def removeCallback(id: Short, rings: scala.collection.concurrent.Map[Short, Poller]): Boolean = + private[this] def removeCallback(id: Short): Boolean = callbacks .remove(id) .map { _ => @@ -216,19 +225,16 @@ object UringSystem extends PollingSystem { // println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") // } releaseId(id) - rings.remove(id) } .isDefined private[UringSystem] def getId( - rings: scala.collection.concurrent.Map[Short, Poller], cb: Either[Throwable, Int] => Unit ): Short = { val id: Short = getUniqueId() pendingSubmissions = true callbacks.put(id, cb) - rings.put(id, this) // if (debug) { // println("GETTING ID") // println(s"CALLBACK MAP UPDATED: $callbacks") @@ -256,7 +262,21 @@ object UringSystem extends PollingSystem { } private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + if (callbacks.contains(id)) { + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + } else { + println("[CANCEL] the cb has already been handled") + false + } + + // private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { + // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") + // enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) + // submit() + // cq.ioUringWaitCqe() + // cq.process(completionQueueCallback) + // sq.submit() > 0 + // } private[UringSystem] def getFd(): Int = ring.fd() @@ -274,9 +294,12 @@ object UringSystem extends PollingSystem { buf.flip() writeEnd.write(buf, 0, 1) } + + private[UringSystem] def wakeup() = + extraRing.sendMsgRing(0, this.getFd()) + private[UringSystem] def poll( - nanos: Long, - rings: scala.collection.concurrent.Map[Short, Poller] + nanos: Long ): Boolean = { val completionQueueCallback = new UringCompletionQueueCallback { @@ -315,7 +338,7 @@ object UringSystem extends PollingSystem { // Handle the callback callbacks.get(data).foreach { cb => handleCallback(res, cb) - removeCallback(data, rings) + removeCallback(data) } } From e01b2a308a1f131515702a34c6a133f6c501a7ff Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 18 Aug 2023 14:06:16 +0200 Subject: [PATCH 130/200] Abstract operations parameters in a class --- .../main/scala/fs2/io/uring/Operation.scala | 67 +++++++++++++++++++ 1 file changed, 67 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/Operation.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala b/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala new file mode 100644 index 00000000..0c58d906 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala @@ -0,0 +1,67 @@ +package fs2.io.uring + +import fs2.io.uring.unsafe.util.OP._ + +trait Operation { + val op: Byte + val flags: Int = 0 + val rwFlags: Int = 0 + val fd: Int = 0 + val bufferAddress: Long = 0 + val length: Int = 0 + val offset: Long = 0 +} + +case class noOperation() extends Operation { + override val op: Byte = IORING_OP_NOP +} + +case class send( + override val flags: Int, + override val fd: Int, + override val bufferAddress: Long, + override val length: Int +) extends Operation { + override val op: Byte = IORING_OP_SEND +} + +case class recv( + override val flags: Int, + override val fd: Int, + override val bufferAddress: Long, + override val length: Int +) extends Operation { + override val op: Byte = IORING_OP_RECV +} + +case class shutdown( + override val fd: Int, + override val length: Int +) extends Operation { + override val op: Byte = IORING_OP_SHUTDOWN +} + +case class accept(override val fd: Int, override val bufferAddress: Long, override val offset: Long) + extends Operation { + override val op: Byte = IORING_OP_ACCEPT +} + +case class connect( + override val fd: Int, + override val bufferAddress: Long, + override val offset: Long +) extends Operation { + + override val op: Byte = IORING_OP_CONNECT +} + +case class open(override val fd: Int, override val length: Int, override val offset: Long) + extends Operation { + override val op: Byte = IORING_OP_SOCKET + +} +case class close(override val fd: Int) extends Operation { + override val op: Byte = IORING_OP_CLOSE +} + + From 6ae4b3323f13cec44711ea9d406a1f637ffc2766 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 18 Aug 2023 16:35:49 +0200 Subject: [PATCH 131/200] Handle cancellation from a different thread --- .../fs2/io/uring/unsafe/UringSystem.scala | 76 +++++++++++++------ 1 file changed, 51 insertions(+), 25 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index b9ce808e..1f1321df 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -129,16 +129,29 @@ object UringSystem extends PollingSystem { - If the ring for current thrad matches the ring we need to send cancel to, submit the cancel right away - otherwise we access the cancel queue for that ring, we add the op and we interrupt the ring so it process the cancel */ - def cancel(id: Short): IO[Boolean] = // We need access to the correct ring + def cancel( + id: Short, + correctRing: Poller + ): IO[Boolean] = // We need access to the correct ring IO.uncancelable { _ => IO.async_[Int] { cb => register { ring => - val cancelId = ring.getId(cb) val opAddressToCancel = Encoder.encode(fd, op, id) println( - s"[CANCEL] from fd: $fd cancel id: $cancelId and op to cancel is in address: $opAddressToCancel" + s"[CANCEL] from fd: ${ring.getFd()} cancel id: and op to cancel is in address: $opAddressToCancel" ) - ring.cancel(opAddressToCancel, cancelId) + if (correctRing == ring) { + val cancelId = ring.getId(cb) + println( + s"[CANCEL] We are cancelling from the same ring!: ${ring.getFd()} and ${correctRing.getFd()}" + ) + ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, cancelId) + } else { + // val cancelId = correctRing.getId(cb) + println(s"[CANCEL] We are cancelling from another ring!: from: ${ring + .getFd()} and we want to cancel in: ${correctRing.getFd()}") + correctRing.cancelFromOtherRing(opAddressToCancel, cb) + } () } } @@ -150,19 +163,19 @@ object UringSystem extends PollingSystem { F: MonadCancelThrow[F] ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => - val submit: IO[Short] = IO.async_[Short] { cb => + val submit: IO[(Short, Poller)] = IO.async_[(Short, Poller)] { cb => register { ring => val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - cb(Right(id)) // pass the pair (id, ring) + cb(Right((id, ring))) // pass the pair (id, ring) } } lift(submit) - .flatMap { id => // (id, ring) + .flatMap { case (id, ring) => // (id, ring) F.onCancel( poll(get), - lift(cancel(id)).ifM( // pass cancel(id, ring) + lift(cancel(id, ring)).ifM( // pass cancel(id, ring) F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => @@ -197,7 +210,7 @@ object UringSystem extends PollingSystem { private[this] val extraRing: UringRing = UringRing() - private[this] val cancelOperations: ConcurrentLinkedDeque[(Long, Short)] = + private[this] val cancelOperations: ConcurrentLinkedDeque[(Long, Either[Throwable, Int] => Unit)] = new ConcurrentLinkedDeque() private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() @@ -261,22 +274,22 @@ object UringSystem extends PollingSystem { sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } - private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = - if (callbacks.contains(id)) { - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - } else { - println("[CANCEL] the cb has already been handled") - false - } + private[this] def cancel(opAddressToCancel: Long, id: Short): Unit = { + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + sq.submit() + () + } - // private[UringSystem] def sendMsgRing(flags: Int, fd: Int): Boolean = { - // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") - // enqueueSqe(IORING_OP_MSG_RING, flags, 0, fd, 0, 0, 0, 0) - // submit() - // cq.ioUringWaitCqe() - // cq.process(completionQueueCallback) - // sq.submit() > 0 - // } + private[UringSystem] def cancelFromOtherRing( + opAddressToCancel: Long, + cb: Either[Throwable, Int] => Unit + ): Boolean = { + // val id = getId(cb) + // println(s"WE GOT THE ID: $id") + println(s"WE ADDED THE OPERATION TO CANCEL") + cancelOperations.add((opAddressToCancel, cb)) + // wakeup() + } private[UringSystem] def getFd(): Int = ring.fd() @@ -316,7 +329,7 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - if (op == 14 || op == 19) { + if (op == 14) { println( s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) @@ -365,6 +378,19 @@ object UringSystem extends PollingSystem { listenFd = true // Set the flag indicating we're now listening } + if (!cancelOperations.isEmpty()) { + println("THE CANCEL QUEUE IT IS NOT EMPTY!") + + cancelOperations.forEach { + case (opAddressToCancel, cb) => { + val id = getId(cb) + cancel(opAddressToCancel, id) + } + } + sq.submit() + cancelOperations.clear() + } + if (debug) println(s"[POLL ${Thread.currentThread().getName()}]Polling with nanos = $nanos") var rtn = -1 From 9f81ecd25fc1e60cae838d6207b23886bae494c7 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 18 Aug 2023 17:13:35 +0200 Subject: [PATCH 132/200] Wake up the canceled ring from the current ring --- .../fs2/io/uring/unsafe/UringSystem.scala | 13 ++++++++++--- .../test/scala/fs2/io/uring/UringSuite.scala | 18 +++++++++--------- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 1f1321df..f88349e2 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -151,6 +151,10 @@ object UringSystem extends PollingSystem { println(s"[CANCEL] We are cancelling from another ring!: from: ${ring .getFd()} and we want to cancel in: ${correctRing.getFd()}") correctRing.cancelFromOtherRing(opAddressToCancel, cb) + + // wake up: + ring.enqueueSqe(40, 0, 0, correctRing.getFd(), 0, 0, 0, 0) + ring.submit() } () } @@ -255,6 +259,8 @@ object UringSystem extends PollingSystem { id } + private[UringSystem] def submit() = sq.submit() + private[UringSystem] def enqueueSqe( op: Byte, flags: Int, @@ -276,7 +282,7 @@ object UringSystem extends PollingSystem { private[this] def cancel(opAddressToCancel: Long, id: Short): Unit = { enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - sq.submit() + // sq.submit() () } @@ -288,7 +294,8 @@ object UringSystem extends PollingSystem { // println(s"WE GOT THE ID: $id") println(s"WE ADDED THE OPERATION TO CANCEL") cancelOperations.add((opAddressToCancel, cb)) - // wakeup() + + // wakeup() TODO: we don't want to interrupt this poller from a different thread } private[UringSystem] def getFd(): Int = ring.fd() @@ -329,7 +336,7 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - if (op == 14) { + if (op == 14 || op == 40) { println( s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 8e159831..c9ef1d7e 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -25,16 +25,16 @@ import scala.concurrent.duration._ abstract class UringSuite extends CatsEffectSuite { - override lazy val munitIORuntime = { - val (pool, poller, shutdown) = - IORuntime.createWorkStealingComputeThreadPool(threads = 2, pollingSystem = UringSystem) - IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() - } + // override lazy val munitIORuntime = { + // val (pool, poller, shutdown) = + // IORuntime.createWorkStealingComputeThreadPool(threads = 2, pollingSystem = UringSystem) + // IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() + // } - // override lazy val munitIORuntime = - // IORuntimeBuilder() - // .setPollingSystem(UringSystem) - // .build() + override lazy val munitIORuntime = + IORuntimeBuilder() + .setPollingSystem(UringSystem) + .build() override def munitIOTimeout: Duration = 3.second } From 8b806531c74aab35e51cde430bc32323c5c54da1 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 19 Aug 2023 01:07:09 +0200 Subject: [PATCH 133/200] Add TODO with bugs to correct in tests --- .../src/test/scala/fs2/io/uring/UringSuite.scala | 2 +- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 16 +++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index c9ef1d7e..c5703349 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -36,5 +36,5 @@ abstract class UringSuite extends CatsEffectSuite { .setPollingSystem(UringSystem) .build() - override def munitIOTimeout: Duration = 3.second + override def munitIOTimeout: Duration = 10.second } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index d111d593..496ca887 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -56,7 +56,7 @@ class TcpSocketSuit extends UringSuite { .head writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 200 OK") + .assertEquals("HTTP/1.1 301 Moved Permanently") } } @@ -136,12 +136,12 @@ class TcpSocketSuit extends UringSuite { Nil ) - test("Start server and wait for a connection during 10 sec") { + test("Start server and wait for a connection during 5 sec") { serverResource.use { case (localAddress, _) => IO { println(s"[TEST] Server started at $localAddress") println(s"[TEST] You can now connect to this server") - } *> IO.sleep(10.second) + } *> IO.sleep(5.second) // Use telnet localhost "port" to connect } } @@ -175,7 +175,6 @@ class TcpSocketSuit extends UringSuite { IO.println("socket created and connection established!") *> echoServer.background.use(_ => - IO.sleep(10.second) *> // TODO server waits for connection but we never connect to it. write.compile.drain *> IO.println("message written!") ) @@ -232,6 +231,10 @@ class TcpSocketSuit extends UringSuite { val repetitions: Int = 1 + /* + TODO: (Very rare) timeout error polling the cancellation + TODO: (Very rare) second cancellation with error -2 (we shouldn't have a second cancellation (?)) + */ test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) val clientCount = 20L @@ -272,6 +275,9 @@ class TcpSocketSuit extends UringSuite { test.replicateA(repetitions).void } + /* + TODO: (Very rare) error -107 + */ test("readN yields chunks of the requested size") { val message = Chunk.array("123456789012345678901234567890".getBytes) val sizes = Vector(1, 2, 3, 4, 3, 2, 1) @@ -318,7 +324,7 @@ class TcpSocketSuit extends UringSuite { // concurrent writes Stream { Stream.eval(socket.write(message)).repeatN(10L) - }.repeatN(2L).parJoinUnbounded + }.repeatN(1L).parJoinUnbounded } client.concurrently(readOnlyServer) From be666a3444fc5f338c12961438f17f671c7aeabf Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 19 Aug 2023 01:07:44 +0200 Subject: [PATCH 134/200] Add poller fd in prints --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index f88349e2..d93ccb66 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -49,7 +49,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false // True to printout operations + private val debug = true // True to printout operations type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -71,7 +71,7 @@ object UringSystem extends PollingSystem { override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { if (debug) println( - s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: $targetPoller in thread: $targetThread" + s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: ${targetPoller.getFd()} in thread: $targetThread" ) // Interrupt using an extra ring // targetPoller.wakeup() @@ -155,6 +155,7 @@ object UringSystem extends PollingSystem { // wake up: ring.enqueueSqe(40, 0, 0, correctRing.getFd(), 0, 0, 0, 0) ring.submit() + } () } @@ -394,7 +395,6 @@ object UringSystem extends PollingSystem { cancel(opAddressToCancel, id) } } - sq.submit() cancelOperations.clear() } From 055684397440169ea9c4003052b5b5fcc6e5e4eb Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 21 Aug 2023 20:54:07 +0200 Subject: [PATCH 135/200] Fix bug related to timeout --- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 17 ++++++++++------- .../test/scala/fs2/io/uring/UringSuite.scala | 2 +- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 3 +-- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index d93ccb66..40b2baf8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -138,7 +138,7 @@ object UringSystem extends PollingSystem { register { ring => val opAddressToCancel = Encoder.encode(fd, op, id) println( - s"[CANCEL] from fd: ${ring.getFd()} cancel id: and op to cancel is in address: $opAddressToCancel" + s"[CANCEL] from fd: ${ring.getFd()} cancel an operation $op with id: $id and op to cancel is in address: $opAddressToCancel" ) if (correctRing == ring) { val cancelId = ring.getId(cb) @@ -146,6 +146,7 @@ object UringSystem extends PollingSystem { s"[CANCEL] We are cancelling from the same ring!: ${ring.getFd()} and ${correctRing.getFd()}" ) ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, cancelId) + ring.submit() } else { // val cancelId = correctRing.getId(cb) println(s"[CANCEL] We are cancelling from another ring!: from: ${ring @@ -153,8 +154,8 @@ object UringSystem extends PollingSystem { correctRing.cancelFromOtherRing(opAddressToCancel, cb) // wake up: - ring.enqueueSqe(40, 0, 0, correctRing.getFd(), 0, 0, 0, 0) - ring.submit() + // ring.enqueueSqe(40, 0, 0, correctRing.getFd(), 0, 0, 0, 0) + // ring.submit() } () @@ -296,7 +297,8 @@ object UringSystem extends PollingSystem { println(s"WE ADDED THE OPERATION TO CANCEL") cancelOperations.add((opAddressToCancel, cb)) - // wakeup() TODO: we don't want to interrupt this poller from a different thread + writeFd() >= 0 + // wakeup() // TODO: we don't want to interrupt this poller from a different thread } private[UringSystem] def getFd(): Int = ring.fd() @@ -337,7 +339,7 @@ object UringSystem extends PollingSystem { ) else cb(Right(res)) - if (op == 14 || op == 40) { + if (op == 14 || op == 40 || res < 0) { println( s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) @@ -389,6 +391,7 @@ object UringSystem extends PollingSystem { if (!cancelOperations.isEmpty()) { println("THE CANCEL QUEUE IT IS NOT EMPTY!") + println(s"THE QUEUE HAS: ${cancelOperations.size()} ELEMENTS") cancelOperations.forEach { case (opAddressToCancel, cb) => { val id = getId(cb) @@ -416,10 +419,10 @@ object UringSystem extends PollingSystem { case _ => if (pendingSubmissions) { - sq.addTimeout(nanos, 0) + sq.addTimeout(1000000000, 0) rtn = sq.submitAndWait() } else { - sq.addTimeout(nanos, 0) + sq.addTimeout(1000000000, 0) sq.submit() cq.ioUringWaitCqe() } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index c5703349..6b87f0e2 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -36,5 +36,5 @@ abstract class UringSuite extends CatsEffectSuite { .setPollingSystem(UringSystem) .build() - override def munitIOTimeout: Duration = 10.second + override def munitIOTimeout: Duration = 20.second } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 496ca887..0c4f411f 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -229,10 +229,9 @@ class TcpSocketSuit extends UringSuite { clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - val repetitions: Int = 1 + val repetitions: Int = 5 /* - TODO: (Very rare) timeout error polling the cancellation TODO: (Very rare) second cancellation with error -2 (we shouldn't have a second cancellation (?)) */ test("echo requests - each concurrent client gets back what it sent") { From 92ce0592da5427f1bf06ec7a168aec9c7b347c40 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 21 Aug 2023 23:17:34 +0200 Subject: [PATCH 136/200] Remove operation --- .../main/scala/fs2/io/uring/Operation.scala | 67 ------------------- 1 file changed, 67 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/Operation.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala b/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala deleted file mode 100644 index 0c58d906..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/Operation.scala +++ /dev/null @@ -1,67 +0,0 @@ -package fs2.io.uring - -import fs2.io.uring.unsafe.util.OP._ - -trait Operation { - val op: Byte - val flags: Int = 0 - val rwFlags: Int = 0 - val fd: Int = 0 - val bufferAddress: Long = 0 - val length: Int = 0 - val offset: Long = 0 -} - -case class noOperation() extends Operation { - override val op: Byte = IORING_OP_NOP -} - -case class send( - override val flags: Int, - override val fd: Int, - override val bufferAddress: Long, - override val length: Int -) extends Operation { - override val op: Byte = IORING_OP_SEND -} - -case class recv( - override val flags: Int, - override val fd: Int, - override val bufferAddress: Long, - override val length: Int -) extends Operation { - override val op: Byte = IORING_OP_RECV -} - -case class shutdown( - override val fd: Int, - override val length: Int -) extends Operation { - override val op: Byte = IORING_OP_SHUTDOWN -} - -case class accept(override val fd: Int, override val bufferAddress: Long, override val offset: Long) - extends Operation { - override val op: Byte = IORING_OP_ACCEPT -} - -case class connect( - override val fd: Int, - override val bufferAddress: Long, - override val offset: Long -) extends Operation { - - override val op: Byte = IORING_OP_CONNECT -} - -case class open(override val fd: Int, override val length: Int, override val offset: Long) - extends Operation { - override val op: Byte = IORING_OP_SOCKET - -} -case class close(override val fd: Int) extends Operation { - override val op: Byte = IORING_OP_CLOSE -} - - From ae0c8fa50d8c724c782ca83a10e565baf6301d0d Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 21 Aug 2023 23:17:55 +0200 Subject: [PATCH 137/200] Remove pipe --- .../main/scala/fs2/io/uring/unsafe/Pipe.scala | 36 ------------------- 1 file changed, 36 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala deleted file mode 100644 index e6aec0c5..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/Pipe.scala +++ /dev/null @@ -1,36 +0,0 @@ -package fs2.io.uring.unsafe - -import com.sun.jna.{Library, Native, Pointer} -import com.sun.jna.Memory - -class Pipe { - private val libc: CLibrary = Native.load("c", classOf[CLibrary]) - private val pipefd = Array.ofDim[Int](2) - - if (libc.pipe(pipefd) != 0) { - throw new RuntimeException("Pipe creation failed!") - } - - def write(message: String): Int = { - val buffer: Pointer = new Memory(message.length + 1) - buffer.setString(0, message) - val bytesWritten = libc.write(pipefd(1), buffer, message.length) - libc.close(pipefd(1)) - bytesWritten - } - - def read(bufferSize: Int = 100): String = { - val buffer: Pointer = new Memory(bufferSize) - libc.read(pipefd(0), buffer, bufferSize) - val message = buffer.getString(0) - libc.close(pipefd(0)) - message - } - - trait CLibrary extends Library { - def pipe(pipefd: Array[Int]): Int - def read(fd: Int, buf: Pointer, count: Int): Int - def write(fd: Int, buf: Pointer, count: Int): Int - def close(fd: Int): Int - } -} From 5f4015625a8415a1ad62508e681a5edd5fd336e0 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 21 Aug 2023 23:20:11 +0200 Subject: [PATCH 138/200] Remove JNA --- build.sbt | 2 - .../scala/fs2/io/uring/JnaPlayground.scala | 56 ------------------- 2 files changed, 58 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala diff --git a/build.sbt b/build.sbt index a1903c5b..a648edf5 100644 --- a/build.sbt +++ b/build.sbt @@ -34,7 +34,6 @@ ThisBuild / githubWorkflowBuild += val ceVersion = "3.6-0142603" val fs2Version = "3.7.0" val munitCEVersion = "2.0.0-M3" -val jnaVersion = "5.8.0" ThisBuild / nativeConfig ~= { c => val arch = System.getProperty("os.arch").toLowerCase() @@ -55,7 +54,6 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, - "net.java.dev.jna" % "jna" % jnaVersion, // JNA to import C code to java ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") .classifier("linux-x86_64") ), diff --git a/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala b/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala deleted file mode 100644 index c5e058b7..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/JnaPlayground.scala +++ /dev/null @@ -1,56 +0,0 @@ -package fs2.io.uring - -import com.sun.jna.Library -import com.sun.jna.Native -import com.sun.jna.Pointer -import com.sun.jna.Memory - -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.{Await, Future} -import scala.concurrent.duration._ - -trait CLibrary extends Library { - def printf(format: String): Int -} - -trait CPipe extends Library { - def pipe(pipefd: Array[Int]): Int - def read(fd: Int, buf: Pointer, count: Int): Int - def write(fd: Int, buf: Pointer, count: Int): Int - def close(fd: Int): Int -} - -object JnaPlayground extends App { - val printer: CLibrary = Native.load("c", classOf[CLibrary]) - printer.printf("Hello, JNA from Scala!") - - val libc: CPipe = Native.load("c", classOf[CPipe]) - val BUFFER_SIZE = 100 - val pipefd = new Array[Int](2) - - if (libc.pipe(pipefd) != 0) { - throw new RuntimeException("Pipe creation failed!") - } - - val reader = Future { - val buffer: Pointer = new Memory(BUFFER_SIZE) - libc.read(pipefd(0), buffer, BUFFER_SIZE) - val message = buffer.getString(0) - libc.close(pipefd(0)) - message - } - - val writer = Future { - val message = "Hello from writer thread!" - val buffer: Pointer = new Memory(message.length + 1) - buffer.setString(0, message) - println("waiting 2 sec before writing...") - Thread.sleep(2000) - libc.write(pipefd(1), buffer, message.length) - libc.close(pipefd(1)) - } - - Await.ready(writer, 10.seconds) - println(s"Reader received: '${Await.result(reader, 10.seconds)}'") - -} From 418638462e9bcce0b0a6473c585f69f05a352f88 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 14:05:25 +0200 Subject: [PATCH 139/200] Remove unnecessary import --- uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 818a5197..27bd4fde 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -22,7 +22,6 @@ import cats.syntax.parallel._ import fs2.io.uring.UringSuite import fs2.io.uring.unsafe.util.OP._ -import scala.io.StdIn class UringSystemSuitd extends UringSuite { From edbac062174d40bc017ae3d7b779cf9a550770a6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 14:05:57 +0200 Subject: [PATCH 140/200] Fix timeout bug --- .../fs2/io/uring/unsafe/UringSystem.scala | 390 ++++++------------ 1 file changed, 120 insertions(+), 270 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 40b2baf8..98841c35 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -32,24 +32,31 @@ import io.netty.incubator.channel.uring.UringRing import io.netty.incubator.channel.uring.UringSubmissionQueue import io.netty.incubator.channel.uring.UringCompletionQueue import io.netty.incubator.channel.uring.UringCompletionQueueCallback +import io.netty.incubator.channel.uring.NativeAccess import io.netty.incubator.channel.uring.Encoder import fs2.io.uring.unsafe.util.OP._ +import scala.collection.mutable.Map + import java.io.IOException +import java.nio.ByteBuffer -import scala.collection.mutable.Map -import java.util.BitSet import io.netty.channel.unix.FileDescriptor -import java.nio.ByteBuffer -import io.netty.incubator.channel.uring.NativeAccess + +import java.util.BitSet import java.util.concurrent.ConcurrentLinkedDeque object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = true // True to printout operations + private val debug = false + private val debugPoll = debug && true + private val debugCancel = debug && true + private val debugInterrupt = debug && true + private val debugSubmissionQueue = debug && true + private val debugHandleCompletionQueue = debug && true type Api = Uring override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) @@ -69,7 +76,7 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { - if (debug) + if (debugInterrupt) println( s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: ${targetPoller.getFd()} in thread: $targetThread" ) @@ -78,12 +85,6 @@ object UringSystem extends PollingSystem { // Interrupt using a pipe targetPoller.writeFd() - - /* - Interrupt for single-thread - It consistently calls the interrupt for a single-threaded context. - A suitable implementation might be required to manage this. (?) - */ () } @@ -124,11 +125,6 @@ object UringSystem extends PollingSystem { offset: Long )(release: Int => IO[Unit]): IO[Int] = { - /* - TODO: - - If the ring for current thrad matches the ring we need to send cancel to, submit the cancel right away - - otherwise we access the cancel queue for that ring, we add the op and we interrupt the ring so it process the cancel - */ def cancel( id: Short, correctRing: Poller @@ -137,27 +133,25 @@ object UringSystem extends PollingSystem { IO.async_[Int] { cb => register { ring => val opAddressToCancel = Encoder.encode(fd, op, id) - println( - s"[CANCEL] from fd: ${ring.getFd()} cancel an operation $op with id: $id and op to cancel is in address: $opAddressToCancel" - ) - if (correctRing == ring) { - val cancelId = ring.getId(cb) + if (debugCancel) println( - s"[CANCEL] We are cancelling from the same ring!: ${ring.getFd()} and ${correctRing.getFd()}" + s"[CANCEL ring:${ring.getFd()}] cancel an operation: $op with id: $id and address: $opAddressToCancel" ) - ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, cancelId) - ring.submit() + if (correctRing == ring) { + val cancelId = ring.getId(cb) + if (debugCancel) + println( + s"[CANCEL ring:${ring.getFd()}] Cancelling from the same ring!" + ) + ring.cancel(opAddressToCancel, cancelId) } else { - // val cancelId = correctRing.getId(cb) - println(s"[CANCEL] We are cancelling from another ring!: from: ${ring - .getFd()} and we want to cancel in: ${correctRing.getFd()}") - correctRing.cancelFromOtherRing(opAddressToCancel, cb) - - // wake up: - // ring.enqueueSqe(40, 0, 0, correctRing.getFd(), 0, 0, 0, 0) - // ring.submit() - + if (debugCancel) + println( + s"[CANCEL ring:${ring.getFd()}] Cancelling from another ring: cancelled operation is in: ${correctRing.getFd()}" + ) + correctRing.cancelFromDifferentRing(opAddressToCancel, cb) } + () } } @@ -173,15 +167,15 @@ object UringSystem extends PollingSystem { register { ring => val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) - cb(Right((id, ring))) // pass the pair (id, ring) + cb(Right((id, ring))) } } lift(submit) - .flatMap { case (id, ring) => // (id, ring) + .flatMap { case (id, ring) => F.onCancel( poll(get), - lift(cancel(id, ring)).ifM( // pass cancel(id, ring) + lift(cancel(id, ring)).ifM( F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => @@ -202,21 +196,17 @@ object UringSystem extends PollingSystem { } - /* - TODO: - If the ring was woke up by interruption, check the cancel queue - Replace the Pipe method with send_ring_msg - */ final class Poller private[UringSystem] (ring: UringRing) { - val interruptFd = FileDescriptor.pipe() - val readEnd = interruptFd(0) - val writeEnd = interruptFd(1) - var listenFd: Boolean = false + private[this] val interruptFd = FileDescriptor.pipe() + private[this] val readEnd = interruptFd(0) + private[this] val writeEnd = interruptFd(1) + private[this] var listenFd: Boolean = false private[this] val extraRing: UringRing = UringRing() - private[this] val cancelOperations: ConcurrentLinkedDeque[(Long, Either[Throwable, Int] => Unit)] = + private[this] val cancelOperations + : ConcurrentLinkedDeque[(Long, Either[Throwable, Int] => Unit)] = new ConcurrentLinkedDeque() private[this] val sq: UringSubmissionQueue = ring.ioUringSubmissionQueue() @@ -233,35 +223,32 @@ object UringSystem extends PollingSystem { newId.toShort } - private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - - private[this] def removeCallback(id: Short): Boolean = - callbacks - .remove(id) - .map { _ => - // if (debug) { - // println(s"REMOVED CB WITH ID: $id") - // println(s"CALLBACK MAP UPDATED AFTER REMOVING: $callbacks") - // } - releaseId(id) - } - .isDefined - private[UringSystem] def getId( cb: Either[Throwable, Int] => Unit ): Short = { val id: Short = getUniqueId() - pendingSubmissions = true callbacks.put(id, cb) - // if (debug) { - // println("GETTING ID") - // println(s"CALLBACK MAP UPDATED: $callbacks") - // } id } - private[UringSystem] def submit() = sq.submit() + private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) + + private[this] def removeCallback(id: Short): Boolean = + callbacks + .remove(id) + .map(_ => releaseId(id)) + .isDefined + + private[UringSystem] def getFd(): Int = ring.fd() + + private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty + + private[UringSystem] def close(): Unit = { + readEnd.close() + writeEnd.close() + ring.close() + } private[UringSystem] def enqueueSqe( op: Byte, @@ -273,43 +260,29 @@ object UringSystem extends PollingSystem { offset: Long, data: Short ): Boolean = { - if (debug) + if (debugSubmissionQueue) println( - s"[SQ] Enqueuing a new Sqe in ringFd: ${ring - .fd()} with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" + s"[SQ ${ring.fd()}] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" ) sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } - private[this] def cancel(opAddressToCancel: Long, id: Short): Unit = { + private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = { enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - // sq.submit() - () + writeFd() >= 0 } - private[UringSystem] def cancelFromOtherRing( + private[UringSystem] def cancelFromDifferentRing( opAddressToCancel: Long, cb: Either[Throwable, Int] => Unit ): Boolean = { - // val id = getId(cb) - // println(s"WE GOT THE ID: $id") - println(s"WE ADDED THE OPERATION TO CANCEL") cancelOperations.add((opAddressToCancel, cb)) - writeFd() >= 0 - // wakeup() // TODO: we don't want to interrupt this poller from a different thread - } - - private[UringSystem] def getFd(): Int = ring.fd() - - private[UringSystem] def close(): Unit = { - readEnd.close() - writeEnd.close() - ring.close() } - private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty + private[UringSystem] def wakeup() = + extraRing.sendMsgRing(0, this.getFd()) private[UringSystem] def writeFd(): Int = { val buf = ByteBuffer.allocateDirect(1) @@ -318,62 +291,18 @@ object UringSystem extends PollingSystem { writeEnd.write(buf, 0, 1) } - private[UringSystem] def wakeup() = - extraRing.sendMsgRing(0, this.getFd()) - private[UringSystem] def poll( nanos: Long ): Boolean = { - val completionQueueCallback = new UringCompletionQueueCallback { - override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if (res < 0) - cb( - Left( - new IOException( - s"Error in completion queue entry of the ring with fd: ${ring - .fd()} with fd: $fd op: $op res: $res and data: $data" - ) - ) - ) - else cb(Right(res)) - - if (op == 14 || op == 40 || res < 0) { - println( - s"[HANDLE CQCB]: ringfd: ${ring.fd()} fd: $fd, res: $res, flags: $flags, op: $op, data: $data" - ) - } - - /* - Instead of using a callback for interrupt handling, we manage the interrupt directly within this block. - Checks for an interrupt by determining if the FileDescriptor (fd) has been written to. - */ - if (fd == readEnd.intValue()) { - val buf = ByteBuffer.allocateDirect(1) - val readed = readEnd.read(buf, 0, 1) // Consume the listener - if (readed > 0) { - buf.clear() - listenFd = false // Sets the flag to stop listening to the FileDescriptor - } - } - - // Handle the callback - callbacks.get(data).foreach { cb => - handleCallback(res, cb) - removeCallback(data) - } - - } - } - - def process( - completionQueueCallback: UringCompletionQueueCallback - ): Boolean = - cq.process(completionQueueCallback) > 0 + if (debugPoll) + println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") // Check if we are listening to the FD. If not, start listening if (!listenFd) { + if (debugPoll) + println(s"[POLL ${Thread.currentThread().getName()}] We are not listening to the FD!") + enqueueSqe( IORING_OP_POLL_ADD, 0, @@ -388,41 +317,39 @@ object UringSystem extends PollingSystem { listenFd = true // Set the flag indicating we're now listening } + // Check for cancel operations if (!cancelOperations.isEmpty()) { - println("THE CANCEL QUEUE IT IS NOT EMPTY!") - - println(s"THE QUEUE HAS: ${cancelOperations.size()} ELEMENTS") - cancelOperations.forEach { - case (opAddressToCancel, cb) => { - val id = getId(cb) - cancel(opAddressToCancel, id) - } + if (debugPoll) + println( + s"[POLL ${Thread.currentThread().getName()}] The Cancel Queue is not empty, it has: ${cancelOperations.size()} elements" + ) + cancelOperations.forEach { case (opAddressToCancel, cb) => + val id = getId(cb) + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + () } cancelOperations.clear() } - if (debug) println(s"[POLL ${Thread.currentThread().getName()}]Polling with nanos = $nanos") - - var rtn = -1 nanos match { case -1 => if (pendingSubmissions) { - rtn = sq.submitAndWait() + sq.submitAndWait() } else { cq.ioUringWaitCqe() } case 0 => if (pendingSubmissions) { - rtn = sq.submit() + sq.submit() } case _ => if (pendingSubmissions) { - sq.addTimeout(1000000000, 0) - rtn = sq.submitAndWait() + sq.addTimeout(nanos, 0) + sq.submitAndWait() } else { - sq.addTimeout(1000000000, 0) + sq.addTimeout(nanos, 0) sq.submit() cq.ioUringWaitCqe() } @@ -430,128 +357,51 @@ object UringSystem extends PollingSystem { val invokedCbs = process(completionQueueCallback) - // If pending submissions exist and we encountered an error, retry submission. - while (pendingSubmissions && rtn <= 0) - rtn = sq.submit() - pendingSubmissions = false invokedCbs + } - // if (!wakingUp) { - // enqueueSqe( - // IORING_OP_POLL_ADD, - // 0, - // NativeAccess.POLLIN, - // readEnd.intValue(), - // 0, - // 0, - // 0, - // NativeAccess.POLLIN.toShort - // ) - // sq.submit() - // wakingUp = true - // } - - // // 1. Submit pending operations if any - // val submitted = submit() - - // // 2. Check for events based on nanos value - // nanos match { - // case -1 => - // if (debug) println(s"[POLL] we are polling with nanos = -1, therefore we wait for a cqe") - // if (submitted && !cq.hasCompletions()) { - // // if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") - // cq.ioUringWaitCqe() - // } else { - // // sq.addTimeout(0, 0)// replace 1 sec with 0 - // // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - // // submit() - // cq.ioUringWaitCqe() - - // // val buf = ByteBuffer.allocateDirect(1) - // // readEnd.read(buf, 0, 1) - // } - // case 0 => - // // do nothing, just check without waiting - // case _ => - // if (debug) println(s"[POLL] we are polling with nanos = $nanos") - - // if (submitted) { - // // if (debug) println("[POLL] We are going to wait cqe (BLOCKING)") - // cq.ioUringWaitCqe() - // // if (sq.count() > 0) sq.submit() - // // if (cq.hasCompletions()) { - // // process(completionQueueCallback) - // // } - // } else { - // // sq.addTimeout(0, 0)// replace 1 sec with 0 - // // enqueueSqe(IORING_OP_POLL_ADD, NativeAccess.POLLIN, 0, readEnd.intValue(), 0, 0, 0, 0) - // // submit() - - // cq.ioUringWaitCqe() - - // // val buf = ByteBuffer.allocateDirect(1) - // // readEnd.read(buf, 0, 1) - // // sq.addTimeout(nanos, 0) // - // } - // } - - // // 3. Process the events - // val proc = process(completionQueueCallback) - // // if (debug) println(s"[POLL] We processed cqe ? : $proc") - - // proc - - // if (!wakingUp) { - // enqueueSqe( - // IORING_OP_POLL_ADD, - // 0, - // NativeAccess.POLLIN, - // readEnd.intValue(), - // 0, - // 0, - // 0, - // NativeAccess.POLLIN.toShort - // ) - // sq.submit() // we start listening, it will be completed only when we call interrupt, therefore we don't want to submit and wait - // wakingUp = true // Now we are listening - // } - - // def handlePendingSubmissions(submitAndWait: Boolean): Boolean = { - // // if (submitAndWait) println("[HANDLE PENDING SUMBISSION] Submiting and waiting...") - // // else println("[HANDLE PENDING SUMBISSION] Submiting...") - // val submitted = if (submitAndWait) sq.submitAndWait() > 0 else sq.submit() > 0 - // if (submitted) pendingSubmissions = false - // // println( - // // s"[HANDLE PENDING SUBMISSION] submitted a positive number of operations: $submitted" - // // ) - // submitted - // } - - // def handleTimeoutAndQueue(nanos: Long, submitAndWait: Boolean): Boolean = { - // // println(s"[HANDLE TIMEOUT AND QUEUE] adding timeout: $nanos") - // sq.addTimeout(nanos, 0) - // val submitted = handlePendingSubmissions(submitAndWait) - // // println(s"[HANDLE TIMEOUT AND QUEUE] waiting CQE") - // cq.ioUringWaitCqe() - // // println(s"[HANDLE TIMEOUT AND QUEUE] processing CQ") - // process(completionQueueCallback) - // // println(s"[HANDLE TIMEOUT AND QUEUE] submitted a positive number of operations: $submitted") - // submitted - // } - - // nanos match { - // case -1 => - // if (pendingSubmissions) handlePendingSubmissions(true) - // else handleTimeoutAndQueue(-1, true) - // case 0 => if (pendingSubmissions) handlePendingSubmissions(false) else false - // case _ => - // if (pendingSubmissions) handlePendingSubmissions(true) - // else handleTimeoutAndQueue(nanos, false) - // } + private[this] def process( + completionQueueCallback: UringCompletionQueueCallback + ): Boolean = + cq.process(completionQueueCallback) > 0 + + private[this] val completionQueueCallback = new UringCompletionQueueCallback { + override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { + def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = + if (res < 0 && op != 14) // Temporarly, ignore error due to race condition on cancellation + cb( + Left( + new IOException( + s"Error in completion queue entry of the ring with fd: ${ring + .fd()} with fd: $fd op: $op res: $res and data: $data" + ) + ) + ) + else cb(Right(res)) - } + if (debugHandleCompletionQueue) + println( + s"[HANDLE CQCB ${ring.fd()}]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data" + ) + /* + Instead of using a callback for interrupt handling, we manage the interrupt directly within this block. + Checks for an interrupt by determining if the FileDescriptor (fd) has been written to. + */ + if (fd == readEnd.intValue()) { + val buf = ByteBuffer.allocateDirect(1) + readEnd.read(buf, 0, 1) + listenFd = false + } else { + // Handle the callback + callbacks.get(data).foreach { cb => + handleCallback(res, cb) + removeCallback(data) + } + } + } + } } } From a6dcf88199cb44d61e60af9a5a60debf1479942b Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 14:16:11 +0200 Subject: [PATCH 141/200] Comment unused import --- uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 6b87f0e2..5058e397 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -17,7 +17,7 @@ package fs2.io.uring import munit.CatsEffectSuite -import cats.effect.unsafe.IORuntime +// import cats.effect.unsafe.IORuntime import fs2.io.uring.unsafe.UringSystem import cats.effect.unsafe.IORuntimeBuilder import scala.concurrent.duration._ From be8649091850bd3138c8612833044544fb4d4eb9 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 16:13:03 +0200 Subject: [PATCH 142/200] Add comment --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 98841c35..346a4fd2 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -218,7 +218,7 @@ object UringSystem extends PollingSystem { private[this] val ids = new BitSet(Short.MaxValue) private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(10) + val newId = ids.nextClearBit(10) // 0-9 are reserved for certain operations ids.set(newId) newId.toShort } @@ -298,7 +298,7 @@ object UringSystem extends PollingSystem { if (debugPoll) println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") - // Check if we are listening to the FD. If not, start listening + // Check if it is listening to the FD. If not, start listening if (!listenFd) { if (debugPoll) println(s"[POLL ${Thread.currentThread().getName()}] We are not listening to the FD!") @@ -314,7 +314,7 @@ object UringSystem extends PollingSystem { NativeAccess.POLLIN.toShort ) pendingSubmissions = true - listenFd = true // Set the flag indicating we're now listening + listenFd = true // Set the flag indicating it is now listening } // Check for cancel operations From 2430e9e0354827942d94cf0d7f9301c5e4387f3b Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 16:13:12 +0200 Subject: [PATCH 143/200] Remove todo --- .../src/main/scala/fs2/io/uring/unsafe/util.scala | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index f355d07b..a0bba1e9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -23,18 +23,6 @@ import io.netty.buffer.ByteBuf private[uring] object util { - /** TODO: We need to choose between heap or direct buffer and pooled or unpooled buffer: (I feel that Direct/Unpooled is the right combination) - * - * - Heap Buffer: Buffer is backed by a byte array located in the JVM's heap. Convenient if we work with API's that requires byte arrays. - * However, reading/writing from I/O channels requires copying data between the JVM heap and the Native heap which is slow. - * - * - Direct Buffer: Buffer is allocated on the Native heap. Read and writes from I/O channels can occur without copying any data which is faster. - * However, interacting with other Java APIs will require additional data copy. (REMEMBER: They are not subject to the JVM garbage collector, we have to free the memory) - * - * - Pooled Buffer: pre-allocated in memory and reused as needed. It is faster but consumes a lot of memory (we need to conserve a pool of buffers). - * - * - Unpooled Buffer: Allocated when we need them and deallocated when we are done. It may be slower but consume only the memory of the buffer that we are using. - */ def createBuffer[F[_]: Sync](size: Int): Resource[F, ByteBuf] = Resource.make( Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(size)) From 54852f88bd9d95d3e141a42359451366ca464e07 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 16:13:42 +0200 Subject: [PATCH 144/200] Add debug --- .../scala/fs2/io/uring/net/UringSocket.scala | 44 ++++++++----- .../fs2/io/uring/net/UringSocketGroup.scala | 54 ++++++++++------ .../fs2/io/uring/net/TcpSocketSuite.scala | 61 +++++++++---------- 3 files changed, 97 insertions(+), 62 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 2b5ee33a..f3604f71 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -49,24 +49,36 @@ private[net] final class UringSocket[F[_]: LiftIO]( )(implicit F: Async[F]) extends Socket[F] { - private[this] def recv(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = - ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + private val debug = false + private val debugRead = debug && true + private val debugWrite = debug && true + + private[this] def recv(bufferAddress: Long, maxBytes: Int, flags: Int): F[Int] = + ring.call(IORING_OP_RECV, flags, 0, sockfd, bufferAddress, maxBytes, 0).to def read(maxBytes: Int): F[Option[Chunk[Byte]]] = readMutex.lock.surround { for { _ <- F.delay(buffer.clear()) // Clear the buffer before writing - _ <- F.delay(println(s"[SOCKET][READ] writing the received message in the buffer...")) - readed <- recv(buffer.memoryAddress(), 0, maxBytes, 0) + _ <- F.whenA(debugRead)( + F.delay(println(s"[SOCKET][READ] writing the received message in the buffer...")) + ) + + readed <- recv(buffer.memoryAddress(), maxBytes, 0) + + _ <- F.whenA(debugRead)( + F.delay( + println(s"[SOCKET][READ] transfering the message from the buffer to a new array...") + ) + ) - _ <- F.delay(println(s"[SOCKET][READ] transfering the message from the buffer to a new array...")) bytes <- F.delay { val arr = new Array[Byte](readed) buffer.getBytes(0, arr) arr } - _ <- F.delay(println(s"[SOCKET][READ] Done reading!")) + _ <- F.whenA(debugRead)(F.delay(println(s"[SOCKET][READ] Done reading!"))) } yield Option.when(readed > 0)(Chunk.array(bytes)) } @@ -78,7 +90,6 @@ private[net] final class UringSocket[F[_]: LiftIO]( readed <- recv( buffer.memoryAddress(), - 0, numBytes, 0 // TODO: Replace with MSG_WAITALL ) @@ -104,28 +115,33 @@ private[net] final class UringSocket[F[_]: LiftIO]( def localAddress: F[SocketAddress[IpAddress]] = F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) - private[this] def send(bufferAddress: Long, pos: Int, maxBytes: Int, flags: Int): F[Int] = - ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress + pos, maxBytes - pos, 0).to + private[this] def send(bufferAddress: Long, maxBytes: Int, flags: Int): F[Int] = + ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress, maxBytes, 0).to def write(bytes: Chunk[Byte]): F[Unit] = writeMutex.lock .surround { for { - _ <- F.delay(println(s"[SOCKET][WRITE] transfering to the buffer the bytes...")) + _ <- F.whenA(debugWrite)( + F.delay(println(s"[SOCKET][WRITE] transfering to the buffer the bytes...")) + ) + _ <- F.delay { buffer.clear() buffer.writeBytes(bytes.toArray) } - _ <- F.delay(println(s"[SOCKET][WRITE] sending the bytes in the buffer...")) + _ <- F.whenA(debugWrite)( + F.delay(println(s"[SOCKET][WRITE] sending the bytes in the buffer...")) + ) + _ <- send( buffer.memoryAddress(), - 0, bytes.size, - 0 // TODO Replace with MSG_NOSIGNAL + 0 // TODO: Replace with MSG_WAITALL ) - _ <- F.delay(println(s"[SOCKET][WRITE] message sent!")) + _ <- F.whenA(debugWrite)(F.delay(println(s"[SOCKET][WRITE] message sent!"))) } yield () } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index 33ac36b8..ed42baa9 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -43,6 +43,10 @@ import java.net.InetSocketAddress private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) extends SocketGroup[F] { + private val debug = false + private val debugClient = debug && true + private val debugServer = debug && true + private[this] def createBufferAux(isIpv6: Boolean): Resource[F, ByteBuf] = if (isIpv6) createBuffer(SIZEOF_SOCKADDR_IN6) else createBuffer(SIZEOF_SOCKADDR_IN) @@ -60,14 +64,17 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect for { length <- F.delay(write(isIpv6, buf.memoryAddress(), address.toInetSocketAddress)) - _ <- F.delay( - println( - s"[CLIENT] Connecting to address: ${address - .toString()}, Buffer length: $length and LinuxSocket fd: ${linuxSocket.fd()}" + + _ <- F.whenA(debugClient)( + F.delay( + println( + s"[CLIENT] Connecting to address: ${address + .toString()}, Buffer length: $length and LinuxSocket fd: ${linuxSocket.fd()}" + ) ) ) - _ <- F.delay(println("[CLIENT] Connecting...")) + _ <- F.whenA(debugClient)(F.delay(println("[CLIENT] Connecting..."))) _ <- ring .call( op = IORING_OP_CONNECT, @@ -82,7 +89,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) - _ <- Resource.eval(F.delay(println("[CLIENT] Connexion established!"))) + _ <- Resource.eval(F.whenA(debugClient)(F.delay(println("[CLIENT] Connexion established!")))) } yield socket @@ -106,13 +113,17 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn for { resolvedAddress <- Resource.eval(address.fold(IpAddress.loopback)(_.resolve)) - _ <- Resource.eval(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) + _ <- Resource.eval( + F.whenA(debugServer)(F.delay(println(s"[SERVER] Resolved Address: $resolvedAddress"))) + ) isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] linuxSocket <- openSocket(ring, isIpv6) - _ <- Resource.eval(F.delay(println(s"[SERVER] LinusSocketFd: ${linuxSocket.fd()}"))) + _ <- Resource.eval( + F.whenA(debugServer)(F.delay(println(s"[SERVER] LinusSocketFd: ${linuxSocket.fd()}"))) + ) _ <- Resource.eval( F.delay( @@ -126,7 +137,8 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn localAddress <- Resource .eval(F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress()))) - _ <- Resource.eval(F.delay(println(s"[SERVER] Local Address: $localAddress"))) + _ <- Resource + .eval(F.whenA(debugServer)(F.delay(println(s"[SERVER] Local Address: $localAddress")))) sockets = Stream .resource(createBufferAux(isIpv6)) @@ -144,7 +156,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = for { // _ <- Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) - _ <- Resource.eval(F.delay(println("[SERVER] accepting connection..."))) + _ <- Resource.eval( + F.whenA(debugServer)(F.delay(println("[SERVER] accepting connection..."))) + ) clientFd <- ring .bracket( op = IORING_OP_ACCEPT, @@ -152,14 +166,16 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufferAddress = buf.memoryAddress(), offset = bufLength.memoryAddress() )(closeSocket(ring, _)) - .mapK { - new cats.~>[IO, IO] { - def apply[A](ioa: IO[A]) = ioa.debug() - } - } + // .mapK { + // new cats.~>[IO, IO] { + // def apply[A](ioa: IO[A]) = ioa.debug() + // } + // } .mapK(LiftIO.liftK) - _ <- Resource.eval(F.delay(println("[SERVER] connexion established!"))) + _ <- Resource.eval( + F.whenA(debugServer)(F.delay(println("[SERVER] connexion established!"))) + ) } yield clientFd @@ -173,7 +189,11 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn clientFd <- accept remoteAddress <- Resource.eval(convert) _ <- Resource - .eval(F.delay(s"[SERVER] connected to $remoteAddress with fd: $clientFd")) + .eval( + F.whenA(debugServer)( + F.delay(s"[SERVER] connected to $remoteAddress with fd: $clientFd") + ) + ) socket <- UringSocket( ring, UringLinuxSocket(clientFd), diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 0c4f411f..8235ad43 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -33,7 +33,7 @@ import fs2.io.net.Socket import java.util.concurrent.TimeoutException class TcpSocketSuit extends UringSuite { - + val debug = false val sg = UringSocketGroup[IO] // Client test: @@ -138,10 +138,10 @@ class TcpSocketSuit extends UringSuite { test("Start server and wait for a connection during 5 sec") { serverResource.use { case (localAddress, _) => - IO { + IO.whenA(debug)(IO { println(s"[TEST] Server started at $localAddress") println(s"[TEST] You can now connect to this server") - } *> IO.sleep(5.second) + }) *> IO.sleep(5.second) // Use telnet localhost "port" to connect } } @@ -149,11 +149,12 @@ class TcpSocketSuit extends UringSuite { test("Start server and connect external client") { serverResource.use { case (localAddress, _) => for { - _ <- IO.println(s"[TEST] Server started at $localAddress") + _ <- IO.whenA(debug)(IO.println(s"[TEST] Server started at $localAddress")) _ <- sg.client(localAddress).use { socket => for { remoteAddress <- socket.remoteAddress - _ <- IO.println(s"[TEST] Socket created and connected to $remoteAddress!") + _ <- IO + .whenA(debug)(IO.println(s"[TEST] Socket created and connected to $remoteAddress!")) _ <- socket.remoteAddress.map(assertEquals(_, localAddress)) } yield () } @@ -173,10 +174,10 @@ class TcpSocketSuit extends UringSuite { val echoServer = serverStream.compile.drain - IO.println("socket created and connection established!") *> + IO.whenA(debug)(IO.println("socket created and connection established!")) *> echoServer.background.use(_ => - write.compile.drain - *> IO.println("message written!") + write.compile.drain + *> IO.whenA(debug)(IO.println("message written!")) ) } } @@ -208,7 +209,7 @@ class TcpSocketSuit extends UringSuite { .compile .drain - IO.println("socket created and connection established!") *> + IO.whenA(debug)(IO.println("socket created and connection established!")) *> echoServer.background.use(_ => IO.sleep( 1.second @@ -221,15 +222,14 @@ class TcpSocketSuit extends UringSuite { } // Server and client tests: - val setup = for { serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) (bindAddress, server) = serverSetup - _ <- Resource.eval(IO.delay(println(s"Bind address: $bindAddress"))) + _ <- Resource.eval(IO.whenA(debug)(IO.delay(println(s"[TEST] Bind address: $bindAddress")))) clients = Stream.resource(sg.client(bindAddress)).repeat } yield server -> clients - val repetitions: Int = 5 + val repetitions: Int = 1 /* TODO: (Very rare) second cancellation with error -2 (we shouldn't have a second cancellation (?)) @@ -366,19 +366,9 @@ class TcpSocketSuit extends UringSuite { // TODO decide about "read after timed out read not allowed" - test("can shutdown a socket that's pending a read") { - val timeout = 2.seconds - val test = sg.serverResource().use { case (bindAddress, clients) => - sg.client(bindAddress).use { _ => - clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { - case _: TimeoutException => () - } - } - } - - // also test that timeouts are working correctly - test.timed.flatMap { case (duration, _) => - IO(assert(clue(duration) < (timeout + 100.millis))) + test("empty write") { + setup.use { case (_, clients) => + clients.take(1).foreach(_.write(Chunk.empty)).compile.drain } } @@ -388,10 +378,19 @@ class TcpSocketSuit extends UringSuite { } } - test("empty write") { - setup.use { case (_, clients) => - clients.take(1).foreach(_.write(Chunk.empty)).compile.drain - } - } - + // test("can shutdown a socket that's pending a read") { + // val timeout = 2.seconds + // val test = sg.serverResource().use { case (bindAddress, clients) => + // sg.client(bindAddress).use { _ => + // clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { + // case _: TimeoutException => () + // } + // } + // } + + // // also test that timeouts are working correctly + // test.timed.flatMap { case (duration, _) => + // IO(assert(clue(duration) < (timeout + 100.millis))) + // } + // } } From 121d0304aa660e5a3662baf1cfcc74fa9c693f1a Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:11:05 +0200 Subject: [PATCH 145/200] Add debug --- .../scala/fs2/io/uring/UringSystemSuite.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 27bd4fde..4b8d15dd 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -23,12 +23,14 @@ import fs2.io.uring.UringSuite import fs2.io.uring.unsafe.util.OP._ -class UringSystemSuitd extends UringSuite { +class UringSystemSuite extends UringSuite { + + val debug = false test("Create a ring") { val test = for { _ <- Uring.get[IO] - _ <- IO.println("[TEST] We got the ring!") + _ <- IO.whenA(debug)(IO.println("[TEST] We got the ring!")) } yield () test.assertEquals(()) @@ -37,7 +39,7 @@ class UringSystemSuitd extends UringSuite { test("submission") { val test = for { ring <- Uring.get[IO] - _ <- IO.println("[TEST] We got the ring!") + _ <- IO.whenA(debug)(IO.println("[TEST] We got the ring!")) res <- { val op: Byte = IORING_OP_NOP val flags: Int = 0 @@ -76,8 +78,8 @@ class UringSystemSuitd extends UringSuite { val list = for { results <- test - _ <- IO.println(results) - _ <- IO.println(results.size) + _ <- IO.whenA(debug)(IO.println(results)) + _ <- IO.whenA(debug)(IO.println(results.size)) } yield results @@ -104,8 +106,8 @@ class UringSystemSuitd extends UringSuite { val list = for { listOfList <- test listFlatten <- IO(listOfList.flatten) - _ <- IO.println(listFlatten) - _ <- IO.println(listFlatten.size) + _ <- IO.whenA(debug)(IO.println(listFlatten)) + _ <- IO.whenA(debug)(IO.println(listFlatten.size)) } yield listFlatten list.map(results => assert(results.forall(_ >= 0))) From f482e39bdfad759f9bc3efdf78e87cfdb5989a87 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:11:28 +0200 Subject: [PATCH 146/200] Remove invalid test for JVM version --- .../fs2/io/uring/net/TcpSocketSuite.scala | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 8235ad43..41fdf6b5 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -30,9 +30,8 @@ import cats.effect.kernel.Resource import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket -import java.util.concurrent.TimeoutException -class TcpSocketSuit extends UringSuite { +class TcpSocketSuite extends UringSuite { val debug = false val sg = UringSocketGroup[IO] @@ -377,20 +376,4 @@ class TcpSocketSuit extends UringSuite { clients.compile.drain.timeoutTo(100.millis, IO.unit) } } - - // test("can shutdown a socket that's pending a read") { - // val timeout = 2.seconds - // val test = sg.serverResource().use { case (bindAddress, clients) => - // sg.client(bindAddress).use { _ => - // clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { - // case _: TimeoutException => () - // } - // } - // } - - // // also test that timeouts are working correctly - // test.timed.flatMap { case (duration, _) => - // IO(assert(clue(duration) < (timeout + 100.millis))) - // } - // } } From 99ababa618b64049c9d758aeba40884bf235e033 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:12:00 +0200 Subject: [PATCH 147/200] Wrap buffer write in Resource --- .../fs2/io/uring/net/UringSocketGroup.scala | 22 +++++-------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index ed42baa9..be93157a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -149,13 +149,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer Stream.resource { - bufLength.writeInt( - buf.capacity() - ) // TODO: Moved to accept, wrapped in Resource[F, A] but introduces errors in tests (echo requests), probably due to the interrupt - // Accept a connection, write the remote address on the buf and get the clientFd val accept: Resource[F, Int] = for { - // _ <- Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) + _ <- Resource.eval(F.delay(bufLength.writeInt(buf.capacity()))) _ <- Resource.eval( F.whenA(debugServer)(F.delay(println("[SERVER] accepting connection..."))) ) @@ -166,11 +162,11 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn bufferAddress = buf.memoryAddress(), offset = bufLength.memoryAddress() )(closeSocket(ring, _)) - // .mapK { - // new cats.~>[IO, IO] { - // def apply[A](ioa: IO[A]) = ioa.debug() - // } - // } + .mapK { + new cats.~>[IO, IO] { + def apply[A](ioa: IO[A]) = if (debugServer) ioa.debug() else ioa + } + } .mapK(LiftIO.liftK) _ <- Resource.eval( @@ -219,12 +215,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn linuxSocket => closeSocket(ring, linuxSocket.fd()).to ) - /* - The bind, listen and getLocalAddress functions are in the LinuxSocket class. - If we open a socket using the ring, we only get a fd => Problem: how to bind and listen ? - - io_uring doesn't have a bind, listen or getLocalAddress operators. - - Is it possible to create a LinuxSocket using the fd created by the ring ? This would solve the problem. - */ // private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { // val domain = if (ipv6) AF_INET6 else AF_INET // ring From 7a25084450f4670ad3fc6dc26a576b0107693075 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:12:09 +0200 Subject: [PATCH 148/200] Update todo --- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index f3604f71..676d431d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -91,7 +91,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( readed <- recv( buffer.memoryAddress(), numBytes, - 0 // TODO: Replace with MSG_WAITALL + 0 ) bytes <- F.delay { @@ -138,7 +138,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( _ <- send( buffer.memoryAddress(), bytes.size, - 0 // TODO: Replace with MSG_WAITALL + 0 ) _ <- F.whenA(debugWrite)(F.delay(println(s"[SOCKET][WRITE] message sent!"))) From e6da2a75cdd7e6f696dda78efa2903ea93be5a90 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:22:29 +0200 Subject: [PATCH 149/200] Remove temporarily DatagramSocket --- .../io/uring/net/UringDatagramSocket.scala | 118 ------------------ 1 file changed, 118 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala deleted file mode 100644 index 04e86db8..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocket.scala +++ /dev/null @@ -1,118 +0,0 @@ -package fs2.io.uring.net -import cats.effect.LiftIO -import cats.effect.kernel.Async -import cats.effect.kernel.Resource -import cats.effect.kernel.Sync -import cats.effect.std.Mutex -import cats.syntax.all._ - -import com.comcast.ip4s.IpAddress -import com.comcast.ip4s.SocketAddress -import com.comcast.ip4s.MulticastJoin - -import fs2.Chunk -import fs2.Pipe -import fs2.Stream -import fs2.io.net.Datagram -import fs2.io.net.DatagramSocket - -import fs2.io.uring.Uring -import fs2.io.uring.unsafe.util.createBuffer -import fs2.io.uring.unsafe.util.OP._ - -import io.netty.buffer.ByteBuf -import io.netty.incubator.channel.uring.UringLinuxSocket - -private[net] final class UringDatagramSocket[F[_]: LiftIO]( - ring: Uring, - linuxSocket: UringLinuxSocket, - sockfd: Int, - buffer: ByteBuf, - defaultReadSize: Int, - readMutex: Mutex[F], - writeMutex: Mutex[F] -)(implicit F: Async[F]) - extends DatagramSocket[F] { - - private[this] def recvfrom( - bufferAddress: Long, - len: Int - ): F[(SocketAddress[IpAddress], Int)] = // TODO: Work around this - // ring.call(IORING_OP_RECVFROM, fd = sockfd, bufferAddress = bufferAddress, length = len).to - ??? - - private[this] def recvMsg(msgHdr: Long) = ??? - - def read: F[Datagram] = - readMutex.lock.surround { - for { - _ <- F.delay(buffer.clear()) - (srcAddress, len) <- recvfrom(buffer.memoryAddress(), defaultReadSize) - bytes <- F.delay { - val arr = new Array[Byte](len) - buffer.getBytes(0, arr) - arr - } - } yield Datagram(srcAddress, Chunk.array(bytes)) - } - - def reads: Stream[F, Datagram] = Stream.repeatEval(read) - - private[this] def sendto( - bufferAddress: Long, - len: Int, - address: SocketAddress[IpAddress] - ): F[Int] = - // ring - // .call(IORING_OP_SEND, fd = sockfd, bufferAddress = bufferAddress, length = len) - // .to - ??? - - private[this] def sendMsg(msgHdr: Long, flags: Int) = ??? - - def write(datagram: Datagram): F[Unit] = - writeMutex.lock - .surround { - for { - _ <- F.delay { - buffer.clear() - buffer.writeBytes(datagram.bytes.toArray) - } - _ <- sendto(buffer.memoryAddress(), datagram.bytes.size, datagram.remote) - } yield () - } - .unlessA(datagram.bytes.isEmpty) - - def writes: Pipe[F, Datagram, Nothing] = _.evalMap(datagram => write(datagram)).drain - - def localAddress: F[SocketAddress[IpAddress]] = - F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) - - def join( - join: MulticastJoin[IpAddress], - interface: DatagramSocket.NetworkInterface - ): F[GroupMembership] = - F.raiseError(new UnsupportedOperationException("Not supported in DatagramSocket")) -} - -object UringDatagramSocket { - private[this] val defaultReadSize = 65535 - - def apply[F[_]: LiftIO](ring: Uring, linuxSocket: UringLinuxSocket, fd: Int)(implicit - F: Async[F] - ): Resource[F, UringDatagramSocket[F]] = - for { - buffer <- createBuffer(defaultReadSize) - readMutex <- Resource.eval(Mutex[F]) - writeMutex <- Resource.eval(Mutex[F]) - socket = new UringDatagramSocket( - ring, - linuxSocket, - fd, - buffer, - defaultReadSize, - readMutex, - writeMutex - ) - } yield socket -} From 8a20450e607c32a7e3256e9f49154c10b785d9f1 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:23:56 +0200 Subject: [PATCH 150/200] Add header and remove unused import --- .../uring/net/UringDatagramSocketGroup.scala | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala index 1441f2b3..9ea0f32d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala @@ -1,28 +1,32 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package fs2.io.uring.net -import cats.effect.IO -import cats.effect.LiftIO -import cats.effect.kernel.Async import cats.effect.kernel.Resource -import cats.syntax.all._ import com.comcast.ip4s._ -import fs2.Stream -import fs2.io.net.Socket -import fs2.io.net.SocketGroup -import fs2.io.net.SocketOption import fs2.io.net.DatagramSocketGroup import fs2.io.net.DatagramSocket import fs2.io.net._ -import fs2.io.uring.Uring -import fs2.io.uring.unsafe.util.OP._ - import java.net.ProtocolFamily -private final class UringDatagramSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dns[F]) - extends DatagramSocketGroup[F] { +private final class UringDatagramSocketGroup[F[_]] extends DatagramSocketGroup[F] { override def openDatagramSocket( address: Option[Host], @@ -34,5 +38,5 @@ private final class UringDatagramSocketGroup[F[_]: LiftIO](implicit F: Async[F], } object UringDatagramSocketGroup { - def apply[F[_]: Async: Dns: LiftIO]: DatagramSocketGroup[F] = new UringDatagramSocketGroup + def apply[F[_]]: DatagramSocketGroup[F] = new UringDatagramSocketGroup } From 4f55885db8f9723ddda26736302c90412a559130 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:26:59 +0200 Subject: [PATCH 151/200] formatting --- uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala | 1 - uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala index 5058e397..44453322 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala @@ -22,7 +22,6 @@ import fs2.io.uring.unsafe.UringSystem import cats.effect.unsafe.IORuntimeBuilder import scala.concurrent.duration._ - abstract class UringSuite extends CatsEffectSuite { // override lazy val munitIORuntime = { diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index 4b8d15dd..d5b5c5ec 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -77,12 +77,11 @@ class UringSystemSuite extends UringSuite { val test: IO[List[Int]] = calls.parSequence val list = for { - results <- test + results <- test _ <- IO.whenA(debug)(IO.println(results)) _ <- IO.whenA(debug)(IO.println(results.size)) } yield results - list.map(results => assert(results.forall(_ >= 0))) } From 048722df9501b77af14b9b1d255bab8484036321 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:33:32 +0200 Subject: [PATCH 152/200] Fix naming error: double definition --- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 4 ++-- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 676d431d..9c5cc00a 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -41,7 +41,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, linuxSocket: UringLinuxSocket, sockfd: Int, - remoteAddress: SocketAddress[IpAddress], + _remoteAddress: SocketAddress[IpAddress], buffer: ByteBuf, defaultReadSize: Int, readMutex: Mutex[F], @@ -110,7 +110,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( def isOpen: F[Boolean] = F.pure(true) - def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(remoteAddress) + def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(_remoteAddress) def localAddress: F[SocketAddress[IpAddress]] = F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 7f98fb1a..39adcdd0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -568,8 +568,8 @@ final class UringMsgHdr { ) } -final class UringMsgHdrMemoryArray(capacity: Int) { - private[this] val msgHdrMemoryArray: MsgHdrMemoryArray = new MsgHdrMemoryArray(capacity) +final class UringMsgHdrMemoryArray(_capacity: Int) { + private[this] val msgHdrMemoryArray: MsgHdrMemoryArray = new MsgHdrMemoryArray(_capacity) def clear(): Unit = msgHdrMemoryArray.clear() From ef64368c1dec79dd6212d40129cde11da76efc16 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:37:26 +0200 Subject: [PATCH 153/200] Update native folder --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index a648edf5..ad06cd4e 100644 --- a/build.sbt +++ b/build.sbt @@ -27,7 +27,7 @@ ThisBuild / githubWorkflowBuildPreamble ++= { ThisBuild / githubWorkflowBuild += WorkflowStep.Run( - List("clang-format --dry-run --Werror uring/src/main/resources/scala-native/*.c"), + List("clang-format --dry-run --Werror uring/native/src/main/resources/scala-native/*.c"), name = Some("Check formatting of C sources") ) From d43e28066a429c22dd8336e2767ac37b1a92fcf9 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:53:49 +0200 Subject: [PATCH 154/200] Add more common errors --- .../fs2/io/uring/IOExceptionHelper.scala | 43 ++++++++++++++++++- .../fs2/io/uring/IOExceptionHelper.scala | 43 ++++++++++++++++++- 2 files changed, 84 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala b/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala index 34f9f124..d6777cc4 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala @@ -19,17 +19,58 @@ package fs2.io.uring import java.io.IOException import java.net.ConnectException import java.net.BindException +import java.net.SocketException +import java.net.SocketTimeoutException +import java.net.NoRouteToHostException private[uring] object IOExceptionHelper { def apply(errno: Int): IOException = errno match { + case 9 => // EBADF + new IOException("Bad file descriptor") + + case 11 => // EAGAIN + new IOException("Resource temporarily unavailable") + + case 13 => // EACCES + new IOException("Permission denied") + + case 14 => // EFAULT + new IOException("Bad address") + + case 22 => // EINVAL + new IOException("Invalid argument") + + case 24 => // EMFILE + new IOException("Too many open files") + + case 28 => // ENOSPC + new IOException("No space left on device") + + case 32 => // EPIPE + new IOException("Broken pipe") + case 98 => // EADDRINUSE new BindException("Address already in use") + case 99 => // EADDRNOTAVAIL new BindException("Cannot assign requested address") + + case 107 => // ECONNABORTED + new SocketException("Connection aborted") + + case 110 => // ETIMEDOUT + new SocketTimeoutException("Connection timed out") + case 111 => // ECONNREFUSED new ConnectException("Connection refused") + + case 113 => // EHOSTUNREACH + new NoRouteToHostException("No route to host") + + case 104 => // ECONNRESET + new SocketException("Connection reset by peer") + case _ => new IOException(errno.toString) } - } diff --git a/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala b/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala index 34f9f124..d6777cc4 100644 --- a/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala +++ b/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala @@ -19,17 +19,58 @@ package fs2.io.uring import java.io.IOException import java.net.ConnectException import java.net.BindException +import java.net.SocketException +import java.net.SocketTimeoutException +import java.net.NoRouteToHostException private[uring] object IOExceptionHelper { def apply(errno: Int): IOException = errno match { + case 9 => // EBADF + new IOException("Bad file descriptor") + + case 11 => // EAGAIN + new IOException("Resource temporarily unavailable") + + case 13 => // EACCES + new IOException("Permission denied") + + case 14 => // EFAULT + new IOException("Bad address") + + case 22 => // EINVAL + new IOException("Invalid argument") + + case 24 => // EMFILE + new IOException("Too many open files") + + case 28 => // ENOSPC + new IOException("No space left on device") + + case 32 => // EPIPE + new IOException("Broken pipe") + case 98 => // EADDRINUSE new BindException("Address already in use") + case 99 => // EADDRNOTAVAIL new BindException("Cannot assign requested address") + + case 107 => // ECONNABORTED + new SocketException("Connection aborted") + + case 110 => // ETIMEDOUT + new SocketTimeoutException("Connection timed out") + case 111 => // ECONNREFUSED new ConnectException("Connection refused") + + case 113 => // EHOSTUNREACH + new NoRouteToHostException("No route to host") + + case 104 => // ECONNRESET + new SocketException("Connection reset by peer") + case _ => new IOException(errno.toString) } - } From 12938171ea269d81412ec77657d958a8bd1008c4 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 17:58:19 +0200 Subject: [PATCH 155/200] Update workflow to find the new native directory --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 06bb3da9..de72707a 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -95,7 +95,7 @@ jobs: run: sbt 'project ${{ matrix.project }}' '++ ${{ matrix.scala }}' doc - name: Check formatting of C sources - run: clang-format --dry-run --Werror uring/src/main/resources/scala-native/*.c + run: clang-format --dry-run --Werror uring/native/src/main/resources/scala-native/*.c - name: Make target directories if: github.event_name != 'pull_request' && (startsWith(github.ref, 'refs/tags/v') || github.ref == 'refs/heads/main') From 9876348b7b8abc0183e1bccc7d5505eec727f779 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 19:15:28 +0200 Subject: [PATCH 156/200] add documentation --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 39adcdd0..130a1dd3 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -594,22 +594,33 @@ final class UringIov() { def readBufferLength(iovAddress: Long): Int = Iov.readBufferLength(iovAddress) } +/** Represents a Linux socket with utility methods like bind, listen or accept. + * @param socket The underlying Linux socket. + */ final class UringLinuxSocket(private[this] val socket: LinuxSocket) { + /** Returns the local address of the socket. */ def getLocalAddress(): InetSocketAddress = socket.localAddress() + /** Returns the remote address of the socket. */ def getRemoteAddress(): InetSocketAddress = socket.remoteAddress() + /** Returns the internet protocol family of the socket. */ def family(): InternetProtocolFamily = socket.family() + /** Checks if the socket is using IPv6. */ def isIpv6(): Boolean = socket.isIpv6() + /** Returns the file descriptor of the socket. */ def fd(): Int = socket.intValue() + /** Binds the socket to the specified address. */ def bind(socketAddress: SocketAddress): Unit = socket.bind(socketAddress) + /** Listens for connections with a specified backlog. */ def listen(backlog: Int): Unit = socket.listen(backlog) + /** Accepts a connection and returns the file descriptor of the new socket. */ def accept(addr: Array[Byte]): Int = socket.accept(addr) } @@ -617,31 +628,40 @@ final class UringLinuxSocket(private[this] val socket: LinuxSocket) { object UringLinuxSocket { def apply(fd: Int): UringLinuxSocket = new UringLinuxSocket(new LinuxSocket(fd)) + /** Creates a new TCP socket and wraps it in a `UringLinuxSocket`. */ def newSocketStream(): UringLinuxSocket = new UringLinuxSocket(LinuxSocket.newSocketStream()) + /** Creates a new TCP socket (with optional IPv6) and wraps it in a `UringLinuxSocket`. */ def newSocketStream(ipv6: Boolean): UringLinuxSocket = new UringLinuxSocket( LinuxSocket.newSocketStream(ipv6) ) + /** Creates a new datagram socket and wraps it in a `UringLinuxSocket`. */ def newSocketDatagram(): UringLinuxSocket = new UringLinuxSocket(LinuxSocket.newSocketDgram()) + /** Creates a new datagram socket (with optional IPv6) and wraps it in a `UringLinuxSocket`. */ def newSocketDatagram(ipv6: Boolean): UringLinuxSocket = new UringLinuxSocket( LinuxSocket.newSocketDgram(ipv6) ) } +/** Provides utility methods to work with IPv4 and IPv6 socket addresses in Netty io_uring. + */ object UringSockaddrIn { val IPV6_ADDRESS_LENGTH = SockaddrIn.IPV6_ADDRESS_LENGTH val IPV4_ADDRESS_LENGTH = SockaddrIn.IPV4_ADDRESS_LENGTH + /** Writes the socket address to memory. */ def write(ipv6: Boolean, memory: Long, address: InetSocketAddress): Int = SockaddrIn.write(ipv6, memory, address) + /** Reads an IPv4 address from memory. */ def readIPv4(memory: Long, tmpArray: Array[Byte]): InetSocketAddress = SockaddrIn.readIPv4(memory, tmpArray) + /** Reads an IPv6 address from memory. */ def readIPv6(memory: Long, ipv6Array: Array[Byte], ipv4Array: Array[Byte]): InetSocketAddress = SockaddrIn.readIPv6(memory, ipv6Array, ipv4Array) } From e98acdd5a4d788ca263daf0fb109c35ef786e8ef Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Tue, 22 Aug 2023 19:21:48 +0200 Subject: [PATCH 157/200] postman echo was moved --- .../native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index b798498b..4f03a6ad 100644 --- a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -49,7 +49,7 @@ class TcpSocketSuite extends UringSuite { .head writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 200 OK") + .assertEquals("HTTP/1.1 301 Moved Permanently") } } From 3028ac011f50c2dfd613c652f55e60126acf7706 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Wed, 23 Aug 2023 20:17:59 +0200 Subject: [PATCH 158/200] Remove errors log --- hs_err_pid4794.log | 1149 -------------------------------------------- hs_err_pid8969.log | 1149 -------------------------------------------- 2 files changed, 2298 deletions(-) delete mode 100644 hs_err_pid4794.log delete mode 100644 hs_err_pid8969.log diff --git a/hs_err_pid4794.log b/hs_err_pid4794.log deleted file mode 100644 index da28a1c2..00000000 --- a/hs_err_pid4794.log +++ /dev/null @@ -1,1149 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x00007f22d2f923e4, pid=4794, tid=4837 -# -# JRE version: OpenJDK Runtime Environment (17.0.7+7) (build 17.0.7+7-Ubuntu-0ubuntu123.04) -# Java VM: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, linux-amd64) -# Problematic frame: -# V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 -# -# Core dump will be written. Default location: Core dumps may be processed with "/usr/share/apport/apport -p%p -s%s -c%c -d%d -P%P -u%u -g%g -- %E" (or dumping to /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/core.4794) -# -# If you would like to submit a bug report, please visit: -# Unknown -# - ---------------- S U M M A R Y ------------ - -Command Line: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring sbt.ForkMain 44215 - -Host: Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz, 8 cores, 15G, Ubuntu 23.04 -Time: Tue Aug 8 13:11:46 2023 CEST elapsed time: 12.336484 seconds (0d 0h 0m 12s) - ---------------- T H R E A D --------------- - -Current thread (0x00007f224c545570): JavaThread "io-compute-3" daemon [_thread_in_vm, id=4837, stack(0x00007f229c0f6000,0x00007f229c1f6000)] - -Stack: [0x00007f229c0f6000,0x00007f229c1f6000], sp=0x00007f229c1f3678, free space=1013k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 -V [libjvm.so+0xe881a4] SystemDictionary::resolve_instance_class_or_null(Symbol*, Handle, Handle, JavaThread*)+0x134 -V [libjvm.so+0xe89f46] SystemDictionary::resolve_or_fail(Symbol*, Handle, Handle, bool, JavaThread*)+0x66 -V [libjvm.so+0x602cc9] ConstantPool::klass_at_impl(constantPoolHandle const&, int, JavaThread*)+0x139 -V [libjvm.so+0x603a0a] ConstantPool::klass_ref_at(int, JavaThread*)+0x7a -V [libjvm.so+0xa948f9] LinkInfo::LinkInfo(constantPoolHandle const&, int, JavaThread*)+0x39 -V [libjvm.so+0xa9b19f] LinkResolver::resolve_invoke(CallInfo&, Handle, constantPoolHandle const&, int, Bytecodes::Code, JavaThread*)+0x1df -V [libjvm.so+0x82c947] InterpreterRuntime::resolve_invoke(JavaThread*, Bytecodes::Code)+0x177 -V [libjvm.so+0x82ce67] InterpreterRuntime::resolve_from_cache(JavaThread*, Bytecodes::Code)+0x37 -j fs2.CompositeFailure$.fromResults(Lscala/util/Either;Lscala/util/Either;)Lscala/util/Either;+65 -j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+89 -j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 -j fs2.Stream$NestedStreamOps$$$Lambda$692+0x00000008013703d0.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 -j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 -j cats.effect.IO$$Lambda$299+0x000000080125da28.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 -j cats.effect.IO$$Lambda$300+0x000000080125ddf8.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -J 1728 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f22b57553e4 [0x00007f22b5754020+0x00000000000013c4] -J 1857 c1 cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V (3533 bytes) @ 0x00007f22b57be284 [0x00007f22b57aec40+0x000000000000f644] -j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 -J 1971 c1 cats.effect.IOFiber.run()V (113 bytes) @ 0x00007f22b580256c [0x00007f22b58020c0+0x00000000000004ac] -J 1751% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f22b5766af4 [0x00007f22b5763da0+0x0000000000002d54] -v ~StubRoutines::call_stub -V [libjvm.so+0x831c02] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x302 -V [libjvm.so+0x8332d2] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x1a2 -V [libjvm.so+0x8fb417] thread_entry(JavaThread*, JavaThread*)+0xa7 -V [libjvm.so+0xed302e] JavaThread::thread_main_inner()+0xce -V [libjvm.so+0xed675f] Thread::call_run()+0xbf -V [libjvm.so+0xc2d5c9] thread_native_entry(Thread*)+0xe9 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j fs2.CompositeFailure$.fromResults(Lscala/util/Either;Lscala/util/Either;)Lscala/util/Either;+65 -j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+89 -j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 -j fs2.Stream$NestedStreamOps$$$Lambda$692+0x00000008013703d0.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 -j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 -j cats.effect.IO$$Lambda$299+0x000000080125da28.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 -j cats.effect.IO$$Lambda$300+0x000000080125ddf8.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -J 1728 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f22b57553e4 [0x00007f22b5754020+0x00000000000013c4] -J 1857 c1 cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V (3533 bytes) @ 0x00007f22b57be284 [0x00007f22b57aec40+0x000000000000f644] -j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 -J 1971 c1 cats.effect.IOFiber.run()V (113 bytes) @ 0x00007f22b580256c [0x00007f22b58020c0+0x00000000000004ac] -J 1751% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f22b5766af4 [0x00007f22b5763da0+0x0000000000002d54] -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr: 0x0000000000001000 - -Register to memory mapping: - -RAX=0x00007f2200000010 points into unknown readable memory: 0x000000000007b000 | 00 b0 07 00 00 00 00 00 -RBX=0x00007f224c0fbad0 points into unknown readable memory: 0x00007f226c6bea9e | 9e ea 6b 6c 22 7f 00 00 -RCX=0x0000000000000003 is an unknown value -RDX=0x00007f22d3d12330: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 -RSP=0x00007f229c1f3678 is pointing into the stack for thread: 0x00007f224c545570 -RBP=0x00007f229c1f36a0 is pointing into the stack for thread: 0x00007f224c545570 -RSI=0x0000000000000060 is an unknown value -RDI=0x0000000000001000 is an unknown value -R8 =0x00007f224c03c230 points into unknown readable memory: 0x63730010ea9e0035 | 35 00 9e ea 10 00 73 63 -R9 =0x00007f224c545570 is a thread -R10=0x00007f22d3dcd000 points into unknown readable memory: 0x0100050403020100 | 00 01 02 03 04 05 00 01 -R11=0x0000000000000001 is an unknown value -R12=0x00007f228c00de00 points into unknown readable memory: 0x00007f2200000010 | 10 00 00 00 22 7f 00 00 -R13=0x0000000708616d18 is an oop: java.security.ProtectionDomain -{0x0000000708616d18} - klass: 'java/security/ProtectionDomain' - - ---- fields (total size 5 words): - - private 'hasAllPerm' 'Z' @12 false - - private final 'staticPermissions' 'Z' @13 false - - private 'codesource' 'Ljava/security/CodeSource;' @16 a 'java/security/CodeSource'{0x0000000708616d40} (e10c2da8) - - private 'classloader' 'Ljava/lang/ClassLoader;' @20 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000007ffd59e70} (fffab3ce) - - private 'principals' '[Ljava/security/Principal;' @24 a 'java/security/Principal'[0] {0x0000000708616ee0} (e10c2ddc) - - private 'permissions' 'Ljava/security/PermissionCollection;' @28 a 'sun/security/util/LazyCodeSourcePermissionCollection'{0x0000000708616ef0} (e10c2dde) - - final 'key' 'Ljava/security/ProtectionDomain$Key;' @32 a 'java/security/ProtectionDomain$Key'{0x00000007086170b0} (e10c2e16) -R14=0x00007f22cc193210 points into unknown readable memory: 0x00007f22cc165008 | 08 50 16 cc 22 7f 00 00 -R15=0x00007f224c545b80 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 - - -Registers: -RAX=0x00007f2200000010, RBX=0x00007f224c0fbad0, RCX=0x0000000000000003, RDX=0x00007f22d3d12330 -RSP=0x00007f229c1f3678, RBP=0x00007f229c1f36a0, RSI=0x0000000000000060, RDI=0x0000000000001000 -R8 =0x00007f224c03c230, R9 =0x00007f224c545570, R10=0x00007f22d3dcd000, R11=0x0000000000000001 -R12=0x00007f228c00de00, R13=0x0000000708616d18, R14=0x00007f22cc193210, R15=0x00007f224c545b80 -RIP=0x00007f22d2f923e4, EFLAGS=0x0000000000010206, CSGSFS=0x002b000000000033, ERR=0x0000000000000004 - TRAPNO=0x000000000000000e - -Top of Stack: (sp=0x00007f229c1f3678) -0x00007f229c1f3678: 00007f22d308ffb8 00007f229c1f36a0 -0x00007f229c1f3688: 00007f224c03c230 00007f224c545570 -0x00007f229c1f3698: 00007f224c545b90 00007f229c1f37d0 -0x00007f229c1f36a8: 00007f22d38881a4 00007f229c1f3820 - -Instructions: (pc=0x00007f22d2f923e4) -0x00007f22d2f922e4: 01 eb df 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f922f4: 31 c0 c3 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92304: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92314: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92324: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92334: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92344: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92354: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92364: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92374: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92384: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92394: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923a4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923b4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923c4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923d4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923e4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f923f4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92404: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92414: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92424: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92434: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92444: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92454: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92464: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92474: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92484: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f92494: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f924a4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f -0x00007f22d2f924b4: 48 85 ff 74 05 e8 e2 45 14 00 4c 89 23 5b 41 5c -0x00007f22d2f924c4: 5d c3 66 2e 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f22d2f924d4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x00007f22d308ffb8: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 -stack at sp + 1 slots: 0x00007f229c1f36a0 is pointing into the stack for thread: 0x00007f224c545570 -stack at sp + 2 slots: 0x00007f224c03c230 points into unknown readable memory: 0x63730010ea9e0035 | 35 00 9e ea 10 00 73 63 -stack at sp + 3 slots: 0x00007f224c545570 is a thread -stack at sp + 4 slots: 0x00007f224c545b90 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 -stack at sp + 5 slots: 0x00007f229c1f37d0 is pointing into the stack for thread: 0x00007f224c545570 -stack at sp + 6 slots: 0x00007f22d38881a4: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f22d2a00000 -stack at sp + 7 slots: 0x00007f229c1f3820 is pointing into the stack for thread: 0x00007f224c545570 - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x00007f220c3e8a50, length=27, elements={ -0x00007f22cc015d20, 0x00007f22cc179870, 0x00007f22cc17ac50, 0x00007f22cc1801d0, -0x00007f22cc181580, 0x00007f22cc182990, 0x00007f22cc184340, 0x00007f22cc185870, -0x00007f22cc18ecd0, 0x00007f22cc1a76b0, 0x00007f22cc1ab290, 0x00007f2264001140, -0x00007f22cc1ac250, 0x00007f22cc1b1120, 0x00007f22cc373ce0, 0x00007f224c542b10, -0x00007f224c5436b0, 0x00007f224c544660, 0x00007f224c545570, 0x00007f224c546530, -0x00007f224c547440, 0x00007f224c548800, 0x00007f224c549b00, 0x00007f2230006780, -0x00007f227838dfb0, 0x00007f221c029620, 0x00007f220c3e7e00 -} - -Java Threads: ( => current thread ) - 0x00007f22cc015d20 JavaThread "main" [_thread_blocked, id=4797, stack(0x00007f22d2500000,0x00007f22d2600000)] - 0x00007f22cc179870 JavaThread "Reference Handler" daemon [_thread_blocked, id=4804, stack(0x00007f22acc32000,0x00007f22acd32000)] - 0x00007f22cc17ac50 JavaThread "Finalizer" daemon [_thread_blocked, id=4805, stack(0x00007f22acb32000,0x00007f22acc32000)] - 0x00007f22cc1801d0 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=4806, stack(0x00007f22aca32000,0x00007f22acb32000)] - 0x00007f22cc181580 JavaThread "Service Thread" daemon [_thread_blocked, id=4807, stack(0x00007f22ac932000,0x00007f22aca32000)] - 0x00007f22cc182990 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=4808, stack(0x00007f22ac832000,0x00007f22ac932000)] - 0x00007f22cc184340 JavaThread "C2 CompilerThread0" daemon [_thread_in_native, id=4809, stack(0x00007f22ac732000,0x00007f22ac832000)] - 0x00007f22cc185870 JavaThread "C1 CompilerThread0" daemon [_thread_in_native, id=4810, stack(0x00007f22ac632000,0x00007f22ac732000)] - 0x00007f22cc18ecd0 JavaThread "Sweeper thread" daemon [_thread_blocked, id=4811, stack(0x00007f22ac532000,0x00007f22ac632000)] - 0x00007f22cc1a76b0 JavaThread "JDWP Transport Listener: dt_socket" daemon [_thread_blocked, id=4812, stack(0x00007f22ac432000,0x00007f22ac532000)] - 0x00007f22cc1ab290 JavaThread "JDWP Event Helper Thread" daemon [_thread_blocked, id=4813, stack(0x00007f22ac332000,0x00007f22ac432000)] - 0x00007f2264001140 JavaThread "JDWP Command Reader" daemon [_thread_in_native, id=4819, stack(0x00007f22ac232000,0x00007f22ac332000)] - 0x00007f22cc1ac250 JavaThread "Notification Thread" daemon [_thread_blocked, id=4822, stack(0x00007f22ac132000,0x00007f22ac232000)] - 0x00007f22cc1b1120 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=4824, stack(0x00007f229d100000,0x00007f229d200000)] - 0x00007f22cc373ce0 JavaThread "pool-1-thread-1" [_thread_blocked, id=4826, stack(0x00007f229d000000,0x00007f229d100000)] - 0x00007f224c542b10 JavaThread "io-compute-0" daemon [_thread_in_native, id=4834, stack(0x00007f229c3f6000,0x00007f229c4f6000)] - 0x00007f224c5436b0 JavaThread "io-compute-1" daemon [_thread_in_native, id=4835, stack(0x00007f229c2f6000,0x00007f229c3f6000)] - 0x00007f224c544660 JavaThread "io-compute-2" daemon [_thread_in_native, id=4836, stack(0x00007f229c1f6000,0x00007f229c2f6000)] -=>0x00007f224c545570 JavaThread "io-compute-3" daemon [_thread_in_vm, id=4837, stack(0x00007f229c0f6000,0x00007f229c1f6000)] - 0x00007f224c546530 JavaThread "io-compute-7" daemon [_thread_in_native, id=4838, stack(0x00007f222bf00000,0x00007f222c000000)] - 0x00007f224c547440 JavaThread "io-compute-5" daemon [_thread_in_native, id=4839, stack(0x00007f222be00000,0x00007f222bf00000)] - 0x00007f224c548800 JavaThread "io-compute-4" daemon [_thread_in_native, id=4840, stack(0x00007f222bd00000,0x00007f222be00000)] - 0x00007f224c549b00 JavaThread "io-compute-6" daemon [_thread_in_Java, id=4841, stack(0x00007f222bc00000,0x00007f222bd00000)] - 0x00007f2230006780 JavaThread "pool-2-thread-1" [_thread_blocked, id=4845, stack(0x00007f229c9f6000,0x00007f229caf6000)] - 0x00007f227838dfb0 JavaThread "C2 CompilerThread1" daemon [_thread_in_native, id=4851, stack(0x00007f222bb00000,0x00007f222bc00000)] - 0x00007f221c029620 JavaThread "io-compute-blocker-8" daemon [_thread_blocked, id=4853, stack(0x00007f222b7fc000,0x00007f222b8fc000)] - 0x00007f220c3e7e00 JavaThread "C2 CompilerThread2" daemon [_thread_in_native, id=4854, stack(0x00007f222b5fa000,0x00007f222b6fa000)] - -Other Threads: - 0x00007f22cc1758f0 VMThread "VM Thread" [stack: 0x00007f22acd34000,0x00007f22ace34000] [id=4803] - 0x00007f22cc1adaa0 WatcherThread [stack: 0x00007f22ac032000,0x00007f22ac132000] [id=4823] - 0x00007f22cc080fb0 GCTaskThread "GC Thread#0" [stack: 0x00007f22d02e5000,0x00007f22d03e5000] [id=4798] - 0x00007f228c005f90 GCTaskThread "GC Thread#1" [stack: 0x00007f229cf00000,0x00007f229d000000] [id=4827] - 0x00007f228c0069c0 GCTaskThread "GC Thread#2" [stack: 0x00007f229cdfe000,0x00007f229cefe000] [id=4828] - 0x00007f228c0073f0 GCTaskThread "GC Thread#3" [stack: 0x00007f229ccfc000,0x00007f229cdfc000] [id=4829] - 0x00007f228c007e20 GCTaskThread "GC Thread#4" [stack: 0x00007f229cbfa000,0x00007f229ccfa000] [id=4830] - 0x00007f228c008850 GCTaskThread "GC Thread#5" [stack: 0x00007f229caf8000,0x00007f229cbf8000] [id=4831] - 0x00007f228c00c9d0 GCTaskThread "GC Thread#6" [stack: 0x00007f222ba00000,0x00007f222bb00000] [id=4843] - 0x00007f228c00cf50 GCTaskThread "GC Thread#7" [stack: 0x00007f222b8fe000,0x00007f222b9fe000] [id=4844] - 0x00007f22cc091d10 ConcurrentGCThread "G1 Main Marker" [stack: 0x00007f22d01e3000,0x00007f22d02e3000] [id=4799] - 0x00007f22cc092c00 ConcurrentGCThread "G1 Conc#0" [stack: 0x00007f22d00e1000,0x00007f22d01e1000] [id=4800] - 0x00007f22a0000ea0 ConcurrentGCThread "G1 Conc#1" [stack: 0x00007f222b6fc000,0x00007f222b7fc000] [id=4848] - 0x00007f22cc146590 ConcurrentGCThread "G1 Refine#0" [stack: 0x00007f22ad009000,0x00007f22ad109000] [id=4801] - 0x00007f22cc147480 ConcurrentGCThread "G1 Service" [stack: 0x00007f22acf07000,0x00007f22ad007000] [id=4802] - -Threads with active compile tasks: -C2 CompilerThread0 12483 1801 4 java.lang.invoke.MethodType::makeImpl (109 bytes) -C1 CompilerThread0 12483 2118 3 cats.effect.unsafe.FiberMonitor::monitorSuspended (45 bytes) -C2 CompilerThread1 12483 2043 ! 4 cats.effect.unsafe.WorkerThread::run (1765 bytes) -C2 CompilerThread2 12483 2120 4 cats.effect.IO$$anon$5::flatMap (10 bytes) - -VM state: not at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: None - -Heap address: 0x0000000708200000, size: 3966 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x0000000800000000-0x0000000800be2000-0x0000000800be2000), size 12460032, SharedBaseAddress: 0x0000000800000000, ArchiveRelocationMode: 0. -Compressed class space mapped at: 0x0000000801000000-0x0000000841000000, reserved size: 1073741824 -Narrow klass base: 0x0000000800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 8 total, 8 available - Memory: 15860M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 248M - Heap Max Capacity: 3966M - Pre-touch: Disabled - Parallel Workers: 8 - Concurrent Workers: 2 - Concurrent Refinement Workers: 8 - Periodic GC: Disabled - -Heap: - garbage-first heap total 49152K, used 17737K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 9 young (18432K), 3 survivors (6144K) - Metaspace used 29726K, committed 30144K, reserved 1114112K - class space used 3318K, committed 3520K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000708200000, 0x000000070835a7f8, 0x0000000708400000| 67%| S|CS|TAMS 0x0000000708200000, 0x0000000708200000| Complete -| 1|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| S|CS|TAMS 0x0000000708400000, 0x0000000708400000| Complete -| 2|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| S|CS|TAMS 0x0000000708600000, 0x0000000708600000| Complete -| 3|0x0000000708800000, 0x0000000708800000, 0x0000000708a00000| 0%| F| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked -| 4|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked -| 5|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked -| 6|0x0000000708e00000, 0x0000000708e00000, 0x0000000709000000| 0%| F| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked -| 7|0x0000000709000000, 0x0000000709000000, 0x0000000709200000| 0%| F| |TAMS 0x0000000709000000, 0x0000000709000000| Untracked -| 8|0x0000000709200000, 0x0000000709200000, 0x0000000709400000| 0%| F| |TAMS 0x0000000709200000, 0x0000000709200000| Untracked -| 9|0x0000000709400000, 0x0000000709400000, 0x0000000709600000| 0%| F| |TAMS 0x0000000709400000, 0x0000000709400000| Untracked -| 10|0x0000000709600000, 0x0000000709600000, 0x0000000709800000| 0%| F| |TAMS 0x0000000709600000, 0x0000000709600000| Untracked -| 11|0x0000000709800000, 0x0000000709800000, 0x0000000709a00000| 0%| F| |TAMS 0x0000000709800000, 0x0000000709800000| Untracked -| 12|0x0000000709a00000, 0x0000000709a00000, 0x0000000709c00000| 0%| F| |TAMS 0x0000000709a00000, 0x0000000709a00000| Untracked -| 13|0x0000000709c00000, 0x0000000709c00000, 0x0000000709e00000| 0%| F| |TAMS 0x0000000709c00000, 0x0000000709c00000| Untracked -| 14|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked -| 15|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked -| 16|0x000000070a200000, 0x000000070a30efe0, 0x000000070a400000| 52%| E| |TAMS 0x000000070a200000, 0x000000070a200000| Complete -| 109|0x0000000715c00000, 0x0000000715e00000, 0x0000000715e00000|100%| E|CS|TAMS 0x0000000715c00000, 0x0000000715c00000| Complete -| 110|0x0000000715e00000, 0x0000000716000000, 0x0000000716000000|100%| E|CS|TAMS 0x0000000715e00000, 0x0000000715e00000| Complete -| 111|0x0000000716000000, 0x0000000716200000, 0x0000000716200000|100%| E|CS|TAMS 0x0000000716000000, 0x0000000716000000| Complete -| 122|0x0000000717600000, 0x0000000717800000, 0x0000000717800000|100%| E|CS|TAMS 0x0000000717600000, 0x0000000717600000| Complete -| 123|0x0000000717800000, 0x0000000717a00000, 0x0000000717a00000|100%| E|CS|TAMS 0x0000000717800000, 0x0000000717800000| Complete -|1981|0x00000007ffc00000, 0x00000007ffd76000, 0x00000007ffe00000| 73%|OA| |TAMS 0x00000007ffd76000, 0x00000007ffc00000| Untracked -|1982|0x00000007ffe00000, 0x00000007ffe82000, 0x0000000800000000| 25%|CA| |TAMS 0x00000007ffe82000, 0x00000007ffe00000| Untracked - -Card table byte_map: [0x00007f22d0ba4000,0x00007f22d1363000] _byte_map_base: 0x00007f22cd363000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x00007f22cc081a00, (CMBitMap*) 0x00007f22cc0819c0 - Prev Bits: [0x00007f22ad810000, 0x00007f22b1608000) - Next Bits: [0x00007f22b1608000, 0x00007f22b5400000) - -Polling page: 0x00007f22d40a2000 - -Metaspace: - -Usage: - Non-class: 25.79 MB used. - Class: 3.24 MB used. - Both: 29.03 MB used. - -Virtual space: - Non-class space: 64.00 MB reserved, 26.00 MB ( 41%) committed, 1 nodes. - Class space: 1.00 GB reserved, 3.44 MB ( <1%) committed, 1 nodes. - Both: 1.06 GB reserved, 29.44 MB ( 3%) committed. - -Chunk freelists: - Non-Class: 5.60 MB - Class: 12.44 MB - Both: 18.03 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 35.12 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 3. -num_arena_births: 442. -num_arena_deaths: 0. -num_vsnodes_births: 2. -num_vsnodes_deaths: 0. -num_space_committed: 471. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 3. -num_chunks_taken_from_freelist: 1465. -num_chunk_merges: 3. -num_chunk_splits: 1034. -num_chunks_enlarged: 816. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=120028Kb used=1311Kb max_used=1311Kb free=118716Kb - bounds [0x00007f22bcec9000, 0x00007f22bd139000, 0x00007f22c4400000] -CodeHeap 'profiled nmethods': size=120028Kb used=4340Kb max_used=4340Kb free=115687Kb - bounds [0x00007f22b5400000, 0x00007f22b5840000, 0x00007f22bc937000] -CodeHeap 'non-nmethods': size=5704Kb used=1348Kb max_used=1414Kb free=4355Kb - bounds [0x00007f22bc937000, 0x00007f22bcba7000, 0x00007f22bcec9000] - total_blobs=2722 nmethods=2097 adapters=539 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 12.260 Thread 0x00007f22cc185870 2022 1 cats.effect.kernel.Outcome$Succeeded::fa (5 bytes) -Event: 12.260 Thread 0x00007f22cc185870 nmethod 2022 0x00007f22bd006a90 code [0x00007f22bd006c20, 0x00007f22bd006cf0] -Event: 12.260 Thread 0x00007f22cc185870 2016 1 fs2.Chunk$Singleton::value (5 bytes) -Event: 12.260 Thread 0x00007f22cc185870 nmethod 2016 0x00007f22bd006d90 code [0x00007f22bd006f20, 0x00007f22bd006ff0] -Event: 12.260 Thread 0x00007f22cc185870 2034 1 fs2.internal.ScopedResource$State::open (5 bytes) -Event: 12.260 Thread 0x00007f22cc185870 nmethod 2034 0x00007f22bd007090 code [0x00007f22bd007220, 0x00007f22bd0072f0] -Event: 12.260 Thread 0x00007f22cc185870 2042 3 cats.effect.CallbackStack$::apply (9 bytes) -Event: 12.261 Thread 0x00007f22cc185870 nmethod 2042 0x00007f22b581a210 code [0x00007f22b581a3c0, 0x00007f22b581a6b0] -Event: 12.261 Thread 0x00007f22cc185870 2025 1 scala.collection.mutable.ArrayBuffer::size0_$eq (6 bytes) -Event: 12.261 Thread 0x00007f22cc185870 nmethod 2025 0x00007f22bd007390 code [0x00007f22bd007520, 0x00007f22bd0075f0] -Event: 12.261 Thread 0x00007f22cc185870 2026 1 scala.collection.mutable.ArrayBuffer::array_$eq (6 bytes) -Event: 12.262 Thread 0x00007f22cc185870 nmethod 2026 0x00007f22bd007690 code [0x00007f22bd007820, 0x00007f22bd007930] -Event: 12.262 Thread 0x00007f22cc185870 2041 1 fs2.internal.ScopedResource$State::leases (5 bytes) -Event: 12.262 Thread 0x00007f22cc185870 nmethod 2041 0x00007f22bd007990 code [0x00007f22bd007b20, 0x00007f22bd007bf0] -Event: 12.265 Thread 0x00007f22cc185870 2044 3 cats.data.Chain$::nil (4 bytes) -Event: 12.265 Thread 0x00007f22cc185870 nmethod 2044 0x00007f22b581a810 code [0x00007f22b581a9a0, 0x00007f22b581aab0] -Event: 12.265 Thread 0x00007f22cc185870 2045 3 cats.Eval:: (5 bytes) -Event: 12.266 Thread 0x00007f22cc185870 nmethod 2045 0x00007f22b581ab90 code [0x00007f22b581ad40, 0x00007f22b581ae90] -Event: 12.272 Thread 0x00007f227838dfb0 nmethod 2017 0x00007f22bd007c90 code [0x00007f22bd007e40, 0x00007f22bd0080e8] -Event: 12.272 Thread 0x00007f227838dfb0 2043 ! 4 cats.effect.unsafe.WorkerThread::run (1765 bytes) - -GC Heap History (8 events): -Event: 2.511 GC heap before -{Heap before GC invocations=0 (full 0): - garbage-first heap total 258048K, used 22496K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 11 young (22528K), 0 survivors (0K) - Metaspace used 7378K, committed 7488K, reserved 1114112K - class space used 779K, committed 832K, reserved 1048576K -} -Event: 2.520 GC heap after -{Heap after GC invocations=1 (full 0): - garbage-first heap total 258048K, used 4789K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 7378K, committed 7488K, reserved 1114112K - class space used 779K, committed 832K, reserved 1048576K -} -Event: 5.614 GC heap before -{Heap before GC invocations=1 (full 0): - garbage-first heap total 258048K, used 35509K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 18 young (36864K), 2 survivors (4096K) - Metaspace used 16783K, committed 17024K, reserved 1114112K - class space used 1732K, committed 1856K, reserved 1048576K -} -Event: 5.626 GC heap after -{Heap after GC invocations=2 (full 0): - garbage-first heap total 258048K, used 5916K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 16783K, committed 17024K, reserved 1114112K - class space used 1732K, committed 1856K, reserved 1048576K -} -Event: 8.132 GC heap before -{Heap before GC invocations=2 (full 0): - garbage-first heap total 258048K, used 28444K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 14 young (28672K), 2 survivors (4096K) - Metaspace used 21250K, committed 21504K, reserved 1114112K - class space used 2228K, committed 2368K, reserved 1048576K -} -Event: 8.146 GC heap after -{Heap after GC invocations=3 (full 0): - garbage-first heap total 258048K, used 6546K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 21250K, committed 21504K, reserved 1114112K - class space used 2228K, committed 2368K, reserved 1048576K -} -Event: 11.186 GC heap before -{Heap before GC invocations=4 (full 0): - garbage-first heap total 49152K, used 35218K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 19 young (38912K), 3 survivors (6144K) - Metaspace used 27948K, committed 28288K, reserved 1114112K - class space used 2972K, committed 3136K, reserved 1048576K -} -Event: 11.200 GC heap after -{Heap after GC invocations=5 (full 0): - garbage-first heap total 49152K, used 7497K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 27948K, committed 28288K, reserved 1114112K - class space used 2972K, committed 3136K, reserved 1048576K -} - -Dll operation events (11 events): -Event: 0.008 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -Event: 0.046 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -Event: 0.902 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -Event: 0.945 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -Event: 1.071 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -Event: 1.133 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -Event: 1.178 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -Event: 1.457 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -Event: 3.415 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -Event: 3.422 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -Event: 3.512 Loaded shared library /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so - -Deoptimization events (20 events): -Event: 11.983 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa268 mode 2 -Event: 11.983 Thread 0x00007f224c5436b0 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c3f4268 mode 2 -Event: 12.032 Thread 0x00007f221c029620 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcf395b0 relative=0x0000000000000210 -Event: 12.032 Thread 0x00007f221c029620 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcf395b0 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 -Event: 12.032 Thread 0x00007f221c029620 DEOPT PACKING pc=0x00007f22bcf395b0 sp=0x00007f222b8fa2c0 -Event: 12.032 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa200 mode 2 -Event: 12.066 Thread 0x00007f224c545570 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcf33fb0 relative=0x0000000000000190 -Event: 12.066 Thread 0x00007f224c545570 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcf33fb0 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 -Event: 12.066 Thread 0x00007f224c545570 DEOPT PACKING pc=0x00007f22bcf33fb0 sp=0x00007f229c1f4270 -Event: 12.066 Thread 0x00007f224c545570 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c1f4218 mode 2 -Event: 12.154 Thread 0x00007f221c029620 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcfefb6c relative=0x00000000000004ec -Event: 12.154 Thread 0x00007f221c029620 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcfefb6c method=scala.Option.isEmpty()Z @ 4 c2 -Event: 12.154 Thread 0x00007f221c029620 DEOPT PACKING pc=0x00007f22bcfefb6c sp=0x00007f222b8fa4f0 -Event: 12.154 Thread 0x00007f221c029620 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f222b8fa078 mode 2 -Event: 12.177 Thread 0x00007f224c5436b0 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f22bcfffc0c relative=0x000000000000048c -Event: 12.177 Thread 0x00007f224c5436b0 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f22bcfffc0c method=scala.Option.isEmpty()Z @ 4 c2 -Event: 12.177 Thread 0x00007f224c5436b0 DEOPT PACKING pc=0x00007f22bcfffc0c sp=0x00007f229c3f44a0 -Event: 12.177 Thread 0x00007f224c5436b0 DEOPT UNPACKING pc=0x00007f22bc98d699 sp=0x00007f229c3f40e8 mode 2 -Event: 12.261 Thread 0x00007f224c547440 DEOPT PACKING pc=0x00007f22b577fcd0 sp=0x00007f222befe660 -Event: 12.261 Thread 0x00007f224c547440 DEOPT UNPACKING pc=0x00007f22bc98de2f sp=0x00007f222befdcf0 mode 0 - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 11.555 Thread 0x00007f224c542b10 Exception (0x000000071775de90) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.559 Thread 0x00007f224c5436b0 Exception (0x000000071778e0d0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.585 Thread 0x00007f224c547440 Exception (0x0000000716001210) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.601 Thread 0x00007f221c029620 Exception (0x00000007177ba870) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.647 Thread 0x00007f221c029620 Exception (0x0000000716087af0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.653 Thread 0x00007f221c029620 Exception (0x000000071608c658) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.657 Thread 0x00007f221c029620 Exception (0x0000000716090e10) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.730 Thread 0x00007f224c545570 Exception (0x000000071611fb28) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.868 Thread 0x00007f224c545570 Exception (0x00000007161e4b60) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.877 Thread 0x00007f224c545570 Exception (0x00000007161f8ab8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.880 Thread 0x00007f224c545570 Exception (0x0000000715e00aa8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.914 Thread 0x00007f224c545570 Exception (0x0000000715e52478) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.917 Thread 0x00007f224c545570 Exception (0x0000000715e72f70) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.920 Thread 0x00007f224c545570 Exception (0x0000000715e77ad8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.923 Thread 0x00007f224c545570 Exception (0x0000000715e80c68) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.927 Thread 0x00007f224c545570 Exception (0x0000000715e8d210) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.930 Thread 0x00007f224c545570 Exception (0x0000000715e91d78) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.984 Thread 0x00007f224c546530 Exception (0x0000000715ea07c0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.999 Thread 0x00007f224c546530 Exception (0x0000000715f64a08) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 12.002 Thread 0x00007f224c546530 Exception (0x0000000715f6cbc0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] - -VM Operations (20 events): -Event: 6.626 Executing VM operation: Cleanup -Event: 6.627 Executing VM operation: Cleanup done -Event: 7.345 Executing VM operation: HandshakeAllThreads -Event: 7.345 Executing VM operation: HandshakeAllThreads done -Event: 8.132 Executing VM operation: CollectForMetadataAllocation -Event: 8.146 Executing VM operation: CollectForMetadataAllocation done -Event: 8.169 Executing VM operation: G1Concurrent -Event: 8.176 Executing VM operation: G1Concurrent done -Event: 8.177 Executing VM operation: G1Concurrent -Event: 8.179 Executing VM operation: G1Concurrent done -Event: 9.179 Executing VM operation: Cleanup -Event: 9.179 Executing VM operation: Cleanup done -Event: 10.119 Executing VM operation: HandshakeAllThreads -Event: 10.121 Executing VM operation: HandshakeAllThreads done -Event: 11.121 Executing VM operation: Cleanup -Event: 11.122 Executing VM operation: Cleanup done -Event: 11.185 Executing VM operation: G1CollectForAllocation -Event: 11.200 Executing VM operation: G1CollectForAllocation done -Event: 12.175 Executing VM operation: HandshakeAllThreads -Event: 12.176 Executing VM operation: HandshakeAllThreads done - -Events (20 events): -Event: 10.120 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 -Event: 10.125 loading class javax/security/cert/X509Certificate -Event: 10.125 loading class javax/security/cert/Certificate -Event: 10.125 loading class javax/security/cert/Certificate done -Event: 10.126 loading class javax/security/cert/X509Certificate done -Event: 10.130 loading class java/nio/ReadOnlyBufferException -Event: 10.131 loading class java/nio/ReadOnlyBufferException done -Event: 10.160 loading class java/util/concurrent/atomic/Striped64$1 -Event: 10.160 loading class java/util/concurrent/atomic/Striped64$1 done -Event: 10.992 Thread 0x00007f226c42fc00 Thread exited: 0x00007f226c42fc00 -Event: 10.993 Thread 0x00007f227838dfb0 Thread exited: 0x00007f227838dfb0 -Event: 11.076 Thread 0x00007f227838dfb0 Thread added: 0x00007f227838dfb0 -Event: 11.076 Protecting memory [0x00007f222bb00000,0x00007f222bb04000] with protection modes 0 -Event: 11.076 Thread 0x00007f22781ddc90 Thread added: 0x00007f22781ddc90 -Event: 11.110 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 -Event: 11.301 Thread 0x00007f22781ddc90 Thread exited: 0x00007f22781ddc90 -Event: 11.466 Thread 0x00007f221c029620 Thread added: 0x00007f221c029620 -Event: 11.466 Protecting memory [0x00007f222b7fc000,0x00007f222b800000] with protection modes 0 -Event: 11.797 Thread 0x00007f220c3e7e00 Thread added: 0x00007f220c3e7e00 -Event: 11.798 Protecting memory [0x00007f222b5fa000,0x00007f222b5fe000] with protection modes 0 - - -Dynamic libraries: -708200000-70a400000 rw-p 00000000 00:00 0 -70a400000-715c00000 ---p 00000000 00:00 0 -715c00000-716200000 rw-p 00000000 00:00 0 -716200000-717600000 ---p 00000000 00:00 0 -717600000-717a00000 rw-p 00000000 00:00 0 -717a00000-7ffc00000 ---p 00000000 00:00 0 -7ffc00000-7ffd00000 rw-p 00000000 00:00 0 -7ffd00000-7ffd76000 rw-p 00c9d000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -7ffd76000-7ffe00000 rw-p 00000000 00:00 0 -7ffe00000-7ffe82000 rw-p 00c1b000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -7ffe82000-800000000 rw-p 00000000 00:00 0 -800000000-800be2000 rw-p 00001000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -800be2000-801000000 ---p 00000000 00:00 0 -801000000-801030000 rw-p 00000000 00:00 0 -801030000-801130000 rw-p 00000000 00:00 0 -801130000-801170000 rw-p 00000000 00:00 0 -801170000-8011b0000 rw-p 00000000 00:00 0 -8011b0000-8011d0000 rw-p 00000000 00:00 0 -8011d0000-801200000 ---p 00000000 00:00 0 -801200000-8013a0000 rw-p 00000000 00:00 0 -8013a0000-841000000 ---p 00000000 00:00 0 -556b97447000-556b97448000 r--p 00000000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -556b97448000-556b97449000 r-xp 00001000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -556b97449000-556b9744a000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -556b9744a000-556b9744b000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -556b9744b000-556b9744c000 rw-p 00003000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -556b98d13000-556b98d5b000 rw-p 00000000 00:00 0 [heap] -7f21f8000000-7f21f8121000 rw-p 00000000 00:00 0 -7f21f8121000-7f21fc000000 ---p 00000000 00:00 0 -7f2200000000-7f220030a000 rw-p 00000000 00:00 0 -7f220030a000-7f2204000000 ---p 00000000 00:00 0 -7f2204000000-7f22040f0000 rw-p 00000000 00:00 0 -7f22040f0000-7f2208000000 ---p 00000000 00:00 0 -7f2208000000-7f2208021000 rw-p 00000000 00:00 0 -7f2208021000-7f220c000000 ---p 00000000 00:00 0 -7f220c000000-7f220c64f000 rw-p 00000000 00:00 0 -7f220c64f000-7f2210000000 ---p 00000000 00:00 0 -7f2210000000-7f2210161000 rw-p 00000000 00:00 0 -7f2210161000-7f2214000000 ---p 00000000 00:00 0 -7f2214000000-7f221419d000 rw-p 00000000 00:00 0 -7f221419d000-7f2218000000 ---p 00000000 00:00 0 -7f2218000000-7f22180ab000 rw-p 00000000 00:00 0 -7f22180ab000-7f221c000000 ---p 00000000 00:00 0 -7f221c000000-7f221c35c000 rw-p 00000000 00:00 0 -7f221c35c000-7f2220000000 ---p 00000000 00:00 0 -7f2220000000-7f22200f3000 rw-p 00000000 00:00 0 -7f22200f3000-7f2224000000 ---p 00000000 00:00 0 -7f2224000000-7f22240f3000 rw-p 00000000 00:00 0 -7f22240f3000-7f2228000000 ---p 00000000 00:00 0 -7f222b214000-7f222b5fa000 rw-p 00000000 00:00 0 -7f222b5fa000-7f222b5fe000 ---p 00000000 00:00 0 -7f222b5fe000-7f222b6fa000 rw-p 00000000 00:00 0 -7f222b6fa000-7f222b6fb000 ---p 00000000 00:00 0 -7f222b6fb000-7f222b7fc000 rw-p 00000000 00:00 0 -7f222b7fc000-7f222b800000 ---p 00000000 00:00 0 -7f222b800000-7f222b8fc000 rw-p 00000000 00:00 0 -7f222b8fc000-7f222b8fd000 ---p 00000000 00:00 0 -7f222b8fd000-7f222b9fe000 rw-p 00000000 00:00 0 -7f222b9fe000-7f222b9ff000 ---p 00000000 00:00 0 -7f222b9ff000-7f222bb00000 rw-p 00000000 00:00 0 -7f222bb00000-7f222bb04000 ---p 00000000 00:00 0 -7f222bb04000-7f222bc00000 rw-p 00000000 00:00 0 -7f222bc00000-7f222bc04000 ---p 00000000 00:00 0 -7f222bc04000-7f222bd00000 rw-p 00000000 00:00 0 -7f222bd00000-7f222bd04000 ---p 00000000 00:00 0 -7f222bd04000-7f222be00000 rw-p 00000000 00:00 0 -7f222be00000-7f222be04000 ---p 00000000 00:00 0 -7f222be04000-7f222bf00000 rw-p 00000000 00:00 0 -7f222bf00000-7f222bf04000 ---p 00000000 00:00 0 -7f222bf04000-7f222c056000 rw-p 00000000 00:00 0 -7f222c056000-7f2230000000 ---p 00000000 00:00 0 -7f2230000000-7f223004f000 rw-p 00000000 00:00 0 -7f223004f000-7f2234000000 ---p 00000000 00:00 0 -7f2234000000-7f2234181000 rw-p 00000000 00:00 0 -7f2234181000-7f2238000000 ---p 00000000 00:00 0 -7f2238000000-7f223807a000 rw-p 00000000 00:00 0 -7f223807a000-7f223c000000 ---p 00000000 00:00 0 -7f223c000000-7f223c021000 rw-p 00000000 00:00 0 -7f223c021000-7f2240000000 ---p 00000000 00:00 0 -7f2240000000-7f2240021000 rw-p 00000000 00:00 0 -7f2240021000-7f2244000000 ---p 00000000 00:00 0 -7f2244000000-7f2244021000 rw-p 00000000 00:00 0 -7f2244021000-7f2248000000 ---p 00000000 00:00 0 -7f2248000000-7f2248021000 rw-p 00000000 00:00 0 -7f2248021000-7f224c000000 ---p 00000000 00:00 0 -7f224c000000-7f224c623000 rw-p 00000000 00:00 0 -7f224c623000-7f2250000000 ---p 00000000 00:00 0 -7f2250000000-7f2250021000 rw-p 00000000 00:00 0 -7f2250021000-7f2254000000 ---p 00000000 00:00 0 -7f2254000000-7f2254021000 rw-p 00000000 00:00 0 -7f2254021000-7f2258000000 ---p 00000000 00:00 0 -7f2258000000-7f2258021000 rw-p 00000000 00:00 0 -7f2258021000-7f225c000000 ---p 00000000 00:00 0 -7f225c000000-7f225c021000 rw-p 00000000 00:00 0 -7f225c021000-7f2260000000 ---p 00000000 00:00 0 -7f2260000000-7f2260021000 rw-p 00000000 00:00 0 -7f2260021000-7f2264000000 ---p 00000000 00:00 0 -7f2264000000-7f2264021000 rw-p 00000000 00:00 0 -7f2264021000-7f2268000000 ---p 00000000 00:00 0 -7f2268000000-7f2268082000 rw-p 00000000 00:00 0 -7f2268082000-7f226c000000 ---p 00000000 00:00 0 -7f226c000000-7f226c6c0000 rw-p 00000000 00:00 0 -7f226c6c0000-7f2270000000 ---p 00000000 00:00 0 -7f2270000000-7f2270021000 rw-p 00000000 00:00 0 -7f2270021000-7f2274000000 ---p 00000000 00:00 0 -7f2274000000-7f2274021000 rw-p 00000000 00:00 0 -7f2274021000-7f2278000000 ---p 00000000 00:00 0 -7f2278000000-7f22789f2000 rw-p 00000000 00:00 0 -7f22789f2000-7f227c000000 ---p 00000000 00:00 0 -7f227c000000-7f227c021000 rw-p 00000000 00:00 0 -7f227c021000-7f2280000000 ---p 00000000 00:00 0 -7f2280000000-7f2280021000 rw-p 00000000 00:00 0 -7f2280021000-7f2284000000 ---p 00000000 00:00 0 -7f2284000000-7f2284021000 rw-p 00000000 00:00 0 -7f2284021000-7f2288000000 ---p 00000000 00:00 0 -7f2288000000-7f2288021000 rw-p 00000000 00:00 0 -7f2288021000-7f228c000000 ---p 00000000 00:00 0 -7f228c000000-7f228c021000 rw-p 00000000 00:00 0 -7f228c021000-7f2290000000 ---p 00000000 00:00 0 -7f2290000000-7f2290021000 rw-p 00000000 00:00 0 -7f2290021000-7f2294000000 ---p 00000000 00:00 0 -7f2294000000-7f22943f0000 rw-p 00000000 00:00 0 -7f22943f0000-7f2294d90000 rw-p 00000000 00:00 0 -7f2294d90000-7f2294db0000 rw-p 00000000 00:00 0 -7f2294db0000-7f2294ef0000 rw-p 00000000 00:00 0 -7f2294ef0000-7f2294f90000 rw-p 00000000 00:00 0 -7f2294f90000-7f2295000000 ---p 00000000 00:00 0 -7f2295000000-7f2295a70000 rw-p 00000000 00:00 0 -7f2295a70000-7f2298000000 ---p 00000000 00:00 0 -7f2298000000-7f2298021000 rw-p 00000000 00:00 0 -7f2298021000-7f229c000000 ---p 00000000 00:00 0 -7f229c0f6000-7f229c0fa000 ---p 00000000 00:00 0 -7f229c0fa000-7f229c1f6000 rw-p 00000000 00:00 0 -7f229c1f6000-7f229c1fa000 ---p 00000000 00:00 0 -7f229c1fa000-7f229c2f6000 rw-p 00000000 00:00 0 -7f229c2f6000-7f229c2fa000 ---p 00000000 00:00 0 -7f229c2fa000-7f229c3f6000 rw-p 00000000 00:00 0 -7f229c3f6000-7f229c3fa000 ---p 00000000 00:00 0 -7f229c3fa000-7f229c4f6000 rw-p 00000000 00:00 0 -7f229c4f6000-7f229c536000 rw-s 10000000 00:0e 48673 anon_inode:[io_uring] -7f229c536000-7f229c55b000 rw-s 00000000 00:0e 48673 anon_inode:[io_uring] -7f229c55b000-7f229c59b000 rw-s 10000000 00:0e 48671 anon_inode:[io_uring] -7f229c59b000-7f229c5c0000 rw-s 00000000 00:0e 48671 anon_inode:[io_uring] -7f229c5c0000-7f229c600000 rw-s 10000000 00:0e 48669 anon_inode:[io_uring] -7f229c600000-7f229c611000 r-xp 00000000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) -7f229c611000-7f229c811000 ---p 00011000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) -7f229c811000-7f229c813000 rw-p 00011000 103:02 3670138 /tmp/libnetty_transport_native_io_uring_x86_64629166417486836025.so (deleted) -7f229c822000-7f229c847000 rw-s 00000000 00:0e 48669 anon_inode:[io_uring] -7f229c847000-7f229c887000 rw-s 10000000 00:0e 48667 anon_inode:[io_uring] -7f229c887000-7f229c8ac000 rw-s 00000000 00:0e 48667 anon_inode:[io_uring] -7f229c8ac000-7f229c8ec000 rw-s 10000000 00:0e 48665 anon_inode:[io_uring] -7f229c8ec000-7f229c911000 rw-s 00000000 00:0e 48665 anon_inode:[io_uring] -7f229c911000-7f229c951000 rw-s 10000000 00:0e 48663 anon_inode:[io_uring] -7f229c951000-7f229c976000 rw-s 00000000 00:0e 48663 anon_inode:[io_uring] -7f229c976000-7f229c9b6000 rw-s 10000000 00:0e 48661 anon_inode:[io_uring] -7f229c9b6000-7f229c9f6000 rw-s 10000000 00:0e 48659 anon_inode:[io_uring] -7f229c9f6000-7f229c9fa000 ---p 00000000 00:00 0 -7f229c9fa000-7f229caf6000 rw-p 00000000 00:00 0 -7f229caf6000-7f229caf7000 ---p 00000000 00:00 0 -7f229caf7000-7f229cbf8000 rw-p 00000000 00:00 0 -7f229cbf8000-7f229cbf9000 ---p 00000000 00:00 0 -7f229cbf9000-7f229ccfa000 rw-p 00000000 00:00 0 -7f229ccfa000-7f229ccfb000 ---p 00000000 00:00 0 -7f229ccfb000-7f229cdfc000 rw-p 00000000 00:00 0 -7f229cdfc000-7f229cdfd000 ---p 00000000 00:00 0 -7f229cdfd000-7f229cefe000 rw-p 00000000 00:00 0 -7f229cefe000-7f229ceff000 ---p 00000000 00:00 0 -7f229ceff000-7f229d000000 rw-p 00000000 00:00 0 -7f229d000000-7f229d004000 ---p 00000000 00:00 0 -7f229d004000-7f229d100000 rw-p 00000000 00:00 0 -7f229d100000-7f229d104000 ---p 00000000 00:00 0 -7f229d104000-7f229d200000 rw-p 00000000 00:00 0 -7f229d200000-7f229dfea000 r--p 00000000 103:02 1055507 /usr/lib/locale/locale-archive -7f229e000000-7f22a0021000 rw-p 00000000 00:00 0 -7f22a0021000-7f22a4000000 ---p 00000000 00:00 0 -7f22a4000000-7f22a4021000 rw-p 00000000 00:00 0 -7f22a4021000-7f22a8000000 ---p 00000000 00:00 0 -7f22a8000000-7f22a8021000 rw-p 00000000 00:00 0 -7f22a8021000-7f22ac000000 ---p 00000000 00:00 0 -7f22ac00b000-7f22ac030000 rw-s 00000000 00:0e 48661 anon_inode:[io_uring] -7f22ac030000-7f22ac031000 ---p 00000000 00:00 0 -7f22ac031000-7f22ac132000 rw-p 00000000 00:00 0 -7f22ac132000-7f22ac136000 ---p 00000000 00:00 0 -7f22ac136000-7f22ac232000 rw-p 00000000 00:00 0 -7f22ac232000-7f22ac236000 ---p 00000000 00:00 0 -7f22ac236000-7f22ac332000 rw-p 00000000 00:00 0 -7f22ac332000-7f22ac336000 ---p 00000000 00:00 0 -7f22ac336000-7f22ac432000 rw-p 00000000 00:00 0 -7f22ac432000-7f22ac436000 ---p 00000000 00:00 0 -7f22ac436000-7f22ac532000 rw-p 00000000 00:00 0 -7f22ac532000-7f22ac536000 ---p 00000000 00:00 0 -7f22ac536000-7f22ac632000 rw-p 00000000 00:00 0 -7f22ac632000-7f22ac636000 ---p 00000000 00:00 0 -7f22ac636000-7f22ac732000 rw-p 00000000 00:00 0 -7f22ac732000-7f22ac736000 ---p 00000000 00:00 0 -7f22ac736000-7f22ac832000 rw-p 00000000 00:00 0 -7f22ac832000-7f22ac836000 ---p 00000000 00:00 0 -7f22ac836000-7f22ac932000 rw-p 00000000 00:00 0 -7f22ac932000-7f22ac936000 ---p 00000000 00:00 0 -7f22ac936000-7f22aca32000 rw-p 00000000 00:00 0 -7f22aca32000-7f22aca36000 ---p 00000000 00:00 0 -7f22aca36000-7f22acb32000 rw-p 00000000 00:00 0 -7f22acb32000-7f22acb36000 ---p 00000000 00:00 0 -7f22acb36000-7f22acc32000 rw-p 00000000 00:00 0 -7f22acc32000-7f22acc36000 ---p 00000000 00:00 0 -7f22acc36000-7f22acd32000 rw-p 00000000 00:00 0 -7f22acd32000-7f22acd33000 ---p 00000000 00:00 0 -7f22acd33000-7f22ace34000 rw-p 00000000 00:00 0 -7f22ace34000-7f22ace39000 r--p 00000000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f22ace39000-7f22ace7a000 r-xp 00005000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f22ace7a000-7f22acf03000 r--p 00046000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f22acf03000-7f22acf04000 r--p 000ce000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f22acf04000-7f22acf05000 rw-p 000cf000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f22acf05000-7f22acf06000 ---p 00000000 00:00 0 -7f22acf06000-7f22ad007000 rw-p 00000000 00:00 0 -7f22ad007000-7f22ad008000 ---p 00000000 00:00 0 -7f22ad008000-7f22ad898000 rw-p 00000000 00:00 0 -7f22ad898000-7f22adb78000 ---p 00000000 00:00 0 -7f22adb78000-7f22adb90000 rw-p 00000000 00:00 0 -7f22adb90000-7f22adbe0000 ---p 00000000 00:00 0 -7f22adbe0000-7f22adbf0000 rw-p 00000000 00:00 0 -7f22adbf0000-7f22b15f8000 ---p 00000000 00:00 0 -7f22b15f8000-7f22b1690000 rw-p 00000000 00:00 0 -7f22b1690000-7f22b1970000 ---p 00000000 00:00 0 -7f22b1970000-7f22b1988000 rw-p 00000000 00:00 0 -7f22b1988000-7f22b19d8000 ---p 00000000 00:00 0 -7f22b19d8000-7f22b19e8000 rw-p 00000000 00:00 0 -7f22b19e8000-7f22b53f0000 ---p 00000000 00:00 0 -7f22b53f0000-7f22b5400000 rw-p 00000000 00:00 0 -7f22b5400000-7f22b5840000 rwxp 00000000 00:00 0 -7f22b5840000-7f22bc937000 ---p 00000000 00:00 0 -7f22bc937000-7f22bcba7000 rwxp 00000000 00:00 0 -7f22bcba7000-7f22bcec9000 ---p 00000000 00:00 0 -7f22bcec9000-7f22bd139000 rwxp 00000000 00:00 0 -7f22bd139000-7f22c4400000 ---p 00000000 00:00 0 -7f22c4400000-7f22cbe42000 r--s 00000000 103:02 1966437 /usr/lib/jvm/java-17-openjdk-amd64/lib/modules -7f22cbe45000-7f22cbe6a000 rw-s 00000000 00:0e 48659 anon_inode:[io_uring] -7f22cbe6a000-7f22cbe6b000 r--p 00000000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f22cbe6b000-7f22cbe6c000 r-xp 00001000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f22cbe6c000-7f22cbe6d000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f22cbe6d000-7f22cbe6e000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f22cbe6e000-7f22cbe6f000 rw-p 00003000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f22cbe7e000-7f22cc3d1000 rw-p 00000000 00:00 0 -7f22cc3d1000-7f22d0000000 ---p 00000000 00:00 0 -7f22d0003000-7f22d0005000 r--p 00000000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f22d0005000-7f22d0008000 r-xp 00002000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f22d0008000-7f22d000a000 r--p 00005000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f22d000a000-7f22d000b000 r--p 00006000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f22d000b000-7f22d000c000 rw-p 00007000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f22d000c000-7f22d000e000 r--p 00000000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f22d000e000-7f22d000f000 r-xp 00002000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f22d000f000-7f22d0010000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f22d0010000-7f22d0011000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f22d0011000-7f22d0012000 rw-p 00004000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f22d0012000-7f22d00df000 rw-p 00000000 00:00 0 -7f22d00df000-7f22d00e0000 ---p 00000000 00:00 0 -7f22d00e0000-7f22d01e1000 rw-p 00000000 00:00 0 -7f22d01e1000-7f22d01e2000 ---p 00000000 00:00 0 -7f22d01e2000-7f22d02e3000 rw-p 00000000 00:00 0 -7f22d02e3000-7f22d02e4000 ---p 00000000 00:00 0 -7f22d02e4000-7f22d03f6000 rw-p 00000000 00:00 0 -7f22d03f6000-7f22d0452000 ---p 00000000 00:00 0 -7f22d0452000-7f22d0455000 rw-p 00000000 00:00 0 -7f22d0455000-7f22d045f000 ---p 00000000 00:00 0 -7f22d045f000-7f22d0461000 rw-p 00000000 00:00 0 -7f22d0461000-7f22d0ba2000 ---p 00000000 00:00 0 -7f22d0ba2000-7f22d0bb5000 rw-p 00000000 00:00 0 -7f22d0bb5000-7f22d0c11000 ---p 00000000 00:00 0 -7f22d0c11000-7f22d0c14000 rw-p 00000000 00:00 0 -7f22d0c14000-7f22d0c1e000 ---p 00000000 00:00 0 -7f22d0c1e000-7f22d0c20000 rw-p 00000000 00:00 0 -7f22d0c20000-7f22d1361000 ---p 00000000 00:00 0 -7f22d1361000-7f22d1374000 rw-p 00000000 00:00 0 -7f22d1374000-7f22d13d0000 ---p 00000000 00:00 0 -7f22d13d0000-7f22d13d3000 rw-p 00000000 00:00 0 -7f22d13d3000-7f22d13dd000 ---p 00000000 00:00 0 -7f22d13dd000-7f22d13df000 rw-p 00000000 00:00 0 -7f22d13df000-7f22d1b20000 ---p 00000000 00:00 0 -7f22d1b20000-7f22d232f000 rw-p 00000000 00:00 0 -7f22d232f000-7f22d2415000 ---p 00000000 00:00 0 -7f22d2415000-7f22d241e000 rw-p 00000000 00:00 0 -7f22d241e000-7f22d2500000 ---p 00000000 00:00 0 -7f22d2500000-7f22d2504000 ---p 00000000 00:00 0 -7f22d2504000-7f22d2600000 rw-p 00000000 00:00 0 -7f22d2600000-7f22d269c000 r--p 00000000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f22d269c000-7f22d27cb000 r-xp 0009c000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f22d27cb000-7f22d2858000 r--p 001cb000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f22d2858000-7f22d2863000 r--p 00257000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f22d2863000-7f22d2866000 rw-p 00262000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f22d2866000-7f22d286a000 rw-p 00000000 00:00 0 -7f22d286a000-7f22d286b000 r--p 00000000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f22d286b000-7f22d286c000 r-xp 00001000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f22d286c000-7f22d286d000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f22d286d000-7f22d286e000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f22d286e000-7f22d286f000 rw-p 00003000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f22d286f000-7f22d2873000 r--p 00000000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f22d2873000-7f22d287a000 r-xp 00004000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f22d287a000-7f22d287c000 r--p 0000b000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f22d287c000-7f22d287e000 r--p 0000c000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f22d287e000-7f22d287f000 rw-p 0000e000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f22d287f000-7f22d2883000 r--p 00000000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f22d2883000-7f22d2891000 r-xp 00004000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f22d2891000-7f22d2895000 r--p 00012000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f22d2895000-7f22d2896000 r--p 00015000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f22d2896000-7f22d2897000 rw-p 00016000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f22d2897000-7f22d2917000 rw-p 00000000 00:00 0 -7f22d2917000-7f22d2925000 r--p 00000000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f22d2925000-7f22d29a3000 r-xp 0000e000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f22d29a3000-7f22d29fe000 r--p 0008c000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f22d29fe000-7f22d29ff000 r--p 000e6000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f22d29ff000-7f22d2a00000 rw-p 000e7000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f22d2a00000-7f22d2c51000 r--p 00000000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f22d2c51000-7f22d39d5000 r-xp 00251000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f22d39d5000-7f22d3c56000 r--p 00fd5000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f22d3c56000-7f22d3d0e000 r--p 01255000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f22d3d0e000-7f22d3d43000 rw-p 0130d000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f22d3d43000-7f22d3d9d000 rw-p 00000000 00:00 0 -7f22d3da1000-7f22d3da2000 r--p 00000000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f22d3da2000-7f22d3da3000 r-xp 00001000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f22d3da3000-7f22d3da4000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f22d3da4000-7f22d3da5000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f22d3da5000-7f22d3da6000 rw-p 00003000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f22d3da6000-7f22d3da8000 r--p 00000000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f22d3da8000-7f22d3dad000 r-xp 00002000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f22d3dad000-7f22d3daf000 r--p 00007000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f22d3daf000-7f22d3db0000 r--p 00008000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f22d3db0000-7f22d3db1000 rw-p 00009000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f22d3db1000-7f22d3db7000 r--p 00000000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f22d3db7000-7f22d3dc0000 r-xp 00006000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f22d3dc0000-7f22d3dc4000 r--p 0000f000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f22d3dc4000-7f22d3dc5000 r--p 00013000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f22d3dc5000-7f22d3dc6000 rw-p 00014000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f22d3dc6000-7f22d3dc7000 r--p 00000000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f22d3dc7000-7f22d3dca000 r-xp 00001000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f22d3dca000-7f22d3dcb000 r--p 00004000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f22d3dcb000-7f22d3dcc000 r--p 00005000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f22d3dcc000-7f22d3dcd000 rw-p 00006000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f22d3dcd000-7f22d3dd2000 rw-p 00000000 00:00 0 -7f22d3dd2000-7f22d3dd9000 ---p 00000000 00:00 0 -7f22d3dd9000-7f22d3de5000 r--p 00000000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f22d3de5000-7f22d3df7000 r-xp 0000c000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f22d3df7000-7f22d3dfd000 r--p 0001e000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f22d3dfd000-7f22d3dfe000 r--p 00023000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f22d3dfe000-7f22d3dff000 rw-p 00024000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f22d3dff000-7f22d3e00000 rw-p 00000000 00:00 0 -7f22d3e00000-7f22d3e22000 r--p 00000000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f22d3e22000-7f22d3f9a000 r-xp 00022000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f22d3f9a000-7f22d3ff2000 r--p 0019a000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f22d3ff2000-7f22d3ff6000 r--p 001f1000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f22d3ff6000-7f22d3ff8000 rw-p 001f5000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f22d3ff8000-7f22d4005000 rw-p 00000000 00:00 0 -7f22d4008000-7f22d400f000 r--s 00000000 103:02 1074285 /usr/lib/x86_64-linux-gnu/gconv/gconv-modules.cache -7f22d400f000-7f22d4017000 rw-s 00000000 103:02 3670137 /tmp/hsperfdata_antonio/4794 -7f22d4017000-7f22d401a000 r--p 00000000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f22d401a000-7f22d4048000 r-xp 00003000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f22d4048000-7f22d4056000 r--p 00031000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f22d4056000-7f22d4057000 r--p 0003e000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f22d4057000-7f22d4058000 rw-p 0003f000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f22d4058000-7f22d4059000 rw-p 00000000 00:00 0 -7f22d4059000-7f22d405c000 r--p 00000000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f22d405c000-7f22d4077000 r-xp 00003000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f22d4077000-7f22d407b000 r--p 0001e000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f22d407b000-7f22d407c000 r--p 00021000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f22d407c000-7f22d407d000 rw-p 00022000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f22d407d000-7f22d4080000 rw-p 00000000 00:00 0 -7f22d4080000-7f22d4083000 r--p 00000000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f22d4083000-7f22d4095000 r-xp 00003000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f22d4095000-7f22d409c000 r--p 00015000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f22d409c000-7f22d409d000 r--p 0001b000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f22d409d000-7f22d409e000 rw-p 0001c000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f22d40a2000-7f22d40a3000 ---p 00000000 00:00 0 -7f22d40a3000-7f22d40a4000 r--p 00000000 00:00 0 -7f22d40a4000-7f22d40a5000 ---p 00000000 00:00 0 -7f22d40a5000-7f22d40a7000 r--p 00000000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f22d40a7000-7f22d40aa000 r-xp 00002000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f22d40aa000-7f22d40ab000 r--p 00005000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f22d40ab000-7f22d40ac000 r--p 00006000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f22d40ac000-7f22d40ad000 rw-p 00007000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f22d40ad000-7f22d40af000 r--p 00000000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f22d40af000-7f22d40b9000 r-xp 00002000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f22d40b9000-7f22d40bc000 r--p 0000c000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f22d40bc000-7f22d40bd000 r--p 0000e000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f22d40bd000-7f22d40be000 rw-p 0000f000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f22d40be000-7f22d40c0000 rw-p 00000000 00:00 0 -7f22d40c0000-7f22d40c1000 r--p 00000000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f22d40c1000-7f22d40e9000 r-xp 00001000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f22d40e9000-7f22d40f3000 r--p 00029000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f22d40f3000-7f22d40f5000 r--p 00033000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f22d40f5000-7f22d40f7000 rw-p 00035000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7fff14656000-7fff14678000 rw-p 00000000 00:00 0 [stack] -7fff147e5000-7fff147e9000 r--p 00000000 00:00 0 [vvar] -7fff147e9000-7fff147eb000 r-xp 00000000 00:00 0 [vdso] -ffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall] - - -VM Arguments: -jvm_args: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring -java_command: sbt.ForkMain 44215 -java_class_path (initial): /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/test-classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scala-lang/scala-library/2.13.10/scala-library-2.13.10.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect_2.13/3.6-0142603/cats-effect_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-io_2.13/3.7.0/fs2-io_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/munit-cats-effect_2.13/2.0.0-M3/munit-cats-effect_2.13-2.0.0-M3.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-native-io_uring/0.0.21.Final/netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-kernel_2.13/3.6-0142603/cats-effect-kernel_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-std_2.13/3.6-0142603/cats-effect-std_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-core_2.13/3.7.0/fs2-core_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/com/comcast/ip4s-core_2.13/3.3.0/ip4s-core_2.13-3.3.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scalameta/munit_2.13/1.0.0-M6/munit_2.13-1.0.0-M6.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-classes-io_uring/0.0.21.Final/netty-incubator-transport-classes-io_uring-0.0.21.Final.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-core_2.13/2.9.0/cats-core_2.13-2.9.0.ja -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 4 {product} {ergonomic} - uint ConcGCThreads = 2 {product} {ergonomic} - uint G1ConcRefinementThreads = 8 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - size_t InitialHeapSize = 260046848 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4158652416 {product} {ergonomic} - size_t MaxNewSize = 2493513728 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 5839372 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4158652416 {manageable} {ergonomic} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64/ -PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/games:/usr/local/games:/snap/bin:/snap/bin:/home/antonio/.local/share/coursier/bin:/home/antonio/.local/share/coursier/bin -USERNAME=antonio -SHELL=/bin/bash -DISPLAY=:0 -LANG=en_US.UTF-8 - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.so, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - - ---------------- S Y S T E M --------------- - -OS: -DISTRIB_ID=Ubuntu -DISTRIB_RELEASE=23.04 -DISTRIB_CODENAME=lunar -DISTRIB_DESCRIPTION="Ubuntu 23.04" -uname: Linux 6.2.0-26-generic #26-Ubuntu SMP PREEMPT_DYNAMIC Mon Jul 10 23:39:54 UTC 2023 x86_64 -OS uptime: 0 days 0:40 hours -libc: glibc 2.37 NPTL 2.37 -rlimit (soft/hard): STACK 8192k/infinity , CORE 0k/infinity , NPROC 63121/63121 , NOFILE 1048576/1048576 , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK 2030188k/2030188k -load average: 4.99 2.00 1.29 - -/proc/meminfo: -MemTotal: 16241528 kB -MemFree: 10347640 kB -MemAvailable: 12248880 kB -Buffers: 82448 kB -Cached: 2370536 kB -SwapCached: 0 kB -Active: 668164 kB -Inactive: 4415084 kB -Active(anon): 2104 kB -Inactive(anon): 2969548 kB -Active(file): 666060 kB -Inactive(file): 1445536 kB -Unevictable: 342388 kB -Mlocked: 48 kB -SwapTotal: 4194300 kB -SwapFree: 4194300 kB -Zswap: 0 kB -Zswapped: 0 kB -Dirty: 48 kB -Writeback: 0 kB -AnonPages: 2973056 kB -Mapped: 655108 kB -Shmem: 358916 kB -KReclaimable: 91204 kB -Slab: 234964 kB -SReclaimable: 91204 kB -SUnreclaim: 143760 kB -KernelStack: 17904 kB -PageTables: 33876 kB -SecPageTables: 0 kB -NFS_Unstable: 0 kB -Bounce: 0 kB -WritebackTmp: 0 kB -CommitLimit: 12315064 kB -Committed_AS: 11130900 kB -VmallocTotal: 34359738367 kB -VmallocUsed: 114780 kB -VmallocChunk: 0 kB -Percpu: 5952 kB -HardwareCorrupted: 0 kB -AnonHugePages: 2048 kB -ShmemHugePages: 0 kB -ShmemPmdMapped: 0 kB -FileHugePages: 0 kB -FilePmdMapped: 0 kB -HugePages_Total: 0 -HugePages_Free: 0 -HugePages_Rsvd: 0 -HugePages_Surp: 0 -Hugepagesize: 2048 kB -Hugetlb: 0 kB -DirectMap4k: 377332 kB -DirectMap2M: 6844416 kB -DirectMap1G: 9437184 kB - -/sys/kernel/mm/transparent_hugepage/enabled: always [madvise] never -/sys/kernel/mm/transparent_hugepage/defrag (defrag/compaction efforts parameter): always defer defer+madvise [madvise] never - -Process Memory: -Virtual Size: 8613960K (peak: 8675504K) -Resident Set Size: 171000K (peak: 171000K) (anon: 141304K, file: 29696K, shmem: 0K) -Swapped out: 0K -C-Heap outstanding allocations: 60271K, retained: 13540K -glibc malloc tunables: (default) - -/proc/sys/kernel/threads-max (system-wide limit on the number of threads): 126243 -/proc/sys/vm/max_map_count (maximum number of memory map areas a process may have): 65530 -/proc/sys/kernel/pid_max (system-wide limit on number of process identifiers): 4194304 - -container (cgroup) information: -container_type: cgroupv2 -cpu_cpuset_cpus: not supported -cpu_memory_nodes: not supported -active_processor_count: 8 -cpu_quota: not supported -cpu_period: not supported -cpu_shares: not supported -memory_limit_in_bytes: unlimited -memory_and_swap_limit_in_bytes: unlimited -memory_soft_limit_in_bytes: unlimited -memory_usage_in_bytes: 2812744 k -memory_max_usage_in_bytes: not supported -memory_swap_current_in_bytes: unlimited -memory_swap_max_limit_in_bytes: unlimited -maximum number of tasks: 18936 -current number of tasks: 364 - -Steal ticks since vm start: 0 -Steal ticks percentage since vm start: 0.000 - -CPU: total 8 (initial active 8) (4 cores per cpu, 2 threads per core) family 6 model 142 stepping 10 microcode 0xf0, cx8, cmov, fxsr, ht, mmx, 3dnowpref, sse, sse2, sse3, ssse3, sse4.1, sse4.2, popcnt, lzcnt, tsc, tscinvbit, avx, avx2, aes, erms, clmul, bmi1, bmi2, adx, fma, vzeroupper, clflush, clflushopt -CPU Model and flags from /proc/cpuinfo: -model name : Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz -flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc art arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc cpuid aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch cpuid_fault epb invpcid_single pti ssbd ibrs ibpb stibp tpr_shadow vnmi flexpriority ept vpid ept_ad fsgsbase tsc_adjust bmi1 avx2 smep bmi2 erms invpcid mpx rdseed adx smap clflushopt intel_pt xsaveopt xsavec xgetbv1 xsaves dtherm ida arat pln pts hwp hwp_notify hwp_act_window hwp_epp md_clear flush_l1d arch_capabilities - -Online cpus: 0-7 -Offline cpus: -BIOS frequency limitation: -Frequency switch latency (ns): 0 -Available cpu frequencies: -Current governor: powersave -Core performance/turbo boost: - -Memory: 4k page, physical 16241528k(10347640k free), swap 4194300k(4194300k free) -Page Sizes: 4k - -vm_info: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04) for linux-amd64 JRE (17.0.7+7-Ubuntu-0ubuntu123.04), built on May 6 2023 14:43:58 by "buildd" with gcc 12.2.0 - -END. diff --git a/hs_err_pid8969.log b/hs_err_pid8969.log deleted file mode 100644 index 12fb274c..00000000 --- a/hs_err_pid8969.log +++ /dev/null @@ -1,1149 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x00007f5294b923e4, pid=8969, tid=9009 -# -# JRE version: OpenJDK Runtime Environment (17.0.7+7) (build 17.0.7+7-Ubuntu-0ubuntu123.04) -# Java VM: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04, mixed mode, sharing, tiered, compressed oops, compressed class ptrs, g1 gc, linux-amd64) -# Problematic frame: -# V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 -# -# Core dump will be written. Default location: Core dumps may be processed with "/usr/share/apport/apport -p%p -s%s -c%c -d%d -P%P -u%u -g%g -- %E" (or dumping to /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/core.8969) -# -# If you would like to submit a bug report, please visit: -# Unknown -# - ---------------- S U M M A R Y ------------ - -Command Line: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring sbt.ForkMain 33543 - -Host: Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz, 8 cores, 15G, Ubuntu 23.04 -Time: Tue Aug 8 13:41:27 2023 CEST elapsed time: 11.994800 seconds (0d 0h 0m 11s) - ---------------- T H R E A D --------------- - -Current thread (0x00007f521453bfc0): JavaThread "io-compute-1" daemon [_thread_in_vm, id=9009, stack(0x00007f5249f1b000,0x00007f524a01b000)] - -Stack: [0x00007f5249f1b000,0x00007f524a01b000], sp=0x00007f524a018828, free space=1014k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.so+0x5923e4] AccessInternal::PostRuntimeDispatch, (AccessInternal::BarrierType)2, 594020ul>::oop_access_barrier(void*)+0x4 -V [libjvm.so+0xe881a4] SystemDictionary::resolve_instance_class_or_null(Symbol*, Handle, Handle, JavaThread*)+0x134 -V [libjvm.so+0xe89f46] SystemDictionary::resolve_or_fail(Symbol*, Handle, Handle, bool, JavaThread*)+0x66 -V [libjvm.so+0x602cc9] ConstantPool::klass_at_impl(constantPoolHandle const&, int, JavaThread*)+0x139 -V [libjvm.so+0x603a0a] ConstantPool::klass_ref_at(int, JavaThread*)+0x7a -V [libjvm.so+0xa948f9] LinkInfo::LinkInfo(constantPoolHandle const&, int, JavaThread*)+0x39 -V [libjvm.so+0xa9b19f] LinkResolver::resolve_invoke(CallInfo&, Handle, constantPoolHandle const&, int, Bytecodes::Code, JavaThread*)+0x1df -V [libjvm.so+0x82c947] InterpreterRuntime::resolve_invoke(JavaThread*, Bytecodes::Code)+0x177 -V [libjvm.so+0x82ce67] InterpreterRuntime::resolve_from_cache(JavaThread*, Bytecodes::Code)+0x37 -j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+85 -j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 -j fs2.Stream$NestedStreamOps$$$Lambda$682+0x000000080136f5d8.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 -j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 -j cats.effect.IO$$Lambda$302+0x000000080125ee68.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 -j cats.effect.IO$$Lambda$303+0x000000080125f238.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -J 1746 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f527975a764 [0x00007f52797593a0+0x00000000000013c4] -j cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V+262 -j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 -j cats.effect.IOFiber.run()V+66 -J 1768% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f5279764f74 [0x00007f5279762220+0x0000000000002d54] -v ~StubRoutines::call_stub -V [libjvm.so+0x831c02] JavaCalls::call_helper(JavaValue*, methodHandle const&, JavaCallArguments*, JavaThread*)+0x302 -V [libjvm.so+0x8332d2] JavaCalls::call_virtual(JavaValue*, Handle, Klass*, Symbol*, Symbol*, JavaThread*)+0x1a2 -V [libjvm.so+0x8fb417] thread_entry(JavaThread*, JavaThread*)+0xa7 -V [libjvm.so+0xed302e] JavaThread::thread_main_inner()+0xce -V [libjvm.so+0xed675f] Thread::call_run()+0xbf -V [libjvm.so+0xc2d5c9] thread_native_entry(Thread*)+0xe9 - -Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) -j fs2.Stream$NestedStreamOps$.onOutcome$1(Lcats/effect/kernel/Outcome;Lscala/util/Either;Lfs2/concurrent/Channel;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/SignallingRef;)Ljava/lang/Object;+85 -j fs2.Stream$NestedStreamOps$.$anonfun$parJoin$25(Lcats/effect/kernel/Outcome;Lcats/effect/kernel/GenConcurrent;Lfs2/concurrent/Channel;Lfs2/concurrent/SignallingRef;Lcats/effect/kernel/Outcome;)Ljava/lang/Object;+77 -j fs2.Stream$NestedStreamOps$$$Lambda$682+0x000000080136f5d8.apply(Ljava/lang/Object;)Ljava/lang/Object;+20 -j cats.effect.IO.$anonfun$guaranteeCase$6(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+17 -j cats.effect.IO$$Lambda$302+0x000000080125ee68.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -j cats.effect.IO.$anonfun$flatTap$1(Lscala/Function1;Ljava/lang/Object;)Lcats/effect/IO;+2 -j cats.effect.IO$$Lambda$303+0x000000080125f238.apply(Ljava/lang/Object;)Ljava/lang/Object;+5 -J 1746 c1 cats.effect.IOFiber.succeeded(Ljava/lang/Object;I)Lcats/effect/IO; (420 bytes) @ 0x00007f527975a764 [0x00007f52797593a0+0x00000000000013c4] -j cats.effect.IOFiber.runLoop(Lcats/effect/IO;II)V+262 -j cats.effect.IOFiber.asyncContinueSuccessfulR()V+29 -j cats.effect.IOFiber.run()V+66 -J 1768% c1 cats.effect.unsafe.WorkerThread.run()V (1765 bytes) @ 0x00007f5279764f74 [0x00007f5279762220+0x0000000000002d54] -v ~StubRoutines::call_stub - -siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr: 0x0000000000001000 - -Register to memory mapping: - -RAX=0x00007f5200000010 points into unknown readable memory: 0x0000000000021000 | 00 10 02 00 00 00 00 00 -RBX=0x00007f52140f3900 points into unknown readable memory: 0x00007f526e73ea9e | 9e ea 73 6e 52 7f 00 00 -RCX=0x0000000000000003 is an unknown value -RDX=0x00007f5295912330: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 -RSP=0x00007f524a018828 is pointing into the stack for thread: 0x00007f521453bfc0 -RBP=0x00007f524a018850 is pointing into the stack for thread: 0x00007f521453bfc0 -RSI=0x0000000000000060 is an unknown value -RDI=0x0000000000001000 is an unknown value -R8 =0x00007f5214034430 points into unknown readable memory: 0x63730010ea9e003b | 3b 00 9e ea 10 00 73 63 -R9 =0x00007f521453bfc0 is a thread -R10=0x00007f5295c10000 points into unknown readable memory: 0x0100050403020100 | 00 01 02 03 04 05 00 01 -R11=0x0000000000000001 is an unknown value -R12=0x00007f524c00c900 points into unknown readable memory: 0x00007f5200000010 | 10 00 00 00 52 7f 00 00 -R13=0x000000070860fac8 is an oop: java.security.ProtectionDomain -{0x000000070860fac8} - klass: 'java/security/ProtectionDomain' - - ---- fields (total size 5 words): - - private 'hasAllPerm' 'Z' @12 false - - private final 'staticPermissions' 'Z' @13 false - - private 'codesource' 'Ljava/security/CodeSource;' @16 a 'java/security/CodeSource'{0x000000070860faf0} (e10c1f5e) - - private 'classloader' 'Ljava/lang/ClassLoader;' @20 a 'jdk/internal/loader/ClassLoaders$AppClassLoader'{0x00000007ffd59e70} (fffab3ce) - - private 'principals' '[Ljava/security/Principal;' @24 a 'java/security/Principal'[0] {0x000000070860fc90} (e10c1f92) - - private 'permissions' 'Ljava/security/PermissionCollection;' @28 a 'sun/security/util/LazyCodeSourcePermissionCollection'{0x000000070860fca0} (e10c1f94) - - final 'key' 'Ljava/security/ProtectionDomain$Key;' @32 a 'java/security/ProtectionDomain$Key'{0x000000070860fe60} (e10c1fcc) -R14=0x00007f5290193210 points into unknown readable memory: 0x00007f5290165008 | 08 50 16 90 52 7f 00 00 -R15=0x00007f521453c5d0 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 - - -Registers: -RAX=0x00007f5200000010, RBX=0x00007f52140f3900, RCX=0x0000000000000003, RDX=0x00007f5295912330 -RSP=0x00007f524a018828, RBP=0x00007f524a018850, RSI=0x0000000000000060, RDI=0x0000000000001000 -R8 =0x00007f5214034430, R9 =0x00007f521453bfc0, R10=0x00007f5295c10000, R11=0x0000000000000001 -R12=0x00007f524c00c900, R13=0x000000070860fac8, R14=0x00007f5290193210, R15=0x00007f521453c5d0 -RIP=0x00007f5294b923e4, EFLAGS=0x0000000000010206, CSGSFS=0x002b000000000033, ERR=0x0000000000000004 - TRAPNO=0x000000000000000e - -Top of Stack: (sp=0x00007f524a018828) -0x00007f524a018828: 00007f5294c8ffb8 00007f524a018850 -0x00007f524a018838: 00007f5214034430 00007f521453bfc0 -0x00007f524a018848: 00007f521453c5e0 00007f524a018980 -0x00007f524a018858: 00007f52954881a4 00007f524a018b10 - -Instructions: (pc=0x00007f5294b923e4) -0x00007f5294b922e4: 01 eb df 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b922f4: 31 c0 c3 66 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92304: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92314: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92324: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92334: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92344: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92354: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92364: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92374: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92384: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92394: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923a4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923b4: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923c4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923d4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923e4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b923f4: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92404: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92414: 48 8b 07 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92424: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92434: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92444: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92454: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92464: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92474: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92484: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b92494: 48 89 37 c3 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b924a4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f -0x00007f5294b924b4: 48 85 ff 74 05 e8 e2 45 14 00 4c 89 23 5b 41 5c -0x00007f5294b924c4: 5d c3 66 2e 0f 1f 84 00 00 00 00 00 f3 0f 1e fa -0x00007f5294b924d4: 55 48 89 e5 41 54 49 89 f4 53 48 89 fb 48 8b 3f - - -Stack slot to memory mapping: -stack at sp + 0 slots: 0x00007f5294c8ffb8: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 -stack at sp + 1 slots: 0x00007f524a018850 is pointing into the stack for thread: 0x00007f521453bfc0 -stack at sp + 2 slots: 0x00007f5214034430 points into unknown readable memory: 0x63730010ea9e003b | 3b 00 9e ea 10 00 73 63 -stack at sp + 3 slots: 0x00007f521453bfc0 is a thread -stack at sp + 4 slots: 0x00007f521453c5e0 points into unknown readable memory: 0x00000007ffd59e70 | 70 9e d5 ff 07 00 00 00 -stack at sp + 5 slots: 0x00007f524a018980 is pointing into the stack for thread: 0x00007f521453bfc0 -stack at sp + 6 slots: 0x00007f52954881a4: in /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so at 0x00007f5294600000 -stack at sp + 7 slots: 0x00007f524a018b10 is pointing into the stack for thread: 0x00007f521453bfc0 - - ---------------- P R O C E S S --------------- - -Threads class SMR info: -_java_thread_list=0x00007f5234390650, length=27, elements={ -0x00007f5290015d20, 0x00007f5290179870, 0x00007f529017ac50, 0x00007f52901801d0, -0x00007f5290181580, 0x00007f5290182990, 0x00007f5290184340, 0x00007f5290185870, -0x00007f529018ecd0, 0x00007f52901a76b0, 0x00007f52901ab290, 0x00007f522c001140, -0x00007f52901ac250, 0x00007f52901b10a0, 0x00007f529037c7f0, 0x00007f521453b510, -0x00007f521453bfc0, 0x00007f521453cf70, 0x00007f521453de80, 0x00007f521453f230, -0x00007f5214540530, 0x00007f52145418f0, 0x00007f5214542800, 0x00007f51f0006af0, -0x00007f523020de90, 0x00007f51ec003790, 0x00007f523438f6f0 -} - -Java Threads: ( => current thread ) - 0x00007f5290015d20 JavaThread "main" [_thread_blocked, id=8972, stack(0x00007f5294100000,0x00007f5294200000)] - 0x00007f5290179870 JavaThread "Reference Handler" daemon [_thread_blocked, id=8979, stack(0x00007f526c865000,0x00007f526c965000)] - 0x00007f529017ac50 JavaThread "Finalizer" daemon [_thread_blocked, id=8980, stack(0x00007f526c765000,0x00007f526c865000)] - 0x00007f52901801d0 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=8981, stack(0x00007f526c665000,0x00007f526c765000)] - 0x00007f5290181580 JavaThread "Service Thread" daemon [_thread_blocked, id=8982, stack(0x00007f526c565000,0x00007f526c665000)] - 0x00007f5290182990 JavaThread "Monitor Deflation Thread" daemon [_thread_blocked, id=8983, stack(0x00007f526c465000,0x00007f526c565000)] - 0x00007f5290184340 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=8984, stack(0x00007f526c365000,0x00007f526c465000)] - 0x00007f5290185870 JavaThread "C1 CompilerThread0" daemon [_thread_blocked, id=8985, stack(0x00007f526c265000,0x00007f526c365000)] - 0x00007f529018ecd0 JavaThread "Sweeper thread" daemon [_thread_blocked, id=8986, stack(0x00007f526c165000,0x00007f526c265000)] - 0x00007f52901a76b0 JavaThread "JDWP Transport Listener: dt_socket" daemon [_thread_blocked, id=8987, stack(0x00007f526c065000,0x00007f526c165000)] - 0x00007f52901ab290 JavaThread "JDWP Event Helper Thread" daemon [_thread_blocked, id=8988, stack(0x00007f524b100000,0x00007f524b200000)] - 0x00007f522c001140 JavaThread "JDWP Command Reader" daemon [_thread_in_native, id=8993, stack(0x00007f524b000000,0x00007f524b100000)] - 0x00007f52901ac250 JavaThread "Notification Thread" daemon [_thread_blocked, id=8996, stack(0x00007f524af00000,0x00007f524b000000)] - 0x00007f52901b10a0 JavaThread "Common-Cleaner" daemon [_thread_blocked, id=8998, stack(0x00007f524acfe000,0x00007f524adfe000)] - 0x00007f529037c7f0 JavaThread "pool-1-thread-1" [_thread_blocked, id=9000, stack(0x00007f524abfe000,0x00007f524acfe000)] - 0x00007f521453b510 JavaThread "io-compute-blocker-0" daemon [_thread_blocked, id=9008, stack(0x00007f524a01b000,0x00007f524a11b000)] -=>0x00007f521453bfc0 JavaThread "io-compute-1" daemon [_thread_in_vm, id=9009, stack(0x00007f5249f1b000,0x00007f524a01b000)] - 0x00007f521453cf70 JavaThread "io-compute-6" daemon [_thread_in_native, id=9010, stack(0x00007f5249e1b000,0x00007f5249f1b000)] - 0x00007f521453de80 JavaThread "io-compute-2" daemon [_thread_blocked, id=9011, stack(0x00007f5249d1b000,0x00007f5249e1b000)] - 0x00007f521453f230 JavaThread "io-compute-4" daemon [_thread_in_native, id=9012, stack(0x00007f5249c1b000,0x00007f5249d1b000)] - 0x00007f5214540530 JavaThread "io-compute-5" daemon [_thread_blocked, id=9013, stack(0x00007f5249b1b000,0x00007f5249c1b000)] - 0x00007f52145418f0 JavaThread "io-compute-0" daemon [_thread_blocked, id=9014, stack(0x00007f5249a1b000,0x00007f5249b1b000)] - 0x00007f5214542800 JavaThread "io-compute-7" daemon [_thread_blocked, id=9015, stack(0x00007f524991b000,0x00007f5249a1b000)] - 0x00007f51f0006af0 JavaThread "pool-2-thread-1" [_thread_blocked, id=9019, stack(0x00007f524a5f4000,0x00007f524a6f4000)] - 0x00007f523020de90 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=9023, stack(0x00007f524981b000,0x00007f524991b000)] - 0x00007f51ec003790 JavaThread "io-compute-3" daemon [_thread_in_native, id=9024, stack(0x00007f5249415000,0x00007f5249515000)] - 0x00007f523438f6f0 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=9025, stack(0x00007f5249315000,0x00007f5249415000)] - -Other Threads: - 0x00007f52901758f0 VMThread "VM Thread" [stack: 0x00007f526c967000,0x00007f526ca67000] [id=8978] _threads_hazard_ptr=0x00007f5234390650 - 0x00007f52901adaa0 WatcherThread [stack: 0x00007f524ae00000,0x00007f524af00000] [id=8997] - 0x00007f5290080fb0 GCTaskThread "GC Thread#0" [stack: 0x00007f526f7cb000,0x00007f526f8cb000] [id=8973] - 0x00007f524c005f90 GCTaskThread "GC Thread#1" [stack: 0x00007f524aafe000,0x00007f524abfe000] [id=9001] - 0x00007f524c0069c0 GCTaskThread "GC Thread#2" [stack: 0x00007f524a9fc000,0x00007f524aafc000] [id=9002] - 0x00007f524c0073f0 GCTaskThread "GC Thread#3" [stack: 0x00007f524a8fa000,0x00007f524a9fa000] [id=9003] - 0x00007f524c007e20 GCTaskThread "GC Thread#4" [stack: 0x00007f524a7f8000,0x00007f524a8f8000] [id=9004] - 0x00007f524c008850 GCTaskThread "GC Thread#5" [stack: 0x00007f524a6f6000,0x00007f524a7f6000] [id=9005] - 0x00007f524c00c9d0 GCTaskThread "GC Thread#6" [stack: 0x00007f524971b000,0x00007f524981b000] [id=9017] - 0x00007f524c00cfa0 GCTaskThread "GC Thread#7" [stack: 0x00007f5249619000,0x00007f5249719000] [id=9018] - 0x00007f5290091d10 ConcurrentGCThread "G1 Main Marker" [stack: 0x00007f526f6c9000,0x00007f526f7c9000] [id=8974] - 0x00007f5290092c00 ConcurrentGCThread "G1 Conc#0" [stack: 0x00007f526f5c7000,0x00007f526f6c7000] [id=8975] - 0x00007f5260000ea0 ConcurrentGCThread "G1 Conc#1" [stack: 0x00007f5249517000,0x00007f5249617000] [id=9021] - 0x00007f5290146590 ConcurrentGCThread "G1 Refine#0" [stack: 0x00007f526ccbd000,0x00007f526cdbd000] [id=8976] - 0x00007f5290147480 ConcurrentGCThread "G1 Service" [stack: 0x00007f526cbbb000,0x00007f526ccbb000] [id=8977] - -Threads with active compile tasks: -C2 CompilerThread0 12047 1809 4 java.lang.invoke.MethodType::makeImpl (109 bytes) -C1 CompilerThread0 12048 1940 ! 3 cats.effect.IOFiber::runLoop (3533 bytes) -C2 CompilerThread1 12048 1783 % ! 4 cats.effect.unsafe.WorkerThread::run @ 44 (1765 bytes) -C2 CompilerThread2 12048 2018 ! 4 java.lang.ref.ReferenceQueue::poll (28 bytes) - -VM state: synchronizing (normal execution) - -VM Mutex/Monitor currently owned by a thread: ([mutex/lock_event]) -[0x00007f5290012a90] Threads_lock - owner thread: 0x00007f52901758f0 - -Heap address: 0x0000000708200000, size: 3966 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 - -CDS archive(s) mapped at: [0x0000000800000000-0x0000000800be2000-0x0000000800be2000), size 12460032, SharedBaseAddress: 0x0000000800000000, ArchiveRelocationMode: 0. -Compressed class space mapped at: 0x0000000801000000-0x0000000841000000, reserved size: 1073741824 -Narrow klass base: 0x0000000800000000, Narrow klass shift: 0, Narrow klass range: 0x100000000 - -GC Precious Log: - CPUs: 8 total, 8 available - Memory: 15860M - Large Page Support: Disabled - NUMA Support: Disabled - Compressed Oops: Enabled (Zero based) - Heap Region Size: 2M - Heap Min Capacity: 8M - Heap Initial Capacity: 248M - Heap Max Capacity: 3966M - Pre-touch: Disabled - Parallel Workers: 8 - Concurrent Workers: 2 - Concurrent Refinement Workers: 8 - Periodic GC: Disabled - -Heap: - garbage-first heap total 49152K, used 16110K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 9 young (18432K), 3 survivors (6144K) - Metaspace used 29626K, committed 30016K, reserved 1114112K - class space used 3307K, committed 3456K, reserved 1048576K - -Heap Regions: E=young(eden), S=young(survivor), O=old, HS=humongous(starts), HC=humongous(continues), CS=collection set, F=free, OA=open archive, CA=closed archive, TAMS=top-at-mark-start (previous, next) -| 0|0x0000000708200000, 0x00000007083c3838, 0x0000000708400000| 88%| S|CS|TAMS 0x0000000708200000, 0x0000000708200000| Complete -| 1|0x0000000708400000, 0x0000000708600000, 0x0000000708600000|100%| S|CS|TAMS 0x0000000708400000, 0x0000000708400000| Complete -| 2|0x0000000708600000, 0x0000000708800000, 0x0000000708800000|100%| S|CS|TAMS 0x0000000708600000, 0x0000000708600000| Complete -| 3|0x0000000708800000, 0x0000000708800000, 0x0000000708a00000| 0%| F| |TAMS 0x0000000708800000, 0x0000000708800000| Untracked -| 4|0x0000000708a00000, 0x0000000708a00000, 0x0000000708c00000| 0%| F| |TAMS 0x0000000708a00000, 0x0000000708a00000| Untracked -| 5|0x0000000708c00000, 0x0000000708c00000, 0x0000000708e00000| 0%| F| |TAMS 0x0000000708c00000, 0x0000000708c00000| Untracked -| 6|0x0000000708e00000, 0x0000000708e00000, 0x0000000709000000| 0%| F| |TAMS 0x0000000708e00000, 0x0000000708e00000| Untracked -| 7|0x0000000709000000, 0x0000000709000000, 0x0000000709200000| 0%| F| |TAMS 0x0000000709000000, 0x0000000709000000| Untracked -| 8|0x0000000709200000, 0x0000000709200000, 0x0000000709400000| 0%| F| |TAMS 0x0000000709200000, 0x0000000709200000| Untracked -| 9|0x0000000709400000, 0x0000000709400000, 0x0000000709600000| 0%| F| |TAMS 0x0000000709400000, 0x0000000709400000| Untracked -| 10|0x0000000709600000, 0x0000000709600000, 0x0000000709800000| 0%| F| |TAMS 0x0000000709600000, 0x0000000709600000| Untracked -| 11|0x0000000709800000, 0x0000000709800000, 0x0000000709a00000| 0%| F| |TAMS 0x0000000709800000, 0x0000000709800000| Untracked -| 12|0x0000000709a00000, 0x0000000709a00000, 0x0000000709c00000| 0%| F| |TAMS 0x0000000709a00000, 0x0000000709a00000| Untracked -| 13|0x0000000709c00000, 0x0000000709c00000, 0x0000000709e00000| 0%| F| |TAMS 0x0000000709c00000, 0x0000000709c00000| Untracked -| 14|0x0000000709e00000, 0x0000000709e00000, 0x000000070a000000| 0%| F| |TAMS 0x0000000709e00000, 0x0000000709e00000| Untracked -| 15|0x000000070a000000, 0x000000070a000000, 0x000000070a200000| 0%| F| |TAMS 0x000000070a000000, 0x000000070a000000| Untracked -| 16|0x000000070a200000, 0x000000070a35a460, 0x000000070a400000| 67%| E| |TAMS 0x000000070a200000, 0x000000070a200000| Complete -| 110|0x0000000715e00000, 0x0000000716000000, 0x0000000716000000|100%| E|CS|TAMS 0x0000000715e00000, 0x0000000715e00000| Complete -| 111|0x0000000716000000, 0x0000000716200000, 0x0000000716200000|100%| E|CS|TAMS 0x0000000716000000, 0x0000000716000000| Complete -| 112|0x0000000716200000, 0x0000000716400000, 0x0000000716400000|100%| E| |TAMS 0x0000000716200000, 0x0000000716200000| Complete -| 122|0x0000000717600000, 0x0000000717800000, 0x0000000717800000|100%| E|CS|TAMS 0x0000000717600000, 0x0000000717600000| Complete -| 123|0x0000000717800000, 0x0000000717a00000, 0x0000000717a00000|100%| E|CS|TAMS 0x0000000717800000, 0x0000000717800000| Complete -|1981|0x00000007ffc00000, 0x00000007ffd76000, 0x00000007ffe00000| 73%|OA| |TAMS 0x00000007ffd76000, 0x00000007ffc00000| Untracked -|1982|0x00000007ffe00000, 0x00000007ffe82000, 0x0000000800000000| 25%|CA| |TAMS 0x00000007ffe82000, 0x00000007ffe00000| Untracked - -Card table byte_map: [0x00007f5277c7a000,0x00007f5278439000] _byte_map_base: 0x00007f5274439000 - -Marking Bits (Prev, Next): (CMBitMap*) 0x00007f5290081a00, (CMBitMap*) 0x00007f52900819c0 - Prev Bits: [0x00007f526f8cb000, 0x00007f52736c3000) - Next Bits: [0x00007f52736c3000, 0x00007f52774bb000) - -Polling page: 0x00007f5295caf000 - -Metaspace: - -Usage: - Non-class: 25.70 MB used. - Class: 3.23 MB used. - Both: 28.93 MB used. - -Virtual space: - Non-class space: 64.00 MB reserved, 25.94 MB ( 41%) committed, 1 nodes. - Class space: 1.00 GB reserved, 3.38 MB ( <1%) committed, 1 nodes. - Both: 1.06 GB reserved, 29.31 MB ( 3%) committed. - -Chunk freelists: - Non-Class: 5.67 MB - Class: 12.45 MB - Both: 18.12 MB - -MaxMetaspaceSize: unlimited -CompressedClassSpaceSize: 1.00 GB -Initial GC threshold: 21.00 MB -Current GC threshold: 35.12 MB -CDS: on -MetaspaceReclaimPolicy: balanced - - commit_granule_bytes: 65536. - - commit_granule_words: 8192. - - virtual_space_node_default_size: 8388608. - - enlarge_chunks_in_place: 1. - - new_chunks_are_fully_committed: 0. - - uncommit_free_chunks: 1. - - use_allocation_guard: 0. - - handle_deallocations: 1. - - -Internal statistics: - -num_allocs_failed_limit: 3. -num_arena_births: 432. -num_arena_deaths: 0. -num_vsnodes_births: 2. -num_vsnodes_deaths: 0. -num_space_committed: 469. -num_space_uncommitted: 0. -num_chunks_returned_to_freelist: 3. -num_chunks_taken_from_freelist: 1438. -num_chunk_merges: 3. -num_chunk_splits: 1034. -num_chunks_enlarged: 816. -num_inconsistent_stats: 0. - -CodeHeap 'non-profiled nmethods': size=120028Kb used=1078Kb max_used=1078Kb free=118949Kb - bounds [0x00007f5280ec9000, 0x00007f5281139000, 0x00007f5288400000] -CodeHeap 'profiled nmethods': size=120028Kb used=3792Kb max_used=3792Kb free=116235Kb - bounds [0x00007f5279400000, 0x00007f52797c0000, 0x00007f5280937000] -CodeHeap 'non-nmethods': size=5704Kb used=1349Kb max_used=1375Kb free=4354Kb - bounds [0x00007f5280937000, 0x00007f5280ba7000, 0x00007f5280ec9000] - total_blobs=2552 nmethods=1927 adapters=539 - compilation: enabled - stopped_count=0, restarted_count=0 - full_count=0 - -Compilation events (20 events): -Event: 11.759 Thread 0x00007f5290184340 nmethod 1888 0x00007f5280fd1810 code [0x00007f5280fd1a00, 0x00007f5280fd2478] -Event: 11.759 Thread 0x00007f5290184340 1968 4 cats.effect.IOFiber::shouldFinalize (20 bytes) -Event: 11.761 Thread 0x00007f5290184340 nmethod 1968 0x00007f5280fd3010 code [0x00007f5280fd31a0, 0x00007f5280fd3268] -Event: 11.761 Thread 0x00007f5290184340 1809 4 java.lang.invoke.MethodType::makeImpl (109 bytes) -Event: 11.790 Thread 0x00007f523438f6f0 nmethod 1839 0x00007f5280fd3310 code [0x00007f5280fd34e0, 0x00007f5280fd4138] -Event: 11.790 Thread 0x00007f523438f6f0 1981 4 java.lang.Object::equals (11 bytes) -Event: 11.791 Thread 0x00007f523438f6f0 nmethod 1981 0x00007f5280fd4790 code [0x00007f5280fd4900, 0x00007f5280fd4998] -Event: 11.791 Thread 0x00007f523438f6f0 1977 4 scala.Option::foreach (20 bytes) -Event: 11.812 Thread 0x00007f523438f6f0 nmethod 1977 0x00007f5280fd4a90 code [0x00007f5280fd4c20, 0x00007f5280fd4d68] -Event: 11.812 Thread 0x00007f523438f6f0 1988 4 cats.effect.IO:: (5 bytes) -Event: 11.813 Thread 0x00007f523438f6f0 nmethod 1988 0x00007f5280fd4e90 code [0x00007f5280fd5000, 0x00007f5280fd5078] -Event: 11.813 Thread 0x00007f523438f6f0 1854 4 io.netty.incubator.channel.uring.UringCompletionQueue::ioUringWaitCqe (8 bytes) -Event: 11.816 Thread 0x00007f523438f6f0 nmethod 1854 0x00007f5280fd5190 code [0x00007f5280fd5320, 0x00007f5280fd5428] -Event: 11.817 Thread 0x00007f523438f6f0 1855 4 io.netty.incubator.channel.uring.UringSubmissionQueue::addTimeout (10 bytes) -Event: 11.879 Thread 0x00007f523438f6f0 nmethod 1855 0x00007f5280fd5510 code [0x00007f5280fd56a0, 0x00007f5280fd59d8] -Event: 11.879 Thread 0x00007f523438f6f0 2000 4 java.lang.ClassValue::get (31 bytes) -Event: 11.895 Thread 0x00007f523438f6f0 nmethod 2000 0x00007f5280fd5e10 code [0x00007f5280fd5fa0, 0x00007f5280fd61c8] -Event: 11.895 Thread 0x00007f523438f6f0 2010 4 cats.effect.tracing.TracingPlatform::calculateTracingEvent (36 bytes) -Event: 11.925 Thread 0x00007f523438f6f0 nmethod 2010 0x00007f5280fd6410 code [0x00007f5280fd65c0, 0x00007f5280fd6868] -Event: 11.925 Thread 0x00007f523438f6f0 2018 ! 4 java.lang.ref.ReferenceQueue::poll (28 bytes) - -GC Heap History (8 events): -Event: 2.547 GC heap before -{Heap before GC invocations=0 (full 0): - garbage-first heap total 258048K, used 22496K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 11 young (22528K), 0 survivors (0K) - Metaspace used 7389K, committed 7488K, reserved 1114112K - class space used 779K, committed 832K, reserved 1048576K -} -Event: 2.558 GC heap after -{Heap after GC invocations=1 (full 0): - garbage-first heap total 258048K, used 4874K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 7389K, committed 7488K, reserved 1114112K - class space used 779K, committed 832K, reserved 1048576K -} -Event: 5.887 GC heap before -{Heap before GC invocations=1 (full 0): - garbage-first heap total 258048K, used 35594K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 18 young (36864K), 2 survivors (4096K) - Metaspace used 16922K, committed 17088K, reserved 1114112K - class space used 1758K, committed 1856K, reserved 1048576K -} -Event: 5.903 GC heap after -{Heap after GC invocations=2 (full 0): - garbage-first heap total 258048K, used 6026K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 2 young (4096K), 2 survivors (4096K) - Metaspace used 16922K, committed 17088K, reserved 1114112K - class space used 1758K, committed 1856K, reserved 1048576K -} -Event: 8.154 GC heap before -{Heap before GC invocations=2 (full 0): - garbage-first heap total 258048K, used 24458K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 13 young (26624K), 2 survivors (4096K) - Metaspace used 21262K, committed 21504K, reserved 1114112K - class space used 2226K, committed 2304K, reserved 1048576K -} -Event: 8.173 GC heap after -{Heap after GC invocations=3 (full 0): - garbage-first heap total 258048K, used 6642K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 21262K, committed 21504K, reserved 1114112K - class space used 2226K, committed 2304K, reserved 1048576K -} -Event: 10.961 GC heap before -{Heap before GC invocations=4 (full 0): - garbage-first heap total 49152K, used 35314K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 19 young (38912K), 3 survivors (6144K) - Metaspace used 27645K, committed 27968K, reserved 1114112K - class space used 2968K, committed 3072K, reserved 1048576K -} -Event: 10.978 GC heap after -{Heap after GC invocations=5 (full 0): - garbage-first heap total 49152K, used 7918K [0x0000000708200000, 0x0000000800000000) - region size 2048K, 3 young (6144K), 3 survivors (6144K) - Metaspace used 27645K, committed 27968K, reserved 1114112K - class space used 2968K, committed 3072K, reserved 1048576K -} - -Dll operation events (11 events): -Event: 0.006 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -Event: 0.043 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -Event: 0.725 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -Event: 0.768 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -Event: 0.919 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -Event: 0.988 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -Event: 1.046 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -Event: 1.367 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -Event: 3.493 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -Event: 3.503 Loaded shared library /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -Event: 3.598 Loaded shared library /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so - -Deoptimization events (20 events): -Event: 11.676 Thread 0x00007f521453b510 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fccbcc relative=0x000000000000032c -Event: 11.676 Thread 0x00007f52145418f0 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fccbcc relative=0x000000000000032c -Event: 11.676 Thread 0x00007f521453b510 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fccbcc method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 -Event: 11.676 Thread 0x00007f521453b510 DEOPT PACKING pc=0x00007f5280fccbcc sp=0x00007f524a1194a0 -Event: 11.677 Thread 0x00007f52145418f0 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fccbcc method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 -Event: 11.678 Thread 0x00007f52145418f0 DEOPT PACKING pc=0x00007f5280fccbcc sp=0x00007f5249b194a0 -Event: 11.678 Thread 0x00007f52145418f0 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249b192e8 mode 2 -Event: 11.679 Thread 0x00007f521453b510 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f524a1192e8 mode 2 -Event: 11.745 Thread 0x00007f51ec003790 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280f34730 relative=0x0000000000000210 -Event: 11.745 Thread 0x00007f51ec003790 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280f34730 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 -Event: 11.745 Thread 0x00007f51ec003790 DEOPT PACKING pc=0x00007f5280f34730 sp=0x00007f52495133f0 -Event: 11.746 Thread 0x00007f51ec003790 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249513358 mode 2 -Event: 11.759 Thread 0x00007f5214540530 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280fd2008 relative=0x0000000000000608 -Event: 11.759 Thread 0x00007f5214540530 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280fd2008 method=scala.collection.mutable.HashMap.get(Ljava/lang/Object;)Lscala/Option; @ 33 c2 -Event: 11.759 Thread 0x00007f5214540530 DEOPT PACKING pc=0x00007f5280fd2008 sp=0x00007f5249c19520 -Event: 11.759 Thread 0x00007f521453b510 Uncommon trap: trap_request=0xffffff45 fr.pc=0x00007f5280f34f30 relative=0x0000000000000190 -Event: 11.759 Thread 0x00007f5214540530 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f5249c19240 mode 2 -Event: 11.759 Thread 0x00007f521453b510 Uncommon trap: reason=unstable_if action=reinterpret pc=0x00007f5280f34f30 method=fs2.io.uring.unsafe.UringSystem$Poller$$anon$2.handle(IIIBS)V @ 4 c2 -Event: 11.759 Thread 0x00007f521453b510 DEOPT PACKING pc=0x00007f5280f34f30 sp=0x00007f524a119330 -Event: 11.759 Thread 0x00007f521453b510 DEOPT UNPACKING pc=0x00007f528098d699 sp=0x00007f524a119308 mode 2 - -Classes unloaded (0 events): -No events - -Classes redefined (0 events): -No events - -Internal exceptions (20 events): -Event: 11.337 Thread 0x00007f521453bfc0 Exception (0x0000000716290440) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.376 Thread 0x00007f521453b510 Exception (0x00000007162b94e0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.377 Thread 0x00007f521453b510 Exception (0x00000007162bded0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.392 Thread 0x00007f52145418f0 Exception (0x00000007162c7948) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.429 Thread 0x00007f52145418f0 Exception (0x0000000716362dd8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.435 Thread 0x00007f52145418f0 Exception (0x0000000716367940) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.436 Thread 0x00007f52145418f0 Exception (0x000000071636c0f8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.495 Thread 0x00007f51ec003790 Exception (0x00000007163fc868) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.560 Thread 0x00007f51ec003790 Exception (0x00000007160b1a30) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.567 Thread 0x00007f51ec003790 Exception (0x00000007160c5998) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.570 Thread 0x00007f51ec003790 Exception (0x00000007160cd300) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.611 Thread 0x00007f521453cf70 Exception (0x00000007161291e8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.614 Thread 0x00007f521453cf70 Exception (0x0000000716135bb0) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.616 Thread 0x00007f521453cf70 Exception (0x000000071613a718) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.620 Thread 0x00007f521453cf70 Exception (0x0000000716143640) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.624 Thread 0x00007f521453cf70 Exception (0x000000071614fe40) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.626 Thread 0x00007f521453cf70 Exception (0x00000007161549a8) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.680 Thread 0x00007f5214542800 Exception (0x00000007161e9d28) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.704 Thread 0x00007f5214542800 Exception (0x00000007161f9e58) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] -Event: 11.722 Thread 0x00007f5214542800 Exception (0x0000000715e67958) -thrown [./src/hotspot/share/interpreter/linkResolver.cpp, line 758] - -VM Operations (20 events): -Event: 6.904 Executing VM operation: Cleanup -Event: 6.906 Executing VM operation: Cleanup done -Event: 7.161 Executing VM operation: HandshakeAllThreads -Event: 7.162 Executing VM operation: HandshakeAllThreads done -Event: 8.154 Executing VM operation: CollectForMetadataAllocation -Event: 8.173 Executing VM operation: CollectForMetadataAllocation done -Event: 8.207 Executing VM operation: G1Concurrent -Event: 8.214 Executing VM operation: G1Concurrent done -Event: 8.214 Executing VM operation: G1Concurrent -Event: 8.215 Executing VM operation: G1Concurrent done -Event: 8.297 Executing VM operation: HandshakeAllThreads -Event: 8.297 Executing VM operation: HandshakeAllThreads done -Event: 9.297 Executing VM operation: Cleanup -Event: 9.298 Executing VM operation: Cleanup done -Event: 9.929 Executing VM operation: HandshakeAllThreads -Event: 9.929 Executing VM operation: HandshakeAllThreads done -Event: 10.929 Executing VM operation: Cleanup -Event: 10.930 Executing VM operation: Cleanup done -Event: 10.961 Executing VM operation: G1CollectForAllocation -Event: 10.978 Executing VM operation: G1CollectForAllocation done - -Events (20 events): -Event: 9.129 Protecting memory [0x00007f524981b000,0x00007f524981f000] with protection modes 0 -Event: 9.928 loading class java/security/cert/X509Certificate -Event: 9.928 loading class java/security/cert/X509Extension -Event: 9.928 loading class java/security/cert/X509Extension done -Event: 9.928 loading class java/security/cert/X509Certificate done -Event: 9.929 loading class javax/security/cert/X509Certificate -Event: 9.930 loading class javax/security/cert/Certificate -Event: 9.930 loading class javax/security/cert/Certificate done -Event: 9.930 loading class javax/security/cert/X509Certificate done -Event: 9.932 loading class java/nio/ReadOnlyBufferException -Event: 9.932 loading class java/nio/ReadOnlyBufferException done -Event: 9.959 loading class java/util/concurrent/atomic/Striped64$1 -Event: 9.959 loading class java/util/concurrent/atomic/Striped64$1 done -Event: 10.831 Thread 0x00007f523020de90 Thread exited: 0x00007f523020de90 -Event: 10.938 Thread 0x00007f523020de90 Thread added: 0x00007f523020de90 -Event: 10.939 Protecting memory [0x00007f524981b000,0x00007f524981f000] with protection modes 0 -Event: 11.294 Thread 0x00007f51ec003790 Thread added: 0x00007f51ec003790 -Event: 11.294 Protecting memory [0x00007f5249415000,0x00007f5249419000] with protection modes 0 -Event: 11.349 Thread 0x00007f523438f6f0 Thread added: 0x00007f523438f6f0 -Event: 11.354 Protecting memory [0x00007f5249315000,0x00007f5249319000] with protection modes 0 - - -Dynamic libraries: -708200000-70a400000 rw-p 00000000 00:00 0 -70a400000-715e00000 ---p 00000000 00:00 0 -715e00000-716400000 rw-p 00000000 00:00 0 -716400000-717600000 ---p 00000000 00:00 0 -717600000-717a00000 rw-p 00000000 00:00 0 -717a00000-7ffc00000 ---p 00000000 00:00 0 -7ffc00000-7ffd00000 rw-p 00000000 00:00 0 -7ffd00000-7ffd76000 rw-p 00c9d000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -7ffd76000-7ffe00000 rw-p 00000000 00:00 0 -7ffe00000-7ffe82000 rw-p 00c1b000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -7ffe82000-800000000 rw-p 00000000 00:00 0 -800000000-800be2000 rw-p 00001000 103:02 1966440 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/classes.jsa -800be2000-801000000 ---p 00000000 00:00 0 -801000000-801030000 rw-p 00000000 00:00 0 -801030000-801130000 rw-p 00000000 00:00 0 -801130000-801170000 rw-p 00000000 00:00 0 -801170000-8011b0000 rw-p 00000000 00:00 0 -8011b0000-8011d0000 rw-p 00000000 00:00 0 -8011d0000-801200000 ---p 00000000 00:00 0 -801200000-801390000 rw-p 00000000 00:00 0 -801390000-841000000 ---p 00000000 00:00 0 -55843ef12000-55843ef13000 r--p 00000000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -55843ef13000-55843ef14000 r-xp 00001000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -55843ef14000-55843ef15000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -55843ef15000-55843ef16000 r--p 00002000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -55843ef16000-55843ef17000 rw-p 00003000 103:02 1966288 /usr/lib/jvm/java-17-openjdk-amd64/bin/java -55844037f000-5584403c7000 rw-p 00000000 00:00 0 [heap] -7f51c0000000-7f51c0052000 rw-p 00000000 00:00 0 -7f51c0052000-7f51c4000000 ---p 00000000 00:00 0 -7f51c4000000-7f51c434a000 rw-p 00000000 00:00 0 -7f51c434a000-7f51c8000000 ---p 00000000 00:00 0 -7f51c8000000-7f51c838b000 rw-p 00000000 00:00 0 -7f51c838b000-7f51cc000000 ---p 00000000 00:00 0 -7f51cc000000-7f51cc4ce000 rw-p 00000000 00:00 0 -7f51cc4ce000-7f51d0000000 ---p 00000000 00:00 0 -7f51d0000000-7f51d00ec000 rw-p 00000000 00:00 0 -7f51d00ec000-7f51d4000000 ---p 00000000 00:00 0 -7f51d4000000-7f51d4021000 rw-p 00000000 00:00 0 -7f51d4021000-7f51d8000000 ---p 00000000 00:00 0 -7f51d8000000-7f51d80b0000 rw-p 00000000 00:00 0 -7f51d80b0000-7f51dc000000 ---p 00000000 00:00 0 -7f51dc000000-7f51dc1f1000 rw-p 00000000 00:00 0 -7f51dc1f1000-7f51e0000000 ---p 00000000 00:00 0 -7f51e0000000-7f51e0021000 rw-p 00000000 00:00 0 -7f51e0021000-7f51e4000000 ---p 00000000 00:00 0 -7f51e4000000-7f51e42c3000 rw-p 00000000 00:00 0 -7f51e42c3000-7f51e8000000 ---p 00000000 00:00 0 -7f51e8000000-7f51e80fd000 rw-p 00000000 00:00 0 -7f51e80fd000-7f51ec000000 ---p 00000000 00:00 0 -7f51ec000000-7f51ec03a000 rw-p 00000000 00:00 0 -7f51ec03a000-7f51f0000000 ---p 00000000 00:00 0 -7f51f0000000-7f51f004a000 rw-p 00000000 00:00 0 -7f51f004a000-7f51f4000000 ---p 00000000 00:00 0 -7f51f4000000-7f51f4161000 rw-p 00000000 00:00 0 -7f51f4161000-7f51f8000000 ---p 00000000 00:00 0 -7f51f8000000-7f51f8021000 rw-p 00000000 00:00 0 -7f51f8021000-7f51fc000000 ---p 00000000 00:00 0 -7f51fc000000-7f51fc1b5000 rw-p 00000000 00:00 0 -7f51fc1b5000-7f5200000000 ---p 00000000 00:00 0 -7f5200000000-7f5200021000 rw-p 00000000 00:00 0 -7f5200021000-7f5204000000 ---p 00000000 00:00 0 -7f5204000000-7f5204021000 rw-p 00000000 00:00 0 -7f5204021000-7f5208000000 ---p 00000000 00:00 0 -7f5208000000-7f5208021000 rw-p 00000000 00:00 0 -7f5208021000-7f520c000000 ---p 00000000 00:00 0 -7f520c000000-7f520c021000 rw-p 00000000 00:00 0 -7f520c021000-7f5210000000 ---p 00000000 00:00 0 -7f5210000000-7f5210021000 rw-p 00000000 00:00 0 -7f5210021000-7f5214000000 ---p 00000000 00:00 0 -7f5214000000-7f5214603000 rw-p 00000000 00:00 0 -7f5214603000-7f5218000000 ---p 00000000 00:00 0 -7f5218000000-7f5218021000 rw-p 00000000 00:00 0 -7f5218021000-7f521c000000 ---p 00000000 00:00 0 -7f521c000000-7f521c021000 rw-p 00000000 00:00 0 -7f521c021000-7f5220000000 ---p 00000000 00:00 0 -7f5220000000-7f5220081000 rw-p 00000000 00:00 0 -7f5220081000-7f5224000000 ---p 00000000 00:00 0 -7f5224000000-7f5224021000 rw-p 00000000 00:00 0 -7f5224021000-7f5228000000 ---p 00000000 00:00 0 -7f5228000000-7f5228021000 rw-p 00000000 00:00 0 -7f5228021000-7f522c000000 ---p 00000000 00:00 0 -7f522c000000-7f522c021000 rw-p 00000000 00:00 0 -7f522c021000-7f5230000000 ---p 00000000 00:00 0 -7f5230000000-7f523080e000 rw-p 00000000 00:00 0 -7f523080e000-7f5234000000 ---p 00000000 00:00 0 -7f5234000000-7f52346cf000 rw-p 00000000 00:00 0 -7f52346cf000-7f5238000000 ---p 00000000 00:00 0 -7f5238000000-7f5238021000 rw-p 00000000 00:00 0 -7f5238021000-7f523c000000 ---p 00000000 00:00 0 -7f523c000000-7f523c021000 rw-p 00000000 00:00 0 -7f523c021000-7f5240000000 ---p 00000000 00:00 0 -7f5240000000-7f5240021000 rw-p 00000000 00:00 0 -7f5240021000-7f5244000000 ---p 00000000 00:00 0 -7f5244000000-7f5244021000 rw-p 00000000 00:00 0 -7f5244021000-7f5248000000 ---p 00000000 00:00 0 -7f5248e2b000-7f5249211000 rw-p 00000000 00:00 0 -7f5249315000-7f5249319000 ---p 00000000 00:00 0 -7f5249319000-7f5249415000 rw-p 00000000 00:00 0 -7f5249415000-7f5249419000 ---p 00000000 00:00 0 -7f5249419000-7f5249515000 rw-p 00000000 00:00 0 -7f5249515000-7f5249516000 ---p 00000000 00:00 0 -7f5249516000-7f5249617000 rw-p 00000000 00:00 0 -7f5249617000-7f5249618000 ---p 00000000 00:00 0 -7f5249618000-7f5249719000 rw-p 00000000 00:00 0 -7f5249719000-7f524971a000 ---p 00000000 00:00 0 -7f524971a000-7f524981b000 rw-p 00000000 00:00 0 -7f524981b000-7f524981f000 ---p 00000000 00:00 0 -7f524981f000-7f524991b000 rw-p 00000000 00:00 0 -7f524991b000-7f524991f000 ---p 00000000 00:00 0 -7f524991f000-7f5249a1b000 rw-p 00000000 00:00 0 -7f5249a1b000-7f5249a1f000 ---p 00000000 00:00 0 -7f5249a1f000-7f5249b1b000 rw-p 00000000 00:00 0 -7f5249b1b000-7f5249b1f000 ---p 00000000 00:00 0 -7f5249b1f000-7f5249c1b000 rw-p 00000000 00:00 0 -7f5249c1b000-7f5249c1f000 ---p 00000000 00:00 0 -7f5249c1f000-7f5249d1b000 rw-p 00000000 00:00 0 -7f5249d1b000-7f5249d1f000 ---p 00000000 00:00 0 -7f5249d1f000-7f5249e1b000 rw-p 00000000 00:00 0 -7f5249e1b000-7f5249e1f000 ---p 00000000 00:00 0 -7f5249e1f000-7f5249f1b000 rw-p 00000000 00:00 0 -7f5249f1b000-7f5249f1f000 ---p 00000000 00:00 0 -7f5249f1f000-7f524a01b000 rw-p 00000000 00:00 0 -7f524a01b000-7f524a01f000 ---p 00000000 00:00 0 -7f524a01f000-7f524a11b000 rw-p 00000000 00:00 0 -7f524a11b000-7f524a15b000 rw-s 10000000 00:0e 65771 anon_inode:[io_uring] -7f524a15b000-7f524a180000 rw-s 00000000 00:0e 65771 anon_inode:[io_uring] -7f524a180000-7f524a1c0000 rw-s 10000000 00:0e 65769 anon_inode:[io_uring] -7f524a1c0000-7f524a200000 rw-s 10000000 00:0e 65767 anon_inode:[io_uring] -7f524a200000-7f524a211000 r-xp 00000000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) -7f524a211000-7f524a411000 ---p 00011000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) -7f524a411000-7f524a413000 rw-p 00011000 103:02 3673027 /tmp/libnetty_transport_native_io_uring_x86_644034392916556200957.so (deleted) -7f524a416000-7f524a43b000 rw-s 00000000 00:0e 65769 anon_inode:[io_uring] -7f524a43b000-7f524a460000 rw-s 00000000 00:0e 65767 anon_inode:[io_uring] -7f524a460000-7f524a4a0000 rw-s 10000000 00:0e 65765 anon_inode:[io_uring] -7f524a4a0000-7f524a4c5000 rw-s 00000000 00:0e 65765 anon_inode:[io_uring] -7f524a4c5000-7f524a505000 rw-s 10000000 00:0e 65763 anon_inode:[io_uring] -7f524a505000-7f524a52a000 rw-s 00000000 00:0e 65763 anon_inode:[io_uring] -7f524a52a000-7f524a56a000 rw-s 10000000 00:0e 65761 anon_inode:[io_uring] -7f524a56a000-7f524a58f000 rw-s 00000000 00:0e 65761 anon_inode:[io_uring] -7f524a58f000-7f524a5cf000 rw-s 10000000 00:0e 65759 anon_inode:[io_uring] -7f524a5cf000-7f524a5f4000 rw-s 00000000 00:0e 65759 anon_inode:[io_uring] -7f524a5f4000-7f524a5f8000 ---p 00000000 00:00 0 -7f524a5f8000-7f524a6f4000 rw-p 00000000 00:00 0 -7f524a6f4000-7f524a6f5000 ---p 00000000 00:00 0 -7f524a6f5000-7f524a7f6000 rw-p 00000000 00:00 0 -7f524a7f6000-7f524a7f7000 ---p 00000000 00:00 0 -7f524a7f7000-7f524a8f8000 rw-p 00000000 00:00 0 -7f524a8f8000-7f524a8f9000 ---p 00000000 00:00 0 -7f524a8f9000-7f524a9fa000 rw-p 00000000 00:00 0 -7f524a9fa000-7f524a9fb000 ---p 00000000 00:00 0 -7f524a9fb000-7f524aafc000 rw-p 00000000 00:00 0 -7f524aafc000-7f524aafd000 ---p 00000000 00:00 0 -7f524aafd000-7f524abfe000 rw-p 00000000 00:00 0 -7f524abfe000-7f524ac02000 ---p 00000000 00:00 0 -7f524ac02000-7f524acfe000 rw-p 00000000 00:00 0 -7f524acfe000-7f524ad02000 ---p 00000000 00:00 0 -7f524ad02000-7f524adfe000 rw-p 00000000 00:00 0 -7f524adfe000-7f524adff000 ---p 00000000 00:00 0 -7f524adff000-7f524af00000 rw-p 00000000 00:00 0 -7f524af00000-7f524af04000 ---p 00000000 00:00 0 -7f524af04000-7f524b000000 rw-p 00000000 00:00 0 -7f524b000000-7f524b004000 ---p 00000000 00:00 0 -7f524b004000-7f524b100000 rw-p 00000000 00:00 0 -7f524b100000-7f524b104000 ---p 00000000 00:00 0 -7f524b104000-7f524b200000 rw-p 00000000 00:00 0 -7f524b200000-7f524bfea000 r--p 00000000 103:02 1055507 /usr/lib/locale/locale-archive -7f524c000000-7f524c021000 rw-p 00000000 00:00 0 -7f524c021000-7f5250000000 ---p 00000000 00:00 0 -7f5250000000-7f5250021000 rw-p 00000000 00:00 0 -7f5250021000-7f5254000000 ---p 00000000 00:00 0 -7f5254000000-7f52543f0000 rw-p 00000000 00:00 0 -7f52543f0000-7f5254d90000 rw-p 00000000 00:00 0 -7f5254d90000-7f5254db0000 rw-p 00000000 00:00 0 -7f5254db0000-7f5254e90000 rw-p 00000000 00:00 0 -7f5254e90000-7f5254f00000 ---p 00000000 00:00 0 -7f5254f00000-7f5255a60000 rw-p 00000000 00:00 0 -7f5255a60000-7f5258000000 ---p 00000000 00:00 0 -7f5258000000-7f5258021000 rw-p 00000000 00:00 0 -7f5258021000-7f525c000000 ---p 00000000 00:00 0 -7f525c000000-7f525c021000 rw-p 00000000 00:00 0 -7f525c021000-7f5260000000 ---p 00000000 00:00 0 -7f5260000000-7f5260021000 rw-p 00000000 00:00 0 -7f5260021000-7f5264000000 ---p 00000000 00:00 0 -7f5264000000-7f5264021000 rw-p 00000000 00:00 0 -7f5264021000-7f5268000000 ---p 00000000 00:00 0 -7f5268000000-7f5268021000 rw-p 00000000 00:00 0 -7f5268021000-7f526c000000 ---p 00000000 00:00 0 -7f526c000000-7f526c040000 rw-s 10000000 00:0e 65757 anon_inode:[io_uring] -7f526c040000-7f526c065000 rw-s 00000000 00:0e 65757 anon_inode:[io_uring] -7f526c065000-7f526c069000 ---p 00000000 00:00 0 -7f526c069000-7f526c165000 rw-p 00000000 00:00 0 -7f526c165000-7f526c169000 ---p 00000000 00:00 0 -7f526c169000-7f526c265000 rw-p 00000000 00:00 0 -7f526c265000-7f526c269000 ---p 00000000 00:00 0 -7f526c269000-7f526c365000 rw-p 00000000 00:00 0 -7f526c365000-7f526c369000 ---p 00000000 00:00 0 -7f526c369000-7f526c465000 rw-p 00000000 00:00 0 -7f526c465000-7f526c469000 ---p 00000000 00:00 0 -7f526c469000-7f526c565000 rw-p 00000000 00:00 0 -7f526c565000-7f526c569000 ---p 00000000 00:00 0 -7f526c569000-7f526c665000 rw-p 00000000 00:00 0 -7f526c665000-7f526c669000 ---p 00000000 00:00 0 -7f526c669000-7f526c765000 rw-p 00000000 00:00 0 -7f526c765000-7f526c769000 ---p 00000000 00:00 0 -7f526c769000-7f526c865000 rw-p 00000000 00:00 0 -7f526c865000-7f526c869000 ---p 00000000 00:00 0 -7f526c869000-7f526c965000 rw-p 00000000 00:00 0 -7f526c965000-7f526c966000 ---p 00000000 00:00 0 -7f526c966000-7f526ca67000 rw-p 00000000 00:00 0 -7f526ca67000-7f526ca6c000 r--p 00000000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f526ca6c000-7f526caad000 r-xp 00005000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f526caad000-7f526cb36000 r--p 00046000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f526cb36000-7f526cb37000 r--p 000ce000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f526cb37000-7f526cb38000 rw-p 000cf000 103:02 1966422 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjsvml.so -7f526cb38000-7f526cbb9000 rw-p 00000000 00:00 0 -7f526cbb9000-7f526cbba000 ---p 00000000 00:00 0 -7f526cbba000-7f526ccbb000 rw-p 00000000 00:00 0 -7f526ccbb000-7f526ccbc000 ---p 00000000 00:00 0 -7f526ccbc000-7f526f5c5000 rw-p 00000000 00:00 0 -7f526f5c5000-7f526f5c6000 ---p 00000000 00:00 0 -7f526f5c6000-7f526f6c7000 rw-p 00000000 00:00 0 -7f526f6c7000-7f526f6c8000 ---p 00000000 00:00 0 -7f526f6c8000-7f526f7c9000 rw-p 00000000 00:00 0 -7f526f7c9000-7f526f7ca000 ---p 00000000 00:00 0 -7f526f7ca000-7f526f953000 rw-p 00000000 00:00 0 -7f526f953000-7f526fc3b000 ---p 00000000 00:00 0 -7f526fc3b000-7f526fc53000 rw-p 00000000 00:00 0 -7f526fc53000-7f526fc9b000 ---p 00000000 00:00 0 -7f526fc9b000-7f526fcab000 rw-p 00000000 00:00 0 -7f526fcab000-7f52736b3000 ---p 00000000 00:00 0 -7f52736b3000-7f527374b000 rw-p 00000000 00:00 0 -7f527374b000-7f5273a33000 ---p 00000000 00:00 0 -7f5273a33000-7f5273a4b000 rw-p 00000000 00:00 0 -7f5273a4b000-7f5273a93000 ---p 00000000 00:00 0 -7f5273a93000-7f5273aa3000 rw-p 00000000 00:00 0 -7f5273aa3000-7f52774ab000 ---p 00000000 00:00 0 -7f52774ab000-7f52774cc000 rw-p 00000000 00:00 0 -7f52774cc000-7f5277529000 ---p 00000000 00:00 0 -7f5277529000-7f527752c000 rw-p 00000000 00:00 0 -7f527752c000-7f5277535000 ---p 00000000 00:00 0 -7f5277535000-7f5277537000 rw-p 00000000 00:00 0 -7f5277537000-7f5277c78000 ---p 00000000 00:00 0 -7f5277c78000-7f5277c8b000 rw-p 00000000 00:00 0 -7f5277c8b000-7f5277ce8000 ---p 00000000 00:00 0 -7f5277ce8000-7f5277ceb000 rw-p 00000000 00:00 0 -7f5277ceb000-7f5277cf4000 ---p 00000000 00:00 0 -7f5277cf4000-7f5277cf6000 rw-p 00000000 00:00 0 -7f5277cf6000-7f5278437000 ---p 00000000 00:00 0 -7f5278437000-7f527844a000 rw-p 00000000 00:00 0 -7f527844a000-7f52784a7000 ---p 00000000 00:00 0 -7f52784a7000-7f52784aa000 rw-p 00000000 00:00 0 -7f52784aa000-7f52784b3000 ---p 00000000 00:00 0 -7f52784b3000-7f52784b5000 rw-p 00000000 00:00 0 -7f52784b5000-7f5278bf6000 ---p 00000000 00:00 0 -7f5278bf6000-7f5279400000 rw-p 00000000 00:00 0 -7f5279400000-7f52797c0000 rwxp 00000000 00:00 0 -7f52797c0000-7f5280937000 ---p 00000000 00:00 0 -7f5280937000-7f5280ba7000 rwxp 00000000 00:00 0 -7f5280ba7000-7f5280ec9000 ---p 00000000 00:00 0 -7f5280ec9000-7f5281139000 rwxp 00000000 00:00 0 -7f5281139000-7f5288400000 ---p 00000000 00:00 0 -7f5288400000-7f528fe42000 r--s 00000000 103:02 1966437 /usr/lib/jvm/java-17-openjdk-amd64/lib/modules -7f528fe43000-7f528fe44000 r--p 00000000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f528fe44000-7f528fe45000 r-xp 00001000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f528fe45000-7f528fe46000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f528fe46000-7f528fe47000 r--p 00002000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f528fe47000-7f528fe48000 rw-p 00003000 103:02 1072681 /usr/lib/x86_64-linux-gnu/libdl.so.2 -7f528fe48000-7f528ff1a000 rw-p 00000000 00:00 0 -7f528ff1a000-7f5290000000 ---p 00000000 00:00 0 -7f5290000000-7f5290412000 rw-p 00000000 00:00 0 -7f5290412000-7f5294000000 ---p 00000000 00:00 0 -7f5294001000-7f5294002000 r--p 00000000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f5294002000-7f5294003000 r-xp 00001000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f5294003000-7f5294004000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f5294004000-7f5294005000 r--p 00002000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f5294005000-7f5294006000 rw-p 00003000 103:02 1073090 /usr/lib/x86_64-linux-gnu/librt.so.1 -7f5294015000-7f529401d000 rw-p 00000000 00:00 0 -7f529401d000-7f5294100000 ---p 00000000 00:00 0 -7f5294100000-7f5294104000 ---p 00000000 00:00 0 -7f5294104000-7f5294200000 rw-p 00000000 00:00 0 -7f5294200000-7f529429c000 r--p 00000000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f529429c000-7f52943cb000 r-xp 0009c000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f52943cb000-7f5294458000 r--p 001cb000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f5294458000-7f5294463000 r--p 00257000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f5294463000-7f5294466000 rw-p 00262000 103:02 1073143 /usr/lib/x86_64-linux-gnu/libstdc++.so.6.0.31 -7f5294466000-7f529446a000 rw-p 00000000 00:00 0 -7f529446d000-7f529446f000 r--p 00000000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f529446f000-7f5294472000 r-xp 00002000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f5294472000-7f5294474000 r--p 00005000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f5294474000-7f5294475000 r--p 00006000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f5294475000-7f5294476000 rw-p 00007000 103:02 1966426 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement_ext.so -7f5294476000-7f5294478000 r--p 00000000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f5294478000-7f5294479000 r-xp 00002000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f5294479000-7f529447a000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f529447a000-7f529447b000 r--p 00003000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f529447b000-7f529447c000 rw-p 00004000 103:02 1966424 /usr/lib/jvm/java-17-openjdk-amd64/lib/libmanagement.so -7f529447c000-7f5294480000 r--p 00000000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f5294480000-7f5294487000 r-xp 00004000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f5294487000-7f5294489000 r--p 0000b000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f5294489000-7f529448b000 r--p 0000c000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f529448b000-7f529448c000 rw-p 0000e000 103:02 1966435 /usr/lib/jvm/java-17-openjdk-amd64/lib/libverify.so -7f529448c000-7f529448e000 r--p 00000000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f529448e000-7f5294493000 r-xp 00002000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f5294493000-7f5294495000 r--p 00007000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f5294495000-7f5294496000 r--p 00008000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f5294496000-7f5294497000 rw-p 00009000 103:02 1966436 /usr/lib/jvm/java-17-openjdk-amd64/lib/libzip.so -7f5294497000-7f5294517000 rw-p 00000000 00:00 0 -7f5294517000-7f5294525000 r--p 00000000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f5294525000-7f52945a3000 r-xp 0000e000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f52945a3000-7f52945fe000 r--p 0008c000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f52945fe000-7f52945ff000 r--p 000e6000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f52945ff000-7f5294600000 rw-p 000e7000 103:02 1072926 /usr/lib/x86_64-linux-gnu/libm.so.6 -7f5294600000-7f5294851000 r--p 00000000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f5294851000-7f52955d5000 r-xp 00251000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f52955d5000-7f5295856000 r--p 00fd5000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f5295856000-7f529590e000 r--p 01255000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f529590e000-7f5295943000 rw-p 0130d000 103:02 1966443 /usr/lib/jvm/java-17-openjdk-amd64/lib/server/libjvm.so -7f5295943000-7f529599d000 rw-p 00000000 00:00 0 -7f52959a0000-7f52959a1000 r--p 00000000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f52959a1000-7f52959a2000 r-xp 00001000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f52959a2000-7f52959a3000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f52959a3000-7f52959a4000 r--p 00002000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f52959a4000-7f52959a5000 rw-p 00003000 103:02 1966408 /usr/lib/jvm/java-17-openjdk-amd64/lib/libextnet.so -7f52959a5000-7f52959a9000 r--p 00000000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f52959a9000-7f52959b7000 r-xp 00004000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f52959b7000-7f52959bb000 r--p 00012000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f52959bb000-7f52959bc000 r--p 00015000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f52959bc000-7f52959bd000 rw-p 00016000 103:02 1966428 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnet.so -7f52959bd000-7f52959c3000 r--p 00000000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f52959c3000-7f52959cc000 r-xp 00006000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f52959cc000-7f52959d0000 r--p 0000f000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f52959d0000-7f52959d1000 r--p 00013000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f52959d1000-7f52959d2000 rw-p 00014000 103:02 1966429 /usr/lib/jvm/java-17-openjdk-amd64/lib/libnio.so -7f52959d2000-7f52959d3000 r--p 00000000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f52959d3000-7f52959d6000 r-xp 00001000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f52959d6000-7f52959d7000 r--p 00004000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f52959d7000-7f52959d8000 r--p 00005000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f52959d8000-7f52959d9000 rw-p 00006000 103:02 1966407 /usr/lib/jvm/java-17-openjdk-amd64/lib/libdt_socket.so -7f52959d9000-7f52959e5000 r--p 00000000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f52959e5000-7f52959f7000 r-xp 0000c000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f52959f7000-7f52959fd000 r--p 0001e000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f52959fd000-7f52959fe000 r--p 00023000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f52959fe000-7f52959ff000 rw-p 00024000 103:02 1966415 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjava.so -7f52959ff000-7f5295a00000 rw-p 00000000 00:00 0 -7f5295a00000-7f5295a22000 r--p 00000000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f5295a22000-7f5295b9a000 r-xp 00022000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f5295b9a000-7f5295bf2000 r--p 0019a000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f5295bf2000-7f5295bf6000 r--p 001f1000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f5295bf6000-7f5295bf8000 rw-p 001f5000 103:02 1072624 /usr/lib/x86_64-linux-gnu/libc.so.6 -7f5295bf8000-7f5295c05000 rw-p 00000000 00:00 0 -7f5295c09000-7f5295c10000 r--s 00000000 103:02 1074285 /usr/lib/x86_64-linux-gnu/gconv/gconv-modules.cache -7f5295c10000-7f5295c15000 rw-p 00000000 00:00 0 -7f5295c15000-7f5295c1c000 ---p 00000000 00:00 0 -7f5295c1c000-7f5295c24000 rw-s 00000000 103:02 3673026 /tmp/hsperfdata_antonio/8969 -7f5295c24000-7f5295c27000 r--p 00000000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f5295c27000-7f5295c55000 r-xp 00003000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f5295c55000-7f5295c63000 r--p 00031000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f5295c63000-7f5295c64000 r--p 0003e000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f5295c64000-7f5295c65000 rw-p 0003f000 103:02 1966417 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjdwp.so -7f5295c65000-7f5295c66000 rw-p 00000000 00:00 0 -7f5295c66000-7f5295c69000 r--p 00000000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f5295c69000-7f5295c84000 r-xp 00003000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f5295c84000-7f5295c88000 r--p 0001e000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f5295c88000-7f5295c89000 r--p 00021000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f5295c89000-7f5295c8a000 rw-p 00022000 103:02 1072753 /usr/lib/x86_64-linux-gnu/libgcc_s.so.1 -7f5295c8a000-7f5295c8d000 rw-p 00000000 00:00 0 -7f5295c8d000-7f5295c90000 r--p 00000000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f5295c90000-7f5295ca2000 r-xp 00003000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f5295ca2000-7f5295ca9000 r--p 00015000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f5295ca9000-7f5295caa000 r--p 0001b000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f5295caa000-7f5295cab000 rw-p 0001c000 103:02 1073270 /usr/lib/x86_64-linux-gnu/libz.so.1.2.13 -7f5295caf000-7f5295cb0000 ---p 00000000 00:00 0 -7f5295cb0000-7f5295cb1000 r--p 00000000 00:00 0 -7f5295cb1000-7f5295cb2000 ---p 00000000 00:00 0 -7f5295cb2000-7f5295cb4000 r--p 00000000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f5295cb4000-7f5295cb7000 r-xp 00002000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f5295cb7000-7f5295cb8000 r--p 00005000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f5295cb8000-7f5295cb9000 r--p 00006000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f5295cb9000-7f5295cba000 rw-p 00007000 103:02 1966418 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjimage.so -7f5295cba000-7f5295cbc000 r--p 00000000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f5295cbc000-7f5295cc6000 r-xp 00002000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f5295cc6000-7f5295cc9000 r--p 0000c000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f5295cc9000-7f5295cca000 r--p 0000e000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f5295cca000-7f5295ccb000 rw-p 0000f000 103:02 1966419 /usr/lib/jvm/java-17-openjdk-amd64/lib/libjli.so -7f5295ccb000-7f5295ccd000 rw-p 00000000 00:00 0 -7f5295ccd000-7f5295cce000 r--p 00000000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f5295cce000-7f5295cf6000 r-xp 00001000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f5295cf6000-7f5295d00000 r--p 00029000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f5295d00000-7f5295d02000 r--p 00033000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7f5295d02000-7f5295d04000 rw-p 00035000 103:02 1072442 /usr/lib/x86_64-linux-gnu/ld-linux-x86-64.so.2 -7ffe4c443000-7ffe4c465000 rw-p 00000000 00:00 0 [stack] -7ffe4c4c6000-7ffe4c4ca000 r--p 00000000 00:00 0 [vvar] -7ffe4c4ca000-7ffe4c4cc000 r-xp 00000000 00:00 0 [vdso] -ffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall] - - -VM Arguments: -jvm_args: -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,quiet=n -Duser.dir=/home/antonio/Programming/Scala/Typelevel/fs2-io_uring -java_command: sbt.ForkMain 33543 -java_class_path (initial): /home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/test-classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/Programming/Scala/Typelevel/fs2-io_uring/.bloop/uringJVM/bloop-bsp-clients-classes/classes-Metals-tBWvfpC2R1qPaYPy-U1v4A==:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scala-lang/scala-library/2.13.10/scala-library-2.13.10.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect_2.13/3.6-0142603/cats-effect_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-io_2.13/3.7.0/fs2-io_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/munit-cats-effect_2.13/2.0.0-M3/munit-cats-effect_2.13-2.0.0-M3.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-native-io_uring/0.0.21.Final/netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-kernel_2.13/3.6-0142603/cats-effect-kernel_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-effect-std_2.13/3.6-0142603/cats-effect-std_2.13-3.6-0142603.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/co/fs2/fs2-core_2.13/3.7.0/fs2-core_2.13-3.7.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/com/comcast/ip4s-core_2.13/3.3.0/ip4s-core_2.13-3.3.0.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/scalameta/munit_2.13/1.0.0-M6/munit_2.13-1.0.0-M6.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/io/netty/incubator/netty-incubator-transport-classes-io_uring/0.0.21.Final/netty-incubator-transport-classes-io_uring-0.0.21.Final.jar:/home/antonio/.cache/coursier/v1/https/repo1.maven.org/maven2/org/typelevel/cats-core_2.13/2.9.0/cats-core_2.13-2.9.0.ja -Launcher Type: SUN_STANDARD - -[Global flags] - intx CICompilerCount = 4 {product} {ergonomic} - uint ConcGCThreads = 2 {product} {ergonomic} - uint G1ConcRefinementThreads = 8 {product} {ergonomic} - size_t G1HeapRegionSize = 2097152 {product} {ergonomic} - uintx GCDrainStackTargetSize = 64 {product} {ergonomic} - size_t InitialHeapSize = 260046848 {product} {ergonomic} - size_t MarkStackSize = 4194304 {product} {ergonomic} - size_t MaxHeapSize = 4158652416 {product} {ergonomic} - size_t MaxNewSize = 2493513728 {product} {ergonomic} - size_t MinHeapDeltaBytes = 2097152 {product} {ergonomic} - size_t MinHeapSize = 8388608 {product} {ergonomic} - uintx NonNMethodCodeHeapSize = 5839372 {pd product} {ergonomic} - uintx NonProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} - uintx ProfiledCodeHeapSize = 122909434 {pd product} {ergonomic} - uintx ReservedCodeCacheSize = 251658240 {pd product} {ergonomic} - bool SegmentedCodeCache = true {product} {ergonomic} - size_t SoftMaxHeapSize = 4158652416 {manageable} {ergonomic} - bool UseCompressedClassPointers = true {product lp64_product} {ergonomic} - bool UseCompressedOops = true {product lp64_product} {ergonomic} - bool UseG1GC = true {product} {ergonomic} - -Logging: -Log output configuration: - #0: stdout all=warning uptime,level,tags - #1: stderr all=off uptime,level,tags - -Environment Variables: -JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64/ -PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/games:/usr/local/games:/snap/bin:/snap/bin:/home/antonio/.local/share/coursier/bin:/home/antonio/.local/share/coursier/bin -USERNAME=antonio -SHELL=/bin/bash -DISPLAY=:0 -LANG=en_US.UTF-8 - -Active Locale: -LC_ALL=en_US.UTF-8 -LC_COLLATE=en_US.UTF-8 -LC_CTYPE=en_US.UTF-8 -LC_MESSAGES=en_US.UTF-8 -LC_MONETARY=en_US.UTF-8 -LC_NUMERIC=en_US.UTF-8 -LC_TIME=en_US.UTF-8 - -Signal Handlers: - SIGSEGV: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGBUS: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGFPE: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGPIPE: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGXFSZ: javaSignalHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGILL: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - SIGUSR2: SR_handler in libjvm.so, mask=00100000000000000000000000000000, flags=SA_RESTART|SA_SIGINFO, blocked - SIGHUP: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGINT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTERM: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGQUIT: UserHandler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, blocked - SIGTRAP: crash_handler in libjvm.so, mask=11100100010111111101111111111110, flags=SA_RESTART|SA_SIGINFO, unblocked - - ---------------- S Y S T E M --------------- - -OS: -DISTRIB_ID=Ubuntu -DISTRIB_RELEASE=23.04 -DISTRIB_CODENAME=lunar -DISTRIB_DESCRIPTION="Ubuntu 23.04" -uname: Linux 6.2.0-26-generic #26-Ubuntu SMP PREEMPT_DYNAMIC Mon Jul 10 23:39:54 UTC 2023 x86_64 -OS uptime: 0 days 1:10 hours -libc: glibc 2.37 NPTL 2.37 -rlimit (soft/hard): STACK 8192k/infinity , CORE 0k/infinity , NPROC 63121/63121 , NOFILE 1048576/1048576 , AS infinity/infinity , CPU infinity/infinity , DATA infinity/infinity , FSIZE infinity/infinity , MEMLOCK 2030188k/2030188k -load average: 4.85 4.29 4.82 - -/proc/meminfo: -MemTotal: 16241528 kB -MemFree: 6777444 kB -MemAvailable: 9738900 kB -Buffers: 107464 kB -Cached: 3577316 kB -SwapCached: 0 kB -Active: 1069368 kB -Inactive: 7386124 kB -Active(anon): 4076 kB -Inactive(anon): 5296692 kB -Active(file): 1065292 kB -Inactive(file): 2089432 kB -Unevictable: 476784 kB -Mlocked: 64 kB -SwapTotal: 4194300 kB -SwapFree: 4194300 kB -Zswap: 0 kB -Zswapped: 0 kB -Dirty: 188 kB -Writeback: 0 kB -AnonPages: 5247632 kB -Mapped: 894504 kB -Shmem: 547584 kB -KReclaimable: 125384 kB -Slab: 285052 kB -SReclaimable: 125384 kB -SUnreclaim: 159668 kB -KernelStack: 21840 kB -PageTables: 51860 kB -SecPageTables: 0 kB -NFS_Unstable: 0 kB -Bounce: 0 kB -WritebackTmp: 0 kB -CommitLimit: 12315064 kB -Committed_AS: 16092828 kB -VmallocTotal: 34359738367 kB -VmallocUsed: 118800 kB -VmallocChunk: 0 kB -Percpu: 6048 kB -HardwareCorrupted: 0 kB -AnonHugePages: 2048 kB -ShmemHugePages: 0 kB -ShmemPmdMapped: 0 kB -FileHugePages: 0 kB -FilePmdMapped: 0 kB -HugePages_Total: 0 -HugePages_Free: 0 -HugePages_Rsvd: 0 -HugePages_Surp: 0 -Hugepagesize: 2048 kB -Hugetlb: 0 kB -DirectMap4k: 379380 kB -DirectMap2M: 7890944 kB -DirectMap1G: 8388608 kB - -/sys/kernel/mm/transparent_hugepage/enabled: always [madvise] never -/sys/kernel/mm/transparent_hugepage/defrag (defrag/compaction efforts parameter): always defer defer+madvise [madvise] never - -Process Memory: -Virtual Size: 8613960K (peak: 8613960K) -Resident Set Size: 172172K (peak: 172172K) (anon: 142092K, file: 30080K, shmem: 0K) -Swapped out: 0K -C-Heap outstanding allocations: 61705K, retained: 12622K -glibc malloc tunables: (default) - -/proc/sys/kernel/threads-max (system-wide limit on the number of threads): 126243 -/proc/sys/vm/max_map_count (maximum number of memory map areas a process may have): 65530 -/proc/sys/kernel/pid_max (system-wide limit on number of process identifiers): 4194304 - -container (cgroup) information: -container_type: cgroupv2 -cpu_cpuset_cpus: not supported -cpu_memory_nodes: not supported -active_processor_count: 8 -cpu_quota: not supported -cpu_period: not supported -cpu_shares: not supported -memory_limit_in_bytes: unlimited -memory_and_swap_limit_in_bytes: unlimited -memory_soft_limit_in_bytes: unlimited -memory_usage_in_bytes: 4508120 k -memory_max_usage_in_bytes: not supported -memory_swap_current_in_bytes: unlimited -memory_swap_max_limit_in_bytes: unlimited -maximum number of tasks: 18936 -current number of tasks: 370 - -Steal ticks since vm start: 0 -Steal ticks percentage since vm start: 0.000 - -CPU: total 8 (initial active 8) (4 cores per cpu, 2 threads per core) family 6 model 142 stepping 10 microcode 0xf0, cx8, cmov, fxsr, ht, mmx, 3dnowpref, sse, sse2, sse3, ssse3, sse4.1, sse4.2, popcnt, lzcnt, tsc, tscinvbit, avx, avx2, aes, erms, clmul, bmi1, bmi2, adx, fma, vzeroupper, clflush, clflushopt -CPU Model and flags from /proc/cpuinfo: -model name : Intel(R) Core(TM) i7-8550U CPU @ 1.80GHz -flags : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov pat pse36 clflush dts acpi mmx fxsr sse sse2 ss ht tm pbe syscall nx pdpe1gb rdtscp lm constant_tsc art arch_perfmon pebs bts rep_good nopl xtopology nonstop_tsc cpuid aperfmperf pni pclmulqdq dtes64 monitor ds_cpl vmx est tm2 ssse3 sdbg fma cx16 xtpr pdcm pcid sse4_1 sse4_2 x2apic movbe popcnt tsc_deadline_timer aes xsave avx f16c rdrand lahf_lm abm 3dnowprefetch cpuid_fault epb invpcid_single pti ssbd ibrs ibpb stibp tpr_shadow vnmi flexpriority ept vpid ept_ad fsgsbase tsc_adjust bmi1 avx2 smep bmi2 erms invpcid mpx rdseed adx smap clflushopt intel_pt xsaveopt xsavec xgetbv1 xsaves dtherm ida arat pln pts hwp hwp_notify hwp_act_window hwp_epp md_clear flush_l1d arch_capabilities - -Online cpus: 0-7 -Offline cpus: -BIOS frequency limitation: -Frequency switch latency (ns): 0 -Available cpu frequencies: -Current governor: powersave -Core performance/turbo boost: - -Memory: 4k page, physical 16241528k(6777444k free), swap 4194300k(4194300k free) -Page Sizes: 4k - -vm_info: OpenJDK 64-Bit Server VM (17.0.7+7-Ubuntu-0ubuntu123.04) for linux-amd64 JRE (17.0.7+7-Ubuntu-0ubuntu123.04), built on May 6 2023 14:43:58 by "buildd" with gcc 12.2.0 - -END. From 664b428622ae4678aa8b84a1fa5f2c34e125a744 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Wed, 23 Aug 2023 20:20:21 +0200 Subject: [PATCH 159/200] It wasn't moved permanetly --- .../native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 4f03a6ad..b798498b 100644 --- a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -49,7 +49,7 @@ class TcpSocketSuite extends UringSuite { .head writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 301 Moved Permanently") + .assertEquals("HTTP/1.1 200 OK") } } From 56e8673e430af1d1966a320d9d77233f41a19936 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Wed, 23 Aug 2023 20:20:41 +0200 Subject: [PATCH 160/200] Add read test with remote disconnection --- .../fs2/io/uring/net/TcpSocketSuite.scala | 39 ++++++++++++++++++- 1 file changed, 38 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 41fdf6b5..43a85a37 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -55,7 +55,7 @@ class TcpSocketSuite extends UringSuite { .head writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 301 Moved Permanently") + .assertEquals("HTTP/1.1 200 OK") } } @@ -310,6 +310,43 @@ class TcpSocketSuite extends UringSuite { test.replicateA(repetitions).void } + test("readN yields chunks of the requested size with remote disconnection") { + val message = Chunk.array("123456789012345678901234567890".getBytes) + val sizes = Vector(1, 2, 3, 4, 3, 2, 1) + val subsetSize: Long = 15 + + val test: IO[Unit] = Stream + .resource(setup) + .flatMap { case (server, clients) => + val junkServer = server.map { socket => + Stream + .chunk(message) + .through(socket.writes) + .take(subsetSize) + }.parJoinUnbounded + + val client = + clients + .take(1) + .flatMap { socket => + Stream + .emits(sizes) + .evalMap(socket.readN(_)) + .map(_.size) + } + .take(subsetSize) + + client.concurrently(junkServer) + } + .compile + .toVector + .assertEquals( + sizes.takeWhile(_ <= subsetSize) + ) + + test.replicateA(100).void + } + test("write - concurrent calls do not cause a WritePendingException") { val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) From 17166afd2f632dad0f5a2dab666ab345725bc69d Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 12:06:55 +0200 Subject: [PATCH 161/200] Correct error -107 (shutdown a disconnected socket) --- .../fs2/io/uring/unsafe/UringSystem.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 346a4fd2..08f2f1e6 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -51,10 +51,10 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false - private val debugPoll = debug && true + private val debug = true + private val debugPoll = debug && false private val debugCancel = debug && true - private val debugInterrupt = debug && true + private val debugInterrupt = debug && false private val debugSubmissionQueue = debug && true private val debugHandleCompletionQueue = debug && true type Api = Uring @@ -260,7 +260,7 @@ object UringSystem extends PollingSystem { offset: Long, data: Short ): Boolean = { - if (debugSubmissionQueue) + if (debugSubmissionQueue && data > 9) println( s"[SQ ${ring.fd()}] Enqueuing a new Sqe with: OP: $op, flags: $flags, rwFlags: $rwFlags, fd: $fd, bufferAddress: $bufferAddress, length: $length, offset: $offset, extraData: $data" ) @@ -370,7 +370,11 @@ object UringSystem extends PollingSystem { private[this] val completionQueueCallback = new UringCompletionQueueCallback { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if (res < 0 && op != 14) // Temporarly, ignore error due to race condition on cancellation + if ( + res < 0 && + (op != 14 && res == -2) && // Temporarly, ignore error due to race condition on cancellation + (op == 34 && res != -107) // Ignore error when shutdown a disconnected socket + ) cb( Left( new IOException( @@ -379,9 +383,12 @@ object UringSystem extends PollingSystem { ) ) ) - else cb(Right(res)) + else { + if (op == 34 && res == -107) cb(Right(0)) + else cb(Right(res)) + } - if (debugHandleCompletionQueue) + if (debugHandleCompletionQueue && data > 9) println( s"[HANDLE CQCB ${ring.fd()}]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) From bb1b3dbdbd9b3d002f2b565fbac3732991a90fe1 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 20:47:05 +0200 Subject: [PATCH 162/200] Add mask --- .../src/main/scala/fs2/io/uring/Uring.scala | 6 ++-- .../fs2/io/uring/unsafe/UringSystem.scala | 32 +++++++++---------- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index 6f28c22b..2de15d3d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -30,7 +30,8 @@ abstract class Uring private[uring] { fd: Int = 0, bufferAddress: Long = 0, length: Int = 0, - offset: Long = 0 + offset: Long = 0, + mask: Int => Boolean = (_ => false) ): IO[Int] def bracket( @@ -40,7 +41,8 @@ abstract class Uring private[uring] { fd: Int = 0, bufferAddress: Long = 0, length: Int = 0, - offset: Long = 0 + offset: Long = 0, + mask: Int => Boolean = (_ => false) )(release: Int => IO[Unit]): Resource[IO, Int] } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 08f2f1e6..e14a050c 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -53,9 +53,9 @@ object UringSystem extends PollingSystem { private val debug = true private val debugPoll = debug && false - private val debugCancel = debug && true + private val debugCancel = debug && false private val debugInterrupt = debug && false - private val debugSubmissionQueue = debug && true + private val debugSubmissionQueue = debug && false private val debugHandleCompletionQueue = debug && true type Api = Uring @@ -98,9 +98,10 @@ object UringSystem extends PollingSystem { fd: Int, bufferAddress: Long, length: Int, - offset: Long + offset: Long, + mask: Int => Boolean ): IO[Int] = - exec(op, flags, rwFlags, fd, bufferAddress, length, offset)(noopRelease) + exec(op, flags, rwFlags, fd, bufferAddress, length, offset, mask)(noopRelease) def bracket( op: Byte, @@ -109,10 +110,11 @@ object UringSystem extends PollingSystem { fd: Int, bufferAddress: Long, length: Int, - offset: Long + offset: Long, + mask: Int => Boolean )(release: Int => IO[Unit]): Resource[IO, Int] = Resource.makeFull[IO, Int](poll => - poll(exec(op, flags, rwFlags, fd, bufferAddress, length, offset)(release(_))) + poll(exec(op, flags, rwFlags, fd, bufferAddress, length, offset, mask)(release(_))) )(release) private def exec( @@ -122,7 +124,8 @@ object UringSystem extends PollingSystem { fd: Int, bufferAddress: Long, length: Int, - offset: Long + offset: Long, + mask: Int => Boolean )(release: Int => IO[Unit]): IO[Int] = { def cancel( @@ -179,13 +182,13 @@ object UringSystem extends PollingSystem { F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => - if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) + if (rtn < 0 && !mask(rtn)) F.raiseError(IOExceptionHelper(-rtn)) else lift(release(rtn)) } ) ) } - .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) + .flatTap(e => F.raiseWhen(e < 0 && !mask(e))(IOExceptionHelper(-e))) } } @@ -372,8 +375,7 @@ object UringSystem extends PollingSystem { def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = if ( res < 0 && - (op != 14 && res == -2) && // Temporarly, ignore error due to race condition on cancellation - (op == 34 && res != -107) // Ignore error when shutdown a disconnected socket + (op != 14 && res == -2) // Temporarly, ignore error due to race condition on cancellation ) cb( Left( @@ -383,12 +385,10 @@ object UringSystem extends PollingSystem { ) ) ) - else { - if (op == 34 && res == -107) cb(Right(0)) - else cb(Right(res)) - } + else + cb(Right(res)) - if (debugHandleCompletionQueue && data > 9) + if (debugHandleCompletionQueue && data > 9 && res == -107) println( s"[HANDLE CQCB ${ring.fd()}]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) From 94d6a238b884b44674bb3ceea75e48d4a27411d2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 20:47:15 +0200 Subject: [PATCH 163/200] Add Error --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index a0bba1e9..f460da51 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -83,4 +83,8 @@ private[uring] object util { val IORING_OP_LAST: Byte = 49 } + object errno { + val ENOTCONN = -107 + } + } From 26ea3d5fa8fdb6628f6cbf337f9e1c5327a496f2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 20:47:28 +0200 Subject: [PATCH 164/200] Mask error ENOTCONN when Shutdown --- .../jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 9c5cc00a..9cfa29a8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -36,6 +36,7 @@ import fs2.io.uring.unsafe.util.OP._ import io.netty.buffer.ByteBuf import io.netty.incubator.channel.uring.UringLinuxSocket +import fs2.io.uring.unsafe.util.errno._ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, @@ -104,9 +105,11 @@ private[net] final class UringSocket[F[_]: LiftIO]( def reads: Stream[F, Byte] = Stream.repeatEval(read(defaultReadSize)).unNoneTerminate.unchunks - def endOfInput: F[Unit] = ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 0).void.to + def endOfInput: F[Unit] = + ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 0, mask = e => e == ENOTCONN).void.to - def endOfOutput: F[Unit] = ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 1).void.to + def endOfOutput: F[Unit] = + ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 1, mask = e => e == ENOTCONN).void.to def isOpen: F[Boolean] = F.pure(true) From 83c39f08037481613e8ac3c1d001bb896450ea51 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 20:53:22 +0200 Subject: [PATCH 165/200] Remove unused parameter --- uring/jvm/src/main/scala/fs2/io/uring/Uring.scala | 4 ++-- uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index 2de15d3d..973a65dd 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -31,7 +31,7 @@ abstract class Uring private[uring] { bufferAddress: Long = 0, length: Int = 0, offset: Long = 0, - mask: Int => Boolean = (_ => false) + mask: Int => Boolean = _ => false ): IO[Int] def bracket( @@ -42,7 +42,7 @@ abstract class Uring private[uring] { bufferAddress: Long = 0, length: Int = 0, offset: Long = 0, - mask: Int => Boolean = (_ => false) + mask: Int => Boolean = _ => false )(release: Int => IO[Unit]): Resource[IO, Int] } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 9cfa29a8..2fc52c67 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -106,10 +106,10 @@ private[net] final class UringSocket[F[_]: LiftIO]( def reads: Stream[F, Byte] = Stream.repeatEval(read(defaultReadSize)).unNoneTerminate.unchunks def endOfInput: F[Unit] = - ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 0, mask = e => e == ENOTCONN).void.to + ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 0, mask = _ == ENOTCONN).void.to def endOfOutput: F[Unit] = - ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 1, mask = e => e == ENOTCONN).void.to + ring.call(op = IORING_OP_SHUTDOWN, fd = sockfd, length = 1, mask = _ == ENOTCONN).void.to def isOpen: F[Boolean] = F.pure(true) From 1dd1e522b9efaad7ad0addd015d05a102bd80ba2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 20:58:19 +0200 Subject: [PATCH 166/200] Add mask in Native version --- .../native/src/main/scala/fs2/io/uring/Uring.scala | 4 ++-- .../scala/fs2/io/uring/unsafe/UringSystem.scala | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/Uring.scala b/uring/native/src/main/scala/fs2/io/uring/Uring.scala index 9d6249c7..78e6541c 100644 --- a/uring/native/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/native/src/main/scala/fs2/io/uring/Uring.scala @@ -26,9 +26,9 @@ import scala.scalanative.unsafe.Ptr abstract class Uring private[uring] { - def call(prep: Ptr[io_uring_sqe] => Unit): IO[Int] + def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false): IO[Int] - def bracket(prep: Ptr[io_uring_sqe] => Unit)(release: Int => IO[Unit]): Resource[IO, Int] + def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false)(release: Int => IO[Unit]): Resource[IO, Int] } object Uring { diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 042dd5bc..e100b46a 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -75,13 +75,13 @@ object UringSystem extends PollingSystem { with FileDescriptorPoller { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit - def call(prep: Ptr[io_uring_sqe] => Unit): IO[Int] = - exec(prep)(noopRelease) + def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean): IO[Int] = + exec(prep, mask)(noopRelease) - def bracket(prep: Ptr[io_uring_sqe] => Unit)(release: Int => IO[Unit]): Resource[IO, Int] = - Resource.makeFull[IO, Int](poll => poll(exec(prep)(release(_))))(release(_)) + def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)(release: Int => IO[Unit]): Resource[IO, Int] = + Resource.makeFull[IO, Int](poll => poll(exec(prep, mask)(release(_))))(release(_)) - private def exec(prep: Ptr[io_uring_sqe] => Unit)(release: Int => IO[Unit]): IO[Int] = + private def exec(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)(release: Int => IO[Unit]): IO[Int] = IO.cont { new Cont[IO, Int, Int] { def apply[F[_]](implicit @@ -104,13 +104,13 @@ object UringSystem extends PollingSystem { F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => - if (rtn < 0) F.raiseError(IOExceptionHelper(-rtn)) + if (rtn < 0 && !mask(rtn)) F.raiseError(IOExceptionHelper(-rtn)) else lift(release(rtn)) } ) ) } - .flatTap(e => F.raiseWhen(e < 0)(IOExceptionHelper(-e))) + .flatTap(e => F.raiseWhen(e < 0 && !mask(e))(IOExceptionHelper(-e))) } } } From e76b9b23b4462239a5e57421505975aa1772205f Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 21:02:24 +0200 Subject: [PATCH 167/200] add mask to shutdown when socket has been disconnected --- .../native/src/main/scala/fs2/io/uring/net/UringSocket.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala index 25fb4978..e44ab248 100644 --- a/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -37,6 +37,7 @@ import scala.scalanative.libc.errno._ import scala.scalanative.posix.sys.socket._ import scala.scalanative.unsafe._ import scala.scalanative.unsigned._ +import scala.scalanative.posix.errno._ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, @@ -70,9 +71,9 @@ private[net] final class UringSocket[F[_]: LiftIO]( def reads: Stream[F, Byte] = Stream.repeatEval(read(defaultReadSize)).unNoneTerminate.unchunks - def endOfInput: F[Unit] = ring.call(io_uring_prep_shutdown(_, fd, 0)).void.to + def endOfInput: F[Unit] = ring.call(io_uring_prep_shutdown(_, fd, 0), _ == ENOTCONN).void.to - def endOfOutput: F[Unit] = ring.call(io_uring_prep_shutdown(_, fd, 1)).void.to + def endOfOutput: F[Unit] = ring.call(io_uring_prep_shutdown(_, fd, 1), _ == ENOTCONN).void.to def isOpen: F[Boolean] = F.pure(true) From 5e095ac11b970ae5356ef7baa9ced0b2194ec53a Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 25 Aug 2023 21:02:53 +0200 Subject: [PATCH 168/200] formatting --- uring/native/src/main/scala/fs2/io/uring/Uring.scala | 4 +++- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 8 ++++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/Uring.scala b/uring/native/src/main/scala/fs2/io/uring/Uring.scala index 78e6541c..1ed81ba6 100644 --- a/uring/native/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/native/src/main/scala/fs2/io/uring/Uring.scala @@ -28,7 +28,9 @@ abstract class Uring private[uring] { def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false): IO[Int] - def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false)(release: Int => IO[Unit]): Resource[IO, Int] + def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false)( + release: Int => IO[Unit] + ): Resource[IO, Int] } object Uring { diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index e100b46a..3d43c1cb 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -78,10 +78,14 @@ object UringSystem extends PollingSystem { def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean): IO[Int] = exec(prep, mask)(noopRelease) - def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)(release: Int => IO[Unit]): Resource[IO, Int] = + def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)( + release: Int => IO[Unit] + ): Resource[IO, Int] = Resource.makeFull[IO, Int](poll => poll(exec(prep, mask)(release(_))))(release(_)) - private def exec(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)(release: Int => IO[Unit]): IO[Int] = + private def exec(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean)( + release: Int => IO[Unit] + ): IO[Int] = IO.cont { new Cont[IO, Int, Int] { def apply[F[_]](implicit From 85cc34f9243fd5738bad2defed0e3f4b6abfd0d4 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 25 Aug 2023 23:40:20 +0200 Subject: [PATCH 169/200] Test ignore ENOTCONN when shutdown using masking feature --- .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 16 ++++++++++++++++ .../scala/fs2/io/uring/net/TcpSocketSuite.scala | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 43a85a37..641c11c8 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -413,4 +413,20 @@ class TcpSocketSuite extends UringSuite { clients.compile.drain.timeoutTo(100.millis, IO.unit) } } + + test("endOfOutput / endOfInput ignores ENOTCONN") { + sg.serverResource().use { case (bindAddress, clients) => + sg.client(bindAddress).surround(IO.sleep(100.millis)).background.surround { + clients + .take(1) + .foreach { socket => + socket.write(Chunk.array("fs2.rocks".getBytes)) *> + IO.sleep(1.second) *> + socket.endOfOutput *> socket.endOfInput + } + .compile + .drain + } + } + } } diff --git a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index b798498b..dcbd19f9 100644 --- a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -230,4 +230,20 @@ class TcpSocketSuite extends UringSuite { } } + test("endOfOutput / endOfInput ignores ENOTCONN") { + sg.serverResource().use { case (bindAddress, clients) => + sg.client(bindAddress).surround(IO.sleep(100.millis)).background.surround { + clients + .take(1) + .foreach { socket => + socket.write(Chunk.array("fs2.rocks".getBytes)) *> + IO.sleep(1.second) *> + socket.endOfOutput *> socket.endOfInput + } + .compile + .drain + } + } + } + } From 58f47834a49ac528776c88bae54ca15ef91cbfa6 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 26 Aug 2023 00:15:14 +0200 Subject: [PATCH 170/200] Negate rtn value --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 3d43c1cb..63a3a32d 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -108,13 +108,13 @@ object UringSystem extends PollingSystem { F.unit, // if cannot cancel, fallback to get get.flatMap { rtn => - if (rtn < 0 && !mask(rtn)) F.raiseError(IOExceptionHelper(-rtn)) + if (rtn < 0 && !mask(-rtn)) F.raiseError(IOExceptionHelper(-rtn)) else lift(release(rtn)) } ) ) } - .flatTap(e => F.raiseWhen(e < 0 && !mask(e))(IOExceptionHelper(-e))) + .flatTap(e => F.raiseWhen(e < 0 && !mask(-e))(IOExceptionHelper(-e))) } } } From f391a26c0824a299159ec4b1434d0f3b4b66bfd1 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 26 Aug 2023 00:18:49 +0200 Subject: [PATCH 171/200] Cache default mask --- uring/jvm/src/main/scala/fs2/io/uring/Uring.scala | 5 +++-- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 7 ++----- uring/native/src/main/scala/fs2/io/uring/Uring.scala | 5 +++-- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala index 973a65dd..ba28678f 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/Uring.scala @@ -22,6 +22,7 @@ import cats.effect.kernel.Resource import cats.syntax.all._ abstract class Uring private[uring] { + private[this] val noopMask: Int => Boolean = _ => false def call( op: Byte, @@ -31,7 +32,7 @@ abstract class Uring private[uring] { bufferAddress: Long = 0, length: Int = 0, offset: Long = 0, - mask: Int => Boolean = _ => false + mask: Int => Boolean = noopMask ): IO[Int] def bracket( @@ -42,7 +43,7 @@ abstract class Uring private[uring] { bufferAddress: Long = 0, length: Int = 0, offset: Long = 0, - mask: Int => Boolean = _ => false + mask: Int => Boolean = noopMask )(release: Int => IO[Unit]): Resource[IO, Int] } diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index e14a050c..9e7d4089 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -51,7 +51,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = true + private val debug = false private val debugPoll = debug && false private val debugCancel = debug && false private val debugInterrupt = debug && false @@ -190,13 +190,10 @@ object UringSystem extends PollingSystem { } .flatTap(e => F.raiseWhen(e < 0 && !mask(e))(IOExceptionHelper(-e))) } - } } } - } - } final class Poller private[UringSystem] (ring: UringRing) { @@ -388,7 +385,7 @@ object UringSystem extends PollingSystem { else cb(Right(res)) - if (debugHandleCompletionQueue && data > 9 && res == -107) + if (debugHandleCompletionQueue && data > 9) println( s"[HANDLE CQCB ${ring.fd()}]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) diff --git a/uring/native/src/main/scala/fs2/io/uring/Uring.scala b/uring/native/src/main/scala/fs2/io/uring/Uring.scala index 1ed81ba6..44fe4221 100644 --- a/uring/native/src/main/scala/fs2/io/uring/Uring.scala +++ b/uring/native/src/main/scala/fs2/io/uring/Uring.scala @@ -25,10 +25,11 @@ import fs2.io.uring.unsafe.uring._ import scala.scalanative.unsafe.Ptr abstract class Uring private[uring] { + private[this] val noopMask: Int => Boolean = _ => false - def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false): IO[Int] + def call(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = noopMask): IO[Int] - def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = _ => false)( + def bracket(prep: Ptr[io_uring_sqe] => Unit, mask: Int => Boolean = noopMask)( release: Int => IO[Unit] ): Resource[IO, Int] } From a59e5d05030de4caec32372930b632dd596c7c67 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 26 Aug 2023 14:32:53 +0200 Subject: [PATCH 172/200] Remove unnecessary interrupt and handleCallback --- .../fs2/io/uring/unsafe/UringSystem.scala | 52 ++++++------------- 1 file changed, 16 insertions(+), 36 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 9e7d4089..6af85616 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -39,7 +39,6 @@ import fs2.io.uring.unsafe.util.OP._ import scala.collection.mutable.Map -import java.io.IOException import java.nio.ByteBuffer import io.netty.channel.unix.FileDescriptor @@ -51,7 +50,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = false + private val debug = true private val debugPoll = debug && false private val debugCancel = debug && false private val debugInterrupt = debug && false @@ -131,14 +130,14 @@ object UringSystem extends PollingSystem { def cancel( id: Short, correctRing: Poller - ): IO[Boolean] = // We need access to the correct ring + ): IO[Boolean] = IO.uncancelable { _ => IO.async_[Int] { cb => register { ring => - val opAddressToCancel = Encoder.encode(fd, op, id) + val operationAddress = Encoder.encode(fd, op, id) if (debugCancel) println( - s"[CANCEL ring:${ring.getFd()}] cancel an operation: $op with id: $id and address: $opAddressToCancel" + s"[CANCEL ring:${ring.getFd()}] cancel an operation: $op with id: $id and address: $operationAddress" ) if (correctRing == ring) { val cancelId = ring.getId(cb) @@ -146,13 +145,13 @@ object UringSystem extends PollingSystem { println( s"[CANCEL ring:${ring.getFd()}] Cancelling from the same ring!" ) - ring.cancel(opAddressToCancel, cancelId) + ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, cancelId) } else { if (debugCancel) println( s"[CANCEL ring:${ring.getFd()}] Cancelling from another ring: cancelled operation is in: ${correctRing.getFd()}" ) - correctRing.cancelFromDifferentRing(opAddressToCancel, cb) + correctRing.enqueueCancelOperation(operationAddress, cb) } () @@ -268,22 +267,14 @@ object UringSystem extends PollingSystem { sq.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, data) } - private[UringSystem] def cancel(opAddressToCancel: Long, id: Short): Boolean = { - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) - writeFd() >= 0 - } - - private[UringSystem] def cancelFromDifferentRing( - opAddressToCancel: Long, + private[UringSystem] def enqueueCancelOperation( + operationAddress: Long, cb: Either[Throwable, Int] => Unit ): Boolean = { - cancelOperations.add((opAddressToCancel, cb)) + cancelOperations.add((operationAddress, cb)) writeFd() >= 0 } - private[UringSystem] def wakeup() = - extraRing.sendMsgRing(0, this.getFd()) - private[UringSystem] def writeFd(): Int = { val buf = ByteBuffer.allocateDirect(1) buf.put(0.toByte) @@ -291,6 +282,9 @@ object UringSystem extends PollingSystem { writeEnd.write(buf, 0, 1) } + private[UringSystem] def wakeup() = + extraRing.sendMsgRing(0, this.getFd()) + private[UringSystem] def poll( nanos: Long ): Boolean = { @@ -323,9 +317,9 @@ object UringSystem extends PollingSystem { println( s"[POLL ${Thread.currentThread().getName()}] The Cancel Queue is not empty, it has: ${cancelOperations.size()} elements" ) - cancelOperations.forEach { case (opAddressToCancel, cb) => + cancelOperations.forEach { case (operationAddress, cb) => val id = getId(cb) - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, opAddressToCancel, 0, 0, id) + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, id) () } cancelOperations.clear() @@ -369,23 +363,9 @@ object UringSystem extends PollingSystem { private[this] val completionQueueCallback = new UringCompletionQueueCallback { override def handle(fd: Int, res: Int, flags: Int, op: Byte, data: Short): Unit = { - def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = - if ( - res < 0 && - (op != 14 && res == -2) // Temporarly, ignore error due to race condition on cancellation - ) - cb( - Left( - new IOException( - s"Error in completion queue entry of the ring with fd: ${ring - .fd()} with fd: $fd op: $op res: $res and data: $data" - ) - ) - ) - else - cb(Right(res)) + def handleCallback(res: Int, cb: Either[Throwable, Int] => Unit): Unit = cb(Right(res)) - if (debugHandleCompletionQueue && data > 9) + if (debugHandleCompletionQueue && data > 9 && res < 0) println( s"[HANDLE CQCB ${ring.fd()}]: fd: $fd, res: $res, flags: $flags, op: $op, data: $data" ) From 892b94ee8d4faee93a96dd1f05bbc3ef8a25be92 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 26 Aug 2023 14:52:47 +0200 Subject: [PATCH 173/200] Move listening and cancelOperation to a specific function --- .../fs2/io/uring/unsafe/UringSystem.scala | 68 ++++++++++--------- 1 file changed, 37 insertions(+), 31 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 6af85616..e35d439b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -50,7 +50,7 @@ object UringSystem extends PollingSystem { private final val MaxEvents = 64 - private val debug = true + private val debug = false private val debugPoll = debug && false private val debugCancel = debug && false private val debugInterrupt = debug && false @@ -293,37 +293,10 @@ object UringSystem extends PollingSystem { println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") // Check if it is listening to the FD. If not, start listening - if (!listenFd) { - if (debugPoll) - println(s"[POLL ${Thread.currentThread().getName()}] We are not listening to the FD!") - - enqueueSqe( - IORING_OP_POLL_ADD, - 0, - NativeAccess.POLLIN, - readEnd.intValue(), - 0, - 0, - 0, - NativeAccess.POLLIN.toShort - ) - pendingSubmissions = true - listenFd = true // Set the flag indicating it is now listening - } + startListening() // Check for cancel operations - if (!cancelOperations.isEmpty()) { - if (debugPoll) - println( - s"[POLL ${Thread.currentThread().getName()}] The Cancel Queue is not empty, it has: ${cancelOperations.size()} elements" - ) - cancelOperations.forEach { case (operationAddress, cb) => - val id = getId(cb) - enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, id) - () - } - cancelOperations.clear() - } + checkCancelOperations() nanos match { case -1 => @@ -356,6 +329,39 @@ object UringSystem extends PollingSystem { invokedCbs } + private[this] def startListening(): Unit = + if (!listenFd) { + if (debugPoll) + println(s"[POLL ${Thread.currentThread().getName()}] We are not listening to the FD!") + + enqueueSqe( + IORING_OP_POLL_ADD, + 0, + NativeAccess.POLLIN, + readEnd.intValue(), + 0, + 0, + 0, + NativeAccess.POLLIN.toShort + ) + pendingSubmissions = true + listenFd = true // Set the flag indicating it is now listening + } + + private[this] def checkCancelOperations(): Unit = + if (!cancelOperations.isEmpty()) { + if (debugPoll) + println( + s"[POLL ${Thread.currentThread().getName()}] The Cancel Queue is not empty, it has: ${cancelOperations.size()} elements" + ) + cancelOperations.forEach { case (operationAddress, cb) => + val id = getId(cb) + enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, id) + () + } + cancelOperations.clear() + } + private[this] def process( completionQueueCallback: UringCompletionQueueCallback ): Boolean = @@ -374,7 +380,7 @@ object UringSystem extends PollingSystem { Instead of using a callback for interrupt handling, we manage the interrupt directly within this block. Checks for an interrupt by determining if the FileDescriptor (fd) has been written to. */ - if (fd == readEnd.intValue()) { + if (fd == readEnd.intValue() && op == IORING_OP_POLL_ADD) { val buf = ByteBuffer.allocateDirect(1) readEnd.read(buf, 0, 1) listenFd = false From 7ed808a42d1edbd97ccae00b6ae07ba7fc142de6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Mon, 28 Aug 2023 13:56:33 +0200 Subject: [PATCH 174/200] Make Poller extends AbstractSelector --- .../fs2/io/uring/unsafe/UringSystem.scala | 54 +++++++++++++------ 1 file changed, 37 insertions(+), 17 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index e35d439b..01540631 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -45,6 +45,11 @@ import io.netty.channel.unix.FileDescriptor import java.util.BitSet import java.util.concurrent.ConcurrentLinkedDeque +import java.nio.channels.spi.AbstractSelector +import java.{util => ju} +import java.nio.channels.Selector +import java.nio.channels.SelectionKey +import java.nio.channels.spi.AbstractSelectableChannel object UringSystem extends PollingSystem { @@ -79,11 +84,7 @@ object UringSystem extends PollingSystem { println( s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: ${targetPoller.getFd()} in thread: $targetThread" ) - // Interrupt using an extra ring - // targetPoller.wakeup() - - // Interrupt using a pipe - targetPoller.writeFd() + targetPoller.wakeup() () } @@ -195,15 +196,13 @@ object UringSystem extends PollingSystem { } } - final class Poller private[UringSystem] (ring: UringRing) { + final class Poller private[UringSystem] (ring: UringRing) extends AbstractSelector(null) { private[this] val interruptFd = FileDescriptor.pipe() private[this] val readEnd = interruptFd(0) private[this] val writeEnd = interruptFd(1) private[this] var listenFd: Boolean = false - private[this] val extraRing: UringRing = UringRing() - private[this] val cancelOperations : ConcurrentLinkedDeque[(Long, Either[Throwable, Int] => Unit)] = new ConcurrentLinkedDeque() @@ -243,12 +242,6 @@ object UringSystem extends PollingSystem { private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty - private[UringSystem] def close(): Unit = { - readEnd.close() - writeEnd.close() - ring.close() - } - private[UringSystem] def enqueueSqe( op: Byte, flags: Int, @@ -275,19 +268,44 @@ object UringSystem extends PollingSystem { writeFd() >= 0 } - private[UringSystem] def writeFd(): Int = { + private[this] def writeFd(): Int = { val buf = ByteBuffer.allocateDirect(1) buf.put(0.toByte) buf.flip() writeEnd.write(buf, 0, 1) } - private[UringSystem] def wakeup() = - extraRing.sendMsgRing(0, this.getFd()) + override def keys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException + + override def selectedKeys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException + + override def selectNow(): Int = throw new UnsupportedOperationException + + override def select(x$1: Long): Int = throw new UnsupportedOperationException + + override def select(): Int = throw new UnsupportedOperationException + + override def wakeup(): Selector = { + writeFd() + this + } + + override protected def implCloseSelector(): Unit = { + readEnd.close() + writeEnd.close() + ring.close() + } + + override protected def register( + x$1: AbstractSelectableChannel, + x$2: Int, + x$3: Object + ): SelectionKey = throw new UnsupportedOperationException private[UringSystem] def poll( nanos: Long ): Boolean = { + begin() if (debugPoll) println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") @@ -326,6 +344,8 @@ object UringSystem extends PollingSystem { pendingSubmissions = false + end() + invokedCbs } From 450602041331792d449e6639fae0c1423eb1cd14 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Mon, 28 Aug 2023 14:04:57 +0200 Subject: [PATCH 175/200] organize code and put end in finally block --- .../fs2/io/uring/unsafe/UringSystem.scala | 165 +++++++++--------- 1 file changed, 86 insertions(+), 79 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 01540631..a3e79019 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -215,11 +215,7 @@ object UringSystem extends PollingSystem { Map.empty[Short, Either[Throwable, Int] => Unit] private[this] val ids = new BitSet(Short.MaxValue) - private[this] def getUniqueId(): Short = { - val newId = ids.nextClearBit(10) // 0-9 are reserved for certain operations - ids.set(newId) - newId.toShort - } + // API private[UringSystem] def getId( cb: Either[Throwable, Int] => Unit @@ -230,14 +226,6 @@ object UringSystem extends PollingSystem { id } - private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - - private[this] def removeCallback(id: Short): Boolean = - callbacks - .remove(id) - .map(_ => releaseId(id)) - .isDefined - private[UringSystem] def getFd(): Int = ring.fd() private[UringSystem] def needsPoll(): Boolean = pendingSubmissions || !callbacks.isEmpty @@ -268,87 +256,77 @@ object UringSystem extends PollingSystem { writeFd() >= 0 } - private[this] def writeFd(): Int = { - val buf = ByteBuffer.allocateDirect(1) - buf.put(0.toByte) - buf.flip() - writeEnd.write(buf, 0, 1) - } - - override def keys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException - - override def selectedKeys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException - - override def selectNow(): Int = throw new UnsupportedOperationException - - override def select(x$1: Long): Int = throw new UnsupportedOperationException - - override def select(): Int = throw new UnsupportedOperationException - - override def wakeup(): Selector = { - writeFd() - this - } + private[UringSystem] def poll( + nanos: Long + ): Boolean = + try { + begin() - override protected def implCloseSelector(): Unit = { - readEnd.close() - writeEnd.close() - ring.close() - } + if (debugPoll) + println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") - override protected def register( - x$1: AbstractSelectableChannel, - x$2: Int, - x$3: Object - ): SelectionKey = throw new UnsupportedOperationException + startListening() // Check if it is listening to the FD. If not, start listening - private[UringSystem] def poll( - nanos: Long - ): Boolean = { - begin() + checkCancelOperations() // Check for cancel operations - if (debugPoll) - println(s"[POLL ${Thread.currentThread().getName()}] Polling with nanos = $nanos") + nanos match { + case -1 => + if (pendingSubmissions) { + sq.submitAndWait() + } else { + cq.ioUringWaitCqe() + } - // Check if it is listening to the FD. If not, start listening - startListening() + case 0 => + if (pendingSubmissions) { + sq.submit() + } - // Check for cancel operations - checkCancelOperations() + case _ => + if (pendingSubmissions) { + sq.addTimeout(nanos, 0) + sq.submitAndWait() + } else { + sq.addTimeout(nanos, 0) + sq.submit() + cq.ioUringWaitCqe() + } + } - nanos match { - case -1 => - if (pendingSubmissions) { - sq.submitAndWait() - } else { - cq.ioUringWaitCqe() - } + val invokedCbs = process(completionQueueCallback) - case 0 => - if (pendingSubmissions) { - sq.submit() - } + pendingSubmissions = false + invokedCbs + } finally + end() - case _ => - if (pendingSubmissions) { - sq.addTimeout(nanos, 0) - sq.submitAndWait() - } else { - sq.addTimeout(nanos, 0) - sq.submit() - cq.ioUringWaitCqe() - } - } + // private - val invokedCbs = process(completionQueueCallback) + // CALLBACKS + private[this] def getUniqueId(): Short = { + val newId = ids.nextClearBit(10) // 0-9 are reserved for certain operations + ids.set(newId) + newId.toShort + } - pendingSubmissions = false + private[this] def releaseId(id: Short): Unit = ids.clear(id.toInt) - end() + private[this] def removeCallback(id: Short): Boolean = + callbacks + .remove(id) + .map(_ => releaseId(id)) + .isDefined - invokedCbs + // INTERRUPT + private[this] def writeFd(): Int = { + val buf = ByteBuffer.allocateDirect(1) + buf.put(0.toByte) + buf.flip() + writeEnd.write(buf, 0, 1) } + // POLL + private[this] def startListening(): Unit = if (!listenFd) { if (debugPoll) @@ -413,5 +391,34 @@ object UringSystem extends PollingSystem { } } } + + // ABSTRACT SELECTOR + override def keys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException + + override def selectedKeys(): ju.Set[SelectionKey] = throw new UnsupportedOperationException + + override def selectNow(): Int = throw new UnsupportedOperationException + + override def select(x$1: Long): Int = throw new UnsupportedOperationException + + override def select(): Int = throw new UnsupportedOperationException + + override def wakeup(): Selector = { + writeFd() + this + } + + override protected def implCloseSelector(): Unit = { + readEnd.close() + writeEnd.close() + ring.close() + } + + override protected def register( + x$1: AbstractSelectableChannel, + x$2: Int, + x$3: Object + ): SelectionKey = throw new UnsupportedOperationException + } } From af9488d0b298afebb90e4b79158a13ed7fd5fa54 Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Mon, 28 Aug 2023 13:04:06 +0000 Subject: [PATCH 176/200] Fixups --- .../jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 5 +++-- uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala | 2 -- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index a3e79019..6ad1e3c8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -53,8 +53,6 @@ import java.nio.channels.spi.AbstractSelectableChannel object UringSystem extends PollingSystem { - private final val MaxEvents = 64 - private val debug = false private val debugPoll = debug && false private val debugCancel = debug && false @@ -68,6 +66,8 @@ object UringSystem extends PollingSystem { override def makePoller(): Poller = new Poller(UringRing()) + override def close(): Unit = () + override def closePoller(poller: Poller): Unit = poller.close() override def poll( @@ -280,6 +280,7 @@ object UringSystem extends PollingSystem { case 0 => if (pendingSubmissions) { sq.submit() + () } case _ => diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala index d5b5c5ec..900146fa 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/UringSystemSuite.scala @@ -19,8 +19,6 @@ package fs2.io.uring import cats.effect.IO import cats.syntax.parallel._ -import fs2.io.uring.UringSuite - import fs2.io.uring.unsafe.util.OP._ class UringSystemSuite extends UringSuite { From 4e332b91fd36c3ecf7d56bd79845865030ca94a8 Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Mon, 28 Aug 2023 13:07:23 +0000 Subject: [PATCH 177/200] Fixup --- uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala index 4297d6fd..84b7c8d2 100644 --- a/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/native/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -38,7 +38,6 @@ import scala.scalanative.posix.sys.socket._ import scala.scalanative.posix.errno._ import scala.scalanative.unsafe._ import scala.scalanative.unsigned._ -import scala.scalanative.posix.errno._ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, From b655bb4f783bc340bfbefa78d8b34409656092fb Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Mon, 28 Aug 2023 13:59:47 +0000 Subject: [PATCH 178/200] Fix netty deps --- build.sbt | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index edf16f1f..ebda58d3 100644 --- a/build.sbt +++ b/build.sbt @@ -34,8 +34,14 @@ ThisBuild / githubWorkflowPublishPreamble += val ceVersion = "3.6-e9aeb8c" val fs2Version = "3.8.0" +val nettyVersion = "0.0.21.Final" val munitCEVersion = "2.0.0-M3" +lazy val classifier = System.getProperty("os.arch") match { + case "amd64" => "linux-x86_64" + case "aarch64" => "linux-aarch_64" +} + ThisBuild / nativeConfig ~= { c => if (Option(System.getenv("CI")).contains("true")) c.withLinkingOptions(c.linkingOptions :+ "-luring") @@ -53,9 +59,10 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) libraryDependencies ++= Seq( "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, + "io.netty.incubator" % "netty-incubator-transport-classes-io_uring" % nettyVersion, "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, - ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % "0.0.21.Final") - .classifier("linux-x86_64") + ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % nettyVersion % Test) + .classifier(classifier) ), Test / testOptions += Tests.Argument("+l") ) From dfbef6b9e0671120ba36f974ff2d981ea8a34d24 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Tue, 29 Aug 2023 00:02:59 +0200 Subject: [PATCH 179/200] Add timeout test --- .../fs2/io/uring/net/TcpSocketSuite.scala | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 641c11c8..5bd3dcdb 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -30,6 +30,7 @@ import cats.effect.kernel.Resource import scala.concurrent.duration._ import java.io.IOException import fs2.io.net.Socket +import java.util.concurrent.TimeoutException class TcpSocketSuite extends UringSuite { val debug = false @@ -230,9 +231,6 @@ class TcpSocketSuite extends UringSuite { val repetitions: Int = 1 - /* - TODO: (Very rare) second cancellation with error -2 (we shouldn't have a second cancellation (?)) - */ test("echo requests - each concurrent client gets back what it sent") { val message = Chunk.array("fs2.rocks".getBytes) val clientCount = 20L @@ -273,9 +271,6 @@ class TcpSocketSuite extends UringSuite { test.replicateA(repetitions).void } - /* - TODO: (Very rare) error -107 - */ test("readN yields chunks of the requested size") { val message = Chunk.array("123456789012345678901234567890".getBytes) val sizes = Vector(1, 2, 3, 4, 3, 2, 1) @@ -429,4 +424,20 @@ class TcpSocketSuite extends UringSuite { } } } + + test("can shutdown a socket that's pending a read") { + val timeout = 2.seconds + val test = sg.serverResource().use { case (bindAddress, clients) => + sg.client(bindAddress).use { _ => + clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { + case _: TimeoutException => () + } + } + } + + // also test that timeouts are working correctly + test.timed.flatMap { case (duration, _) => + IO(assert(clue(duration) < (timeout + 100.millis))) + } + } } From 51d90f842c337a02d9781af60aa3155a634a4a42 Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Mon, 28 Aug 2023 22:49:09 +0000 Subject: [PATCH 180/200] Fixups --- build.sbt | 9 ++++- .../fs2/io/uring/unsafe/UringSystem.scala | 40 +++++++++---------- 2 files changed, 26 insertions(+), 23 deletions(-) diff --git a/build.sbt b/build.sbt index ebda58d3..3554f1b8 100644 --- a/build.sbt +++ b/build.sbt @@ -59,10 +59,15 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) libraryDependencies ++= Seq( "org.typelevel" %%% "cats-effect" % ceVersion, "co.fs2" %%% "fs2-io" % fs2Version, + "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test + ), + Test / testOptions += Tests.Argument("+l") + ) + .jvmSettings( + libraryDependencies ++= Seq( "io.netty.incubator" % "netty-incubator-transport-classes-io_uring" % nettyVersion, - "org.typelevel" %%% "munit-cats-effect" % munitCEVersion % Test, ("io.netty.incubator" % "netty-incubator-transport-native-io_uring" % nettyVersion % Test) .classifier(classifier) ), - Test / testOptions += Tests.Argument("+l") + fork := true ) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 6ad1e3c8..c104a418 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -132,31 +132,29 @@ object UringSystem extends PollingSystem { id: Short, correctRing: Poller ): IO[Boolean] = - IO.uncancelable { _ => - IO.async_[Int] { cb => - register { ring => - val operationAddress = Encoder.encode(fd, op, id) + IO.async_[Int] { cb => + register { ring => + val operationAddress = Encoder.encode(fd, op, id) + if (debugCancel) + println( + s"[CANCEL ring:${ring.getFd()}] cancel an operation: $op with id: $id and address: $operationAddress" + ) + if (correctRing == ring) { + val cancelId = ring.getId(cb) if (debugCancel) println( - s"[CANCEL ring:${ring.getFd()}] cancel an operation: $op with id: $id and address: $operationAddress" + s"[CANCEL ring:${ring.getFd()}] Cancelling from the same ring!" ) - if (correctRing == ring) { - val cancelId = ring.getId(cb) - if (debugCancel) - println( - s"[CANCEL ring:${ring.getFd()}] Cancelling from the same ring!" - ) - ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, cancelId) - } else { - if (debugCancel) - println( - s"[CANCEL ring:${ring.getFd()}] Cancelling from another ring: cancelled operation is in: ${correctRing.getFd()}" - ) - correctRing.enqueueCancelOperation(operationAddress, cb) - } - - () + ring.enqueueSqe(IORING_OP_ASYNC_CANCEL, 0, 0, -1, operationAddress, 0, 0, cancelId) + } else { + if (debugCancel) + println( + s"[CANCEL ring:${ring.getFd()}] Cancelling from another ring: cancelled operation is in: ${correctRing.getFd()}" + ) + correctRing.enqueueCancelOperation(operationAddress, cb) } + + () } }.map(_ == 0) From d9b481b5a1fd4c6874e933d3fba0ecc698d89adb Mon Sep 17 00:00:00 2001 From: antjim1 Date: Fri, 1 Sep 2023 12:56:34 +0200 Subject: [PATCH 181/200] Update netty version --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 3554f1b8..f1f83ff7 100644 --- a/build.sbt +++ b/build.sbt @@ -34,7 +34,7 @@ ThisBuild / githubWorkflowPublishPreamble += val ceVersion = "3.6-e9aeb8c" val fs2Version = "3.8.0" -val nettyVersion = "0.0.21.Final" +val nettyVersion = "0.0.22.Final" val munitCEVersion = "2.0.0-M3" lazy val classifier = System.getProperty("os.arch") match { From fba387505deb6d39f7a455fa11244194c4febb9e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 2 Sep 2023 00:14:27 +0200 Subject: [PATCH 182/200] Add warmup for timeout test --- .../fs2/io/uring/net/TcpSocketSuite.scala | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 5bd3dcdb..f05f1331 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -426,6 +426,26 @@ class TcpSocketSuite extends UringSuite { } test("can shutdown a socket that's pending a read") { + val warmup = sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => + val msg = + """|GET /get HTTP/1.1 + |Host: postman-echo.com + | + |""".stripMargin + + val writeRead = + Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .head + + writeRead.compile.lastOrError + .assertEquals("HTTP/1.1 200 OK") + } + val timeout = 2.seconds val test = sg.serverResource().use { case (bindAddress, clients) => sg.client(bindAddress).use { _ => @@ -436,7 +456,7 @@ class TcpSocketSuite extends UringSuite { } // also test that timeouts are working correctly - test.timed.flatMap { case (duration, _) => + warmup *> test.timed.flatMap { case (duration, _) => IO(assert(clue(duration) < (timeout + 100.millis))) } } From cea2b125a0b710c342b346c28897808083607568 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 2 Sep 2023 00:15:14 +0200 Subject: [PATCH 183/200] Use send_msg for interruption and sketch getCurrentPollerIfAvailable --- .../scala/fs2/io/uring/unsafe/UringRing.scala | 6 +-- .../fs2/io/uring/unsafe/UringSystem.scala | 38 +++++++++++++------ 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 130a1dd3..8056ecb5 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -74,11 +74,11 @@ final class UringRing(private[this] val ringBuffer: RingBuffer) { */ def fd(): Int = ringBuffer.fd() - def sendMsgRing(flags: Int, fd: Int): Boolean = { // TODO bypass the submission queue + def sendMsgRing(flags: Int, fd: Int): Unit = { // println(s"[SENDMESSAGE] current thread: ${Thread.currentThread().getName()}]") uringSubmissionQueue.enqueueSqe(40, flags, 0, fd, 0, 0, 0, 0) - uringSubmissionQueue.submit() - uringCompletionQueue.hasCompletions() + uringSubmissionQueue.submitAndWait() + () } /** Closes the Ring, realising any associated resources. diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index c104a418..b01ccf9b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -79,12 +79,17 @@ object UringSystem extends PollingSystem { override def needsPoll(poller: Poller): Boolean = poller.needsPoll() + def getCurrentPollerIfAvailable(): Option[Poller] = None // TODO + override def interrupt(targetThread: Thread, targetPoller: Poller): Unit = { if (debugInterrupt) println( s"[INTERRUPT ${Thread.currentThread().getName()}] waking up poller: ${targetPoller.getFd()} in thread: $targetThread" ) - targetPoller.wakeup() + getCurrentPollerIfAvailable() match { + case Some(poller) => poller.sendMsg(targetPoller.getFd()) + case None => targetPoller.wakeup() + } () } @@ -201,6 +206,8 @@ object UringSystem extends PollingSystem { private[this] val writeEnd = interruptFd(1) private[this] var listenFd: Boolean = false + private[this] val interruptRing: UringRing = UringRing() + private[this] val cancelOperations : ConcurrentLinkedDeque[(Long, Either[Throwable, Int] => Unit)] = new ConcurrentLinkedDeque() @@ -249,9 +256,15 @@ object UringSystem extends PollingSystem { private[UringSystem] def enqueueCancelOperation( operationAddress: Long, cb: Either[Throwable, Int] => Unit - ): Boolean = { + ): Unit = { cancelOperations.add((operationAddress, cb)) - writeFd() >= 0 + wakeup() + () + } + + private[UringSystem] def sendMsg(fd: Int): Unit = { + enqueueSqe(op = IORING_OP_MSG_RING, 0, 0, fd, 0, 0, 0, 0) + () } private[UringSystem] def poll( @@ -317,15 +330,14 @@ object UringSystem extends PollingSystem { .isDefined // INTERRUPT - private[this] def writeFd(): Int = { - val buf = ByteBuffer.allocateDirect(1) - buf.put(0.toByte) - buf.flip() - writeEnd.write(buf, 0, 1) - } + // private[this] def writeFd(): Int = { + // val buf = ByteBuffer.allocateDirect(1) + // buf.put(0.toByte) + // buf.flip() + // writeEnd.write(buf, 0, 1) + // } // POLL - private[this] def startListening(): Unit = if (!listenFd) { if (debugPoll) @@ -403,7 +415,11 @@ object UringSystem extends PollingSystem { override def select(): Int = throw new UnsupportedOperationException override def wakeup(): Selector = { - writeFd() + val interruptCq = interruptRing.ioUringCompletionQueue() + if (interruptCq.hasCompletions()) { + interruptCq.process(completionQueueCallback) + } + interruptRing.sendMsgRing(0, this.getFd()) this } From c8678d7ad5fdedac26e3252a61a0a810cee0f3cb Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sat, 2 Sep 2023 00:35:08 +0200 Subject: [PATCH 184/200] fix discarded non-Unit error --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index b01ccf9b..89b80a28 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -418,6 +418,7 @@ object UringSystem extends PollingSystem { val interruptCq = interruptRing.ioUringCompletionQueue() if (interruptCq.hasCompletions()) { interruptCq.process(completionQueueCallback) + () } interruptRing.sendMsgRing(0, this.getFd()) this From 5f9922c17587926b2d56060e3db101e00415089e Mon Sep 17 00:00:00 2001 From: antjim1 Date: Thu, 21 Sep 2023 01:20:35 +0200 Subject: [PATCH 185/200] Remove datagramsocket --- .../uring/net/UringDatagramSocketGroup.scala | 42 ------------------- .../scala/fs2/io/uring/net/UringNetwork.scala | 10 ++--- 2 files changed, 5 insertions(+), 47 deletions(-) delete mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala deleted file mode 100644 index 9ea0f32d..00000000 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringDatagramSocketGroup.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2.io.uring.net - -import cats.effect.kernel.Resource - -import com.comcast.ip4s._ - -import fs2.io.net.DatagramSocketGroup -import fs2.io.net.DatagramSocket -import fs2.io.net._ - -import java.net.ProtocolFamily - -private final class UringDatagramSocketGroup[F[_]] extends DatagramSocketGroup[F] { - - override def openDatagramSocket( - address: Option[Host], - port: Option[Port], - options: List[DatagramSocketOption], - protocolFamily: Option[ProtocolFamily] - ): Resource[F, DatagramSocket[F]] = ??? - -} - -object UringDatagramSocketGroup { - def apply[F[_]]: DatagramSocketGroup[F] = new UringDatagramSocketGroup -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala index 1740a337..9ae27c64 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringNetwork.scala @@ -39,18 +39,19 @@ import fs2.io.net.DatagramSocketOption import java.net.ProtocolFamily import java.util.concurrent.ThreadFactory +import cats.MonadError private[net] final class UringNetwork[F[_]]( sg: UringSocketGroup[F], - dsg: UringDatagramSocketGroup[F], val tlsContext: TLSContext.Builder[F] -) extends Network.UnsealedNetwork[F] { +)(implicit F: MonadError[F, Throwable]) + extends Network.UnsealedNetwork[F] { def socketGroup(threadCount: Int, threadFactory: ThreadFactory): Resource[F, SocketGroup[F]] = Resource.pure[F, SocketGroup[F]](sg) def datagramSocketGroup(threadFactory: ThreadFactory): Resource[F, DatagramSocketGroup[F]] = - Resource.pure[F, DatagramSocketGroup[F]](dsg) + Resource.eval(F.raiseError(new NotImplementedError("This operation is not yet implemented"))) def client(to: SocketAddress[Host], options: List[SocketOption]): Resource[F, Socket[F]] = sg.client(to, options) @@ -74,14 +75,13 @@ private[net] final class UringNetwork[F[_]]( options: List[DatagramSocketOption], protocolFamily: Option[ProtocolFamily] ): Resource[F, DatagramSocket[F]] = - dsg.openDatagramSocket(address, port, options, protocolFamily) + Resource.eval(F.raiseError(new NotImplementedError("This operation is not yet implemented"))) } object UringNetwork { def apply[F[_]: Async: Dns: LiftIO]: Network[F] = new UringNetwork( new UringSocketGroup[F], - new UringDatagramSocketGroup[F], TLSContext.Builder.forAsync[F] ) } From 7afdf024dd03e233890231e4810c01a053cfc8e2 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Thu, 21 Sep 2023 21:17:04 +0000 Subject: [PATCH 186/200] Share App, implicits and ExceptionHelper --- .../fs2/io/uring/IOExceptionHelper.scala | 76 ------------------- .../main/scala/fs2/io/uring/UringApp.scala | 30 -------- .../main/scala/fs2/io/uring/implicits.scala | 29 ------- .../fs2/io/uring/IOExceptionHelper.scala | 0 .../main/scala/fs2/io/uring/UringApp.scala | 0 .../main/scala/fs2/io/uring/implicits.scala | 0 6 files changed, 135 deletions(-) delete mode 100644 uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala delete mode 100644 uring/native/src/main/scala/fs2/io/uring/UringApp.scala delete mode 100644 uring/native/src/main/scala/fs2/io/uring/implicits.scala rename uring/{jvm => shared}/src/main/scala/fs2/io/uring/IOExceptionHelper.scala (100%) rename uring/{jvm => shared}/src/main/scala/fs2/io/uring/UringApp.scala (100%) rename uring/{jvm => shared}/src/main/scala/fs2/io/uring/implicits.scala (100%) diff --git a/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala b/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala deleted file mode 100644 index d6777cc4..00000000 --- a/uring/native/src/main/scala/fs2/io/uring/IOExceptionHelper.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2.io.uring - -import java.io.IOException -import java.net.ConnectException -import java.net.BindException -import java.net.SocketException -import java.net.SocketTimeoutException -import java.net.NoRouteToHostException - -private[uring] object IOExceptionHelper { - - def apply(errno: Int): IOException = errno match { - case 9 => // EBADF - new IOException("Bad file descriptor") - - case 11 => // EAGAIN - new IOException("Resource temporarily unavailable") - - case 13 => // EACCES - new IOException("Permission denied") - - case 14 => // EFAULT - new IOException("Bad address") - - case 22 => // EINVAL - new IOException("Invalid argument") - - case 24 => // EMFILE - new IOException("Too many open files") - - case 28 => // ENOSPC - new IOException("No space left on device") - - case 32 => // EPIPE - new IOException("Broken pipe") - - case 98 => // EADDRINUSE - new BindException("Address already in use") - - case 99 => // EADDRNOTAVAIL - new BindException("Cannot assign requested address") - - case 107 => // ECONNABORTED - new SocketException("Connection aborted") - - case 110 => // ETIMEDOUT - new SocketTimeoutException("Connection timed out") - - case 111 => // ECONNREFUSED - new ConnectException("Connection refused") - - case 113 => // EHOSTUNREACH - new NoRouteToHostException("No route to host") - - case 104 => // ECONNRESET - new SocketException("Connection reset by peer") - - case _ => new IOException(errno.toString) - } -} diff --git a/uring/native/src/main/scala/fs2/io/uring/UringApp.scala b/uring/native/src/main/scala/fs2/io/uring/UringApp.scala deleted file mode 100644 index ddf7903b..00000000 --- a/uring/native/src/main/scala/fs2/io/uring/UringApp.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2.io.uring - -import cats.effect.IOApp -import fs2.io.uring.unsafe.UringSystem - -trait UringApp extends IOApp { - - override protected final def pollingSystem = UringSystem - -} - -object UringApp { - trait Simple extends IOApp.Simple with UringApp -} diff --git a/uring/native/src/main/scala/fs2/io/uring/implicits.scala b/uring/native/src/main/scala/fs2/io/uring/implicits.scala deleted file mode 100644 index 90473190..00000000 --- a/uring/native/src/main/scala/fs2/io/uring/implicits.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2.io.uring - -import cats.effect.LiftIO -import cats.effect.kernel.Async -import com.comcast.ip4s.Dns -import fs2.io.net.Network -import fs2.io.uring.net.UringNetwork - -object implicits { - - @inline implicit def network[F[_]: Async: Dns: LiftIO]: Network[F] = UringNetwork[F] - -} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala b/uring/shared/src/main/scala/fs2/io/uring/IOExceptionHelper.scala similarity index 100% rename from uring/jvm/src/main/scala/fs2/io/uring/IOExceptionHelper.scala rename to uring/shared/src/main/scala/fs2/io/uring/IOExceptionHelper.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala b/uring/shared/src/main/scala/fs2/io/uring/UringApp.scala similarity index 100% rename from uring/jvm/src/main/scala/fs2/io/uring/UringApp.scala rename to uring/shared/src/main/scala/fs2/io/uring/UringApp.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/implicits.scala b/uring/shared/src/main/scala/fs2/io/uring/implicits.scala similarity index 100% rename from uring/jvm/src/main/scala/fs2/io/uring/implicits.scala rename to uring/shared/src/main/scala/fs2/io/uring/implicits.scala From b4cb55726691010ffaf5c23ce0225288d46a804b Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 22 Sep 2023 22:31:02 +0000 Subject: [PATCH 187/200] Share UringSuite and TcpSocketSuite --- build.sbt | 4 +- .../test/scala/fs2/io/uring/UringSuite.scala | 39 -- .../fs2/io/uring/net/TcpSocketSuite.scala | 498 +++++++++--------- .../test/scala/fs2/io/uring/UringSuite.scala | 8 +- .../fs2/io/uring/net/TcpSocketSuite.scala | 52 +- 5 files changed, 275 insertions(+), 326 deletions(-) delete mode 100644 uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala rename uring/{native => shared}/src/test/scala/fs2/io/uring/UringSuite.scala (86%) rename uring/{jvm => shared}/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala (92%) diff --git a/build.sbt b/build.sbt index f1f83ff7..71564986 100644 --- a/build.sbt +++ b/build.sbt @@ -53,7 +53,7 @@ ThisBuild / nativeConfig ~= { c => lazy val root = tlCrossRootProject.aggregate(uring) lazy val uring = crossProject(NativePlatform, JVMPlatform) - .in(file("uring")) + .in(file("uring/shared")) .settings( name := "fs2-io_uring", libraryDependencies ++= Seq( @@ -71,3 +71,5 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) ), fork := true ) + .jvmConfigure(_.in(file("uring/jvm"))) + .nativeConfigure(_.in(file("uring/native"))) \ No newline at end of file diff --git a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala deleted file mode 100644 index 44453322..00000000 --- a/uring/jvm/src/test/scala/fs2/io/uring/UringSuite.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2.io.uring - -import munit.CatsEffectSuite -// import cats.effect.unsafe.IORuntime -import fs2.io.uring.unsafe.UringSystem -import cats.effect.unsafe.IORuntimeBuilder -import scala.concurrent.duration._ - -abstract class UringSuite extends CatsEffectSuite { - - // override lazy val munitIORuntime = { - // val (pool, poller, shutdown) = - // IORuntime.createWorkStealingComputeThreadPool(threads = 2, pollingSystem = UringSystem) - // IORuntime.builder().setCompute(pool, shutdown).addPoller(poller, () => ()).build() - // } - - override lazy val munitIORuntime = - IORuntimeBuilder() - .setPollingSystem(UringSystem) - .build() - - override def munitIOTimeout: Duration = 20.second -} diff --git a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index c8223fde..7d9244d3 100644 --- a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -1,249 +1,249 @@ -/* - * Copyright 2022 Arman Bilge - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fs2 -package io.uring -package net - -import cats.effect.IO -import cats.syntax.all._ -import com.comcast.ip4s._ -import fs2.text._ - -import java.net.BindException -import java.net.ConnectException -import java.util.concurrent.TimeoutException -import scala.concurrent.duration._ - -class TcpSocketSuite extends UringSuite { - - val sg = UringSocketGroup[IO] - - test("postman echo") { - sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => - val msg = - """|GET /get HTTP/1.1 - |Host: postman-echo.com - | - |""".stripMargin - - val writeRead = Stream(msg) - .through(utf8.encode[IO]) - .through(socket.writes) ++ - socket.reads - .through(utf8.decode[IO]) - .through(lines) - .head - - val http11 = "HTTP/1.1" - writeRead.compile.lastOrError.map(_.take(http11.length)).assertEquals(http11) - } - } - - val setup = for { - serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) - (bindAddress, server) = serverSetup - clients = Stream.resource(sg.client(bindAddress)).repeat - } yield server -> clients - - test("echo requests - each concurrent client gets back what it sent") { - val message = Chunk.array("fs2.rocks".getBytes) - val clientCount = 20L - - Stream - .resource(setup) - .flatMap { case (server, clients) => - val echoServer = server.map { socket => - socket.reads - .through(socket.writes) - .onFinalize(socket.endOfOutput) - }.parJoinUnbounded - - val msgClients = clients - .take(clientCount) - .map { socket => - Stream - .chunk(message) - .through(socket.writes) - .onFinalize(socket.endOfOutput) ++ - socket.reads.chunks - .map(bytes => new String(bytes.toArray)) - } - .parJoin(10) - .take(clientCount) - - msgClients.concurrently(echoServer) - } - .compile - .toVector - .map { it => - assertEquals(it.size.toLong, clientCount) - assert(it.forall(_ == "fs2.rocks")) - } - } - - test("readN yields chunks of the requested size") { - val message = Chunk.array("123456789012345678901234567890".getBytes) - val sizes = Vector(1, 2, 3, 4, 3, 2, 1) - - Stream - .resource(setup) - .flatMap { case (server, clients) => - val junkServer = server.map { socket => - Stream - .chunk(message) - .through(socket.writes) - .onFinalize(socket.endOfOutput) - }.parJoinUnbounded - - val client = - clients - .take(1) - .flatMap { socket => - Stream - .emits(sizes) - .evalMap(socket.readN(_)) - .map(_.size) - } - .take(sizes.length.toLong) - - client.concurrently(junkServer) - } - .compile - .toVector - .assertEquals(sizes) - } - - test("write - concurrent calls do not cause a WritePendingException") { - val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) - - Stream - .resource(setup) - .flatMap { case (server, clients) => - val readOnlyServer = server.map(_.reads).parJoinUnbounded - val client = - clients.take(1).flatMap { socket => - // concurrent writes - Stream { - Stream.eval(socket.write(message)).repeatN(10L) - }.repeatN(2L).parJoinUnbounded - } - - client.concurrently(readOnlyServer) - } - .compile - .drain - } - - test("addresses - should match across client and server sockets") { - Stream - .resource(setup) - .flatMap { case (server, clients) => - val serverSocketAddresses = server.evalMap { socket => - socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) - } - - val clientSocketAddresses = - clients - .take(1) - .evalMap { socket => - socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) - } - - serverSocketAddresses.parZip(clientSocketAddresses).map { - case ((serverLocal, serverRemote), (clientLocal, clientRemote)) => - assertEquals(clientRemote, serverLocal) - assertEquals(clientLocal, serverRemote) - } - - } - .compile - .drain - } - - test("errors - should be captured in the effect") { - (for { - bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).use(s => IO.pure(s._1)) - _ <- sg.client(bindAddress).use(_ => IO.unit).recover { case ex: ConnectException => - assertEquals(ex.getMessage, "Connection refused") - } - } yield ()) >> (for { - bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).map(_._1) - _ <- sg - .serverResource(Some(bindAddress.host), Some(bindAddress.port)) - .void - .recover { case ex: BindException => - assertEquals(ex.getMessage, "Address already in use") - } - } yield ()).use_ >> (for { - _ <- sg.client(SocketAddress.fromString("not.example.com:80").get).use_.recover { - case ex: UnknownHostException => - assert( - ex.getMessage == "not.example.com: Name or service not known" || ex.getMessage == "not.example.com: nodename nor servname provided, or not known" - ) - } - } yield ()) - } - - // TODO options test - - // TODO decide about "read after timed out read not allowed" - - test("can shutdown a socket that's pending a read") { - val timeout = 2.seconds - val test = sg.serverResource().use { case (bindAddress, clients) => - sg.client(bindAddress).use { _ => - clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { - case _: TimeoutException => () - } - } - } - - // also test that timeouts are working correctly - test.timed.flatMap { case (duration, _) => - IO(assert(clue(duration) < (timeout + 100.millis))) - } - } - - test("accept is cancelable") { - sg.serverResource().use { case (_, clients) => - clients.compile.drain.timeoutTo(100.millis, IO.unit) - } - } - - test("empty write") { - setup.use { case (_, clients) => - clients.take(1).foreach(_.write(Chunk.empty)).compile.drain - } - } - - test("endOfOutput / endOfInput ignores ENOTCONN") { - sg.serverResource().use { case (bindAddress, clients) => - sg.client(bindAddress).surround(IO.sleep(100.millis)).background.surround { - clients - .take(1) - .foreach { socket => - socket.write(Chunk.array("fs2.rocks".getBytes)) *> - IO.sleep(1.second) *> - socket.endOfOutput *> socket.endOfInput - } - .compile - .drain - } - } - } - -} +// /* +// * Copyright 2022 Arman Bilge +// * +// * Licensed under the Apache License, Version 2.0 (the "License"); +// * you may not use this file except in compliance with the License. +// * You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ + +// package fs2 +// package io.uring +// package net + +// import cats.effect.IO +// import cats.syntax.all._ +// import com.comcast.ip4s._ +// import fs2.text._ + +// import java.net.BindException +// import java.net.ConnectException +// import java.util.concurrent.TimeoutException +// import scala.concurrent.duration._ + +// class TcpSocketSuite extends UringSuite { + +// val sg = UringSocketGroup[IO] + +// test("postman echo") { +// sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => +// val msg = +// """|GET /get HTTP/1.1 +// |Host: postman-echo.com +// | +// |""".stripMargin + +// val writeRead = Stream(msg) +// .through(utf8.encode[IO]) +// .through(socket.writes) ++ +// socket.reads +// .through(utf8.decode[IO]) +// .through(lines) +// .head + +// val http11 = "HTTP/1.1" +// writeRead.compile.lastOrError.map(_.take(http11.length)).assertEquals(http11) +// } +// } + +// val setup = for { +// serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) +// (bindAddress, server) = serverSetup +// clients = Stream.resource(sg.client(bindAddress)).repeat +// } yield server -> clients + +// test("echo requests - each concurrent client gets back what it sent") { +// val message = Chunk.array("fs2.rocks".getBytes) +// val clientCount = 20L + +// Stream +// .resource(setup) +// .flatMap { case (server, clients) => +// val echoServer = server.map { socket => +// socket.reads +// .through(socket.writes) +// .onFinalize(socket.endOfOutput) +// }.parJoinUnbounded + +// val msgClients = clients +// .take(clientCount) +// .map { socket => +// Stream +// .chunk(message) +// .through(socket.writes) +// .onFinalize(socket.endOfOutput) ++ +// socket.reads.chunks +// .map(bytes => new String(bytes.toArray)) +// } +// .parJoin(10) +// .take(clientCount) + +// msgClients.concurrently(echoServer) +// } +// .compile +// .toVector +// .map { it => +// assertEquals(it.size.toLong, clientCount) +// assert(it.forall(_ == "fs2.rocks")) +// } +// } + +// test("readN yields chunks of the requested size") { +// val message = Chunk.array("123456789012345678901234567890".getBytes) +// val sizes = Vector(1, 2, 3, 4, 3, 2, 1) + +// Stream +// .resource(setup) +// .flatMap { case (server, clients) => +// val junkServer = server.map { socket => +// Stream +// .chunk(message) +// .through(socket.writes) +// .onFinalize(socket.endOfOutput) +// }.parJoinUnbounded + +// val client = +// clients +// .take(1) +// .flatMap { socket => +// Stream +// .emits(sizes) +// .evalMap(socket.readN(_)) +// .map(_.size) +// } +// .take(sizes.length.toLong) + +// client.concurrently(junkServer) +// } +// .compile +// .toVector +// .assertEquals(sizes) +// } + +// test("write - concurrent calls do not cause a WritePendingException") { +// val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) + +// Stream +// .resource(setup) +// .flatMap { case (server, clients) => +// val readOnlyServer = server.map(_.reads).parJoinUnbounded +// val client = +// clients.take(1).flatMap { socket => +// // concurrent writes +// Stream { +// Stream.eval(socket.write(message)).repeatN(10L) +// }.repeatN(2L).parJoinUnbounded +// } + +// client.concurrently(readOnlyServer) +// } +// .compile +// .drain +// } + +// test("addresses - should match across client and server sockets") { +// Stream +// .resource(setup) +// .flatMap { case (server, clients) => +// val serverSocketAddresses = server.evalMap { socket => +// socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) +// } + +// val clientSocketAddresses = +// clients +// .take(1) +// .evalMap { socket => +// socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) +// } + +// serverSocketAddresses.parZip(clientSocketAddresses).map { +// case ((serverLocal, serverRemote), (clientLocal, clientRemote)) => +// assertEquals(clientRemote, serverLocal) +// assertEquals(clientLocal, serverRemote) +// } + +// } +// .compile +// .drain +// } + +// test("errors - should be captured in the effect") { +// (for { +// bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).use(s => IO.pure(s._1)) +// _ <- sg.client(bindAddress).use(_ => IO.unit).recover { case ex: ConnectException => +// assertEquals(ex.getMessage, "Connection refused") +// } +// } yield ()) >> (for { +// bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).map(_._1) +// _ <- sg +// .serverResource(Some(bindAddress.host), Some(bindAddress.port)) +// .void +// .recover { case ex: BindException => +// assertEquals(ex.getMessage, "Address already in use") +// } +// } yield ()).use_ >> (for { +// _ <- sg.client(SocketAddress.fromString("not.example.com:80").get).use_.recover { +// case ex: UnknownHostException => +// assert( +// ex.getMessage == "not.example.com: Name or service not known" || ex.getMessage == "not.example.com: nodename nor servname provided, or not known" +// ) +// } +// } yield ()) +// } + +// // TODO options test + +// // TODO decide about "read after timed out read not allowed" + +// test("can shutdown a socket that's pending a read") { +// val timeout = 2.seconds +// val test = sg.serverResource().use { case (bindAddress, clients) => +// sg.client(bindAddress).use { _ => +// clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { +// case _: TimeoutException => () +// } +// } +// } + +// // also test that timeouts are working correctly +// test.timed.flatMap { case (duration, _) => +// IO(assert(clue(duration) < (timeout + 100.millis))) +// } +// } + +// test("accept is cancelable") { +// sg.serverResource().use { case (_, clients) => +// clients.compile.drain.timeoutTo(100.millis, IO.unit) +// } +// } + +// test("empty write") { +// setup.use { case (_, clients) => +// clients.take(1).foreach(_.write(Chunk.empty)).compile.drain +// } +// } + +// test("endOfOutput / endOfInput ignores ENOTCONN") { +// sg.serverResource().use { case (bindAddress, clients) => +// sg.client(bindAddress).surround(IO.sleep(100.millis)).background.surround { +// clients +// .take(1) +// .foreach { socket => +// socket.write(Chunk.array("fs2.rocks".getBytes)) *> +// IO.sleep(1.second) *> +// socket.endOfOutput *> socket.endOfInput +// } +// .compile +// .drain +// } +// } +// } + +// } diff --git a/uring/native/src/test/scala/fs2/io/uring/UringSuite.scala b/uring/shared/src/test/scala/fs2/io/uring/UringSuite.scala similarity index 86% rename from uring/native/src/test/scala/fs2/io/uring/UringSuite.scala rename to uring/shared/src/test/scala/fs2/io/uring/UringSuite.scala index d9d9cf38..eaf8a128 100644 --- a/uring/native/src/test/scala/fs2/io/uring/UringSuite.scala +++ b/uring/shared/src/test/scala/fs2/io/uring/UringSuite.scala @@ -16,13 +16,15 @@ package fs2.io.uring -import cats.effect.unsafe.IORuntime -import fs2.io.uring.unsafe.UringSystem import munit.CatsEffectSuite +import fs2.io.uring.unsafe.UringSystem +import cats.effect.unsafe.IORuntimeBuilder abstract class UringSuite extends CatsEffectSuite { override lazy val munitIORuntime = - IORuntime.builder().setPollingSystem(UringSystem).build() + IORuntimeBuilder() + .setPollingSystem(UringSystem) + .build() } diff --git a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala similarity index 92% rename from uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala rename to uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index f05f1331..43b23511 100644 --- a/uring/jvm/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -32,31 +32,28 @@ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException -class TcpSocketSuite extends UringSuite { +class TcpSocketSuitd extends UringSuite { val debug = false val sg = UringSocketGroup[IO] - // Client test: - test("postman echo") { sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => val msg = """|GET /get HTTP/1.1 - |Host: postman-echo.com - | - |""".stripMargin - - val writeRead = - Stream(msg) - .through(utf8.encode[IO]) - .through(socket.writes) ++ - socket.reads - .through(utf8.decode[IO]) - .through(lines) - .head - - writeRead.compile.lastOrError - .assertEquals("HTTP/1.1 200 OK") + |Host: postman-echo.com + | + |""".stripMargin + + val writeRead = Stream(msg) + .through(utf8.encode[IO]) + .through(socket.writes) ++ + socket.reads + .through(utf8.decode[IO]) + .through(lines) + .head + + val http11 = "HTTP/1.1" + writeRead.compile.lastOrError.map(_.take(http11.length)).assertEquals(http11) } } @@ -127,8 +124,6 @@ class TcpSocketSuite extends UringSuite { } } - // Server tests: - val serverResource: Resource[IO, (SocketAddress[IpAddress], Stream[IO, Socket[IO]])] = sg.serverResource( Some(Host.fromString("localhost").get), @@ -136,16 +131,6 @@ class TcpSocketSuite extends UringSuite { Nil ) - test("Start server and wait for a connection during 5 sec") { - serverResource.use { case (localAddress, _) => - IO.whenA(debug)(IO { - println(s"[TEST] Server started at $localAddress") - println(s"[TEST] You can now connect to this server") - }) *> IO.sleep(5.second) - // Use telnet localhost "port" to connect - } - } - test("Start server and connect external client") { serverResource.use { case (localAddress, _) => for { @@ -162,7 +147,7 @@ class TcpSocketSuite extends UringSuite { } } - test("Create server connect external client and writes") { + test("Create server connect external client and write") { serverResource.use { case (localAddress, serverStream) => sg.client(localAddress).use { socket => val msg = "Hello, echo server!\n" @@ -221,7 +206,6 @@ class TcpSocketSuite extends UringSuite { } } - // Server and client tests: val setup = for { serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) (bindAddress, server) = serverSetup @@ -244,7 +228,7 @@ class TcpSocketSuite extends UringSuite { .through(socket.writes) .onFinalize(socket.endOfOutput) } - .parJoin(1) + .parJoinUnbounded val msgClients = Stream.sleep_[IO](1.second) ++ clients .take(clientCount) @@ -339,7 +323,7 @@ class TcpSocketSuite extends UringSuite { sizes.takeWhile(_ <= subsetSize) ) - test.replicateA(100).void + test.replicateA(repetitions).void } test("write - concurrent calls do not cause a WritePendingException") { From 53f32de83dba61377470574b868699b09c56a1fc Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 22 Sep 2023 22:45:39 +0000 Subject: [PATCH 188/200] formatting --- .../test/scala/fs2/io/uring/net/TcpSocketSuite.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 43b23511..41a2af41 100644 --- a/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -222,13 +222,11 @@ class TcpSocketSuitd extends UringSuite { val test: IO[Unit] = Stream .resource(setup) .flatMap { case (server, clients) => - val echoServer = server - .map { socket => - socket.reads - .through(socket.writes) - .onFinalize(socket.endOfOutput) - } - .parJoinUnbounded + val echoServer = server.map { socket => + socket.reads + .through(socket.writes) + .onFinalize(socket.endOfOutput) + }.parJoinUnbounded val msgClients = Stream.sleep_[IO](1.second) ++ clients .take(clientCount) From 6342f80d95d5268d3bc6380bda910ff056a7eec6 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 22 Sep 2023 23:16:49 +0000 Subject: [PATCH 189/200] undo shared in build --- build.sbt | 6 +- .../fs2/io/uring/net/TcpSocketSuite.scala | 249 ------------------ 2 files changed, 2 insertions(+), 253 deletions(-) delete mode 100644 uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala diff --git a/build.sbt b/build.sbt index 71564986..d4bfef7d 100644 --- a/build.sbt +++ b/build.sbt @@ -53,7 +53,7 @@ ThisBuild / nativeConfig ~= { c => lazy val root = tlCrossRootProject.aggregate(uring) lazy val uring = crossProject(NativePlatform, JVMPlatform) - .in(file("uring/shared")) + .in(file("uring")) .settings( name := "fs2-io_uring", libraryDependencies ++= Seq( @@ -70,6 +70,4 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) .classifier(classifier) ), fork := true - ) - .jvmConfigure(_.in(file("uring/jvm"))) - .nativeConfigure(_.in(file("uring/native"))) \ No newline at end of file + ) \ No newline at end of file diff --git a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala deleted file mode 100644 index 7d9244d3..00000000 --- a/uring/native/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ /dev/null @@ -1,249 +0,0 @@ -// /* -// * Copyright 2022 Arman Bilge -// * -// * Licensed under the Apache License, Version 2.0 (the "License"); -// * you may not use this file except in compliance with the License. -// * You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ - -// package fs2 -// package io.uring -// package net - -// import cats.effect.IO -// import cats.syntax.all._ -// import com.comcast.ip4s._ -// import fs2.text._ - -// import java.net.BindException -// import java.net.ConnectException -// import java.util.concurrent.TimeoutException -// import scala.concurrent.duration._ - -// class TcpSocketSuite extends UringSuite { - -// val sg = UringSocketGroup[IO] - -// test("postman echo") { -// sg.client(SocketAddress(host"postman-echo.com", port"80")).use { socket => -// val msg = -// """|GET /get HTTP/1.1 -// |Host: postman-echo.com -// | -// |""".stripMargin - -// val writeRead = Stream(msg) -// .through(utf8.encode[IO]) -// .through(socket.writes) ++ -// socket.reads -// .through(utf8.decode[IO]) -// .through(lines) -// .head - -// val http11 = "HTTP/1.1" -// writeRead.compile.lastOrError.map(_.take(http11.length)).assertEquals(http11) -// } -// } - -// val setup = for { -// serverSetup <- sg.serverResource(address = Some(ip"127.0.0.1")) -// (bindAddress, server) = serverSetup -// clients = Stream.resource(sg.client(bindAddress)).repeat -// } yield server -> clients - -// test("echo requests - each concurrent client gets back what it sent") { -// val message = Chunk.array("fs2.rocks".getBytes) -// val clientCount = 20L - -// Stream -// .resource(setup) -// .flatMap { case (server, clients) => -// val echoServer = server.map { socket => -// socket.reads -// .through(socket.writes) -// .onFinalize(socket.endOfOutput) -// }.parJoinUnbounded - -// val msgClients = clients -// .take(clientCount) -// .map { socket => -// Stream -// .chunk(message) -// .through(socket.writes) -// .onFinalize(socket.endOfOutput) ++ -// socket.reads.chunks -// .map(bytes => new String(bytes.toArray)) -// } -// .parJoin(10) -// .take(clientCount) - -// msgClients.concurrently(echoServer) -// } -// .compile -// .toVector -// .map { it => -// assertEquals(it.size.toLong, clientCount) -// assert(it.forall(_ == "fs2.rocks")) -// } -// } - -// test("readN yields chunks of the requested size") { -// val message = Chunk.array("123456789012345678901234567890".getBytes) -// val sizes = Vector(1, 2, 3, 4, 3, 2, 1) - -// Stream -// .resource(setup) -// .flatMap { case (server, clients) => -// val junkServer = server.map { socket => -// Stream -// .chunk(message) -// .through(socket.writes) -// .onFinalize(socket.endOfOutput) -// }.parJoinUnbounded - -// val client = -// clients -// .take(1) -// .flatMap { socket => -// Stream -// .emits(sizes) -// .evalMap(socket.readN(_)) -// .map(_.size) -// } -// .take(sizes.length.toLong) - -// client.concurrently(junkServer) -// } -// .compile -// .toVector -// .assertEquals(sizes) -// } - -// test("write - concurrent calls do not cause a WritePendingException") { -// val message = Chunk.array(("123456789012345678901234567890" * 10000).getBytes) - -// Stream -// .resource(setup) -// .flatMap { case (server, clients) => -// val readOnlyServer = server.map(_.reads).parJoinUnbounded -// val client = -// clients.take(1).flatMap { socket => -// // concurrent writes -// Stream { -// Stream.eval(socket.write(message)).repeatN(10L) -// }.repeatN(2L).parJoinUnbounded -// } - -// client.concurrently(readOnlyServer) -// } -// .compile -// .drain -// } - -// test("addresses - should match across client and server sockets") { -// Stream -// .resource(setup) -// .flatMap { case (server, clients) => -// val serverSocketAddresses = server.evalMap { socket => -// socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) -// } - -// val clientSocketAddresses = -// clients -// .take(1) -// .evalMap { socket => -// socket.endOfOutput *> socket.localAddress.product(socket.remoteAddress) -// } - -// serverSocketAddresses.parZip(clientSocketAddresses).map { -// case ((serverLocal, serverRemote), (clientLocal, clientRemote)) => -// assertEquals(clientRemote, serverLocal) -// assertEquals(clientLocal, serverRemote) -// } - -// } -// .compile -// .drain -// } - -// test("errors - should be captured in the effect") { -// (for { -// bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).use(s => IO.pure(s._1)) -// _ <- sg.client(bindAddress).use(_ => IO.unit).recover { case ex: ConnectException => -// assertEquals(ex.getMessage, "Connection refused") -// } -// } yield ()) >> (for { -// bindAddress <- sg.serverResource(Some(ip"127.0.0.1")).map(_._1) -// _ <- sg -// .serverResource(Some(bindAddress.host), Some(bindAddress.port)) -// .void -// .recover { case ex: BindException => -// assertEquals(ex.getMessage, "Address already in use") -// } -// } yield ()).use_ >> (for { -// _ <- sg.client(SocketAddress.fromString("not.example.com:80").get).use_.recover { -// case ex: UnknownHostException => -// assert( -// ex.getMessage == "not.example.com: Name or service not known" || ex.getMessage == "not.example.com: nodename nor servname provided, or not known" -// ) -// } -// } yield ()) -// } - -// // TODO options test - -// // TODO decide about "read after timed out read not allowed" - -// test("can shutdown a socket that's pending a read") { -// val timeout = 2.seconds -// val test = sg.serverResource().use { case (bindAddress, clients) => -// sg.client(bindAddress).use { _ => -// clients.head.flatMap(_.reads).compile.drain.timeout(2.seconds).recover { -// case _: TimeoutException => () -// } -// } -// } - -// // also test that timeouts are working correctly -// test.timed.flatMap { case (duration, _) => -// IO(assert(clue(duration) < (timeout + 100.millis))) -// } -// } - -// test("accept is cancelable") { -// sg.serverResource().use { case (_, clients) => -// clients.compile.drain.timeoutTo(100.millis, IO.unit) -// } -// } - -// test("empty write") { -// setup.use { case (_, clients) => -// clients.take(1).foreach(_.write(Chunk.empty)).compile.drain -// } -// } - -// test("endOfOutput / endOfInput ignores ENOTCONN") { -// sg.serverResource().use { case (bindAddress, clients) => -// sg.client(bindAddress).surround(IO.sleep(100.millis)).background.surround { -// clients -// .take(1) -// .foreach { socket => -// socket.write(Chunk.array("fs2.rocks".getBytes)) *> -// IO.sleep(1.second) *> -// socket.endOfOutput *> socket.endOfInput -// } -// .compile -// .drain -// } -// } -// } - -// } From 347123154620c7a745bd87ad23f0634310b7500f Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Fri, 22 Sep 2023 23:19:51 +0000 Subject: [PATCH 190/200] formatting --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index d4bfef7d..f1f83ff7 100644 --- a/build.sbt +++ b/build.sbt @@ -70,4 +70,4 @@ lazy val uring = crossProject(NativePlatform, JVMPlatform) .classifier(classifier) ), fork := true - ) \ No newline at end of file + ) From a61f5895921bde5cd2ffd555b7dab65a55a03efc Mon Sep 17 00:00:00 2001 From: Antonio Jimenez <100201872+antoniojimeneznieto@users.noreply.github.com> Date: Sat, 23 Sep 2023 12:18:36 +0200 Subject: [PATCH 191/200] Update uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala Co-authored-by: Arman Bilge --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index f460da51..6f34938d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -84,7 +84,7 @@ private[uring] object util { } object errno { - val ENOTCONN = -107 + final val ENOTCONN = 107 } } From b8ee8d3287f5e21918bb24af006f860a41bdc4d3 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 23 Sep 2023 17:59:33 +0000 Subject: [PATCH 192/200] Open sockets using uring --- .../scala/fs2/io/uring/net/UringSocket.scala | 5 +-- .../fs2/io/uring/net/UringSocketGroup.scala | 45 +++++++++---------- 2 files changed, 22 insertions(+), 28 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index 2fc52c67..c5e20b6d 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -40,7 +40,6 @@ import fs2.io.uring.unsafe.util.errno._ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, - linuxSocket: UringLinuxSocket, sockfd: Int, _remoteAddress: SocketAddress[IpAddress], buffer: ByteBuf, @@ -116,7 +115,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( def remoteAddress: F[SocketAddress[IpAddress]] = F.pure(_remoteAddress) def localAddress: F[SocketAddress[IpAddress]] = - F.delay(SocketAddress.fromInetSocketAddress(linuxSocket.getLocalAddress())) + F.delay(SocketAddress.fromInetSocketAddress(UringLinuxSocket(sockfd).getLocalAddress())) private[this] def send(bufferAddress: Long, maxBytes: Int, flags: Int): F[Int] = ring.call(IORING_OP_SEND, flags, 0, sockfd, bufferAddress, maxBytes, 0).to @@ -158,7 +157,6 @@ private[net] object UringSocket { def apply[F[_]: LiftIO]( ring: Uring, - linuxSocket: UringLinuxSocket, fd: Int, remote: SocketAddress[IpAddress] )(implicit @@ -170,7 +168,6 @@ private[net] object UringSocket { writeMutex <- Resource.eval(Mutex[F]) socket = new UringSocket( ring, - linuxSocket, fd, remote, buffer, diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index be93157a..fbb14118 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -58,7 +58,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn isIpv6 = address.host.isInstanceOf[Ipv6Address] - linuxSocket <- openSocket(ring, isIpv6) + fd <- openSocket(ring, isIpv6) + + // linuxSocket <- Resource.eval(F.delay(UringLinuxSocket(fd))) _ <- Resource.eval( createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect @@ -69,7 +71,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn F.delay( println( s"[CLIENT] Connecting to address: ${address - .toString()}, Buffer length: $length and LinuxSocket fd: ${linuxSocket.fd()}" + .toString()}, Buffer length: $length and LinuxSocket fd: $fd" ) ) ) @@ -78,7 +80,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn _ <- ring .call( op = IORING_OP_CONNECT, - fd = linuxSocket.fd(), + fd = fd, bufferAddress = buf.memoryAddress(), offset = length.toLong ) @@ -87,7 +89,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } ) - socket <- UringSocket(ring, linuxSocket, linuxSocket.fd(), address) + socket <- UringSocket(ring, fd, address) _ <- Resource.eval(F.whenA(debugClient)(F.delay(println("[CLIENT] Connexion established!")))) @@ -119,7 +121,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn isIpv6 = resolvedAddress.isInstanceOf[Ipv6Address] - linuxSocket <- openSocket(ring, isIpv6) + fd <- openSocket(ring, isIpv6) + + linuxSocket <- Resource.eval(F.delay(UringLinuxSocket(fd))) _ <- Resource.eval( F.whenA(debugServer)(F.delay(println(s"[SERVER] LinusSocketFd: ${linuxSocket.fd()}"))) @@ -158,7 +162,7 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn clientFd <- ring .bracket( op = IORING_OP_ACCEPT, - fd = linuxSocket.fd(), + fd = fd, bufferAddress = buf.memoryAddress(), offset = bufLength.memoryAddress() )(closeSocket(ring, _)) @@ -192,7 +196,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn ) socket <- UringSocket( ring, - UringLinuxSocket(clientFd), clientFd, remoteAddress ) @@ -207,25 +210,19 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn } yield (localAddress, sockets.unNone) } - private[this] def openSocket( - ring: Uring, - ipv6: Boolean - ): Resource[F, UringLinuxSocket] = - Resource.make[F, UringLinuxSocket](F.delay(UringLinuxSocket.newSocketStream(ipv6)))( - linuxSocket => closeSocket(ring, linuxSocket.fd()).to - ) - - // private[this] def uringOpenSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { - // val domain = if (ipv6) AF_INET6 else AF_INET - // ring - // .bracket(op = IORING_OP_SOCKET, fd = domain, length = 0, offset = SOCK_STREAM)( - // closeSocket(ring, _) - // ) - // .mapK(LiftIO.liftK) - // } + private[this] def openSocket(ring: Uring, ipv6: Boolean): Resource[F, Int] = { + val domain = if (ipv6) AF_INET6 else AF_INET + ring + .bracket(op = IORING_OP_SOCKET, fd = domain.toInt, length = 0, offset = SOCK_STREAM.toLong)( + closeSocket(ring, _) + ) + .mapK(LiftIO.liftK) + } private[this] def closeSocket(ring: Uring, fd: Int): IO[Unit] = - ring.call(op = IORING_OP_CLOSE, fd = fd).void + IO.whenA(debug)(IO.println(s"The fd to close is: $fd")) *> ring + .call(op = IORING_OP_CLOSE, fd = fd) + .void } From 58e628f09c684b5c768b542aa4ae97307359c32d Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 23 Sep 2023 18:00:23 +0000 Subject: [PATCH 193/200] Fix buffer release bug --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala index 6f34938d..9879f255 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/util.scala @@ -18,6 +18,7 @@ package fs2.io.uring.unsafe import cats.effect.kernel.Sync import cats.effect.kernel.Resource + import io.netty.buffer.UnpooledByteBufAllocator import io.netty.buffer.ByteBuf @@ -26,7 +27,7 @@ private[uring] object util { def createBuffer[F[_]: Sync](size: Int): Resource[F, ByteBuf] = Resource.make( Sync[F].delay(UnpooledByteBufAllocator.DEFAULT.directBuffer(size)) - )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release() })) + )(buf => Sync[F].delay(if (buf.refCnt() > 0) { val _ = buf.release(buf.refCnt()) })) /** Defines constants for various operation types supported by the io_uring interface. */ From fc7f1a4afbc66a1a6379d89449af5efa5f8389c4 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 23 Sep 2023 18:00:47 +0000 Subject: [PATCH 194/200] Change masking error to negative --- uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 2 +- .../shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 89b80a28..c99946d0 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -191,7 +191,7 @@ object UringSystem extends PollingSystem { ) ) } - .flatTap(e => F.raiseWhen(e < 0 && !mask(e))(IOExceptionHelper(-e))) + .flatTap(e => F.raiseWhen(e < 0 && !mask(-e))(IOExceptionHelper(-e))) } } } diff --git a/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala index 41a2af41..9f777b04 100644 --- a/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala +++ b/uring/shared/src/test/scala/fs2/io/uring/net/TcpSocketSuite.scala @@ -32,7 +32,7 @@ import java.io.IOException import fs2.io.net.Socket import java.util.concurrent.TimeoutException -class TcpSocketSuitd extends UringSuite { +class TcpSocketSuite extends UringSuite { val debug = false val sg = UringSocketGroup[IO] From 21b2af771845a4303dd729b6145f8ed8ee71d0ef Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sat, 23 Sep 2023 22:32:37 +0000 Subject: [PATCH 195/200] Use resizable buffer for reading and wriing --- .../fs2/io/uring/net/ResizableBuffer.scala | 38 +++++++++++++++++++ .../scala/fs2/io/uring/net/UringSocket.scala | 31 ++++++++------- 2 files changed, 55 insertions(+), 14 deletions(-) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala new file mode 100644 index 00000000..30a8e878 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala @@ -0,0 +1,38 @@ +package fs2.io.uring.net + +import cats.effect.kernel.Resource +import cats.effect.kernel.Sync + +import io.netty.buffer.ByteBuf +import io.netty.buffer.UnpooledByteBufAllocator + +private[net] final class ResizableBuffer[F[_]] private ( + private var buffer: ByteBuf +)(implicit F: Sync[F]) { + + def get(minCapacity: Int): F[ByteBuf] = F.delay { + if (buffer.capacity() >= minCapacity) { + buffer + } else { + val newBuffer = UnpooledByteBufAllocator.DEFAULT.directBuffer(minCapacity) + newBuffer.writeBytes(buffer) + buffer.release(buffer.refCnt()) + buffer = newBuffer + buffer + } + } + + def currentBuffer: ByteBuf = buffer + +} + +private[net] object ResizableBuffer { + + def apply[F[_]](size: Int)(implicit F: Sync[F]): Resource[F, ResizableBuffer[F]] = + Resource.make { + F.delay { + val initialBuffer = UnpooledByteBufAllocator.DEFAULT.directBuffer(size) + new ResizableBuffer(initialBuffer) + } + }(buf => F.delay { val _ = buf.currentBuffer.release(buf.currentBuffer.refCnt()) }) +} diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala index c5e20b6d..c7715e70 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocket.scala @@ -31,10 +31,8 @@ import fs2.Stream import fs2.io.net.Socket import fs2.io.uring.Uring -import fs2.io.uring.unsafe.util.createBuffer import fs2.io.uring.unsafe.util.OP._ -import io.netty.buffer.ByteBuf import io.netty.incubator.channel.uring.UringLinuxSocket import fs2.io.uring.unsafe.util.errno._ @@ -42,7 +40,8 @@ private[net] final class UringSocket[F[_]: LiftIO]( ring: Uring, sockfd: Int, _remoteAddress: SocketAddress[IpAddress], - buffer: ByteBuf, + readBuffer: ResizableBuffer[F], + writeBuffer: ResizableBuffer[F], defaultReadSize: Int, readMutex: Mutex[F], writeMutex: Mutex[F] @@ -59,13 +58,13 @@ private[net] final class UringSocket[F[_]: LiftIO]( def read(maxBytes: Int): F[Option[Chunk[Byte]]] = readMutex.lock.surround { for { - _ <- F.delay(buffer.clear()) // Clear the buffer before writing + buf <- readBuffer.get(maxBytes) _ <- F.whenA(debugRead)( F.delay(println(s"[SOCKET][READ] writing the received message in the buffer...")) ) - readed <- recv(buffer.memoryAddress(), maxBytes, 0) + readed <- recv(buf.memoryAddress(), maxBytes, 0) _ <- F.whenA(debugRead)( F.delay( @@ -75,7 +74,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( bytes <- F.delay { val arr = new Array[Byte](readed) - buffer.getBytes(0, arr) + buf.getBytes(0, arr) arr } _ <- F.whenA(debugRead)(F.delay(println(s"[SOCKET][READ] Done reading!"))) @@ -86,17 +85,17 @@ private[net] final class UringSocket[F[_]: LiftIO]( def readN(numBytes: Int): F[Chunk[Byte]] = readMutex.lock.surround { for { - _ <- F.delay(buffer.clear()) + buf <- readBuffer.get(numBytes) readed <- recv( - buffer.memoryAddress(), + buf.memoryAddress(), numBytes, 0 ) bytes <- F.delay { val arr = new Array[Byte](readed) - buffer.getBytes(0, arr) + buf.getBytes(0, arr) arr } } yield if (readed > 0) Chunk.array(bytes) else Chunk.empty @@ -128,9 +127,10 @@ private[net] final class UringSocket[F[_]: LiftIO]( F.delay(println(s"[SOCKET][WRITE] transfering to the buffer the bytes...")) ) + buf <- writeBuffer.get(bytes.size) + _ <- F.delay { - buffer.clear() - buffer.writeBytes(bytes.toArray) + buf.writeBytes(bytes.toArray) } _ <- F.whenA(debugWrite)( @@ -138,7 +138,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( ) _ <- send( - buffer.memoryAddress(), + buf.memoryAddress(), bytes.size, 0 ) @@ -146,6 +146,7 @@ private[net] final class UringSocket[F[_]: LiftIO]( _ <- F.whenA(debugWrite)(F.delay(println(s"[SOCKET][WRITE] message sent!"))) } yield () + } .unlessA(bytes.isEmpty) @@ -163,14 +164,16 @@ private[net] object UringSocket { F: Async[F] ): Resource[F, UringSocket[F]] = for { - buffer <- createBuffer(defaultReadSize) + readBuffer <- ResizableBuffer(defaultReadSize) + writeBuffer <- ResizableBuffer(defaultReadSize) readMutex <- Resource.eval(Mutex[F]) writeMutex <- Resource.eval(Mutex[F]) socket = new UringSocket( ring, fd, remote, - buffer, + readBuffer, + writeBuffer, defaultReadSize, readMutex, writeMutex From b678106bbb17756da325f11beaccdff081aff33c Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 1 Oct 2023 01:17:19 +0200 Subject: [PATCH 196/200] Replace buffer size with a constant value --- .../src/main/scala/fs2/io/uring/net/UringSocketGroup.scala | 7 ++----- .../jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala | 2 ++ 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala index fbb14118..66456982 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/UringSocketGroup.scala @@ -60,8 +60,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn fd <- openSocket(ring, isIpv6) - // linuxSocket <- Resource.eval(F.delay(UringLinuxSocket(fd))) - _ <- Resource.eval( createBufferAux(isIpv6).use { buf => // Write address in the buffer and call connect for { @@ -148,9 +146,9 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn .resource(createBufferAux(isIpv6)) .flatMap { buf => // Buffer that will contain the remote address Stream - .resource(createBuffer(4)) + .resource(createBuffer(BUFFER_SIZE)) .flatMap { - bufLength => // ACCEPT_OP needs a pointer to a buffer containing the size of the first buffer + bufLength => // IORING_OP_ACCEPT needs a pointer to a buffer containing the size of the first buffer Stream.resource { // Accept a connection, write the remote address on the buf and get the clientFd @@ -223,7 +221,6 @@ private final class UringSocketGroup[F[_]: LiftIO](implicit F: Async[F], dns: Dn IO.whenA(debug)(IO.println(s"The fd to close is: $fd")) *> ring .call(op = IORING_OP_CLOSE, fd = fd) .void - } object UringSocketGroup { diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala index 8056ecb5..e4e23d5e 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringRing.scala @@ -432,6 +432,8 @@ object NativeAccess { val POLLOUT = Native.POLLOUT + val BUFFER_SIZE = 4 + /** Creates a RingBuffer with the default size and IO sequence async threshold. * @return A new RingBuffer instance. */ From eee6522c0f229ea7021bd39175f094445ce4df01 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 1 Oct 2023 01:17:43 +0200 Subject: [PATCH 197/200] Rename register to access --- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 6 +++--- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index c99946d0..c4ac5a97 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -93,7 +93,7 @@ object UringSystem extends PollingSystem { () } - private final class ApiImpl(register: (Poller => Unit) => Unit) extends Uring { + private final class ApiImpl(access: (Poller => Unit) => Unit) extends Uring { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit def call( @@ -138,7 +138,7 @@ object UringSystem extends PollingSystem { correctRing: Poller ): IO[Boolean] = IO.async_[Int] { cb => - register { ring => + access { ring => val operationAddress = Encoder.encode(fd, op, id) if (debugCancel) println( @@ -170,7 +170,7 @@ object UringSystem extends PollingSystem { ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => val submit: IO[(Short, Poller)] = IO.async_[(Short, Poller)] { cb => - register { ring => + access { ring => val id = ring.getId(resume) ring.enqueueSqe(op, flags, rwFlags, fd, bufferAddress, length, offset, id) cb(Right((id, ring))) diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index 8e9322e8..ad2bfb5d 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -78,7 +78,7 @@ object UringSystem extends PollingSystem { def interrupt(targetThread: Thread, targetPoller: Poller): Unit = () - private final class ApiImpl(register: (Poller => Unit) => Unit) + private final class ApiImpl(access: (Poller => Unit) => Unit) extends Uring with FileDescriptorPoller { private[this] val noopRelease: Int => IO[Unit] = _ => IO.unit @@ -101,7 +101,7 @@ object UringSystem extends PollingSystem { ): (Either[Throwable, Int] => Unit, F[Int], IO ~> F) => F[Int] = { (resume, get, lift) => F.uncancelable { poll => val submit = IO.async_[ULong] { cb => - register { ring => + access { ring => val sqe = ring.getSqe(resume) prep(sqe) cb(Right(sqe.user_data)) @@ -130,7 +130,7 @@ object UringSystem extends PollingSystem { private[this] def cancel(addr: __u64): IO[Boolean] = IO.async_[Int] { cb => - register { ring => + access { ring => val sqe = ring.getSqe(cb) io_uring_prep_cancel64(sqe, addr, 0) } From 32a3282a0f960cf5c133191b797a3e1ae78d769a Mon Sep 17 00:00:00 2001 From: antjim1 Date: Sun, 1 Oct 2023 01:22:43 +0200 Subject: [PATCH 198/200] Rename register to access --- .../jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 2 +- .../src/main/scala/fs2/io/uring/unsafe/UringSystem.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index c4ac5a97..49304f4b 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -61,7 +61,7 @@ object UringSystem extends PollingSystem { private val debugHandleCompletionQueue = debug && true type Api = Uring - override def makeApi(register: (Poller => Unit) => Unit): Api = new ApiImpl(register) + override def makeApi(access: (Poller => Unit) => Unit): Api = new ApiImpl(access) override def makePoller(): Poller = new Poller(UringRing()) diff --git a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala index ad2bfb5d..8b165e2e 100644 --- a/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala +++ b/uring/native/src/main/scala/fs2/io/uring/unsafe/UringSystem.scala @@ -47,8 +47,8 @@ object UringSystem extends PollingSystem { def close(): Unit = () - def makeApi(register: (Poller => Unit) => Unit): Api = - new ApiImpl(register) + def makeApi(access: (Poller => Unit) => Unit): Api = + new ApiImpl(access) def makePoller(): Poller = { val ring = util.malloc[io_uring]() From b5cfd0be14f16d165c2ccad5c073f914f8c11838 Mon Sep 17 00:00:00 2001 From: antjim1 Date: Wed, 1 May 2024 22:47:48 +0200 Subject: [PATCH 199/200] Add license --- .../scala/fs2/io/uring/net/ResizableBuffer.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala b/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala index 30a8e878..81f0b4b8 100644 --- a/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala +++ b/uring/jvm/src/main/scala/fs2/io/uring/net/ResizableBuffer.scala @@ -1,3 +1,19 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package fs2.io.uring.net import cats.effect.kernel.Resource From 6880b0ccb21b18350cc189136df14eaf8d8662c5 Mon Sep 17 00:00:00 2001 From: Antonio Jimenez Date: Sun, 4 Aug 2024 12:22:25 +0200 Subject: [PATCH 200/200] Mock Cheshire API --- .../io/uring/unsafe/CheshireUringRing.scala | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) create mode 100644 uring/jvm/src/main/scala/fs2/io/uring/unsafe/CheshireUringRing.scala diff --git a/uring/jvm/src/main/scala/fs2/io/uring/unsafe/CheshireUringRing.scala b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/CheshireUringRing.scala new file mode 100644 index 00000000..cc6a5262 --- /dev/null +++ b/uring/jvm/src/main/scala/fs2/io/uring/unsafe/CheshireUringRing.scala @@ -0,0 +1,91 @@ +/* + * Copyright 2022 Arman Bilge + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.netty.incubator.channel.uring + +case class MemorySegment() { + def address(): Long = 0L +} + +case class io_uring() { + def address(): Long = 0L +} + +case class io_uring_cqe() { + def address(): Long = 0L +} + +case class io_uring_sqe() { + def address(): Long = 0L +} + +case class kernel_timespec() { + def address(): Long = 0L +} + +// Mock Cherishire API for UringRing +object CheshireUringRing { + def io_uring_peek_cqe(ring: io_uring, cqePtr: io_uring_cqe, nrAvailable: MemorySegment) = ??? + def io_uring_peek_cqe(ring: io_uring, cqePtr: io_uring_cqe) = ??? + def io_uring_wait_cqe(ring: io_uring, cqePtr: io_uring_cqe) = ??? + def io_uring_cqe_seen(ring: io_uring, cqePtr: io_uring_cqe) = ??? + def io_uring_queue_init(entries: Int, ring: io_uring, flags: Int) = ??? + def io_uring_queue_exit(ring: io_uring) = ??? + def io_uring_get_sqe(ring: io_uring) = ??? + def io_uring_submit(ring: io_uring) = ??? + def io_uring_submit_and_wait(ring: io_uring, waitNr: Int) = ??? + def io_uring_submit_and_wait_timeout( + ring: io_uring, + cqePtr: io_uring_cqe, + waitNr: Int, + ts: kernel_timespec, + sigmask: MemorySegment + ) = ??? + def io_uring_wait_cqe_timeout(ring: io_uring, cqePtr: io_uring_cqe, ts: kernel_timespec) = ??? + def io_uring_peek_batch_cqe(ring: io_uring, cqes: MemorySegment, count: Int) = ??? + def io_uring_cq_advance(ring: io_uring, nr: Int) = ??? + def io_uring_cq_ready(ring: io_uring) = ??? + def io_uring_prep_rw( + op: Int, + sqe: io_uring_sqe, + fd: Int, + addr: MemorySegment, + len: Int, + offset: Long + ) = ??? + def io_uring_prep_nop(sqe: io_uring_sqe) = ??? + def io_uring_prep_accept( + sqe: io_uring_sqe, + fd: Int, + addr: MemorySegment, + addrlen: MemorySegment, + flags: Int + ) = ??? + def io_uring_prep_cancel64(sqe: io_uring_sqe, userData: Long, flags: Int) = ??? + def io_uring_prep_close(sqe: io_uring_sqe, fd: Int) = ??? + def io_uring_prep_connect(sqe: io_uring_sqe, fd: Int, addr: MemorySegment, addrlen: Int) = ??? + def io_uring_prep_recv(sqe: io_uring_sqe, sockfd: Int, buf: MemorySegment, len: Int, flags: Int) = + ??? + def io_uring_prep_send(sqe: io_uring_sqe, sockfd: Int, buf: MemorySegment, len: Int, flags: Int) = + ??? + def io_uring_sqe_set_data64(sqe: io_uring_sqe, data: Long) = ??? + def io_uring_cqe_get_data64(cqe: io_uring_cqe) = ??? + def io_uring_prep_shutdown(sqe: io_uring_sqe, fd: Int, how: Int) = ??? + def io_uring_prep_socket(sqe: io_uring_sqe, domain: Int, _type: Int, protocol: Int, flags: Int) = + ??? + def io_uring_prep_timeout(sqe: io_uring_sqe, ts: kernel_timespec, count: Int, flags: Int) = ??? + +}