feat(compiler): Add warnings subsystem [fixes LNG117] (#906)

* ErrorsAlgebra -> ReportAlgebra

* Refactor ReportAlgebra

* Refactor

* Refactor AquaError

* Fixes

* Add warnings, refactor

* Refactor parser

* Move semantics

* Savepoint

* Refactor semantics and compiler

* Refactor types

* Refactor compiler

* Refactor compiler

* Refactor types

* Refactor retunr types

* Return warnings

* Add simple warning

* Refactor to ValidatedNec

* Refactor

* Add comment

* Propagate warnings to LspContext

* Propagate warnings to LSP

* Add warnings to js api

* Update LSP js api

* Use export declare

* Add comment

* Refactor span rendering

* Remove variable name warning

* Add warning on unused call results

* Add unit tests

* Remove println
This commit is contained in:
InversionSpaces
2023-09-25 15:00:43 +02:00
committed by GitHub
parent 1c708c8bb0
commit 27a781dd3f
51 changed files with 1642 additions and 1241 deletions

View File

@ -1,22 +1,23 @@
import { ServiceDef, FunctionCallDef } from "@fluencelabs/interfaces"; import { ServiceDef, FunctionCallDef } from "@fluencelabs/interfaces";
export class AquaFunction { export declare class AquaFunction {
funcDef: FunctionCallDef; funcDef: FunctionCallDef;
script: string; script: string;
} }
export class GeneratedSource { export declare class GeneratedSource {
name: string; name: string;
tsSource?: string; tsSource?: string;
jsSource?: string; jsSource?: string;
tsTypes?: string; tsTypes?: string;
} }
class CompilationResult { export declare class CompilationResult {
services: Record<string, ServiceDef>; services: Record<string, ServiceDef>;
functions: Record<string, AquaFunction>; functions: Record<string, AquaFunction>;
functionCall?: AquaFunction; functionCall?: AquaFunction;
errors: string[]; errors: string[];
warnings: string[];
generatedSources: GeneratedSource[]; generatedSources: GeneratedSource[];
} }

View File

@ -1,9 +1,9 @@
package api package api
import api.types.{AquaConfig, AquaFunction, CompilationResult, GeneratedSource, Input} import api.types.{AquaConfig, AquaFunction, CompilationResult, GeneratedSource, Input}
import aqua.ErrorRendering.showError import aqua.Rendering.given
import aqua.raw.value.ValueRaw import aqua.raw.value.ValueRaw
import aqua.api.{APICompilation, AquaAPIConfig} import aqua.api.{APICompilation, APIResult, AquaAPIConfig}
import aqua.api.TargetType.* import aqua.api.TargetType.*
import aqua.backend.air.AirBackend import aqua.backend.air.AirBackend
import aqua.backend.{AirFunction, Backend, Generated} import aqua.backend.{AirFunction, Backend, Generated}
@ -13,18 +13,24 @@ import aqua.logging.{LogFormatter, LogLevels}
import aqua.constants.Constants import aqua.constants.Constants
import aqua.io.* import aqua.io.*
import aqua.raw.ops.Call import aqua.raw.ops.Call
import aqua.run.{CallInfo, CallPreparer, CliFunc, FuncCompiler, RunPreparer} import aqua.run.{CliFunc, FuncCompiler}
import aqua.parser.lexer.{LiteralToken, Token} import aqua.parser.lexer.{LiteralToken, Token}
import aqua.parser.lift.FileSpan.F import aqua.parser.lift.FileSpan.F
import aqua.parser.lift.{FileSpan, Span} import aqua.parser.lift.{FileSpan, Span}
import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError} import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError}
import aqua.semantics.{CompilerState, HeaderError, RulesViolated, WrongAST}
import aqua.{AquaIO, SpanParser} import aqua.{AquaIO, SpanParser}
import aqua.model.transform.{Transform, TransformConfig} import aqua.model.transform.{Transform, TransformConfig}
import aqua.backend.api.APIBackend import aqua.backend.api.APIBackend
import aqua.backend.js.JavaScriptBackend import aqua.backend.js.JavaScriptBackend
import aqua.backend.ts.TypeScriptBackend import aqua.backend.ts.TypeScriptBackend
import aqua.definitions.FunctionDef import aqua.definitions.FunctionDef
import aqua.js.{FunctionDefJs, ServiceDefJs, VarJson}
import aqua.model.AquaContext
import aqua.raw.ops.CallArrowRawTag
import aqua.raw.value.{LiteralRaw, VarRaw}
import aqua.res.AquaRes
import cats.Applicative
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec} import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid} import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.syntax.applicative.* import cats.syntax.applicative.*
@ -35,6 +41,7 @@ import cats.effect.IO
import cats.effect.unsafe.implicits.global import cats.effect.unsafe.implicits.global
import cats.syntax.show.* import cats.syntax.show.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.either.*
import fs2.io.file.{Files, Path} import fs2.io.file.{Files, Path}
import scribe.Logging import scribe.Logging
@ -44,12 +51,6 @@ import scala.scalajs.js.{|, undefined, Promise, UndefOr}
import scala.scalajs.js import scala.scalajs.js
import scala.scalajs.js.JSConverters.* import scala.scalajs.js.JSConverters.*
import scala.scalajs.js.annotation.* import scala.scalajs.js.annotation.*
import aqua.js.{FunctionDefJs, ServiceDefJs, VarJson}
import aqua.model.AquaContext
import aqua.raw.ops.CallArrowRawTag
import aqua.raw.value.{LiteralRaw, VarRaw}
import aqua.res.AquaRes
import cats.Applicative
@JSExportTopLevel("Aqua") @JSExportTopLevel("Aqua")
object AquaAPI extends App with Logging { object AquaAPI extends App with Logging {
@ -68,11 +69,9 @@ object AquaAPI extends App with Logging {
aquaConfigJS: js.UndefOr[AquaConfig] aquaConfigJS: js.UndefOr[AquaConfig]
): Promise[CompilationResult] = { ): Promise[CompilationResult] = {
aquaConfigJS.toOption aquaConfigJS.toOption
.map(cjs => AquaConfig.fromJS(cjs)) .map(AquaConfig.fromJS)
.getOrElse( .getOrElse(validNec(AquaAPIConfig()))
validNec(AquaAPIConfig()) .traverse { config =>
)
.map { config =>
val importsList = imports.toList val importsList = imports.toList
input match { input match {
@ -82,10 +81,10 @@ object AquaAPI extends App with Logging {
compileCall(c, importsList, config) compileCall(c, importsList, config)
} }
} match {
case Valid(v) => v.unsafeToFuture().toJSPromise
case Invalid(errs) => js.Promise.resolve(CompilationResult.errs(errs.toChain.toList))
} }
.map(_.leftMap(errs => CompilationResult.errs(errs.toChain.toList)).merge)
.unsafeToFuture()
.toJSPromise
} }
// Compile all non-call inputs // Compile all non-call inputs
@ -100,15 +99,17 @@ object AquaAPI extends App with Logging {
case JavaScriptType => JavaScriptBackend() case JavaScriptType => JavaScriptBackend()
} }
extension (res: IO[ValidatedNec[String, Chain[AquaCompiled[FileModuleId]]]]) extension (res: APIResult[Chain[AquaCompiled[FileModuleId]]])
def toResult: IO[CompilationResult] = res.map { compiledV => def toResult: CompilationResult = {
compiledV.map { compiled => val (warnings, result) = res.value.run
config.targetType match {
case AirType => generatedToAirResult(compiled) result.map { compiled =>
case TypeScriptType => compiledToTsSourceResult(compiled) (config.targetType match {
case JavaScriptType => compiledToJsSourceResult(compiled) case AirType => generatedToAirResult
} case TypeScriptType => compiledToTsSourceResult
}.leftMap(errorsToResult).merge case JavaScriptType => compiledToJsSourceResult
}).apply(compiled, warnings)
}.leftMap(errorsToResult(_, warnings)).merge
} }
input match { input match {
@ -120,7 +121,7 @@ object AquaAPI extends App with Logging {
config, config,
backend backend
) )
.toResult .map(_.toResult)
case p: types.Path => case p: types.Path =>
APICompilation APICompilation
.compilePath( .compilePath(
@ -129,23 +130,33 @@ object AquaAPI extends App with Logging {
config, config,
backend backend
) )
.toResult .map(_.toResult)
} }
} }
private def compileCall(call: types.Call, imports: List[String], config: AquaAPIConfig) = { // Compile a function call
private def compileCall(
call: types.Call,
imports: List[String],
config: AquaAPIConfig
): IO[CompilationResult] = {
val path = call.input match { val path = call.input match {
case i: types.Input => i.input case i: types.Input => i.input
case p: types.Path => p.path case p: types.Path => p.path
} }
extension (res: IO[ValidatedNec[String, (FunctionDef, String)]]) extension (res: APIResult[(FunctionDef, String)])
def callToResult: IO[CompilationResult] = res.map( def callToResult: CompilationResult = {
_.map { case (definitions, air) => val (warnings, result) = res.value.run
CompilationResult.result(call = Some(AquaFunction(FunctionDefJs(definitions), air)))
}.leftMap(errorsToResult).merge result.map { case (definitions, air) =>
CompilationResult.result(
call = Some(AquaFunction(FunctionDefJs(definitions), air)),
warnings = warnings.toList
) )
}.leftMap(errorsToResult(_, warnings)).merge
}
APICompilation APICompilation
.compileCall( .compileCall(
@ -155,34 +166,36 @@ object AquaAPI extends App with Logging {
config, config,
vr => VarJson.checkDataGetServices(vr, Some(call.arguments)).map(_._1) vr => VarJson.checkDataGetServices(vr, Some(call.arguments)).map(_._1)
) )
.callToResult .map(_.callToResult)
} }
private def errorsToResult(errors: NonEmptyChain[String]): CompilationResult = { private def errorsToResult(
CompilationResult.errs(errors.toChain.toList) errors: NonEmptyChain[String],
} warnings: Chain[String]
): CompilationResult = CompilationResult.errs(
extension (res: List[GeneratedSource]) errors.toChain.toList,
warnings.toList
def toSourcesResult: CompilationResult = )
CompilationResult.result(sources = res.toJSArray)
private def compiledToTsSourceResult( private def compiledToTsSourceResult(
compiled: Chain[AquaCompiled[FileModuleId]] compiled: Chain[AquaCompiled[FileModuleId]],
): CompilationResult = warnings: Chain[String]
compiled.toList ): CompilationResult = CompilationResult.result(
sources = compiled.toList
.flatMap(c => .flatMap(c =>
c.compiled c.compiled
.find(_.suffix == TypeScriptBackend.ext) .find(_.suffix == TypeScriptBackend.ext)
.map(_.content) .map(_.content)
.map(GeneratedSource.tsSource(c.sourceId.toString, _)) .map(GeneratedSource.tsSource(c.sourceId.toString, _))
),
warnings = warnings.toList
) )
.toSourcesResult
private def compiledToJsSourceResult( private def compiledToJsSourceResult(
compiled: Chain[AquaCompiled[FileModuleId]] compiled: Chain[AquaCompiled[FileModuleId]],
): CompilationResult = warnings: Chain[String]
compiled.toList.flatMap { c => ): CompilationResult = CompilationResult.result(
sources = compiled.toList.flatMap { c =>
for { for {
dtsContent <- c.compiled dtsContent <- c.compiled
.find(_.suffix == JavaScriptBackend.dtsExt) .find(_.suffix == JavaScriptBackend.dtsExt)
@ -191,20 +204,24 @@ object AquaAPI extends App with Logging {
.find(_.suffix == JavaScriptBackend.ext) .find(_.suffix == JavaScriptBackend.ext)
.map(_.content) .map(_.content)
} yield GeneratedSource.jsSource(c.sourceId.toString, jsContent, dtsContent) } yield GeneratedSource.jsSource(c.sourceId.toString, jsContent, dtsContent)
}.toSourcesResult },
warnings = warnings.toList
)
private def generatedToAirResult( private def generatedToAirResult(
compiled: Chain[AquaCompiled[FileModuleId]] compiled: Chain[AquaCompiled[FileModuleId]],
warnings: Chain[String]
): CompilationResult = { ): CompilationResult = {
val generated = compiled.toList.flatMap(_.compiled) val generated = compiled.toList.flatMap(_.compiled)
val serviceDefs = generated.flatMap(_.services).map(s => s.name -> ServiceDefJs(s)) val serviceDefs = generated.flatMap(_.services).map(s => s.name -> ServiceDefJs(s))
val functions = generated.flatMap( val functions = generated.flatMap(
_.air.map(as => (as.name, AquaFunction(FunctionDefJs(as.funcDef), as.air))) _.air.map(as => as.name -> AquaFunction(FunctionDefJs(as.funcDef), as.air))
) )
CompilationResult.result( CompilationResult.result(
js.Dictionary.apply(serviceDefs: _*), services = serviceDefs.toMap,
js.Dictionary.apply(functions: _*) functions = functions.toMap,
warnings = warnings.toList
) )
} }

View File

@ -2,7 +2,8 @@ package api.types
import aqua.js.{FunctionDefJs, ServiceDefJs} import aqua.js.{FunctionDefJs, ServiceDefJs}
import aqua.model.transform.TransformConfig import aqua.model.transform.TransformConfig
import cats.data.Validated.{Invalid, Valid, invalidNec, validNec}
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec} import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import scala.scalajs.js import scala.scalajs.js
@ -31,8 +32,12 @@ case class GeneratedSource(
) )
object GeneratedSource { object GeneratedSource {
def tsSource(name: String, tsSource: String) = new GeneratedSource(name, tsSource, null, null)
def jsSource(name: String, jsSource: String, tsTypes: String) = new GeneratedSource(name, null, jsSource, tsTypes) def tsSource(name: String, tsSource: String) =
new GeneratedSource(name, tsSource, null, null)
def jsSource(name: String, jsSource: String, tsTypes: String) =
new GeneratedSource(name, null, jsSource, tsTypes)
} }
@JSExportTopLevel("CompilationResult") @JSExportTopLevel("CompilationResult")
@ -46,21 +51,39 @@ class CompilationResult(
@JSExport @JSExport
val generatedSources: js.Array[GeneratedSource], val generatedSources: js.Array[GeneratedSource],
@JSExport @JSExport
val errors: js.Array[String] val errors: js.Array[String],
@JSExport
val warnings: js.Array[String]
) )
object CompilationResult { object CompilationResult {
def result( def result(
services: js.Dictionary[ServiceDefJs] = js.Dictionary(), services: Map[String, ServiceDefJs] = Map.empty,
functions: js.Dictionary[AquaFunction] = js.Dictionary(), functions: Map[String, AquaFunction] = Map.empty,
call: Option[AquaFunction] = None, call: Option[AquaFunction] = None,
sources: js.Array[GeneratedSource] = js.Array() sources: List[GeneratedSource] = List.empty,
warnings: List[String] = List.empty
): CompilationResult = ): CompilationResult =
new CompilationResult(services, functions, call.orNull, sources, js.Array()) new CompilationResult(
services.toJSDictionary,
functions.toJSDictionary,
call.orNull,
sources.toJSArray,
js.Array(),
warnings.toJSArray
)
def errs( def errs(
errors: List[String] errors: List[String] = List.empty,
warnings: List[String] = List.empty
): CompilationResult = ): CompilationResult =
CompilationResult(js.Dictionary(), js.Dictionary(), null, null, errors.toJSArray) new CompilationResult(
js.Dictionary.empty,
js.Dictionary.empty,
null,
null,
errors.toJSArray,
warnings.toJSArray
)
} }

View File

@ -9,7 +9,7 @@ import cats.data.Chain
import cats.data.Validated.{Invalid, Valid} import cats.data.Validated.{Invalid, Valid}
import cats.effect.{IO, IOApp} import cats.effect.{IO, IOApp}
import fs2.io.file.{Files, Path} import fs2.io.file.{Files, Path}
import fs2.{Stream, text} import fs2.{text, Stream}
object Test extends IOApp.Simple { object Test extends IOApp.Simple {
@ -21,19 +21,31 @@ object Test extends IOApp.Simple {
AquaAPIConfig(targetType = TypeScriptType), AquaAPIConfig(targetType = TypeScriptType),
TypeScriptBackend(false, "IFluenceClient$$") TypeScriptBackend(false, "IFluenceClient$$")
) )
.flatMap { .flatMap { res =>
case Valid(res) => val (warnings, result) = res.value.run
val content = res.get(0).get.compiled.head.content
IO.delay {
warnings.toList.foreach(println)
} *> result.fold(
errors =>
IO.delay {
errors.toChain.toList.foreach(println)
},
compiled => {
val content = compiled.get(0).get.compiled.head.content
val targetPath = Path("./target/antithesis.ts") val targetPath = Path("./target/antithesis.ts")
Stream.emit(content) Stream
.emit(content)
.through(text.utf8.encode) .through(text.utf8.encode)
.through(Files[IO].writeAll(targetPath)) .through(Files[IO].writeAll(targetPath))
.attempt .attempt
.compile .compile
.last.flatMap(_ => IO.delay(println(s"File: ${targetPath.absolute.normalize}"))) .last *> IO.delay(
case Invalid(e) => println(s"File: ${targetPath.absolute.normalize}")
IO.delay(println(e)) )
}
)
} }
} }

View File

@ -1,7 +1,8 @@
package aqua.api package aqua.api
import aqua.ErrorRendering.showError import aqua.Rendering.given
import aqua.raw.value.ValueRaw import aqua.raw.value.ValueRaw
import aqua.raw.ConstantRaw
import aqua.api.AquaAPIConfig import aqua.api.AquaAPIConfig
import aqua.backend.{AirFunction, Backend, Generated} import aqua.backend.{AirFunction, Backend, Generated}
import aqua.compiler.* import aqua.compiler.*
@ -10,21 +11,31 @@ import aqua.logging.{LogFormatter, LogLevels}
import aqua.constants.Constants import aqua.constants.Constants
import aqua.io.* import aqua.io.*
import aqua.raw.ops.Call import aqua.raw.ops.Call
import aqua.run.{CallInfo, CallPreparer, CliFunc, FuncCompiler, RunPreparer} import aqua.run.{CliFunc, FuncCompiler, RunPreparer}
import aqua.parser.lexer.{LiteralToken, Token} import aqua.parser.lexer.{LiteralToken, Token}
import aqua.parser.lift.FileSpan.F import aqua.parser.lift.FileSpan.F
import aqua.parser.lift.{FileSpan, Span} import aqua.parser.lift.{FileSpan, Span}
import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError} import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError}
import aqua.semantics.{CompilerState, HeaderError, RulesViolated, WrongAST}
import aqua.{AquaIO, SpanParser} import aqua.{AquaIO, SpanParser}
import aqua.model.transform.{Transform, TransformConfig} import aqua.model.transform.{Transform, TransformConfig}
import aqua.backend.api.APIBackend import aqua.backend.api.APIBackend
import aqua.definitions.FunctionDef import aqua.definitions.FunctionDef
import aqua.model.AquaContext import aqua.model.AquaContext
import aqua.res.AquaRes import aqua.res.AquaRes
import cats.Applicative import cats.Applicative
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec} import cats.~>
import cats.data.Validated.{Invalid, Valid, invalid, invalidNec, validNec} import cats.data.{
Chain,
EitherT,
NonEmptyChain,
NonEmptyList,
Validated,
ValidatedNec,
ValidatedNel,
Writer
}
import cats.data.Validated.{invalid, invalidNec, validNec, Invalid, Valid}
import cats.syntax.applicative.* import cats.syntax.applicative.*
import cats.syntax.apply.* import cats.syntax.apply.*
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
@ -33,8 +44,9 @@ import cats.effect.IO
import cats.effect.unsafe.implicits.global import cats.effect.unsafe.implicits.global
import cats.syntax.show.* import cats.syntax.show.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.either.*
import fs2.io.file.{Files, Path} import fs2.io.file.{Files, Path}
import scribe.Logging import scribe.{Level, Logging}
object APICompilation { object APICompilation {
@ -44,13 +56,13 @@ object APICompilation {
imports: List[String], imports: List[String],
aquaConfig: AquaAPIConfig, aquaConfig: AquaAPIConfig,
fillWithTypes: List[ValueRaw] => ValidatedNec[String, List[ValueRaw]] fillWithTypes: List[ValueRaw] => ValidatedNec[String, List[ValueRaw]]
): IO[ValidatedNec[String, (FunctionDef, String)]] = { ): IO[APIResult[(FunctionDef, String)]] = {
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO] given AquaIO[IO] = new AquaFilesIO[IO]
( (
LogLevels.levelFromString(aquaConfig.logLevel), LogLevels.levelFromString(aquaConfig.logLevel),
Constants.parse(aquaConfig.constants) Constants.parse(aquaConfig.constants)
).mapN { (level, constants) => ).tupled.toResult.flatTraverse { case (level, constants) =>
val transformConfig = aquaConfig.getTransformConfig.copy(constants = constants) val transformConfig = aquaConfig.getTransformConfig.copy(constants = constants)
LogFormatter.initLogger(Some(level)) LogFormatter.initLogger(Some(level))
@ -60,43 +72,38 @@ object APICompilation {
imports.map(Path.apply), imports.map(Path.apply),
transformConfig transformConfig
).compile().map { contextV => ).compile().map { contextV =>
contextV.andThen { context => for {
CliFunc context <- contextV.toResult
cliFunc <- CliFunc
.fromString(functionStr) .fromString(functionStr)
.leftMap(errs => NonEmptyChain.fromNonEmptyList(errs)) .toResult
.andThen { cliFunc => arrow <- FuncCompiler
FuncCompiler.findFunction(context, cliFunc).andThen { arrow => .findFunction(context, cliFunc)
fillWithTypes(cliFunc.args).andThen { argsWithTypes => .toResult
val func = cliFunc.copy(args = argsWithTypes) argsWithTypes <- fillWithTypes(cliFunc.args).toResult
val preparer = new RunPreparer( func = cliFunc.copy(args = argsWithTypes)
preparer = new RunPreparer(
func, func,
arrow, arrow,
transformConfig transformConfig
) )
preparer.prepare().map { ci => ci <- preparer.prepare().toResult
(ci.definitions, ci.air) } yield ci.definitions -> ci.air
} }
} }
} }
}
}.leftMap(_.map(_.show).distinct)
}
} match {
case Valid(pr) => pr
case Invalid(errs) => IO.pure(Invalid(NonEmptyChain.fromNonEmptyList(errs)))
}
}
def compilePath( def compilePath(
pathStr: String, pathStr: String,
imports: List[String], imports: List[String],
aquaConfig: AquaAPIConfig, aquaConfig: AquaAPIConfig,
backend: Backend backend: Backend
): IO[ValidatedNec[String, Chain[AquaCompiled[FileModuleId]]]] = { ): IO[APIResult[Chain[AquaCompiled[FileModuleId]]]] = {
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO] given AquaIO[IO] = new AquaFilesIO[IO]
val path = Path(pathStr) val path = Path(pathStr)
val sources = new AquaFileSources[IO](path, imports.map(Path.apply)) val sources = new AquaFileSources[IO](path, imports.map(Path.apply))
compileRaw( compileRaw(
aquaConfig, aquaConfig,
sources, sources,
@ -109,8 +116,9 @@ object APICompilation {
imports: List[String], imports: List[String],
aquaConfig: AquaAPIConfig, aquaConfig: AquaAPIConfig,
backend: Backend backend: Backend
): IO[ValidatedNec[String, Chain[AquaCompiled[FileModuleId]]]] = { ): IO[APIResult[Chain[AquaCompiled[FileModuleId]]]] = {
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO] given AquaIO[IO] = new AquaFilesIO[IO]
val path = Path("") val path = Path("")
val strSources: AquaFileSources[IO] = val strSources: AquaFileSources[IO] =
@ -119,6 +127,7 @@ object APICompilation {
IO.pure(Valid(Chain.one((FileModuleId(path), input)))) IO.pure(Valid(Chain.one((FileModuleId(path), input))))
} }
} }
compileRaw( compileRaw(
aquaConfig, aquaConfig,
strSources, strSources,
@ -130,13 +139,10 @@ object APICompilation {
aquaConfig: AquaAPIConfig, aquaConfig: AquaAPIConfig,
sources: AquaSources[IO, AquaFileError, FileModuleId], sources: AquaSources[IO, AquaFileError, FileModuleId],
backend: Backend backend: Backend
): IO[ValidatedNec[String, Chain[AquaCompiled[FileModuleId]]]] = { ): IO[APIResult[Chain[AquaCompiled[FileModuleId]]]] = (
(
LogLevels.levelFromString(aquaConfig.logLevel), LogLevels.levelFromString(aquaConfig.logLevel),
Constants.parse(aquaConfig.constants) Constants.parse(aquaConfig.constants)
).traverseN { (level, constants) => ).tupled.toResult.flatTraverse { case (level, constants) =>
LogFormatter.initLogger(Some(level)) LogFormatter.initLogger(Some(level))
val transformConfig = aquaConfig.getTransformConfig val transformConfig = aquaConfig.getTransformConfig
@ -151,14 +157,33 @@ object APICompilation {
override def validate(airs: List[AirFunction]): IO[ValidatedNec[String, Unit]] = override def validate(airs: List[AirFunction]): IO[ValidatedNec[String, Unit]] =
Applicative[IO].pure(validNec(())) Applicative[IO].pure(validNec(()))
}, },
new Backend.Transform: new Backend.Transform {
override def transform(ex: AquaContext): AquaRes = override def transform(ex: AquaContext): AquaRes =
Transform.contextRes(ex, transformConfig) Transform.contextRes(ex, transformConfig)
override def generate(aqua: AquaRes): Seq[Generated] = backend.generate(aqua) override def generate(aqua: AquaRes): Seq[Generated] = backend.generate(aqua)
, },
config config
).map(_.leftMap(_.map(_.show).distinct)) )
}.map(_.leftMap(NonEmptyChain.fromNonEmptyList).andThen(identity)) .map(_.toResult)
}
extension [A](v: ValidatedNec[String, A]) {
def toResult: APIResult[A] =
v.toEither.toEitherT
}
extension [A](v: CompileResult[FileModuleId, AquaFileError, FileSpan.F][A]) {
def toResult: APIResult[A] =
v.leftMap(_.map(_.show))
.mapK(
new (CompileWarnings[FileSpan.F] ~> APIWarnings) {
override def apply[A](w: CompileWarnings[FileSpan.F][A]): APIWarnings[A] =
w.mapWritten(_.map(_.show))
}
)
} }
} }

