Converting compiler to a pure function (#217)

* Trying to make the compiler a pure function

* Pure compiler WIP

* Compiler should be working now

* printlns

* printlns

* printlns

* fix, delete printlns

* more logs

* fix import resolving, more tests

* fix imports, add tests for imports resolving

* fix test

* correct paths to targets, correct output

* refactoring, create directories to file if not exist

* small changes

* fix test

* Tiny fixes WIP

* Tiny fixes

* Incrementing base version, as host_peer_id is added (fixes #218)

* render error messages, WIP

* small fix

* get src for lexer error

* wrap parser error

* add list of errors

* Handle file write errors accurately

* Use show syntax

* fix test

* fix test

* fix test

* println

Co-authored-by: DieMyst <dmitry.shakhtarin@fluence.ai>
This commit is contained in:
Dmitry Kurinskiy 2021-08-03 17:23:28 +03:00 committed by GitHub
parent 2eb4598093
commit e3716f6f31
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
55 changed files with 924 additions and 643 deletions

View File

@ -12,7 +12,7 @@ Therefore, Aqua scripts are compiled into several targets at once, with AIR and
## Using Aqua ## Using Aqua
Please refer to [aqua-playground](https://github.com/fluencelabs/aqua-playground) to learn how to use Aqua. Please refer to [Aqua Book](https://doc.fluence.dev/aqua-book/) to learn how to use Aqua.
## Compiler CLI ## Compiler CLI
@ -34,6 +34,8 @@ Input directory should contain files with `aqua` scripts.
- **[model](./model)** - middle-end, internal representation of the code, optimizations and transfromations - **[model](./model)** - middle-end, internal representation of the code, optimizations and transfromations
- **[semantics](./semantics)** - rules to convert source AST into the model - **[semantics](./semantics)** - rules to convert source AST into the model
- **[linker](./linker)** - checks dependencies between modules, builds and combines an abstract dependencies tree - **[linker](./linker)** - checks dependencies between modules, builds and combines an abstract dependencies tree
- **[backend](./backend)** - compilation backend interface
- **[compiler](./compiler)** - compiler as a pure function made from _linker_, _semantics_ and _backend_
- **[backend/air](./backend/air)** generates AIR code from the middle-end model - **[backend/air](./backend/air)** generates AIR code from the middle-end model
- **[backend/ts](./backend/ts)** - generates AIR code and Typescript wrappers for use with Fluence JS SDK - **[backend/ts](./backend/ts)** - generates AIR code and Typescript wrappers for use with Fluence JS SDK
- **[cli](./cli)** - CLI interface - **[cli](./cli)** - CLI interface

View File

@ -1,17 +1,17 @@
package aqua.backend.air package aqua.backend.air
import aqua.backend.{Backend, Compiled} import aqua.backend.{Backend, Generated}
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.implicits.toShow import cats.implicits.toShow
object AirBackend extends Backend { object AirBackend extends Backend {
val ext = ".air" val ext = ".air"
override def generate(context: AquaContext, bc: BodyConfig): Seq[Compiled] = { override def generate(context: AquaContext, genConf: GenerationConfig): Seq[Generated] = {
context.funcs.values.toList.map(fc => context.funcs.values.toList.map(fc =>
Compiled("." + fc.funcName + ext, FuncAirGen(fc).generateAir(bc).show) Generated("." + fc.funcName + ext, FuncAirGen(fc).generateAir(genConf).show)
) )
} }
} }

View File

@ -1,14 +1,14 @@
package aqua.backend.air package aqua.backend.air
import aqua.model.func.FuncCallable import aqua.model.func.FuncCallable
import aqua.model.transform.{BodyConfig, Transform} import aqua.model.transform.{GenerationConfig, Transform}
case class FuncAirGen(func: FuncCallable) { case class FuncAirGen(func: FuncCallable) {
/** /**
* Generates AIR from the function body * Generates AIR from the function body
*/ */
def generateAir(conf: BodyConfig = BodyConfig()): Air = def generateAir(conf: GenerationConfig = GenerationConfig()): Air =
AirGen( AirGen(
Transform.forClient(func, conf) Transform.forClient(func, conf)
).generate ).generate

View File

@ -1,21 +1,29 @@
package aqua.backend.js package aqua.backend.js
import aqua.backend.{Backend, Compiled} import aqua.backend.{Backend, Generated}
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.data.Chain import cats.data.NonEmptyChain
object JavaScriptBackend extends Backend { object JavaScriptBackend extends Backend {
val ext = ".js" val ext = ".js"
override def generate(context: AquaContext, bc: BodyConfig): Seq[Compiled] = { override def generate(context: AquaContext, genConf: GenerationConfig): Seq[Generated] = {
val funcs = Chain.fromSeq(context.funcs.values.toSeq).map(JavaScriptFunc(_)) val funcs = NonEmptyChain.fromSeq(context.funcs.values.toSeq).map(_.map(JavaScriptFunc(_)))
Seq( funcs
Compiled( .map(fs =>
ext, Seq(
JavaScriptFile.Header + "\n\n" + funcs.map(_.generateJavascript(bc)).toList.mkString("\n\n") Generated(
ext,
JavaScriptFile.Header + "\n\n" + fs
.map(_.generateJavascript(genConf))
.toChain
.toList
.mkString("\n\n")
)
)
) )
) .getOrElse(Seq.empty)
} }
} }

View File

@ -1,7 +1,7 @@
package aqua.backend.js package aqua.backend.js
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.data.Chain import cats.data.Chain
case class JavaScriptFile(context: AquaContext) { case class JavaScriptFile(context: AquaContext) {
@ -9,7 +9,7 @@ case class JavaScriptFile(context: AquaContext) {
def funcs: Chain[JavaScriptFunc] = def funcs: Chain[JavaScriptFunc] =
Chain.fromSeq(context.funcs.values.toSeq).map(JavaScriptFunc(_)) Chain.fromSeq(context.funcs.values.toSeq).map(JavaScriptFunc(_))
def generateJS(conf: BodyConfig = BodyConfig()): String = def generateJS(conf: GenerationConfig = GenerationConfig()): String =
JavaScriptFile.Header + "\n\n" + funcs.map(_.generateJavascript(conf)).toList.mkString("\n\n") JavaScriptFile.Header + "\n\n" + funcs.map(_.generateJavascript(conf)).toList.mkString("\n\n")
} }

View File

@ -2,7 +2,7 @@ package aqua.backend.js
import aqua.backend.air.FuncAirGen import aqua.backend.air.FuncAirGen
import aqua.model.func.{ArgDef, FuncCallable} import aqua.model.func.{ArgDef, FuncCallable}
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import aqua.types._ import aqua.types._
import cats.syntax.show._ import cats.syntax.show._
@ -38,7 +38,7 @@ case class JavaScriptFunc(func: FuncCallable) {
|""".stripMargin |""".stripMargin
} }
def generateJavascript(conf: BodyConfig = BodyConfig()): String = { def generateJavascript(conf: GenerationConfig = GenerationConfig()): String = {
val tsAir = FuncAirGen(func).generateAir(conf) val tsAir = FuncAirGen(func).generateAir(conf)

View File

@ -1,18 +1,19 @@
package aqua.backend package aqua.backend
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
/** /**
* Compilation result * Compiler backend generates output based on the processed model
* @param suffix extension or another info that will be added to a resulted file
* @param content a code that is used as an output
*/
case class Compiled(suffix: String, content: String)
/**
* Describes how context can be finalized
*/ */
trait Backend { trait Backend {
def generate(context: AquaContext, bc: BodyConfig): Seq[Compiled]
/**
* Generate the result based on the given [[AquaContext]] and [[GenerationConfig]]
*
* @param context Source file context, processed, transformed
* @param genConf Generation configuration
* @return Zero or more [[Generated]] objects, based on arguments
*/
def generate(context: AquaContext, genConf: GenerationConfig): Seq[Generated]
} }

View File

@ -0,0 +1,9 @@
package aqua.backend
/**
* Compilation result
*
* @param suffix extension or another info that will be added to a resulted file
* @param content compiled code
*/
case class Generated(suffix: String, content: String)

View File

@ -1,21 +1,29 @@
package aqua.backend.ts package aqua.backend.ts
import aqua.backend.{Backend, Compiled} import aqua.backend.{Backend, Generated}
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.data.Chain import cats.data.NonEmptyChain
object TypeScriptBackend extends Backend { object TypeScriptBackend extends Backend {
val ext = ".ts" val ext = ".ts"
override def generate(context: AquaContext, bc: BodyConfig): Seq[Compiled] = { override def generate(context: AquaContext, genConf: GenerationConfig): Seq[Generated] = {
val funcs = Chain.fromSeq(context.funcs.values.toSeq).map(TypeScriptFunc(_)) val funcs = NonEmptyChain.fromSeq(context.funcs.values.toSeq).map(_.map(TypeScriptFunc(_)))
Seq( funcs
Compiled( .map(fs =>
ext, Seq(
TypeScriptFile.Header + "\n\n" + funcs.map(_.generateTypescript(bc)).toList.mkString("\n\n") Generated(
ext,
TypeScriptFile.Header + "\n\n" + fs
.map(_.generateTypescript(genConf))
.toChain
.toList
.mkString("\n\n")
)
)
) )
) .getOrElse(Seq.empty)
} }
} }

View File

@ -1,7 +1,7 @@
package aqua.backend.ts package aqua.backend.ts
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.data.Chain import cats.data.Chain
case class TypeScriptFile(context: AquaContext) { case class TypeScriptFile(context: AquaContext) {
@ -9,7 +9,7 @@ case class TypeScriptFile(context: AquaContext) {
def funcs: Chain[TypeScriptFunc] = def funcs: Chain[TypeScriptFunc] =
Chain.fromSeq(context.funcs.values.toSeq).map(TypeScriptFunc(_)) Chain.fromSeq(context.funcs.values.toSeq).map(TypeScriptFunc(_))
def generateTS(conf: BodyConfig = BodyConfig()): String = def generateTS(conf: GenerationConfig = GenerationConfig()): String =
TypeScriptFile.Header + "\n\n" + funcs.map(_.generateTypescript(conf)).toList.mkString("\n\n") TypeScriptFile.Header + "\n\n" + funcs.map(_.generateTypescript(conf)).toList.mkString("\n\n")
} }

View File

@ -2,7 +2,7 @@ package aqua.backend.ts
import aqua.backend.air.FuncAirGen import aqua.backend.air.FuncAirGen
import aqua.model.func.{ArgDef, FuncCallable} import aqua.model.func.{ArgDef, FuncCallable}
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import aqua.types._ import aqua.types._
import cats.syntax.show._ import cats.syntax.show._
@ -46,7 +46,7 @@ case class TypeScriptFunc(func: FuncCallable) {
|""".stripMargin |""".stripMargin
} }
def generateTypescript(conf: BodyConfig = BodyConfig()): String = { def generateTypescript(conf: GenerationConfig = GenerationConfig()): String = {
val tsAir = FuncAirGen(func).generateAir(conf) val tsAir = FuncAirGen(func).generateAir(conf)

View File

@ -28,7 +28,7 @@ val cats = "org.typelevel" %% "cats-core" % catsV
name := "aqua-hll" name := "aqua-hll"
val commons = Seq( val commons = Seq(
baseAquaVersion := "0.1.10", baseAquaVersion := "0.1.11",
version := baseAquaVersion.value + "-" + sys.env.getOrElse("BUILD_NUMBER", "SNAPSHOT"), version := baseAquaVersion.value + "-" + sys.env.getOrElse("BUILD_NUMBER", "SNAPSHOT"),
scalaVersion := dottyVersion, scalaVersion := dottyVersion,
libraryDependencies ++= Seq( libraryDependencies ++= Seq(

View File

@ -1,7 +1,7 @@
package aqua package aqua
import aqua.model.LiteralModel import aqua.model.LiteralModel
import aqua.model.transform.Constant import aqua.model.transform.GenerationConfig
import aqua.parser.expr.ConstantExpr import aqua.parser.expr.ConstantExpr
import aqua.parser.lift.LiftParser import aqua.parser.lift.LiftParser
import cats.data.Validated.{Invalid, Valid} import cats.data.Validated.{Invalid, Valid}
@ -95,7 +95,7 @@ object AppOps {
} }
.withDefault(List.empty) .withDefault(List.empty)
def constantOpts[F[_]: LiftParser: Comonad]: Opts[List[Constant]] = def constantOpts[F[_]: LiftParser: Comonad]: Opts[List[GenerationConfig.Const]] =
Opts Opts
.options[String]("const", "Constant that will be used in an aqua code", "c") .options[String]("const", "Constant that will be used in an aqua code", "c")
.mapValidated { strs => .mapValidated { strs =>
@ -108,8 +108,9 @@ object AppOps {
NonEmptyList NonEmptyList
.fromList(errors) .fromList(errors)
.fold( .fold(
Validated.validNel[String, List[Constant]](parsed.collect { case Right(v) => Validated.validNel[String, List[GenerationConfig.Const]](parsed.collect {
Constant(v._1.value, LiteralModel(v._2.value, v._2.ts)) case Right(v) =>
GenerationConfig.Const(v._1.value, LiteralModel(v._2.value, v._2.ts))
}) })
) { errors => ) { errors =>
Validated.invalid(errors.map(_.toString)) Validated.invalid(errors.map(_.toString))

View File

@ -4,8 +4,8 @@ import aqua.backend.Backend
import aqua.backend.air.AirBackend import aqua.backend.air.AirBackend
import aqua.backend.js.JavaScriptBackend import aqua.backend.js.JavaScriptBackend
import aqua.backend.ts.TypeScriptBackend import aqua.backend.ts.TypeScriptBackend
import aqua.compiler.{AquaCompiler, AquaIO} import aqua.files.AquaFilesIO
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import aqua.parser.lift.LiftParser.Implicits.idLiftParser import aqua.parser.lift.LiftParser.Implicits.idLiftParser
import cats.Id import cats.Id
import cats.data.Validated import cats.data.Validated
@ -18,17 +18,7 @@ import com.monovore.decline.effect.CommandIOApp
import fs2.io.file.Files import fs2.io.file.Files
import org.typelevel.log4cats.slf4j.Slf4jLogger import org.typelevel.log4cats.slf4j.Slf4jLogger
import org.typelevel.log4cats.{Logger, SelfAwareStructuredLogger} import org.typelevel.log4cats.{Logger, SelfAwareStructuredLogger}
import wvlet.log.LogFormatter.{appendStackTrace, highlightLog} import wvlet.log.{LogSupport, Logger => WLogger}
import wvlet.log.{LogFormatter, LogRecord, LogSupport, Logger => WLogger}
object CustomLogFormatter extends LogFormatter {
override def formatLog(r: LogRecord): String = {
val log =
s"[${highlightLog(r.level, r.level.name)}] ${highlightLog(r.level, r.getMessage)}"
appendStackTrace(log, r)
}
}
object AquaCli extends IOApp with LogSupport { object AquaCli extends IOApp with LogSupport {
import AppOps._ import AppOps._
@ -82,11 +72,11 @@ object AquaCli extends IOApp with LogSupport {
else if (toJs) JavaScriptTarget else if (toJs) JavaScriptTarget
else TypescriptTarget else TypescriptTarget
val bc = { val bc = {
val bc = BodyConfig(wrapWithXor = !noXor, constants = constants) val bc = GenerationConfig(wrapWithXor = !noXor, constants = constants)
bc.copy(relayVarName = bc.relayVarName.filterNot(_ => noRelay)) bc.copy(relayVarName = bc.relayVarName.filterNot(_ => noRelay))
} }
info(s"Aqua Compiler ${versionStr}") info(s"Aqua Compiler ${versionStr}")
AquaCompiler AquaPathCompiler
.compileFilesTo[F]( .compileFilesTo[F](
input, input,
imports, imports,
@ -96,10 +86,10 @@ object AquaCli extends IOApp with LogSupport {
) )
.map { .map {
case Validated.Invalid(errs) => case Validated.Invalid(errs) =>
errs.map(println) errs.map(System.out.println)
ExitCode.Error ExitCode.Error
case Validated.Valid(results) => case Validated.Valid(results) =>
results.map(println) results.map(info(_))
ExitCode.Success ExitCode.Success
} }
} }

View File

@ -1,7 +1,6 @@
package aqua.compiler package aqua
import aqua.compiler.io.AquaFileError import aqua.io.AquaFileError
import aqua.parser.lift.FileSpan
import cats.data.{Chain, EitherT, ValidatedNec} import cats.data.{Chain, EitherT, ValidatedNec}
import java.nio.file.Path import java.nio.file.Path
@ -10,7 +9,6 @@ trait AquaIO[F[_]] {
def readFile(file: Path): EitherT[F, AquaFileError, String] def readFile(file: Path): EitherT[F, AquaFileError, String]
def resolve( def resolve(
focus: FileSpan.Focus,
src: Path, src: Path,
imports: List[Path] imports: List[Path]
): EitherT[F, AquaFileError, Path] ): EitherT[F, AquaFileError, Path]

View File

@ -0,0 +1,39 @@
package aqua
import aqua.backend.Backend
import aqua.compiler.{AquaCompiler, AquaError}
import aqua.files.{AquaFileSources, FileModuleId}
import aqua.io._
import aqua.model.transform.GenerationConfig
import aqua.parser.lift.FileSpan
import cats.data._
import cats.syntax.functor._
import cats.syntax.show._
import cats.{Monad, Show}
import wvlet.log.LogSupport
import java.nio.file.Path
object AquaPathCompiler extends LogSupport {
def compileFilesTo[F[_]: AquaIO: Monad](
srcPath: Path,
imports: List[Path],
targetPath: Path,
backend: Backend,
bodyConfig: GenerationConfig
): F[ValidatedNec[String, Chain[String]]] = {
import ErrorRendering.showError
val sources = new AquaFileSources[F](srcPath, imports)
AquaCompiler
.compileTo[F, AquaFileError, FileModuleId, FileSpan.F, String](
sources,
(fmid, src) => FileSpan.fileSpanLiftParser(fmid.file.toString, src),
backend,
bodyConfig,
sources.write(targetPath)
)
.map(_.leftMap(_.map(_.show)))
}
}

View File

@ -0,0 +1,13 @@
package aqua
import wvlet.log.LogFormatter.{appendStackTrace, highlightLog}
import wvlet.log.{LogFormatter, LogRecord}
object CustomLogFormatter extends LogFormatter {
override def formatLog(r: LogRecord): String = {
val log =
s"[${highlightLog(r.level, r.level.name)}] ${highlightLog(r.level, r.getMessage)}"
appendStackTrace(log, r)
}
}

View File

@ -0,0 +1,72 @@
package aqua
import aqua.compiler._
import aqua.files.FileModuleId
import aqua.io.AquaFileError
import aqua.parser.lift.FileSpan
import aqua.parser.{BlockIndentError, FuncReturnError, LexerError}
import aqua.semantics.{RulesViolated, WrongAST}
import cats.Show
object ErrorRendering {
def showForConsole(span: FileSpan, message: String): String =
span
.focus(3)
.map(
_.toConsoleStr(
message,
Console.RED
)
)
.getOrElse(
"(offset is beyond the script, syntax errors) Error: " + Console.RED + message
.mkString(", ")
) + Console.RESET + "\n"
implicit val showError: Show[AquaError[FileModuleId, AquaFileError, FileSpan.F]] = Show.show {
case ParserErr(err) =>
err match {
case BlockIndentError(indent, message) => showForConsole(indent._1, message)
case FuncReturnError(point, message) => showForConsole(point._1, message)
case LexerError((span, e)) =>
span
.focus(3)
.map(spanFocus =>
spanFocus.toConsoleStr(
s"Syntax error, expected: ${e.expected.toList.mkString(", ")}",
Console.RED
)
)
.getOrElse(
"(offset is beyond the script, syntax errors) " + Console.RED + e.expected.toList
.mkString(", ")
) + Console.RESET + "\n"
}
case SourcesErr(err) =>
Console.RED + err.showForConsole + Console.RESET
case ResolveImportsErr(_, token, err) =>
val span = token.unit._1
showForConsole(span, s"Cannot resolve imports: ${err.showForConsole}")
case ImportErr(token) =>
val span = token.unit._1
showForConsole(span, s"Cannot resolve import")
case CycleError(modules) =>
s"Cycle loops detected in imports: ${modules.map(_.file.getFileName)}"
case CompileError(err) =>
err match {
case RulesViolated(token, message) =>
token.unit._1
.focus(2)
.map(_.toConsoleStr(message, Console.CYAN))
.getOrElse("(Dup error, but offset is beyond the script)") + "\n"
case WrongAST(ast) =>
s"Semantic error"
}
case OutputError(_, err) =>
Console.RED + err.showForConsole + Console.RESET
}
}

View File

@ -1,8 +1,8 @@
package aqua package aqua
import aqua.backend.ts.TypeScriptBackend import aqua.backend.ts.TypeScriptBackend
import aqua.compiler.{AquaCompiler, AquaIO} import aqua.files.AquaFilesIO
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import cats.data.Validated import cats.data.Validated
import cats.effect.{IO, IOApp, Sync} import cats.effect.{IO, IOApp, Sync}
import org.typelevel.log4cats.SelfAwareStructuredLogger import org.typelevel.log4cats.SelfAwareStructuredLogger
@ -18,19 +18,19 @@ object Test extends IOApp.Simple {
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO] implicit val aio: AquaIO[IO] = new AquaFilesIO[IO]
override def run: IO[Unit] = override def run: IO[Unit] =
AquaCompiler AquaPathCompiler
.compileFilesTo[IO]( .compileFilesTo[IO](
Paths.get("./aqua-src"), Paths.get("./aqua-src"),
List(Paths.get("./aqua")), List(Paths.get("./aqua")),
Paths.get("./target"), Paths.get("./target"),
TypeScriptBackend, TypeScriptBackend,
BodyConfig() GenerationConfig()
) )
.map { .map {
case Validated.Invalid(errs) => case Validated.Invalid(errs) =>
errs.map(println) errs.map(System.err.println)
case Validated.Valid(_) => case Validated.Valid(res) =>
res.map(println)
} }
} }

View File

@ -0,0 +1,124 @@
package aqua.files
import aqua.AquaIO
import aqua.compiler.{AquaCompiled, AquaSources}
import aqua.io.{AquaFileError, FileSystemError, ListAquaErrors}
import cats.Monad
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.implicits.catsSyntaxApplicativeId
import cats.syntax.either._
import cats.syntax.flatMap._
import cats.syntax.functor._
import cats.syntax.traverse._
import java.nio.file.{Path, Paths}
import scala.util.Try
class AquaFileSources[F[_]: AquaIO: Monad](sourcesPath: Path, importFrom: List[Path])
extends AquaSources[F, AquaFileError, FileModuleId] {
private val filesIO = implicitly[AquaIO[F]]
override def sources: F[ValidatedNec[AquaFileError, Chain[(FileModuleId, String)]]] =
filesIO.listAqua(sourcesPath).flatMap {
case Validated.Valid(files) =>
files
.map(f =>
filesIO
.readFile(f)
.value
.map[ValidatedNec[AquaFileError, Chain[(FileModuleId, String)]]] {
case Left(err) => Validated.invalidNec(err)
case Right(content) => Validated.validNec(Chain.one(FileModuleId(f) -> content))
}
)
.traverse(identity)
.map(
_.foldLeft[ValidatedNec[AquaFileError, Chain[(FileModuleId, String)]]](
Validated.validNec(Chain.nil)
)(_ combine _)
)
case Validated.Invalid(e) =>
Validated
.invalidNec[AquaFileError, Chain[(FileModuleId, String)]](ListAquaErrors(e))
.pure[F]
}
// Resolve an import that was written in a 'from' file
// Try to find it in a list of given imports or near 'from' file
override def resolveImport(
from: FileModuleId,
imp: String
): F[ValidatedNec[AquaFileError, FileModuleId]] = {
Validated.fromEither(Try(Paths.get(imp)).toEither.leftMap(FileSystemError)) match {
case Validated.Valid(importP) =>
filesIO
.resolve(importP, from.file.getParent +: importFrom)
.bimap(NonEmptyChain.one, FileModuleId(_))
.value
.map(Validated.fromEither)
case Validated.Invalid(err) => Validated.invalidNec[AquaFileError, FileModuleId](err).pure[F]
}
}
override def load(file: FileModuleId): F[ValidatedNec[AquaFileError, String]] =
filesIO.readFile(file.file).leftMap(NonEmptyChain.one).value.map(Validated.fromEither)
/**
* @param srcFile aqua source
* @param targetPath a main path where all output files will be written
* @param suffix `.aqua` will be replaced with this suffix
* @return
*/
def resolveTargetPath(
srcFile: Path,
targetPath: Path,
suffix: String
): Validated[Throwable, Path] =
Validated.catchNonFatal {
val srcDir = if (sourcesPath.toFile.isDirectory) sourcesPath else sourcesPath.getParent
val srcFilePath = srcDir.toAbsolutePath
.normalize()
.relativize(srcFile.toAbsolutePath.normalize())
val targetDir =
targetPath.toAbsolutePath
.normalize()
.resolve(
srcFilePath
)
targetDir.getParent.resolve(srcFile.getFileName.toString.stripSuffix(".aqua") + suffix)
}
def write(
targetPath: Path
)(ac: AquaCompiled[FileModuleId]): F[Seq[Validated[AquaFileError, String]]] =
if (ac.compiled.isEmpty)
Seq(
Validated.valid[AquaFileError, String](
s"Source ${ac.sourceId.file}: compilation OK (nothing to emit)"
)
).pure[F]
else
ac.compiled.map { compiled =>
resolveTargetPath(
ac.sourceId.file,
targetPath,
compiled.suffix
).leftMap(FileSystemError)
.map { target =>
filesIO
.writeFile(
target,
compiled.content
)
.as(s"Result $target: compilation OK (${ac.compiled.size} functions)")
.value
.map(Validated.fromEither)
}
// TODO: we use both EitherT and F[Validated] to handle errors, that's why so weird
.traverse(identity)
}.traverse(identity)
.map(_.map(_.andThen(identity)))
}

View File

@ -1,24 +1,18 @@
package aqua package aqua.files
import aqua.compiler.AquaIO import aqua.AquaIO
import aqua.compiler.io.{ import aqua.io._
AquaFileError,
EmptyFileError,
FileNotFound,
FileSystemError,
FileWriteError
}
import aqua.parser.lift.FileSpan
import cats.data.Validated.{Invalid, Valid} import cats.data.Validated.{Invalid, Valid}
import cats.data.{Chain, EitherT, NonEmptyChain, Validated, ValidatedNec} import cats.data._
import cats.syntax.functor._
import cats.syntax.either._
import cats.effect.kernel.Concurrent import cats.effect.kernel.Concurrent
import cats.syntax.applicative._
import cats.syntax.apply._
import cats.syntax.either._
import cats.syntax.flatMap._
import cats.syntax.functor._
import cats.syntax.applicativeError._
import fs2.io.file.Files import fs2.io.file.Files
import fs2.text import fs2.text
import cats.syntax.applicative._
import cats.syntax.flatMap._
import cats.syntax.apply._
import java.nio.file.Path import java.nio.file.Path
import scala.util.Try import scala.util.Try
@ -43,6 +37,10 @@ class AquaFilesIO[F[_]: Files: Concurrent] extends AquaIO[F] {
) )
) )
/**
* Find the first file that exists in the given list of paths
* If there is no such file - error
*/
private def findFirstF( private def findFirstF(
in: List[Path], in: List[Path],
notFound: EitherT[F, AquaFileError, Path] notFound: EitherT[F, AquaFileError, Path]
@ -67,29 +65,29 @@ class AquaFilesIO[F[_]: Files: Concurrent] extends AquaIO[F] {
* Checks if a file exists in the list of possible paths * Checks if a file exists in the list of possible paths
*/ */
def resolve( def resolve(
focus: FileSpan.Focus,
src: Path, src: Path,
imports: List[Path] imports: List[Path]
): EitherT[F, AquaFileError, Path] = ): EitherT[F, AquaFileError, Path] =
findFirstF( findFirstF(
imports imports
.map(_.resolve(src)), .map(_.resolve(src)),
EitherT.leftT(FileNotFound(focus, src, imports)) EitherT.leftT(FileNotFound(src, imports))
) )
override def listAqua(folder: Path): F[ValidatedNec[AquaFileError, Chain[Path]]] = override def listAqua(folder: Path): F[ValidatedNec[AquaFileError, Chain[Path]]] =
Validated Validated
.fromTry( .fromEither(
Try { Try {
val f = folder.toFile val f = folder.toFile
if (f.isDirectory) { if (!f.exists()) {
f.listFiles().toList Left(FileNotFound(folder, Nil))
} else if (f.isDirectory) {
Right(f.listFiles().toList)
} else { } else {
f :: Nil Right(f :: Nil)
} }
} }.toEither.leftMap[AquaFileError](FileSystemError).flatMap(identity)
) )
.leftMap[AquaFileError](FileSystemError)
.leftMap(NonEmptyChain.one) .leftMap(NonEmptyChain.one)
.pure[F] .pure[F]
.flatMap { .flatMap {
@ -113,27 +111,27 @@ class AquaFilesIO[F[_]: Files: Concurrent] extends AquaIO[F] {
Validated.invalid[NonEmptyChain[AquaFileError], Chain[Path]](errs).pure[F] Validated.invalid[NonEmptyChain[AquaFileError], Chain[Path]](errs).pure[F]
} }
private def deleteIfExists(file: Path): EitherT[F, AquaFileError, Boolean] =
Files[F].deleteIfExists(file).attemptT.leftMap(FileSystemError)
private def createDirectories(path: Path): EitherT[F, AquaFileError, Path] =
Files[F].createDirectories(path).attemptT.leftMap(FileSystemError)
// Writes to a file, creates directories if they do not exist
override def writeFile(file: Path, content: String): EitherT[F, AquaFileError, Unit] = override def writeFile(file: Path, content: String): EitherT[F, AquaFileError, Unit] =
EitherT deleteIfExists(file) >> createDirectories(file.getParent) >>
.right[AquaFileError](Files[F].deleteIfExists(file)) EitherT(
.flatMap(_ => fs2.Stream
EitherT[F, AquaFileError, Unit]( .emit(content)
fs2.Stream .through(text.utf8Encode)
.emit( .through(Files[F].writeAll(file))
content .attempt
) .compile
.through(text.utf8Encode) .last
.through(Files[F].writeAll(file)) .map(_.getOrElse(Right()))
.attempt
.map { e =>
e.left
.map(t => FileWriteError(file, t))
}
.compile
.drain
.map(_ => Right(()))
)
) )
.leftMap(FileWriteError(file, _))
} }
object AquaFilesIO { object AquaFilesIO {

View File

@ -0,0 +1,11 @@
package aqua.files
import java.nio.file.Path
case class FileModuleId private (file: Path)
object FileModuleId {
def apply(file: Path): FileModuleId =
new FileModuleId(file.toAbsolutePath.normalize())
}

View File

@ -1,7 +1,5 @@
package aqua.compiler.io package aqua.io
import aqua.compiler.AquaError
import aqua.parser.lift.FileSpan
import cats.data.NonEmptyChain import cats.data.NonEmptyChain
import java.nio.file.Path import java.nio.file.Path
@ -12,13 +10,19 @@ sealed trait AquaFileError {
override def toString: String = showForConsole override def toString: String = showForConsole
} }
case class FileNotFound(focus: FileSpan.Focus, name: Path, imports: Seq[Path]) case class ListAquaErrors(errors: NonEmptyChain[AquaFileError]) extends AquaFileError {
extends AquaFileError {
override def showForConsole: String = focus.toConsoleStr( override def showForConsole: String =
s"File not found at $name, looking in ${imports.mkString(", ")}", s"Cannot read '*.aqua' files:\n" + errors.map(_.showForConsole)
Console.YELLOW }
)
case class FileNotFound(name: Path, imports: Seq[Path]) extends AquaFileError {
override def showForConsole: String =
if (imports.nonEmpty)
s"File '$name' not found, looking in ${imports.mkString(", ")}"
else
s"File '$name' not found"
} }
case class EmptyFileError(path: Path) extends AquaFileError { case class EmptyFileError(path: Path) extends AquaFileError {
@ -26,20 +30,13 @@ case class EmptyFileError(path: Path) extends AquaFileError {
} }
case class FileSystemError(err: Throwable) extends Exception(err) with AquaFileError { case class FileSystemError(err: Throwable) extends Exception(err) with AquaFileError {
override def showForConsole: String = s"File system error: ${err.getMessage}" override def showForConsole: String = s"File system error: $err"
} }
case class FileWriteError(file: Path, err: Throwable) extends Exception(err) with AquaFileError { case class FileWriteError(file: Path, err: Throwable) extends Exception(err) with AquaFileError {
override def showForConsole: String = s"Cannot write a file $file: ${err.getMessage}" override def showForConsole: String = s"Cannot write a file $file: $err"
} }
case class Unresolvable(msg: String) extends AquaFileError { case class Unresolvable(msg: String) extends AquaFileError {
override def showForConsole: String = s"Unresolvable: $msg" override def showForConsole: String = s"Unresolvable: $msg"
} }
// TODO there should be no AquaErrors, as they does not fit
case class AquaScriptErrors(errors: NonEmptyChain[AquaError]) extends AquaFileError {
override def showForConsole: String =
errors.map(_.showForConsole).toChain.toList.mkString("\n")
}

View File

@ -0,0 +1,157 @@
import aqua.AquaIO
import aqua.backend.Generated
import aqua.compiler.AquaCompiled
import aqua.files.{AquaFileSources, AquaFilesIO, FileModuleId}
import cats.data.Chain
import cats.effect.IO
import cats.effect.unsafe.implicits.global
import fs2.io.file.Files
import fs2.text
import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers
import java.nio.file.Paths
class SourcesSpec extends AnyFlatSpec with Matchers {
implicit val aquaIO: AquaIO[IO] = AquaFilesIO.summon[IO]
"AquaFileSources" should "generate correct fileId with imports" in {
val path = Paths.get("cli/src/test/test-dir/path-test")
val importPath = path.resolve("imports")
val sourceGen = new AquaFileSources[IO](path, importPath :: Nil)
val result = sourceGen.sources.unsafeRunSync()
result.isValid shouldBe true
val listResult = result
.getOrElse(Chain.empty)
.toList
.map { case (fid, s) =>
(fid.file.toString.split("/").last, s)
}
.sortBy(_._1) // sort cause different systems have different order of file reading
val (id, importFile) = listResult(1)
id shouldBe "index.aqua"
importFile.nonEmpty shouldBe true
val (importNearId, importFileNear) = listResult.head
importNearId shouldBe "importNear.aqua"
importFileNear.nonEmpty shouldBe true
}
"AquaFileSources" should "throw an error if a source file doesn't exist" in {
val path = Paths.get("some/random/path")
val sourceGen = new AquaFileSources[IO](path, Nil)
val result = sourceGen.sources.unsafeRunSync()
result.isInvalid shouldBe true
}
"AquaFileSources" should "throw an error if there is no import that is indicated in a source" in {
val path = Paths.get("cli/src/test/test-dir")
val importPath = path.resolve("random/import/path")
val sourceGen = new AquaFileSources[IO](path, importPath :: Nil)
val result =
sourceGen.resolveImport(FileModuleId(path.resolve("no-file.aqua")), "no/file").unsafeRunSync()
result.isInvalid shouldBe true
}
"AquaFileSources" should "find correct imports" in {
val srcPath = Paths.get("cli/src/test/test-dir/index.aqua")
val importPath = srcPath.resolve("imports")
val sourceGen = new AquaFileSources[IO](srcPath, importPath :: Nil)
// should be found in importPath
val result =
sourceGen
.resolveImport(FileModuleId(srcPath), "imports/import.aqua")
.unsafeRunSync()
result.isValid shouldBe true
result.getOrElse(FileModuleId(Paths.get("/some/random"))).file.toFile.exists() shouldBe true
// should be found near src file
val result2 =
sourceGen
.resolveImport(FileModuleId(srcPath), "importNear.aqua")
.unsafeRunSync()
result2.isValid shouldBe true
result2.getOrElse(FileModuleId(Paths.get("/some/random"))).file.toFile.exists() shouldBe true
// near src file but in another directory
val sourceGen2 = new AquaFileSources[IO](srcPath, Nil)
val result3 =
sourceGen2
.resolveImport(FileModuleId(srcPath), "imports/import.aqua")
.unsafeRunSync()
result3.isValid shouldBe true
result3.getOrElse(FileModuleId(Paths.get("/some/random"))).file.toFile.exists() shouldBe true
}
"AquaFileSources" should "resolve correct path for target" in {
val path = Paths.get("cli/src/test/test-dir")
val filePath = path.resolve("some-dir/file.aqua")
val targetPath = Paths.get("/target/dir/")
val sourceGen = new AquaFileSources[IO](path, Nil)
val suffix = "_custom.super"
val resolved = sourceGen.resolveTargetPath(filePath, targetPath, suffix)
resolved.isValid shouldBe true
val targetFilePath = resolved.toOption.get
targetFilePath.toString shouldBe "/target/dir/some-dir/file_custom.super"
}
"AquaFileSources" should "write correct file with correct path" in {
val path = Paths.get("cli/src/test/test-dir")
val filePath = path.resolve("imports/import.aqua")
val targetPath = path.resolve("target/")
// clean up
val resultPath = Paths.get("cli/src/test/test-dir/target/imports/import_hey.custom")
Files[IO].deleteIfExists(resultPath).unsafeRunSync()
val sourceGen = new AquaFileSources[IO](path, Nil)
val content = "some random content"
val compiled = AquaCompiled[FileModuleId](
FileModuleId(filePath),
Seq(Generated("_hey.custom", content))
)
val resolved = sourceGen.write(targetPath)(compiled).unsafeRunSync()
resolved.size shouldBe 1
resolved.head.isValid shouldBe true
Files[IO].exists(resultPath).unsafeRunSync() shouldBe true
val resultText = Files[IO]
.readAll(resultPath, 1000)
.fold(
Vector
.empty[Byte]
)((acc, b) => acc :+ b)
.flatMap(fs2.Stream.emits)
.through(text.utf8Decode)
.attempt
.compile
.last
.unsafeRunSync()
.get
.right
.get
resultText shouldBe content
Files[IO].deleteIfExists(resultPath).unsafeRunSync()
}
}

View File

@ -1,8 +1,8 @@
import aqua.AquaPathCompiler
import aqua.backend.air.AirBackend import aqua.backend.air.AirBackend
import aqua.backend.js.JavaScriptBackend import aqua.backend.js.JavaScriptBackend
import aqua.backend.ts.TypeScriptBackend import aqua.backend.ts.TypeScriptBackend
import aqua.compiler.AquaCompiler import aqua.model.transform.GenerationConfig
import aqua.model.transform.BodyConfig
import cats.effect.IO import cats.effect.IO
import cats.effect.unsafe.implicits.global import cats.effect.unsafe.implicits.global
import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.flatspec.AnyFlatSpec
@ -17,10 +17,10 @@ class WriteFileSpec extends AnyFlatSpec with Matchers {
val targetJs = Files.createTempDirectory("js") val targetJs = Files.createTempDirectory("js")
val targetAir = Files.createTempDirectory("air") val targetAir = Files.createTempDirectory("air")
import aqua.AquaFilesIO.summon import aqua.files.AquaFilesIO.summon
val bc = BodyConfig() val bc = GenerationConfig()
AquaCompiler AquaPathCompiler
.compileFilesTo[IO](src, List.empty, targetTs, TypeScriptBackend, bc) .compileFilesTo[IO](src, List.empty, targetTs, TypeScriptBackend, bc)
.unsafeRunSync() .unsafeRunSync()
.leftMap { err => .leftMap { err =>
@ -32,7 +32,7 @@ class WriteFileSpec extends AnyFlatSpec with Matchers {
targetTsFile.toFile.exists() should be(true) targetTsFile.toFile.exists() should be(true)
Files.deleteIfExists(targetTsFile) Files.deleteIfExists(targetTsFile)
AquaCompiler AquaPathCompiler
.compileFilesTo[IO](src, List.empty, targetJs, JavaScriptBackend, bc) .compileFilesTo[IO](src, List.empty, targetJs, JavaScriptBackend, bc)
.unsafeRunSync() .unsafeRunSync()
.leftMap { err => .leftMap { err =>
@ -44,7 +44,7 @@ class WriteFileSpec extends AnyFlatSpec with Matchers {
targetJsFile.toFile.exists() should be(true) targetJsFile.toFile.exists() should be(true)
Files.deleteIfExists(targetJsFile) Files.deleteIfExists(targetJsFile)
AquaCompiler AquaPathCompiler
.compileFilesTo[IO](src, List.empty, targetAir, AirBackend, bc) .compileFilesTo[IO](src, List.empty, targetAir, AirBackend, bc)
.unsafeRunSync() .unsafeRunSync()
.leftMap { err => .leftMap { err =>

View File

@ -0,0 +1,4 @@
import "random/import/import.aqua"
func indexCall():
Println.print("it is true")

View File

@ -0,0 +1,5 @@
service Println("println-service-id"):
print: string -> ()
func print(str: string):
Println.print(str)

View File

@ -0,0 +1,5 @@
service Println("println-service-id"):
print: string -> ()
func print(str: string):
Println.print(str)

View File

@ -0,0 +1,4 @@
import "imports/import.aqua"
func indexCall():
Println.print("it is true")

View File

@ -0,0 +1,5 @@
service Println("println-service-id"):
print: string -> ()
func print(str: string):
Println.print(str)

View File

@ -0,0 +1,4 @@
import "imports/import.aqua"
func indexCall():
Println.print("it is true")

View File

@ -0,0 +1,5 @@
package aqua.compiler
import aqua.backend.Generated
case class AquaCompiled[I](sourceId: I, compiled: Seq[Generated])

View File

@ -1,167 +1,90 @@
package aqua.compiler package aqua.compiler
import aqua.backend.Backend import aqua.backend.Backend
import aqua.compiler.io._
import aqua.linker.Linker import aqua.linker.Linker
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.model.transform.BodyConfig import aqua.model.transform.GenerationConfig
import aqua.parser.lift.FileSpan import aqua.parser.lift.LiftParser
import aqua.semantics.{RulesViolated, SemanticError, Semantics, WrongAST} import aqua.semantics.Semantics
import cats.data._ import cats.data.Validated.{validNec, Invalid, Valid}
import cats.kernel.Monoid import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.syntax.applicative._
import cats.syntax.flatMap._ import cats.syntax.flatMap._
import cats.syntax.functor._ import cats.syntax.functor._
import cats.{Applicative, Monad} import cats.syntax.traverse._
import wvlet.log.LogSupport import cats.{Comonad, Monad}
import java.nio.file.Path object AquaCompiler {
object AquaCompiler extends LogSupport { def compile[F[_]: Monad, E, I, S[_]: Comonad](
sources: AquaSources[F, E, I],
private def gatherPreparedFiles( liftI: (I, String) => LiftParser[S],
srcPath: Path,
targetPath: Path,
files: Map[FileModuleId, ValidatedNec[SemanticError[FileSpan.F], AquaContext]]
): ValidatedNec[String, Chain[Prepared]] = {
val (errs, _, preps) = files.toSeq.foldLeft[(Chain[String], Set[String], Chain[Prepared])](
(Chain.empty, Set.empty, Chain.empty)
) { case ((errs, errsSet, preps), (modId, proc)) =>
proc.fold(
es => {
val newErrs = showProcErrors(es.toChain).filterNot(errsSet.contains)
(errs ++ newErrs, errsSet ++ newErrs.iterator, preps)
},
c => {
Prepared(modId.file, srcPath, targetPath, c) match {
case Validated.Valid(p)
(errs, errsSet, preps :+ p)
case Validated.Invalid(err)
(errs :+ err.getMessage, errsSet, preps)
}
}
)
}
NonEmptyChain
.fromChain(errs)
.fold(Validated.validNec[String, Chain[Prepared]](preps))(Validated.invalid)
}
/**
* Create a structure that will be used to create output by a backend
*/
def prepareFiles[F[_]: AquaIO: Monad](
srcPath: Path,
imports: List[Path],
targetPath: Path
)(implicit aqum: Monoid[AquaContext]): F[ValidatedNec[String, Chain[Prepared]]] =
AquaFiles
.readAndResolve[F, ValidatedNec[SemanticError[FileSpan.F], AquaContext]](
srcPath,
imports,
ast => context => context.andThen(ctx => Semantics.process(ast, ctx))
)
.value
.map {
case Left(fileErrors) =>
Validated.invalid(fileErrors.map(_.showForConsole))
case Right(modules) =>
Linker[FileModuleId, AquaFileError, ValidatedNec[SemanticError[FileSpan.F], AquaContext]](
modules,
ids => Unresolvable(ids.map(_.id.file.toString).mkString(" -> "))
) match {
case Validated.Valid(files)
gatherPreparedFiles(srcPath, targetPath, files)
case Validated.Invalid(errs)
Validated.invalid(
errs
.map(_.showForConsole)
)
}
}
def showProcErrors(
errors: Chain[SemanticError[FileSpan.F]]
): Chain[String] =
errors.map {
case RulesViolated(token, hint) =>
token.unit._1
.focus(2)
.map(_.toConsoleStr(hint, Console.CYAN))
.getOrElse("(Dup error, but offset is beyond the script)") + "\n"
case WrongAST(_) =>
"Semantic error"
}
private def gatherResults[F[_]: Monad](
results: List[EitherT[F, String, Unit]]
): F[ValidatedNec[String, Chain[String]]] = {
results
.foldLeft(
EitherT.rightT[F, NonEmptyChain[String]](Chain.empty[String])
) { case (accET, writeET) =>
EitherT(for {
acc <- accET.value
writeResult <- writeET.value
} yield (acc, writeResult) match {
case (Left(errs), Left(err)) => Left(errs :+ err)
case (Right(res), Right(_)) => Right(res)
case (Left(errs), _) => Left(errs)
case (_, Left(err)) => Left(NonEmptyChain.of(err))
})
}
.value
.map(Validated.fromEither)
}
def compileFilesTo[F[_]: AquaIO: Monad](
srcPath: Path,
imports: List[Path],
targetPath: Path,
backend: Backend, backend: Backend,
bodyConfig: BodyConfig config: GenerationConfig
): F[ValidatedNec[String, Chain[String]]] = { ): F[ValidatedNec[AquaError[I, E, S], Chain[AquaCompiled[I]]]] = {
import bodyConfig.aquaContextMonoid import config.aquaContextMonoid
prepareFiles(srcPath, imports, targetPath) type Err = AquaError[I, E, S]
.map(_.map(_.filter { p => new AquaParser[F, E, I, S](sources, liftI)
val hasOutput = p.hasOutput .resolve[ValidatedNec[Err, AquaContext]](ast =>
if (!hasOutput) info(s"Source ${p.srcFile}: compilation OK (nothing to emit)") context =>
hasOutput context.andThen(ctx => Semantics.process(ast, ctx).leftMap(_.map[Err](CompileError(_))))
})) )
.flatMap[ValidatedNec[String, Chain[String]]] { .map {
case Validated.Invalid(e) => case Valid(modules) =>
Applicative[F].pure(Validated.invalid(e)) Linker.link[I, AquaError[I, E, S], ValidatedNec[Err, AquaContext]](
case Validated.Valid(preps) => modules,
val results = preps.toList cycle => CycleError[I, E, S](cycle.map(_.id))
.flatMap(p => ) match {
backend.generate(p.context, bodyConfig).map { compiled => case Valid(filesWithContext) =>
val targetPath = p.targetPath( filesWithContext
p.srcFile.getFileName.toString.stripSuffix(".aqua") + compiled.suffix .foldLeft[ValidatedNec[Err, Chain[AquaProcessed[I]]]](
validNec(Chain.nil)
) {
case (acc, (i, Valid(context))) =>
acc combine validNec(Chain.one(AquaProcessed(i, context)))
case (acc, (_, Invalid(errs))) =>
acc combine Invalid(errs)
}
.map(
_.map { ap =>
val compiled = backend.generate(ap.context, config)
AquaCompiled(ap.id, compiled)
}
) )
case i @ Invalid(_) => i
targetPath.fold( }
t => EitherT.leftT[F, Unit](t.getMessage), case i @ Invalid(_) => i
tp =>
AquaIO[F]
.writeFile(tp, compiled.content)
.flatTap { _ =>
EitherT.pure(
Validated.catchNonFatal(
info(
s"Result ${tp.toAbsolutePath}: compilation OK (${p.context.funcs.size} functions)"
)
)
)
}
.leftMap(_.showForConsole)
)
}
)
gatherResults(results)
} }
} }
def compileTo[F[_]: Monad, E, I, S[_]: Comonad, T](
sources: AquaSources[F, E, I],
liftI: (I, String) => LiftParser[S],
backend: Backend,
config: GenerationConfig,
write: AquaCompiled[I] => F[Seq[Validated[E, T]]]
): F[ValidatedNec[AquaError[I, E, S], Chain[T]]] =
compile[F, E, I, S](sources, liftI, backend, config).flatMap {
case Valid(compiled) =>
compiled.map { ac =>
write(ac).map(
_.map(
_.bimap[NonEmptyChain[AquaError[I, E, S]], Chain[T]](
e => NonEmptyChain.one(OutputError(ac, e)),
Chain.one
)
)
)
}.toList
.traverse(identity)
.map(
_.flatten
.foldLeft[ValidatedNec[AquaError[I, E, S], Chain[T]]](validNec(Chain.nil))(
_ combine _
)
)
case Validated.Invalid(errs) =>
Validated.invalid[NonEmptyChain[AquaError[I, E, S]], Chain[T]](errs).pure[F]
}
} }

View File

@ -1,43 +1,18 @@
package aqua.compiler package aqua.compiler
import aqua.parser.lift.FileSpan import aqua.parser.ParserError
import cats.data.NonEmptyList import aqua.parser.lexer.Token
import cats.parse.Parser.Expectation import aqua.semantics.SemanticError
sealed trait AquaError { trait AquaError[I, E, S[_]]
def showForConsole: String case class SourcesErr[I, E, S[_]](err: E) extends AquaError[I, E, S]
} case class ParserErr[I, E, S[_]](err: ParserError[S]) extends AquaError[I, E, S]
case class CustomSyntaxError(span: FileSpan, message: String) extends AquaError { case class ResolveImportsErr[I, E, S[_]](fromFile: I, token: Token[S], err: E)
extends AquaError[I, E, S]
case class ImportErr[I, E, S[_]](token: Token[S]) extends AquaError[I, E, S]
override def showForConsole: String = case class CycleError[I, E, S[_]](modules: List[I]) extends AquaError[I, E, S]
span
.focus(3)
.map(
_.toConsoleStr(
message,
Console.RED
)
)
.getOrElse(
"(offset is beyond the script, syntax errors) Error: " + Console.RED + message
.mkString(", ")
) + Console.RESET + "\n"
}
case class SyntaxError(span: FileSpan, expectations: NonEmptyList[Expectation]) extends AquaError { case class CompileError[I, E, S[_]](err: SemanticError[S]) extends AquaError[I, E, S]
case class OutputError[I, E, S[_]](compiled: AquaCompiled[I], err: E) extends AquaError[I, E, S]
override def showForConsole: String =
span
.focus(3)
.map(spanFocus =>
spanFocus.toConsoleStr(
s"Syntax error, expected: ${expectations.toList.mkString(", ")}",
Console.RED
)
)
.getOrElse(
"(offset is beyond the script, syntax errors) " + Console.RED + expectations.toList
.mkString(", ")
) + Console.RESET + "\n"
}

View File

@ -0,0 +1,111 @@
package aqua.compiler
import aqua.linker.{AquaModule, Modules}
import aqua.parser.Ast
import aqua.parser.head.ImportExpr
import aqua.parser.lift.LiftParser
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.syntax.applicative._
import cats.syntax.flatMap._
import cats.syntax.functor._
import cats.syntax.traverse._
import cats.{Comonad, Monad}
// TODO: add tests
class AquaParser[F[_]: Monad, E, I, S[_]: Comonad](
sources: AquaSources[F, E, I],
liftI: (I, String) => LiftParser[S]
) {
type Body = Ast[S]
type Err = AquaError[I, E, S]
// Parse all the source files
def parseSources: F[ValidatedNec[Err, Chain[(I, Body)]]] =
sources.sources
.map(
_.leftMap(_.map[Err](SourcesErr(_))).andThen(_.map { case (i, s) =>
implicit val lift: LiftParser[S] = liftI(i, s)
Ast.fromString[S](s).bimap(_.map[Err](ParserErr(_)), ast => Chain.one(i -> ast))
}.foldLeft(Validated.validNec[Err, Chain[(I, Body)]](Chain.nil))(_ combine _))
)
// Resolve imports (not parse, just resolve) of the given file
def resolveImports(id: I, ast: Ast[S]): F[ValidatedNec[Err, Map[I, Err]]] =
ast.head.tailForced
.map(_.head)
.collect { case ImportExpr(filename) =>
sources
.resolveImport(id, filename.value.drop(1).dropRight(1))
.map(
_.bimap(
_.map(ResolveImportsErr(id, filename, _)),
importId => Chain.one[(I, Err)](importId -> ImportErr(filename))
)
)
}
.traverse(identity)
.map(
_.foldLeft(Validated.validNec[Err, Chain[(I, Err)]](Chain.nil))(_ combine _)
.map(_.toList.toMap)
)
// Parse sources, convert to modules
def sourceModules: F[ValidatedNec[Err, Modules[I, Err, Body]]] =
parseSources.flatMap {
case Validated.Valid(srcs) =>
srcs.traverse { case (id, ast) =>
resolveImports(id, ast).map(_.map(AquaModule(id, _, ast)).map(Chain.one))
}.map(
_.foldLeft(Validated.validNec[Err, Chain[AquaModule[I, Err, Body]]](Chain.empty))(
_ combine _
)
)
case Validated.Invalid(errs) =>
Validated.invalid[NonEmptyChain[Err], Chain[AquaModule[I, Err, Body]]](errs).pure[F]
}.map(_.map(_.foldLeft(Modules[I, Err, Body]())(_.add(_, export = true))))
def loadModule(imp: I): F[ValidatedNec[Err, AquaModule[I, Err, Ast[S]]]] =
sources
.load(imp)
.map(_.leftMap(_.map[Err](SourcesErr(_))).andThen { src =>
implicit val lift: LiftParser[S] = liftI(imp, src)
Ast.fromString[S](src).leftMap(_.map[Err](ParserErr(_)))
})
.flatMap {
case Validated.Valid(ast) =>
resolveImports(imp, ast).map(_.map(AquaModule(imp, _, ast)))
case Validated.Invalid(errs) =>
Validated.invalid[NonEmptyChain[Err], AquaModule[I, Err, Ast[S]]](errs).pure[F]
}
def resolveModules(
modules: Modules[I, Err, Body]
): F[ValidatedNec[Err, Modules[I, Err, Ast[S]]]] =
modules.dependsOn.map { case (moduleId, unresolvedErrors) =>
loadModule(moduleId).map(_.leftMap(_ ++ unresolvedErrors))
}.toList
.traverse(identity)
.map(_.foldLeft[ValidatedNec[Err, Modules[I, Err, Ast[S]]]](Validated.validNec(modules)) {
case (mods, m) =>
mods.andThen(ms => m.map(ms.add(_)))
})
.flatMap {
case Validated.Valid(ms) if ms.isResolved =>
Validated.validNec[Err, Modules[I, Err, Ast[S]]](ms).pure[F]
case Validated.Valid(ms) =>
resolveModules(ms)
case err =>
err.pure[F]
}
def resolveSources: F[ValidatedNec[Err, Modules[I, Err, Ast[S]]]] =
sourceModules.flatMap {
case Validated.Valid(ms) => resolveModules(ms)
case err => err.pure[F]
}
def resolve[T](transpile: Ast[S] => T => T): F[ValidatedNec[Err, Modules[I, Err, T => T]]] =
resolveSources.map(_.map(_.map(transpile)))
}

View File

@ -0,0 +1,7 @@
package aqua.compiler
import aqua.model.AquaContext
case class AquaProcessed[I](id: I, context: AquaContext) {
def hasOutput: Boolean = context.funcs.nonEmpty
}

View File

@ -0,0 +1,14 @@
package aqua.compiler
import cats.data.{Chain, ValidatedNec}
trait AquaSources[F[_], Err, I] {
// Read the sources in the sources directory as (Id, String Content) pairs
def sources: F[ValidatedNec[Err, Chain[(I, String)]]]
// Resolve id of the imported imp string from I file
def resolveImport(from: I, imp: String): F[ValidatedNec[Err, I]]
// Load file by its resolved I
def load(file: I): F[ValidatedNec[Err, String]]
}

View File

@ -1,54 +0,0 @@
package aqua.compiler
import aqua.model.AquaContext
import cats.data.Validated
import java.nio.file.Path
object Prepared {
/**
* @param srcFile aqua source
* @param srcPath a main source path with all aqua files
* @param targetPath a main path where all output files will be written
* @param context processed aqua code
* @return
*/
def apply(
srcFile: Path,
srcPath: Path,
targetPath: Path,
context: AquaContext
): Validated[Throwable, Prepared] =
Validated.catchNonFatal {
val srcDir = if (srcPath.toFile.isDirectory) srcPath else srcPath.getParent
val srcFilePath = srcDir.toAbsolutePath
.normalize()
.relativize(srcFile.toAbsolutePath.normalize())
val targetDir =
targetPath.toAbsolutePath
.normalize()
.resolve(
srcFilePath
)
new Prepared(targetDir, srcFile, context)
}
}
/**
* All info that can be used to write a final output.
* @param targetDir a directory to write to
* @param srcFile file with a source (aqua code)
* @param context processed code
*/
case class Prepared private (targetDir: Path, srcFile: Path, context: AquaContext) {
def hasOutput: Boolean = context.funcs.nonEmpty
def targetPath(fileName: String): Validated[Throwable, Path] =
Validated.catchNonFatal {
targetDir.getParent.resolve(fileName)
}
}

View File

@ -1,114 +0,0 @@
package aqua.compiler.io
import aqua.compiler.io.AquaFiles.ETC
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, Monad}
import cats.data.{EitherT, NonEmptyChain}
import cats.parse.LocationMap
import cats.syntax.apply._
import cats.syntax.functor._
import java.nio.file.{Path, Paths}
import scala.collection.immutable
case class AquaFile(
id: FileModuleId,
imports: Map[String, FileSpan.Focus],
source: String,
ast: Ast[FileSpan.F]
) {
/**
* Gathers all errors and results
*/
private def gatherResolvedResults[F[_]: Monad](
results: immutable.Iterable[EitherT[F, AquaFileError, (FileModuleId, FileNotFound)]]
): ETC[F, Map[FileModuleId, AquaFileError]] =
results
.foldLeft[AquaFiles.ETC[F, Map[FileModuleId, AquaFileError]]](EitherT.rightT(Map())) {
case (files, nextFile) =>
EitherT((files.value, nextFile.value).mapN {
case (files, Right(resolvedImport)) =>
files.map(_ + resolvedImport)
case (Right(_), Left(err)) =>
Left(NonEmptyChain(err))
case (Left(errs), Left(err)) =>
Left(errs.append(err))
})
}
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) =>
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)))
}
for {
importsWithInfo <- gatherResolvedResults(resolvedImports)
} yield AquaModule(
id,
importsWithInfo,
transpile(ast)
)
}
}
object AquaFile {
def parseAst(name: String, input: String): Either[AquaFileError, Ast[F]] = {
implicit val fileLift: LiftParser[FileSpan.F] = FileSpan.fileSpanLiftParser(name, input)
Ast
.fromString[FileSpan.F](input)
.leftMap(_.map {
case BlockIndentError(indent, message) => CustomSyntaxError(indent._1, message)
case FuncReturnError(point, message) => CustomSyntaxError(point._1, message)
case LexerError(pe) =>
val fileSpan =
FileSpan(
name,
input,
Eval.later(LocationMap(input)),
Span(pe.failedAtOffset, pe.failedAtOffset + 1)
)
SyntaxError(fileSpan, pe.expected)
})
.toEither
.left
.map(AquaScriptErrors(_))
}
def read[F[_]: AquaIO: Monad](file: Path): EitherT[F, AquaFileError, AquaFile] =
for {
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)
.collect { case ImportExpr(filename) =>
val path = filename.value.drop(1).dropRight(1)
val focus = filename.unit._1.focus(1)
path -> focus
}
.collect { case (path, Some(focus)) =>
path -> focus
}
.toList
.toMap
} yield AquaFile(
FileModuleId(file.toAbsolutePath.normalize()),
imports,
source,
ast
)
}

View File

@ -1,90 +0,0 @@
package aqua.compiler.io
import aqua.compiler.AquaIO
import aqua.linker.Modules
import aqua.parser.Ast
import aqua.parser.lift.FileSpan
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
object AquaFiles {
type Mods[T] = Modules[FileModuleId, AquaFileError, T]
type ETC[F[_], T] = EitherT[F, NonEmptyChain[AquaFileError], T]
def readSources[F[_]: AquaIO: Monad](
sourcePath: Path
): ETC[F, Chain[AquaFile]] =
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)
.value
.map(Validated.fromEither)
}
.map(_.toEither)
)
def createModules[F[_]: AquaIO: Monad, T](
sources: Chain[AquaFile],
importFromPaths: List[Path],
transpile: Ast[FileSpan.F] => T => T
): ETC[F, Mods[T]] =
sources
.map(_.createModule(transpile, importFromPaths))
.foldLeft[ETC[F, Mods[T]]](
EitherT.rightT(Modules())
) { case (modulesF, modF) =>
for {
ms <- modulesF
m <- modF
} yield ms.add(m, export = true)
}
def resolveModules[F[_]: AquaIO: Monad, T](
modules: Modules[FileModuleId, AquaFileError, T],
importFromPaths: List[Path],
transpile: Ast[FileSpan.F] => T => T
): ETC[F, Mods[T]] =
modules.dependsOn.map { case (moduleId, unresolvedErrors) =>
AquaFile
.read[F](moduleId.file)
.leftMap(unresolvedErrors.prepend)
.flatMap(_.createModule(transpile, importFromPaths))
}.foldLeft[ETC[F, Mods[T]]](
EitherT.rightT(modules)
) { case (modulesF, modF) =>
for {
ms <- modulesF
m <- modF
} yield ms.add(m)
}.flatMap {
case ms if ms.isResolved =>
EitherT.rightT(ms)
case ms => resolveModules(ms, importFromPaths, transpile)
}
def readAndResolve[F[_]: AquaIO: Monad, T](
sourcePath: Path,
importFromPaths: List[Path],
transpile: Ast[FileSpan.F] => T => T
): ETC[F, Mods[T]] =
for {
sources <- readSources(sourcePath)
sourceModules <- createModules(sources, importFromPaths, transpile)
resolvedModules <- resolveModules(sourceModules, importFromPaths, transpile)
} yield resolvedModules
}

View File

@ -1,5 +0,0 @@
package aqua.compiler.io
import java.nio.file.Path
case class FileModuleId(file: Path)

View File

@ -1,8 +1,8 @@
package aqua.linker package aqua.linker
// HACK: here E is a FileNotFound error with Focus that the code will 'throw' case class AquaModule[I, E, T](id: I, dependsOn: Map[I, E], body: T) {
// if not found it in the list of loaded modules in `Modules` class. def map[TT](f: T => TT): AquaModule[I, E, TT] = copy(body = f(body))
// Essentially this error is a container with import information
// and a future error if the file for this import is not found def mapErr[EE](f: E => EE): AquaModule[I, EE, T] =
// TODO: fix it copy(dependsOn = dependsOn.view.mapValues(f).toMap)
case class AquaModule[I, E, T](id: I, dependsOn: Map[I, E], body: T => T) }

View File

@ -11,15 +11,18 @@ object Linker extends LogSupport {
@tailrec @tailrec
def iter[I, E, T: Semigroup]( def iter[I, E, T: Semigroup](
mods: List[AquaModule[I, E, T]], mods: List[AquaModule[I, E, T => T]],
proc: Map[I, T => T], proc: Map[I, T => T],
cycleError: List[AquaModule[I, E, T]] => E cycleError: List[AquaModule[I, E, T => T]] => E
): Either[E, Map[I, T => T]] = ): Either[E, Map[I, T => T]] =
mods match { mods match {
case Nil => Right(proc) case Nil =>
Right(proc)
case _ => case _ =>
val (canHandle, postpone) = mods.partition(_.dependsOn.keySet.forall(proc.contains)) val (canHandle, postpone) = mods.partition(_.dependsOn.keySet.forall(proc.contains))
debug("ITERATE, can handle: " + canHandle.map(_.id)) debug("ITERATE, can handle: " + canHandle.map(_.id))
debug(s"dependsOn = ${mods.map(_.dependsOn.keySet)}")
debug(s"postpone = ${postpone.map(_.id)}")
debug(s"proc = ${proc.keySet}") debug(s"proc = ${proc.keySet}")
if (canHandle.isEmpty && postpone.nonEmpty) if (canHandle.isEmpty && postpone.nonEmpty)
@ -47,17 +50,20 @@ object Linker extends LogSupport {
} }
} }
def apply[I, E, T: Monoid]( def link[I, E, T: Monoid](
modules: Modules[I, E, T], modules: Modules[I, E, T => T],
cycleError: List[AquaModule[I, E, T]] => E cycleError: List[AquaModule[I, E, T => T]] => E
): ValidatedNec[E, Map[I, T]] = ): ValidatedNec[E, Map[I, T]] =
if (modules.dependsOn.nonEmpty) Validated.invalid(modules.dependsOn.values.reduce(_ ++ _)) if (modules.dependsOn.nonEmpty) Validated.invalid(modules.dependsOn.values.reduce(_ ++ _))
else else {
val result = iter(modules.loaded.values.toList, Map.empty[I, T => T], cycleError)
Validated.fromEither( Validated.fromEither(
iter(modules.loaded.values.toList, Map.empty[I, T => T], cycleError) result
.map(_.view.filterKeys(modules.exports).mapValues(_.apply(Monoid[T].empty)).toMap) .map(_.view.filterKeys(modules.exports).mapValues(_.apply(Monoid[T].empty)).toMap)
.left .left
.map(NonEmptyChain.one) .map(NonEmptyChain.one)
) )
}
} }

View File

@ -24,4 +24,13 @@ case class Modules[I, E, T](
) )
def isResolved: Boolean = dependsOn.isEmpty def isResolved: Boolean = dependsOn.isEmpty
def map[TT](f: T => TT): Modules[I, E, TT] =
copy(loaded = loaded.view.mapValues(_.map(f)).toMap)
def mapErr[EE](f: E => EE): Modules[I, EE, T] =
copy(
loaded = loaded.view.mapValues(_.mapErr(f)).toMap,
dependsOn = dependsOn.view.mapValues(_.map(f)).toMap
)
} }

View File

@ -8,17 +8,21 @@ class LinkerSpec extends AnyFlatSpec with Matchers {
"linker" should "resolve dependencies" in { "linker" should "resolve dependencies" in {
val empty = Modules[String, String, String]() val empty = Modules[String, String, String => String]()
val withMod1 = val withMod1 =
empty empty
.add( .add(
AquaModule("mod1", Map("mod2" -> "unresolved mod2 in mod1"), _ ++ " | mod1"), AquaModule[String, String, String => String](
"mod1",
Map("mod2" -> "unresolved mod2 in mod1"),
_ ++ " | mod1"
),
export = true export = true
) )
withMod1.isResolved should be(false) withMod1.isResolved should be(false)
Linker[String, String, String]( Linker.link[String, String, String](
withMod1, withMod1,
cycle => cycle.map(_.id).mkString(" -> ") cycle => cycle.map(_.id).mkString(" -> ")
) should be(Validated.invalidNec("unresolved mod2 in mod1")) ) should be(Validated.invalidNec("unresolved mod2 in mod1"))
@ -28,7 +32,7 @@ class LinkerSpec extends AnyFlatSpec with Matchers {
withMod2.isResolved should be(true) withMod2.isResolved should be(true)
Linker[String, String, String]( Linker.link[String, String, String](
withMod2, withMod2,
cycle => cycle.map(_.id + "?").mkString(" -> ") cycle => cycle.map(_.id + "?").mkString(" -> ")
) should be(Validated.validNec(Map("mod1" -> " | mod2 | mod1"))) ) should be(Validated.validNec(Map("mod1" -> " | mod2 | mod1")))

View File

@ -1,12 +1,10 @@
package aqua.model.transform package aqua.model.transform
import aqua.model.{AquaContext, LiteralModel, ValueModel, VarModel} import aqua.model.{AquaContext, LiteralModel, ValueModel, VarModel}
import aqua.types.{DataType, OptionType} import aqua.types.ScalarType
import cats.kernel.Monoid import cats.kernel.Monoid
case class Constant(name: String, value: ValueModel) case class GenerationConfig(
case class BodyConfig(
getDataService: String = "getDataSrv", getDataService: String = "getDataSrv",
callbackService: String = "callbackSrv", callbackService: String = "callbackSrv",
errorHandlingService: String = "errorHandlingSrv", errorHandlingService: String = "errorHandlingSrv",
@ -14,7 +12,7 @@ case class BodyConfig(
respFuncName: String = "response", respFuncName: String = "response",
relayVarName: Option[String] = Some("-relay-"), relayVarName: Option[String] = Some("-relay-"),
wrapWithXor: Boolean = true, wrapWithXor: Boolean = true,
constants: List[Constant] = Nil constants: List[GenerationConfig.Const] = Nil
) { ) {
val errorId: ValueModel = LiteralModel.quote(errorFuncName) val errorId: ValueModel = LiteralModel.quote(errorFuncName)
@ -22,8 +20,16 @@ case class BodyConfig(
val callbackSrvId: ValueModel = LiteralModel.quote(callbackService) val callbackSrvId: ValueModel = LiteralModel.quote(callbackService)
val dataSrvId: ValueModel = LiteralModel.quote(getDataService) val dataSrvId: ValueModel = LiteralModel.quote(getDataService)
// Host peer id holds %init_peer_id% in case Aqua is not compiled to be executed behind a relay,
// or relay's variable otherwise
val hostPeerId: GenerationConfig.Const =
GenerationConfig.Const(
"host_peer_id",
relayVarName.fold[ValueModel](LiteralModel.initPeerId)(r => VarModel(r, ScalarType.string))
)
implicit val aquaContextMonoid: Monoid[AquaContext] = { implicit val aquaContextMonoid: Monoid[AquaContext] = {
val constantsMap = constants.map(c => c.name -> c.value).toMap val constantsMap = (hostPeerId :: constants).map(c => c.name -> c.value).toMap
AquaContext AquaContext
.implicits( .implicits(
AquaContext.blank AquaContext.blank
@ -38,3 +44,10 @@ case class BodyConfig(
} }
} }
object GenerationConfig {
case class Const(name: String, value: ValueModel)
def forHost: GenerationConfig =
GenerationConfig(wrapWithXor = false, relayVarName = None)
}

View File

@ -22,7 +22,7 @@ object Transform extends LogSupport {
): Cofree[Chain, ResolvedOp] = ): Cofree[Chain, ResolvedOp] =
tree.copy(tail = tree.tail.map(_.filter(t => filter(t.head)).map(clear(_, filter)))) tree.copy(tail = tree.tail.map(_.filter(t => filter(t.head)).map(clear(_, filter))))
def forClient(func: FuncCallable, conf: BodyConfig): Cofree[Chain, ResolvedOp] = { def forClient(func: FuncCallable, conf: GenerationConfig): Cofree[Chain, ResolvedOp] = {
val initCallable: InitPeerCallable = InitViaRelayCallable( val initCallable: InitPeerCallable = InitViaRelayCallable(
Chain.fromOption(conf.relayVarName).map(VarModel(_, ScalarType.string)) Chain.fromOption(conf.relayVarName).map(VarModel(_, ScalarType.string))
) )

View File

@ -3,7 +3,7 @@ package aqua
import aqua.model.func.Call import aqua.model.func.Call
import aqua.model.func.raw._ import aqua.model.func.raw._
import aqua.model.func.resolved.{CallServiceRes, MakeRes, MatchMismatchRes, ResolvedOp} import aqua.model.func.resolved.{CallServiceRes, MakeRes, MatchMismatchRes, ResolvedOp}
import aqua.model.transform.{BodyConfig, ErrorsCatcher} import aqua.model.transform.{ErrorsCatcher, GenerationConfig}
import aqua.model.{LiteralModel, ValueModel, VarModel} import aqua.model.{LiteralModel, ValueModel, VarModel}
import aqua.types.{ArrayType, LiteralType, ScalarType} import aqua.types.{ArrayType, LiteralType, ScalarType}
import cats.Eval import cats.Eval
@ -88,7 +88,7 @@ object Node {
) )
) )
def errorCall(bc: BodyConfig, i: Int, on: ValueModel = initPeer): Res = Node[ResolvedOp]( def errorCall(bc: GenerationConfig, i: Int, on: ValueModel = initPeer): Res = Node[ResolvedOp](
CallServiceRes( CallServiceRes(
bc.errorHandlingCallback, bc.errorHandlingCallback,
bc.errorFuncName, bc.errorFuncName,
@ -103,7 +103,7 @@ object Node {
) )
) )
def respCall(bc: BodyConfig, value: ValueModel, on: ValueModel = initPeer): Res = def respCall(bc: GenerationConfig, value: ValueModel, on: ValueModel = initPeer): Res =
Node[ResolvedOp]( Node[ResolvedOp](
CallServiceRes( CallServiceRes(
bc.callbackSrvId, bc.callbackSrvId,
@ -113,14 +113,15 @@ object Node {
) )
) )
def dataCall(bc: BodyConfig, name: String, on: ValueModel = initPeer): Res = Node[ResolvedOp]( def dataCall(bc: GenerationConfig, name: String, on: ValueModel = initPeer): Res =
CallServiceRes( Node[ResolvedOp](
bc.dataSrvId, CallServiceRes(
name, bc.dataSrvId,
Call(Nil, Some(Call.Export(name, ScalarType.string))), name,
on Call(Nil, Some(Call.Export(name, ScalarType.string))),
on
)
) )
)
def on(peer: ValueModel, via: List[ValueModel], body: Raw*) = def on(peer: ValueModel, via: List[ValueModel], body: Raw*) =
Node( Node(

View File

@ -26,7 +26,7 @@ class TransformSpec extends AnyFlatSpec with Matchers {
Map.empty Map.empty
) )
val bc = BodyConfig() val bc = GenerationConfig()
val fc = Transform.forClient(func, bc) val fc = Transform.forClient(func, bc)
@ -76,7 +76,7 @@ class TransformSpec extends AnyFlatSpec with Matchers {
Map.empty Map.empty
) )
val bc = BodyConfig(wrapWithXor = false) val bc = GenerationConfig(wrapWithXor = false)
val fc = Transform.forClient(func, bc) val fc = Transform.forClient(func, bc)
@ -137,7 +137,7 @@ class TransformSpec extends AnyFlatSpec with Matchers {
Map.empty Map.empty
) )
val bc = BodyConfig(wrapWithXor = false) val bc = GenerationConfig(wrapWithXor = false)
val res = Transform.forClient(f2, bc): Node.Res val res = Transform.forClient(f2, bc): Node.Res

View File

@ -3,6 +3,7 @@ package aqua.parser
import aqua.parser.expr._ import aqua.parser.expr._
import aqua.parser.head.{HeadExpr, HeaderExpr} import aqua.parser.head.{HeadExpr, HeaderExpr}
import aqua.parser.lift.LiftParser import aqua.parser.lift.LiftParser
import aqua.parser.lift.LiftParser._
import cats.data.{Chain, Validated, ValidatedNec} import cats.data.{Chain, Validated, ValidatedNec}
import cats.free.Cofree import cats.free.Cofree
import cats.parse.{Parser0 => P0} import cats.parse.{Parser0 => P0}
@ -27,7 +28,7 @@ object Ast {
parser[F]() parser[F]()
.parseAll(script) match { .parseAll(script) match {
case Right(value) => value case Right(value) => value
case Left(e) => Validated.invalidNec(LexerError[F](e)) case Left(e) => Validated.invalidNec(LexerError[F](e.wrapErr))
} }
} }

View File

@ -4,6 +4,6 @@ import cats.parse.Parser
trait ParserError[F[_]] trait ParserError[F[_]]
case class LexerError[F[_]](err: Parser.Error) extends ParserError[F] case class LexerError[F[_]](err: F[Parser.Error]) extends ParserError[F]
case class BlockIndentError[F[_]](indent: F[String], message: String) extends ParserError[F] case class BlockIndentError[F[_]](indent: F[String], message: String) extends ParserError[F]
case class FuncReturnError[F[_]](point: F[Unit], message: String) extends ParserError[F] case class FuncReturnError[F[_]](point: F[Unit], message: String) extends ParserError[F]

View File

@ -6,7 +6,8 @@ import cats.{Comonad, Eval}
import scala.language.implicitConversions import scala.language.implicitConversions
// TODO: rewrite FileSpan and Span under one trait // TODO: rewrite FileSpan and Span under one trait
case class FileSpan(name: String, source: String, locationMap: Eval[LocationMap], span: Span) { // TODO: move FileSpan to another package?
case class FileSpan(name: String, locationMap: Eval[LocationMap], span: Span) {
def focus(ctx: Int): Option[FileSpan.Focus] = def focus(ctx: Int): Option[FileSpan.Focus] =
span.focus(locationMap, ctx).map(FileSpan.Focus(name, locationMap, ctx, _)) span.focus(locationMap, ctx).map(FileSpan.Focus(name, locationMap, ctx, _))
@ -37,18 +38,27 @@ object FileSpan {
def fileSpanLiftParser(name: String, source: String): LiftParser[F] = new LiftParser[F] { def fileSpanLiftParser(name: String, source: String): LiftParser[F] = new LiftParser[F] {
val memoizedLocationMap = Eval.later(LocationMap(source)).memoize private val memoizedLocationMap = Eval.later(LocationMap(source)).memoize
override def lift[T](p: P[T]): P[F[T]] = { override def lift[T](p: P[T]): P[F[T]] = {
implicitly[LiftParser[Span.F]].lift(p).map { case (span, value) => implicitly[LiftParser[Span.F]].lift(p).map { case (span, value) =>
(FileSpan(name, source, memoizedLocationMap, span), value) (FileSpan(name, memoizedLocationMap, span), value)
} }
} }
override def lift0[T](p0: Parser0[T]): Parser0[(FileSpan, T)] = { override def lift0[T](p0: Parser0[T]): Parser0[(FileSpan, T)] = {
implicitly[LiftParser[Span.F]].lift0(p0).map { case (span, value) => implicitly[LiftParser[Span.F]].lift0(p0).map { case (span, value) =>
(FileSpan(name, source, memoizedLocationMap, span), value) (FileSpan(name, memoizedLocationMap, span), value)
} }
} }
override def wrapErr(e: P.Error): (FileSpan, P.Error) = (
FileSpan(
name,
memoizedLocationMap,
Span(e.failedAtOffset, e.failedAtOffset + 1)
),
e
)
} }
} }

View File

@ -7,10 +7,16 @@ trait LiftParser[F[_]] {
def lift[T](p: Parser[T]): Parser[F[T]] def lift[T](p: Parser[T]): Parser[F[T]]
def lift0[T](p0: Parser0[T]): Parser0[F[T]] def lift0[T](p0: Parser0[T]): Parser0[F[T]]
def wrapErr(e: Parser.Error): F[Parser.Error]
} }
object LiftParser { object LiftParser {
implicit class LiftErrorOps[F[_]: LiftParser, T](e: Parser.Error) {
def wrapErr: F[Parser.Error] = implicitly[LiftParser[F]].wrapErr(e)
}
implicit class LiftParserOps[F[_]: LiftParser, T](parser: Parser[T]) { implicit class LiftParserOps[F[_]: LiftParser, T](parser: Parser[T]) {
def lift: Parser[F[T]] = implicitly[LiftParser[F]].lift(parser) def lift: Parser[F[T]] = implicitly[LiftParser[F]].lift(parser)
} }
@ -24,6 +30,7 @@ object LiftParser {
implicit object idLiftParser extends LiftParser[Id] { implicit object idLiftParser extends LiftParser[Id] {
override def lift[T](p: Parser[T]): Parser[Id[T]] = p override def lift[T](p: Parser[T]): Parser[Id[T]] = p
override def lift0[T](p0: Parser0[T]): Parser0[Id[T]] = p0 override def lift0[T](p0: Parser0[T]): Parser0[Id[T]] = p0
override def wrapErr(e: Parser.Error): Id[Parser.Error] = e
} }
} }

View File

@ -105,6 +105,9 @@ object Span {
(P.index ~ p0).map { case (i, v) (P.index ~ p0).map { case (i, v)
(Span(i, i), v) (Span(i, i), v)
} }
override def wrapErr(e: P.Error): (Span, P.Error) =
(Span(e.failedAtOffset, e.failedAtOffset + 1), e)
} }
} }

View File

@ -29,7 +29,7 @@ class SemanticsSpec extends AnyFlatSpec with Matchers {
val ast = Ast.fromString(script).toList.head val ast = Ast.fromString(script).toList.head
val ctx = AquaContext.blank val ctx = AquaContext.blank
val bc = BodyConfig() val bc = GenerationConfig()
import bc.aquaContextMonoid import bc.aquaContextMonoid
val p = Semantics.process(ast, ctx) val p = Semantics.process(ast, ctx)