You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tvm.apache.org by GitBox <gi...@apache.org> on 2020/06/26 03:01:08 UTC

[GitHub] [incubator-tvm] jroesch opened a new pull request #5932: [Frontend][Relay] Add Parser 2.0

jroesch opened a new pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932


   This PR implements a pure C++ parser for Relay's text format and starts to lay ground work for improved error reporting and full program parsing work that I will send an RFC for sometime next week. The goal is to remove the external dependency on ANTLR and make it easier for non-parsing experts to make simple modifications or tweaks to the parser. 
   
   I have implemented nearly all the expression and definition parsing, I have some remaining work to do on parsing types and ensuring end to end examples are working. I am opening the PR now in draft form to solicit some initial feedback. 
   
   ### Features
   - [ ] graph definitions and variables
   - [ ] comments
   - [ ] integer literals 
   - [ ] float literals
   - [ ] boolean literals 
   - [ ] unary operations
   - [ ] binary operations 
   - [ ] parens 
   - [ ] operator table an operator precedence 
   - [ ] let bindings 
   - [ ] sequencing 
   - [ ] tuple expressions
   - [ ] function literals 
   - [ ] top-level global functions
   - [ ] recursive calls
   - [ ] if-then-else
   - [ ] function calls 
   - [ ] incomplete types 
   - [ ] builtin types 
   - [ ] tuple types 
   - [ ] adt definitions
   - [ ] match expression 
   - [ ] extern types 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] tqchen commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
tqchen commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r449303920



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {

Review comment:
       the template F is the most efficient way if you want it to be inlined




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651433312


   cc @antinucleon and @jwfromm 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451064556



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }

Review comment:
       with the method above. seems like there's shared structure at a glance




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451088080



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }

Review comment:
       There isn't really any easy way to refactor because you really templatize the code cleanly due to the need to pass dtypes around and perform the correct casting based on dtype and container type. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448249240



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {

Review comment:
       cc @tqchen do you know what the most efficient thing to do is? in the past I've ignored typechecking and just done ```
   ```
   template <typename F> 
     void Foo(F f) { ... }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448220277



##########
File path: src/parser/op_table.h
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file token.h
+ * \brief A operator table for parsing.
+ *
+ * Provides symbolic token sequences to map to TVM operators, with a given associativity and arity.
+ */
+
+#ifndef TVM_PARSER_OP_TABLE_H_
+#define TVM_PARSER_OP_TABLE_H_
+
+#include <fstream>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/container.h>
+#include <tvm/ir/op.h>
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+
+struct Rule {
+  std::vector<TokenType> tokens;
+  int precedence;
+  int arity;
+  tvm::Op op;
+  bool left_assoc;
+
+  Rule() : tokens(), precedence(0), arity(0 ), op(tvm::Op()), left_assoc(false) {}
+
+  Rule(std::vector<TokenType> tokens, tvm::Op op, int precedence, int arity = 2, bool left_assoc = false)
+      : tokens(tokens), precedence(precedence), arity(arity), op(op), left_assoc(false) {}
+
+  Rule(const Rule& rule) {
+    this->tokens = rule.tokens;
+    this->op = rule.op;
+    this->precedence = rule.precedence;
+    this->arity = rule.arity;
+    this->left_assoc = rule.left_assoc;
+  }
+};
+
+struct OperatorTable {
+  std::vector<Rule> rules;
+  std::unordered_map<std::string, Rule> this_is_a_hack;

Review comment:
       This was to codify that I need a better solution here, ideally I need some kind of `std::trie` but I don't really want to implement a prefix tree for this relatively simple use case. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451086755



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }
+
+  /*! \brief Parse zero or more Relay definitions. */
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+      auto next = Peek();
+      switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        case TokenType::Extern: {
+          Consume(TokenType::Extern);
+          // TODO(@jroesch): add some validation here?
+          defs.types.push_back(ParseTypeDef());

Review comment:
       yeah we need to do errors (which is the goal of my next PR, so I figured I would do all in one). 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448250021



##########
File path: tests/python/relay/test_ir_parser.py
##########
@@ -76,6 +76,7 @@ def graph_equal(lhs, rhs):
 
 def roundtrip(expr):
     x = relay.fromtext(expr.astext())
+    import pdb; pdb.set_trace()

Review comment:
       more debugging for last failing test case will remove when I'm done
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451143165



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }
+
+  /*! \brief Parse zero or more Relay definitions. */
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+      auto next = Peek();
+      switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        case TokenType::Extern: {
+          Consume(TokenType::Extern);
+          // TODO(@jroesch): add some validation here?
+          defs.types.push_back(ParseTypeDef());

Review comment:
       sounds good




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451068428



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }
+
+  /*! \brief Parse zero or more Relay definitions. */
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+      auto next = Peek();
+      switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        case TokenType::Extern: {
+          Consume(TokenType::Extern);
+          // TODO(@jroesch): add some validation here?
+          defs.types.push_back(ParseTypeDef());

Review comment:
       oh my bad. you meant to ensure there aren't any constructors. should be a two-line fix, right? just store the parsed def, then `CHECK_EQ(def->constructors.length, 0)`, before pushing it back.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451087346



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       No the problem is that that isn't a valid token, and trying to hack in is going to be a huge hack because its incredibly contextual and overlaps with a lots of other lexing rules. I don't really want to do it given that I WILL rip it out soon, and the old parser is still in place for now. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451142304



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }

Review comment:
       sounds good. wasn't sure




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r450743817



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {

Review comment:
       I looked into doing that, there are some annoying template deduction issues. Going to leave this way for now, after doing research and concluding this will be best performance besides using template parameters. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448250952



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {

Review comment:
       This is confusingly correct, the goal is to run n - 1 times. The implementation was sort of a quick hack to add lookahead for the one case we need it in the parser. My general goal is to only do lookahead by 1 (which is Peek()).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451054166



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       There are annoying issues with how this is done right now. I would like to move away from some ugly lexing hacks but in order to do that I need to change the semver. I would like to introduce module level attributes and instead provide general parsing for those instead of continue to hack this in. I will make sure this works before we purge the old parser. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451087346



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       No the problem is that that isn't a valid token, and trying to hack in is going to be a huge hack because its incredibly contextual and overlaps with a lots of other lexing rules. I don't really want to do it given that I WILL rip it out soon.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651432640


   I just marked this as ready for review, my suggestion is that we review the existing code and land it in an experimental state. I will finish the metadata parsing and integration tests on real models in follow up work. My fear is that if I do it on this branch we are looking at 4kloc+ to review all at once. The PR is already pretty big and will take time to review. Thoughts?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] MarisaKirisame commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
MarisaKirisame commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r447985013



##########
File path: tests/python/relay/test_ir_parser.py
##########
@@ -76,6 +76,7 @@ def graph_equal(lhs, rhs):
 
 def roundtrip(expr):
     x = relay.fromtext(expr.astext())
+    import pdb; pdb.set_trace()

Review comment:
       remove

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {

Review comment:
       const & for std::function

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  template <typename R>
+  R Block(std::function<R()> parser) {

Review comment:
       const & for std::function

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  Expr ParseBindingExpr() {
+    // We use a loop here so that the stack depth
+    // does not grow linearly with a sequence of
+    // graph or let bindings.
+    //
+    // Assuming we start at call depth k, we will
+    // enter k + c call frames to parse the RHS
+    // of the bindings where `c` is the depth
+    // of recursion needed by RHS.
+    //
+    // If RHS is a call expresssion the c=1.
+    //
+    // Once we have parsed the RHS we will be
+    // back at depth K, and will return to
+    // this loop header to parse another
+    // graph or let binding.
+    //
+    // This ensures for n sequential bindings
+    // the call depth will be the same before
+    // and after parsing the n bindings.
+    std::vector<std::pair<Var, Expr>> bindings;
+    int scopes = 0;
+
+    while (true) {
+      auto next = Peek();
+      if (next->token_type == TokenType::Graph && Lookahead(2)->token_type == TokenType::Equal) {
+        Match(TokenType::Graph);
+        Match(TokenType::Equal);
+        auto val = this->ParseExprBinOp();
+        Match(TokenType::Semicolon);
+        AddGraphBinding(next, val);
+      } else if (next->token_type == TokenType::Let) {
+        // Parse the 'let'.
+        Consume(TokenType::Let);
+
+        // Parse the local '%<id>'.
+        auto local_tok = Match(TokenType::Local);
+        auto string = local_tok.ToString();
+
+        // Parse the optional type annotation (':' <type>).
+        Type type;
+        if (WhenMatch(TokenType::Colon)) {
+          type = ParseType();
+        }
+
+        auto var = BindVar(string, type);
+
+        // Parse the '=';
+        Match(TokenType::Equal);
+
+        // Parse the body, and the ';'.
+        auto val = this->ParseExprBinOp();
+        Consume(TokenType::Semicolon);
+
+        // Add the bindings to the local data structure.
+        bindings.push_back({ var, val });
+        scopes++;
+        PushScope();
+      } else {
+        // This is the only case we will increase the stack
+        // depth.
+        //
+        // If we parse a program which is a sequence of N bindings
+        // followed by a single body expression we will end up with
+        // a call depth of 3, the first call to ParseExpr, then
+        // ParseBindingExpr, then finally ParseExpr once more.
+
+        auto body = this->ParseExpr();
+
+        // Remove the same number of scopes we added.
+        PopScopes(scopes);
+
+        if (bindings.size() == 0) {
+          return body;
+        } else {
+          // We can now build the let binding up backwards.
+          for (auto binding = bindings.rbegin(); binding != bindings.rend(); binding++) {
+            body = relay::Let(binding->first, binding->second, body);
+          }
+          return body;
+        }
+      }
+    }
+  }
+
+  std::string HackTokensAsString(int n) {
+    std::stringstream key;
+    n = std::min((int)(tokens.size() - pos), n);
+    for (int i = 0; i < n; i++) {
+      key << ToString(tokens.at(pos + i)->token_type);
+    }
+    return key.str();
+  }
+
+  std::vector<Rule> ParseOp() {
+    std::vector<Rule> matched;
+    Peek();
+    for (int i = 4; i > 0; i--) {
+      auto key = HackTokensAsString(i);
+      auto it = this->op_table.this_is_a_hack.find(key);
+      if (it != this->op_table.this_is_a_hack.end()) {
+        pos = pos + i;
+        matched.push_back(it->second);
+      }
+    }
+
+    return matched;
+  }
+
+  void DebugStack(const std::vector<Expr>& exprs, const std::vector<Rule>& rules) {
+      std::cout << "Expr Stack: ";
+      for (auto expr : exprs) {
+        std::cout << expr << ", ";
+      }
+
+      std::cout << std::endl;
+      std::cout << "Op Stack: ";
+      for (auto rule : rules) {
+        std::cout << rule.op << ", ";
+      }
+
+      std::cout << std::endl;
+  }
+
+
+  // Provides parsing a sequence of the form: <star> (T <sep>)* <tokens_for_before_stop> <stop>.
+  // the intended use case of the before stop parser to is allow a customized parsing rule for things
+  // such as attributes.
+  template<typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse, std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = { data };
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) { before_stop(); }
+      if (WhenMatch(stop)) {
+        return elements;
+      // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+      // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) { before_stop(); }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  Array<tvm::PrimExpr> ParseShape() {
+    auto dims = ParseSequence<tvm::PrimExpr>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto tok = Match(TokenType::Integer);
+      return Downcast<tvm::PrimExpr>(tok->data);
+    });
+    return dims;
+  }
+
+  Type ParseFunctionType() {
+    auto ty_params = ParseSequence<Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+      return ParseType();
+    });
+
+    Match(TokenType::Minus);
+    Match(TokenType::RAngle);
+    auto ret_type = ParseType();
+
+    return relay::FuncType(ty_params, ret_type, {}, {});
+  }
+
+  // Parses a user defined ADT or type variable.
+  Type ParseNonPrimitiveType(const Token& tok) {
+    std::cout << "inside of prim type " << tok << std::endl;
+    auto name = tok.ToString();
+    Type head_type;
+    auto global_type = type_names.Get(name);
+
+    if (!global_type.defined()) {
+      head_type = LookupTypeVar(tok);
+    } else {
+      head_type = global_type;
+    }
+
+    CHECK(head_type.defined())
+      << "head type must be defined";
+
+    Array<Type> arg_types;
+    if (Peek()->token_type == TokenType::LSquare) {
+      arg_types = ParseSequence<Type>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        return ParseType();
+      });
+    }
+
+    if (arg_types.size()) {
+      return TypeCall(head_type, arg_types);
+    } else {
+      return head_type;
+    }
+  }
+
+  Type ParseType() {
+    auto tok = Peek();
+
+    if (tok->token_type == TokenType::OpenParen) {
+      auto tys = ParseSequence<relay::Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+        return ParseType();
+      });
+      return relay::TupleType(tys);
+    } else if (WhenMatch(TokenType::Fn)) {
+      return ParseFunctionType();
+    } else if (WhenMatch(TokenType::Identifier)) {
+      auto id = tok.ToString();
+      if (id == "Tensor") {
+        Match(TokenType::LSquare);
+        auto shape = ParseShape();
+        Match(TokenType::Comma);
+        auto dtype_tok = Match(TokenType::Identifier);
+        auto dtype = DataType(String2DLDataType(dtype_tok.ToString()));
+        Match(TokenType::RSquare);
+        return TensorType(shape, dtype);
+      } else {
+        auto ty = tok.ToString();
+        if (ty.rfind("int", 0) == 0 || ty.find("float", 0) == 0 || ty.find("uint", 0) == 0 || ty.find("bool", 0) == 0) {
+          // Need to do better error handling here.
+          auto dtype = DataType(String2DLDataType(tok.ToString()));
+          return TensorType({}, dtype);
+        } else {
+          return ParseNonPrimitiveType(tok);
+        }
+      }
+    } if (WhenMatch(TokenType::Underscore)) {
+      return IncompleteType();
+    } else {
+      std::stringstream msg;
+      msg << "failed to parse type found ";
+      msg << tok;
+      diag_ctx.Emit({ tok->line, tok->column, msg.str() });
+      diag_ctx.Render();
+      return Type();
+    }
+  }
+
+  Attrs ParseAttrs(const std::string& type_key) {
+    Map<String, ObjectRef> kwargs;
+    auto attrs = tvm::ReflectionVTable::Global()->CreateObject(type_key, kwargs);
+    LOG(FATAL) << Attrs();
+    return Attrs();
+  }
+
+  Function ParseFunctionDef() {
+    PushScope();
+    PushTypeScope();
+
+    Array<TypeVar> generics;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    auto params = ParseSequence<Var>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto token = Match(TokenType::Local);
+      auto string = token.ToString();
+      Type type;
+      if (WhenMatch(TokenType::Colon)) {
+        type = ParseType();
+      }
+      return BindVar(string, type);
+    });
+
+    Type ret_type;
+    if (WhenMatch(TokenType::Minus)) {
+      Match(TokenType::RAngle);
+      ret_type = ParseType();
+    }
+
+    auto body = Block<Expr>([&]() {
+      return ParseExpr();
+    });
+
+    PopTypeScopes(1);
+    PopScopes(1);
+
+    return relay::Function(params, body, ret_type, generics);
+  }
+
+  Expr ParseIf() {
+    Consume(TokenType::If);
+    auto guard = Parens<Expr>([&] {
+      return ParseExpr();
+    });
+
+    auto true_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    Match(TokenType::Else);
+
+    auto false_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    return relay::If(guard, true_branch, false_branch);
+  }
+
+  Expr ParseMatch(bool is_partial) {
+    LOG(FATAL) << "parse match";
+  }
+
+  Expr ParseExpr() {
+    return ConsumeWhitespace<Expr>([this] {
+      std::vector<Expr> exprs;
+
+      while (true) {
+        auto next = Peek();
+        switch (next->token_type) {
+          // For graph or let, match first rhs, then invoke ParseBindingExpr
+          // ParseBindingExpression then parse_lhs() parse_rhs() ';' continue
+          case TokenType::Let:
+            exprs.push_back(ParseBindingExpr());
+            break;
+          case TokenType::Match:
+          case TokenType::PartialMatch:
+            bool is_partial = next->token_type == PartialMatch;
+            Consume(nest->token_type);
+            exprs.push_back(ParseMatch(is_partial));
+            break;
+          case TokenType::If: {
+            exprs.push_back(ParseIf());
+            break;
+          }
+          case TokenType::Graph:
+            if (Lookahead(2)->token_type == TokenType::Equal) {
+              exprs.push_back(ParseBindingExpr());
+              break;
+            }
+            // intentional fall through here.
+          default: {
+            DisplayNextN(100);

Review comment:
       name the magic number

##########
File path: tests/python/relay/test_ir_parser2.py
##########
@@ -0,0 +1,920 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import tvm
+from tvm import te
+from tvm import relay
+import pytest
+from numpy import isclose
+from typing import Union
+from functools import wraps
+raises_parse_error = pytest.mark.xfail(raises=tvm._ffi.base.TVMError)
+
+SEMVER = "v0.0.4"
+
+BINARY_OPS = {
+    "*": relay.multiply,
+    "/": relay.divide,
+    "+": relay.add,
+    "-": relay.subtract,
+    "<": relay.less,
+    ">": relay.greater,
+    "<=": relay.less_equal,
+    ">=": relay.greater_equal,
+    "==": relay.equal,
+    "!=": relay.not_equal,
+}
+
+TYPES = {
+    "int8",
+    "int16",
+    "int32",
+    "int64",
+
+    "uint8",
+    "uint16",
+    "uint32",
+    "uint64",
+
+    "float16",
+    "float32",
+    "float64",
+
+    "bool",
+
+    "int8x4",
+    "uint1x4",
+    "float16x4",
+}
+
+LIST_DEFN = """
+type List[A] {
+    Cons(A, List[A]),
+    Nil,
+}
+"""
+
+def assert_graph_equal(lhs, rhs):
+    tvm.ir.assert_structural_equal(lhs, rhs, map_free_vars=True)
+
+def graph_equal(lhs, rhs):
+    return tvm.ir.structural_equal(lhs, rhs, map_free_vars=True)
+
+
+def roundtrip(expr):
+    x = relay.fromtext(expr.astext())
+    assert_graph_equal(x, expr)
+
+
+def parse_text(code):
+    expr = tvm.parser.parse_expr(code)
+    roundtrip(expr)
+    return expr
+
+
+def parses_as(code, expr):
+    # type: (str, relay.Expr) -> bool
+    parsed = parse_text(code)
+    result = graph_equal(parsed, expr)
+    return result
+
+def parse_module(code):
+    mod = tvm.parser.parse(code)
+    roundtrip(mod)
+    return mod
+
+
+def assert_parses_as(code, expr):
+    parsed = parse_text(code)
+    assert_graph_equal(parsed, expr)
+
+def assert_parse_module_as(code, mod):
+    import pdb; pdb.set_trace()

Review comment:
       remove other pdb

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {

Review comment:
       const & for std::function
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451005507



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.

Review comment:
       ```suggestion
    * In order to properly handle scoping we must maintain a stack of scopes.
   ```

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)

Review comment:
       remove?

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.

Review comment:
       ```suggestion
    * which encode the parsing context and thus how to interpret the parenthesis.
   ```

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.

Review comment:
       ```suggestion
    * into a mode for parsing `[`, a shape, a comma, a data type and then a `]`.
   ```

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }

Review comment:
       does it make sense to refactor this?

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       even if we only support the current version, we should still validate the given version matches that, right?

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }
+
+  /*! \brief Parse zero or more Relay definitions. */
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+      auto next = Peek();
+      switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        case TokenType::Extern: {
+          Consume(TokenType::Extern);
+          // TODO(@jroesch): add some validation here?
+          defs.types.push_back(ParseTypeDef());

Review comment:
       i'm not sure if it _can_ be validated, since it's opaque. unless you mean something else

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.

Review comment:
       ```suggestion
      * Useful for matching optional tokens, effectively looks ahead by one.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448248019



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+

Review comment:
       I added a preamble to the class, the code below has *a lot* of comments esp. compared to the rest of TVM. I explained the high level bits but unfortunately to really understand the parser one will probably need to read the parsing code. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451066449



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       ohh. so the todo above means we remove the semver from the text format? we can discuss whether or not to do so later, but yeah, _until_ we do so, we should at least have a hack that checks for "v0.0.4", rather than the current half measure.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-654525930


   @ANSHUMAN87 here is some initial details https://discuss.tvm.ai/t/rfc-meta-rfc-3-pronged-plan-for-improving-error-messages-in-tvm/7214 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] zhiics edited a comment on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
zhiics edited a comment on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651437762


   I agree we should incrementally add the support of these language features to make review smoother. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] ANSHUMAN87 commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