View File

@ -0,0 +1,17 @@
package aqua
import cats.data.{Chain, EitherT, NonEmptyChain, Writer}
package object api {
type APIWarnings = [A] =>> Writer[
Chain[String],
A
]
type APIResult = [A] =>> EitherT[
APIWarnings,
NonEmptyChain[String],
A
]
}

View File

@ -5,8 +5,8 @@ import aqua.parser.lift.Span
import aqua.raw.value.{CollectionRaw, LiteralRaw, ValueRaw, VarRaw} import aqua.raw.value.{CollectionRaw, LiteralRaw, ValueRaw, VarRaw}
import aqua.types.{ArrayType, BottomType} import aqua.types.{ArrayType, BottomType}
import cats.data.{NonEmptyList, Validated, ValidatedNel} import cats.data.{NonEmptyChain, NonEmptyList, Validated, ValidatedNec}
import cats.data.Validated.{invalid, invalidNel, validNel} import cats.data.Validated.{invalid, invalidNec, validNec}
import cats.{~>, Id} import cats.{~>, Id}
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.validated.* import cats.syntax.validated.*
@ -18,25 +18,27 @@ case class CliFunc(name: String, args: List[ValueRaw] = Nil)
object CliFunc { object CliFunc {
def spanToId: Span.S ~> Id = new (Span.S ~> Id) { private val spanToId: Span.S ~> Id = new (Span.S ~> Id) {
override def apply[A](span: Span.S[A]): Id[A] = span.extract override def apply[A](span: Span.S[A]): Id[A] = span.extract
} }
def fromString(func: String): ValidatedNel[String, CliFunc] = { def fromString(func: String): ValidatedNec[String, CliFunc] = {
CallArrowToken.callArrow CallArrowToken.callArrow
.parseAll(func.trim) .parseAll(func.trim)
.toValidated .leftMap(error =>
.leftMap( NonEmptyChain
_.expected.map(_.context.mkString("\n")) .fromNonEmptyList(error.expected)
.map(_.context.mkString("\n"))
) )
.toValidated
.map(_.mapK(spanToId)) .map(_.mapK(spanToId))
.andThen(expr => .andThen(expr =>
expr.args.traverse { expr.args.traverse {
case LiteralToken(value, ts) => case LiteralToken(value, ts) =>
LiteralRaw(value, ts).valid LiteralRaw(value, ts).validNec
case VarToken(name) => case VarToken(name) =>
VarRaw(name.value, BottomType).valid VarRaw(name.value, BottomType).validNec
case CollectionToken(_, values) => case CollectionToken(_, values) =>
values.traverse { values.traverse {
case LiteralToken(value, ts) => case LiteralToken(value, ts) =>
@ -53,11 +55,11 @@ object CliFunc {
.map(l => CollectionRaw(l, ArrayType(l.head.baseType))) .map(l => CollectionRaw(l, ArrayType(l.head.baseType)))
.getOrElse(ValueRaw.Nil) .getOrElse(ValueRaw.Nil)
) )
.toValidatedNel .toValidatedNec
case CallArrowToken(_, _, _) => case CallArrowToken(_, _, _) =>
"Function calls as arguments are not supported.".invalidNel "Function calls as arguments are not supported.".invalidNec
case _ => case _ =>
"Unsupported argument.".invalidNel "Unsupported argument.".invalidNec
}.map(args => CliFunc(expr.funcName.value, args)) }.map(args => CliFunc(expr.funcName.value, args))
) )
} }

View File

@ -1,7 +1,7 @@
package aqua.run package aqua.run
import aqua.ErrorRendering.showError import aqua.Rendering.given
import aqua.compiler.{AquaCompiler, AquaCompilerConf, CompilerAPI} import aqua.compiler.{AquaCompiler, AquaCompilerConf, CompileResult, CompilerAPI}
import aqua.files.{AquaFileSources, FileModuleId} import aqua.files.{AquaFileSources, FileModuleId}
import aqua.{AquaIO, SpanParser} import aqua.{AquaIO, SpanParser}
import aqua.io.{AquaFileError, AquaPath, PackagePath, Prelude} import aqua.io.{AquaFileError, AquaPath, PackagePath, Prelude}
@ -21,6 +21,9 @@ import cats.syntax.monad.*
import cats.syntax.show.* import cats.syntax.show.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.option.* import cats.syntax.option.*
import cats.syntax.either.*
import cats.syntax.validated.*
import cats.syntax.apply.*
import fs2.io.file.{Files, Path} import fs2.io.file.{Files, Path}
import scribe.Logging import scribe.Logging
@ -32,51 +35,48 @@ class FuncCompiler[F[_]: Files: AquaIO: Async](
transformConfig: TransformConfig transformConfig: TransformConfig
) extends Logging { ) extends Logging {
type Result = [A] =>> CompileResult[FileModuleId, AquaFileError, FileSpan.F][A]
private def compileToContext( private def compileToContext(
path: Path, path: Path,
imports: List[Path], imports: List[Path],
config: AquaCompilerConf = AquaCompilerConf(transformConfig.constantsList) config: AquaCompilerConf = AquaCompilerConf(transformConfig.constantsList)
) = { ): F[Result[Chain[AquaContext]]] = {
val sources = new AquaFileSources[F](path, imports) val sources = new AquaFileSources[F](path, imports)
CompilerAPI CompilerAPI.compileToContext[F, AquaFileError, FileModuleId, FileSpan.F](
.compileToContext[F, AquaFileError, FileModuleId, FileSpan.F](
sources, sources,
SpanParser.parser, SpanParser.parser,
config config
) )
.map(_.leftMap(_.map(_.show)))
} }
private def compileBuiltins() = { private def compileBuiltins(): F[Result[Chain[AquaContext]]] =
for { for {
path <- PackagePath.builtin.getPath() path <- PackagePath.builtin.getPath()
context <- compileToContext(path, Nil) context <- compileToContext(path, Nil)
} yield { } yield context
context
}
}
// Compile and get only one function // Compile and get only one function
def compile( def compile(
preludeImports: List[Path] = Nil, preludeImports: List[Path] = Nil,
withBuiltins: Boolean = false withBuiltins: Boolean = false
): F[ValidatedNec[String, Chain[AquaContext]]] = { ): F[Result[Chain[AquaContext]]] = {
for { for {
// compile builtins and add it to context // compile builtins and add it to context
builtinsV <- builtinsV <-
if (withBuiltins) compileBuiltins() if (withBuiltins) compileBuiltins()
else validNec[String, Chain[AquaContext]](Chain.empty).pure[F] else Chain.empty.pure[Result].pure[F]
compileResult <- input.map { ap => compileResult <- input.traverse { ap =>
// compile only context to wrap and call function later // compile only context to wrap and call function later
Clock[F].timed( Clock[F].timed(
ap.getPath().flatMap(p => compileToContext(p, preludeImports ++ imports)) ap.getPath().flatMap(p => compileToContext(p, preludeImports ++ imports))
) )
}.getOrElse((Duration.Zero, validNec[String, Chain[AquaContext]](Chain.empty)).pure[F]) }
(compileTime, contextV) = compileResult (compileTime, contextV) = compileResult.orEmpty
} yield { } yield {
logger.debug(s"Compile time: ${compileTime.toMillis}ms") logger.debug(s"Compile time: ${compileTime.toMillis}ms")
// add builtins to the end of context // add builtins to the end of context
contextV.andThen(c => builtinsV.map(bc => c ++ bc)) (contextV, builtinsV).mapN(_ ++ _)
} }
} }
} }

View File

@ -2,4 +2,9 @@ package aqua.compiler
import aqua.backend.Generated import aqua.backend.Generated
case class AquaCompiled[I](sourceId: I, compiled: Seq[Generated], funcsCount: Int, servicesCount: Int) case class AquaCompiled[+I](
sourceId: I,
compiled: Seq[Generated],
funcsCount: Int,
servicesCount: Int
)

View File

@ -1,5 +1,6 @@
package aqua.compiler package aqua.compiler
import aqua.compiler.AquaError.{ParserError as AquaParserError, *}
import aqua.backend.Backend import aqua.backend.Backend
import aqua.linker.{AquaModule, Linker, Modules} import aqua.linker.{AquaModule, Linker, Modules}
import aqua.model.AquaContext import aqua.model.AquaContext
@ -10,6 +11,7 @@ import aqua.raw.{RawContext, RawPart}
import aqua.res.AquaRes import aqua.res.AquaRes
import aqua.semantics.{CompilerState, Semantics} import aqua.semantics.{CompilerState, Semantics}
import aqua.semantics.header.{HeaderHandler, HeaderSem, Picker} import aqua.semantics.header.{HeaderHandler, HeaderSem, Picker}
import aqua.semantics.{SemanticError, SemanticWarning}
import cats.data.* import cats.data.*
import cats.data.Validated.{validNec, Invalid, Valid} import cats.data.Validated.{validNec, Invalid, Valid}
@ -20,7 +22,9 @@ import cats.syntax.functor.*
import cats.syntax.monoid.* import cats.syntax.monoid.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.semigroup.* import cats.syntax.semigroup.*
import cats.syntax.either.*
import cats.{~>, Comonad, Functor, Monad, Monoid, Order} import cats.{~>, Comonad, Functor, Monad, Monoid, Order}
import cats.arrow.FunctionK
import scribe.Logging import scribe.Logging
class AquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad, C: Monoid: Picker]( class AquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad, C: Monoid: Picker](
@ -31,71 +35,115 @@ class AquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad, C: Monoid: Picker](
type Err = AquaError[I, E, S] type Err = AquaError[I, E, S]
type Ctx = NonEmptyMap[I, C] type Ctx = NonEmptyMap[I, C]
type ValidatedCtx = ValidatedNec[Err, Ctx] type CompileWarns = [A] =>> CompileWarnings[S][A]
type ValidatedCtxT = ValidatedCtx => ValidatedCtx type CompileRes = [A] =>> CompileResult[I, E, S][A]
type CompiledCtx = CompileRes[Ctx]
type CompiledCtxT = CompiledCtx => CompiledCtx
private def linkModules( private def linkModules(
modules: Modules[ modules: Modules[I, Err, CompiledCtxT],
I, cycleError: Linker.DepCycle[AquaModule[I, Err, CompiledCtxT]] => Err
Err, ): CompileRes[Map[I, C]] = {
ValidatedCtxT
],
cycleError: Linker.DepCycle[AquaModule[I, Err, ValidatedCtxT]] => Err
): ValidatedNec[Err, Map[I, ValidatedCtx]] = {
logger.trace("linking modules...") logger.trace("linking modules...")
Linker
.link(
modules,
cycleError,
// By default, provide an empty context for this module's id // By default, provide an empty context for this module's id
i => validNec(NonEmptyMap.one(i, Monoid.empty[C])) val empty: I => CompiledCtx = i => NonEmptyMap.one(i, Monoid[C].empty).pure[CompileRes]
for {
linked <- Linker
.link(modules, cycleError, empty)
.toEither
.toEitherT[CompileWarns]
res <- EitherT(
linked.toList.traverse { case (id, ctx) =>
ctx
.map(
/**
* NOTE: This should be safe
* as result for id should contain itself
*/
_.apply(id).map(id -> _).get
) )
.toValidated
}.map(_.sequence.toEither)
)
} yield res.toMap
} }
def compileRaw( def compileRaw(
sources: AquaSources[F, E, I], sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]] parser: I => String => ValidatedNec[ParserError[S], Ast[S]]
): F[Validated[NonEmptyChain[Err], Map[I, ValidatedCtx]]] = { ): F[CompileRes[Map[I, C]]] = {
logger.trace("starting resolving sources...") logger.trace("starting resolving sources...")
new AquaParser[F, E, I, S](sources, parser) new AquaParser[F, E, I, S](sources, parser)
.resolve[ValidatedCtx](mod => .resolve[CompiledCtx](mod =>
context => context =>
for {
// Context with prepared imports // Context with prepared imports
context.andThen { ctx => ctx <- context
val imports = mod.imports.view imports = mod.imports.flatMap { case (fn, id) =>
.mapValues(ctx(_)) ctx.apply(id).map(fn -> _)
.collect { case (fn, Some(fc)) => fn -> fc } }
.toMap header = mod.body.head
val header = mod.body.head headerSem <- headerHandler
// To manage imports, exports run HeaderHandler .sem(imports, header)
headerHandler .toCompileRes
.sem(
imports,
header
)
.andThen { headerSem =>
// Analyze the body, with prepared initial context // Analyze the body, with prepared initial context
logger.trace("semantic processing...") _ = logger.trace("semantic processing...")
semantics processed <- semantics
.process( .process(
mod.body, mod.body,
headerSem.initCtx headerSem.initCtx
) )
.toCompileRes
// Handle exports, declares - finalize the resulting context // Handle exports, declares - finalize the resulting context
.andThen { ctx => rc <- headerSem
headerSem.finCtx(ctx) .finCtx(processed)
} .toCompileRes
.map { rc => NonEmptyMap.one(mod.id, rc) } /**
} * Here we build a map of contexts while processing modules.
// The whole chain returns a semantics error finally * Should not linker provide this info inside this process?
.leftMap(_.map[Err](CompileError(_))) * Building this map complicates things a lot.
} */
} yield NonEmptyMap.one(mod.id, rc)
) )
.map( .value
_.andThen { modules => linkModules(modules, cycle => CycleError[I, E, S](cycle.map(_.id))) } .map(resolved =>
for {
modules <- resolved.toEitherT[CompileWarns]
linked <- linkModules(
modules,
cycle => CycleError(cycle.map(_.id))
)
} yield linked
) )
} }
private val warningsK: semantics.Warnings ~> CompileWarns =
new FunctionK[semantics.Warnings, CompileWarns] {
override def apply[A](
fa: semantics.Warnings[A]
): CompileWarns[A] =
fa.mapWritten(_.map(AquaWarning.CompileWarning.apply))
}
extension (res: semantics.ProcessResult) {
def toCompileRes: CompileRes[C] =
res
.leftMap(_.map(CompileError.apply))
.mapK(warningsK)
}
extension [A](res: ValidatedNec[SemanticError[S], A]) {
def toCompileRes: CompileRes[A] =
res.toEither
.leftMap(_.map(CompileError.apply))
.toEitherT[CompileWarns]
}
} }

View File

@ -1,20 +1,20 @@
package aqua.compiler package aqua.compiler
import aqua.parser.ParserError import aqua.parser
import aqua.parser.lexer.Token import aqua.parser.lexer.Token
import aqua.semantics.SemanticError import aqua.semantics
import cats.data.NonEmptyChain import cats.data.NonEmptyChain
trait AquaError[I, E, S[_]] enum AquaError[+I, +E, S[_]] {
case class SourcesErr[I, E, S[_]](err: E) extends AquaError[I, E, S] case SourcesError(err: E)
case class ParserErr[I, E, S[_]](err: ParserError[S]) extends AquaError[I, E, S] case ParserError(err: parser.ParserError[S])
case class ResolveImportsErr[I, E, S[_]](fromFile: I, token: Token[S], err: E) case ResolveImportsError(fromFile: I, token: Token[S], err: E)
extends AquaError[I, E, S] case ImportError(token: Token[S])
case class ImportErr[I, E, S[_]](token: Token[S]) extends AquaError[I, E, S] case CycleError(modules: NonEmptyChain[I])
case class CycleError[I, E, S[_]](modules: NonEmptyChain[I]) extends AquaError[I, E, S] case CompileError(err: semantics.SemanticError[S])
case OutputError(compiled: AquaCompiled[I], err: E)
case class CompileError[I, E, S[_]](err: SemanticError[S]) extends AquaError[I, E, S] case AirValidationError(errors: NonEmptyChain[String])
case class OutputError[I, E, S[_]](compiled: AquaCompiled[I], err: E) extends AquaError[I, E, S] }
case class AirValidationError[I, E, S[_]](errors: NonEmptyChain[String]) extends AquaError[I, E, S]

View File

