mirror of
https://github.com/fluencelabs/aqua.git
synced 2024-12-12 17:55:33 +00:00
Removing IO from compiler submodule (#186)
* Removing IO from compiler submodule (wip) * move targets to cli
This commit is contained in:
parent
f15bd0558b
commit
d24e77b5e3
19
build.sbt
19
build.sbt
@ -22,6 +22,8 @@ val declineEnumV = "1.3.0"
|
||||
val airframeLog = "org.wvlet.airframe" %% "airframe-log" % airframeLogV
|
||||
val catsEffect = "org.typelevel" %% "cats-effect" % catsEffectV
|
||||
val fs2Io = "co.fs2" %% "fs2-io" % fs2V
|
||||
val catsFree = "org.typelevel" %% "cats-free" % catsV
|
||||
val cats = "org.typelevel" %% "cats-core" % catsV
|
||||
|
||||
name := "aqua-hll"
|
||||
|
||||
@ -49,6 +51,7 @@ lazy val cli = project
|
||||
"com.monovore" %% "decline" % declineV,
|
||||
"com.monovore" %% "decline-effect" % declineV,
|
||||
catsEffect,
|
||||
fs2Io,
|
||||
"org.typelevel" %% "log4cats-slf4j" % log4catsV,
|
||||
"com.beachape" %% "enumeratum" % enumeratumV,
|
||||
"org.slf4j" % "slf4j-jdk14" % slf4jV,
|
||||
@ -61,7 +64,7 @@ lazy val types = project
|
||||
.settings(commons)
|
||||
.settings(
|
||||
libraryDependencies ++= Seq(
|
||||
"org.typelevel" %% "cats-core" % catsV
|
||||
cats
|
||||
)
|
||||
)
|
||||
|
||||
@ -70,7 +73,7 @@ lazy val parser = project
|
||||
.settings(
|
||||
libraryDependencies ++= Seq(
|
||||
"org.typelevel" %% "cats-parse" % catsParseV,
|
||||
"org.typelevel" %% "cats-free" % catsV
|
||||
catsFree
|
||||
)
|
||||
)
|
||||
.dependsOn(types)
|
||||
@ -79,7 +82,7 @@ lazy val linker = project
|
||||
.settings(commons: _*)
|
||||
.settings(
|
||||
libraryDependencies ++= Seq(
|
||||
"org.wvlet.airframe" %% "airframe-log" % airframeLogV
|
||||
airframeLog
|
||||
)
|
||||
)
|
||||
.dependsOn(parser)
|
||||
@ -88,7 +91,7 @@ lazy val model = project
|
||||
.settings(commons: _*)
|
||||
.settings(
|
||||
libraryDependencies ++= Seq(
|
||||
"org.typelevel" %% "cats-free" % catsV
|
||||
catsFree
|
||||
)
|
||||
)
|
||||
.dependsOn(types)
|
||||
@ -111,13 +114,7 @@ lazy val semantics = project
|
||||
lazy val compiler = project
|
||||
.in(file("compiler"))
|
||||
.settings(commons: _*)
|
||||
.settings(
|
||||
libraryDependencies ++= Seq(
|
||||
catsEffect,
|
||||
fs2Io
|
||||
)
|
||||
)
|
||||
.dependsOn(model, semantics, linker, backend)
|
||||
.dependsOn(semantics, linker, backend)
|
||||
|
||||
lazy val backend = project
|
||||
.in(file("backend"))
|
||||
|
@ -4,8 +4,7 @@ import aqua.backend.Backend
|
||||
import aqua.backend.air.AirBackend
|
||||
import aqua.backend.js.JavaScriptBackend
|
||||
import aqua.backend.ts.TypeScriptBackend
|
||||
import aqua.compiler.AquaCompiler
|
||||
import aqua.compiler.AquaCompiler.{AirTarget, CompileTarget, JavaScriptTarget, TypescriptTarget}
|
||||
import aqua.compiler.{AquaCompiler, AquaIO}
|
||||
import aqua.model.transform.BodyConfig
|
||||
import aqua.parser.lift.LiftParser.Implicits.idLiftParser
|
||||
import cats.Id
|
||||
@ -34,6 +33,11 @@ object CustomLogFormatter extends LogFormatter {
|
||||
object AquaCli extends IOApp with LogSupport {
|
||||
import AppOps._
|
||||
|
||||
sealed trait CompileTarget
|
||||
case object TypescriptTarget extends CompileTarget
|
||||
case object JavaScriptTarget extends CompileTarget
|
||||
case object AirTarget extends CompileTarget
|
||||
|
||||
def targetToBackend(target: CompileTarget): Backend = {
|
||||
target match {
|
||||
case TypescriptTarget =>
|
||||
@ -68,13 +72,15 @@ object AquaCli extends IOApp with LogSupport {
|
||||
WLogger.setDefaultLogLevel(LogLevel.toLogLevel(logLevel))
|
||||
WLogger.setDefaultFormatter(CustomLogFormatter)
|
||||
|
||||
implicit val aio: AquaIO[F] = new AquaFilesIO[F]
|
||||
|
||||
// if there is `--help` or `--version` flag - show help and version
|
||||
// otherwise continue program execution
|
||||
h.map(_ => helpAndExit) orElse v.map(_ => versionAndExit) getOrElse {
|
||||
val target =
|
||||
if (toAir) AquaCompiler.AirTarget
|
||||
else if (toJs) AquaCompiler.JavaScriptTarget
|
||||
else AquaCompiler.TypescriptTarget
|
||||
if (toAir) AirTarget
|
||||
else if (toJs) JavaScriptTarget
|
||||
else TypescriptTarget
|
||||
val bc = {
|
||||
val bc = BodyConfig(wrapWithXor = !noXor, constants = constants)
|
||||
bc.copy(relayVarName = bc.relayVarName.filterNot(_ => noRelay))
|
||||
|
141
cli/src/main/scala/aqua/AquaFilesIO.scala
Normal file
141
cli/src/main/scala/aqua/AquaFilesIO.scala
Normal file
@ -0,0 +1,141 @@
|
||||
package aqua
|
||||
|
||||
import aqua.compiler.AquaIO
|
||||
import aqua.compiler.io.{
|
||||
AquaFileError,
|
||||
EmptyFileError,
|
||||
FileNotFound,
|
||||
FileSystemError,
|
||||
FileWriteError
|
||||
}
|
||||
import aqua.parser.lift.FileSpan
|
||||
import cats.data.Validated.{Invalid, Valid}
|
||||
import cats.data.{Chain, EitherT, NonEmptyChain, Validated, ValidatedNec}
|
||||
import cats.syntax.functor._
|
||||
import cats.syntax.either._
|
||||
import cats.effect.kernel.Concurrent
|
||||
import fs2.io.file.Files
|
||||
import fs2.text
|
||||
import cats.syntax.applicative._
|
||||
import cats.syntax.flatMap._
|
||||
import cats.syntax.apply._
|
||||
|
||||
import java.nio.file.Path
|
||||
import scala.util.Try
|
||||
|
||||
class AquaFilesIO[F[_]: Files: Concurrent] extends AquaIO[F] {
|
||||
|
||||
override def readFile(file: Path): EitherT[F, AquaFileError, String] =
|
||||
EitherT(
|
||||
Files[F]
|
||||
.readAll(file, 4096)
|
||||
.fold(Vector.empty[Byte])((acc, b) => acc :+ b)
|
||||
// TODO fix for comment on last line in air
|
||||
// TODO should be fixed by parser
|
||||
.map(_.appendedAll("\n\r".getBytes))
|
||||
.flatMap(fs2.Stream.emits)
|
||||
.through(text.utf8Decode)
|
||||
.attempt
|
||||
.compile
|
||||
.last
|
||||
.map(
|
||||
_.fold((EmptyFileError(file): AquaFileError).asLeft[String])(_.left.map(FileSystemError))
|
||||
)
|
||||
)
|
||||
|
||||
private def findFirstF(
|
||||
in: List[Path],
|
||||
notFound: EitherT[F, AquaFileError, Path]
|
||||
): EitherT[F, AquaFileError, Path] =
|
||||
in.headOption.fold(notFound)(p =>
|
||||
EitherT(
|
||||
Concurrent[F].attempt(p.toFile.isFile.pure[F])
|
||||
)
|
||||
.leftMap[AquaFileError](FileSystemError)
|
||||
.recover({ case _ => false })
|
||||
.flatMap {
|
||||
case true =>
|
||||
EitherT(
|
||||
Concurrent[F].attempt(p.toAbsolutePath.normalize().pure[F])
|
||||
).leftMap[AquaFileError](FileSystemError)
|
||||
case false =>
|
||||
findFirstF(in.tail, notFound)
|
||||
}
|
||||
)
|
||||
|
||||
/**
|
||||
* Checks if a file exists in the list of possible paths
|
||||
*/
|
||||
def resolve(
|
||||
focus: FileSpan.Focus,
|
||||
src: Path,
|
||||
imports: List[Path]
|
||||
): EitherT[F, AquaFileError, Path] =
|
||||
findFirstF(
|
||||
imports
|
||||
.map(_.resolve(src)),
|
||||
EitherT.leftT(FileNotFound(focus, src, imports))
|
||||
)
|
||||
|
||||
override def listAqua(folder: Path): F[ValidatedNec[AquaFileError, Chain[Path]]] =
|
||||
Validated
|
||||
.fromTry(
|
||||
Try {
|
||||
val f = folder.toFile
|
||||
if (f.isDirectory) {
|
||||
f.listFiles().toList
|
||||
} else {
|
||||
f :: Nil
|
||||
}
|
||||
}
|
||||
)
|
||||
.leftMap[AquaFileError](FileSystemError)
|
||||
.leftMap(NonEmptyChain.one)
|
||||
.pure[F]
|
||||
.flatMap {
|
||||
case Valid(files) =>
|
||||
files.collect {
|
||||
case f if f.isFile && f.getName.endsWith(".aqua") =>
|
||||
Validated
|
||||
.fromTry(
|
||||
Try(Chain.one(f.toPath.toAbsolutePath.normalize()))
|
||||
)
|
||||
.leftMap(FileSystemError)
|
||||
.leftMap(NonEmptyChain.one)
|
||||
.pure[F]
|
||||
case f if f.isDirectory =>
|
||||
listAqua(f.toPath)
|
||||
}.foldLeft(Validated.validNec[AquaFileError, Chain[Path]](Chain.nil).pure[F]) {
|
||||
case (acc, v) =>
|
||||
(acc, v).mapN(_ combine _)
|
||||
}
|
||||
case Invalid(errs) =>
|
||||
Validated.invalid[NonEmptyChain[AquaFileError], Chain[Path]](errs).pure[F]
|
||||
}
|
||||
|
||||
override def writeFile(file: Path, content: String): EitherT[F, AquaFileError, Unit] =
|
||||
EitherT
|
||||
.right[AquaFileError](Files[F].deleteIfExists(file))
|
||||
.flatMap(_ =>
|
||||
EitherT[F, AquaFileError, Unit](
|
||||
fs2.Stream
|
||||
.emit(
|
||||
content
|
||||
)
|
||||
.through(text.utf8Encode)
|
||||
.through(Files[F].writeAll(file))
|
||||
.attempt
|
||||
.map { e =>
|
||||
e.left
|
||||
.map(t => FileWriteError(file, t))
|
||||
}
|
||||
.compile
|
||||
.drain
|
||||
.map(_ => Right(()))
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
object AquaFilesIO {
|
||||
implicit def summon[F[_]: Files: Concurrent]: AquaIO[F] = new AquaFilesIO[F]
|
||||
}
|
@ -1,7 +1,7 @@
|
||||
package aqua
|
||||
|
||||
import aqua.backend.ts.TypeScriptBackend
|
||||
import aqua.compiler.AquaCompiler
|
||||
import aqua.compiler.{AquaCompiler, AquaIO}
|
||||
import aqua.model.transform.BodyConfig
|
||||
import cats.data.Validated
|
||||
import cats.effect.{IO, IOApp, Sync}
|
||||
@ -15,6 +15,8 @@ object Test extends IOApp.Simple {
|
||||
implicit def logger[F[_]: Sync]: SelfAwareStructuredLogger[F] =
|
||||
Slf4jLogger.getLogger[F]
|
||||
|
||||
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO]
|
||||
|
||||
override def run: IO[Unit] =
|
||||
AquaCompiler
|
||||
.compileFilesTo[IO](
|
||||
|
@ -17,6 +17,8 @@ class WriteFileSpec extends AnyFlatSpec with Matchers {
|
||||
val targetJs = Files.createTempDirectory("js")
|
||||
val targetAir = Files.createTempDirectory("air")
|
||||
|
||||
import aqua.AquaFilesIO.summon
|
||||
|
||||
val bc = BodyConfig()
|
||||
AquaCompiler
|
||||
.compileFilesTo[IO](src, List.empty, targetTs, TypeScriptBackend, bc)
|
||||
|
@ -8,21 +8,15 @@ import aqua.model.transform.BodyConfig
|
||||
import aqua.parser.lift.FileSpan
|
||||
import aqua.semantics.{RulesViolated, SemanticError, Semantics}
|
||||
import cats.data._
|
||||
import cats.effect.kernel.Concurrent
|
||||
import cats.kernel.Monoid
|
||||
import cats.syntax.flatMap._
|
||||
import cats.syntax.functor._
|
||||
import cats.{Applicative, Monad}
|
||||
import fs2.io.file.Files
|
||||
import wvlet.log.LogSupport
|
||||
|
||||
import java.nio.file.Path
|
||||
|
||||
object AquaCompiler extends LogSupport {
|
||||
sealed trait CompileTarget
|
||||
case object TypescriptTarget extends CompileTarget
|
||||
case object JavaScriptTarget extends CompileTarget
|
||||
case object AirTarget extends CompileTarget
|
||||
|
||||
private def gatherPreparedFiles(
|
||||
srcPath: Path,
|
||||
@ -56,7 +50,7 @@ object AquaCompiler extends LogSupport {
|
||||
/**
|
||||
* Create a structure that will be used to create output by a backend
|
||||
*/
|
||||
def prepareFiles[F[_]: Files: Concurrent](
|
||||
def prepareFiles[F[_]: AquaIO: Monad](
|
||||
srcPath: Path,
|
||||
imports: List[Path],
|
||||
targetPath: Path
|
||||
@ -103,7 +97,7 @@ object AquaCompiler extends LogSupport {
|
||||
|
||||
private def gatherResults[F[_]: Monad](
|
||||
results: List[EitherT[F, String, Unit]]
|
||||
): F[Validated[NonEmptyChain[String], Chain[String]]] = {
|
||||
): F[ValidatedNec[String, Chain[String]]] = {
|
||||
results
|
||||
.foldLeft(
|
||||
EitherT.rightT[F, NonEmptyChain[String]](Chain.empty[String])
|
||||
@ -122,7 +116,7 @@ object AquaCompiler extends LogSupport {
|
||||
.map(Validated.fromEither)
|
||||
}
|
||||
|
||||
def compileFilesTo[F[_]: Files: Concurrent](
|
||||
def compileFilesTo[F[_]: AquaIO: Monad](
|
||||
srcPath: Path,
|
||||
imports: List[Path],
|
||||
targetPath: Path,
|
||||
@ -150,11 +144,8 @@ object AquaCompiler extends LogSupport {
|
||||
targetPath.fold(
|
||||
t => EitherT.leftT[F, Unit](t.getMessage),
|
||||
tp =>
|
||||
FileOps
|
||||
.writeFile(
|
||||
tp,
|
||||
compiled.content
|
||||
)
|
||||
AquaIO[F]
|
||||
.writeFile(tp, compiled.content)
|
||||
.flatTap { _ =>
|
||||
EitherT.pure(
|
||||
Validated.catchNonFatal(
|
||||
@ -164,6 +155,7 @@ object AquaCompiler extends LogSupport {
|
||||
)
|
||||
)
|
||||
}
|
||||
.leftMap(_.showForConsole)
|
||||
)
|
||||
}
|
||||
)
|
||||
|
25
compiler/src/main/scala/aqua/compiler/AquaIO.scala
Normal file
25
compiler/src/main/scala/aqua/compiler/AquaIO.scala
Normal file
@ -0,0 +1,25 @@
|
||||
package aqua.compiler
|
||||
|
||||
import aqua.compiler.io.AquaFileError
|
||||
import aqua.parser.lift.FileSpan
|
||||
import cats.data.{Chain, EitherT, ValidatedNec}
|
||||
|
||||
import java.nio.file.Path
|
||||
|
||||
trait AquaIO[F[_]] {
|
||||
def readFile(file: Path): EitherT[F, AquaFileError, String]
|
||||
|
||||
def resolve(
|
||||
focus: FileSpan.Focus,
|
||||
src: Path,
|
||||
imports: List[Path]
|
||||
): EitherT[F, AquaFileError, Path]
|
||||
|
||||
def listAqua(folder: Path): F[ValidatedNec[AquaFileError, Chain[Path]]]
|
||||
|
||||
def writeFile(file: Path, content: String): EitherT[F, AquaFileError, Unit]
|
||||
}
|
||||
|
||||
object AquaIO {
|
||||
def apply[F[_]](implicit aio: AquaIO[F]): AquaIO[F] = aio
|
||||
}
|
@ -1,19 +1,17 @@
|
||||
package aqua.compiler.io
|
||||
|
||||
import aqua.compiler.io.AquaFiles.ETC
|
||||
import aqua.compiler.{CustomSyntaxError, SyntaxError}
|
||||
import aqua.compiler.{AquaIO, CustomSyntaxError, SyntaxError}
|
||||
import aqua.linker.AquaModule
|
||||
import aqua.parser.head.ImportExpr
|
||||
import aqua.parser.lift.FileSpan.F
|
||||
import aqua.parser.lift.{FileSpan, LiftParser, Span}
|
||||
import aqua.parser.{Ast, BlockIndentError, FuncReturnError, LexerError}
|
||||
import cats.Eval
|
||||
import cats.{Eval, Monad}
|
||||
import cats.data.{EitherT, NonEmptyChain}
|
||||
import cats.effect.Concurrent
|
||||
import cats.parse.LocationMap
|
||||
import cats.syntax.apply._
|
||||
import cats.syntax.functor._
|
||||
import fs2.io.file.Files
|
||||
|
||||
import java.nio.file.{Path, Paths}
|
||||
import scala.collection.immutable
|
||||
@ -28,9 +26,9 @@ case class AquaFile(
|
||||
/**
|
||||
* Gathers all errors and results
|
||||
*/
|
||||
private def gatherResolvedResults[F[_]: Concurrent](
|
||||
private def gatherResolvedResults[F[_]: Monad](
|
||||
results: immutable.Iterable[EitherT[F, AquaFileError, (FileModuleId, FileNotFound)]]
|
||||
): ETC[F, Map[FileModuleId, AquaFileError]] = {
|
||||
): ETC[F, Map[FileModuleId, AquaFileError]] =
|
||||
results
|
||||
.foldLeft[AquaFiles.ETC[F, Map[FileModuleId, AquaFileError]]](EitherT.rightT(Map())) {
|
||||
case (files, nextFile) =>
|
||||
@ -43,15 +41,15 @@ case class AquaFile(
|
||||
Left(errs.append(err))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
def createModule[F[_]: Concurrent, T](
|
||||
def createModule[F[_]: AquaIO: Monad, T](
|
||||
transpile: Ast[FileSpan.F] => T => T,
|
||||
importFrom: List[Path]
|
||||
): AquaFiles.ETC[F, AquaModule[FileModuleId, AquaFileError, T]] = {
|
||||
val resolvedImports = imports.map { case (pathString, focus) =>
|
||||
FileModuleId
|
||||
AquaIO[F]
|
||||
.resolve(focus, Paths.get(pathString), id.file.getParent +: importFrom)
|
||||
.map(FileModuleId)
|
||||
// 'FileNotFound' will be used later if there will be problems in compilation
|
||||
.map(id => (id -> FileNotFound(focus, id.file, importFrom)))
|
||||
}
|
||||
@ -90,22 +88,10 @@ object AquaFile {
|
||||
.map(AquaScriptErrors(_))
|
||||
}
|
||||
|
||||
def read[F[_]: Files: Concurrent](file: Path): EitherT[F, AquaFileError, AquaFile] = {
|
||||
def read[F[_]: AquaIO: Monad](file: Path): EitherT[F, AquaFileError, AquaFile] =
|
||||
for {
|
||||
sourceOp <- EitherT.right(
|
||||
FileOps
|
||||
.readSourceText[F](file)
|
||||
.map {
|
||||
_.left
|
||||
.map(t => FileSystemError(t))
|
||||
}
|
||||
.compile
|
||||
.last
|
||||
)
|
||||
source <- EitherT.fromEither(sourceOp.getOrElse(Left(EmptyFileError(file))))
|
||||
_ <- EitherT.fromEither(
|
||||
if (source.isEmpty) Left(EmptyFileError(file): AquaFileError) else Right(())
|
||||
)
|
||||
source <- AquaIO[F].readFile(file)
|
||||
_ <- EitherT.cond[F](source.nonEmpty, (), EmptyFileError(file))
|
||||
ast <- EitherT.fromEither(parseAst(file.toString, source))
|
||||
imports = ast.head.tailForced
|
||||
.map(_.head)
|
||||
@ -119,13 +105,10 @@ object AquaFile {
|
||||
}
|
||||
.toList
|
||||
.toMap
|
||||
} yield {
|
||||
AquaFile(
|
||||
} yield AquaFile(
|
||||
FileModuleId(file.toAbsolutePath.normalize()),
|
||||
imports,
|
||||
source,
|
||||
ast
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -29,6 +29,10 @@ case class FileSystemError(err: Throwable) extends Exception(err) with AquaFileE
|
||||
override def showForConsole: String = s"File system error: ${err.getMessage}"
|
||||
}
|
||||
|
||||
case class FileWriteError(file: Path, err: Throwable) extends Exception(err) with AquaFileError {
|
||||
override def showForConsole: String = s"Cannot write a file $file: ${err.getMessage}"
|
||||
}
|
||||
|
||||
case class Unresolvable(msg: String) extends AquaFileError {
|
||||
override def showForConsole: String = s"Unresolvable: $msg"
|
||||
}
|
||||
|
@ -1,64 +1,42 @@
|
||||
package aqua.compiler.io
|
||||
|
||||
import aqua.compiler.AquaIO
|
||||
import aqua.linker.Modules
|
||||
import aqua.parser.Ast
|
||||
import aqua.parser.lift.FileSpan
|
||||
import cats.data.{Chain, EitherT, NonEmptyChain}
|
||||
import cats.effect.kernel.Concurrent
|
||||
import cats.syntax.apply._
|
||||
import fs2.io.file.Files
|
||||
import cats.Monad
|
||||
import cats.data.{Chain, EitherT, NonEmptyChain, Validated, ValidatedNec}
|
||||
import cats.syntax.functor._
|
||||
import cats.syntax.traverse._
|
||||
import cats.syntax.flatMap._
|
||||
import cats.syntax.applicative._
|
||||
|
||||
import java.nio.file.Path
|
||||
import scala.util.Try
|
||||
|
||||
object AquaFiles {
|
||||
type Mods[T] = Modules[FileModuleId, AquaFileError, T]
|
||||
type ETC[F[_], T] = EitherT[F, NonEmptyChain[AquaFileError], T]
|
||||
|
||||
def readSources[F[_]: Files: Concurrent](
|
||||
def readSources[F[_]: AquaIO: Monad](
|
||||
sourcePath: Path
|
||||
): ETC[F, Chain[AquaFile]] =
|
||||
// TODO use effect instead of Try
|
||||
EitherT
|
||||
.fromEither[F](
|
||||
Try {
|
||||
val f = sourcePath.toFile
|
||||
if (f.isDirectory) {
|
||||
f.listFiles().toList
|
||||
} else {
|
||||
List(f)
|
||||
}
|
||||
}.toEither
|
||||
)
|
||||
.leftMap[AquaFileError](FileSystemError)
|
||||
EitherT(
|
||||
AquaIO[F]
|
||||
.listAqua(sourcePath)
|
||||
.flatMap[ValidatedNec[AquaFileError, Chain[AquaFile]]] {
|
||||
case Validated.Invalid(e) =>
|
||||
Validated.invalid[NonEmptyChain[AquaFileError], Chain[AquaFile]](e).pure[F]
|
||||
case Validated.Valid(paths) =>
|
||||
paths
|
||||
.traverse(AquaFile.read(_))
|
||||
.leftMap(NonEmptyChain.one)
|
||||
.flatMap(
|
||||
_.collect {
|
||||
case f if f.isFile && f.getName.endsWith(".aqua") =>
|
||||
AquaFile
|
||||
.read(f.toPath.toAbsolutePath.normalize())
|
||||
.map(Chain(_))
|
||||
.leftMap(NonEmptyChain.one)
|
||||
case f if f.isDirectory =>
|
||||
readSources(f.toPath)
|
||||
}
|
||||
.foldLeft[ETC[F, Chain[AquaFile]]](
|
||||
EitherT.rightT(Chain.empty)
|
||||
) { case (accF, nextF) =>
|
||||
EitherT((accF.value, nextF.value).mapN {
|
||||
case (Right(acc), Right(v)) =>
|
||||
Right(acc ++ v)
|
||||
case (Left(acc), Left(v)) =>
|
||||
Left(acc ++ v)
|
||||
case (Left(acc), _) =>
|
||||
Left(acc)
|
||||
case (_, Left(v)) =>
|
||||
Left(v)
|
||||
})
|
||||
.value
|
||||
.map(Validated.fromEither)
|
||||
}
|
||||
.map(_.toEither)
|
||||
)
|
||||
|
||||
def createModules[F[_]: Concurrent, T](
|
||||
def createModules[F[_]: AquaIO: Monad, T](
|
||||
sources: Chain[AquaFile],
|
||||
importFromPaths: List[Path],
|
||||
transpile: Ast[FileSpan.F] => T => T
|
||||
@ -74,7 +52,7 @@ object AquaFiles {
|
||||
} yield ms.add(m, export = true)
|
||||
}
|
||||
|
||||
def resolveModules[F[_]: Files: Concurrent, T](
|
||||
def resolveModules[F[_]: AquaIO: Monad, T](
|
||||
modules: Modules[FileModuleId, AquaFileError, T],
|
||||
importFromPaths: List[Path],
|
||||
transpile: Ast[FileSpan.F] => T => T
|
||||
@ -98,7 +76,7 @@ object AquaFiles {
|
||||
case ms => resolveModules(ms, importFromPaths, transpile)
|
||||
}
|
||||
|
||||
def readAndResolve[F[_]: Files: Concurrent, T](
|
||||
def readAndResolve[F[_]: AquaIO: Monad, T](
|
||||
sourcePath: Path,
|
||||
importFromPaths: List[Path],
|
||||
transpile: Ast[FileSpan.F] => T => T
|
||||
|
@ -1,47 +1,5 @@
|
||||
package aqua.compiler.io
|
||||
|
||||
import aqua.parser.lift.FileSpan
|
||||
import cats.data.EitherT
|
||||
import cats.effect.kernel.Concurrent
|
||||
import cats.syntax.applicative._
|
||||
|
||||
import java.nio.file.Path
|
||||
|
||||
case class FileModuleId(file: Path)
|
||||
|
||||
object FileModuleId {
|
||||
|
||||
private def findFirstF[F[_]: Concurrent](
|
||||
in: List[Path],
|
||||
notFound: EitherT[F, AquaFileError, FileModuleId]
|
||||
): EitherT[F, AquaFileError, FileModuleId] =
|
||||
in.headOption.fold(notFound)(p =>
|
||||
EitherT(
|
||||
Concurrent[F].attempt(p.toFile.isFile.pure[F])
|
||||
)
|
||||
.leftMap[AquaFileError](FileSystemError)
|
||||
.recover({ case _ => false })
|
||||
.flatMap {
|
||||
case true =>
|
||||
EitherT(
|
||||
Concurrent[F].attempt(FileModuleId(p.toAbsolutePath.normalize()).pure[F])
|
||||
).leftMap[AquaFileError](FileSystemError)
|
||||
case false =>
|
||||
findFirstF(in.tail, notFound)
|
||||
}
|
||||
)
|
||||
|
||||
/**
|
||||
* Checks if a file existed in the list of possible paths
|
||||
*/
|
||||
def resolve[F[_]: Concurrent](
|
||||
focus: FileSpan.Focus,
|
||||
src: Path,
|
||||
imports: List[Path]
|
||||
): EitherT[F, AquaFileError, FileModuleId] =
|
||||
findFirstF(
|
||||
imports
|
||||
.map(_.resolve(src)),
|
||||
EitherT.leftT(FileNotFound(focus, src, imports))
|
||||
)
|
||||
}
|
||||
|
@ -1,47 +0,0 @@
|
||||
package aqua.compiler.io
|
||||
|
||||
import cats.data.EitherT
|
||||
import cats.effect.Concurrent
|
||||
import cats.implicits.toFunctorOps
|
||||
import fs2.io.file.Files
|
||||
import fs2.text
|
||||
|
||||
import java.nio.file.Path
|
||||
|
||||
object FileOps {
|
||||
|
||||
def writeFile[F[_]: Files: Concurrent](file: Path, content: String): EitherT[F, String, Unit] =
|
||||
EitherT
|
||||
.right[String](Files[F].deleteIfExists(file))
|
||||
.flatMap(_ =>
|
||||
EitherT[F, String, Unit](
|
||||
fs2.Stream
|
||||
.emit(
|
||||
content
|
||||
)
|
||||
.through(text.utf8Encode)
|
||||
.through(Files[F].writeAll(file))
|
||||
.attempt
|
||||
.map { e =>
|
||||
e.left
|
||||
.map(t => s"Error on writing file $file" + t)
|
||||
}
|
||||
.compile
|
||||
.drain
|
||||
.map(_ => Right(()))
|
||||
)
|
||||
)
|
||||
|
||||
def readSourceText[F[_]: Files: Concurrent](
|
||||
file: Path
|
||||
): fs2.Stream[F, Either[Throwable, String]] =
|
||||
Files[F]
|
||||
.readAll(file, 4096)
|
||||
.fold(Vector.empty[Byte])((acc, b) => acc :+ b)
|
||||
// TODO fix for comment on last line in air
|
||||
// TODO should be fixed by parser
|
||||
.map(_.appendedAll("\n\r".getBytes))
|
||||
.flatMap(fs2.Stream.emits)
|
||||
.through(text.utf8Decode)
|
||||
.attempt
|
||||
}
|
Loading…
Reference in New Issue
Block a user