// Basic streaming parsing framework
//
// Copyright (C) 2014-2022 Ryan Specialty Group, LLC.
//
// This file is part of TAME.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.
//
// You should have received a copy of the GNU General Public License
// along with this program. If not, see .
//! Basic streaming parser framework for lowering operations.
//!
//! _TODO: Some proper docs and examples!_
use crate::diagnose::{Annotate, AnnotatedSpan, Diagnostic};
use crate::iter::{TripIter, TrippableIterator};
use crate::span::{Span, UNKNOWN_SPAN};
use std::fmt::Debug;
use std::hint::unreachable_unchecked;
use std::iter::{self, Empty};
use std::mem::take;
use std::ops::{ControlFlow, Deref, DerefMut, FromResidual, Try};
use std::{convert::Infallible, error::Error, fmt::Display};
/// Result of applying a [`Token`] to a [`ParseState`],
/// with any error having been wrapped in a [`ParseError`].
pub type ParsedResult = ParseResult::Object>>;
/// Result of some non-parsing operation on a [`Parser`],
/// with any error having been wrapped in a [`ParseError`].
pub type ParseResult =
Result::Token, ::Error>>;
/// A single datum from a streaming IR with an associated [`Span`].
///
/// A token may be a lexeme with associated data,
/// or a more structured object having been lowered from other IRs.
pub trait Token: Display + Debug + PartialEq {
/// Retrieve the [`Span`] representing the source location of the token.
fn span(&self) -> Span;
}
impl From for Span {
fn from(tok: T) -> Self {
tok.span()
}
}
/// An IR object produced by a lowering operation on one or more [`Token`]s.
///
/// Note that an [`Object`] may also be a [`Token`] if it will be in turn
/// fed to another [`Parser`] for lowering.
///
/// This trait exists to disambiguate an otherwise unbounded type for
/// [`From`] conversions,
/// used in the [`Transition`] API to provide greater flexibility.
pub trait Object: Debug + PartialEq {}
impl Object for () {}
/// An infallible [`Token`] stream.
///
/// If the token stream originates from an operation that could potentially
/// fail and ought to be propagated,
/// use [`TokenResultStream`].
///
/// The name "stream" in place of "iterator" is intended to convey that this
/// type is expected to be processed in real-time as a stream,
/// not read into memory.
pub trait TokenStream = Iterator- ;
/// A [`Token`] stream that may encounter errors during parsing.
///
/// If the stream cannot fail,
/// consider using [`TokenStream`].
pub trait TokenResultStream = Iterator
- >;
/// A [`ParseState`] capable of being automatically stitched together with
/// a parent [`ParseState`] `SP` to create a composite parser.
///
/// Conceptually,
/// this can be visualized as combining the state machines of multiple
/// parsers into one larger state machine.
///
/// The term _state stitching_ refers to a particular pattern able to be
/// performed automatically by this parsing framework;
/// it is not necessary for parser composition,
/// provided that you perform the necessary wiring yourself in absence
/// of state stitching.
pub trait StitchableParseState = ParseState
where
SP: ParseState::Token>,
::Object: Into<::Object>,
::Error: Into<::Error>;
/// A parsing automaton.
///
/// These states are utilized by a [`Parser`].
///
/// A [`ParseState`] is also responsible for storing data about the
/// accepted input,
/// and handling appropriate type conversions into the final type.
/// That is---an
/// automaton may store metadata that is subsequently emitted once an
/// accepting state has been reached.
/// Whatever the underlying automaton,
/// a `(state, token, context)` triple must uniquely determine the next
/// parser action.
pub trait ParseState: Default + PartialEq + Eq + Display + Debug {
/// Input tokens to the parser.
type Token: Token;
/// Objects produced by a parser utilizing these states.
type Object: Object;
/// Errors specific to this set of states.
type Error: Debug + Diagnostic + PartialEq;
/// Object provided to parser alongside each token.
///
/// This may be used in situations where Rust/LLVM are unable to
/// optimize away moves of interior data associated with the
/// otherwise-immutable [`ParseState`].
type Context: Debug = EmptyContext;
/// Construct a parser.
///
/// Whether this method is helpful or provides any clarity depends on
/// the context and the types that are able to be inferred.
fn parse>(toks: I) -> Parser
where
Self::Context: Default,
{
Parser::from(toks)
}
/// Construct a parser with a non-default [`ParseState::Context`].
///
/// This is useful in two ways:
///
/// 1. To allow for parsing using a context that does not implement
/// [`Default`],
/// or whose default is not sufficient; and
/// 2. To re-use a context from a previous [`Parser`].
///
/// If neither of these apply to your situation,
/// consider [`ParseState::parse`] instead.
///
/// To retrieve a context from a parser for re-use,
/// see [`Parser::finalize`].
fn parse_with_context>(
toks: I,
ctx: Self::Context,
) -> Parser {
Parser::from((toks, ctx))
}
/// Parse a single [`Token`] and optionally perform a state transition.
///
/// The current state is represented by `self`.
/// The result of a parsing operation is a state transition with
/// associated [`ParseStatus`] data.
///
/// Note that `self` is owned,
/// for a couple primary reasons:
///
/// 1. This forces the parser to explicitly consider and document all
/// state transitions,
/// rather than potentially missing unintended behavior through
/// implicit behavior; and
/// 2. It allows for more natural functional composition of state,
/// which in turn makes it easier to compose parsers
/// (which conceptually involves stitching together state
/// machines).
///
/// Since a [`ParseState`] produces a new version of itself with each
/// invocation,
/// it is functionally pure.
/// Generally,
/// Rust/LLVM are able to optimize moves into direct assignments.
/// However,
/// there are circumstances where this is _not_ the case,
/// in which case [`Context`] can be used to provide a mutable context
/// owned by the caller (e.g. [`Parser`]) to store additional
/// information that is not subject to Rust's move semantics.
/// If this is not necessary,
/// see [`NoContext`].
fn parse_token(
self,
tok: Self::Token,
ctx: &mut Self::Context,
) -> TransitionResult;
/// Whether the current state represents an accepting state.
///
/// An accepting state represents a valid state to stop parsing.
/// If parsing stops at a state that is _not_ accepting,
/// then the [`TokenStream`] has ended unexpectedly and should produce
/// a [`ParseError::UnexpectedEof`].
///
/// It makes sense for there to be exist multiple accepting states for a
/// parser.
/// For example:
/// A parser that parses a list of attributes may be used to parse one
/// or more attributes,
/// or the entire list of attributes.
/// It is acceptable to attempt to parse just one of those attributes,
/// or it is acceptable to parse all the way until the end.
fn is_accepting(&self) -> bool;
/// Delegate parsing from a compatible, stitched [`ParseState`]~`SP`.
///
/// This helps to combine two state machines that speak the same input
/// language
/// (share the same [`Self::Token`]),
/// handling the boilerplate of delegating [`Self::Token`] from a
/// parent state~`SP` to `Self`.
///
/// Token delegation happens after [`Self`] has been entered from a
/// parent [`ParseState`] context~`SP`,
/// so stitching the start and accepting states must happen elsewhere
/// (for now).
///
/// This assumes that no lookahead token from [`ParseStatus::Dead`] will
/// need to be handled by the parent state~`SP`.
/// To handle a token of lookahead,
/// use [`Self::delegate_lookahead`] instead.
///
/// _TODO: More documentation once this is finalized._
fn delegate(
self,
mut context: C,
tok: ::Token,
into: impl FnOnce(Self) -> SP,
) -> TransitionResult
where
Self: StitchableParseState,
C: AsMut<::Context>,
{
use ParseStatus::{Dead, Incomplete, Object as Obj};
let (Transition(newst), result) =
self.parse_token(tok, context.as_mut()).into();
// This does not use `delegate_lookahead` so that we can have
// `into: impl FnOnce` instead of `Fn`.
Transition(into(newst)).result(match result {
Ok(Incomplete) => Ok(Incomplete),
Ok(Obj(obj)) => Ok(Obj(obj.into())),
Ok(Dead(tok)) => Ok(Dead(tok)),
Err(e) => Err(e.into()),
})
}
/// Delegate parsing from a compatible, stitched [`ParseState`]~`SP` with
/// support for a lookahead token.
///
/// This does the same thing as [`Self::delegate`],
/// but allows for the handling of a lookahead token from [`Self`]
/// rather than simply proxying [`ParseStatus::Dead`].
///
/// _TODO: More documentation once this is finalized._
fn delegate_lookahead(
self,
mut context: C,
tok: ::Token,
into: impl FnOnce(Self) -> SP,
) -> ControlFlow, (Self, ::Token, C)>
where
Self: StitchableParseState,
C: AsMut<::Context>,
{
use ControlFlow::*;
use ParseStatus::{Dead, Incomplete, Object as Obj};
// NB: Rust/LLVM are generally able to elide these moves into direct
// assignments,
// but sometimes this does not work
// (e.g. XIRF's use of `ArrayVec`).
// If your [`ParseState`] has a lot of `memcpy`s or other
// performance issues,
// move heavy objects into `context`.
let (Transition(newst), result) =
self.parse_token(tok, context.as_mut()).into();
match result {
Ok(Incomplete) => Break(Transition(into(newst)).incomplete()),
Ok(Obj(obj)) => Break(Transition(into(newst)).ok(obj.into())),
Ok(Dead(tok)) => Continue((newst, tok, context)),
Err(e) => Break(Transition(into(newst)).err(e)),
}
}
}
/// Empty [`Context`] for [`ParseState`]s with pure functional
/// implementations with no mutable state.
///
/// Using this value means that a [`ParseState`] does not require a
/// context.
/// All [`Context`]s implement [`AsMut`](AsMut),
/// and so all pure [`ParseState`]s have contexts compatible with every
/// other parser for composition
/// (provided that the other invariants in [`StitchableParseState`] are
/// met).
///
/// This can be clearly represented in function signatures using
/// [`EmptyContext`].
#[derive(Debug, PartialEq, Eq, Default)]
pub struct EmptyContext;
impl AsMut for EmptyContext {
fn as_mut(&mut self) -> &mut EmptyContext {
self
}
}
/// A [`ParseState`] does not require any mutable [`Context`].
///
/// A [`ParseState`] using this context is pure
/// (has no mutable state),
/// returning a new version of itself on each state change.
///
/// This type is intended to be self-documenting:
/// `_: EmptyContext` is nicer to readers than `_: &mut EmptyContext`.
///
/// See [`EmptyContext`] for more information.
pub type NoContext<'a> = &'a mut EmptyContext;
/// Mutable context for [`ParseState`].
///
/// [`ParseState`]s are immutable and pure---they
/// are invoked via [`ParseState::parse_token`] and return a new version
/// of themselves representing their new state.
/// Rust/LLVM are generally able to elide intermediate values and moves,
/// optimizing these parsers away into assignments.
///
/// However,
/// there are circumstances where moves may not be elided and may retain
/// their `memcpy` equivalents.
/// To work around this,
/// [`ParseState::parse_token`] accepts a mutable [`Context`] reference
/// which is held by the parent [`Parser`],
/// which can be mutated in-place without worrying about Rust's move
/// semantics.
///
/// Plainly: you should only use this if you have to.
/// This was added because certain parsers may be invoked millions of times
/// for each individual token in systems with many source packages,
/// which may otherwise result in millions of `memcpy`s.
///
/// When composing two [`ParseState`]s `A`,
/// a [`Context`](Context) must be contravariant over `B` and~`C`.
/// Concretely,
/// this means that [`AsMut`](AsMut) and
/// [`AsMut`](AsMut) must be implemented for `A::Context`.
/// This almost certainly means that `A::Context` is a product type.
/// Consequently,
/// a single [`Parser`] is able to hold a composite [`Context`] in a
/// single memory location.
///
/// [`Context`](Context) implements [`Deref`](Deref) for convenience.
///
/// If your [`ParseState`] does not require a mutable [`Context`],
/// see [`NoContext`].
#[derive(Debug, Default)]
pub struct Context(T, EmptyContext);
impl AsMut for Context {
fn as_mut(&mut self) -> &mut EmptyContext {
&mut self.1
}
}
impl Deref for Context {
type Target = T;
fn deref(&self) -> &Self::Target {
&self.0
}
}
impl DerefMut for Context {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.0
}
}
impl From for Context {
fn from(x: T) -> Self {
Context(x, EmptyContext)
}
}
/// Result of applying a [`Token`] to a [`ParseState`].
///
/// This is used by [`ParseState::parse_token`];
/// see that function for rationale.
pub type ParseStateResult
= Result, ::Error>;
/// A state transition with associated data.
///
/// Conceptually,
/// imagine the act of a state transition producing data.
/// See [`Transition`] for convenience methods for producing this tuple.
#[derive(Debug, PartialEq)]
pub struct TransitionResult(
pub Transition,
pub ParseStateResult,
);
/// Denotes a state transition.
///
/// This newtype was created to produce clear, self-documenting code;
/// parsers can get confusing to read with all of the types involved,
/// so this provides a mental synchronization point.
///
/// This also provides some convenience methods to help remote boilerplate
/// and further improve code clarity.
#[derive(Debug, PartialEq, Eq)]
pub struct Transition(pub S);
impl Transition {
/// A state transition with corresponding data.
///
/// This allows [`ParseState::parse_token`] to emit a parsed object and
/// corresponds to [`ParseStatus::Object`].
pub fn ok(self, obj: T) -> TransitionResult
where
T: Into>,
{
TransitionResult(self, Ok(obj.into()))
}
/// A transition with corresponding error.
///
/// This indicates a parsing failure.
/// The state ought to be suitable for error recovery.
pub fn err>(self, err: E) -> TransitionResult {
TransitionResult(self, Err(err.into()))
}
/// A state transition with corresponding [`Result`].
///
/// This translates the provided [`Result`] in a manner equivalent to
/// [`Transition::ok`] and [`Transition::err`].
pub fn result(self, result: Result) -> TransitionResult
where
T: Into>,
E: Into,
{
TransitionResult(self, result.map(Into::into).map_err(Into::into))
}
/// A state transition indicating that more data is needed before an
/// object can be emitted.
///
/// This corresponds to [`ParseStatus::Incomplete`].
pub fn incomplete(self) -> TransitionResult {
TransitionResult(self, Ok(ParseStatus::Incomplete))
}
/// A dead state transition.
///
/// This corresponds to [`ParseStatus::Dead`],
/// and a calling parser should use the provided [`Token`] as
/// lookahead.
pub fn dead(self, tok: S::Token) -> TransitionResult {
TransitionResult(self, Ok(ParseStatus::Dead(tok)))
}
}
impl Into<(Transition, ParseStateResult)>
for TransitionResult
{
fn into(self) -> (Transition, ParseStateResult) {
(self.0, self.1)
}
}
impl Try for TransitionResult {
type Output = (Transition, ParseStateResult);
type Residual = (Transition, ParseStateResult);
fn from_output(output: Self::Output) -> Self {
match output {
(st, result) => Self(st, result),
}
}
fn branch(self) -> ControlFlow {
match self.into() {
(st, Ok(x)) => ControlFlow::Continue((st, Ok(x))),
(st, Err(e)) => ControlFlow::Break((st, Err(e))),
}
}
}
impl FromResidual<(Transition, ParseStateResult)>
for TransitionResult
{
fn from_residual(residual: (Transition, ParseStateResult)) -> Self {
match residual {
(st, result) => Self(st, result),
}
}
}
impl FromResidual>>
for TransitionResult
{
fn from_residual(
residual: Result>,
) -> Self {
match residual {
Err(e) => e,
// SAFETY: This match arm doesn't seem to be required in
// core::result::Result's FromResidual implementation,
// but as of 1.61 nightly it is here.
// Since this is Infallable,
// it cannot occur.
Ok(_) => unsafe { unreachable_unchecked() },
}
}
}
impl FromResidual, Infallible>>
for TransitionResult
{
fn from_residual(
residual: ControlFlow, Infallible>,
) -> Self {
match residual {
ControlFlow::Break(result) => result,
// SAFETY: Infallible, so cannot hit.
ControlFlow::Continue(_) => unsafe { unreachable_unchecked() },
}
}
}
/// An object able to be used as data for a state [`Transition`].
///
/// This flips the usual order of things:
/// rather than using a method of [`Transition`] to provide data,
/// this starts with the data and produces a transition from it.
/// This is sometimes necessary to satisfy ownership/borrowing rules.
///
/// This trait simply removes boilerplate associated with storing
/// intermediate values and translating into the resulting type.
pub trait Transitionable {
/// Perform a state transition to `S` using [`Self`] as the associated
/// data.
///
/// This may be necessary to satisfy ownership/borrowing rules when
/// state data from `S` is used to compute [`Self`].
fn transition(self, to: S) -> TransitionResult;
}
impl Transitionable for Result, E>
where
S: ParseState,
::Error: From,
{
fn transition(self, to: S) -> TransitionResult {
Transition(to).result(self)
}
}
impl Transitionable for Result<(), E>
where
S: ParseState,
::Error: From,
{
fn transition(self, to: S) -> TransitionResult {
Transition(to).result(self.map(|_| ParseStatus::Incomplete))
}
}
/// A streaming parser defined by a [`ParseState`] with exclusive
/// mutable access to an underlying [`TokenStream`].
///
/// This parser handles operations that are common among all types of
/// parsers,
/// such that specialized parsers need only implement logic that is
/// unique to their operation.
/// This also simplifies combinators,
/// since there is more uniformity among distinct parser types.
///
/// After you have finished with a parser,
/// if you have not consumed the entire iterator,
/// call [`finalize`](Parser::finalize) to ensure that parsing has
/// completed in an accepting state.
#[derive(Debug, PartialEq, Eq)]
pub struct Parser> {
toks: I,
state: S,
last_span: Span,
ctx: S::Context,
}
impl> Parser {
/// Indicate that no further parsing will take place using this parser,
/// retrieve any final aggregate state (the context),
/// and [`drop`] it.
///
/// Invoking the method is equivalent to stating that the stream has
/// ended,
/// since the parser will have no later opportunity to continue
/// parsing.
/// Consequently,
/// the caller should expect [`ParseError::UnexpectedEof`] if the
/// parser is not in an accepting state.
///
/// To re-use the context returned by this method,
/// see [`ParseState::parse_with_context`].
/// Note that whether the context is permitted to be reused,
/// or is useful independently to the caller,
/// is a decision made by the [`ParseState`].
pub fn finalize(
self,
) -> Result)> {
match self.assert_accepting() {
Ok(()) => Ok(self.ctx),
Err(err) => Err((self, err)),
}
}
/// Return [`Ok`] if the parser is in an accepting state,
/// otherwise [`Err`] with [`ParseError::UnexpectedEof`].
///
/// See [`finalize`](Self::finalize) for the public-facing method.
fn assert_accepting(&self) -> Result<(), ParseError> {
if self.state.is_accepting() {
Ok(())
} else {
let endpoints = self.last_span.endpoints();
Err(ParseError::UnexpectedEof(
endpoints.1.unwrap_or(endpoints.0),
self.state.to_string(),
))
}
}
/// Feed an input token to the parser.
///
/// This _pushes_ data into the parser,
/// rather than the typical pull system used by [`Parser`]'s
/// [`Iterator`] implementation.
/// The pull system also uses this method to provided data to the
/// parser.
///
/// This method is intentionally private,
/// since push parsers are currently supported only internally.
/// The only thing preventing this being public is formalization and a
/// commitment to maintain it.
fn feed_tok(&mut self, tok: S::Token) -> ParsedResult {
// Store the most recently encountered Span for error
// reporting in case we encounter an EOF.
self.last_span = tok.span();
let result;
TransitionResult(Transition(self.state), result) =
take(&mut self.state).parse_token(tok, &mut self.ctx);
use ParseStatus::*;
match result {
// Nothing handled this dead state,
// and we cannot discard a lookahead token,
// so we have no choice but to produce an error.
Ok(Dead(invalid)) => Err(ParseError::UnexpectedToken(
invalid,
self.state.to_string(),
)),
Ok(parsed @ (Incomplete | Object(..))) => Ok(parsed.into()),
Err(e) => Err(e.into()),
}
}
/// Lower the IR produced by this [`Parser`] into another IR by piping
/// the output to a new parser defined by the [`ParseState`] `LS`.
///
/// This parser consumes tokens `S::Token` and produces the IR
/// `S::Output`.
/// If there is some other [`ParseState`] `LS` such that
/// `LS::Token == S::Output`
/// (that is—the output of this parser is the input to another),
/// then this method will wire the two together into a new iterator
/// that produces `LS::Output`.
///
/// Visually, we have,
/// within the provided closure `f`,
/// a [`LowerIter`] that acts as this pipeline:
///
/// ```text
/// (S::Token) -> (S::Output == LS::Token) -> (LS::Output)
/// ```
///
/// The new iterator is a [`LowerIter`],
/// and scoped to the provided closure `f`.
/// The outer [`Result`] of `Self`'s [`ParsedResult`] is stripped by
/// a [`TripIter`] before being provided as input to a new push
/// [`Parser`] utilizing `LS`.
/// A push parser,
/// rather than pulling tokens from a [`TokenStream`],
/// has tokens pushed into it;
/// this parser is created automatically for you.
///
/// _TODO_: There's no way to access the inner parser for error recovery
/// after tripping the [`TripIter`].
/// Consequently,
/// this API (likely the return type) will change.
#[inline]
pub fn lower(
&mut self,
f: impl FnOnce(&mut LowerIter, LS>) -> Result,
) -> Result
where
LS: ParseState,
::Object: Token,
::Context: Default,
ParseError: Into,
{
self.while_ok(|toks| {
// TODO: This parser is not accessible after error recovery!
let lower = LS::parse(iter::empty());
let mut iter = LowerIter { lower, toks };
f(&mut iter)
})
.map_err(Into::into)
}
}
/// An IR lowering operation that pipes the output of one [`Parser`] to the
/// input of another.
///
/// This is produced by [`Parser::lower`].
pub struct LowerIter<'a, 'b, S, I, LS>
where
S: ParseState,
I: Iterator- >,
LS: ParseState,
::Object: Token,
{
/// A push [`Parser`].
lower: Parser>,
/// Source tokens from higher-level [`Parser`],
/// with the outer [`Result`] having been stripped by a [`TripIter`].
toks: &'a mut TripIter<
'b,
I,
Parsed,
ParseError,
>,
}
impl<'a, 'b, S, I, LS> LowerIter<'a, 'b, S, I, LS>
where
S: ParseState,
I: Iterator- >,
LS: ParseState,
::Object: Token,
{
/// Consume inner parser and yield its context.
#[inline]
fn finalize(self) -> Result> {
self.lower.finalize().map_err(|(_, e)| e)
}
}
/// Lowering operation from one [`ParseState`] to another.
pub trait Lower
where
S: ParseState,
LS: ParseState,
::Object: Token,
{
/// Perform a lowering operation between two parsers where the context
/// is both received and returned.
///
/// This allows state to be shared among parsers.
///
/// See [`ParseState::parse_with_context`] for more information.
fn lower_with_context(
&mut self,
ctx: LS::Context,
f: impl FnOnce(&mut LowerIter) -> Result,
) -> Result<(U, LS::Context), E>
where
Self: Iterator- > + Sized,
ParseError: Into,
ParseError: Into,
{
self.while_ok(|toks| {
let lower = LS::parse_with_context(iter::empty(), ctx);
let mut iter = LowerIter { lower, toks };
let val = f(&mut iter)?;
iter.finalize().map_err(Into::into).map(|ctx| (val, ctx))
})
}
}
impl
Lower for I
where
I: Iterator- > + Sized,
S: ParseState,
LS: ParseState,
::Object: Token,
{
}
impl<'a, 'b, S, I, LS> Iterator for LowerIter<'a, 'b, S, I, LS>
where
S: ParseState,
I: Iterator- >,
LS: ParseState,
::Object: Token,
{
type Item = ParsedResult;
/// Pull a token through the higher-level [`Parser`],
/// push it to the lowering parser,
/// and yield the resulting [`ParseResult`].
#[inline]
fn next(&mut self) -> Option {
match self.toks.next() {
None => None,
Some(Parsed::Incomplete) => Some(Ok(Parsed::Incomplete)),
Some(Parsed::Object(obj)) => Some(self.lower.feed_tok(obj)),
}
}
}
impl> Iterator for Parser {
type Item = ParsedResult;
/// Parse a single [`Token`] according to the current
/// [`ParseState`],
/// if available.
///
/// If the underlying [`TokenStream`] yields [`None`],
/// then the [`ParseState`] must be in an accepting state;
/// otherwise, [`ParseError::UnexpectedEof`] will occur.
///
/// This is intended to be invoked by [`Iterator::next`].
/// Accepting a token rather than the [`TokenStream`] allows the caller
/// to inspect the token first
/// (e.g. to store a copy of the [`Span`][crate::span::Span]).
#[inline]
fn next(&mut self) -> Option {
let otok = self.toks.next();
match otok {
None => match self.assert_accepting() {
Ok(()) => None,
Err(e) => Some(Err(e)),
},
Some(tok) => Some(self.feed_tok(tok)),
}
}
}
/// Common parsing errors produced by [`Parser`].
///
/// These errors are common enough that they are handled in a common way,
/// such that individual parsers needn't check for these situations
/// themselves.
///
/// Having a common type also allows combinators to handle error types in a
/// consistent way when composing parsers.
///
/// Parsers may return their own unique errors via the
/// [`StateError`][ParseError::StateError] variant.
#[derive(Debug, PartialEq)]
pub enum ParseError {
/// Token stream ended unexpectedly.
///
/// This error means that the parser was expecting more input before
/// reaching an accepting state.
/// This could represent a truncated file,
/// a malformed stream,
/// or maybe just a user that's not done typing yet
/// (e.g. in the case of an LSP implementation).
///
/// If no span is available,
/// then parsing has not even had the chance to begin.
/// If this parser follows another,
/// then the combinator ought to substitute a missing span with
/// whatever span preceded this invocation.
///
/// The string is intended to describe what was expected to have been
/// available based on the current [`ParseState`].
/// It is a heap-allocated string so that a copy of [`ParseState`]
/// needn't be stored.
UnexpectedEof(Span, String),
/// The parser reached an unhandled dead state.
///
/// Once a parser returns [`ParseStatus::Dead`],
/// a parent context must use that provided token as a lookahead.
/// If that does not occur,
/// [`Parser`] produces this error.
///
/// The string is intended to describe what was expected to have been
/// available based on the current [`ParseState`].
/// It is a heap-allocated string so that a copy of [`ParseState`]
/// needn't be stored.
UnexpectedToken(T, String),
/// A parser-specific error associated with an inner
/// [`ParseState`].
StateError(E),
}
impl ParseError {
pub fn inner_into(
self,
) -> ParseError
where
EA: Into,
{
use ParseError::*;
match self {
UnexpectedEof(span, desc) => UnexpectedEof(span, desc),
UnexpectedToken(x, desc) => UnexpectedToken(x, desc),
StateError(e) => StateError(e.into()),
}
}
}
impl From for ParseError {
fn from(e: E) -> Self {
Self::StateError(e)
}
}
impl Display for ParseError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
Self::UnexpectedEof(_, desc) => {
write!(f, "unexpected end of input while {desc}")
}
Self::UnexpectedToken(_, desc) => {
write!(f, "unexpected input while {desc}")
}
Self::StateError(e) => Display::fmt(e, f),
}
}
}
impl Error for ParseError {
fn source(&self) -> Option<&(dyn Error + 'static)> {
match self {
Self::StateError(e) => Some(e),
_ => None,
}
}
}
impl Diagnostic
for ParseError
{
fn describe(&self) -> Vec {
use ParseError::*;
match self {
UnexpectedEof(span, desc) => span.error(desc).into(),
UnexpectedToken(tok, desc) => tok.span().error(desc).into(),
// TODO: Is there any additional useful context we can augment
// this with?
StateError(e) => e.describe(),
}
}
}
impl From for Parser
where
S: ParseState,
I: TokenStream,
::Context: Default,
{
/// Create a new parser with a default context.
///
/// This can only be used if the associated [`ParseState::Context`] does
/// not implement [`Default`];
/// otherwise,
/// consider instantiating from a `(TokenStream, Context)` pair.
/// See also [`ParseState::parse`] and
/// [`ParseState::parse_with_context`].
fn from(toks: I) -> Self {
Self {
toks,
state: Default::default(),
last_span: UNKNOWN_SPAN,
ctx: Default::default(),
}
}
}
impl From<(I, C)> for Parser
where
S: ParseState,
I: TokenStream,
{
/// Create a new parser with a provided context.
///
/// For more information,
/// see [`ParseState::parse_with_context`].
///
/// See also [`ParseState::parse`].
fn from((toks, ctx): (I, C)) -> Self {
Self {
toks,
state: Default::default(),
last_span: UNKNOWN_SPAN,
ctx,
}
}
}
/// Result of a parsing operation.
#[derive(Debug, PartialEq, Eq)]
pub enum ParseStatus {
/// Additional tokens are needed to complete parsing of the next object.
Incomplete,
/// Parsing of an object is complete.
///
/// This does not indicate that the parser is complete,
/// as more objects may be able to be emitted.
Object(S::Object),
/// Parser encountered a dead state relative to the given token.
///
/// A dead state is an empty accepting state that has no state
/// transition for the given token.
/// A state is empty if a [`ParseStatus::Object`] will not be lost if
/// parsing ends at this point
/// (that is---there is no partially-built object).
/// This could simply mean that the parser has completed its job and
/// that control must be returned to a parent context.
///
/// If a parser is _not_ in an accepting state,
/// then an error ought to occur rather than a dead state;
/// the difference between the two is that the token associated with
/// a dead state can be used as a lookahead token in order to
/// produce a state transition at a higher level,
/// whereas an error indicates that parsing has failed.
/// Intuitively,
/// this means that a [`ParseStatus::Object`] had just been emitted
/// and that the token following it isn't something that can be
/// parsed.
///
/// If there is no parent context to handle the token,
/// [`Parser`] must yield an error.
Dead(S::Token),
}
impl, T: Object> From for ParseStatus {
fn from(obj: T) -> Self {
Self::Object(obj)
}
}
/// Result of a parsing operation.
///
/// Whereas [`ParseStatus`] is used by [`ParseState`] to influence parser
/// operation,
/// this type is public-facing and used by [`Parser`].
#[derive(Debug, PartialEq, Eq)]
pub enum Parsed {
/// Additional tokens are needed to complete parsing of the next object.
Incomplete,
/// Parsing of an object is complete.
///
/// This does not indicate that the parser is complete,
/// as more objects may be able to be emitted.
Object(O),
}
impl From> for Parsed {
fn from(status: ParseStatus) -> Self {
match status {
ParseStatus::Incomplete => Parsed::Incomplete,
ParseStatus::Object(x) => Parsed::Object(x),
ParseStatus::Dead(_) => {
unreachable!("Dead status must be filtered by Parser")
}
}
}
}
#[cfg(test)]
pub mod test {
use std::{assert_matches::assert_matches, iter::once};
use super::*;
use crate::{span::DUMMY_SPAN as DS, sym::GlobalSymbolIntern};
#[derive(Debug, PartialEq, Eq, Clone)]
enum TestToken {
Close(Span),
MarkDone(Span),
Text(Span),
SetCtxVal(u8),
}
impl Display for TestToken {
fn fmt(&self, _f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
unimplemented!("fmt::Display")
}
}
impl Token for TestToken {
fn span(&self) -> Span {
use TestToken::*;
match self {
Close(span) | MarkDone(span) | Text(span) => *span,
_ => UNKNOWN_SPAN,
}
}
}
impl Object for TestToken {}
#[derive(Debug, PartialEq, Eq)]
enum EchoState {
Empty,
Done,
}
impl Default for EchoState {
fn default() -> Self {
Self::Empty
}
}
#[derive(Debug, PartialEq, Default)]
struct StubContext {
val: u8,
}
impl ParseState for EchoState {
type Token = TestToken;
type Object = TestToken;
type Error = EchoStateError;
type Context = StubContext;
fn parse_token(
self,
tok: TestToken,
ctx: &mut StubContext,
) -> TransitionResult {
match tok {
TestToken::MarkDone(..) => Transition(Self::Done).ok(tok),
TestToken::Close(..) => {
Transition(self).err(EchoStateError::InnerError(tok))
}
TestToken::Text(..) => Transition(self).dead(tok),
TestToken::SetCtxVal(val) => {
ctx.val = val;
Transition(Self::Done).incomplete()
}
}
}
fn is_accepting(&self) -> bool {
*self == Self::Done
}
}
impl Display for EchoState {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(f, "::fmt")
}
}
#[derive(Debug, PartialEq, Eq)]
enum EchoStateError {
InnerError(TestToken),
}
impl Display for EchoStateError {
fn fmt(&self, _: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
unimplemented!()
}
}
impl Error for EchoStateError {
fn source(&self) -> Option<&(dyn Error + 'static)> {
None
}
}
impl Diagnostic for EchoStateError {
fn describe(&self) -> Vec {
unimplemented!()
}
}
type Sut = Parser;
#[test]
fn successful_parse_in_accepting_state_with_spans() {
// EchoState is placed into a Done state given Comment.
let tok = TestToken::MarkDone(DS);
let mut toks = once(tok.clone());
let mut sut = Sut::from(&mut toks);
// The first token should be processed normally.
// EchoState proxies the token back.
assert_eq!(Some(Ok(Parsed::Object(tok))), sut.next());
// This is now the end of the token stream,
// which should be okay provided that the first token put us into
// a proper accepting state.
assert_eq!(None, sut.next());
// Further, finalizing should work in this state.
assert!(sut.finalize().is_ok());
}
#[test]
fn fails_on_end_of_stream_when_not_in_accepting_state() {
let span = Span::new(10, 20, "ctx".intern());
let mut toks = [TestToken::Close(span)].into_iter();
let mut sut = Sut::from(&mut toks);
// The first token is fine,
// and allows us to acquire our most recent span.
sut.next();
// Given that we have no tokens,
// and that EchoState::default does not start in an accepting
// state,
// we must fail when we encounter the end of the stream.
assert_eq!(
Some(Err(ParseError::UnexpectedEof(
span.endpoints().1.unwrap(),
// All the states have the same string
// (at time of writing).
EchoState::default().to_string(),
))),
sut.next()
);
}
#[test]
fn returns_state_specific_error() {
// TestToken::Close causes EchoState to produce an error.
let errtok = TestToken::Close(DS);
let mut toks = [errtok.clone()].into_iter();
let mut sut = Sut::from(&mut toks);
assert_eq!(
Some(Err(ParseError::StateError(EchoStateError::InnerError(
errtok
)))),
sut.next()
);
// The token must have been consumed.
// It is up to a recovery process to either bail out or provide
// recovery tokens;
// continuing without recovery is unlikely to make sense.
assert_eq!(0, toks.len());
}
#[test]
fn fails_when_parser_is_finalized_in_non_accepting_state() {
let span = Span::new(10, 10, "ctx".intern());
// Set up so that we have a single token that we can use for
// recovery as part of the same iterator.
let recovery = TestToken::MarkDone(DS);
let mut toks = [
// Used purely to populate a Span.
TestToken::Close(span),
// Recovery token here:
recovery.clone(),
]
.into_iter();
let mut sut = Sut::from(&mut toks);
// Populate our most recently seen token's span.
sut.next();
// Attempting to finalize now in a non-accepting state should fail
// in the same way that encountering an end-of-stream does,
// since we're effectively saying "we're done with the stream"
// and the parser will have no further opportunity to reach an
// accepting state.
let result = sut.finalize();
assert_matches!(
result,
Err((_, ParseError::UnexpectedEof(s, _))) if s == span.endpoints().1.unwrap()
);
// The sut should have been re-returned,
// allowing for attempted error recovery if the caller can manage
// to produce a sequence of tokens that will be considered valid.
// `toks` above is set up already for this,
// which allows us to assert that we received back the same `sut`.
let mut sut = result.unwrap_err().0;
assert_eq!(Some(Ok(Parsed::Object(recovery))), sut.next());
// And so we should now be in an accepting state,
// able to finalize.
assert!(sut.finalize().is_ok());
}
#[test]
fn unhandled_dead_state_results_in_error() {
// A Text will cause our parser to return Dead.
let tok = TestToken::Text(DS);
let mut toks = once(tok.clone());
let mut sut = Sut::from(&mut toks);
// Our parser returns a Dead status,
// which is unhandled by any parent context
// (since we're not composing parsers),
// which causes an error due to an unhandled Dead state.
assert_eq!(
sut.next(),
Some(Err(ParseError::UnexpectedToken(
tok,
EchoState::default().to_string()
))),
);
}
// A context can be both retrieved from a finished parser and provided
// to a new one.
#[test]
fn provide_and_retrieve_context() {
// First, verify that it's initialized to a default context.
let mut toks = vec![TestToken::MarkDone(DS)].into_iter();
let mut sut = Sut::from(&mut toks);
sut.next().unwrap().unwrap();
let ctx = sut.finalize().unwrap();
assert_eq!(ctx, Default::default());
// Next, verify that the context that is manipulated is the context
// that is returned to us.
let val = 5;
let mut toks = vec![TestToken::SetCtxVal(5)].into_iter();
let mut sut = Sut::from(&mut toks);
sut.next().unwrap().unwrap();
let ctx = sut.finalize().unwrap();
assert_eq!(ctx, StubContext { val });
// Finally, verify that the context provided is the context that is
// used.
let val = 10;
let given_ctx = StubContext { val };
let mut toks = vec![TestToken::MarkDone(DS)].into_iter();
let mut sut = EchoState::parse_with_context(&mut toks, given_ctx);
sut.next().unwrap().unwrap();
let ctx = sut.finalize().unwrap();
assert_eq!(ctx, StubContext { val });
}
}