@ -1,16 +1,20 @@
package aqua.compiler package aqua.compiler
import aqua.compiler.AquaError.{ParserError as AquaParserError, *}
import aqua.linker.{AquaModule, Modules} import aqua.linker.{AquaModule, Modules}
import aqua.parser.head.{FilenameExpr, ImportExpr} import aqua.parser.head.{FilenameExpr, ImportExpr}
import aqua.parser.lift.{LiftParser, Span} import aqua.parser.lift.{LiftParser, Span}
import aqua.parser.{Ast, ParserError} import aqua.parser.{Ast, ParserError}
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.data.{Chain, EitherNec, EitherT, NonEmptyChain, Validated, ValidatedNec}
import cats.parse.Parser0 import cats.parse.Parser0
import cats.syntax.either.*
import cats.syntax.applicative.* import cats.syntax.applicative.*
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
import cats.syntax.functor.* import cats.syntax.functor.*
import cats.syntax.monad.* import cats.syntax.monad.*
import cats.syntax.foldable.* import cats.syntax.foldable.*
import cats.syntax.traverse.*
import cats.syntax.validated.* import cats.syntax.validated.*
import cats.data.Chain.* import cats.data.Chain.*
import cats.data.Validated.* import cats.data.Validated.*
@ -19,46 +23,43 @@ import cats.{~>, Comonad, Monad}
import scribe.Logging import scribe.Logging
// TODO: add tests // TODO: add tests
class AquaParser[F[_], E, I, S[_]: Comonad]( class AquaParser[F[_]: Monad, E, I, S[_]: Comonad](
sources: AquaSources[F, E, I], sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]] parser: I => String => ValidatedNec[ParserError[S], Ast[S]]
)(implicit F: Monad[F]) ) extends Logging {
extends Logging {
type Body = Ast[S] type Body = Ast[S]
type Err = AquaError[I, E, S] type Err = AquaError[I, E, S]
private type FE[A] = EitherT[F, NonEmptyChain[Err], A]
// Parse all the source files // Parse all the source files
def parseSources: F[ValidatedNec[Err, Chain[(I, Body)]]] = private def parseSources: F[ValidatedNec[Err, Chain[(I, Body)]]] =
sources.sources.map( sources.sources.map(
_.leftMap(_.map[Err](SourcesErr(_))).andThen(_.map { case (i, s) => _.leftMap(_.map(SourcesError.apply)).andThen(
parser(i)(s) _.traverse { case (i, s) =>
.bimap( parser(i)(s).bimap(
_.map[Err](ParserErr(_)), _.map(AquaParserError.apply),
ast => Chain.one(i -> ast) ast => i -> ast
)
}
) )
}.foldA)
) )
// Resolve imports (not parse, just resolve) of the given file // Resolve imports (not parse, just resolve) of the given file
def resolveImports(id: I, ast: Body): F[ValidatedNec[Err, AquaModule[I, Err, Body]]] = private def resolveImports(id: I, ast: Body): F[ValidatedNec[Err, AquaModule[I, Err, Body]]] =
ast.head.tailForced ast.collectHead { case fe: FilenameExpr[S] =>
.map(_.head) fe.fileValue -> fe.token
.collect { case fe: FilenameExpr[F] => }.value.traverse { case (filename, token) =>
F.map(
sources sources
.resolveImport(id, fe.fileValue) .resolveImport(id, filename)
)(
_.bimap(
_.map[Err](ResolveImportsErr(id, fe.filename, _)),
importId =>
Chain.one[(I, (String, Err))](importId -> (fe.fileValue, ImportErr(fe.filename)))
)
)
}
.sequence
.map( .map(
_.foldA.map { collected => _.bimap(
_.map(ResolveImportsError(id, token, _): Err),
importId => importId -> (filename, ImportError(token): Err)
)
)
}.map(_.sequence.map { collected =>
AquaModule[I, Err, Body]( AquaModule[I, Err, Body](
id, id,
// How filenames correspond to the resolved IDs // How filenames correspond to the resolved IDs
@ -71,25 +72,30 @@ class AquaParser[F[_], E, I, S[_]: Comonad](
}.toList.toMap[I, Err], }.toList.toMap[I, Err],
ast ast
) )
} })
)
// Parse sources, convert to modules // Parse sources, convert to modules
def sourceModules: F[ValidatedNec[Err, Modules[I, Err, Body]]] = private def sourceModules: F[ValidatedNec[Err, Modules[I, Err, Body]]] =
parseSources.flatMap { parseSources.flatMap {
case Validated.Valid(srcs) => case Validated.Valid(srcs) =>
srcs.traverse { case (id, ast) => srcs.traverse { case (id, ast) =>
resolveImports(id, ast).map(_.map(Chain.one)) resolveImports(id, ast)
}.map(_.foldA) }.map(_.sequence)
case Validated.Invalid(errs) => case Validated.Invalid(errs) =>
errs.invalid.pure[F] errs.invalid.pure[F]
}.map(_.map(_.foldLeft(Modules[I, Err, Body]())(_.add(_, toExport = true)))) }.map(
_.map(
_.foldLeft(Modules[I, Err, Body]())(
_.add(_, toExport = true)
)
)
)
def loadModule(imp: I): F[ValidatedNec[Err, AquaModule[I, Err, Body]]] = private def loadModule(imp: I): F[ValidatedNec[Err, AquaModule[I, Err, Body]]] =
sources sources
.load(imp) .load(imp)
.map(_.leftMap(_.map[Err](SourcesErr(_))).andThen { src => .map(_.leftMap(_.map(SourcesError.apply)).andThen { src =>
parser(imp)(src).leftMap(_.map[Err](ParserErr(_))) parser(imp)(src).leftMap(_.map(AquaParserError.apply))
}) })
.flatMap { .flatMap {
case Validated.Valid(ast) => case Validated.Valid(ast) =>
@ -98,18 +104,16 @@ class AquaParser[F[_], E, I, S[_]: Comonad](
errs.invalid.pure[F] errs.invalid.pure[F]
} }
def resolveModules( private def resolveModules(
modules: Modules[I, Err, Body] modules: Modules[I, Err, Body]
): F[ValidatedNec[Err, Modules[I, Err, Ast[S]]]] = ): F[ValidatedNec[Err, Modules[I, Err, Ast[S]]]] =
modules.dependsOn.map { case (moduleId, unresolvedErrors) => modules.dependsOn.toList.traverse { case (moduleId, unresolvedErrors) =>
loadModule(moduleId).map(_.leftMap(_ ++ unresolvedErrors)) loadModule(moduleId).map(_.leftMap(_ ++ unresolvedErrors))
}.toList.sequence }.map(
.map( _.sequence.map(
_.foldLeft(modules.validNec[Err]) { case (mods, m) => _.foldLeft(modules)(_ add _)
mods.andThen(ms => m.map(ms.add(_)))
}
) )
.flatMap { ).flatMap {
case Validated.Valid(ms) if ms.isResolved => case Validated.Valid(ms) if ms.isResolved =>
ms.validNec.pure[F] ms.validNec.pure[F]
case Validated.Valid(ms) => case Validated.Valid(ms) =>
@ -118,15 +122,19 @@ class AquaParser[F[_], E, I, S[_]: Comonad](
err.pure[F] err.pure[F]
} }
def resolveSources: F[ValidatedNec[Err, Modules[I, Err, Ast[S]]]] = private def resolveSources: FE[Modules[I, Err, Ast[S]]] =
sourceModules.flatMap { for {
case Validated.Valid(ms) => resolveModules(ms) ms <- EitherT(
case err => err.pure[F] sourceModules.map(_.toEither)
} )
res <- EitherT(
resolveModules(ms).map(_.toEither)
)
} yield res
def resolve[T]( def resolve[T](
transpile: AquaModule[I, Err, Body] => T => T transpile: AquaModule[I, Err, Body] => T => T
): F[ValidatedNec[Err, Modules[I, Err, T => T]]] = ): FE[Modules[I, Err, T => T]] =
resolveSources.map(_.map(_.mapModuleToBody(transpile))) resolveSources.map(_.mapModuleToBody(transpile))
} }

View File

@ -0,0 +1,7 @@
package aqua.compiler
import aqua.semantics
enum AquaWarning[S[_]] {
case CompileWarning(warning: semantics.SemanticWarning[S])
}

View File

@ -1,5 +1,6 @@
package aqua.compiler package aqua.compiler
import aqua.compiler.AquaError.*
import aqua.backend.{AirFunction, Backend} import aqua.backend.{AirFunction, Backend}
import aqua.linker.{AquaModule, Linker, Modules} import aqua.linker.{AquaModule, Linker, Modules}
import aqua.model.AquaContext import aqua.model.AquaContext
@ -10,8 +11,9 @@ import aqua.raw.{RawContext, RawPart}
import aqua.res.AquaRes import aqua.res.AquaRes
import aqua.semantics.header.{HeaderHandler, HeaderSem} import aqua.semantics.header.{HeaderHandler, HeaderSem}
import aqua.semantics.{CompilerState, RawSemantics, Semantics} import aqua.semantics.{CompilerState, RawSemantics, Semantics}
import cats.data.* import cats.data.*
import cats.data.Validated.{Invalid, Valid, invalid, validNec} import cats.data.Validated.{invalid, validNec, Invalid, Valid}
import cats.parse.Parser0 import cats.parse.Parser0
import cats.syntax.applicative.* import cats.syntax.applicative.*
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
@ -20,7 +22,8 @@ import cats.syntax.functor.*
import cats.syntax.monoid.* import cats.syntax.monoid.*
import cats.syntax.semigroup.* import cats.syntax.semigroup.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.{Comonad, Monad, Monoid, Order, ~>} import cats.syntax.either.*
import cats.{~>, Comonad, Monad, Monoid, Order}
import scribe.Logging import scribe.Logging
import scala.collection.MapView import scala.collection.MapView
@ -28,18 +31,13 @@ import scala.collection.MapView
object CompilerAPI extends Logging { object CompilerAPI extends Logging {
private def toAquaProcessed[I: Order, E, S[_]: Comonad]( private def toAquaProcessed[I: Order, E, S[_]: Comonad](
filesWithContext: Map[ filesWithContext: Map[I, RawContext]
I, ): Chain[AquaProcessed[I]] = {
ValidatedNec[AquaError[I, E, S], NonEmptyMap[I, RawContext]]
]
): ValidatedNec[AquaError[I, E, S], Chain[AquaProcessed[I]]] = {
logger.trace("linking finished") logger.trace("linking finished")
filesWithContext.values.toList filesWithContext.toList
// Gather all RawContext in List inside ValidatedNec
.flatTraverse(_.map(_.toNel.toList))
// Process all contexts maintaining Cache // Process all contexts maintaining Cache
.traverse(_.traverse { case (i, rawContext) => .traverse { case (i, rawContext) =>
for { for {
cache <- State.get[AquaContext.Cache] cache <- State.get[AquaContext.Cache]
_ = logger.trace(s"Going to prepare exports for $i...") _ = logger.trace(s"Going to prepare exports for $i...")
@ -47,25 +45,32 @@ object CompilerAPI extends Logging {
_ = logger.trace(s"AquaProcessed prepared for $i") _ = logger.trace(s"AquaProcessed prepared for $i")
_ <- State.set(expCache) _ <- State.set(expCache)
} yield AquaProcessed(i, exp) } yield AquaProcessed(i, exp)
}.runA(AquaContext.Cache())) }
.runA(AquaContext.Cache())
// Convert result List to Chain // Convert result List to Chain
.map(_.map(Chain.fromSeq)) .map(Chain.fromSeq)
.value .value
} }
private def getAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad]( private def getAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad](
config: AquaCompilerConf config: AquaCompilerConf
): AquaCompiler[F, E, I, S, RawContext] = { ): AquaCompiler[F, E, I, S, RawContext] = {
implicit val rc: Monoid[RawContext] = RawContext given Monoid[RawContext] = RawContext
.implicits( .implicits(
RawContext.blank RawContext.blank.copy(
.copy(parts = Chain.fromSeq(config.constantsList).map(const => RawContext.blank -> const)) parts = Chain
.fromSeq(config.constantsList)
.map(const => RawContext.blank -> const)
)
) )
.rawContextMonoid .rawContextMonoid
val semantics = new RawSemantics[S]() val semantics = new RawSemantics[S]()
new AquaCompiler[F, E, I, S, RawContext](new HeaderHandler[S, RawContext](), semantics) new AquaCompiler[F, E, I, S, RawContext](
new HeaderHandler(),
semantics
)
} }
// Get result generated by backend // Get result generated by backend
@ -75,75 +80,53 @@ object CompilerAPI extends Logging {
airValidator: AirValidator[F], airValidator: AirValidator[F],
backend: Backend.Transform, backend: Backend.Transform,
config: AquaCompilerConf config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], Chain[AquaCompiled[I]]]] = { ): F[CompileResult[I, E, S][Chain[AquaCompiled[I]]]] = {
val compiler = getAquaCompiler[F, E, I, S](config) val compiler = getAquaCompiler[F, E, I, S](config)
for { for {
compiledRaw <- compiler.compileRaw(sources, parser) compiledRaw <- compiler.compileRaw(sources, parser)
compiledV = compiledRaw.andThen(toAquaProcessed) compiledV = compiledRaw.map(toAquaProcessed)
_ <- airValidator.init() _ <- airValidator.init()
result <- compiledV.traverse { compiled => result <- compiledV.flatTraverse { compiled =>
compiled.traverse { ap => compiled.traverse { ap =>
logger.trace("generating output...") logger.trace("generating output...")
val res = backend.transform(ap.context) val res = backend.transform(ap.context)
val compiled = backend.generate(res) val generated = backend.generate(res)
airValidator val air = generated.toList.flatMap(_.air)
.validate( val compiled = AquaCompiled(
compiled.toList.flatMap(_.air) sourceId = ap.id,
compiled = generated,
funcsCount = res.funcs.length.toInt,
servicesCount = res.services.length.toInt
) )
airValidator
.validate(air)
.map( .map(
_.leftMap(errs => AirValidationError(errs): AquaError[I, E, S]) _.leftMap(errs => AirValidationError(errs): AquaError[I, E, S])
.as( .as(compiled)
AquaCompiled(ap.id, compiled, res.funcs.length.toInt, res.services.length.toInt)
)
.toValidatedNec .toValidatedNec
) )
}.map(_.sequence) }.map(_.sequence.toEither.toEitherT)
}.map(_.andThen(identity)) // There is no flatTraverse for Validated }
} yield result } yield result
} }
def compileTo[F[_]: Monad, E, I: Order, S[_]: Comonad, T](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
airValidator: AirValidator[F],
backend: Backend.Transform,
config: AquaCompilerConf,
write: AquaCompiled[I] => F[Seq[Validated[E, T]]]
): F[ValidatedNec[AquaError[I, E, S], Chain[T]]] =
compile[F, E, I, S](sources, parser, airValidator, backend, config)
.flatMap(
_.traverse(compiled =>
compiled.toList.flatTraverse { ac =>
write(ac).map(
_.toList.map(
_.bimap(
e => OutputError(ac, e): AquaError[I, E, S],
Chain.one
).toValidatedNec
)
)
}.map(_.foldA)
).map(_.andThen(identity)) // There is no flatTraverse for Validated
)
def compileToContext[F[_]: Monad, E, I: Order, S[_]: Comonad]( def compileToContext[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I], sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]], parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], Chain[AquaContext]]] = { ): F[CompileResult[I, E, S][Chain[AquaContext]]] = {
val compiler = getAquaCompiler[F, E, I, S](config) val compiler = getAquaCompiler[F, E, I, S](config)
compiler val compiledRaw = compiler.compileRaw(sources, parser)
.compileRaw(sources, parser)
.map(_.andThen { filesWithContext => compiledRaw.map(
toAquaProcessed(filesWithContext) _.map(toAquaProcessed)
}) .map(_.map { ap =>
.map(_.map { compiled =>
compiled.map { ap =>
logger.trace("generating output...") logger.trace("generating output...")
ap.context ap.context
}
}) })
)
} }
} }

View File

@ -0,0 +1,12 @@
package aqua
import cats.data.{Chain, EitherT, NonEmptyChain, Writer}
package object compiler {
type CompileWarnings[S[_]] =
[A] =>> Writer[Chain[AquaWarning[S]], A]
type CompileResult[I, E, S[_]] =
[A] =>> EitherT[CompileWarnings[S], NonEmptyChain[AquaError[I, E, S]], A]
}

View File

