Go-to definitions location between files (#516)

This commit is contained in:
Dima 2022-06-02 14:31:31 +04:00 committed by GitHub
parent 6064659610
commit 5a7450bce1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 929 additions and 498 deletions

View File

@ -17,7 +17,7 @@ val scribeV = "3.7.1"
name := "aqua-hll"
val commons = Seq(
baseAquaVersion := "0.7.2",
baseAquaVersion := "0.7.3",
version := baseAquaVersion.value + "-" + sys.env.getOrElse("BUILD_NUMBER", "SNAPSHOT"),
scalaVersion := dottyVersion,
libraryDependencies ++= Seq(

View File

@ -1,6 +1,6 @@
package aqua
import aqua.compiler.{AquaCompiler, AquaCompilerConf}
import aqua.compiler.{AquaCompiler, AquaCompilerConf, CompilerAPI}
import aqua.ErrorRendering.showError
import aqua.files.{AquaFileSources, AquaFilesIO, FileModuleId}
import aqua.io.AquaFileError
@ -50,7 +50,7 @@ class FuncCompiler[F[_]: Files: AquaIO: Async](
sources = new AquaFileSources[F](input, prelude.importPaths ++ imports)
// compile only context to wrap and call function later
compileResult <- Clock[F].timed(
AquaCompiler
CompilerAPI
.compileToContext[F, AquaFileError, FileModuleId, FileSpan.F](
sources,
SpanParser.parser,
@ -61,7 +61,7 @@ class FuncCompiler[F[_]: Files: AquaIO: Async](
(compileTime, contextV) = compileResult
} yield {
logger.debug(s"Compile time: ${compileTime.toMillis}ms")
contextV.andThen(c => findFunction(c._2, func))
contextV.andThen(c => findFunction(c, func))
}
}
}

View File

@ -1,7 +1,7 @@
package aqua
import aqua.backend.{Backend, Generated}
import aqua.compiler.{AquaCompiled, AquaCompiler, AquaCompilerConf, AquaError}
import aqua.compiler.{AquaCompiled, AquaCompiler, AquaCompilerConf, AquaError, CompilerAPI}
import aqua.files.{AquaFileSources, FileModuleId}
import aqua.io.*
import aqua.model.AquaContext
@ -44,7 +44,7 @@ object AquaPathCompiler extends Logging {
(for {
prelude <- Prelude.init()
sources = new AquaFileSources[F](srcPath, imports ++ prelude.importPaths)
compiler <- AquaCompiler
compiler <- CompilerAPI
.compileTo[F, AquaFileError, FileModuleId, FileSpan.F, String](
sources,
SpanParser.parser,

View File

@ -9,7 +9,7 @@ import aqua.raw.RawPart.Parts
import aqua.raw.{RawContext, RawPart}
import aqua.res.AquaRes
import aqua.semantics.{CompilerState, Semantics}
import aqua.semantics.header.HeaderSem
import aqua.semantics.header.{HeaderHandler, HeaderSem, Picker}
import cats.data.*
import cats.data.Validated.{validNec, Invalid, Valid}
import cats.parse.Parser0
@ -22,25 +22,25 @@ import cats.syntax.semigroup.*
import cats.{~>, Comonad, Monad, Monoid, Order}
import scribe.Logging
trait AquaCompiler[C] extends Logging {
class AquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad, C: Monoid: Picker](
headerHandler: HeaderHandler[S, C],
semantics: Semantics[S, C]
) extends Logging {
type Err[I, E, S[_]] = AquaError[I, E, S]
// TODO: find the way to replace RawContext with C; maybe move some functions to RawContext-specific subclasses, etc.
type Ctx[I] = NonEmptyMap[I, RawContext]
// TODO: remove CompilerState[S] from the right
type ValidatedCtx[I, E, S[_]] = ValidatedNec[Err[I, E, S], (CompilerState[S], Ctx[I])]
type ValidatedCtxT[I, E, S[_]] = ValidatedCtx[I, E, S] => ValidatedCtx[I, E, S]
type Err = AquaError[I, E, S]
type Ctx = NonEmptyMap[I, C]
private def linkModules[E, I: Order, S[_]: Comonad](
type ValidatedCtx = ValidatedNec[Err, Ctx]
type ValidatedCtxT = ValidatedCtx => ValidatedCtx
private def linkModules(
modules: Modules[
I,
Err[I, E, S],
ValidatedCtxT[I, E, S]
Err,
ValidatedCtxT
],
cycleError: List[AquaModule[I, Err[I, E, S], ValidatedCtxT[I, E, S]]] => Err[I, E, S]
)(implicit
rc: Monoid[RawContext]
): ValidatedNec[Err[I, E, S], (Chain[CompilerState[S]], Chain[AquaProcessed[I]])] = {
cycleError: List[AquaModule[I, Err, ValidatedCtxT]] => Err
): ValidatedNec[Err, Map[I, ValidatedCtx]] = {
logger.trace("linking modules...")
Linker
@ -48,65 +48,23 @@ trait AquaCompiler[C] extends Logging {
modules,
cycleError,
// By default, provide an empty context for this module's id
i => validNec((CompilerState[S](), NonEmptyMap.one(i, Monoid.empty[RawContext])))
i => validNec(NonEmptyMap.one(i, Monoid.empty[C]))
)
.andThen { filesWithContext =>
logger.trace("linking finished")
filesWithContext
.foldLeft[
(
ValidatedNec[Err[I, E, S], (Chain[CompilerState[S]], Chain[AquaProcessed[I]])],
AquaContext.Cache
)
](
validNec((Chain.nil, Chain.nil)) -> AquaContext.Cache()
) {
case ((acc, cache), (i, Valid(result))) =>
val (processed, cacheProcessed) =
result._2.toNel.toList.foldLeft[
((Chain[CompilerState[S]], Chain[AquaProcessed[I]]), AquaContext.Cache)
](
(Chain.nil, Chain.nil) -> cache
) { case ((acc, accCache), (i, c)) =>
logger.trace(s"Going to prepare exports for ${i}...")
val (exp, expCache) = AquaContext.exportsFromRaw(c, accCache)
logger.trace(s"AquaProcessed prepared for ${i}")
(acc._1 :+ result._1, acc._2 :+ AquaProcessed(i, exp)) -> expCache
}
acc.combine(
validNec(
processed
)
) -> cacheProcessed
case ((acc, cache), (_, Invalid(errs))) =>
acc.combine(Invalid(errs)) -> cache
}
._1
}
}
private def compileRaw[F[_]: Monad, E, I: Order, S[_]: Comonad](
def compileRaw(
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], (Chain[CompilerState[S]], Chain[AquaProcessed[I]])]] = {
implicit val rc: Monoid[RawContext] = RawContext
.implicits(
RawContext.blank
.copy(parts = Chain.fromSeq(config.constantsList).map(const => RawContext.blank -> const))
)
.rawContextMonoid
type CErr = Err[I, E, S]
type VCtx = ValidatedCtx[I, E, S]
parser: I => String => ValidatedNec[ParserError[S], Ast[S]]
): F[Validated[NonEmptyChain[Err], Map[I, ValidatedCtx]]] = {
logger.trace("starting resolving sources...")
new AquaParser[F, E, I, S](sources, parser)
.resolve[VCtx](mod =>
.resolve[ValidatedCtx](mod =>
context =>
// Context with prepared imports
context.andThen { case (_, ctx) =>
// To manage imports, exports run HeaderSem
HeaderSem
context.andThen { ctx =>
// To manage imports, exports run HeaderHandler
headerHandler
.sem(
mod.imports.view
.mapValues(ctx(_))
@ -117,19 +75,19 @@ trait AquaCompiler[C] extends Logging {
.andThen { headerSem =>
// Analyze the body, with prepared initial context
logger.trace("semantic processing...")
Semantics
semantics
.process(
mod.body,
headerSem.initCtx
)
// Handle exports, declares - finalize the resulting context
.andThen { case (state, ctx) =>
headerSem.finCtx(ctx).map(r => (state, r))
.andThen { ctx =>
headerSem.finCtx(ctx)
}
.map { case (state, rc) => (state, NonEmptyMap.one(mod.id, rc)) }
.map { rc => NonEmptyMap.one(mod.id, rc) }
}
// The whole chain returns a semantics error finally
.leftMap(_.map[CErr](CompileError(_)))
.leftMap(_.map[Err](CompileError(_)))
}
)
.map(
@ -137,70 +95,4 @@ trait AquaCompiler[C] extends Logging {
)
}
// Get only compiled model
def compileToContext[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], (Chain[CompilerState[S]], Chain[AquaContext])]] = {
compileRaw(sources, parser, config).map(_.map { case (st, compiled) =>
(
st,
compiled.map { ap =>
logger.trace("generating output...")
ap.context
}
)
})
}
// Get result generated by backend
def compile[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
backend: Backend.Transform,
config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], Chain[AquaCompiled[I]]]] = {
compileRaw(sources, parser, config).map(_.map { case (_, compiled) =>
compiled.map { ap =>
logger.trace("generating output...")
val res = backend.transform(ap.context)
val compiled = backend.generate(res)
AquaCompiled(ap.id, compiled, res.funcs.length.toInt, res.services.length.toInt)
}
})
}
def compileTo[F[_]: Monad, E, I: Order, S[_]: Comonad, T](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
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, backend, config).flatMap {
case Valid(compiled) =>
compiled.map { ac =>
write(ac).map(
_.map(
_.bimap[NonEmptyChain[AquaError[I, E, S]], Chain[T]](
e => NonEmptyChain.one(OutputError(ac, e)),
Chain.one
)
)
)
}.toList
.traverse(identity)
.map(
_.flatten
.foldLeft[ValidatedNec[AquaError[I, E, S], Chain[T]]](validNec(Chain.nil))(
_ combine _
)
)
case Validated.Invalid(errs) =>
Validated.invalid[NonEmptyChain[AquaError[I, E, S]], Chain[T]](errs).pure[F]
}
}
object AquaCompiler extends AquaCompiler[RawContext]

View File

@ -1,6 +1,5 @@
package aqua.compiler
import aqua.compiler.AquaCompiler.logger
import aqua.linker.{AquaModule, Modules}
import aqua.parser.head.{FilenameExpr, ImportExpr}
import aqua.parser.lift.{LiftParser, Span}

View File

@ -0,0 +1,229 @@
package aqua.compiler
import aqua.backend.Backend
import aqua.linker.{AquaModule, Linker, Modules}
import aqua.model.AquaContext
import aqua.parser.lift.{LiftParser, Span}
import aqua.parser.{Ast, ParserError}
import aqua.raw.RawPart.Parts
import aqua.raw.{RawContext, RawPart}
import aqua.res.AquaRes
import aqua.semantics.{CompilerState, LspSemantics, RawSemantics, Semantics}
import aqua.semantics.header.{HeaderHandler, HeaderSem}
import aqua.semantics.lsp.LspContext
import cats.data.*
import cats.data.Validated.{validNec, Invalid, Valid}
import cats.parse.Parser0
import cats.syntax.applicative.*
import cats.syntax.flatMap.*
import cats.syntax.functor.*
import cats.syntax.monoid.*
import cats.syntax.traverse.*
import cats.syntax.semigroup.*
import cats.{~>, Comonad, Monad, Monoid, Order}
import scribe.Logging
import scala.collection.MapView
object CompilerAPI extends Logging {
private def toAquaProcessed[I: Order, E, S[_]: Comonad](
filesWithContext: Map[
I,
ValidatedNec[AquaError[I, E, S], NonEmptyMap[I, RawContext]]
]
): ValidatedNec[AquaError[I, E, S], Chain[AquaProcessed[I]]] = {
logger.trace("linking finished")
filesWithContext
.foldLeft[
(
ValidatedNec[AquaError[I, E, S], Chain[AquaProcessed[I]]],
AquaContext.Cache
)
](
validNec(Chain.nil) -> AquaContext.Cache()
) {
case ((acc, cache), (_, Valid(result))) =>
val (processed, cacheProcessed) =
result.toNel.toList.foldLeft[
(Chain[AquaProcessed[I]], AquaContext.Cache)
](
Chain.nil -> cache
) { case ((acc, accCache), (i, rawContext)) =>
logger.trace(s"Going to prepare exports for $i...")
val (exp, expCache) = AquaContext.exportsFromRaw(rawContext, accCache)
logger.trace(s"AquaProcessed prepared for $i")
(acc :+ AquaProcessed(i, exp)) -> expCache
}
acc.combine(
validNec(
processed
)
) -> cacheProcessed
case ((acc, cache), (_, Invalid(errs))) =>
acc.combine(Invalid(errs)) -> cache
}
._1
}
private def getLspAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad](
config: AquaCompilerConf
): AquaCompiler[F, E, I, S, LspContext[S]] = {
implicit val rc: Monoid[LspContext[S]] = LspContext
.implicits(
LspContext
.blank[S]
.copy(raw =
RawContext.blank.copy(parts =
Chain.fromSeq(config.constantsList).map(const => RawContext.blank -> const)
)
)
)
.lspContextMonoid
implicit val headerSemMonoid: Monoid[HeaderSem[S, LspContext[S]]] =
new Monoid[HeaderSem[S, LspContext[S]]] {
override def empty: HeaderSem[S, LspContext[S]] = HeaderSem(rc.empty, (c, _) => validNec(c))
override def combine(
a: HeaderSem[S, LspContext[S]],
b: HeaderSem[S, LspContext[S]]
): HeaderSem[S, LspContext[S]] = {
HeaderSem(
a.initCtx |+| b.initCtx,
(c, i) => a.finInitCtx(c, i).andThen(b.finInitCtx(_, i))
)
}
}
val semantics = new LspSemantics[S]()
new AquaCompiler[F, E, I, S, LspContext[S]](new HeaderHandler[S, LspContext[S]](), semantics)
}
private def getAquaCompiler[F[_]: Monad, E, I: Order, S[_]: Comonad](
config: AquaCompilerConf
): AquaCompiler[F, E, I, S, RawContext] = {
implicit val rc: Monoid[RawContext] = RawContext
.implicits(
RawContext.blank
.copy(parts = Chain.fromSeq(config.constantsList).map(const => RawContext.blank -> const))
)
.rawContextMonoid
implicit val headerSemMonoid: Monoid[HeaderSem[S, RawContext]] =
new Monoid[HeaderSem[S, RawContext]] {
override def empty: HeaderSem[S, RawContext] = HeaderSem(rc.empty, (c, _) => validNec(c))
override def combine(
a: HeaderSem[S, RawContext],
b: HeaderSem[S, RawContext]
): HeaderSem[S, RawContext] =
HeaderSem(
a.initCtx |+| b.initCtx,
(c, i) => a.finInitCtx(c, i).andThen(b.finInitCtx(_, i))
)
}
val semantics = new RawSemantics[S]()
new AquaCompiler[F, E, I, S, RawContext](new HeaderHandler[S, RawContext](), semantics)
}
// Get result generated by backend
def compile[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
backend: Backend.Transform,
config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], Chain[AquaCompiled[I]]]] = {
val compiler = getAquaCompiler[F, E, I, S](config)
compiler
.compileRaw(sources, parser)
.map(_.andThen { filesWithContext =>
toAquaProcessed(filesWithContext)
})
.map(_.map { compiled =>
compiled.map { ap =>
logger.trace("generating output...")
val res = backend.transform(ap.context)
val compiled = backend.generate(res)
AquaCompiled(ap.id, compiled, res.funcs.length.toInt, res.services.length.toInt)
}
})
}
def compileTo[F[_]: Monad, E, I: Order, S[_]: Comonad, T](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
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, backend, config).flatMap {
case Valid(compiled) =>
compiled.map { ac =>
write(ac).map(
_.map(
_.bimap[NonEmptyChain[AquaError[I, E, S]], Chain[T]](
e => NonEmptyChain.one(OutputError(ac, e)),
Chain.one
)
)
)
}.toList
.traverse(identity)
.map(
_.flatten
.foldLeft[ValidatedNec[AquaError[I, E, S], Chain[T]]](validNec(Chain.nil))(
_ combine _
)
)
case Validated.Invalid(errs) =>
Validated.invalid[NonEmptyChain[AquaError[I, E, S]], Chain[T]](errs).pure[F]
}
def compileToLsp[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf
): F[Validated[NonEmptyChain[AquaError[I, E, S]], Map[I, Validated[NonEmptyChain[
AquaError[I, E, S]
], Map[I, LspContext[S]]]]]] = {
val compiler = getLspAquaCompiler[F, E, I, S](config)
compiler
.compileRaw(sources, parser)
.map { v =>
v.map { innerMap =>
innerMap.view.mapValues { vCtx =>
vCtx.map {
_.toSortedMap.toMap
}
}.toMap
}
}
}
def compileToContext[F[_]: Monad, E, I: Order, S[_]: Comonad](
sources: AquaSources[F, E, I],
parser: I => String => ValidatedNec[ParserError[S], Ast[S]],
config: AquaCompilerConf
): F[ValidatedNec[AquaError[I, E, S], Chain[AquaContext]]] = {
val compiler = getAquaCompiler[F, E, I, S](config)
compiler
.compileRaw(sources, parser)
.map(_.andThen { filesWithContext =>
toAquaProcessed(filesWithContext)
})
.map(_.map { compiled =>
compiled.map { ap =>
logger.trace("generating output...")
ap.context
}
})
}
}

View File

@ -7,6 +7,7 @@ import aqua.parser.ParserError
import aqua.parser.Ast
import aqua.parser.Parser
import aqua.parser.lift.Span
import aqua.parser.lift.Span.S
import aqua.raw.value.{LiteralRaw, ValueRaw, VarRaw}
import aqua.res.{
ApRes,
@ -19,38 +20,50 @@ import aqua.res.{
RestrictionRes,
SeqRes
}
import aqua.semantics.lsp.LspContext
import aqua.types.{ArrayType, LiteralType, ScalarType, StreamType, Type}
import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers
import cats.Id
import cats.data.{Chain, NonEmptyChain, Validated, ValidatedNec}
import cats.data.{Chain, NonEmptyChain, NonEmptyMap, Validated, ValidatedNec}
import cats.instances.string.*
import cats.syntax.show.*
class AquaCompilerSpec extends AnyFlatSpec with Matchers {
private def aquaSource(src: Map[String, String], imports: Map[String, String]) = {
new AquaSources[Id, String, String] {
override def sources: Id[ValidatedNec[String, Chain[(String, String)]]] =
Validated.validNec(Chain.fromSeq(src.toSeq))
override def resolveImport(from: String, imp: String): Id[ValidatedNec[String, String]] =
Validated.validNec(imp)
override def load(file: String): Id[ValidatedNec[String, String]] =
Validated.fromEither(
(imports ++ src)
.get(file)
.toRight(NonEmptyChain.one(s"Cannot load imported file $file"))
)
}
}
private def compileToContext(src: Map[String, String], imports: Map[String, String]) =
AquaCompiler
CompilerAPI
.compileToContext[Id, String, String, Span.S](
new AquaSources[Id, String, String] {
override def sources: Id[ValidatedNec[String, Chain[(String, String)]]] =
Validated.validNec(Chain.fromSeq(src.toSeq))
override def resolveImport(from: String, imp: String): Id[ValidatedNec[String, String]] =
Validated.validNec(imp)
override def load(file: String): Id[ValidatedNec[String, String]] =
Validated.fromEither(
(imports ++ src)
.get(file)
.toRight(NonEmptyChain.one(s"Cannot load imported file $file"))
)
},
aquaSource(src, imports),
id => txt => Parser.parse(Parser.parserSchema)(txt),
AquaCompilerConf()
)
private def compileToLsp(src: Map[String, String], imports: Map[String, String]) =
CompilerAPI
.compileToLsp[Id, String, String, Span.S](
aquaSource(src, imports),
id => txt => Parser.parse(Parser.parserSchema)(txt),
AquaCompilerConf()
)
.map(_._2)
"aqua compiler" should "compile a simple snipped to the right context" in {
@ -296,5 +309,4 @@ class AquaCompilerSpec extends AnyFlatSpec with Matchers {
) should be(true)
}
}

View File

@ -4,12 +4,14 @@ import aqua.compiler.*
import aqua.files.{AquaFileSources, AquaFilesIO, FileModuleId}
import aqua.io.*
import aqua.parser.lexer.Token
import aqua.parser.lift.FileSpan.F
import aqua.parser.lift.{FileSpan, Span}
import aqua.parser.{ArrowReturnError, BlockIndentError, LexerError, ParserError}
import aqua.semantics.{HeaderError, RulesViolated, WrongAST}
import aqua.semantics.lsp.LspContext
import aqua.semantics.{CompilerState, HeaderError, RulesViolated, WrongAST}
import aqua.{AquaIO, SpanParser}
import cats.data.NonEmptyChain
import cats.data.Validated.{Invalid, Valid}
import cats.data.{NonEmptyChain, Validated}
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.effect.IO
import cats.effect.unsafe.implicits.global
import fs2.io.file.{Files, Path}
@ -29,15 +31,24 @@ case class CompilationResult(
)
@JSExportAll
case class TokenLocation(name: String, start: Int, end: Int)
case class TokenLocation(name: String, startLine: Int, startCol: Int, endLine: Int, endCol: Int)
@JSExportAll
case class TokenLink(current: TokenLocation, definition: TokenLocation)
object TokenLocation {
def apply(span: FileSpan): TokenLocation = {
TokenLocation(span.name, span.span.startIndex, span.span.endIndex)
def fromSpan(span: FileSpan): Option[TokenLocation] = {
val start = span.locationMap.value.toLineCol(span.span.startIndex)
val end = span.locationMap.value.toLineCol(span.span.endIndex)
for {
startLC <- start
endLC <- end
} yield {
TokenLocation(span.name, startLC._1, startLC._2, endLC._1, endLC._2)
}
}
}
@ -116,40 +127,72 @@ object AquaLSP extends App with Logging {
pathStr: String,
imports: scalajs.js.Array[String]
): scalajs.js.Promise[CompilationResult] = {
logger.debug(s"Compiling '$pathStr' with imports: $imports")
implicit val aio: AquaIO[IO] = new AquaFilesIO[IO]
val sources = new AquaFileSources[IO](Path(pathStr), imports.toList.map(Path.apply))
val path = Path(pathStr)
val pathId = FileModuleId(path)
val sources = new AquaFileSources[IO](path, imports.toList.map(Path.apply))
val config = AquaCompilerConf()
val proc = for {
// TODO: should run a custom AquaCompiler that collects RawContext + token definitions from the CompilerState to enable cross-file ctrl+click support
res <- AquaCompiler
.compileToContext[IO, AquaFileError, FileModuleId, FileSpan.F](
res <- CompilerAPI
.compileToLsp[IO, AquaFileError, FileModuleId, FileSpan.F](
sources,
SpanParser.parser,
config
)
} yield {
val fileRes: Validated[NonEmptyChain[
AquaError[FileModuleId, AquaFileError, FileSpan.F]
], LspContext[FileSpan.F]] = res
.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"))
)
)
)
logger.debug("Compilation done.")
val result = res match {
case Valid((state, _)) =>
val result = fileRes match {
case Valid(lsp) =>
logger.debug("No errors on compilation.")
CompilationResult(
List.empty.toJSArray,
state.toList
.flatMap(s =>
(s.names.locations ++ s.abilities.locations).flatMap { case (t, tInfo) =>
tInfo.definition match {
case None => Nil
case Some(d) =>
TokenLink(TokenLocation(t.unit._1), TokenLocation(d.unit._1)) :: Nil
lsp.locations.flatMap { case (t, tInfo) =>
tInfo.definition match {
case None => Nil
case Some(d) =>
val fromOp = TokenLocation.fromSpan(t.unit._1)
val toOp = TokenLocation.fromSpan(d.unit._1)
val link = for {
from <- fromOp
to <- toOp
} yield {
TokenLink(from, to)
}
}
)
.toJSArray
if (link.isEmpty)
logger.warn(s"Incorrect coordinates for token '${t.unit._1.name}'")
link.toList
}
}.toJSArray
)
case Invalid(e: NonEmptyChain[AquaError[FileModuleId, AquaFileError, FileSpan.F]]) =>
val errors = e.toNonEmptyList.toList.flatMap(errorToInfo)

View File

@ -1,7 +1,9 @@
export interface TokenLocation {
name: string,
start: number,
end: number
startLine: number,
startCol: number,
endLine: number,
endCol: number
}
export interface TokenLink {

View File

@ -40,38 +40,12 @@ case class RawContext(
def nonEmpty: Boolean = !isEmpty
def pick(
name: String,
rename: Option[String],
declared: Boolean = module.nonEmpty
): Option[RawContext] =
Option
.when(!declared || declares(name)) {
RawContext.blank
.copy(parts = parts.filter(_._2.name == name).map { case (partContext, part) =>
(partContext, rename.fold(part)(part.rename))
})
}
.filter(_.nonEmpty)
private def prefixFirst[T](prefix: String, pair: (String, T)): (String, T) =
(prefix + pair._1, pair._2)
private def collectPartsMap[T](f: PartialFunction[RawPart, T]): Map[String, T] =
parts.collect { case (_, p) if f.isDefinedAt(p) => p.name -> f(p) }.toList.toMap
def pickHeader: RawContext =
RawContext.blank.copy(module = module, declares = declares, exports = exports)
def pickDeclared(implicit semi: Semigroup[RawContext]): RawContext =
if (module.isEmpty) this
else
declares.toList
.flatMap(pick(_, None))
.foldLeft(pickHeader)(
_ |+| _
)
private def prefixFirst[T](prefix: String, pair: (String, T)): (String, T) =
(prefix + pair._1, pair._2)
lazy val services: Map[String, ServiceRaw] = collectPartsMap { case srv: ServiceRaw => srv }
lazy val allServices: Map[String, ServiceRaw] =
@ -117,6 +91,8 @@ case class RawContext(
)
)
.map(StructType(name, _))
override def toString: String = s"module: $module\ndeclares: $declares\nexports: $exports"
}
object RawContext {
@ -145,5 +121,6 @@ object RawContext {
override def combine(x: RawContext, y: RawContext): RawContext =
semiRC.combine(x, y)
}
}
}

View File

@ -43,33 +43,23 @@ case class AquaContext(
lazy val allFuncs: Map[String, FuncArrow] =
all(_.funcs)
// TODO remove this ugliness
def pick(name: String, maybeRename: Option[String]): AquaContext =
funcs
.get(name)
.fold(AquaContext.blank)(p =>
AquaContext.blank.copy(funcs = Map(maybeRename.getOrElse(name) -> p))
) |+|
types
.get(name)
.fold(AquaContext.blank)(p =>
AquaContext.blank.copy(types = Map(maybeRename.getOrElse(name) -> p))
) |+|
values
.get(name)
.fold(AquaContext.blank)(p =>
AquaContext.blank.copy(values = Map(maybeRename.getOrElse(name) -> p))
) |+|
abilities
.get(name)
.fold(AquaContext.blank)(p =>
AquaContext.blank.copy(abilities = Map(maybeRename.getOrElse(name) -> p))
) |+|
services
.get(name)
.fold(AquaContext.blank)(p =>
AquaContext.blank.copy(services = Map(maybeRename.getOrElse(name) -> p))
)
private def pickOne[T](
name: String,
newName: String,
ctx: Map[String, T],
add: (AquaContext, Map[String, T]) => AquaContext
): AquaContext = {
ctx.get(name).fold(AquaContext.blank)(t => add(AquaContext.blank, Map(newName -> t)))
}
def pick(name: String, maybeRename: Option[String]): AquaContext = {
val newName = maybeRename.getOrElse(name)
pickOne(name, newName, funcs, (ctx, el) => ctx.copy(funcs = el)) |+|
pickOne(name, newName, types, (ctx, el) => ctx.copy(types = el)) |+|
pickOne(name, newName, values, (ctx, el) => ctx.copy(values = el)) |+|
pickOne(name, newName, abilities, (ctx, el) => ctx.copy(abilities = el)) |+|
pickOne(name, newName, services, (ctx, el) => ctx.copy(services = el))
}
}
object AquaContext extends Logging {

View File

@ -1,6 +1,6 @@
package aqua.parser.lift
import cats.parse.{LocationMap, Parser0, Parser => P}
import cats.parse.{LocationMap, Parser => P, Parser0}
import cats.{Comonad, Eval}
import scala.language.implicitConversions

View File

@ -27,17 +27,6 @@ object CompilerState {
types = TypesState.init[F](ctx)
)
// TODO remove it after switching from RawContext to LspContext[S]
implicit def semigroupState[S[_]]: Semigroup[CompilerState[S]] =
(x: CompilerState[S], y: CompilerState[S]) => {
CompilerState[S](
x.errors ++ y.errors,
x.names |+| y.names,
x.abilities |+| y.abilities,
x.types |+| y.types
)
}
implicit def compilerStateMonoid[S[_]]: Monoid[St[S]] = new Monoid[St[S]] {
override def empty: St[S] = State.pure(Raw.Empty("compiler state monoid empty"))

View File

@ -4,6 +4,7 @@ import aqua.raw.ops.{FuncOp, SeqGroupTag}
import aqua.raw.{Raw, RawContext, RawPart}
import aqua.parser.lexer.Token
import aqua.parser.{Ast, Expr}
import aqua.semantics.header.Picker
import aqua.semantics.rules.abilities.{AbilitiesAlgebra, AbilitiesInterpreter, AbilitiesState}
import aqua.semantics.rules.names.{NamesAlgebra, NamesInterpreter, NamesState}
import aqua.semantics.rules.types.{TypesAlgebra, TypesInterpreter, TypesState}
@ -21,6 +22,75 @@ import cats.{Eval, Monad, Semigroup}
import monocle.Lens
import monocle.macros.GenLens
import scribe.{log, Logging}
import Picker.*
import aqua.semantics.lsp.LspContext
sealed trait Semantics[S[_], C] {
def process(
ast: Ast[S],
init: C
): ValidatedNec[SemanticError[S], C]
}
class RawSemantics[S[_]](implicit p: Picker[RawContext]) extends Semantics[S, RawContext] {
def process(
ast: Ast[S],
init: RawContext
): ValidatedNec[SemanticError[S], RawContext] =
Semantics
.interpret(ast, CompilerState.init(init), init)
.map { case (state, ctx) =>
NonEmptyChain
.fromChain(state.errors)
.fold[ValidatedNec[SemanticError[S], RawContext]](
Valid(ctx)
)(Invalid(_))
}
// TODO: return as Eval
.value
}
class LspSemantics[S[_]](implicit p: Picker[LspContext[S]]) extends Semantics[S, LspContext[S]] {
def process(
ast: Ast[S],
init: LspContext[S]
): ValidatedNec[SemanticError[S], LspContext[S]] = {
val rawState = CompilerState.init[S](init.raw)
val initState = rawState.copy(
names = rawState.names.copy(
rootArrows = rawState.names.rootArrows ++ init.rootArrows,
constants = rawState.names.constants ++ init.constants
),
abilities = rawState.abilities.copy(
definitions = rawState.abilities.definitions ++ init.abDefinitions
)
)
Semantics
.interpret(ast, initState, init.raw)
.map { case (state, ctx) =>
NonEmptyChain
.fromChain(state.errors)
.fold[ValidatedNec[SemanticError[S], LspContext[S]]] {
Valid(
LspContext(
raw = ctx,
rootArrows = state.names.rootArrows,
constants = state.names.constants,
abDefinitions = state.abilities.definitions,
locations = state.names.locations ++ state.abilities.locations
)
)
}(Invalid(_))
}
// TODO: return as Eval
.value
}
}
object Semantics extends Logging {
@ -78,10 +148,13 @@ object Semantics extends Logging {
transpile[S](ast)
// If there are any errors, they're inside CompilerState[S]
// TODO: pass external token definitions for the RawContext somehow
def interpret[S[_]](ast: Ast[S], init: RawContext): Eval[(CompilerState[S], RawContext)] =
def interpret[S[_]](
ast: Ast[S],
initState: CompilerState[S],
init: RawContext
): Eval[(CompilerState[S], RawContext)] =
astToState[S](ast)
.run(CompilerState.init[S](init))
.run(initState)
.map {
case (state, _: Raw.Empty) =>
// No `parts`, but has `init`
@ -112,19 +185,4 @@ object Semantics extends Logging {
.filter(_ != RawContext.blank)
)
}
// TODO: return just RawContext on the right side
def process[S[_]](
ast: Ast[S],
init: RawContext
): ValidatedNec[SemanticError[S], (CompilerState[S], RawContext)] =
interpret(ast, init).map { case (state, ctx) =>
NonEmptyChain
.fromChain(state.errors)
.fold[ValidatedNec[SemanticError[S], (CompilerState[S], RawContext)]](
Valid(state -> ctx)
)(Invalid(_))
}
// TODO: return as Eval
.value
}

View File

@ -0,0 +1,214 @@
package aqua.semantics.header
import aqua.parser.Ast
import aqua.parser.head.*
import aqua.parser.lexer.{Ability, Token}
import aqua.raw.RawContext
import aqua.semantics.header.Picker.*
import aqua.semantics.{HeaderError, SemanticError}
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.data.*
import cats.free.Cofree
import cats.instances.list.*
import cats.instances.option.*
import cats.kernel.Semigroup
import cats.syntax.foldable.*
import cats.syntax.monoid
import cats.syntax.semigroup.*
import cats.{Comonad, Eval, Monoid}
class HeaderHandler[S[_]: Comonad, C](implicit
acm: Monoid[C],
headMonoid: Monoid[HeaderSem[S, C]],
picker: Picker[C]
) {
type Res[S[_], C] = ValidatedNec[SemanticError[S], HeaderSem[S, C]]
type ResAC[S[_]] = ValidatedNec[SemanticError[S], C]
type ResT[S[_], T] = ValidatedNec[SemanticError[S], T]
// Helper: monoidal combine of all the childrens after parent res
private def combineAnd(children: Chain[Res[S, C]])(
parent: Res[S, C]
): Eval[Res[S, C]] =
Eval.later(parent |+| children.combineAll)
// Error generator with token pointer
private def error[T](token: Token[S], msg: String): ValidatedNec[SemanticError[S], T] =
invalidNec(HeaderError(token, msg))
def sem(imports: Map[String, C], header: Ast.Head[S]): Res[S, C] = {
// Resolve a filename from given imports or fail
def resolve(f: FilenameExpr[S]): ResAC[S] =
imports
.get(f.fileValue)
.map(_.pickDeclared)
.fold[ResAC[S]](
error(f.token, "Cannot resolve the import")
)(validNec)
// Get part of the declared context (for import/use ... from ... expressions)
def getFrom(f: FromExpr[S], ctx: C): ResAC[S] =
f.imports
.map[ResAC[S]](
_.fold[ResAC[S]](
{ case (n, rn) =>
ctx
.pick(n.value, rn.map(_.value), ctx.module.nonEmpty)
.map(validNec)
.getOrElse(
error(
n,
s"Imported file `declares ${ctx.declares.mkString(", ")}`, no ${n.value} declared. Try adding `declares ${n.value}` to that file."
)
)
},
{ case (n, rn) =>
ctx
.pick(n.value, rn.map(_.value), ctx.module.nonEmpty)
.map(validNec)
.getOrElse(
error(
n,
s"Imported file `declares ${ctx.declares.mkString(", ")}`, no ${n.value} declared. Try adding `declares ${n.value}` to that file."
)
)
}
)
)
.foldLeft[ResAC[S]](validNec(ctx.pickHeader))(_ |+| _)
// Convert an imported context into a module (ability)
def toModule(ctx: C, tkn: Token[S], rename: Option[Ability[S]]): ResAC[S] =
rename
.map(_.value)
.orElse(ctx.module)
.fold[ResAC[S]](
error(
tkn,
s"Used module has no `module` header. Please add `module` header or use ... as ModuleName, or switch to import"
)
)(modName => validNec(picker.blank.setAbility(modName, ctx)))
// Handler for every header expression, will be combined later
val onExpr: PartialFunction[HeaderExpr[S], Res[S, C]] = {
// Module header, like `module A declares *`
case ModuleExpr(name, declareAll, declareNames, declareCustom) =>
val shouldDeclare = declareNames.map(_.value).toSet ++ declareCustom.map(_.value)
validNec(
HeaderSem[S, C](
// Save module header info
acm.empty.setModule(
name.value,
shouldDeclare
),
(ctx, _) =>
// When file is handled, check that all the declarations exists
if (declareAll.nonEmpty) {
validNec(
ctx.setModule(name.value, declares = ctx.all)
)
} else
(
declareNames.map(n => n.value -> n) ::: declareCustom.map(a => a.value -> a)
).map[ValidatedNec[SemanticError[S], Int]] { case (n, t) =>
ctx
.pick(n, None, ctx.module.nonEmpty)
// We just validate, nothing more
.map(_ => validNec(1))
.getOrElse(
error(
t,
s"`${n}` is expected to be declared, but declaration is not found in the file"
)
)
}.combineAll
.map(_ =>
// TODO: why module name and declares is lost? where is it lost?
ctx.setModule(name.value, declares = shouldDeclare)
)
)
)
case f @ ImportExpr(_) =>
// Import everything from a file
resolve(f).map(fc => HeaderSem[S, C](fc, (c, _) => validNec(c)))
case f @ ImportFromExpr(_, _) =>
// Import, map declarations
resolve(f)
.andThen(getFrom(f, _))
.map { ctx =>
HeaderSem[S, C](ctx, (c, _) => validNec(c))
}
case f @ UseExpr(_, asModule) =>
// Import, move into a module scope
resolve(f)
.andThen(toModule(_, f.token, asModule))
.map { fc =>
HeaderSem[S, C](fc, (c, _) => validNec(c))
}
case f @ UseFromExpr(_, _, asModule) =>
// Import, cherry-pick declarations, move to a module scope
resolve(f)
.andThen(getFrom(f, _))
.andThen(toModule(_, f.token, Some(asModule)))
.map { fc =>
HeaderSem[S, C](fc, (c, _) => validNec(c))
}
case ExportExpr(pubs) =>
// Save exports, finally handle them
validNec(
HeaderSem[S, C](
// Nothing there
picker.blank,
(ctx, initCtx) =>
pubs
.map(
_.fold[(Token[S], String, Option[String])](
nrn => (nrn._1, nrn._1.value, nrn._2.map(_.value)),
nrn => (nrn._1, nrn._1.value, nrn._2.map(_.value))
)
)
.map { case (token, name, rename) =>
(initCtx |+| ctx)
.pick(name, rename, declared = false)
.map(_ => Map(name -> rename))
.map(validNec)
.getOrElse(
error(
token,
s"File has no $name declaration or import, cannot export, available funcs: ${(initCtx |+| ctx).funcNames
.mkString(", ")}"
)
)
}
.foldLeft[ResT[S, Map[String, Option[String]]]](
validNec(ctx.exports.getOrElse(Map.empty))
)(_ |+| _)
.map(expCtx => ctx.setExports(expCtx))
)
)
case HeadExpr(token) =>
// Old file exports everything that it declares
validNec(
HeaderSem[S, C](
acm.empty,
(ctx, _) => validNec(ctx.setExports(Map.empty))
)
)
case f: FilenameExpr[S] =>
resolve(f).map(fc => HeaderSem[S, C](fc, (c, _) => validNec(c)))
}
Cofree
.cata[Chain, HeaderExpr[S], Res[S, C]](header) { case (expr, children) =>
onExpr.lift.apply(expr).fold(Eval.later(children.combineAll))(combineAnd(children)(_))
}
.value
}
}

View File

@ -1,229 +1,13 @@
package aqua.semantics.header
import cats.data.Validated.{invalidNec, validNec, Invalid, Valid}
import cats.{Comonad, Eval, Monoid}
import cats.data.{Chain, NonEmptyChain, NonEmptyMap, Validated, ValidatedNec}
import aqua.parser.Ast
import aqua.parser.head.*
import aqua.parser.lexer.{Ability, Token}
import aqua.raw.RawContext
import aqua.semantics.{HeaderError, SemanticError}
import cats.syntax.foldable.*
import cats.syntax.semigroup.*
import cats.instances.list.*
import cats.instances.option.*
import cats.free.Cofree
import cats.kernel.Semigroup
import aqua.semantics.SemanticError
import cats.data.*
case class HeaderSem[S[_]](
initCtx: RawContext,
finInitCtx: (RawContext, RawContext) => ValidatedNec[SemanticError[S], RawContext]
case class HeaderSem[S[_], C](
initCtx: C,
finInitCtx: (C, C) => ValidatedNec[SemanticError[S], C]
) {
def finCtx: RawContext => ValidatedNec[SemanticError[S], RawContext] =
def finCtx: C => ValidatedNec[SemanticError[S], C] =
finInitCtx(_, initCtx)
}
object HeaderSem {
type Res[S[_]] = ValidatedNec[SemanticError[S], HeaderSem[S]]
type ResAC[S[_]] = ValidatedNec[SemanticError[S], RawContext]
type ResT[S[_], T] = ValidatedNec[SemanticError[S], T]
private implicit def headerSemMonoid[S[_]](implicit
acm: Monoid[RawContext]
): Monoid[HeaderSem[S]] =
new Monoid[HeaderSem[S]] {
override def empty: HeaderSem[S] = HeaderSem(acm.empty, (c, _) => validNec(c))
override def combine(a: HeaderSem[S], b: HeaderSem[S]): HeaderSem[S] =
HeaderSem(
a.initCtx |+| b.initCtx,
(c, i) => a.finInitCtx(c, i).andThen(b.finInitCtx(_, i))
)
}
// Helper: monoidal combine of all the childrens after parent res
private def combineAnd[S[_]](children: Chain[Res[S]])(parent: Res[S])(implicit
acm: Monoid[RawContext]
): Eval[Res[S]] =
Eval.later(parent |+| children.combineAll)
// Error generator with token pointer
private def error[S[_], T](token: Token[S], msg: String): ValidatedNec[SemanticError[S], T] =
invalidNec(HeaderError(token, msg))
def sem[S[_]: Comonad](imports: Map[String, RawContext], header: Ast.Head[S])(implicit
acm: Monoid[RawContext]
): Res[S] = {
// Resolve a filename from given imports or fail
def resolve(f: FilenameExpr[S]): ResAC[S] =
imports
.get(f.fileValue)
.map(_.pickDeclared)
.fold[ResAC[S]](
error(f.token, "Cannot resolve the import")
)(validNec)
// Get part of the declared context (for import/use ... from ... expressions)
def getFrom(f: FromExpr[S], ctx: RawContext): ResAC[S] =
f.imports
.map[ResAC[S]](
_.fold[ResAC[S]](
{ case (n, rn) =>
ctx
.pick(n.value, rn.map(_.value))
.map(validNec)
.getOrElse(
error(
n,
s"Imported file `declares ${ctx.declares.mkString(", ")}`, no ${n.value} declared. Try adding `declares ${n.value}` to that file."
)
)
},
{ case (n, rn) =>
ctx
.pick(n.value, rn.map(_.value))
.map(validNec)
.getOrElse(
error(
n,
s"Imported file `declares ${ctx.declares.mkString(", ")}`, no ${n.value} declared. Try adding `declares ${n.value}` to that file."
)
)
}
)
)
.foldLeft[ResAC[S]](validNec(ctx.pickHeader))(_ |+| _)
// Convert an imported context into a module (ability)
def toModule(ctx: RawContext, tkn: Token[S], rename: Option[Ability[S]]): ResAC[S] =
rename
.map(_.value)
.orElse(ctx.module)
.fold[ResAC[S]](
error(
tkn,
s"Used module has no `module` header. Please add `module` header or use ... as ModuleName, or switch to import"
)
)(modName => validNec(RawContext.blank.copy(abilities = Map(modName -> ctx))))
// Handler for every header expression, will be combined later
val onExpr: PartialFunction[HeaderExpr[S], Res[S]] = {
// Module header, like `module A declares *`
case ModuleExpr(name, declareAll, declareNames, declareCustom) =>
val shouldDeclare = declareNames.map(_.value).toSet ++ declareCustom.map(_.value)
validNec(
HeaderSem[S](
// Save module header info
acm.empty.copy(
module = Some(name.value),
declares = shouldDeclare
),
(ctx, _) =>
// When file is handled, check that all the declarations exists
if (declareAll.nonEmpty) {
val all =
ctx.`type`("").map(_.fields.toNel.map(_._1).toList.toSet).getOrElse(Set.empty)
validNec(
ctx.copy(module = Some(name.value), declares = all)
)
} else
(
declareNames.map(n => n.value -> n) ::: declareCustom.map(a => a.value -> a)
).map[ValidatedNec[SemanticError[S], Int]] { case (n, t) =>
ctx
.pick(n, None)
// We just validate, nothing more
.map(_ => validNec(1))
.getOrElse(
error(
t,
s"`${n}` is expected to be declared, but declaration is not found in the file"
)
)
}.combineAll
.map(_ =>
// TODO: why module name and declares is lost? where is it lost?
ctx.copy(module = Some(name.value), declares = shouldDeclare)
)
)
)
case f @ ImportExpr(_) =>
// Import everything from a file
resolve(f).map(fc => HeaderSem[S](fc, (c, _) => validNec(c)))
case f @ ImportFromExpr(_, _) =>
// Import, map declarations
resolve(f)
.andThen(getFrom(f, _))
.map { ctx =>
HeaderSem[S](ctx, (c, _) => validNec(c))
}
case f @ UseExpr(_, asModule) =>
// Import, move into a module scope
resolve(f)
.andThen(toModule(_, f.token, asModule))
.map { fc =>
HeaderSem[S](fc, (c, _) => validNec(c))
}
case f @ UseFromExpr(_, _, asModule) =>
// Import, cherry-pick declarations, move to a module scope
resolve(f)
.andThen(getFrom(f, _))
.andThen(toModule(_, f.token, Some(asModule)))
.map { fc =>
HeaderSem[S](fc, (c, _) => validNec(c))
}
case ExportExpr(pubs) =>
// Save exports, finally handle them
validNec(
HeaderSem[S](
// Nothing there
RawContext.blank,
(ctx, initCtx) =>
pubs
.map(
_.fold[(Token[S], String, Option[String])](
nrn => (nrn._1, nrn._1.value, nrn._2.map(_.value)),
nrn => (nrn._1, nrn._1.value, nrn._2.map(_.value))
)
)
.map { case (token, name, rename) =>
(initCtx |+| ctx)
.pick(name, rename, declared = false)
.map(_ => Map(name -> rename))
.map(validNec)
.getOrElse(
error(
token,
s"File has no $name declaration or import, cannot export, available funcs: ${(initCtx |+| ctx).funcs.keys
.mkString(", ")}"
)
)
}
.foldLeft[ResT[S, Map[String, Option[String]]]](
validNec(ctx.exports.getOrElse(Map.empty))
)(_ |+| _)
.map(expCtx => ctx.copy(exports = Some(expCtx)))
)
)
case HeadExpr(token) =>
// Old file exports everything that it declares
validNec(HeaderSem[S](acm.empty, (ctx, _) => validNec(ctx.copy(exports = Some(Map.empty)))))
case f: FilenameExpr[S] =>
resolve(f).map(fc => HeaderSem[S](fc, (c, _) => validNec(c)))
}
Cofree
.cata[Chain, HeaderExpr[S], Res[S]](header) { case (expr, children) =>
onExpr.lift.apply(expr).fold(Eval.later(children.combineAll))(combineAnd(children)(_))
}
.value
}
}

View File

@ -0,0 +1,123 @@
package aqua.semantics.header
import aqua.raw.{RawContext, RawPart}
import aqua.semantics.CompilerState
import aqua.semantics.lsp.{LspContext, TokenArrowInfo, TokenTypeInfo}
import aqua.semantics.rules.abilities.AbilitiesState
import aqua.semantics.rules.names.NamesState
import aqua.semantics.rules.types.TypesState
import cats.{Comonad, Semigroup}
import cats.syntax.semigroup.*
// Able to pick info from different contexts
trait Picker[A] {
def all(ctx: A): Set[String]
def funcNames(ctx: A): List[String]
def blank: A
def pick(ctx: A, name: String, rename: Option[String], declared: Boolean): Option[A]
def pickDeclared(ctx: A)(implicit semi: Semigroup[A]): A
def pickHeader(ctx: A): A
def module(ctx: A): Option[String]
def exports(ctx: A): Option[Map[String, Option[String]]]
def declares(ctx: A): Set[String]
def setAbility(ctx: A, name: String, ctxAb: A): A
def setModule(ctx: A, name: Option[String], declares: Set[String]): A
def setExports(ctx: A, exports: Map[String, Option[String]]): A
def setInit(ctx: A, ctxInit: Option[A]): A
def addPart(ctx: A, part: (A, RawPart)): A
}
final class PickerOps[A: Picker](p: A) {
def blank: A = Picker[A].blank
def all: Set[String] = Picker[A].all(p)
def funcNames: List[String] = Picker[A].funcNames(p)
def pick(name: String, rename: Option[String], declared: Boolean): Option[A] =
Picker[A].pick(p, name, rename, declared)
def pickDeclared(implicit semi: Semigroup[A]): A = Picker[A].pickDeclared(p)
def pickHeader: A = Picker[A].pickHeader(p)
def module: Option[String] = Picker[A].module(p)
def exports: Option[Map[String, Option[String]]] = Picker[A].exports(p)
def declares: Set[String] = Picker[A].declares(p)
def setAbility(name: String, ctx: A): A = Picker[A].setAbility(p, name, ctx)
def setInit(ctx: Option[A]): A = Picker[A].setInit(p, ctx)
def addPart(part: (A, RawPart)): A = Picker[A].addPart(p, part)
def setModule(name: String, declares: Set[String]): A =
Picker[A].setModule(p, Some(name), declares)
def setOptModule(name: Option[String], declares: Set[String]): A =
Picker[A].setModule(p, name, declares)
def setExports(exports: Map[String, Option[String]]): A =
Picker[A].setExports(p, exports)
}
object Picker {
implicit final def apply[A](implicit ev: Picker[A]): Picker[A] = ev
implicit final def syntaxPicker[A: Picker](a: A): PickerOps[A] =
new PickerOps[A](a)
given Picker[RawContext] with {
override def blank: RawContext = RawContext.blank
override def exports(ctx: RawContext): Option[Map[String, Option[String]]] = ctx.exports
override def funcNames(ctx: RawContext): List[String] = ctx.funcs.keys.toList
override def addPart(ctx: RawContext, part: (RawContext, RawPart)): RawContext =
ctx.copy(parts = ctx.parts :+ part)
override def setInit(ctx: RawContext, ctxInit: Option[RawContext]): RawContext =
ctx.copy(init = ctxInit)
override def all(ctx: RawContext): Set[String] =
ctx.`type`("").map(_.fields.toNel.map(_._1).toList.toSet).getOrElse(Set.empty)
override def module(ctx: RawContext): Option[String] = ctx.module
override def declares(ctx: RawContext): Set[String] = ctx.declares
override def setAbility(ctx: RawContext, name: String, ctxAb: RawContext): RawContext =
ctx.copy(abilities = Map(name -> ctxAb))
override def setModule(
ctx: RawContext,
name: Option[String],
declares: Set[String]
): RawContext =
ctx.copy(module = name, declares = declares)
override def setExports(ctx: RawContext, exports: Map[String, Option[String]]): RawContext =
ctx.copy(exports = Some(exports))
override def pick(
ctx: RawContext,
name: String,
rename: Option[String],
declared: Boolean
): Option[RawContext] =
Option
.when(!declared || ctx.declares(name)) {
RawContext.blank
.copy(parts = ctx.parts.filter(_._2.name == name).map { case (partContext, part) =>
(partContext, rename.fold(part)(part.rename))
})
}
.filter(_.nonEmpty)
override def pickHeader(ctx: RawContext): RawContext =
RawContext.blank.copy(module = ctx.module, declares = ctx.declares, exports = ctx.exports)
override def pickDeclared(ctx: RawContext)(implicit semi: Semigroup[RawContext]): RawContext =
if (ctx.module.isEmpty) ctx
else
ctx.declares.toList
.flatMap(n => pick(ctx, n, None, ctx.module.nonEmpty))
.foldLeft(pickHeader(ctx))(
_ |+| _
)
}
}

View File

@ -0,0 +1,114 @@
package aqua.semantics.lsp
import aqua.parser.lexer.{Ability, Name, Token}
import aqua.raw.{RawContext, RawPart}
import aqua.types.ArrowType
import cats.{Monoid, Semigroup}
import cats.syntax.monoid.*
import RawContext.semiRC
import aqua.semantics.header.{Picker, PickerOps}
// Context with info that necessary for language server
case class LspContext[S[_]](
raw: RawContext,
abDefinitions: Map[String, (Ability[S], List[(Name[S], ArrowType)])] =
Map.empty[String, (Ability[S], List[(Name[S], ArrowType)])],
rootArrows: Map[String, TokenArrowInfo[S]] = Map.empty[String, TokenArrowInfo[S]],
constants: Map[String, TokenType[S]] = Map.empty[String, TokenType[S]],
locations: List[(Token[S], TokenInfo[S])] = Nil
)
object LspContext {
def blank[S[_]]: LspContext[S] = LspContext[S](raw = RawContext())
implicit def semiLsp[S[_]]: Semigroup[LspContext[S]] =
(x: LspContext[S], y: LspContext[S]) =>
LspContext[S](
raw = x.raw |+| y.raw,
abDefinitions = x.abDefinitions ++ y.abDefinitions,
rootArrows = x.rootArrows ++ y.rootArrows,
constants = x.constants ++ y.constants,
locations = x.locations ++ y.locations
)
trait Implicits[S[_]] {
implicit val lspContextMonoid: Monoid[LspContext[S]]
}
def implicits[S[_]](init: LspContext[S]): Implicits[S] = new Implicits[S] {
override implicit val lspContextMonoid: Monoid[LspContext[S]] = new Monoid[LspContext[S]] {
override def empty: LspContext[S] = init
override def combine(x: LspContext[S], y: LspContext[S]): LspContext[S] = {
semiLsp[S].combine(x, y)
}
}
}
given [S[_]]: Picker[LspContext[S]] with {
private def ops[S[_]](ctx: LspContext[S]) = PickerOps[RawContext](ctx.raw)
override def blank: LspContext[S] = LspContext[S](Picker[RawContext].blank, Map.empty)
override def exports(ctx: LspContext[S]): Option[Map[String, Option[String]]] = ops(ctx).exports
override def funcNames(ctx: LspContext[S]): List[String] = ops(ctx).funcNames
override def addPart(ctx: LspContext[S], part: (LspContext[S], RawPart)): LspContext[S] =
ctx.copy(raw = ops(ctx).addPart(part._1.raw -> part._2))
override def setInit(ctx: LspContext[S], ctxInit: Option[LspContext[S]]): LspContext[S] =
ctx.copy(raw = ops(ctx).setInit(ctxInit.map(_.raw)))
override def all(ctx: LspContext[S]): Set[String] =
ops(ctx).all
override def module(ctx: LspContext[S]): Option[String] = ops(ctx).module
override def declares(ctx: LspContext[S]): Set[String] = ops(ctx).declares
override def setAbility(ctx: LspContext[S], name: String, ctxAb: LspContext[S]): LspContext[S] =
ctx.copy(raw = ops(ctx).setAbility(name, ctxAb.raw))
override def setModule(
ctx: LspContext[S],
name: Option[String],
declares: Set[String]
): LspContext[S] =
ctx.copy(raw = ops(ctx).setOptModule(name, declares))
override def setExports(
ctx: LspContext[S],
exports: Map[String, Option[String]]
): LspContext[S] =
ctx.copy(raw = ops(ctx).setExports(exports))
override def pick(
ctx: LspContext[S],
name: String,
rename: Option[String],
declared: Boolean
): Option[LspContext[S]] =
ops(ctx)
.pick(name, rename, declared)
.map(rc =>
ctx.copy(
raw = rc,
abDefinitions =
ctx.abDefinitions.get(name).fold(Map.empty)(t => Map(rename.getOrElse(name) -> t)),
rootArrows =
ctx.rootArrows.get(name).fold(Map.empty)(t => Map(rename.getOrElse(name) -> t)),
constants =
ctx.constants.get(name).fold(Map.empty)(t => Map(rename.getOrElse(name) -> t))
)
)
override def pickHeader(ctx: LspContext[S]): LspContext[S] =
ctx.copy(raw = ops(ctx).pickHeader)
override def pickDeclared(
ctx: LspContext[S]
)(implicit semi: Semigroup[LspContext[S]]): LspContext[S] =
ctx.copy(raw = ops(ctx).pickDeclared)
}
}

View File

@ -125,7 +125,10 @@ class AbilitiesInterpreter[S[_], X](implicit
)
).as(Option.empty[ArrowType])
) { fn =>
addServiceArrowLocation(name, arrow).as(Some(fn.arrow.`type`))
// TODO: add name and arrow separately
// TODO: find tokens somewhere
// addServiceArrowLocation(name, arrow).as(Some(fn.arrow.`type`))
State.pure(Some(fn.arrow.`type`))
}
case None =>
report(name, "Ability with this name is undefined").as(Option.empty[ArrowType])

View File

@ -35,9 +35,11 @@ class SemanticsSpec extends AnyFlatSpec with Matchers {
val ctx = RawContext.blank
val p = Semantics.process(ast, ctx)
val semantics = new RawSemantics[Span.S]()
val func = p.toList.head._2.funcs("parFunc")
val p = semantics.process(ast, ctx)
val func = p.toList.head.funcs("parFunc")
val proc = func.arrow.body