ANSHUMAN87 commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-653846659


   @jroesch : Thanks for the PR! Great work :+1: 
   I totally agree with the motivations behind this, you have mentioned.
   I am sorry i could not find any RFC for this PR. Will it be possible to share some initial HLD draft ?
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r450743403



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  template <typename R>
+  R Block(std::function<R()> parser) {

Review comment:
       It seems that if you immediately consume you get better behavior with move semantics which is what we are doing here. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] tqchen commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
tqchen commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-655728286


   Thanks @jroesch @zhiics @weberlo 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-654736887


   Okay I addressed the vast majority of comments directly and hopefully got everything, CI is building if people can do another pass. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448252018



##########
File path: src/parser/tokenizer.h
##########
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.h
+ * \brief A parser for TVM IR.
+ */
+#ifndef TVM_PARSER_TOKENIZER_H_
+#define TVM_PARSER_TOKENIZER_H_
+
+#include <fstream>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/container.h>
+
+#include "./token.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace runtime;
+
+bool IsDigit(char c) {
+    return '0' <= c && c <= '9';
+}
+
+bool IsWhitespace(char c) {
+    return ' ' == c || c == '\t' || c == '\n';
+}
+
+bool IsNumeric(char c) {
+    return (IsDigit(c) || c == '.' || c == 'e' || c == '-' || c == '+' || c == 'E') && !IsWhitespace(c);
+}
+
+bool IsIdentLetter(char c) {
+    return '_' == c || ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z');
+}
+
+bool IsIdent(char c) {
+    return IsIdentLetter(c) || IsDigit(c);
+}
+
+static std::unordered_map<std::string, TokenType> KEYWORD_TABLE = {
+    { "let", TokenType::Let },
+    { "fn", TokenType::Fn },
+    { "def", TokenType::Defn },
+    { "if", TokenType::If },
+    { "else", TokenType::Else },
+    { "type", TokenType::TypeDef },
+    { "match", TokenType::Match }
+};
+
+struct Tokenizer {
+    int pos;
+    int col;
+    int line;
+    char next_char;
+    const std::string& source;
+    std::vector<Token> tokens;
+
+    char Next() {
+        char c = this->source.at(this->pos);
+        if (c == '\n') {
+            this->line += 1;
+            this->col = 1;
+        } else {
+            this->col += 1;
+        }
+        pos += 1;
+        return c;
+    }
+
+    bool More() {
+        return this->pos < this->source.size();
+    }
+
+    char Peek() {
+        CHECK(pos < this->source.size());
+        return this->source.at(this->pos);
+    }
+
+    Token NewToken(TokenType token_type, ObjectRef data = ObjectRef()) {
+        return Token(this->line, this->col, token_type, data);
+    }
+
+    enum CommentParserState {
+        Proceed,
+        Forward,
+        Backward,
+    };
+
+    void MatchComment(std::string& buffer) {
+        // We only invoke this after we have matched the first start
+        // token assume, we are proceeding the parse forward with
+        // nesting = 1.
+        //
+        // When we are done we should be at nesting zero and be
+        // in the stop state.
+        CommentParserState state = CommentParserState::Proceed;
+        int nesting = 1;
+
+        while (true) {
+            switch (state) {
+                case CommentParserState::Proceed: {
+                    if (Peek() == '/') {
+                        state = CommentParserState::Forward;
+                    } else if (Peek() == '*') {
+                        state = CommentParserState::Backward;
+                    }
+                    buffer += Next();
+                    continue;
+                }
+                case CommentParserState::Forward: {
+                    if (Peek() == '*') {
+                        nesting += 1;
+                        buffer += Next();
+                    }
+                    state = CommentParserState::Proceed;
+                    continue;
+                }
+                case CommentParserState::Backward: {
+                    if (Peek() == '/') {
+                        nesting -= 1;
+                        if (nesting == 0) {
+                            Next();
+                            buffer.pop_back();
+                            return;
+                        } else {
+                            buffer += Next();
+                            state = CommentParserState::Proceed;
+                        };
+                    }
+                    continue;
+                }
+            }
+        }
+    }
+
+    Token ParseNumber(bool is_pos, std::string number) {
+        CHECK(number.size() > 0)
+            << "an empty string is an invalid number";
+
+        try {
+            auto token = NewToken(TokenType::Integer);
+            size_t index = 0;
+            int value = std::stoi(number, &index);
+            if (number.size() > index) {
+                throw std::invalid_argument("floating point");
+            }
+            value = is_pos ? value : -value;
+            token->data = tvm::Integer(value);
+            return token;
+        } catch (const std::invalid_argument& ia) {
+            auto token = NewToken(TokenType::Float);
+
+            if (number.back() == 'f') {
+                number.pop_back();
+            }
+
+            double value = stod(number);
+            value = is_pos ? value : -value;
+            token->data = tvm::FloatImm(DataType::Float(64), value);
+            return token;
+        }
+    }
+
+    inline Token TokenizeOnce() {
+        auto next = Peek();
+        if (next == '\n') {
+            auto token = NewToken(TokenType::Newline);
+            Next();
+            return token;
+        } else if (this->Peek() == '\r' && this->Peek() == '\n') {

Review comment:
       there is actually a fix me comment below lol




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] tqchen merged pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
tqchen merged pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451055189



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }

Review comment:
       WDYM? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] tqchen edited a comment on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