@ -30,6 +30,7 @@ import cats.data.{Chain, NonEmptyChain, NonEmptyMap, Validated, ValidatedNec}
import cats.instances.string.* import cats.instances.string.*
import cats.syntax.show.* import cats.syntax.show.*
import cats.syntax.option.* import cats.syntax.option.*
import cats.syntax.either.*
class AquaCompilerSpec extends AnyFlatSpec with Matchers { class AquaCompilerSpec extends AnyFlatSpec with Matchers {
import ModelBuilder.* import ModelBuilder.*
@ -59,8 +60,11 @@ class AquaCompilerSpec extends AnyFlatSpec with Matchers {
id => txt => Parser.parse(Parser.parserSchema)(txt), id => txt => Parser.parse(Parser.parserSchema)(txt),
AquaCompilerConf(ConstantRaw.defaultConstants(None)) AquaCompilerConf(ConstantRaw.defaultConstants(None))
) )
.value
.value
.toValidated
"aqua compiler" should "compile a simple snipped to the right context" in { "aqua compiler" should "compile a simple snippet to the right context" in {
val res = compileToContext( val res = compileToContext(
Map( Map(
@ -93,7 +97,6 @@ class AquaCompilerSpec extends AnyFlatSpec with Matchers {
val const = ctx.allValues.get("X") val const = ctx.allValues.get("X")
const.nonEmpty should be(true) const.nonEmpty should be(true)
const.get should be(LiteralModel.number(5)) const.get should be(LiteralModel.number(5))
} }
def through(peer: ValueModel) = def through(peer: ValueModel) =

View File

@ -1,24 +1,26 @@
package aqua package aqua
import aqua.compiler.AquaError.{ParserError as AquaParserError, *}
import aqua.compiler.* import aqua.compiler.*
import aqua.files.FileModuleId import aqua.files.FileModuleId
import aqua.io.AquaFileError import aqua.io.AquaFileError
import aqua.parser.lift.{FileSpan, Span} import aqua.parser.lift.{FileSpan, Span}
import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError} import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError}
import aqua.semantics.{HeaderError, RulesViolated, WrongAST} import aqua.semantics.{HeaderError, RulesViolated, SemanticWarning, WrongAST}
import cats.parse.LocationMap import cats.parse.LocationMap
import cats.parse.Parser.Expectation import cats.parse.Parser.Expectation
import cats.parse.Parser.Expectation.* import cats.parse.Parser.Expectation.*
import cats.{Eval, Show} import cats.{Eval, Show}
object ErrorRendering { object Rendering {
def showForConsole(errorType: String, span: FileSpan, messages: List[String]): String = def showForConsole(messageType: String, span: FileSpan, messages: List[String]): String =
span span
.focus(3) .focus(3)
.map( .map(
_.toConsoleStr( _.toConsoleStr(
errorType, messageType,
messages, messages,
Console.RED Console.RED
) )
@ -29,8 +31,18 @@ object ErrorRendering {
) )
) + Console.RESET + "\n" ) + Console.RESET + "\n"
implicit val showError: Show[AquaError[FileModuleId, AquaFileError, FileSpan.F]] = Show.show { given Show[AquaWarning[FileSpan.F]] = Show.show { case AquaWarning.CompileWarning(warning) =>
case ParserErr(err) => warning match {
case SemanticWarning(token, hints) =>
token.unit._1
.focus(0)
.map(_.toConsoleStr("Warning", hints, Console.YELLOW))
.getOrElse("(Dup warning, but offset is beyond the script)")
}
}
given Show[AquaError[FileModuleId, AquaFileError, FileSpan.F]] = Show.show {
case AquaParserError(err) =>
err match { err match {
case BlockIndentError(indent, message) => case BlockIndentError(indent, message) =>
showForConsole("Syntax error", indent._1, message :: Nil) showForConsole("Syntax error", indent._1, message :: Nil)
@ -63,15 +75,15 @@ object ErrorRendering {
.reverse .reverse
.mkString("\n") .mkString("\n")
} }
case SourcesErr(err) => case SourcesError(err) =>
Console.RED + err.showForConsole + Console.RESET Console.RED + err.showForConsole + Console.RESET
case AirValidationError(errors) => case AirValidationError(errors) =>
Console.RED + errors.toChain.toList.mkString("\n") + Console.RESET Console.RED + errors.toChain.toList.mkString("\n") + Console.RESET
case ResolveImportsErr(_, token, err) => case ResolveImportsError(_, token, err) =>
val span = token.unit._1 val span = token.unit._1
showForConsole("Cannot resolve imports", span, err.showForConsole :: Nil) showForConsole("Cannot resolve imports", span, err.showForConsole :: Nil)
case ImportErr(token) => case ImportError(token) =>
val span = token.unit._1 val span = token.unit._1
showForConsole("Cannot resolve import", span, "Cannot resolve import" :: Nil) showForConsole("Cannot resolve import", span, "Cannot resolve import" :: Nil)
case CycleError(modules) => case CycleError(modules) =>

View File

@ -1,6 +1,8 @@
package aqua.lsp package aqua.lsp
import aqua.compiler.* import aqua.compiler.*
import aqua.compiler.AquaError.{ParserError as AquaParserError, *}
import aqua.compiler.AquaWarning.*
import aqua.files.{AquaFileSources, AquaFilesIO, FileModuleId} import aqua.files.{AquaFileSources, AquaFilesIO, FileModuleId}
import aqua.io.* import aqua.io.*
import aqua.parser.lexer.{LiteralToken, Token} import aqua.parser.lexer.{LiteralToken, Token}
@ -8,27 +10,29 @@ import aqua.parser.lift.FileSpan.F
import aqua.parser.lift.{FileSpan, Span} import aqua.parser.lift.{FileSpan, Span}
import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError} import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError}
import aqua.raw.ConstantRaw import aqua.raw.ConstantRaw
import aqua.semantics.{HeaderError, RulesViolated, WrongAST} import aqua.semantics.{HeaderError, RulesViolated, SemanticWarning, WrongAST}
import aqua.{AquaIO, SpanParser} import aqua.{AquaIO, SpanParser}
import cats.data.Validated.{Invalid, Valid, invalidNec, validNec}
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.data.{NonEmptyChain, Validated} import cats.data.{NonEmptyChain, Validated}
import cats.effect.IO import cats.effect.IO
import cats.syntax.option.*
import cats.effect.unsafe.implicits.global import cats.effect.unsafe.implicits.global
import fs2.io.file.{Files, Path} import fs2.io.file.{Files, Path}
import scribe.Logging import scribe.Logging
import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.Future import scala.concurrent.Future
import scala.scalajs.js import scala.scalajs.js
import scala.scalajs.js.JSConverters.* import scala.scalajs.js.JSConverters.*
import scala.scalajs.js.annotation.* import scala.scalajs.js.annotation.*
import scala.scalajs.js.{UndefOr, undefined} import scala.scalajs.js.{undefined, UndefOr}
@JSExportAll @JSExportAll
case class CompilationResult( case class CompilationResult(
errors: js.Array[ErrorInfo], errors: js.Array[ErrorInfo],
locations: js.Array[TokenLink], warnings: js.Array[WarningInfo] = js.Array(),
importLocations: js.Array[TokenImport] locations: js.Array[TokenLink] = js.Array(),
importLocations: js.Array[TokenImport] = js.Array()
) )
@JSExportAll @JSExportAll
@ -72,12 +76,25 @@ object ErrorInfo {
} }
} }
@JSExportAll
case class WarningInfo(start: Int, end: Int, message: String, location: UndefOr[String])
object WarningInfo {
def apply(fileSpan: FileSpan, message: String): WarningInfo = {
val start = fileSpan.span.startIndex
val end = fileSpan.span.endIndex
WarningInfo(start, end, message, fileSpan.name)
}
}
@JSExportTopLevel("AquaLSP") @JSExportTopLevel("AquaLSP")
object AquaLSP extends App with Logging { object AquaLSP extends App with Logging {
def errorToInfo(error: AquaError[FileModuleId, AquaFileError, FileSpan.F]): List[ErrorInfo] = { private def errorToInfo(
error match { error: AquaError[FileModuleId, AquaFileError, FileSpan.F]
case ParserErr(err) => ): List[ErrorInfo] = error match {
case AquaParserError(err) =>
err match { err match {
case BlockIndentError(indent, message) => case BlockIndentError(indent, message) =>
ErrorInfo(indent._1, message) :: Nil ErrorInfo(indent._1, message) :: Nil
@ -98,11 +115,11 @@ object AquaLSP extends App with Logging {
.map(_._2) .map(_._2)
.reverse .reverse
} }
case SourcesErr(err) => case SourcesError(err) =>
ErrorInfo.applyOp(0, 0, err.showForConsole, None) :: Nil ErrorInfo.applyOp(0, 0, err.showForConsole, None) :: Nil
case ResolveImportsErr(_, token, err) => case ResolveImportsError(_, token, err) =>
ErrorInfo(token.unit._1, err.showForConsole) :: Nil ErrorInfo(token.unit._1, err.showForConsole) :: Nil
case ImportErr(token) => case ImportError(token) =>
ErrorInfo(token.unit._1, "Cannot resolve import") :: Nil ErrorInfo(token.unit._1, "Cannot resolve import") :: Nil
case CycleError(modules) => case CycleError(modules) =>
ErrorInfo.applyOp( ErrorInfo.applyOp(
@ -123,7 +140,15 @@ object AquaLSP extends App with Logging {
} }
case OutputError(_, err) => case OutputError(_, err) =>
ErrorInfo.applyOp(0, 0, err.showForConsole, None) :: Nil ErrorInfo.applyOp(0, 0, err.showForConsole, None) :: Nil
case AirValidationError(errors) =>
errors.toChain.toList.map(ErrorInfo.applyOp(0, 0, _, None))
} }
private def warningToInfo(
warning: AquaWarning[FileSpan.F]
): List[WarningInfo] = warning match {
case CompileWarning(SemanticWarning(token, messages)) =>
WarningInfo(token.unit._1, messages.mkString("\n")) :: Nil
} }
@JSExport @JSExport
@ -133,7 +158,7 @@ object AquaLSP extends App with Logging {
): scalajs.js.Promise[CompilationResult] = { ): scalajs.js.Promise[CompilationResult] = {
logger.debug(s"Compiling '$pathStr' with imports: $imports") logger.debug(s"Compiling '$pathStr' with imports: $imports")
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO] given AquaIO[IO] = new AquaFilesIO[IO]
val path = Path(pathStr) val path = Path(pathStr)
val pathId = FileModuleId(path) val pathId = FileModuleId(path)
@ -141,32 +166,15 @@ object AquaLSP extends App with Logging {
val config = AquaCompilerConf(ConstantRaw.defaultConstants(None)) val config = AquaCompilerConf(ConstantRaw.defaultConstants(None))
val proc = for { val proc = for {
res <- LSPCompiler.compileToLsp[IO, AquaFileError, FileModuleId, FileSpan.F](
res <- LSPCompiler
.compileToLsp[IO, AquaFileError, FileModuleId, FileSpan.F](
sources, sources,
SpanParser.parser, SpanParser.parser,
config config
) )
} yield { } yield {
val fileRes: Validated[NonEmptyChain[ val fileRes = res.andThen(
AquaError[FileModuleId, AquaFileError, FileSpan.F] _.get(pathId).toValidNec(
], LspContext[FileSpan.F]] = res SourcesError(Unresolvable(s"Unexpected. No file $pathStr in compiler results"))
.andThen(
_.getOrElse(
pathId,
invalidNec(
SourcesErr(Unresolvable(s"Unexpected. No file $pathStr in compiler results"))
)
)
)
.andThen(
_.get(pathId)
.map(l => validNec(l))
.getOrElse(
invalidNec(
SourcesErr(Unresolvable(s"Unexpected. No file $pathStr in compiler results"))
)
) )
) )
@ -176,16 +184,13 @@ object AquaLSP extends App with Logging {
locations: List[(Token[FileSpan.F], Token[FileSpan.F])] locations: List[(Token[FileSpan.F], Token[FileSpan.F])]
): js.Array[TokenLink] = { ): js.Array[TokenLink] = {
locations.flatMap { case (from, to) => locations.flatMap { case (from, to) =>
val fromOp = TokenLocation.fromSpan(from.unit._1) val fromOp = TokenLocation.fromSpan(from.unit._1)
val toOp = TokenLocation.fromSpan(to.unit._1) val toOp = TokenLocation.fromSpan(to.unit._1)
val link = for { val link = for {
from <- fromOp from <- fromOp
to <- toOp to <- toOp
} yield { } yield TokenLink(from, to)
TokenLink(from, to)
}
if (link.isEmpty) if (link.isEmpty)
logger.warn(s"Incorrect coordinates for token '${from.unit._1.name}'") logger.warn(s"Incorrect coordinates for token '${from.unit._1.name}'")
@ -204,6 +209,7 @@ object AquaLSP extends App with Logging {
val result = fileRes match { val result = fileRes match {
case Valid(lsp) => case Valid(lsp) =>
val errors = lsp.errors.map(CompileError.apply).flatMap(errorToInfo) val errors = lsp.errors.map(CompileError.apply).flatMap(errorToInfo)
val warnings = lsp.warnings.map(CompileWarning.apply).flatMap(warningToInfo)
errors match errors match
case Nil => case Nil =>
logger.debug("No errors on compilation.") logger.debug("No errors on compilation.")
@ -212,13 +218,14 @@ object AquaLSP extends App with Logging {
CompilationResult( CompilationResult(
errors.toJSArray, errors.toJSArray,
warnings.toJSArray,
locationsToJs(lsp.locations), locationsToJs(lsp.locations),
importsToTokenImport(lsp.importTokens) importsToTokenImport(lsp.importTokens)
) )
case Invalid(e: NonEmptyChain[AquaError[FileModuleId, AquaFileError, FileSpan.F]]) => case Invalid(e) =>
val errors = e.toNonEmptyList.toList.flatMap(errorToInfo) val errors = e.toChain.toList.flatMap(errorToInfo)
logger.debug("Errors: " + errors.mkString("\n")) logger.debug("Errors: " + errors.mkString("\n"))
CompilationResult(errors.toJSArray, List.empty.toJSArray, List.empty.toJSArray) CompilationResult(errors.toJSArray)
} }
result result
} }

View File

@ -7,6 +7,7 @@ import aqua.lsp.LSPCompiler
import aqua.parser.lift.FileSpan import aqua.parser.lift.FileSpan
import aqua.raw.ConstantRaw import aqua.raw.ConstantRaw
import aqua.{AquaIO, SpanParser} import aqua.{AquaIO, SpanParser}
import cats.data.Validated import cats.data.Validated
import cats.effect.{IO, IOApp, Sync} import cats.effect.{IO, IOApp, Sync}
import fs2.io.file.Path import fs2.io.file.Path
@ -31,9 +32,9 @@ object Test extends IOApp.Simple {
) )
.map { .map {
case Validated.Invalid(errs) => case Validated.Invalid(errs) =>
errs.map(System.err.println): Unit errs.toChain.toList.foreach(System.err.println)
case Validated.Valid(res) => case Validated.Valid(res) =>
res.map(println): Unit res.foreach(println)
} }
_ <- IO.println("Compilation ends in: " + (System.currentTimeMillis() - start) + " ms") _ <- IO.println("Compilation ends in: " + (System.currentTimeMillis() - start) + " ms")
} yield () } yield ()

View File

@ -4,6 +4,7 @@ import aqua.compiler.{AquaCompiler, AquaCompilerConf, AquaError, AquaSources}
import aqua.parser.{Ast, ParserError} import aqua.parser.{Ast, ParserError}
import aqua.raw.RawContext import aqua.raw.RawContext
import aqua.semantics.header.{HeaderHandler, HeaderSem} import aqua.semantics.header.{HeaderHandler, HeaderSem}
import cats.data.Validated.validNec import cats.data.Validated.validNec
import cats.syntax.semigroup.* import cats.syntax.semigroup.*
import cats.syntax.applicative.* import cats.syntax.applicative.*
@ -11,6 +12,7 @@ import cats.syntax.flatMap.*
import cats.syntax.functor.* import cats.syntax.functor.*
import cats.syntax.monoid.* import cats.syntax.monoid.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.either.*
import cats.{Comonad, Monad, Monoid, Order} import cats.{Comonad, Monad, Monoid, Order}
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec} import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
@ -19,21 +21,21 @@ object LSPCompiler {
private def getLspAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad]( private def getLspAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad](
config: AquaCompilerConf config: AquaCompilerConf
): AquaCompiler[F, E, I, S, LspContext[S]] = { ): AquaCompiler[F, E, I, S, LspContext[S]] = {
implicit val rc: Monoid[LspContext[S]] = LspContext given Monoid[LspContext[S]] = LspContext
.implicits( .implicits(
LspContext LspContext.blank.copy(raw =
.blank[S] RawContext.blank.copy(
.copy(raw = parts = Chain
RawContext.blank.copy(parts = .fromSeq(config.constantsList)
Chain.fromSeq(config.constantsList).map(const => RawContext.blank -> const) .map(const => RawContext.blank -> const)
) )
) )
) )
.lspContextMonoid .lspContextMonoid
implicit val headerSemMonoid: Monoid[HeaderSem[S, LspContext[S]]] = given Monoid[HeaderSem[S, LspContext[S]]] with {
new Monoid[HeaderSem[S, LspContext[S]]] { override def empty: HeaderSem[S, LspContext[S]] =
override def empty: HeaderSem[S, LspContext[S]] = HeaderSem(rc.empty, (c, _) => validNec(c)) HeaderSem(Monoid[LspContext[S]].empty, (c, _) => validNec(c))
override def combine( override def combine(
a: HeaderSem[S, LspContext[S]], a: HeaderSem[S, LspContext[S]],
@ -48,28 +50,23 @@ object LSPCompiler {
val semantics = new LspSemantics[S]() val semantics = new LspSemantics[S]()
new AquaCompiler[F, E, I, S, LspContext[S]](new HeaderHandler[S, LspContext[S]](), semantics) new AquaCompiler[F, E, I, S, LspContext[S]](
new HeaderHandler(),
semantics
)
} }
def compileToLsp[F[_]: Monad, E, I: Order, S[_]: Comonad]( def compileToLsp[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I], sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]], parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf config: AquaCompilerConf
): F[Validated[NonEmptyChain[AquaError[I, E, S]], Map[I, Validated[NonEmptyChain[ ): F[ValidatedNec[AquaError[I, E, S], Map[I, LspContext[S]]]] = {
AquaError[I, E, S]
], Map[I, LspContext[S]]]]]] = {
val compiler = getLspAquaCompiler[F, E, I, S](config) val compiler = getLspAquaCompiler[F, E, I, S](config)
compiler compiler
.compileRaw(sources, parser) .compileRaw(sources, parser)
.map { v => // NOTE: Ignore warnings here as
v.map { innerMap => // they are collected inside context
innerMap.view.mapValues { vCtx => .map(_.value.value.toValidated)
vCtx.map {
_.toSortedMap.toMap
}
}.toMap
}
}
} }
} }

View File

@ -2,16 +2,15 @@ package aqua.lsp
import aqua.parser.lexer.Token import aqua.parser.lexer.Token
import aqua.semantics.rules.StackInterpreter import aqua.semantics.rules.StackInterpreter
import aqua.semantics.rules.errors.ReportErrors
import aqua.semantics.rules.locations.{LocationsAlgebra, LocationsState} import aqua.semantics.rules.locations.{LocationsAlgebra, LocationsState}
import cats.data.State import cats.data.State
import monocle.Lens import monocle.Lens
import monocle.macros.GenLens import monocle.macros.GenLens
import scribe.Logging import scribe.Logging
class LocationsInterpreter[S[_], X](implicit class LocationsInterpreter[S[_], X](using
lens: Lens[X, LocationsState[S]], lens: Lens[X, LocationsState[S]]
error: ReportErrors[S, X]
) extends LocationsAlgebra[S, State[X, *]] with Logging { ) extends LocationsAlgebra[S, State[X, *]] with Logging {
type SX[A] = State[X, A] type SX[A] = State[X, A]
@ -20,7 +19,7 @@ class LocationsInterpreter[S[_], X](implicit
GenLens[LocationsState[S]](_.stack) GenLens[LocationsState[S]](_.stack)
) )
import stack.{getState, mapStackHead, modify, report} import stack.*
override def addToken(name: String, token: Token[S]): State[X, Unit] = modify { st => override def addToken(name: String, token: Token[S]): State[X, Unit] = modify { st =>
st.copy(tokens = st.tokens.updated(name, token)) st.copy(tokens = st.tokens.updated(name, token))

View File

@ -1,11 +1,11 @@
package aqua.lsp package aqua.lsp
import aqua.parser.lexer.{LiteralToken, NamedTypeToken, Token} import aqua.parser.lexer.{LiteralToken, NamedTypeToken, Token}
import aqua.raw.RawContext.semiRC
import aqua.raw.{RawContext, RawPart} import aqua.raw.{RawContext, RawPart}
import aqua.semantics.SemanticError import aqua.semantics.{SemanticError, SemanticWarning}
import aqua.semantics.header.Picker import aqua.semantics.header.Picker
import aqua.types.{ArrowType, Type} import aqua.types.{ArrowType, Type}
import cats.syntax.monoid.* import cats.syntax.monoid.*
import cats.{Monoid, Semigroup} import cats.{Monoid, Semigroup}
@ -18,14 +18,15 @@ case class LspContext[S[_]](
tokens: Map[String, Token[S]] = Map.empty[String, Token[S]], tokens: Map[String, Token[S]] = Map.empty[String, Token[S]],
locations: List[(Token[S], Token[S])] = Nil, locations: List[(Token[S], Token[S])] = Nil,
importTokens: List[LiteralToken[S]] = Nil, importTokens: List[LiteralToken[S]] = Nil,
errors: List[SemanticError[S]] = Nil errors: List[SemanticError[S]] = Nil,
warnings: List[SemanticWarning[S]] = Nil
) )
object LspContext { object LspContext {
def blank[S[_]]: LspContext[S] = LspContext[S](raw = RawContext()) def blank[S[_]]: LspContext[S] = LspContext[S](raw = RawContext())
implicit def semiLsp[S[_]]: Semigroup[LspContext[S]] = given [S[_]]: Semigroup[LspContext[S]] =
(x: LspContext[S], y: LspContext[S]) => (x: LspContext[S], y: LspContext[S]) =>
LspContext[S]( LspContext[S](
raw = x.raw |+| y.raw, raw = x.raw |+| y.raw,
@ -33,20 +34,22 @@ object LspContext {
rootArrows = x.rootArrows ++ y.rootArrows, rootArrows = x.rootArrows ++ y.rootArrows,
constants = x.constants ++ y.constants, constants = x.constants ++ y.constants,
locations = x.locations ++ y.locations, locations = x.locations ++ y.locations,
tokens = x.tokens ++ y.tokens tokens = x.tokens ++ y.tokens,
errors = x.errors ++ y.errors,
warnings = x.warnings ++ y.warnings
) )
trait Implicits[S[_]] { trait Implicits[S[_]] {
implicit val lspContextMonoid: Monoid[LspContext[S]] val lspContextMonoid: Monoid[LspContext[S]]
} }
def implicits[S[_]](init: LspContext[S]): Implicits[S] = new Implicits[S] { def implicits[S[_]](init: LspContext[S]): Implicits[S] = new Implicits[S] {
override implicit val lspContextMonoid: Monoid[LspContext[S]] = new Monoid[LspContext[S]] { override val lspContextMonoid: Monoid[LspContext[S]] = new Monoid[LspContext[S]] {
override def empty: LspContext[S] = init override def empty: LspContext[S] = init
override def combine(x: LspContext[S], y: LspContext[S]): LspContext[S] = { override def combine(x: LspContext[S], y: LspContext[S]): LspContext[S] = {
semiLsp[S].combine(x, y) Semigroup[LspContext[S]].combine(x, y)
} }
} }

View File

@ -3,15 +3,16 @@ package aqua.lsp
import aqua.parser.Ast import aqua.parser.Ast
import aqua.parser.head.{ImportExpr, ImportFromExpr, UseExpr, UseFromExpr} import aqua.parser.head.{ImportExpr, ImportFromExpr, UseExpr, UseFromExpr}
import aqua.parser.lexer.{LiteralToken, Token} import aqua.parser.lexer.{LiteralToken, Token}
import aqua.semantics.rules.errors.ReportErrors
import aqua.semantics.rules.locations.LocationsState import aqua.semantics.rules.locations.LocationsState
import aqua.semantics.{CompilerState, RawSemantics, RulesViolated, SemanticError, Semantics} import aqua.semantics.{CompilerState, RawSemantics, SemanticError, SemanticWarning, Semantics}
import cats.data.Validated.{Invalid, Valid} import cats.data.Validated.{Invalid, Valid}
import cats.syntax.applicative.* import cats.syntax.applicative.*
import cats.syntax.apply.* import cats.syntax.apply.*
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
import cats.syntax.functor.* import cats.syntax.functor.*
import cats.syntax.foldable.* import cats.syntax.foldable.*
import cats.syntax.either.*
import cats.syntax.reducible.* import cats.syntax.reducible.*
import cats.data.{NonEmptyChain, ValidatedNec} import cats.data.{NonEmptyChain, ValidatedNec}
import monocle.Lens import monocle.Lens
@ -19,22 +20,23 @@ import monocle.macros.GenLens
class LspSemantics[S[_]] extends Semantics[S, LspContext[S]] { class LspSemantics[S[_]] extends Semantics[S, LspContext[S]] {
def getImportTokens(ast: Ast[S]): List[LiteralToken[S]] = { private def getImportTokens(ast: Ast[S]): List[LiteralToken[S]] =
ast.head.foldLeft[List[LiteralToken[S]]](Nil) { case (l, header) => ast.collectHead {
header match { case ImportExpr(fn) => fn
case ImportExpr(fn) => l :+ fn case ImportFromExpr(_, fn) => fn
case ImportFromExpr(_, fn) => l :+ fn case UseExpr(fn, _) => fn
case UseExpr(fn, _) => l :+ fn case UseFromExpr(_, fn, _) => fn
case UseFromExpr(_, fn, _) => l :+ fn }.value.toList
case _ => l
}
}
}
/**
* Process the AST and return the semantics result.
* NOTE: LspSemantics never return errors or warnings,
* they are collected in LspContext.
*/
def process( def process(
ast: Ast[S], ast: Ast[S],
init: LspContext[S] init: LspContext[S]
): ValidatedNec[SemanticError[S], LspContext[S]] = { ): ProcessResult = {
val rawState = CompilerState.init[S](init.raw) val rawState = CompilerState.init[S](init.raw)
@ -53,22 +55,15 @@ class LspSemantics[S[_]] extends Semantics[S, LspContext[S]] {
val importTokens = getImportTokens(ast) val importTokens = getImportTokens(ast)
implicit val ls: Lens[CompilerState[S], LocationsState[S]] = given Lens[CompilerState[S], LocationsState[S]] =
GenLens[CompilerState[S]](_.locations) GenLens[CompilerState[S]](_.locations)
import monocle.syntax.all.* given LocationsInterpreter[S, CompilerState[S]] =
implicit val re: ReportErrors[S, CompilerState[S]] =
(st: CompilerState[S], token: Token[S], hints: List[String]) =>
st.focus(_.errors).modify(_.append(RulesViolated(token, hints)))
implicit val locationsInterpreter: LocationsInterpreter[S, CompilerState[S]] =
new LocationsInterpreter[S, CompilerState[S]]() new LocationsInterpreter[S, CompilerState[S]]()
RawSemantics RawSemantics
.interpret(ast, initState, init.raw) .interpret(ast, initState, init.raw)
.map { case (state, ctx) => .map { case (state, ctx) =>
// TODO: better to change return type in `process` method
Valid(
LspContext( LspContext(
raw = ctx, raw = ctx,
rootArrows = state.names.rootArrows, rootArrows = state.names.rootArrows,
@ -77,9 +72,9 @@ class LspSemantics[S[_]] extends Semantics[S, LspContext[S]] {
locations = state.locations.allLocations, locations = state.locations.allLocations,
importTokens = importTokens, importTokens = importTokens,
tokens = state.locations.tokens, tokens = state.locations.tokens,
errors = state.errors.toList errors = state.errors.toList,
) warnings = state.warnings.toList
) ).pure[Result]
} }
// TODO: return as Eval // TODO: return as Eval
.value .value

View File

@ -23,8 +23,16 @@ export interface ErrorInfo {
location: string | null location: string | null
} }
export interface WarningInfo {
start: number,
end: number,
message: string,
location: string | null
}
export interface CompilationResult { export interface CompilationResult {
errors: ErrorInfo[], errors: ErrorInfo[],
warnings: WarningInfo[],
locations: TokenLink[], locations: TokenLink[],
importLocations: TokenImport[] importLocations: TokenImport[]
} }

View File

@ -105,7 +105,7 @@ object Linker extends Logging {
val importKeys = m.dependsOn.keySet val importKeys = m.dependsOn.keySet
logger.debug(s"${m.id} dependsOn $importKeys") logger.debug(s"${m.id} dependsOn $importKeys")
val deps: T => T = val deps: T => T =
importKeys.map(acc).foldLeft[T => T](identity) { case (fAcc, f) => importKeys.map(acc).foldLeft(identity[T]) { case (fAcc, f) =>
logger.debug("COMBINING ONE TIME ") logger.debug("COMBINING ONE TIME ")
t => { t => {
logger.debug(s"call combine $t") logger.debug(s"call combine $t")
@ -132,7 +132,10 @@ object Linker extends Logging {
else { else {
val result = iter(modules.loaded.values.toList, Map.empty, cycleError) val result = iter(modules.loaded.values.toList, Map.empty, cycleError)
result.map(_.collect { case (i, f) if modules.exports(i) => i -> f(empty(i)) }) result.map(_.collect {
case (i, f) if modules.exports(i) =>
i -> f(empty(i))
})
} }
} }

View File

@ -114,7 +114,7 @@ case class RawContext(
object RawContext { object RawContext {
val blank: RawContext = RawContext() val blank: RawContext = RawContext()
implicit val semiRC: Semigroup[RawContext] = given Semigroup[RawContext] =
(x: RawContext, y: RawContext) => (x: RawContext, y: RawContext) =>
RawContext( RawContext(
x.init.flatMap(xi => y.init.map(xi |+| _)) orElse x.init orElse y.init, x.init.flatMap(xi => y.init.map(xi |+| _)) orElse x.init orElse y.init,
@ -126,16 +126,16 @@ object RawContext {
) )
trait Implicits { trait Implicits {
implicit val rawContextMonoid: Monoid[RawContext] val rawContextMonoid: Monoid[RawContext]
} }
def implicits(init: RawContext): Implicits = new Implicits { def implicits(init: RawContext): Implicits = new Implicits {
override implicit val rawContextMonoid: Monoid[RawContext] = new Monoid[RawContext] { override val rawContextMonoid: Monoid[RawContext] = new Monoid[RawContext] {
override def empty: RawContext = init override def empty: RawContext = init
override def combine(x: RawContext, y: RawContext): RawContext = override def combine(x: RawContext, y: RawContext): RawContext =
semiRC.combine(x, y) Semigroup[RawContext].combine(x, y)
} }
} }

View File

@ -7,6 +7,7 @@ import aqua.parser.lift.LiftParser.*
import aqua.helpers.Tree import aqua.helpers.Tree
import cats.data.{Chain, Validated, ValidatedNec} import cats.data.{Chain, Validated, ValidatedNec}
import cats.syntax.flatMap.*
import cats.free.Cofree import cats.free.Cofree
import cats.{Comonad, Eval} import cats.{Comonad, Eval}
import cats.~> import cats.~>
@ -19,6 +20,14 @@ case class Ast[S[_]](head: Ast.Head[S], tree: Ast.Tree[S]) {
def cataHead[T](folder: (HeaderExpr[S], Chain[T]) => Eval[T]): Eval[T] = def cataHead[T](folder: (HeaderExpr[S], Chain[T]) => Eval[T]): Eval[T] =
Cofree.cata[Chain, HeaderExpr[S], T](head)(folder) Cofree.cata[Chain, HeaderExpr[S], T](head)(folder)
def collectHead[T](pf: PartialFunction[HeaderExpr[S], T]): Eval[Chain[T]] =
cataHead((e, acc: Chain[Chain[T]]) =>
Eval.later {
val flatAcc = acc.flatten
if (pf.isDefinedAt(e)) flatAcc :+ pf(e) else flatAcc
}
)
} }
object Ast { object Ast {

View File

@ -9,8 +9,14 @@ import scala.language.implicitConversions
// TODO: move FileSpan to another package? // TODO: move FileSpan to another package?
case class FileSpan(name: String, locationMap: Eval[LocationMap], span: Span) { case class FileSpan(name: String, locationMap: Eval[LocationMap], span: Span) {
/**
* Focus on the line pointed by the span
*
* @param ctx How many lines to capture before and after the line
* @return FileSpan.Focus
*/
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.value, ctx).map(FileSpan.Focus(name, locationMap, ctx, _))
} }
object FileSpan { object FileSpan {
@ -18,12 +24,12 @@ object FileSpan {
case class Focus(name: String, locationMap: Eval[LocationMap], ctx: Int, spanFocus: Span.Focus) { case class Focus(name: String, locationMap: Eval[LocationMap], ctx: Int, spanFocus: Span.Focus) {
def toConsoleStr( def toConsoleStr(
errorType: String, messageType: String,
msgs: List[String], msgs: List[String],
onLeft: String, onLeft: String,
onRight: String = Console.RESET onRight: String = Console.RESET
): String = ): String =
onLeft + "---- " + errorType + ": " + s"$name:${spanFocus.line._1 + 1}:${spanFocus.column + 1}" + onRight + onLeft + "---- " + messageType + ": " + s"$name:${spanFocus.focus.number + 1}:${spanFocus.column + 1}" + onRight +
spanFocus.toConsoleStr( spanFocus.toConsoleStr(
msgs, msgs,
onLeft, onLeft,

View File

@ -1,86 +1,127 @@
package aqua.parser.lift package aqua.parser.lift
import cats.data.NonEmptyList import cats.data.NonEmptyList
import cats.parse.{LocationMap, Parser0, Parser as P} import cats.parse.{LocationMap, Parser as P, Parser0}
import cats.{Comonad, Eval} import cats.{Comonad, Eval}
import scala.language.implicitConversions import scala.language.implicitConversions
case class Span(startIndex: Int, endIndex: Int) { case class Span(startIndex: Int, endIndex: Int) {
def focus(locationMap: Eval[LocationMap], ctx: Int): Option[Span.Focus] = { /**
val map = locationMap.value * Focus on the line pointed by the span
map.toLineCol(startIndex).flatMap { case (line, column) => *
map * @param locationMap Locations Map
.getLine(line) * @param ctx how many lines to capture before and after the line
.map { l => * @return Span.Focus
val pre = */
(Math.max(0, line - ctx) until line).map(i => map.getLine(i).map(i -> _)).toList.flatten def focus(locationMap: LocationMap, ctx: Int): Option[Span.Focus] =
val linePos = { for {
val (l1, l2) = l.splitAt(column) lineCol <- locationMap.toLineCol(startIndex)
val (lc, l3) = l2.splitAt(endIndex - startIndex) (lineNum, columnNum) = lineCol
(line, l1, lc, l3) line <- locationMap.getLine(lineNum)
} focused = Span.focus(line, columnNum, endIndex - startIndex)
val post = pre = Span.getLines(locationMap, lineNum - ctx, lineNum)
((line + 1) to (line + ctx)).map(i => map.getLine(i).map(i -> _)).toList.flatten post = Span.getLines(locationMap, lineNum + 1, lineNum + ctx + 1)
Span.Focus( } yield Span.Focus(
pre, pre,
linePos, focused.numbered(lineNum),
post, post,
column columnNum
) )
} }
}
}
}
object Span { object Span {
private def getLines(
locationMap: LocationMap,
from: Int,
to: Int
): List[NumberedLine[String]] =
(from until to)
.map(i =>
locationMap
.getLine(i)
.map(NumberedLine(i, _))
)
.toList
.flatten
private def focus(
str: String,
idx: Int,
len: Int
): FocusedLine = FocusedLine(
str.substring(0, idx),
str.substring(idx, idx + len),
str.substring(idx + len)
)
final case class NumberedLine[T](
number: Int,
line: T
)
final case class FocusedLine(
pre: String,
focus: String,
post: String
) {
def numbered(n: Int): NumberedLine[FocusedLine] =
NumberedLine(n, this)
}
case class Focus( case class Focus(
pre: List[(Int, String)], pre: List[NumberedLine[String]],
line: (Int, String, String, String), focus: NumberedLine[FocusedLine],
post: List[(Int, String)], post: List[NumberedLine[String]],
column: Int column: Int
) { ) {
private lazy val lastN = post.lastOption.map(_._1).getOrElse(line._1) + 1 private lazy val lastN = post.lastOption.map(_.number).getOrElse(focus.number) + 1
private lazy val lastNSize = lastN.toString.length private lazy val lastNSize = lastN.toString.length
private def formatLine(l: (Int, String), onLeft: String, onRight: String) = private def formatLine(l: NumberedLine[String], onLeft: String, onRight: String) =
formatLN(l._1, onLeft, onRight) + l._2 formatLN(l.number, onLeft, onRight) + l.line
private def formatLN(ln: Int, onLeft: String, onRight: String) = { private def formatLN(ln: Int, onLeft: String, onRight: String) = {
val s = (ln + 1).toString val s = (ln + 1).toString
onLeft + s + (" " * (lastNSize - s.length)) + onRight + " " onLeft + s + (" " * (lastNSize - s.length)) + onRight + " "
} }
/**
* Format the focus for console output
*
* @param msgs Messages to display
* @param onLeft Control sequence to put on the left
* @param onRight Control sequence to put on the right
*/
def toConsoleStr( def toConsoleStr(
msgs: List[String], msgs: List[String],
onLeft: String, onLeft: String,
onRight: String = Console.RESET onRight: String = Console.RESET
): String = { ): String = {
val line3Length = line._3.length val focusLength = focus.line.focus.length
val line3Mult = if (line3Length == 0) 1 else line3Length val focusMult = if (focusLength == 0) 1 else focusLength
val message = msgs.map(m => (" " * (line._2.length + lastNSize + 1)) + m).mkString("\n") val message = msgs
.map(m => (" " * (focus.line.focus.length + lastNSize + 1)) + m)
.mkString("\n")
pre.map(formatLine(_, onLeft, onRight)).mkString("\n") + pre.map(formatLine(_, onLeft, onRight)).mkString("\n") +
"\n" + "\n" +
formatLN(line._1, onLeft, onRight) + formatLN(focus.number, onLeft, onRight) +
line._2 + focus.line.pre +
onLeft + onLeft + focus.line.focus + onRight +
line._3 + focus.line.post +
onRight +
line._4 +
"\n" + "\n" +
(" " * (line._2.length + lastNSize + 1)) + (" " * (focus.line.pre.length + lastNSize + 1)) +
onLeft + onLeft +
("^" * line3Mult) + ("^" * focusMult) +
("=" * line._4.length) + ("=" * focus.line.post.length) +
onRight + onRight +
"\n" + "\n" +
onLeft + onLeft + message + onRight +
message +
onRight +
"\n" + "\n" +
post.map(formatLine(_, onLeft, onRight)).mkString("\n") post.map(formatLine(_, onLeft, onRight)).mkString("\n")
} }
@ -104,7 +145,6 @@ object Span {
def lift0: Parser0[Span.S[T]] = Span.spanLiftParser.lift0(p) def lift0: Parser0[Span.S[T]] = Span.spanLiftParser.lift0(p)
} }
implicit object spanLiftParser extends LiftParser[S] { implicit object spanLiftParser extends LiftParser[S] {
override def lift[T](p: P[T]): P[S[T]] = override def lift[T](p: P[T]): P[S[T]] =

View File

@ -9,7 +9,7 @@ import aqua.semantics.rules.locations.LocationsState
import aqua.semantics.rules.names.NamesState import aqua.semantics.rules.names.NamesState
import aqua.semantics.rules.types.TypesState import aqua.semantics.rules.types.TypesState
import aqua.semantics.rules.mangler.ManglerState import aqua.semantics.rules.mangler.ManglerState
import aqua.semantics.rules.errors.ReportErrors import aqua.semantics.rules.report.ReportState
import cats.Semigroup import cats.Semigroup
import cats.data.{Chain, State} import cats.data.{Chain, State}
@ -19,14 +19,18 @@ import monocle.Lens
import monocle.macros.GenLens import monocle.macros.GenLens
case class CompilerState[S[_]]( case class CompilerState[S[_]](
errors: Chain[SemanticError[S]] = Chain.empty[SemanticError[S]], report: ReportState[S] = ReportState[S](),
mangler: ManglerState = ManglerState(), mangler: ManglerState = ManglerState(),
names: NamesState[S] = NamesState[S](), names: NamesState[S] = NamesState[S](),
abilities: AbilitiesState[S] = AbilitiesState[S](), abilities: AbilitiesState[S] = AbilitiesState[S](),
types: TypesState[S] = TypesState[S](), types: TypesState[S] = TypesState[S](),
definitions: DefinitionsState[S] = DefinitionsState[S](), definitions: DefinitionsState[S] = DefinitionsState[S](),
locations: LocationsState[S] = LocationsState[S]() locations: LocationsState[S] = LocationsState[S]()
) ) {
lazy val errors: Chain[SemanticError[S]] = report.errors
lazy val warnings: Chain[SemanticWarning[S]] = report.warnings
}
object CompilerState { object CompilerState {
type St[S[_]] = State[CompilerState[S], Raw] type St[S[_]] = State[CompilerState[S], Raw]
@ -38,6 +42,9 @@ object CompilerState {
types = TypesState.init[F](ctx) types = TypesState.init[F](ctx)
) )
given [S[_]]: Lens[CompilerState[S], ReportState[S]] =
GenLens[CompilerState[S]](_.report)
given [S[_]]: Lens[CompilerState[S], NamesState[S]] = given [S[_]]: Lens[CompilerState[S], NamesState[S]] =
GenLens[CompilerState[S]](_.names) GenLens[CompilerState[S]](_.names)
@ -53,18 +60,6 @@ object CompilerState {
given [S[_]]: Lens[CompilerState[S], DefinitionsState[S]] = given [S[_]]: Lens[CompilerState[S], DefinitionsState[S]] =
GenLens[CompilerState[S]](_.definitions) GenLens[CompilerState[S]](_.definitions)
given [S[_]]: ReportErrors[S, CompilerState[S]] =
new ReportErrors[S, CompilerState[S]] {
import monocle.syntax.all.*
override def apply(
st: CompilerState[S],
token: Token[S],
hints: List[String]
): CompilerState[S] =
st.focus(_.errors).modify(_.append(RulesViolated(token, hints)))
}
given [S[_]]: Monoid[St[S]] with { given [S[_]]: Monoid[St[S]] with {
override def empty: St[S] = State.pure(Raw.Empty("compiler state monoid empty")) override def empty: St[S] = State.pure(Raw.Empty("compiler state monoid empty"))
@ -73,7 +68,7 @@ object CompilerState {
b <- y.get b <- y.get
_ <- State.set( _ <- State.set(
CompilerState[S]( CompilerState[S](
a.errors ++ b.errors, a.report |+| b.report,
a.mangler |+| b.mangler, a.mangler |+| b.mangler,
a.names |+| b.names, a.names |+| b.names,
a.abilities |+| b.abilities, a.abilities |+| b.abilities,

View File

@ -0,0 +1,361 @@
package aqua.semantics
import aqua.errors.Errors.internalError
import aqua.raw.ops.*
import aqua.semantics.rules.abilities.{AbilitiesAlgebra, AbilitiesInterpreter, AbilitiesState}
import aqua.semantics.rules.definitions.{DefinitionsAlgebra, DefinitionsInterpreter}
import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra}
import aqua.semantics.rules.names.{NamesAlgebra, NamesInterpreter}
import aqua.semantics.rules.mangler.{ManglerAlgebra, ManglerInterpreter}
import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter}
import aqua.semantics.rules.report.{ReportAlgebra, ReportInterpreter}
import aqua.semantics.header.Picker
import aqua.semantics.header.Picker.*
import aqua.raw.{Raw, RawContext, RawPart}
import aqua.parser.{Ast, Expr}
import aqua.parser.lexer.{LiteralToken, Token}
import cats.{Eval, Monad}
import cats.data.{Chain, EitherT, NonEmptyChain, State, StateT, ValidatedNec, Writer}
import cats.syntax.applicative.*
import cats.syntax.option.*
import cats.syntax.apply.*
import cats.syntax.flatMap.*
import cats.syntax.functor.*
import cats.syntax.foldable.*
import cats.syntax.reducible.*
import cats.syntax.traverse.*
import cats.syntax.semigroup.*
import scribe.Logging
class RawSemantics[S[_]](using
Picker[RawContext]
) extends Semantics[S, RawContext] {
override def process(
ast: Ast[S],
init: RawContext
): ProcessResult = {
given LocationsAlgebra[S, State[CompilerState[S], *]] =
new DummyLocationsInterpreter[S, CompilerState[S]]()
RawSemantics
.interpret(ast, CompilerState.init(init), init)
.map { case (state, ctx) =>
EitherT(
Writer
.tell(state.warnings)
.as(
NonEmptyChain
.fromChain(state.errors)
.toLeft(ctx)
)
)
}
// TODO: return as Eval
.value
}
}
object RawSemantics extends Logging {
/**
* [[RawTag.Tree]] with [[Token]] used for error reporting
*/
private final case class RawTagWithToken[S[_]](
tree: RawTag.Tree,
token: Token[S]
) {
lazy val tag: RawTag = tree.head
private def modifyTree(f: RawTag.Tree => RawTag.Tree): RawTagWithToken[S] =
copy(tree = f(tree))
/**
* Wrap tail of @param next in [[SeqTag]]
* and append it to current tree tail
*/
def append(next: RawTagWithToken[S]): RawTagWithToken[S] = modifyTree(tree =>
tree.copy(
tail = (
tree.tail,
// SeqTag.wrap will return single node as is
next.tree.tail.map(SeqTag.wrap)
).mapN(_ :+ _)
)
)
def wrapIn(tag: GroupTag): RawTagWithToken[S] = modifyTree(tree => tag.wrap(tree))
def toRaw: RawWithToken[S] = RawWithToken(FuncOp(tree), token)
}
private def elseWithoutIf[S[_], G[_]](
token: Token[S]
)(using report: ReportAlgebra[S, G]): G[Unit] =
report.error(token, "Unexpected `else` without `if`" :: Nil)
private def catchWithoutTry[S[_], G[_]](
token: Token[S]
)(using report: ReportAlgebra[S, G]): G[Unit] =
report.error(token, "Unexpected `catch` without `try`" :: Nil)
private def otherwiseWithoutPrev[S[_], G[_]](
token: Token[S]
)(using report: ReportAlgebra[S, G]): G[Unit] =
report.error(token, "Unexpected `otherwise` without previous instruction" :: Nil)
private def parWithoutPrev[S[_], G[_]](
token: Token[S]
)(using report: ReportAlgebra[S, G]): G[Unit] =
report.error(token, "Unexpected `par` without previous instruction" :: Nil)
/**
* Optionally combine two [[RawTag.Tree]] into one.
* Used to combine `if` and `else`,
* `try` and `catch` (`otherwise`);
* to create [[ParTag]] from `par`,
* [[TryTag]] from `otherwise`
*
* @param prev Previous tag
* @param next Next tag
* @param E Algebra for error reporting
* @return [[Some]] with result of combination
* [[None]] if tags should not be combined
* or error occuried
*/
private def rawTagCombine[S[_], G[_]: Monad](
prev: RawTagWithToken[S],
next: RawTagWithToken[S]
)(using E: ReportAlgebra[S, G]): G[Option[RawTagWithToken[S]]] =
(prev.tag, next.tag) match {
case (_: IfTag, IfTag.Else) =>
prev.append(next).some.pure
case (_, IfTag.Else) | (IfTag.Else, _) =>
val token = prev.tag match {
case IfTag.Else => prev.token
case _ => next.token
}
elseWithoutIf(token).as(none)
case (TryTag, TryTag.Catch) =>
prev.append(next).some.pure
case (_, TryTag.Catch) | (TryTag.Catch, _) =>
val token = prev.tag match {
case TryTag.Catch => prev.token
case _ => next.token
}
catchWithoutTry(token).as(none)
case (TryTag.Otherwise, _) =>
otherwiseWithoutPrev(prev.token).as(none)
case (TryTag, TryTag.Otherwise) =>
prev.append(next).some.pure
case (_, TryTag.Otherwise) =>
prev
.wrapIn(TryTag)
.append(next)
.some
.pure
case (ParTag.Par, _) =>
parWithoutPrev(prev.token).as(none)
case (ParTag, ParTag.Par) =>
prev.append(next).some.pure
case (_, ParTag.Par) =>
prev
.wrapIn(ParTag)
.append(next)
.some
.pure
case _ => none.pure
}
/**
* Check if tag is valid to be single
*
* @param single tag
* @param E Algebra for error reporting
* @return [[Some]] if tag is valid to be single
* [[None]] otherwise
*/
private def rawTagSingleCheck[S[_], G[_]: Monad](
single: RawTagWithToken[S]
)(using E: ReportAlgebra[S, G]): G[Option[RawTagWithToken[S]]] =
single.tag match {
case IfTag.Else => elseWithoutIf(single.token).as(none)
case TryTag.Catch => catchWithoutTry(single.token).as(none)
case TryTag.Otherwise => otherwiseWithoutPrev(single.token).as(none)
case ParTag.Par => parWithoutPrev(single.token).as(none)
case _ => single.some.pure
}
/**
* [[Raw]] with [[Token]] used for error reporting
*/
private final case class RawWithToken[S[_]](
raw: Raw,
token: Token[S]
) {
def toTag: Option[RawTagWithToken[S]] =
raw match {
case FuncOp(tree) => RawTagWithToken(tree, token).some
case _ => none
}
}
/**
* State for folding [[Raw]] results of children
*
* @param last Last seen [[Raw]] with [[Token]]
* @param acc All previous [[Raw]]
*/
private final case class InnersFoldState[S[_]](
last: Option[RawWithToken[S]] = None,
acc: Chain[Raw] = Chain.empty
) {
/**
* Process new incoming [[Raw]]
*/
def step[G[_]: Monad](
next: RawWithToken[S]
)(using ReportAlgebra[S, G]): G[InnersFoldState[S]] =
last.fold(copy(last = next.some).pure)(prev =>
(prev.toTag, next.toTag)
.traverseN(rawTagCombine)
.map(
_.flatten.fold(
// No combination - just update last and acc
copy(
last = next.some,
acc = prev.raw +: acc
)
)(combined =>
// Result of combination is the new last
copy(
last = combined.toRaw.some
)
)
)
)
/**
* Produce result of folding
*/
def result[G[_]: Monad](using
ReportAlgebra[S, G]
): G[Option[Raw]] =
if (acc.isEmpty)
// Hack to report error if single tag in block is incorrect
last.flatTraverse(single =>
single.toTag.fold(single.raw.some.pure)(singleTag =>
for {
checked <- rawTagSingleCheck(singleTag)
maybeRaw = checked.map(_.toRaw.raw)
} yield maybeRaw
)
)
else
last
.fold(acc)(_.raw +: acc)
.reverse
.reduceLeftOption(_ |+| _)
.pure
}
private def folder[S[_], G[_]: Monad](using
A: AbilitiesAlgebra[S, G],
N: NamesAlgebra[S, G],
T: TypesAlgebra[S, G],
D: DefinitionsAlgebra[S, G],
L: LocationsAlgebra[S, G],
E: ReportAlgebra[S, G]
): (Expr[S], Chain[G[RawWithToken[S]]]) => Eval[G[RawWithToken[S]]] = (expr, inners) =>
Eval later ExprSem
.getProg[S, G](expr)
.apply(for {
children <- inners.sequence
resultState <- children
.traverse(raw => StateT.modifyF((state: InnersFoldState[S]) => state.step(raw)))
.runS(InnersFoldState())
result <- resultState.result
} yield result.getOrElse(Raw.empty("AST is empty")))
.map(raw => RawWithToken(raw, expr.token))
type Interpreter[S[_], A] = State[CompilerState[S], A]
def transpile[S[_]](ast: Ast[S])(using
LocationsAlgebra[S, Interpreter[S, *]]
): Interpreter[S, Raw] = {
given ReportAlgebra[S, Interpreter[S, *]] =
new ReportInterpreter[S, CompilerState[S]]
given TypesAlgebra[S, Interpreter[S, *]] =
new TypesInterpreter[S, CompilerState[S]]
given ManglerAlgebra[Interpreter[S, *]] =
new ManglerInterpreter[CompilerState[S]]
given AbilitiesAlgebra[S, Interpreter[S, *]] =
new AbilitiesInterpreter[S, CompilerState[S]]
given NamesAlgebra[S, Interpreter[S, *]] =
new NamesInterpreter[S, CompilerState[S]]
given DefinitionsAlgebra[S, Interpreter[S, *]] =
new DefinitionsInterpreter[S, CompilerState[S]]
ast
.cata(folder[S, Interpreter[S, *]])
.value
.map(_.raw)
}
private def astToState[S[_]](ast: Ast[S])(using
locations: LocationsAlgebra[S, Interpreter[S, *]]
): Interpreter[S, Raw] =
transpile[S](ast)
// If there are any errors, they're inside CompilerState[S]
def interpret[S[_]](
ast: Ast[S],
initState: CompilerState[S],
init: RawContext
)(using
LocationsAlgebra[S, Interpreter[S, *]]
): Eval[(CompilerState[S], RawContext)] =
astToState[S](ast)
.run(initState)
.map {
case (state, _: Raw.Empty) =>
// No `parts`, but has `init`
(
state,
RawContext.blank.copy(
init = Some(init.copy(module = init.module.map(_ + "|init")))
.filter(_ != RawContext.blank)
)
)
case (state, part: (RawPart | RawPart.Parts)) =>
state -> RawPart
.contextPart(part)
.parts
.foldLeft(
RawContext.blank.copy(
init = Some(init.copy(module = init.module.map(_ + "|init")))
.filter(_ != RawContext.blank)
)
) { case (ctx, p) =>
ctx.copy(parts = ctx.parts :+ (ctx -> p))
}
case (_, m) =>
internalError(
s"Unexpected Raw ($m)"
)
}
}

View File

@ -0,0 +1,8 @@
package aqua.semantics
import aqua.parser.lexer.Token
final case class SemanticWarning[S[_]](
token: Token[S],
hints: List[String]
)

View File

@ -1,370 +1,27 @@
package aqua.semantics package aqua.semantics
import aqua.errors.Errors.internalError import aqua.parser.Ast
import aqua.parser.head.{HeadExpr, HeaderExpr, ImportExpr, ImportFromExpr} import aqua.semantics.SemanticError
import aqua.parser.lexer.{LiteralToken, Token}
import aqua.parser.{Ast, Expr}
import aqua.raw.ops.{FuncOp, SeqGroupTag}
import aqua.raw.{Raw, RawContext, RawPart}
import aqua.semantics.header.Picker
import aqua.semantics.header.Picker.*
import aqua.semantics.rules.abilities.{AbilitiesAlgebra, AbilitiesInterpreter, AbilitiesState}
import aqua.semantics.rules.definitions.{DefinitionsAlgebra, DefinitionsInterpreter}
import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra}
import aqua.semantics.rules.names.{NamesAlgebra, NamesInterpreter}
import aqua.semantics.rules.mangler.{ManglerAlgebra, ManglerInterpreter}
import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter}
import aqua.semantics.rules.errors.ReportErrors
import aqua.semantics.rules.errors.ErrorsAlgebra
import aqua.raw.ops.*
import cats.arrow.FunctionK import cats.data.{Chain, EitherNec, EitherT, NonEmptyChain, ValidatedNec, Writer}
import cats.data.*
import cats.Reducible
import cats.data.Chain.*
import cats.data.Validated.{Invalid, Valid}
import cats.kernel.Monoid
import cats.syntax.applicative.*
import cats.syntax.option.*
import cats.syntax.apply.*
import cats.syntax.flatMap.*
import cats.syntax.functor.*
import cats.syntax.foldable.*
import cats.syntax.reducible.*
import cats.syntax.traverse.*
import cats.free.CofreeInstances
import cats.syntax.semigroup.*
import cats.{Eval, Monad, Semigroup}
import monocle.Lens
import monocle.macros.GenLens
import scribe.{log, Logging}
import cats.free.Cofree
trait Semantics[S[_], C] { trait Semantics[S[_], C] {
final type Warnings = [A] =>> Writer[
Chain[SemanticWarning[S]],
A
]
final type Result = [A] =>> EitherT[
Warnings,
NonEmptyChain[SemanticError[S]],
A
]
final type ProcessResult = Result[C]
def process( def process(
ast: Ast[S], ast: Ast[S],
init: C init: C
): ValidatedNec[SemanticError[S], C] ): ProcessResult
}
class RawSemantics[S[_]](implicit p: Picker[RawContext]) extends Semantics[S, RawContext] {
def process(
ast: Ast[S],
init: RawContext
): ValidatedNec[SemanticError[S], RawContext] = {
implicit val locationsInterpreter: DummyLocationsInterpreter[S, CompilerState[S]] =
new DummyLocationsInterpreter[S, CompilerState[S]]()
RawSemantics
.interpret(ast, CompilerState.init(init), init)
.map { case (state, ctx) =>
NonEmptyChain
.fromChain(state.errors)
.toInvalid(ctx)
}
// TODO: return as Eval
.value
}
}
object RawSemantics extends Logging {
/**
* [[RawTag.Tree]] with [[Token]] used for error reporting
*/
private final case class RawTagWithToken[S[_]](
tree: RawTag.Tree,
token: Token[S]
) {
lazy val tag: RawTag = tree.head
private def modifyTree(f: RawTag.Tree => RawTag.Tree): RawTagWithToken[S] =
copy(tree = f(tree))
/**
* Wrap tail of @param next in [[SeqTag]]
* and append it to current tree tail
*/
def append(next: RawTagWithToken[S]): RawTagWithToken[S] = modifyTree(tree =>
tree.copy(
tail = (
tree.tail,
// SeqTag.wrap will return single node as is
next.tree.tail.map(SeqTag.wrap)
).mapN(_ :+ _)
)
)
def wrapIn(tag: GroupTag): RawTagWithToken[S] = modifyTree(tree => tag.wrap(tree))
def toRaw: RawWithToken[S] = RawWithToken(FuncOp(tree), token)
}
private def elseWithoutIf[S[_], G[_]](
token: Token[S]
)(using E: ErrorsAlgebra[S, G]): G[Unit] =
E.report(token, "Unexpected `else` without `if`" :: Nil)
private def catchWithoutTry[S[_], G[_]](
token: Token[S]
)(using E: ErrorsAlgebra[S, G]): G[Unit] =
E.report(token, "Unexpected `catch` without `try`" :: Nil)
private def otherwiseWithoutPrev[S[_], G[_]](
token: Token[S]
)(using E: ErrorsAlgebra[S, G]): G[Unit] =
E.report(token, "Unexpected `otherwise` without previous instruction" :: Nil)
private def parWithoutPrev[S[_], G[_]](
token: Token[S]
)(using E: ErrorsAlgebra[S, G]): G[Unit] =
E.report(token, "Unexpected `par` without previous instruction" :: Nil)
/**
* Optionally combine two [[RawTag.Tree]] into one.
* Used to combine `if` and `else`,
* `try` and `catch` (`otherwise`);
* to create [[ParTag]] from `par`,
* [[TryTag]] from `otherwise`
*
* @param prev Previous tag
* @param next Next tag
* @param E Algebra for error reporting
* @return [[Some]] with result of combination
* [[None]] if tags should not be combined
* or error occuried
*/
private def rawTagCombine[S[_], G[_]: Monad](
prev: RawTagWithToken[S],
next: RawTagWithToken[S]
)(using E: ErrorsAlgebra[S, G]): G[Option[RawTagWithToken[S]]] =
(prev.tag, next.tag) match {
case (_: IfTag, IfTag.Else) =>
prev.append(next).some.pure
case (_, IfTag.Else) | (IfTag.Else, _) =>
val token = prev.tag match {
case IfTag.Else => prev.token
case _ => next.token
}
elseWithoutIf(token).as(none)
case (TryTag, TryTag.Catch) =>
prev.append(next).some.pure
case (_, TryTag.Catch) | (TryTag.Catch, _) =>
val token = prev.tag match {
case TryTag.Catch => prev.token
case _ => next.token
}
catchWithoutTry(token).as(none)
case (TryTag.Otherwise, _) =>
otherwiseWithoutPrev(prev.token).as(none)
case (TryTag, TryTag.Otherwise) =>
prev.append(next).some.pure
case (_, TryTag.Otherwise) =>
prev
.wrapIn(TryTag)
.append(next)
.some
.pure
case (ParTag.Par, _) =>
parWithoutPrev(prev.token).as(none)
case (ParTag, ParTag.Par) =>
prev.append(next).some.pure
case (_, ParTag.Par) =>
prev
.wrapIn(ParTag)
.append(next)
.some
.pure
case _ => none.pure
}
/**
* Check if tag is valid to be single
*
* @param single tag
* @param E Algebra for error reporting
* @return [[Some]] if tag is valid to be single
* [[None]] otherwise
*/
private def rawTagSingleCheck[S[_], G[_]: Monad](
single: RawTagWithToken[S]
)(using E: ErrorsAlgebra[S, G]): G[Option[RawTagWithToken[S]]] =
single.tag match {
case IfTag.Else => elseWithoutIf(single.token).as(none)
case TryTag.Catch => catchWithoutTry(single.token).as(none)
case TryTag.Otherwise => otherwiseWithoutPrev(single.token).as(none)
case ParTag.Par => parWithoutPrev(single.token).as(none)
case _ => single.some.pure
}
/**
* [[Raw]] with [[Token]] used for error reporting
*/
private final case class RawWithToken[S[_]](
raw: Raw,
token: Token[S]
) {
def toTag: Option[RawTagWithToken[S]] =
raw match {
case FuncOp(tree) => RawTagWithToken(tree, token).some
case _ => none
}
}
/**
* State for folding [[Raw]] results of children
*
* @param last Last seen [[Raw]] with [[Token]]
* @param acc All previous [[Raw]]
*/
private final case class InnersFoldState[S[_]](
last: Option[RawWithToken[S]] = None,
acc: Chain[Raw] = Chain.empty
) {
/**
* Process new incoming [[Raw]]
*/
def step[G[_]: Monad](
next: RawWithToken[S]
)(using ErrorsAlgebra[S, G]): G[InnersFoldState[S]] =
last.fold(copy(last = next.some).pure)(prev =>
(prev.toTag, next.toTag)
.traverseN(rawTagCombine)
.map(
_.flatten.fold(
// No combination - just update last and acc
copy(
last = next.some,
acc = prev.raw +: acc
)
)(combined =>
// Result of combination is the new last
copy(
last = combined.toRaw.some
)
)
)
)
/**
* Produce result of folding
*/
def result[G[_]: Monad](using
ErrorsAlgebra[S, G]
): G[Option[Raw]] =
if (acc.isEmpty)
// Hack to report error if single tag in block is incorrect
last.flatTraverse(single =>
single.toTag.fold(single.raw.some.pure)(singleTag =>
for {
checked <- rawTagSingleCheck(singleTag)
maybeRaw = checked.map(_.toRaw.raw)
} yield maybeRaw
)
)
else
last
.fold(acc)(_.raw +: acc)
.reverse
.reduceLeftOption(_ |+| _)
.pure
}
private def folder[S[_], G[_]: Monad](implicit
A: AbilitiesAlgebra[S, G],
N: NamesAlgebra[S, G],
T: TypesAlgebra[S, G],
D: DefinitionsAlgebra[S, G],
L: LocationsAlgebra[S, G],
E: ErrorsAlgebra[S, G]
): (Expr[S], Chain[G[RawWithToken[S]]]) => Eval[G[RawWithToken[S]]] = (expr, inners) =>
Eval later ExprSem
.getProg[S, G](expr)
.apply(for {
children <- inners.sequence
resultState <- children
.traverse(raw => StateT.modifyF((state: InnersFoldState[S]) => state.step(raw)))
.runS(InnersFoldState())
result <- resultState.result
} yield result.getOrElse(Raw.empty("AST is empty")))
.map(raw => RawWithToken(raw, expr.token))
type Interpreter[S[_], A] = State[CompilerState[S], A]
def transpile[S[_]](ast: Ast[S])(using
LocationsAlgebra[S, Interpreter[S, *]]
): Interpreter[S, Raw] = {
given TypesAlgebra[S, Interpreter[S, *]] =
new TypesInterpreter[S, CompilerState[S]]
given ManglerAlgebra[Interpreter[S, *]] =
new ManglerInterpreter[CompilerState[S]]
given AbilitiesAlgebra[S, Interpreter[S, *]] =
new AbilitiesInterpreter[S, CompilerState[S]]
given NamesAlgebra[S, Interpreter[S, *]] =
new NamesInterpreter[S, CompilerState[S]]
given DefinitionsAlgebra[S, Interpreter[S, *]] =
new DefinitionsInterpreter[S, CompilerState[S]]
ast
.cata(folder[S, Interpreter[S, *]])
.value
.map(_.raw)
}
private def astToState[S[_]](ast: Ast[S])(implicit
locations: LocationsAlgebra[S, Interpreter[S, *]]
): Interpreter[S, Raw] =
transpile[S](ast)
// If there are any errors, they're inside CompilerState[S]
def interpret[S[_]](
ast: Ast[S],
initState: CompilerState[S],
init: RawContext
)(implicit
locations: LocationsAlgebra[S, Interpreter[S, *]]
): Eval[(CompilerState[S], RawContext)] =
astToState[S](ast)
.run(initState)
.map {
case (state, _: Raw.Empty) =>
// No `parts`, but has `init`
(
state,
RawContext.blank.copy(
init = Some(init.copy(module = init.module.map(_ + "|init")))
.filter(_ != RawContext.blank)
)
)
case (state, part: (RawPart | RawPart.Parts)) =>
state -> RawPart
.contextPart(part)
.parts
.foldLeft(
RawContext.blank.copy(
init = Some(init.copy(module = init.module.map(_ + "|init")))
.filter(_ != RawContext.blank)
)
) { case (ctx, p) =>
ctx.copy(parts = ctx.parts :+ (ctx -> p))
}
case (_, m) =>
internalError(
s"Unexpected Raw ($m)"
)
}
} }

View File

@ -10,19 +10,21 @@ import aqua.semantics.rules.ValuesAlgebra
import aqua.semantics.rules.names.NamesAlgebra import aqua.semantics.rules.names.NamesAlgebra
import aqua.semantics.rules.types.TypesAlgebra import aqua.semantics.rules.types.TypesAlgebra
import aqua.types.{StreamType, Type} import aqua.types.{StreamType, Type}
import cats.Monad import cats.Monad
import cats.syntax.flatMap.* import cats.syntax.flatMap.*
import cats.syntax.functor.* import cats.syntax.functor.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.option.* import cats.syntax.option.*
import cats.syntax.applicative.* import cats.syntax.applicative.*
import cats.syntax.apply.*
import cats.syntax.comonad.* import cats.syntax.comonad.*
class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal { class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal {
import expr.* import expr.*
private def getExports[Alg[_]: Monad](callArrow: CallArrowRaw)(implicit private def getExports[Alg[_]: Monad](callArrow: CallArrowRaw)(using
N: NamesAlgebra[S, Alg], N: NamesAlgebra[S, Alg],
T: TypesAlgebra[S, Alg] T: TypesAlgebra[S, Alg]
): Alg[List[Call.Export]] = ): Alg[List[Call.Export]] =
@ -42,16 +44,11 @@ class CallArrowSem[S[_]](val expr: CallArrowExpr[S]) extends AnyVal {
): Alg[Option[FuncOp]] = for { ): Alg[Option[FuncOp]] = for {
// TODO: Accept other expressions // TODO: Accept other expressions
callArrowRaw <- V.valueToCallArrowRaw(expr.callArrow) callArrowRaw <- V.valueToCallArrowRaw(expr.callArrow)
maybeOp <- callArrowRaw.traverse(car => tag <- callArrowRaw.traverse(car =>
variables getExports(car).map(CallArrowRawTag(_, car)) <*
.drop(car.baseType.codomain.length) T.checkArrowCallResults(callArrow, car.baseType, variables)
.headOption
.fold(getExports(car))(
T.expectNoExport(_).as(Nil)
) )
.map(maybeExports => CallArrowRawTag(maybeExports, car).funcOpLeaf) } yield tag.map(_.funcOpLeaf)
)
} yield maybeOp
def program[Alg[_]: Monad](implicit def program[Alg[_]: Monad](implicit
N: NamesAlgebra[S, Alg], N: NamesAlgebra[S, Alg],

View File

@ -1,7 +1,6 @@
package aqua.semantics.rules package aqua.semantics.rules
import aqua.parser.lexer.Token import aqua.parser.lexer.Token
import aqua.semantics.rules.errors.ReportErrors
import cats.data.State import cats.data.State
import cats.syntax.functor.* import cats.syntax.functor.*
@ -10,21 +9,12 @@ import monocle.Lens
case class StackInterpreter[S[_], X, St, Fr]( case class StackInterpreter[S[_], X, St, Fr](
stackLens: Lens[St, List[Fr]] stackLens: Lens[St, List[Fr]]
)(using )(using lens: Lens[X, St]) {
lens: Lens[X, St],
error: ReportErrors[S, X]
) {
type SX[A] = State[X, A] type SX[A] = State[X, A]
def getState: SX[St] = State.get.map(lens.get) def getState: SX[St] = State.get.map(lens.get)
def setState(st: St): SX[Unit] = State.modify(s => lens.replace(st)(s)) def setState(st: St): SX[Unit] = State.modify(s => lens.replace(st)(s))
def reportError(t: Token[S], hints: List[String]): SX[Unit] =
State.modify(error(_, t, hints))
def report(t: Token[S], hint: String): SX[Unit] =
State.modify(error(_, t, hint :: Nil))
def modify(f: St => St): SX[Unit] = def modify(f: St => St): SX[Unit] =
State.modify(lens.modify(f)) State.modify(lens.modify(f))

View File

@ -7,7 +7,7 @@ import aqua.raw.value.*
import aqua.semantics.rules.abilities.AbilitiesAlgebra import aqua.semantics.rules.abilities.AbilitiesAlgebra
import aqua.semantics.rules.names.NamesAlgebra import aqua.semantics.rules.names.NamesAlgebra
import aqua.semantics.rules.types.TypesAlgebra import aqua.semantics.rules.types.TypesAlgebra
import aqua.semantics.rules.errors.ErrorsAlgebra import aqua.semantics.rules.report.ReportAlgebra
import aqua.types.* import aqua.types.*
import cats.Monad import cats.Monad
@ -30,8 +30,8 @@ import scala.collection.immutable.SortedMap
class ValuesAlgebra[S[_], Alg[_]: Monad](using class ValuesAlgebra[S[_], Alg[_]: Monad](using
N: NamesAlgebra[S, Alg], N: NamesAlgebra[S, Alg],
T: TypesAlgebra[S, Alg], T: TypesAlgebra[S, Alg],
E: ErrorsAlgebra[S, Alg], A: AbilitiesAlgebra[S, Alg],
A: AbilitiesAlgebra[S, Alg] report: ReportAlgebra[S, Alg]
) extends Logging { ) extends Logging {
private def resolveSingleProperty(rootType: Type, op: PropertyOp[S]): Alg[Option[PropertyRaw]] = private def resolveSingleProperty(rootType: Type, op: PropertyOp[S]): Alg[Option[PropertyRaw]] =
@ -305,7 +305,7 @@ class ValuesAlgebra[S[_], Alg[_]: Monad](using
_.flatTraverse { _.flatTraverse {
case ca: CallArrowRaw => ca.some.pure[Alg] case ca: CallArrowRaw => ca.some.pure[Alg]
// TODO: better error message (`raw` formatting) // TODO: better error message (`raw` formatting)
case raw => E.report(v, s"Expected arrow call, got $raw").as(none) case raw => report.error(v, s"Expected arrow call, got $raw").as(none)
} }
) )
@ -419,7 +419,7 @@ object ValuesAlgebra {
N: NamesAlgebra[S, Alg], N: NamesAlgebra[S, Alg],
T: TypesAlgebra[S, Alg], T: TypesAlgebra[S, Alg],
A: AbilitiesAlgebra[S, Alg], A: AbilitiesAlgebra[S, Alg],
E: ErrorsAlgebra[S, Alg] E: ReportAlgebra[S, Alg]
): ValuesAlgebra[S, Alg] = ): ValuesAlgebra[S, Alg] =
new ValuesAlgebra[S, Alg] new ValuesAlgebra[S, Alg]
} }

View File

@ -4,7 +4,7 @@ import aqua.parser.lexer.{Name, NamedTypeToken, Token, ValueToken}
import aqua.raw.value.ValueRaw import aqua.raw.value.ValueRaw
import aqua.raw.{RawContext, ServiceRaw} import aqua.raw.{RawContext, ServiceRaw}
import aqua.semantics.Levenshtein import aqua.semantics.Levenshtein
import aqua.semantics.rules.errors.ReportErrors import aqua.semantics.rules.report.ReportAlgebra
import aqua.semantics.rules.mangler.ManglerAlgebra import aqua.semantics.rules.mangler.ManglerAlgebra
import aqua.semantics.rules.locations.LocationsAlgebra import aqua.semantics.rules.locations.LocationsAlgebra
import aqua.semantics.rules.{abilities, StackInterpreter} import aqua.semantics.rules.{abilities, StackInterpreter}
@ -22,7 +22,7 @@ import monocle.macros.GenLens
class AbilitiesInterpreter[S[_], X](using class AbilitiesInterpreter[S[_], X](using
lens: Lens[X, AbilitiesState[S]], lens: Lens[X, AbilitiesState[S]],
error: ReportErrors[S, X], report: ReportAlgebra[S, State[X, *]],
mangler: ManglerAlgebra[State[X, *]], mangler: ManglerAlgebra[State[X, *]],
locations: LocationsAlgebra[S, State[X, *]] locations: LocationsAlgebra[S, State[X, *]]
) extends AbilitiesAlgebra[S, State[X, *]] { ) extends AbilitiesAlgebra[S, State[X, *]] {
@ -33,7 +33,7 @@ class AbilitiesInterpreter[S[_], X](using
GenLens[AbilitiesState[S]](_.stack) GenLens[AbilitiesState[S]](_.stack)
) )
import stackInt.{getState, mapStackHead, mapStackHeadM, modify, report} import stackInt.*
override def defineService( override def defineService(
name: NamedTypeToken[S], name: NamedTypeToken[S],
@ -45,10 +45,12 @@ class AbilitiesInterpreter[S[_], X](using
getState getState
.map(_.definitions.get(name.value).exists(_ == name)) .map(_.definitions.get(name.value).exists(_ == name))
.flatMap(exists => .flatMap(exists =>
report( report
.error(
name, name,
"Service with this name was already defined" "Service with this name was already defined"
).whenA(!exists) )
.whenA(!exists)
) )
.as(false) .as(false)
case false => case false =>
@ -74,21 +76,23 @@ class AbilitiesInterpreter[S[_], X](using
abCtx.funcs abCtx.funcs
.get(arrow.value) .get(arrow.value)
.fold( .fold(
report( report
.error(
arrow, arrow,
Levenshtein.genMessage( Levenshtein.genMessage(
s"Ability is found, but arrow '${arrow.value}' isn't found in scope", s"Ability is found, but arrow '${arrow.value}' isn't found in scope",
arrow.value, arrow.value,
abCtx.funcs.keys.toList abCtx.funcs.keys.toList
) )
).as(none) )
.as(none)
) { fn => ) { fn =>
// TODO: add name and arrow separately // TODO: add name and arrow separately
// TODO: find tokens somewhere // TODO: find tokens somewhere
addServiceArrowLocation(name, arrow).as(fn.arrow.`type`.some) addServiceArrowLocation(name, arrow).as(fn.arrow.`type`.some)
} }
case None => case None =>
report(name, "Ability with this name is undefined").as(none) report.error(name, "Ability with this name is undefined").as(none)
} }
override def renameService(name: NamedTypeToken[S]): SX[Option[String]] = override def renameService(name: NamedTypeToken[S]): SX[Option[String]] =
@ -102,7 +106,7 @@ class AbilitiesInterpreter[S[_], X](using
.map(newName => h.setServiceRename(name.value, newName) -> newName) .map(newName => h.setServiceRename(name.value, newName) -> newName)
).map(_.some) ).map(_.some)
case false => case false =>
report(name, "Service with this name is not registered").as(none) report.error(name, "Service with this name is not registered").as(none)
} }
override def getServiceRename(name: NamedTypeToken[S]): State[X, Option[String]] = override def getServiceRename(name: NamedTypeToken[S]): State[X, Option[String]] =
@ -111,8 +115,8 @@ class AbilitiesInterpreter[S[_], X](using
getState.map(_.getServiceRename(name.value)) getState.map(_.getServiceRename(name.value))
).flatMapN { ).flatMapN {
case (true, Some(rename)) => rename.some.pure case (true, Some(rename)) => rename.some.pure
case (false, _) => report(name, "Service with this name is undefined").as(none) case (false, _) => report.error(name, "Service with this name is undefined").as(none)
case (_, None) => report(name, "Service ID is undefined").as(none) case (_, None) => report.error(name, "Service ID is undefined").as(none)
} }
override def beginScope(token: Token[S]): SX[Unit] = override def beginScope(token: Token[S]): SX[Unit] =

View File

@ -2,7 +2,7 @@ package aqua.semantics.rules.definitions
import aqua.parser.lexer.{Name, NamedTypeToken, Token} import aqua.parser.lexer.{Name, NamedTypeToken, Token}
import aqua.semantics.rules.StackInterpreter import aqua.semantics.rules.StackInterpreter
import aqua.semantics.rules.errors.ReportErrors import aqua.semantics.rules.report.ReportAlgebra
import aqua.semantics.rules.abilities.AbilitiesState import aqua.semantics.rules.abilities.AbilitiesState
import aqua.semantics.rules.locations.{LocationsAlgebra, LocationsState} import aqua.semantics.rules.locations.{LocationsAlgebra, LocationsState}
import aqua.semantics.rules.types.TypesState import aqua.semantics.rules.types.TypesState
@ -21,7 +21,7 @@ import scala.collection.immutable.SortedMap
class DefinitionsInterpreter[S[_], X](implicit class DefinitionsInterpreter[S[_], X](implicit
lens: Lens[X, DefinitionsState[S]], lens: Lens[X, DefinitionsState[S]],
error: ReportErrors[S, X], report: ReportAlgebra[S, State[X, *]],
locations: LocationsAlgebra[S, State[X, *]] locations: LocationsAlgebra[S, State[X, *]]
) extends DefinitionsAlgebra[S, State[X, *]] { ) extends DefinitionsAlgebra[S, State[X, *]] {
type SX[A] = State[X, A] type SX[A] = State[X, A]
@ -31,9 +31,6 @@ class DefinitionsInterpreter[S[_], X](implicit
private def modify(f: DefinitionsState[S] => DefinitionsState[S]): SX[Unit] = private def modify(f: DefinitionsState[S] => DefinitionsState[S]): SX[Unit] =
State.modify(lens.modify(f)) State.modify(lens.modify(f))
def report(t: Token[S], hint: String): SX[Unit] =
State.modify(error(_, t, hint :: Nil))
def define(name: Name[S], `type`: Type, defName: String): SX[Boolean] = def define(name: Name[S], `type`: Type, defName: String): SX[Boolean] =
getState.map(_.definitions.get(name.value)).flatMap { getState.map(_.definitions.get(name.value)).flatMap {
case None => case None =>
@ -47,7 +44,8 @@ class DefinitionsInterpreter[S[_], X](implicit
) )
.as(true) .as(true)
case Some(_) => case Some(_) =>
report(name, s"Cannot define $defName `${name.value}`, it was already defined above") report
.error(name, s"Cannot define $defName `${name.value}`, it was already defined above")
.as(false) .as(false)
} }
@ -82,7 +80,8 @@ class DefinitionsInterpreter[S[_], X](implicit
st.copy(definitions = Map.empty) st.copy(definitions = Map.empty)
}.as(arrs.some) }.as(arrs.some)
case None => case None =>
report(token, "Cannot purge arrows, no arrows provided") report
.error(token, "Cannot purge arrows, no arrows provided")
.as(none) .as(none)
} }
} }

View File

@ -1,10 +0,0 @@
package aqua.semantics.rules.errors
import aqua.parser.lexer.Token
trait ErrorsAlgebra[S[_], Alg[_]] {
def report(token: Token[S], hints: List[String]): Alg[Unit]
def report(token: Token[S], hint: String): Alg[Unit] =
report(token, hint :: Nil)
}

View File

@ -1,12 +0,0 @@
package aqua.semantics.rules.errors
import aqua.parser.lexer.Token
import cats.data.State
trait ReportErrors[S[_], X] extends ErrorsAlgebra[S, State[X, *]] {
def apply(st: X, token: Token[S], hints: List[String]): X
def report(token: Token[S], hints: List[String]): State[X, Unit] =
State.modify(apply(_, token, hints))
}

View File

@ -3,7 +3,7 @@ package aqua.semantics.rules.names
import aqua.parser.lexer.{Name, Token} import aqua.parser.lexer.{Name, Token}
import aqua.semantics.Levenshtein import aqua.semantics.Levenshtein
import aqua.semantics.rules.StackInterpreter import aqua.semantics.rules.StackInterpreter
import aqua.semantics.rules.errors.ReportErrors import aqua.semantics.rules.report.ReportAlgebra
import aqua.semantics.rules.locations.LocationsAlgebra import aqua.semantics.rules.locations.LocationsAlgebra
import aqua.types.{AbilityType, ArrowType, StreamType, Type} import aqua.types.{AbilityType, ArrowType, StreamType, Type}
@ -15,9 +15,9 @@ import cats.syntax.all.*
import monocle.Lens import monocle.Lens
import monocle.macros.GenLens import monocle.macros.GenLens
class NamesInterpreter[S[_], X](implicit class NamesInterpreter[S[_], X](using
lens: Lens[X, NamesState[S]], lens: Lens[X, NamesState[S]],
error: ReportErrors[S, X], report: ReportAlgebra[S, State[X, *]],
locations: LocationsAlgebra[S, State[X, *]] locations: LocationsAlgebra[S, State[X, *]]
) extends NamesAlgebra[S, State[X, *]] { ) extends NamesAlgebra[S, State[X, *]] {
@ -25,7 +25,7 @@ class NamesInterpreter[S[_], X](implicit
GenLens[NamesState[S]](_.stack) GenLens[NamesState[S]](_.stack)
) )
import stackInt.{getState, mapStackHead, mapStackHeadM, mapStackHead_, modify, report} import stackInt.*
type SX[A] = State[X, A] type SX[A] = State[X, A]
@ -44,7 +44,7 @@ class NamesInterpreter[S[_], X](implicit
.flatTap { .flatTap {
case None if mustBeDefined => case None if mustBeDefined =>
getState.flatMap(st => getState.flatMap(st =>
report( report.error(
name, name,
Levenshtein Levenshtein
.genMessage( .genMessage(
@ -73,7 +73,8 @@ class NamesInterpreter[S[_], X](implicit
locations.pointLocation(name.value, name).map(_ => Option(at)) locations.pointLocation(name.value, name).map(_ => Option(at))
case _ => case _ =>
getState.flatMap(st => getState.flatMap(st =>
report( report
.error(
name, name,
Levenshtein.genMessage( Levenshtein.genMessage(
s"Name '${name.value}' not found in scope", s"Name '${name.value}' not found in scope",
@ -98,11 +99,11 @@ class NamesInterpreter[S[_], X](implicit
case Some(_) => case Some(_) =>
getState.map(_.definitions.get(name.value).exists(_ == name)).flatMap { getState.map(_.definitions.get(name.value).exists(_ == name)).flatMap {
case true => State.pure(false) case true => State.pure(false)
case false => report(name, "This name was already defined in the scope").as(false) case false => report.error(name, "This name was already defined in the scope").as(false)
} }
case None => case None =>
mapStackHeadM(report(name, "Cannot define a variable in the root scope").as(false))(fr => mapStackHeadM(report.error(name, "Cannot define a variable in the root scope").as(false))(
(fr.addName(name, `type`) -> true).pure fr => (fr.addName(name, `type`) -> true).pure
) <* locations.addToken(name.value, name) ) <* locations.addToken(name.value, name)
} }
@ -121,7 +122,7 @@ class NamesInterpreter[S[_], X](implicit
override def defineConstant(name: Name[S], `type`: Type): SX[Boolean] = override def defineConstant(name: Name[S], `type`: Type): SX[Boolean] =
readName(name.value).flatMap { readName(name.value).flatMap {
case Some(_) => case Some(_) =>
report(name, "This name was already defined in the scope").as(false) report.error(name, "This name was already defined in the scope").as(false)
case None => case None =>
modify(st => modify(st =>
st.copy( st.copy(
@ -135,7 +136,7 @@ class NamesInterpreter[S[_], X](implicit
case Some(_) => case Some(_) =>
getState.map(_.definitions.get(name.value).exists(_ == name)).flatMap { getState.map(_.definitions.get(name.value).exists(_ == name)).flatMap {
case true => State.pure(false) case true => State.pure(false)
case false => report(name, "This arrow was already defined in the scope").as(false) case false => report.error(name, "This arrow was already defined in the scope").as(false)
} }
case None => case None =>
@ -149,7 +150,8 @@ class NamesInterpreter[S[_], X](implicit
) )
.as(true) .as(true)
else else
report(name, "Cannot define a variable in the root scope") report
.error(name, "Cannot define a variable in the root scope")
.as(false) .as(false)
)(fr => (fr.addArrow(name, arrowType) -> true).pure) )(fr => (fr.addArrow(name, arrowType) -> true).pure)
}.flatTap(_ => locations.addToken(name.value, name)) }.flatTap(_ => locations.addToken(name.value, name))

View File

@ -0,0 +1,15 @@
package aqua.semantics.rules.report
import aqua.parser.lexer.Token
trait ReportAlgebra[S[_], Alg[_]] {
def error(token: Token[S], hints: List[String]): Alg[Unit]
def error(token: Token[S], hint: String): Alg[Unit] =
error(token, hint :: Nil)
def warning(token: Token[S], hints: List[String]): Alg[Unit]
def warning(token: Token[S], hint: String): Alg[Unit] =
warning(token, hint :: Nil)
}

View File

@ -0,0 +1,25 @@
package aqua.semantics.rules.report
import aqua.parser.lexer.Token
import cats.data.State
import monocle.Lens
class ReportInterpreter[S[_], X](using
lens: Lens[X, ReportState[S]]
) extends ReportAlgebra[S, State[X, *]] {
override def error(token: Token[S], hints: List[String]): State[X, Unit] =
State.modify(
lens.modify(
_.reportError(token, hints)
)
)
override def warning(token: Token[S], hints: List[String]): State[X, Unit] =
State.modify(
lens.modify(
_.reportWarning(token, hints)
)
)
}

View File

@ -0,0 +1,32 @@
package aqua.semantics.rules.report
import aqua.semantics.{RulesViolated, SemanticError, SemanticWarning}
import aqua.parser.lexer.Token
import cats.data.Chain
import cats.kernel.Monoid
final case class ReportState[S[_]](
errors: Chain[SemanticError[S]] = Chain.empty[SemanticError[S]],
warnings: Chain[SemanticWarning[S]] = Chain.empty[SemanticWarning[S]]
) {
def reportError(token: Token[S], hints: List[String]): ReportState[S] =
copy(errors = errors.append(RulesViolated(token, hints)))
def reportWarning(token: Token[S], hints: List[String]): ReportState[S] =
copy(warnings = warnings.append(SemanticWarning(token, hints)))
}
object ReportState {
given [S[_]]: Monoid[ReportState[S]] with {
override val empty: ReportState[S] = ReportState()
override def combine(x: ReportState[S], y: ReportState[S]): ReportState[S] =
ReportState(
errors = x.errors ++ y.errors,
warnings = x.warnings ++ y.warnings
)
}
}

View File

@ -61,7 +61,11 @@ trait TypesAlgebra[S[_], Alg[_]] {
givenType: Type givenType: Type
): Alg[Option[Type]] ): Alg[Option[Type]]
def expectNoExport(token: Token[S]): Alg[Unit] def checkArrowCallResults(
token: Token[S],
arrowType: ArrowType,
results: List[Name[S]]
): Alg[Unit]
def checkArgumentsNumber(token: Token[S], expected: Int, givenNum: Int): Alg[Boolean] def checkArgumentsNumber(token: Token[S], expected: Int, givenNum: Int): Alg[Boolean]

View File

@ -12,7 +12,7 @@ import aqua.raw.value.{
} }
import aqua.semantics.rules.locations.LocationsAlgebra import aqua.semantics.rules.locations.LocationsAlgebra
import aqua.semantics.rules.StackInterpreter import aqua.semantics.rules.StackInterpreter
import aqua.semantics.rules.errors.ReportErrors import aqua.semantics.rules.report.ReportAlgebra
import aqua.semantics.rules.types.TypesStateHelper.{TypeResolution, TypeResolutionError} import aqua.semantics.rules.types.TypesStateHelper.{TypeResolution, TypeResolutionError}
import aqua.types.* import aqua.types.*
@ -31,9 +31,9 @@ import monocle.macros.GenLens
import scala.collection.immutable.SortedMap import scala.collection.immutable.SortedMap
class TypesInterpreter[S[_], X](implicit class TypesInterpreter[S[_], X](using
lens: Lens[X, TypesState[S]], lens: Lens[X, TypesState[S]],
error: ReportErrors[S, X], report: ReportAlgebra[S, State[X, *]],
locations: LocationsAlgebra[S, State[X, *]] locations: LocationsAlgebra[S, State[X, *]]
) extends TypesAlgebra[S, State[X, *]] { ) extends TypesAlgebra[S, State[X, *]] {
@ -55,7 +55,7 @@ class TypesInterpreter[S[_], X](implicit
locations.pointLocations(tokensLocs).as(typ.some) locations.pointLocations(tokensLocs).as(typ.some)
case None => case None =>
// TODO: Give more specific error message // TODO: Give more specific error message
report(token, s"Unresolved type").as(None) report.error(token, s"Unresolved type").as(None)
} }
override def resolveArrowDef(arrowDef: ArrowTypeToken[S]): State[X, Option[ArrowType]] = override def resolveArrowDef(arrowDef: ArrowTypeToken[S]): State[X, Option[ArrowType]] =
@ -65,7 +65,7 @@ class TypesInterpreter[S[_], X](implicit
locations.pointLocations(tokensLocs).as(tt.some) locations.pointLocations(tokensLocs).as(tt.some)
case Invalid(errs) => case Invalid(errs) =>
errs.traverse_ { case TypeResolutionError(token, hint) => errs.traverse_ { case TypeResolutionError(token, hint) =>
report(token, hint) report.error(token, hint)
}.as(none) }.as(none)
} }
@ -74,9 +74,9 @@ class TypesInterpreter[S[_], X](implicit
case Some(serviceType: ServiceType) => case Some(serviceType: ServiceType) =>
serviceType.some.pure serviceType.some.pure
case Some(t) => case Some(t) =>
report(name, s"Type `$t` is not a service").as(none) report.error(name, s"Type `$t` is not a service").as(none)
case None => case None =>
report(name, s"Type `${name.value}` is not defined").as(none) report.error(name, s"Type `${name.value}` is not defined").as(none)
} }
override def defineAbilityType( override def defineAbilityType(
@ -87,7 +87,8 @@ class TypesInterpreter[S[_], X](implicit
val types = fields.view.mapValues { case (_, t) => t }.toMap val types = fields.view.mapValues { case (_, t) => t }.toMap
NonEmptyMap NonEmptyMap
.fromMap(SortedMap.from(types)) .fromMap(SortedMap.from(types))
.fold(report(name, s"Ability `${name.value}` has no fields").as(none))(nonEmptyFields => .fold(report.error(name, s"Ability `${name.value}` has no fields").as(none))(
nonEmptyFields =>
val `type` = AbilityType(name.value, nonEmptyFields) val `type` = AbilityType(name.value, nonEmptyFields)
modify(_.defineType(name, `type`)).as(`type`.some) modify(_.defineType(name, `type`)).as(`type`.some)
@ -104,20 +105,22 @@ class TypesInterpreter[S[_], X](implicit
OptionT OptionT
.when(t.codomain.length <= 1)(field -> t) .when(t.codomain.length <= 1)(field -> t)
.flatTapNone( .flatTapNone(
report(fieldName, "Service functions cannot have multiple results") report.error(fieldName, "Service functions cannot have multiple results")
) )
case (field, (fieldName, t)) => case (field, (fieldName, t)) =>
OptionT( OptionT(
report( report
.error(
fieldName, fieldName,
s"Field '$field' has unacceptable for service field type '$t'" s"Field '$field' has unacceptable for service field type '$t'"
).as(none) )
.as(none)
) )
}.flatMapF(arrows => }.flatMapF(arrows =>
NonEmptyMap NonEmptyMap
.fromMap(SortedMap.from(arrows)) .fromMap(SortedMap.from(arrows))
.fold( .fold(
report(name, s"Service `${name.value}` has no fields").as(none) report.error(name, s"Service `${name.value}` has no fields").as(none)
)(_.some.pure) )(_.some.pure)
).semiflatMap(nonEmptyArrows => ).semiflatMap(nonEmptyArrows =>
val `type` = ServiceType(name.value, nonEmptyArrows) val `type` = ServiceType(name.value, nonEmptyArrows)
@ -134,20 +137,23 @@ class TypesInterpreter[S[_], X](implicit
fields.toList.traverse { fields.toList.traverse {
case (field, (fieldName, t: DataType)) => case (field, (fieldName, t: DataType)) =>
t match { t match {
case _: StreamType => report(fieldName, s"Field '$field' has stream type").as(none) case _: StreamType =>
report.error(fieldName, s"Field '$field' has stream type").as(none)
case _ => (field -> t).some.pure[ST] case _ => (field -> t).some.pure[ST]
} }
case (field, (fieldName, t)) => case (field, (fieldName, t)) =>
report( report
.error(
fieldName, fieldName,
s"Field '$field' has unacceptable for struct field type '$t'" s"Field '$field' has unacceptable for struct field type '$t'"
).as(none) )
.as(none)
}.map(_.sequence.map(_.toMap)) }.map(_.sequence.map(_.toMap))
.flatMap( .flatMap(
_.map(SortedMap.from) _.map(SortedMap.from)
.flatMap(NonEmptyMap.fromMap) .flatMap(NonEmptyMap.fromMap)
.fold( .fold(
report(name, s"Struct `${name.value}` has no fields").as(none) report.error(name, s"Struct `${name.value}` has no fields").as(none)
)(nonEmptyFields => )(nonEmptyFields =>
val `type` = StructType(name.value, nonEmptyFields) val `type` = StructType(name.value, nonEmptyFields)
@ -159,7 +165,7 @@ class TypesInterpreter[S[_], X](implicit
override def defineAlias(name: NamedTypeToken[S], target: Type): State[X, Boolean] = override def defineAlias(name: NamedTypeToken[S], target: Type): State[X, Boolean] =
getState.map(_.definitions.get(name.value)).flatMap { getState.map(_.definitions.get(name.value)).flatMap {
case Some(n) if n == name => State.pure(false) case Some(n) if n == name => State.pure(false)
case Some(_) => report(name, s"Type `${name.value}` was already defined").as(false) case Some(_) => report.error(name, s"Type `${name.value}` was already defined").as(false)
case None => case None =>
modify(_.defineType(name, target)) modify(_.defineType(name, target))
.productL(locations.addToken(name.value, name)) .productL(locations.addToken(name.value, name))
@ -171,10 +177,12 @@ class TypesInterpreter[S[_], X](implicit
case nt: NamedType => case nt: NamedType =>
nt.fields(op.value) nt.fields(op.value)
.fold( .fold(
report( report
.error(
op, op,
s"Field `${op.value}` not found in type `${nt.name}`, available: ${nt.fields.toNel.toList.map(_._1).mkString(", ")}" s"Field `${op.value}` not found in type `${nt.name}`, available: ${nt.fields.toNel.toList.map(_._1).mkString(", ")}"
).as(None) )
.as(None)
) { t => ) { t =>
locations.pointFieldLocation(nt.name, op.value, op).as(Some(IntoFieldRaw(op.value, t))) locations.pointFieldLocation(nt.name, op.value, op).as(Some(IntoFieldRaw(op.value, t)))
} }
@ -182,10 +190,12 @@ class TypesInterpreter[S[_], X](implicit
t.properties t.properties
.get(op.value) .get(op.value)
.fold( .fold(
report( report
.error(
op, op,
s"Expected data type to resolve a field '${op.value}' or a type with this property. Got: $rootT" s"Expected data type to resolve a field '${op.value}' or a type with this property. Got: $rootT"
).as(None) )
.as(None)
)(t => State.pure(Some(FunctorRaw(op.value, t)))) )(t => State.pure(Some(FunctorRaw(op.value, t))))
} }
@ -199,10 +209,12 @@ class TypesInterpreter[S[_], X](implicit
rootT match { rootT match {
case AbilityType(name, fieldsAndArrows) => case AbilityType(name, fieldsAndArrows) =>
fieldsAndArrows(op.name.value).fold( fieldsAndArrows(op.name.value).fold(
report( report
.error(
op, op,
s"Arrow `${op.name.value}` not found in type `$name`, available: ${fieldsAndArrows.toNel.toList.map(_._1).mkString(", ")}" s"Arrow `${op.name.value}` not found in type `$name`, available: ${fieldsAndArrows.toNel.toList.map(_._1).mkString(", ")}"
).as(None) )
.as(None)
) { t => ) { t =>
val resolvedType = t match { val resolvedType = t match {
// TODO: is it a correct way to resolve `IntoArrow` type? // TODO: is it a correct way to resolve `IntoArrow` type?
@ -217,10 +229,12 @@ class TypesInterpreter[S[_], X](implicit
t.properties t.properties
.get(op.name.value) .get(op.name.value)
.fold( .fold(
report( report
.error(
op, op,
s"Expected scope type to resolve an arrow '${op.name.value}' or a type with this property. Got: $rootT" s"Expected scope type to resolve an arrow '${op.name.value}' or a type with this property. Got: $rootT"
).as(None) )
.as(None)
)(t => State.pure(Some(FunctorRaw(op.name.value, t)))) )(t => State.pure(Some(FunctorRaw(op.name.value, t))))
} }
@ -238,12 +252,12 @@ class TypesInterpreter[S[_], X](implicit
st.fields.lookup(fieldName) match { st.fields.lookup(fieldName) match {
case Some(t) => case Some(t) =>
ensureTypeMatches(op.fields.lookup(fieldName).getOrElse(op), t, value.`type`) ensureTypeMatches(op.fields.lookup(fieldName).getOrElse(op), t, value.`type`)
case None => report(op, s"No field with name '$fieldName' in $rootT").as(false) case None => report.error(op, s"No field with name '$fieldName' in $rootT").as(false)
} }
}.map(res => if (res.forall(identity)) Some(IntoCopyRaw(st, fields)) else None) }.map(res => if (res.forall(identity)) Some(IntoCopyRaw(st, fields)) else None)
case _ => case _ =>
report(op, s"Expected $rootT to be a data type").as(None) report.error(op, s"Expected $rootT to be a data type").as(None)
} }
// TODO actually it's stateless, exists there just for reporting needs // TODO actually it's stateless, exists there just for reporting needs
@ -253,17 +267,17 @@ class TypesInterpreter[S[_], X](implicit
idx: ValueRaw idx: ValueRaw
): State[X, Option[PropertyRaw]] = ): State[X, Option[PropertyRaw]] =
if (!ScalarType.i64.acceptsValueOf(idx.`type`)) if (!ScalarType.i64.acceptsValueOf(idx.`type`))
report(op, s"Expected numeric index, got $idx").as(None) report.error(op, s"Expected numeric index, got $idx").as(None)
else else
rootT match { rootT match {
case ot: OptionType => case ot: OptionType =>
op.idx.fold( op.idx.fold(
State.pure(Some(IntoIndexRaw(idx, ot.element))) State.pure(Some(IntoIndexRaw(idx, ot.element)))
)(v => report(v, s"Options might have only one element, use ! to get it").as(None)) )(v => report.error(v, s"Options might have only one element, use ! to get it").as(None))
case rt: BoxType => case rt: BoxType =>
State.pure(Some(IntoIndexRaw(idx, rt.element))) State.pure(Some(IntoIndexRaw(idx, rt.element)))
case _ => case _ =>
report(op, s"Expected $rootT to be a collection type").as(None) report.error(op, s"Expected $rootT to be a collection type").as(None)
} }
override def ensureValuesComparable( override def ensureValuesComparable(
@ -299,7 +313,7 @@ class TypesInterpreter[S[_], X](implicit
} }
if (isComparable(left, right)) State.pure(true) if (isComparable(left, right)) State.pure(true)
else report(token, s"Cannot compare '$left' with '$right''").as(false) else report.error(token, s"Cannot compare '$left' with '$right''").as(false)
} }
override def ensureTypeMatches( override def ensureTypeMatches(
@ -315,10 +329,12 @@ class TypesInterpreter[S[_], X](implicit
val typeFields = typeNamedType.fields val typeFields = typeNamedType.fields
// value can have more fields // value can have more fields
if (valueFields.length < typeFields.length) { if (valueFields.length < typeFields.length) {
report( report
.error(
token, token,
s"Number of fields doesn't match the data type, expected: $expected, given: $givenType" s"Number of fields doesn't match the data type, expected: $expected, given: $givenType"
).as(false) )
.as(false)
} else { } else {
valueFields.toSortedMap.toList.traverse { (name, `type`) => valueFields.toSortedMap.toList.traverse { (name, `type`) =>
typeFields.lookup(name) match { typeFields.lookup(name) match {
@ -333,10 +349,12 @@ class TypesInterpreter[S[_], X](implicit
} }
ensureTypeMatches(nextToken, `type`, t) ensureTypeMatches(nextToken, `type`, t)
case None => case None =>
report( report
.error(
token, token,
s"Wrong value type, expected: $expected, given: $givenType" s"Wrong value type, expected: $expected, given: $givenType"
).as(false) )
.as(false)
} }
}.map(_.forall(identity)) }.map(_.forall(identity))
} }
@ -349,7 +367,8 @@ class TypesInterpreter[S[_], X](implicit
"You can extract value with `!`, but be aware it may trigger join behaviour." :: "You can extract value with `!`, but be aware it may trigger join behaviour." ::
Nil Nil
else Nil else Nil
reportError( report
.error(
token, token,
"Types mismatch." :: s"expected: $expected" :: s"given: $givenType" :: Nil ++ notes "Types mismatch." :: s"expected: $expected" :: s"given: $givenType" :: Nil ++ notes
) )
@ -361,10 +380,12 @@ class TypesInterpreter[S[_], X](implicit
givenType match { givenType match {
case _: DataType => true.pure case _: DataType => true.pure
case _ => case _ =>
report( report
.error(
token, token,
s"Value of type '$givenType' could not be put into a collection" s"Value of type '$givenType' could not be put into a collection"
).as(false) )
.as(false)
} }
override def ensureTypeOneOf[T <: Type]( override def ensureTypeOneOf[T <: Type](
@ -374,19 +395,46 @@ class TypesInterpreter[S[_], X](implicit
): State[X, Option[Type]] = expected ): State[X, Option[Type]] = expected
.find(_ acceptsValueOf givenType) .find(_ acceptsValueOf givenType)
.fold( .fold(
reportError( report
.error(
token, token,
"Types mismatch." :: "Types mismatch." ::
s"expected one of: ${expected.mkString(", ")}" :: s"expected one of: ${expected.mkString(", ")}" ::
s"given: $givenType" :: Nil s"given: $givenType" :: Nil
).as(none) )
.as(none)
)(_.some.pure) )(_.some.pure)
override def expectNoExport(token: Token[S]): State[X, Unit] = override def checkArrowCallResults(
report( token: Token[S],
arrowType: ArrowType,
results: List[Name[S]]
): State[X, Unit] = for {
_ <- results
.drop(arrowType.codomain.length)
.traverse_(result =>
report
.error(
result,
"Types mismatch. Cannot assign to a variable " +
"the result of a call that returns nothing"
)
)
_ <- report
.warning(
token, token,
"Types mismatch. Cannot assign to a variable the result of a call that returns nothing" s"Arrow returns ${arrowType.codomain.length match {
).as(()) case 0 => "no values"
case 1 => "a value"
case i => s"$i values"
}} values, but ${results.length match {
case 0 => "none are"
case 1 => "only one is"
case i => s"only $i are"
}} used"
)
.whenA(arrowType.codomain.length > results.length)
} yield ()
override def checkArgumentsNumber( override def checkArgumentsNumber(
token: Token[S], token: Token[S],
@ -395,10 +443,12 @@ class TypesInterpreter[S[_], X](implicit
): State[X, Boolean] = ): State[X, Boolean] =
if (expected == givenNum) State.pure(true) if (expected == givenNum) State.pure(true)
else else
report( report
.error(
token, token,
s"Number of arguments doesn't match the function type, expected: ${expected}, given: $givenNum" s"Number of arguments doesn't match the function type, expected: ${expected}, given: $givenNum"
).as(false) )
.as(false)
override def beginArrowScope(token: ArrowTypeToken[S]): State[X, ArrowType] = override def beginArrowScope(token: ArrowTypeToken[S]): State[X, ArrowType] =
Applicative[ST] Applicative[ST]
@ -434,37 +484,49 @@ class TypesInterpreter[S[_], X](implicit
values: NonEmptyList[(ValueToken[S], ValueRaw)] values: NonEmptyList[(ValueToken[S], ValueRaw)]
): State[X, Boolean] = ): State[X, Boolean] =
mapStackHeadM[Boolean]( mapStackHeadM[Boolean](
report(values.head._1, "Fatal: checkArrowReturn has no matching beginArrowScope").as(false) report
.error(values.head._1, "Fatal: checkArrowReturn has no matching beginArrowScope")
.as(false)
)(frame => )(frame =>
if (frame.retVals.nonEmpty) if (frame.retVals.nonEmpty)
report( report
.error(
values.head._1, values.head._1,
"Return expression was already used in scope; you can use only one Return in an arrow declaration, use conditional return pattern if you need to return based on condition" "Return expression was already used in scope; you can use only one Return in an arrow declaration, use conditional return pattern if you need to return based on condition"
).as(frame -> false) )
.as(frame -> false)
else if (frame.token.res.isEmpty) else if (frame.token.res.isEmpty)
report( report
.error(
values.head._1, values.head._1,
"No return type declared for this arrow, please remove `<- ...` expression or add `-> ...` return type(s) declaration to the arrow" "No return type declared for this arrow, please remove `<- ...` expression or add `-> ...` return type(s) declaration to the arrow"
).as(frame -> false) )
.as(frame -> false)
else if (frame.token.res.length > values.length) else if (frame.token.res.length > values.length)
report( report
.error(
values.last._1, values.last._1,
s"Expected ${frame.token.res.length - values.length} more values to be returned, see return type declaration" s"Expected ${frame.token.res.length - values.length} more values to be returned, see return type declaration"
).as(frame -> false) )
.as(frame -> false)
else if (frame.token.res.length < values.length) else if (frame.token.res.length < values.length)
report( report
.error(
values.toList.drop(frame.token.res.length).headOption.getOrElse(values.last)._1, values.toList.drop(frame.token.res.length).headOption.getOrElse(values.last)._1,
s"Too many values are returned from this arrow, this one is unexpected. Defined return type: ${frame.arrowType.codomain}" s"Too many values are returned from this arrow, this one is unexpected. Defined return type: ${frame.arrowType.codomain}"
).as(frame -> false) )
.as(frame -> false)
else else
frame.arrowType.codomain.toList frame.arrowType.codomain.toList
.zip(values.toList) .zip(values.toList)
.traverse { case (returnType, (token, returnValue)) => .traverse { case (returnType, (token, returnValue)) =>
if (!returnType.acceptsValueOf(returnValue.`type`)) if (!returnType.acceptsValueOf(returnValue.`type`))
report( report
.error(
token, token,
s"Wrong value type, expected: $returnType, given: ${returnValue.`type`}" s"Wrong value type, expected: $returnType, given: ${returnValue.`type`}"
).as(none) )
.as(none)
else returnValue.some.pure[SX] else returnValue.some.pure[SX]
} }
.map(_.sequence) .map(_.sequence)
@ -473,14 +535,16 @@ class TypesInterpreter[S[_], X](implicit
override def endArrowScope(token: Token[S]): State[X, List[ValueRaw]] = override def endArrowScope(token: Token[S]): State[X, List[ValueRaw]] =
mapStackHeadM( mapStackHeadM(
report(token, "Fatal: endArrowScope has no matching beginArrowScope").as(Nil) report.error(token, "Fatal: endArrowScope has no matching beginArrowScope").as(Nil)
)(frame => )(frame =>
if (frame.token.res.isEmpty) (frame -> Nil).pure if (frame.token.res.isEmpty) (frame -> Nil).pure
else if (frame.retVals.isEmpty) else if (frame.retVals.isEmpty)
report( report
.error(
frame.token.res.headOption.getOrElse(frame.token), frame.token.res.headOption.getOrElse(frame.token),
"Return type is defined for the arrow, but nothing returned. Use `<- value, ...` as the last expression inside function body." "Return type is defined for the arrow, but nothing returned. Use `<- value, ...` as the last expression inside function body."
).as(frame -> Nil) )
.as(frame -> Nil)
else (frame -> frame.retVals.getOrElse(Nil)).pure else (frame -> frame.retVals.getOrElse(Nil)).pure
) <* stack.endScope ) <* stack.endScope
@ -495,10 +559,12 @@ class TypesInterpreter[S[_], X](implicit
.flatMap { .flatMap {
case Some(_) => case Some(_) =>
// TODO: Point to both locations here // TODO: Point to both locations here
report( report
.error(
token, token,
s"Name `${name}` was already defined here" s"Name `${name}` was already defined here"
).as(ifDefined) )
.as(ifDefined)
case None => ifNotDefined case None => ifNotDefined
} }
} }

View File

@ -13,8 +13,7 @@ import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.Inside import org.scalatest.Inside
import cats.~> import cats.~>
import cats.data.Chain import cats.data.{Chain, EitherNec, NonEmptyChain}
import cats.data.NonEmptyChain
import cats.syntax.show.* import cats.syntax.show.*
import cats.syntax.traverse.* import cats.syntax.traverse.*
import cats.syntax.foldable.* import cats.syntax.foldable.*
@ -32,20 +31,30 @@ class SemanticsSpec extends AnyFlatSpec with Matchers with Inside {
val semantics = new RawSemantics[Span.S]() val semantics = new RawSemantics[Span.S]()
def insideBody(script: String)(test: RawTag.Tree => Any): Unit = def insideResult(script: String)(
inside(parser(script)) { case Validated.Valid(ast) => test: PartialFunction[
(
Chain[SemanticWarning[Span.S]],
EitherNec[SemanticError[Span.S], RawContext]
),
Any
]
): Unit = inside(parser(script)) { case Validated.Valid(ast) =>
val init = RawContext.blank val init = RawContext.blank
inside(semantics.process(ast, init)) { case Validated.Valid(ctx) => inside(semantics.process(ast, init).value.run)(test)
}
def insideBody(script: String)(test: RawTag.Tree => Any): Unit =
insideResult(script) { case (_, Right(ctx)) =>
inside(ctx.funcs.headOption) { case Some((_, func)) => inside(ctx.funcs.headOption) { case Some((_, func)) =>
test(func.arrow.body) test(func.arrow.body)
} }
} }
}
def insideSemErrors(script: String)(test: NonEmptyChain[SemanticError[Span.S]] => Any): Unit = def insideSemErrors(script: String)(test: NonEmptyChain[SemanticError[Span.S]] => Any): Unit =
inside(parser(script)) { case Validated.Valid(ast) => inside(parser(script)) { case Validated.Valid(ast) =>
val init = RawContext.blank val init = RawContext.blank
inside(semantics.process(ast, init)) { case Validated.Invalid(errors) => inside(semantics.process(ast, init).value.value) { case Left(errors) =>
test(errors) test(errors)
} }
} }
@ -648,4 +657,21 @@ class SemanticsSpec extends AnyFlatSpec with Matchers with Inside {
atLeast(1, errors.toChain.toList) shouldBe a[RulesViolated[Span.S]] atLeast(1, errors.toChain.toList) shouldBe a[RulesViolated[Span.S]]
} }
} }
it should "produce warning on unused call results" in {
val script = """|func test() -> string, string:
| stream: *string
| stream <<- "a"
| stream <<- "b"
| <- stream[0], stream[1]
|
|func main() -> string:
| a <- test()
| <- a
|""".stripMargin
insideResult(script) { case (warnings, Right(_)) =>
warnings.exists(_.hints.exists(_.contains("used"))) should be(true)
}
}
} }

View File

@ -5,11 +5,12 @@ import aqua.parser.lexer.{Name, Token}
import aqua.parser.lift.Span import aqua.parser.lift.Span
import aqua.raw.{Raw, RawContext} import aqua.raw.{Raw, RawContext}
import aqua.semantics.expr.func.ClosureSem import aqua.semantics.expr.func.ClosureSem
import aqua.semantics.rules.errors.ReportErrors
import aqua.semantics.rules.abilities.{AbilitiesAlgebra, AbilitiesInterpreter, AbilitiesState} import aqua.semantics.rules.abilities.{AbilitiesAlgebra, AbilitiesInterpreter, AbilitiesState}
import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra} import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra}
import aqua.semantics.rules.names.{NamesAlgebra, NamesInterpreter, NamesState} import aqua.semantics.rules.names.{NamesAlgebra, NamesInterpreter, NamesState}
import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter, TypesState} import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter, TypesState}
import aqua.semantics.rules.mangler.{ManglerAlgebra, ManglerInterpreter}
import aqua.semantics.rules.report.{ReportAlgebra, ReportInterpreter}
import aqua.types.* import aqua.types.*
import cats.data.State import cats.data.State
@ -17,11 +18,12 @@ import cats.{~>, Id}
import monocle.Lens import monocle.Lens
import monocle.macros.GenLens import monocle.macros.GenLens
import monocle.syntax.all.* import monocle.syntax.all.*
import aqua.semantics.rules.mangler.ManglerAlgebra
import aqua.semantics.rules.mangler.ManglerInterpreter
object Utils { object Utils {
given ReportAlgebra[Id, State[CompilerState[Id], *]] =
new ReportInterpreter[Id, CompilerState[Id]]
given ManglerAlgebra[State[CompilerState[Id], *]] = given ManglerAlgebra[State[CompilerState[Id], *]] =
new ManglerInterpreter[CompilerState[Id]] new ManglerInterpreter[CompilerState[Id]]

View File

@ -7,6 +7,7 @@ import aqua.semantics.rules.definitions.{DefinitionsAlgebra, DefinitionsInterpre
import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter, TypesState} import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter, TypesState}
import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra} import aqua.semantics.rules.locations.{DummyLocationsInterpreter, LocationsAlgebra}
import aqua.semantics.rules.mangler.{ManglerAlgebra, ManglerInterpreter} import aqua.semantics.rules.mangler.{ManglerAlgebra, ManglerInterpreter}
import aqua.semantics.rules.report.{ReportAlgebra, ReportInterpreter}
import aqua.raw.value.{ApplyBinaryOpRaw, LiteralRaw} import aqua.raw.value.{ApplyBinaryOpRaw, LiteralRaw}
import aqua.raw.RawContext import aqua.raw.RawContext
import aqua.types.* import aqua.types.*
@ -32,9 +33,10 @@ class ValuesAlgebraSpec extends AnyFlatSpec with Matchers with Inside {
def algebra() = { def algebra() = {
type Interpreter[A] = State[TestState, A] type Interpreter[A] = State[TestState, A]
given ReportAlgebra[Id, Interpreter] =
new ReportInterpreter[Id, CompilerState[Id]]
given LocationsAlgebra[Id, Interpreter] = given LocationsAlgebra[Id, Interpreter] =
new DummyLocationsInterpreter[Id, CompilerState[Id]] new DummyLocationsInterpreter[Id, CompilerState[Id]]
given ManglerAlgebra[Interpreter] = given ManglerAlgebra[Interpreter] =
new ManglerInterpreter[CompilerState[Id]] new ManglerInterpreter[CompilerState[Id]]
given TypesAlgebra[Id, Interpreter] = given TypesAlgebra[Id, Interpreter] =

View File

@ -4,26 +4,20 @@ import aqua.parser.expr.ConstantExpr
import aqua.raw.ConstantRaw import aqua.raw.ConstantRaw
import aqua.raw.value.LiteralRaw import aqua.raw.value.LiteralRaw
import cats.data.{NonEmptyList, Validated, ValidatedNel} import cats.data.{NonEmptyList, Validated, ValidatedNec}
import cats.syntax.traverse.*
import cats.syntax.either.*
object Constants { object Constants {
def parse(strs: List[String]): ValidatedNel[String, List[ConstantRaw]] = { def parse(strs: List[String]): ValidatedNec[String, List[ConstantRaw]] =
val parsed = strs.map(s => ConstantExpr.onlyLiteral.parseAll(s)) strs.traverse(s =>
ConstantExpr.onlyLiteral
val errors = parsed.zip(strs).collect { case (Left(_), str) => .parseAll(s)
str .leftMap(_ => s"Invalid constant definition '$s'.")
} .toValidatedNec
.map { case (name, literal) =>
NonEmptyList ConstantRaw(name.value, LiteralRaw(literal.value, literal.ts), false)
.fromList(errors)
.fold(
Validated.validNel[String, List[ConstantRaw]](parsed.collect { case Right(v) =>
ConstantRaw(v._1.value, LiteralRaw(v._2.value, v._2.ts), false)
})
) { errors =>
val errorMsgs = errors.map(str => s"Invalid constant definition '$str'.")
Validated.invalid(errorMsgs)
}
} }
)
} }

View File

@ -3,8 +3,9 @@ package aqua.logging
import cats.syntax.option.* import cats.syntax.option.*
import cats.syntax.either.* import cats.syntax.either.*
import cats.syntax.foldable.* import cats.syntax.foldable.*
import cats.data.Validated.{invalidNel, validNel} import cats.syntax.validated.*
import cats.data.{NonEmptyList, Validated, ValidatedNel} import cats.data.Validated.*
import cats.data.{NonEmptyList, Validated, ValidatedNec}
import scribe.Level import scribe.Level
case class LogLevels( case class LogLevels(
@ -20,10 +21,10 @@ object LogLevels {
def apply(level: Level): LogLevels = LogLevels(level, level, level) def apply(level: Level): LogLevels = LogLevels(level, level, level)
def levelFromString(s: String): ValidatedNel[String, Level] = def levelFromString(s: String): ValidatedNec[String, Level] =
LogLevel.stringToLogLevel LogLevel.stringToLogLevel
.get(s.toLowerCase.trim()) .get(s.toLowerCase.trim())
.toValidNel(s"Invalid log-level '$s'. $logHelpMessage") .toValidNec(s"Invalid log-level '$s'. $logHelpMessage")
lazy val error = lazy val error =
s"Invalid log-level format. $logHelpMessage" s"Invalid log-level format. $logHelpMessage"
@ -32,17 +33,17 @@ object LogLevels {
name: String, name: String,
level: String, level: String,
logLevels: LogLevels logLevels: LogLevels
): Validated[NonEmptyList[String], LogLevels] = { ): ValidatedNec[String, LogLevels] = {
levelFromString(level).andThen { level => levelFromString(level).andThen { level =>
name.trim().toLowerCase() match { name.trim().toLowerCase() match {
case "compiler" => case "compiler" =>
validNel(logLevels.copy(compiler = level)) logLevels.copy(compiler = level).validNec
case "fluencejs" => case "fluencejs" =>
validNel(logLevels.copy(fluencejs = level)) logLevels.copy(fluencejs = level).validNec
case "aquavm" => case "aquavm" =>
validNel(logLevels.copy(aquavm = level)) logLevels.copy(aquavm = level).validNec
case s => case s =>
invalidNel( invalidNec(
s"Unknown component '$s' in log-level. Please use one of these: 'aquavm', 'compiler' and 'fluencejs'" s"Unknown component '$s' in log-level. Please use one of these: 'aquavm', 'compiler' and 'fluencejs'"
) )
} }
@ -51,14 +52,14 @@ object LogLevels {
// Format: '<log-level>' or 'compiler=<log-level>,fluencejs=<log-level>,aquavm=<log-level>', // Format: '<log-level>' or 'compiler=<log-level>,fluencejs=<log-level>,aquavm=<log-level>',
// where <log-level> is one of these strings: 'all', 'trace', 'debug', 'info', 'warn', 'error', 'off' // where <log-level> is one of these strings: 'all', 'trace', 'debug', 'info', 'warn', 'error', 'off'
def fromString(s: String): ValidatedNel[String, LogLevels] = def fromString(s: String): ValidatedNec[String, LogLevels] =
s.split(",") s.split(",")
.toList .toList
.foldLeftM(LogLevels()) { case (levels, level) => .foldLeftM(LogLevels()) { case (levels, level) =>
level.split("=").toList match { level.split("=").toList match {
case n :: l :: Nil => fromStrings(n, l, levels).toEither case n :: l :: Nil => fromStrings(n, l, levels).toEither
case l :: Nil => levelFromString(l).map(apply).toEither case l :: Nil => levelFromString(l).map(apply).toEither
case _ => invalidNel(error).toEither case _ => error.invalidNec.toEither
} }
} }
.toValidated .toValidated