tqchen edited a comment on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-655728286


   Thanks @jroesch @zhiics @weberlo @electriclilies @MarisaKirisame 
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451087346



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       No the problem is that that isn't a valid token, and trying to hack in is going to be a huge hack because its incredibly contextual and overlaps with a lots of other lexing rules. I don't really want to do it given that I WILL rip it out soon. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451066449



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       ohh. so the todo above means we remove the semver from the text format? we can discuss whether or not to do so later, but _until_ we do so, we should at least have a hack that checks for "v0.0.4", rather than the current half measure.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451054661



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }
+
+  /*! \brief Parse zero or more Relay definitions. */
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+      auto next = Peek();
+      switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        case TokenType::Extern: {
+          Consume(TokenType::Extern);
+          // TODO(@jroesch): add some validation here?
+          defs.types.push_back(ParseTypeDef());

Review comment:
       The parser is the only place we can reject this if it has non-zero fields. I will come back to this in the final clean up. Trying to land the initial infra and then can do a polish pass or two. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] electriclilies commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
electriclilies commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448050480



##########
File path: src/parser/tokenizer.h
##########
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.h
+ * \brief A parser for TVM IR.
+ */
+#ifndef TVM_PARSER_TOKENIZER_H_
+#define TVM_PARSER_TOKENIZER_H_
+
+#include <fstream>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/container.h>
+
+#include "./token.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace runtime;
+
+bool IsDigit(char c) {
+    return '0' <= c && c <= '9';
+}
+
+bool IsWhitespace(char c) {
+    return ' ' == c || c == '\t' || c == '\n';
+}
+
+bool IsNumeric(char c) {
+    return (IsDigit(c) || c == '.' || c == 'e' || c == '-' || c == '+' || c == 'E') && !IsWhitespace(c);
+}
+
+bool IsIdentLetter(char c) {
+    return '_' == c || ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z');
+}
+
+bool IsIdent(char c) {
+    return IsIdentLetter(c) || IsDigit(c);
+}
+
+static std::unordered_map<std::string, TokenType> KEYWORD_TABLE = {
+    { "let", TokenType::Let },
+    { "fn", TokenType::Fn },
+    { "def", TokenType::Defn },
+    { "if", TokenType::If },
+    { "else", TokenType::Else },
+    { "type", TokenType::TypeDef },
+    { "match", TokenType::Match }
+};
+
+struct Tokenizer {
+    int pos;
+    int col;
+    int line;
+    char next_char;
+    const std::string& source;
+    std::vector<Token> tokens;
+
+    char Next() {
+        char c = this->source.at(this->pos);
+        if (c == '\n') {
+            this->line += 1;
+            this->col = 1;
+        } else {
+            this->col += 1;
+        }
+        pos += 1;
+        return c;
+    }
+
+    bool More() {
+        return this->pos < this->source.size();
+    }
+
+    char Peek() {
+        CHECK(pos < this->source.size());
+        return this->source.at(this->pos);
+    }
+
+    Token NewToken(TokenType token_type, ObjectRef data = ObjectRef()) {
+        return Token(this->line, this->col, token_type, data);
+    }
+
+    enum CommentParserState {
+        Proceed,
+        Forward,
+        Backward,
+    };
+
+    void MatchComment(std::string& buffer) {
+        // We only invoke this after we have matched the first start
+        // token assume, we are proceeding the parse forward with
+        // nesting = 1.
+        //
+        // When we are done we should be at nesting zero and be
+        // in the stop state.
+        CommentParserState state = CommentParserState::Proceed;
+        int nesting = 1;
+
+        while (true) {
+            switch (state) {
+                case CommentParserState::Proceed: {
+                    if (Peek() == '/') {
+                        state = CommentParserState::Forward;
+                    } else if (Peek() == '*') {
+                        state = CommentParserState::Backward;
+                    }
+                    buffer += Next();
+                    continue;
+                }
+                case CommentParserState::Forward: {
+                    if (Peek() == '*') {
+                        nesting += 1;
+                        buffer += Next();
+                    }
+                    state = CommentParserState::Proceed;
+                    continue;
+                }
+                case CommentParserState::Backward: {
+                    if (Peek() == '/') {
+                        nesting -= 1;
+                        if (nesting == 0) {
+                            Next();
+                            buffer.pop_back();
+                            return;
+                        } else {
+                            buffer += Next();
+                            state = CommentParserState::Proceed;
+                        };
+                    }
+                    continue;
+                }
+            }
+        }
+    }
+
+    Token ParseNumber(bool is_pos, std::string number) {
+        CHECK(number.size() > 0)
+            << "an empty string is an invalid number";
+
+        try {
+            auto token = NewToken(TokenType::Integer);
+            size_t index = 0;
+            int value = std::stoi(number, &index);
+            if (number.size() > index) {
+                throw std::invalid_argument("floating point");
+            }
+            value = is_pos ? value : -value;
+            token->data = tvm::Integer(value);
+            return token;
+        } catch (const std::invalid_argument& ia) {
+            auto token = NewToken(TokenType::Float);
+
+            if (number.back() == 'f') {
+                number.pop_back();
+            }
+
+            double value = stod(number);
+            value = is_pos ? value : -value;
+            token->data = tvm::FloatImm(DataType::Float(64), value);
+            return token;
+        }
+    }
+
+    inline Token TokenizeOnce() {
+        auto next = Peek();
+        if (next == '\n') {
+            auto token = NewToken(TokenType::Newline);
+            Next();
+            return token;
+        } else if (this->Peek() == '\r' && this->Peek() == '\n') {

Review comment:
       This is weird to me -- why are you getting this->Peek() when you already put it in next? Also, a character can't be '/r' and '/n' at the same time so it's dead code

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+

Review comment:
       More doc in this file would be helpful in general, I'm having trouble seeing how all the classes fit together

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)

Review comment:
        CHECK_GT(n, 1) equivalent and clearer, but ensures that n > 1, in direct contradiction to the error message on next line.. 

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {

Review comment:
       Why does this fn have arguments? You prevent lookahead of more than 1, so the only arguments that work are args <= 0, which just return Peek() and doesn't update pos, or 1

##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {

Review comment:
       If n = 1, i < (n -1) becomes i < 0 -- so the loop won't execute when i = 1. Change to i < n or i <= n-1




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448218064



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  Expr ParseBindingExpr() {
+    // We use a loop here so that the stack depth
+    // does not grow linearly with a sequence of
+    // graph or let bindings.
+    //
+    // Assuming we start at call depth k, we will
+    // enter k + c call frames to parse the RHS
+    // of the bindings where `c` is the depth
+    // of recursion needed by RHS.
+    //
+    // If RHS is a call expresssion the c=1.
+    //
+    // Once we have parsed the RHS we will be
+    // back at depth K, and will return to
+    // this loop header to parse another
+    // graph or let binding.
+    //
+    // This ensures for n sequential bindings
+    // the call depth will be the same before
+    // and after parsing the n bindings.
+    std::vector<std::pair<Var, Expr>> bindings;
+    int scopes = 0;
+
+    while (true) {
+      auto next = Peek();
+      if (next->token_type == TokenType::Graph && Lookahead(2)->token_type == TokenType::Equal) {
+        Match(TokenType::Graph);
+        Match(TokenType::Equal);
+        auto val = this->ParseExprBinOp();
+        Match(TokenType::Semicolon);
+        AddGraphBinding(next, val);
+      } else if (next->token_type == TokenType::Let) {
+        // Parse the 'let'.
+        Consume(TokenType::Let);
+
+        // Parse the local '%<id>'.
+        auto local_tok = Match(TokenType::Local);
+        auto string = local_tok.ToString();
+
+        // Parse the optional type annotation (':' <type>).
+        Type type;
+        if (WhenMatch(TokenType::Colon)) {
+          type = ParseType();
+        }
+
+        auto var = BindVar(string, type);
+
+        // Parse the '=';
+        Match(TokenType::Equal);
+
+        // Parse the body, and the ';'.
+        auto val = this->ParseExprBinOp();
+        Consume(TokenType::Semicolon);
+
+        // Add the bindings to the local data structure.
+        bindings.push_back({ var, val });
+        scopes++;
+        PushScope();
+      } else {
+        // This is the only case we will increase the stack
+        // depth.
+        //
+        // If we parse a program which is a sequence of N bindings
+        // followed by a single body expression we will end up with
+        // a call depth of 3, the first call to ParseExpr, then
+        // ParseBindingExpr, then finally ParseExpr once more.
+
+        auto body = this->ParseExpr();
+
+        // Remove the same number of scopes we added.
+        PopScopes(scopes);
+
+        if (bindings.size() == 0) {
+          return body;
+        } else {
+          // We can now build the let binding up backwards.
+          for (auto binding = bindings.rbegin(); binding != bindings.rend(); binding++) {
+            body = relay::Let(binding->first, binding->second, body);
+          }
+          return body;
+        }
+      }
+    }
+  }
+
+  std::string HackTokensAsString(int n) {
+    std::stringstream key;
+    n = std::min((int)(tokens.size() - pos), n);
+    for (int i = 0; i < n; i++) {
+      key << ToString(tokens.at(pos + i)->token_type);
+    }
+    return key.str();
+  }
+
+  std::vector<Rule> ParseOp() {
+    std::vector<Rule> matched;
+    Peek();
+    for (int i = 4; i > 0; i--) {
+      auto key = HackTokensAsString(i);
+      auto it = this->op_table.this_is_a_hack.find(key);
+      if (it != this->op_table.this_is_a_hack.end()) {
+        pos = pos + i;
+        matched.push_back(it->second);
+      }
+    }
+
+    return matched;
+  }
+
+  void DebugStack(const std::vector<Expr>& exprs, const std::vector<Rule>& rules) {
+      std::cout << "Expr Stack: ";
+      for (auto expr : exprs) {
+        std::cout << expr << ", ";
+      }
+
+      std::cout << std::endl;
+      std::cout << "Op Stack: ";
+      for (auto rule : rules) {
+        std::cout << rule.op << ", ";
+      }
+
+      std::cout << std::endl;
+  }
+
+
+  // Provides parsing a sequence of the form: <star> (T <sep>)* <tokens_for_before_stop> <stop>.
+  // the intended use case of the before stop parser to is allow a customized parsing rule for things
+  // such as attributes.
+  template<typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse, std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = { data };
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) { before_stop(); }
+      if (WhenMatch(stop)) {
+        return elements;
+      // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+      // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) { before_stop(); }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  Array<tvm::PrimExpr> ParseShape() {
+    auto dims = ParseSequence<tvm::PrimExpr>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto tok = Match(TokenType::Integer);
+      return Downcast<tvm::PrimExpr>(tok->data);
+    });
+    return dims;
+  }
+
+  Type ParseFunctionType() {
+    auto ty_params = ParseSequence<Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+      return ParseType();
+    });
+
+    Match(TokenType::Minus);
+    Match(TokenType::RAngle);
+    auto ret_type = ParseType();
+
+    return relay::FuncType(ty_params, ret_type, {}, {});
+  }
+
+  // Parses a user defined ADT or type variable.
+  Type ParseNonPrimitiveType(const Token& tok) {
+    std::cout << "inside of prim type " << tok << std::endl;
+    auto name = tok.ToString();
+    Type head_type;
+    auto global_type = type_names.Get(name);
+
+    if (!global_type.defined()) {
+      head_type = LookupTypeVar(tok);
+    } else {
+      head_type = global_type;
+    }
+
+    CHECK(head_type.defined())
+      << "head type must be defined";
+
+    Array<Type> arg_types;
+    if (Peek()->token_type == TokenType::LSquare) {
+      arg_types = ParseSequence<Type>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        return ParseType();
+      });
+    }
+
+    if (arg_types.size()) {
+      return TypeCall(head_type, arg_types);
+    } else {
+      return head_type;
+    }
+  }
+
+  Type ParseType() {
+    auto tok = Peek();
+
+    if (tok->token_type == TokenType::OpenParen) {
+      auto tys = ParseSequence<relay::Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+        return ParseType();
+      });
+      return relay::TupleType(tys);
+    } else if (WhenMatch(TokenType::Fn)) {
+      return ParseFunctionType();
+    } else if (WhenMatch(TokenType::Identifier)) {
+      auto id = tok.ToString();
+      if (id == "Tensor") {
+        Match(TokenType::LSquare);
+        auto shape = ParseShape();
+        Match(TokenType::Comma);
+        auto dtype_tok = Match(TokenType::Identifier);
+        auto dtype = DataType(String2DLDataType(dtype_tok.ToString()));
+        Match(TokenType::RSquare);
+        return TensorType(shape, dtype);
+      } else {
+        auto ty = tok.ToString();
+        if (ty.rfind("int", 0) == 0 || ty.find("float", 0) == 0 || ty.find("uint", 0) == 0 || ty.find("bool", 0) == 0) {
+          // Need to do better error handling here.
+          auto dtype = DataType(String2DLDataType(tok.ToString()));
+          return TensorType({}, dtype);
+        } else {
+          return ParseNonPrimitiveType(tok);
+        }
+      }
+    } if (WhenMatch(TokenType::Underscore)) {
+      return IncompleteType();
+    } else {
+      std::stringstream msg;
+      msg << "failed to parse type found ";
+      msg << tok;
+      diag_ctx.Emit({ tok->line, tok->column, msg.str() });
+      diag_ctx.Render();
+      return Type();
+    }
+  }
+
+  Attrs ParseAttrs(const std::string& type_key) {
+    Map<String, ObjectRef> kwargs;
+    auto attrs = tvm::ReflectionVTable::Global()->CreateObject(type_key, kwargs);
+    LOG(FATAL) << Attrs();
+    return Attrs();
+  }
+
+  Function ParseFunctionDef() {
+    PushScope();
+    PushTypeScope();
+
+    Array<TypeVar> generics;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    auto params = ParseSequence<Var>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto token = Match(TokenType::Local);
+      auto string = token.ToString();
+      Type type;
+      if (WhenMatch(TokenType::Colon)) {
+        type = ParseType();
+      }
+      return BindVar(string, type);
+    });
+
+    Type ret_type;
+    if (WhenMatch(TokenType::Minus)) {
+      Match(TokenType::RAngle);
+      ret_type = ParseType();
+    }
+
+    auto body = Block<Expr>([&]() {
+      return ParseExpr();
+    });
+
+    PopTypeScopes(1);
+    PopScopes(1);
+
+    return relay::Function(params, body, ret_type, generics);
+  }
+
+  Expr ParseIf() {
+    Consume(TokenType::If);
+    auto guard = Parens<Expr>([&] {
+      return ParseExpr();
+    });
+
+    auto true_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    Match(TokenType::Else);
+
+    auto false_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    return relay::If(guard, true_branch, false_branch);
+  }
+
+  Expr ParseMatch(bool is_partial) {
+    LOG(FATAL) << "parse match";
+  }
+
+  Expr ParseExpr() {
+    return ConsumeWhitespace<Expr>([this] {
+      std::vector<Expr> exprs;
+
+      while (true) {
+        auto next = Peek();
+        switch (next->token_type) {
+          // For graph or let, match first rhs, then invoke ParseBindingExpr
+          // ParseBindingExpression then parse_lhs() parse_rhs() ';' continue
+          case TokenType::Let:
+            exprs.push_back(ParseBindingExpr());
+            break;
+          case TokenType::Match:
+          case TokenType::PartialMatch:
+            bool is_partial = next->token_type == PartialMatch;
+            Consume(nest->token_type);
+            exprs.push_back(ParseMatch(is_partial));
+            break;
+          case TokenType::If: {
+            exprs.push_back(ParseIf());
+            break;
+          }
+          case TokenType::Graph:
+            if (Lookahead(2)->token_type == TokenType::Equal) {
+              exprs.push_back(ParseBindingExpr());
+              break;
+            }
+            // intentional fall through here.
+          default: {
+            DisplayNextN(100);

Review comment:
       This is just an arbitrary call to debugging code that is there while I fix the last couple CI cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651432041


   cc @MarisaKirisame @joshpoll @wweic @zhiics 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r451143002



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/node/reflection.h>
+#include <tvm/relay/adt.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+/*! \brief A wrapper structure for capturing the result of parsing
+ * a global definition *before* we add it to the IRModule.
+ *
+ * This enables the parser to parse everything in one pass before
+ * constructing the IRModule.
+ */
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+/*! \brief A wrapper structure for capturing all top-level definitions
+ * when parsing a module.
+ */
+struct Definitions {
+  /*! \brief The set of global functions. */
+  std::vector<GlobalFunc> funcs;
+  /*! \brief The set of type definitions. */
+  std::vector<TypeData> types;
+  // TODO(@jroesch): contain meta-table below
+};
+
+/*! \brief A structure representing the semantic versioning information
+ * for a Relay program.
+ */
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+/*! \brief A reference to a "meta-expression".
+ *
+ * In the text format we allow referencing metadata which
+ * uses a compact serialization that proceeds the main
+ * program body.
+ *
+ * We can reference this table using an expression of
+ * the form `meta[Type][index]`.
+ *
+ * We must later resolve these references to actual in-memory
+ * AST nodes but this requires first parsing the full program
+ * then expanding these temporary AST nodes into their corresponding
+ * nodes.
+ *
+ * For example the nth large constant will be pretty-printed as meta[relay.Constant][n]
+ * with its compact binary serialization residing in the metadata section at the end
+ * of the program.
+ */
+class MetaRefExprNode : public TempExprNode {
+ public:
+  /*! \brief The type key of the meta expression. */
+  std::string type_key;
+  /*! \brief The index into the type key's table. */
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  // TODO(@jroesch): we probably will need to manually
+  // expand these with a pass.
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor for MetaRefExpr
+   * \param type_key The type key of the object in the meta section.
+   * \param kind The index into that subfield.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+/*! \brief A simple wrapper around a mapping from raw string names
+ * to a TVM variable, type variable or other binder type.
+ */
+template <typename T>
+struct Scope {
+  /*! \brief The internal map. */
+  std::unordered_map<std::string, T> name_map;
+};
+
+/*! \brief A stack of scopes.
+ *
+ * In order to properly handle scoping we must maintain a scope of stacks.
+ *
+ * A stack allows users to write programs which contain repeated variable
+ * names and to properly handle both nested scopes and removal of variables
+ * when they go out of scope.
+ *
+ * This is the classic approach to lexical scoping.
+ */
+template <typename T>
+class ScopeStack {
+ private:
+  std::vector<Scope<T>> scope_stack;
+
+ public:
+  /*! \brief Adds a variable binding to the current scope. */
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({name, value});
+  }
+
+  /*! \brief Looks up a variable name in the scope stack returning the matching variable
+   * in most recent scope. */
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); ++scope) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  /*! \brief Adds a fresh scope. */
+  void PushStack() { this->scope_stack.push_back(Scope<T>()); }
+
+  /*! \brief Removes the most recent scope. */
+  void PopStack() { this->scope_stack.pop_back(); }
+};
+
+/*! \brief A table of interning strings as global function and type names. */
+template <typename T>
+struct InternTable {
+  /*! \brief The internal table mapping strings to a unique allocation. */
+  std::unordered_map<std::string, T> table;
+
+  /*! \brief Add the unique allocation. */
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({name, t});
+    }
+  }
+
+  /*! \brief Return the unique allocation. */
+  Optional<T> Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return Optional<T>(it->second);
+    } else {
+      return Optional<T>();
+    }
+  }
+};
+
+/*! \brief The parser class is the main interface to the parser.
+ * the parser is not currently exposed beyond this .cc file.
+ *
+ * The parser is initialized with a diagnostic context, an
+ * operator table, and a token stream.
+ *
+ * The rest of the internal state is used to map the human readable
+ * form to in-memory IR representation.
+ *
+ * The main entry point to the parser are a set of parsing methods
+ * such as `ParseModule` and `ParseExpr`.
+ *
+ * As with traditional recursive descent parsers the parsing methods
+ * are factored recursively just as one would do with a formal language
+ * grammar.
+ *
+ * You can view a recursive descent parser as a human friendly way to specify
+ * a state machine, and thus this factoring is necessary as the 'state' of this
+ * machine is the combination of the current parsing method and the next token.
+ *
+ * Parsing proceeds by matching a token and then dispatching to the appropriate
+ * method to parse the next tokens in the stream.
+ *
+ * For example if we are parsing a type and encounter a "Tensor" token we switch
+ * into a mode for parsing `[`, a shape, a comma, a data type and then a ']'.
+ *
+ * Certain matches like this are unambiguous and proceed in a straight line fashion
+ * once the initial token is found. Other parsing is more complex and requires some
+ * tricks to correctly parse.
+ *
+ * For example when we find a '(' in an expression context, it may be part of
+ * a tuple, the arguments to a call, or a parenthesized expression. The below code
+ * disambiguate these cases by factoring expression parsing into a series of methods
+ * which encode the parsing context the and thus how to interpret the parenthesis.
+ *
+ * For more information one should be able to read the code in order starting with
+ * `ParseModule` or `ParseExpr`.
+ */
+class Parser {
+ public:
+  /*! \brief The version that the parser is parsing. */
+  SemVer version;
+
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A global mapping for constructor names. */
+  InternTable<Constructor> ctors;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  /*! \brief Examine the next token in the stream, the current parser is configured to be
+   * whitespace insensitive so we will skip all whitespace or comment tokens. */
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() && ignore_whitespace &&
+           (tokens.at(pos)->token_type == TokenType::Whitespace ||
+            tokens.at(pos)->token_type == TokenType::Newline ||
+            tokens.at(pos)->token_type == TokenType::LineComment ||
+            tokens.at(pos)->token_type == TokenType::Comment)) {
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  /*! \brief Lookahead by N tokens.
+   * \param n The number of tokens to lookahead.
+   * \return The Nth token.
+   */
+  Token Lookahead(int n) {
+    CHECK_GE(n, 1) << "lookahead is only valid when n >= 1";
+
+    // We intend to skip n - 1 tokens, then return the nth.
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  /*! \brief Consume a token, this method is the lowest level way to consume a token
+   * and will not ignore white space or look ahead in anyway.
+   *
+   * /param token_type The token type to match.
+   */
+  void Consume(const TokenType& token_type) {
+    if (tokens[pos]->token_type != token_type) {
+      std::string message =
+          "expected a " + Pretty(token_type) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render(std::cout);
+    }
+    pos++;
+  }
+
+  /*! Match a token in the stream, this will first invoke Peek, ignoring tokens such
+   * as whitespace or comments returning the first meaningful token.
+   *
+   * We then try and consume the requested token, this will trigger an error if the
+   * current token does not match the token_type.
+   */
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  /*! Conditionally consume a token when it matches, this will never trigger an error
+   * as we guard against consuming the token before we do.
+   *
+   * Useful for matching optional tokens, effectively looksahead by one.
+   */
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /* \brief Add a graph binding to the parsing context
+   *
+   * For example if we parse %0 = add(...), map 0 -> add(...), etc.
+   */
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  /* \brief Lookup a previously bound graph variable.
+   *
+   * Note: we take tokens in all lookup methods so that we
+   * that we can do error reporting based on token location.
+   */
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  /*! \brief Bind a local variable in the expression scope.
+   *
+   * "x" -> Var("x"), these are needed to map from the raw string names
+   * to unique variable nodes.
+   */
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  /*! \brief Bind a type variable in the type scope.
+   *
+   * "A" -> TypeVar("A", ...), these are needed to map from raw string names
+   * to unique type variable nodes.
+   */
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  /*! \brief Lookup a variable in the expression scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  /*! \brief Lookup a variable in the type scope.
+   *
+   * Note: all lookup methods take tokens intentionally for error reporting information.
+   */
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit(
+          {ident->line, ident->column,
+           "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  /*! \brief Add an expression scope to the scope stack. */
+  void PushScope() { this->expr_scopes.PushStack(); }
+
+  /*! \brief Remove N expression scopes from the scope stack. */
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Add an type scope to the scope stack. */
+  void PushTypeScope() { this->type_scopes.PushStack(); }
+
+  /*! \brief Remove N type scopes from the scope stack. */
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  /*! \brief Convert a numeric token to an NDArray for embedding into the Relay program. */
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      LOG(FATAL) << "internal error: should only call this function on numeric tokens";
+      return NDArray();
+    }
+  }
+
+  /*! \brief Convert a boolean value to an NDArray for embedding into the Relay program. */
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  /*! \brief A parsing helper for a bracketed expression <start> <parser> <stop>. */
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  /*! \brief Parse `(` parser() `)`. */
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  /*! \brief Parse `{` parser() `}`. */
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  /*! \brief Parses a sequence beginning with a start token, seperated by a seperator token, and
+   * ending with a stop token.
+   *
+   * The simple form being <start> (<parse()> <seperator>)* <stop>.
+   *
+   * This also provides a fourth argument which is allowed to run when the sequence which matches
+   * the inner sequence can not proceed.
+   *
+   * This is useful for parsing things like attributes which don't match the standard expression
+   * parsers but are contained within the stop token.
+   */
+  template <typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse,
+                         std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = {data};
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) {
+        before_stop();
+      }
+      if (WhenMatch(stop)) {
+        return elements;
+        // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+        // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) {
+          before_stop();
+        }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  /*! \brief Parse a full IRModule. */
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    this->version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    auto mod = IRModule({}, types);
+
+    for (auto func : defs.funcs) {
+      mod->Add(func.global, func.function);
+    }
+
+    return mod;
+  }
+
+  /*! \brief Parse the semantic versioning header. */
+  SemVer ParseSemVer() {
+    // TODO(@jroesch): convert semver to module level attribute.
+    auto id = Peek();
+    if (id->token_type == TokenType::Identifier && id.ToString() == "v0") {
+      auto id = Match(TokenType::Identifier);
+      Consume(TokenType::Period);
+      // CHECK_EQ(minor_and_patch)
+      Consume(TokenType::Float);
+    }
+    // For now we only support current version.
+    return SemVer{.major = 0, .minor = 0, .patch = 4};
+  }

Review comment:
       okay. if it's a big change, then we can leave it as is for now




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-654518640


   @ANSHUMAN87 I have been super busy and will post one soon.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448250340



##########
File path: src/parser/tokenizer.h
##########
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.h
+ * \brief A parser for TVM IR.
+ */
+#ifndef TVM_PARSER_TOKENIZER_H_
+#define TVM_PARSER_TOKENIZER_H_
+
+#include <fstream>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/container.h>
+
+#include "./token.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace runtime;
+
+bool IsDigit(char c) {
+    return '0' <= c && c <= '9';
+}
+
+bool IsWhitespace(char c) {
+    return ' ' == c || c == '\t' || c == '\n';
+}
+
+bool IsNumeric(char c) {
+    return (IsDigit(c) || c == '.' || c == 'e' || c == '-' || c == '+' || c == 'E') && !IsWhitespace(c);
+}
+
+bool IsIdentLetter(char c) {
+    return '_' == c || ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z');
+}
+
+bool IsIdent(char c) {
+    return IsIdentLetter(c) || IsDigit(c);
+}
+
+static std::unordered_map<std::string, TokenType> KEYWORD_TABLE = {
+    { "let", TokenType::Let },
+    { "fn", TokenType::Fn },
+    { "def", TokenType::Defn },
+    { "if", TokenType::If },
+    { "else", TokenType::Else },
+    { "type", TokenType::TypeDef },
+    { "match", TokenType::Match }
+};
+
+struct Tokenizer {
+    int pos;
+    int col;
+    int line;
+    char next_char;
+    const std::string& source;
+    std::vector<Token> tokens;
+
+    char Next() {
+        char c = this->source.at(this->pos);
+        if (c == '\n') {
+            this->line += 1;
+            this->col = 1;
+        } else {
+            this->col += 1;
+        }
+        pos += 1;
+        return c;
+    }
+
+    bool More() {
+        return this->pos < this->source.size();
+    }
+
+    char Peek() {
+        CHECK(pos < this->source.size());
+        return this->source.at(this->pos);
+    }
+
+    Token NewToken(TokenType token_type, ObjectRef data = ObjectRef()) {
+        return Token(this->line, this->col, token_type, data);
+    }
+
+    enum CommentParserState {
+        Proceed,
+        Forward,
+        Backward,
+    };
+
+    void MatchComment(std::string& buffer) {
+        // We only invoke this after we have matched the first start
+        // token assume, we are proceeding the parse forward with
+        // nesting = 1.
+        //
+        // When we are done we should be at nesting zero and be
+        // in the stop state.
+        CommentParserState state = CommentParserState::Proceed;
+        int nesting = 1;
+
+        while (true) {
+            switch (state) {
+                case CommentParserState::Proceed: {
+                    if (Peek() == '/') {
+                        state = CommentParserState::Forward;
+                    } else if (Peek() == '*') {
+                        state = CommentParserState::Backward;
+                    }
+                    buffer += Next();
+                    continue;
+                }
+                case CommentParserState::Forward: {
+                    if (Peek() == '*') {
+                        nesting += 1;
+                        buffer += Next();
+                    }
+                    state = CommentParserState::Proceed;
+                    continue;
+                }
+                case CommentParserState::Backward: {
+                    if (Peek() == '/') {
+                        nesting -= 1;
+                        if (nesting == 0) {
+                            Next();
+                            buffer.pop_back();
+                            return;
+                        } else {
+                            buffer += Next();
+                            state = CommentParserState::Proceed;
+                        };
+                    }
+                    continue;
+                }
+            }
+        }
+    }
+
+    Token ParseNumber(bool is_pos, std::string number) {
+        CHECK(number.size() > 0)
+            << "an empty string is an invalid number";
+
+        try {
+            auto token = NewToken(TokenType::Integer);
+            size_t index = 0;
+            int value = std::stoi(number, &index);
+            if (number.size() > index) {
+                throw std::invalid_argument("floating point");
+            }
+            value = is_pos ? value : -value;
+            token->data = tvm::Integer(value);
+            return token;
+        } catch (const std::invalid_argument& ia) {
+            auto token = NewToken(TokenType::Float);
+
+            if (number.back() == 'f') {
+                number.pop_back();
+            }
+
+            double value = stod(number);
+            value = is_pos ? value : -value;
+            token->data = tvm::FloatImm(DataType::Float(64), value);
+            return token;
+        }
+    }
+
+    inline Token TokenizeOnce() {
+        auto next = Peek();
+        if (next == '\n') {
+            auto token = NewToken(TokenType::Newline);
+            Next();
+            return token;
+        } else if (this->Peek() == '\r' && this->Peek() == '\n') {

Review comment:
       This is just a bug, this code will only ever need to run on windows so it hasn't been tested. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] zhiics commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
zhiics commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651437762


   I agree we should incrementally add the support of these language features to make review more smooth. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-650369009


   A few thoughts:
   It's not clear to me that modifying this parser is any easier than the current parser.  One could make a case that the current parser is suboptimal, because ANTLR does a sort of "covering parse", and [_parser.py](https://github.com/apache/incubator-tvm/blob/master/python/tvm/relay/_parser.py) then does **another** stage of parsing that incorporates context, but I would argue there's value in this separation of concerns, because you no longer need to worry about the syntactic components of parsing (e.g., [precedence and associativity](https://github.com/apache/incubator-tvm/pull/5932/files#diff-807cc0a7f01f9113c1903d4614a3649dR749-R769)).
   
   Another benefit of using a parser generator like ANTLR is that you have a [specification](https://github.com/apache/incubator-tvm/blob/master/python/tvm/relay/grammar/Relay.g4) of the language that serves as documentation **and** defines the parsing behavior, keeping the documentation always up to date.
   
   I see the value in error reporting integration and removing the external dependency, but it would be good to further motivate these changes and maybe find ways to further modularize version 2.0 to make it noob-friendly.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] weberlo commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
weberlo commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r446348563



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,968 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// expr
+//   // operators
+//   : '(' expr ')'                             # paren
+//   // function application
+//   | expr '(' callList ')'                    # call
+//   | '-' expr                                 # neg
+//   | expr op=('*'|'/') expr                   # binOp
+//   | expr op=('+'|'-') expr                   # binOp
+//   | expr op=('<'|'>'|'<='|'>=') expr         # binOp
+//   | expr op=('=='|'!=') expr                 # binOp
+//   // function definition
+//   | func                                     # funcExpr
+//   // tuples and tensors
+//   | '(' ')'                                  # tuple
+//   | '(' expr ',' ')'                         # tuple
+//   | '(' expr (',' expr)+ ')'                 # tuple
+//   | '[' (expr (',' expr)*)? ']'              # tensor
+//   | 'if' '(' expr ')' body 'else' body       # ifElse
+//   | matchType expr '{' matchClauseList? '}'  # match
+//   | expr '.' NAT                             # projection
+//   // sequencing
+//   | 'let' var '=' expr ';' expr              # let
+//   // sugar for let %_ = expr; expr
+//   | expr ';;' expr                           # let
+//   | graphVar '=' expr ';' expr               # graph
+//   | ident                                    # identExpr
+//   | scalar                                   # scalarExpr
+//   | meta                                     # metaExpr
+//   | QUOTED_STRING                            # stringExpr
+//   ;
+
+// func: 'fn' typeParamList? '(' argList ')' ('->' typeExpr)? body ;
+// defn
+//   : 'def' globalVar typeParamList? '(' argList ')' ('->' typeExpr)? body  # funcDefn
+//   | 'extern' 'type' generalIdent typeParamList?                           # externAdtDefn
+//   | 'type' generalIdent typeParamList? '{' adtConsDefnList? '}'           # adtDefn
+//   ;
+
+// constructorName: CNAME ;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+// adtCons: constructorName adtConsParamList? ;
+// adtConsParamList: '(' adtConsParam (',' adtConsParam)* ')' ;
+// adtConsParam: localVar | constructorName ;
+
+// argList
+//   : varList             # argNoAttr
+//   | (var ',')* attrSeq  # argWithAttr
+//   ;
+
+// varList: (var (',' var)*)? ;
+// var: localVar (':' typeExpr)? ;
+
+// attrSeq: attr (',' attr)* ;
+// attr: CNAME '=' expr ;
+
+// typeExpr
+//   : '(' ')'                                                                # tupleType
+//   | '(' typeExpr ')'                                                       # typeParen
+//   | '(' typeExpr ',' ')'                                                   # tupleType
+//   | '(' typeExpr (',' typeExpr)+ ')'                                       # tupleType
+//   | generalIdent typeParamList                                             # typeCallType
+//   | generalIdent                                                           # typeIdentType
+//   | 'Tensor' '[' shapeList ',' typeExpr ']'                                # tensorType
+//   | 'fn' typeParamList? '(' (typeExpr (',' typeExpr)*)? ')' '->' typeExpr  # funcType
+//   | '_'                                                                    # incompleteType
+//   ;
+
+// typeParamList: '[' typeExpr (',' typeExpr)* ']' ;
+
+// shapeList
+//   : '(' ')'
+//   | '(' shape (',' shape)+ ')'
+//   | shape
+//   ;
+
+// meta : 'meta' '[' CNAME ']' '[' NAT ']';
+
+// shape
+//   : meta           # metaShape
+//   | '(' shape ')'  # parensShape
+//   | NAT            # intShape
+//   ;
+
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+struct Rule {
+  std::vector<TokenType> tokens;
+  int precedence;
+  int arity;
+  tvm::Op op;
+  bool left_assoc;
+
+  Rule() : tokens(), precedence(0), arity(0), op(tvm::Op()), left_assoc(false) {}
+
+  Rule(std::vector<TokenType> tokens, tvm::Op op, int precedence, int arity = 2, bool left_assoc = false)
+      : tokens(tokens), precedence(precedence), arity(arity), op(op), left_assoc(false) {}
+
+  Rule(const Rule& rule) {
+    this->tokens = rule.tokens;
+    this->op = rule.op;
+    this->precedence = rule.precedence;
+    this->arity = rule.arity;
+    this->left_assoc = rule.left_assoc;
+  }
+};
+
+
+struct OperatorTable {
+  std::vector<Rule> rules;
+  std::unordered_map<std::string, Rule> this_is_a_hack;
+
+  OperatorTable(std::vector<Rule> rules) : rules(rules), this_is_a_hack() {
+    for (auto rule : rules) {
+      std::stringstream key;
+      for (auto token : rule.tokens) {
+        key << ToString(token);
+      }
+      this->this_is_a_hack.insert({ key.str(), rule });
+    }
+  }
+};
+
+struct Scope {
+  std::unordered_map<std::string, Var> name_map;
+  Scope() : name_map() {}
+};
+
+struct Parser {
+  int pos;
+  std::vector<Token> tokens;
+  OperatorTable op_table;
+  bool ignore_whitespace;
+
+  std::unordered_map<int, Expr> graph_ctx;
+  std::vector<Scope> scopes = { Scope() };
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table)
+      : pos(0), tokens(tokens), op_table(op_table) {
+        // DisplayNextN(100);
+  }
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      throw std::runtime_error(
+          "expected a " + ToString(token) + " found " + ToString(Peek()->token_type) + " at " +
+          std::to_string(tokens[pos]->line) + ":" + std::to_string(tokens[pos]->column));
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(std::string name, relay::Type type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->scopes.back().name_map.insert({name, var});
+    return var;
+  }
+
+  Var LookupVarByString(const std::string& var) {
+    for (auto scope = this->scopes.rbegin(); scope != this->scopes.rend(); scope++) {
+      auto it = scope->name_map.find(var);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    LOG(FATAL) << "foo";
+    return Var();
+  }
+
+  void PushScope() {
+    this->scopes.push_back(Scope());
+  }
+
+  void PopScope(int n) {

Review comment:
       rename to `PopScopes`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on a change in pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on a change in pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#discussion_r448249840



##########
File path: src/parser/parser.cc
##########
@@ -0,0 +1,1103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*!
+ * \file parser.cc
+ * \brief A parser for TVM IR.
+ */
+#include <tvm/ir/module.h>
+#include <tvm/relay/expr.h>
+#include <tvm/relay/function.h>
+#include <tvm/runtime/object.h>
+#include <tvm/runtime/registry.h>
+#include <tvm/node/reflection.h>
+
+#include <fstream>
+
+#include "./diagnostic.h"
+#include "./op_table.h"
+#include "./tokenizer.h"
+
+namespace tvm {
+namespace parser {
+
+using namespace relay;
+using Expr = relay::Expr;
+
+// adtConsDefnList: adtConsDefn (',' adtConsDefn)* ','? ;
+// adtConsDefn: constructorName ('(' typeExpr (',' typeExpr)* ')')? ;
+// matchClauseList: matchClause (',' matchClause)* ','? ;
+// matchClause: pattern '=>' ('{' expr '}' | expr) ;
+// // complete or incomplete match, respectively
+// matchType : 'match' | 'match?' ;
+
+// patternList: '(' pattern (',' pattern)* ')';
+// pattern
+//   : '_'                             # wildcardPattern
+//   | localVar (':' typeExpr)?        # varPattern
+//   | constructorName patternList?    # constructorPattern
+//   | patternList                     # tuplePattern
+//   ;
+
+struct GlobalFunc {
+  GlobalVar global;
+  Function function;
+  GlobalFunc() : global(), function() {}
+  GlobalFunc(GlobalVar global, Function function) : global(global), function(function) {}
+  GlobalFunc(const GlobalFunc& gfunc) {
+    this->global = gfunc.global;
+    this->function = gfunc.function;
+  }
+};
+
+struct Definitions {
+  std::vector<GlobalFunc> funcs;
+  std::vector<TypeData> types;
+};
+
+struct SemVer {
+  int major;
+  int minor;
+  int patch;
+};
+
+class MetaRefExpr;
+class MetaRefExprNode : public TempExprNode {
+ public:
+  std::string type_key;
+  uint64_t node_index;
+
+  void VisitAttrs(tvm::AttrVisitor* v) {}
+
+  Expr Realize() const final { return Expr(); }
+
+  static constexpr const char* _type_key = "relay.MetaRefExpr";
+  TVM_DECLARE_FINAL_OBJECT_INFO(MetaRefExprNode, TempExprNode);
+};
+
+class MetaRefExpr : public TempExpr {
+ public:
+  /*!
+   * \brief The constructor
+   * \param expr The original relay expression.
+   * \param kind The annotation kind.
+   */
+  TVM_DLL MetaRefExpr(std::string type_key, uint64_t node_index);
+
+  TVM_DEFINE_OBJECT_REF_METHODS(MetaRefExpr, TempExpr, MetaRefExprNode);
+};
+
+MetaRefExpr::MetaRefExpr(std::string type_key, uint64_t node_index) {
+  auto rnode = make_object<MetaRefExprNode>();
+  rnode->type_key = type_key;
+  rnode->node_index = node_index;
+  data_ = std::move(rnode);
+}
+
+template<typename T>
+struct Scope {
+  std::unordered_map<std::string, T> name_map;
+  Scope() : name_map() {}
+};
+
+template<typename T>
+struct ScopeStack {
+  std::vector<Scope<T>> scope_stack;
+
+  void Add(const std::string& name, const T& value) {
+    if (!this->scope_stack.size()) {
+      LOG(FATAL) << "internal issue";
+    }
+    this->scope_stack.back().name_map.insert({ name, value });
+  }
+
+  T Lookup(const std::string& name) {
+    for (auto scope = this->scope_stack.rbegin(); scope != this->scope_stack.rend(); scope++) {
+      auto it = scope->name_map.find(name);
+      if (it != scope->name_map.end()) {
+        return it->second;
+      }
+    }
+    return T();
+  }
+
+  void PushStack() {
+    this->scope_stack.push_back(Scope<T>());
+  }
+
+  void PopStack() {
+    this->scope_stack.pop_back();
+  }
+};
+
+template<typename T>
+struct InternTable {
+  std::unordered_map<std::string, T> table;
+  void Add(const std::string& name, const T& t) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      LOG(FATAL) << "duplicate name";
+    } else {
+      table.insert({ name, t});
+    }
+  }
+
+  T Get(const std::string& name) {
+    auto it = table.find(name);
+    if (it != table.end()) {
+      return it->second;
+    } else {
+      return T();
+    }
+  }
+};
+
+struct Parser {
+  /*! \brief The diagnostic context used for error reporting. */
+  DiagnosticContext diag_ctx;
+
+  /*! \brief The current position in the token stream. */
+  int pos;
+
+  /*! \brief The token stream for the parser. */
+  std::vector<Token> tokens;
+
+  /*! \brief The configured operator table. */
+  OperatorTable op_table;
+
+  /*! \brief Configure the whitespace mode, right now we ignore all whitespace. */
+  bool ignore_whitespace;
+
+  /*! \brief A global mapping for GlobalVar. */
+  InternTable<GlobalVar> global_names;
+
+  /*! \brief A global mapping for type definitions. */
+  InternTable<GlobalTypeVar> type_names;
+
+  /*! \brief A mapping from graph variable to expression, i.e., `%0 = expr`. */
+  std::unordered_map<int, Expr> graph_ctx;
+
+  /*! \brief The set of type scopes used for generics. */
+  ScopeStack<TypeVar> type_scopes;
+
+  /*! \brief The set of expression scopes used for lexical scope. */
+  ScopeStack<Var> expr_scopes;
+
+  Parser(std::vector<Token> tokens, OperatorTable op_table, Source source)
+      : diag_ctx(source), pos(0), tokens(tokens), op_table(op_table), ignore_whitespace(true) {}
+
+  void DisplayNextN(int n) {
+    std::cout << "remaining tokens: " << std::endl;
+    auto bound = std::min(pos + n, (int)tokens.size());
+    for (int i = 0; i < bound - pos; i++) {
+      std::cout << tokens[pos + i] << std::endl;
+    }
+  }
+
+  Token Peek() {
+    // For now we ignore all whitespace tokens and comments.
+    // We can tweak this behavior later to enable white space sensitivity in the parser.
+    while (pos < tokens.size() &&
+           ignore_whitespace && (tokens.at(pos)->token_type == TokenType::Whitespace ||
+                                 tokens.at(pos)->token_type == TokenType::Newline ||
+                                 tokens.at(pos)->token_type == TokenType::LineComment ||
+                                 tokens.at(pos)->token_type == TokenType::Comment)) {
+      // std::cout << "pos: " << pos << std::endl;
+      // std::cout << "tokens: " << tokens.size() << std::endl;
+      pos++;
+    }
+
+    if (pos < tokens.size()) {
+      return Token(this->tokens.at(pos));
+    } else {
+      return Token::Null();
+    }
+  }
+
+  // Allow lookahead into the token stream.
+  Token Lookahead(int n) {
+    CHECK_LE(1, n)
+      << "lookahead by > 1 is invalid";
+
+    auto old_pos = pos;
+    for (int i = 0; i < n - 1; i++) {
+      Peek();
+      pos++;
+    }
+
+    auto tok = Peek();
+    pos = old_pos;
+    return tok;
+  }
+
+  void Consume(const TokenType& token) {
+    if (tokens[pos]->token_type != token) {
+      std::string message =  "expected a " + Pretty(token) + " found " + Pretty(Peek()->token_type);
+      this->diag_ctx.Emit({tokens[pos]->line, tokens[pos]->column, message});
+      this->diag_ctx.Render();
+    }
+    pos++;
+  }
+
+  Token Match(const TokenType& token_type) {
+    auto tok = Peek();
+    Consume(token_type);
+    return tok;
+  }
+
+  bool WhenMatch(const TokenType& token_type) {
+    if (Peek()->token_type == token_type) {
+      Consume(token_type);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  void AddGraphBinding(const Token& token, const Expr& expr) {
+    auto graph_no = token.ToNumber();
+    this->graph_ctx.insert({graph_no, expr});
+  }
+
+  Expr LookupGraphBinding(const Token& token) {
+    auto graph_no = token.ToNumber();
+    return this->graph_ctx.at(graph_no);
+  }
+
+  Var BindVar(const std::string& name, const relay::Type& type_annotation) {
+    auto var = Var(name, type_annotation);
+    this->expr_scopes.Add(name, var);
+    return var;
+  }
+
+  TypeVar BindTypeVar(const std::string& name, const TypeKind type_kind) {
+    auto type_var = TypeVar(name, type_kind);
+    this->type_scopes.Add(name, type_var);
+    return type_var;
+  }
+
+  Var LookupLocal(const Token& local) {
+    auto var = this->expr_scopes.Lookup(local.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ local->line, local->column, "this local variable has not been previously declared"});
+    }
+    return var;
+  }
+
+  TypeVar LookupTypeVar(const Token& ident) {
+    auto var = this->type_scopes.Lookup(ident.ToString());
+    if (!var.defined()) {
+      diag_ctx.Emit({ ident->line, ident->column, "this type variable has not been previously declared anywhere, perhaps a typo?"});
+    }
+    return var;
+  }
+
+  void PushScope() {
+    this->expr_scopes.PushStack();
+  }
+
+  void PopScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->expr_scopes.PopStack();
+    }
+  }
+
+  void PushTypeScope() {
+    this->type_scopes.PushStack();
+  }
+
+  void PopTypeScopes(int n) {
+    for (int i = 0; i < n; i++) {
+      this->type_scopes.PopStack();
+    }
+  }
+
+  NDArray NumberToNDArray(const Token& token) {
+    if (token->token_type == TokenType::Integer) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("int32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<int32_t*>(data->data);
+      // revisit this, literal node issue.
+      int64_t value = Downcast<tvm::Integer>(token->data);
+      array[0] = (int32_t)value;
+      return data;
+    } else if (token->token_type == TokenType::Float) {
+      DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+      auto dtype = String2DLDataType("float32");
+      auto data = NDArray::Empty({}, dtype, ctx);
+      auto array = reinterpret_cast<float*>(data->data);
+      // revisit this, literal node issue.
+      float value = Downcast<tvm::FloatImm>(token->data)->value;
+      array[0] = value;
+      return data;
+    } else {
+      throw "foo";
+    }
+  }
+
+  NDArray BooleanToNDarray(bool value) {
+    DLContext ctx({.device_type = DLDeviceType::kDLCPU, .device_id = 0});
+    auto dtype = String2DLDataType("bool");
+    auto data = NDArray::Empty({}, dtype, ctx);
+    auto array = reinterpret_cast<bool*>(data->data);
+    array[0] = value;
+    return data;
+  }
+
+  [[noreturn]] void ParseError(const Token& token, const std::string& msg) {
+    throw std::runtime_error(msg);
+  }
+
+  IRModule ParseModule() {
+    // Parse the semver header at the top of the module.
+    auto _version = ParseSemVer();
+    // Parse the definitions.
+    auto defs = ParseDefinitions();
+    // Parse the metadata section at the end.
+    auto metadata = ParseMetadata();
+    Match(TokenType::EndOfFile);
+    Map<tvm::GlobalVar, BaseFunc> funcs;
+    Map<tvm::GlobalTypeVar, TypeData> types;
+
+    for (auto func : defs.funcs) {
+      funcs.Set(func.global, func.function);
+    }
+
+    for (auto type_def : defs.types) {
+      types.Set(type_def->header, type_def);
+    }
+
+    return IRModule(funcs, types);
+  }
+
+  SemVer ParseSemVer() {
+    // Consume(TokenType::Unknown);
+    return SemVer{.major = 0, .minor = 0, .patch = 0};
+  }
+
+  Definitions ParseDefinitions() {
+    Definitions defs;
+
+    while (true) {
+     auto next = Peek();
+     switch (next->token_type) {
+        case TokenType::Defn: {
+          Consume(TokenType::Defn);
+          auto global_name = Match(TokenType::Global).ToString();
+          auto global = GlobalVar(global_name);
+          global_names.Add(global_name, global);
+          auto func = ParseFunctionDef();
+          defs.funcs.push_back(GlobalFunc(global, func));
+          continue;
+        }
+        case TokenType::TypeDef: {
+          defs.types.push_back(ParseTypeDef());
+          continue;
+        }
+        default:
+          return defs;
+      }
+    }
+  }
+
+  TypeData ParseTypeDef() {
+    // Match the `type` keyword.
+    Match(TokenType::TypeDef);
+    // Parse the type's identifier.
+    auto type_id = Match(TokenType::Identifier).ToString();
+    auto type_global = tvm::GlobalTypeVar(type_id, TypeKind::kTypeData);
+    type_names.Add(type_id, type_global);
+
+    Array<TypeVar> generics;
+
+    bool should_pop = false;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      should_pop = true;
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    // Parse the list of constructors.
+    auto ctors = ParseSequence<tvm::Constructor>(TokenType::LCurly, TokenType::Comma, TokenType::RCurly, [&]() {
+      // First match the name of the constructor.
+      auto ctor = Match(TokenType::Identifier).ToString();
+      // Match the optional field list.
+      if (Peek()->token_type != TokenType::OpenParen) {
+        return tvm::Constructor(ctor, {}, type_global);
+      } else {
+        auto arg_types = ParseSequence<Type>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+          return ParseType();
+        });
+        return tvm::Constructor(ctor, arg_types, type_global);
+      }
+    });
+
+    // Now pop the type scope.
+    if (should_pop) {
+      PopTypeScopes(1);
+    }
+
+    return TypeData(type_global, generics, ctors);
+  }
+
+  template <typename R>
+  R Bracket(TokenType open, TokenType close, std::function<R()> parser) {
+    Match(open);
+    R result = parser();
+    Match(close);
+    return result;
+  }
+
+  template <typename R>
+  R Parens(std::function<R()> parser) {
+    return Bracket(TokenType::OpenParen, TokenType::CloseParen, parser);
+  }
+
+  template <typename R>
+  R Block(std::function<R()> parser) {
+    return Bracket(TokenType::LCurly, TokenType::RCurly, parser);
+  }
+
+  Expr ParseBindingExpr() {
+    // We use a loop here so that the stack depth
+    // does not grow linearly with a sequence of
+    // graph or let bindings.
+    //
+    // Assuming we start at call depth k, we will
+    // enter k + c call frames to parse the RHS
+    // of the bindings where `c` is the depth
+    // of recursion needed by RHS.
+    //
+    // If RHS is a call expresssion the c=1.
+    //
+    // Once we have parsed the RHS we will be
+    // back at depth K, and will return to
+    // this loop header to parse another
+    // graph or let binding.
+    //
+    // This ensures for n sequential bindings
+    // the call depth will be the same before
+    // and after parsing the n bindings.
+    std::vector<std::pair<Var, Expr>> bindings;
+    int scopes = 0;
+
+    while (true) {
+      auto next = Peek();
+      if (next->token_type == TokenType::Graph && Lookahead(2)->token_type == TokenType::Equal) {
+        Match(TokenType::Graph);
+        Match(TokenType::Equal);
+        auto val = this->ParseExprBinOp();
+        Match(TokenType::Semicolon);
+        AddGraphBinding(next, val);
+      } else if (next->token_type == TokenType::Let) {
+        // Parse the 'let'.
+        Consume(TokenType::Let);
+
+        // Parse the local '%<id>'.
+        auto local_tok = Match(TokenType::Local);
+        auto string = local_tok.ToString();
+
+        // Parse the optional type annotation (':' <type>).
+        Type type;
+        if (WhenMatch(TokenType::Colon)) {
+          type = ParseType();
+        }
+
+        auto var = BindVar(string, type);
+
+        // Parse the '=';
+        Match(TokenType::Equal);
+
+        // Parse the body, and the ';'.
+        auto val = this->ParseExprBinOp();
+        Consume(TokenType::Semicolon);
+
+        // Add the bindings to the local data structure.
+        bindings.push_back({ var, val });
+        scopes++;
+        PushScope();
+      } else {
+        // This is the only case we will increase the stack
+        // depth.
+        //
+        // If we parse a program which is a sequence of N bindings
+        // followed by a single body expression we will end up with
+        // a call depth of 3, the first call to ParseExpr, then
+        // ParseBindingExpr, then finally ParseExpr once more.
+
+        auto body = this->ParseExpr();
+
+        // Remove the same number of scopes we added.
+        PopScopes(scopes);
+
+        if (bindings.size() == 0) {
+          return body;
+        } else {
+          // We can now build the let binding up backwards.
+          for (auto binding = bindings.rbegin(); binding != bindings.rend(); binding++) {
+            body = relay::Let(binding->first, binding->second, body);
+          }
+          return body;
+        }
+      }
+    }
+  }
+
+  std::string HackTokensAsString(int n) {
+    std::stringstream key;
+    n = std::min((int)(tokens.size() - pos), n);
+    for (int i = 0; i < n; i++) {
+      key << ToString(tokens.at(pos + i)->token_type);
+    }
+    return key.str();
+  }
+
+  std::vector<Rule> ParseOp() {
+    std::vector<Rule> matched;
+    Peek();
+    for (int i = 4; i > 0; i--) {
+      auto key = HackTokensAsString(i);
+      auto it = this->op_table.this_is_a_hack.find(key);
+      if (it != this->op_table.this_is_a_hack.end()) {
+        pos = pos + i;
+        matched.push_back(it->second);
+      }
+    }
+
+    return matched;
+  }
+
+  void DebugStack(const std::vector<Expr>& exprs, const std::vector<Rule>& rules) {
+      std::cout << "Expr Stack: ";
+      for (auto expr : exprs) {
+        std::cout << expr << ", ";
+      }
+
+      std::cout << std::endl;
+      std::cout << "Op Stack: ";
+      for (auto rule : rules) {
+        std::cout << rule.op << ", ";
+      }
+
+      std::cout << std::endl;
+  }
+
+
+  // Provides parsing a sequence of the form: <star> (T <sep>)* <tokens_for_before_stop> <stop>.
+  // the intended use case of the before stop parser to is allow a customized parsing rule for things
+  // such as attributes.
+  template<typename T>
+  Array<T> ParseSequence(TokenType start, TokenType sep, TokenType stop, std::function<T()> parse, std::function<void()> before_stop = nullptr) {
+    Match(start);
+    if (WhenMatch(stop)) {
+      return Array<T>();
+    } else {
+      auto data = parse();
+      Array<T> elements = { data };
+
+      // parse '(' expr ')'
+      // if we are at the end invoke leftover parser
+      if (Peek()->token_type == stop && before_stop) { before_stop(); }
+      if (WhenMatch(stop)) {
+        return elements;
+      // parse '( expr ',' * ')'
+      } else if (WhenMatch(sep)) {
+      // if we are at the end invoke leftover parser
+        if (Peek()->token_type == stop && before_stop) { before_stop(); }
+        while (true) {
+          if (WhenMatch(stop)) {
+            break;
+          } else {
+            auto data = parse();
+            WhenMatch(sep);
+            elements.push_back(data);
+          }
+        }
+        return elements;
+      } else {
+        LOG(FATAL) << "issue";
+        return Array<T>(nullptr);
+      }
+    }
+  }
+
+  Array<tvm::PrimExpr> ParseShape() {
+    auto dims = ParseSequence<tvm::PrimExpr>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto tok = Match(TokenType::Integer);
+      return Downcast<tvm::PrimExpr>(tok->data);
+    });
+    return dims;
+  }
+
+  Type ParseFunctionType() {
+    auto ty_params = ParseSequence<Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+      return ParseType();
+    });
+
+    Match(TokenType::Minus);
+    Match(TokenType::RAngle);
+    auto ret_type = ParseType();
+
+    return relay::FuncType(ty_params, ret_type, {}, {});
+  }
+
+  // Parses a user defined ADT or type variable.
+  Type ParseNonPrimitiveType(const Token& tok) {
+    std::cout << "inside of prim type " << tok << std::endl;
+    auto name = tok.ToString();
+    Type head_type;
+    auto global_type = type_names.Get(name);
+
+    if (!global_type.defined()) {
+      head_type = LookupTypeVar(tok);
+    } else {
+      head_type = global_type;
+    }
+
+    CHECK(head_type.defined())
+      << "head type must be defined";
+
+    Array<Type> arg_types;
+    if (Peek()->token_type == TokenType::LSquare) {
+      arg_types = ParseSequence<Type>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        return ParseType();
+      });
+    }
+
+    if (arg_types.size()) {
+      return TypeCall(head_type, arg_types);
+    } else {
+      return head_type;
+    }
+  }
+
+  Type ParseType() {
+    auto tok = Peek();
+
+    if (tok->token_type == TokenType::OpenParen) {
+      auto tys = ParseSequence<relay::Type>(
+        TokenType::OpenParen,
+        TokenType::Comma,
+        TokenType::CloseParen, [&]() {
+        return ParseType();
+      });
+      return relay::TupleType(tys);
+    } else if (WhenMatch(TokenType::Fn)) {
+      return ParseFunctionType();
+    } else if (WhenMatch(TokenType::Identifier)) {
+      auto id = tok.ToString();
+      if (id == "Tensor") {
+        Match(TokenType::LSquare);
+        auto shape = ParseShape();
+        Match(TokenType::Comma);
+        auto dtype_tok = Match(TokenType::Identifier);
+        auto dtype = DataType(String2DLDataType(dtype_tok.ToString()));
+        Match(TokenType::RSquare);
+        return TensorType(shape, dtype);
+      } else {
+        auto ty = tok.ToString();
+        if (ty.rfind("int", 0) == 0 || ty.find("float", 0) == 0 || ty.find("uint", 0) == 0 || ty.find("bool", 0) == 0) {
+          // Need to do better error handling here.
+          auto dtype = DataType(String2DLDataType(tok.ToString()));
+          return TensorType({}, dtype);
+        } else {
+          return ParseNonPrimitiveType(tok);
+        }
+      }
+    } if (WhenMatch(TokenType::Underscore)) {
+      return IncompleteType();
+    } else {
+      std::stringstream msg;
+      msg << "failed to parse type found ";
+      msg << tok;
+      diag_ctx.Emit({ tok->line, tok->column, msg.str() });
+      diag_ctx.Render();
+      return Type();
+    }
+  }
+
+  Attrs ParseAttrs(const std::string& type_key) {
+    Map<String, ObjectRef> kwargs;
+    auto attrs = tvm::ReflectionVTable::Global()->CreateObject(type_key, kwargs);
+    LOG(FATAL) << Attrs();
+    return Attrs();
+  }
+
+  Function ParseFunctionDef() {
+    PushScope();
+    PushTypeScope();
+
+    Array<TypeVar> generics;
+    if (Peek()->token_type == TokenType::LSquare) {
+      // If we have generics we need to add a type scope.
+      PushTypeScope();
+      generics = ParseSequence<TypeVar>(TokenType::LSquare, TokenType::Comma, TokenType::RSquare, [&]() {
+        auto type_var_name = Match(TokenType::Identifier).ToString();
+        return BindTypeVar(type_var_name, TypeKind::kType);
+      });
+    }
+
+    auto params = ParseSequence<Var>(TokenType::OpenParen, TokenType::Comma, TokenType::CloseParen, [&]() {
+      auto token = Match(TokenType::Local);
+      auto string = token.ToString();
+      Type type;
+      if (WhenMatch(TokenType::Colon)) {
+        type = ParseType();
+      }
+      return BindVar(string, type);
+    });
+
+    Type ret_type;
+    if (WhenMatch(TokenType::Minus)) {
+      Match(TokenType::RAngle);
+      ret_type = ParseType();
+    }
+
+    auto body = Block<Expr>([&]() {
+      return ParseExpr();
+    });
+
+    PopTypeScopes(1);
+    PopScopes(1);
+
+    return relay::Function(params, body, ret_type, generics);
+  }
+
+  Expr ParseIf() {
+    Consume(TokenType::If);
+    auto guard = Parens<Expr>([&] {
+      return ParseExpr();
+    });
+
+    auto true_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    Match(TokenType::Else);
+
+    auto false_branch = Block<Expr>([&] {
+      return ParseExpr();
+    });
+
+    return relay::If(guard, true_branch, false_branch);
+  }
+
+  Expr ParseMatch(bool is_partial) {
+    LOG(FATAL) << "parse match";
+  }
+
+  Expr ParseExpr() {
+    return ConsumeWhitespace<Expr>([this] {
+      std::vector<Expr> exprs;
+
+      while (true) {
+        auto next = Peek();
+        switch (next->token_type) {
+          // For graph or let, match first rhs, then invoke ParseBindingExpr
+          // ParseBindingExpression then parse_lhs() parse_rhs() ';' continue
+          case TokenType::Let:
+            exprs.push_back(ParseBindingExpr());
+            break;
+          case TokenType::Match:
+          case TokenType::PartialMatch:
+            bool is_partial = next->token_type == PartialMatch;
+            Consume(nest->token_type);
+            exprs.push_back(ParseMatch(is_partial));
+            break;
+          case TokenType::If: {
+            exprs.push_back(ParseIf());
+            break;
+          }
+          case TokenType::Graph:
+            if (Lookahead(2)->token_type == TokenType::Equal) {
+              exprs.push_back(ParseBindingExpr());
+              break;
+            }
+            // intentional fall through here.
+          default: {
+            DisplayNextN(100);

Review comment:
       i.e it won't be in there when I land




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-tvm] jroesch commented on pull request #5932: [Frontend][Relay] Add Parser 2.0

Posted by GitBox <gi...@apache.org>.
jroesch commented on pull request #5932:
URL: https://github.com/apache/incubator-tvm/pull/5932#issuecomment-651316682


   @weberlo I think ANTLR only provides those benefits if you assume the people working on the project actually know ANTLR, which in so far as I can tell is not true. Josh and you were pretty much the only ones to work on the previous parser. Not to mention as we extend the parser to TIR and the rest of TVM it will become increasingly hard for anyone to make even small tweaks. 
   
   The current parser was also incomplete and failed to handle many tricky cases which can often be solved with small amounts of constant lookahead tokens. 
   
   Furthermore many of the grammar gymnastics required to parse in ANTLR are complex and easy for new users to break while this might still require some understanding the ordering is explicit in code for users to read and learn from.
    
   ANTLR is also a painful deployment dependency as we need Java, Python, and C++ to build the current parser. Furthermore the parser necessitated a rewrite given that it was in Python and needs to be in C++ or another statically linkable language. 
   
   Finally error reporting the main reason to write it by hand, if you look at most production quality compilers they have hand written parsers mostly for error reporting and recovery reasons. Most generated parsers fail on the first invalid token, or parse issue such as an invalid identifier. The above parser can continue even after encountering a parse error enabling better error reporting.
   
   In my exp. compilers which use parser generators i.e OCaml or F* have a horrible user exp. when compared to languages with hand rolled parsers such as Rust or Lean. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org