You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/10/27 20:17:35 UTC

Change in asterixdb[master]: [ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH

From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13823 )


Change subject: [ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH
......................................................................

[ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH

- user model changes: yes
- storage format changes: yes
- interface changes: no

Details:
- Add grammar support for graph view definitions, which in the future
  will provide support for graph queries on these graph views.
- Create a new Metadata dataset called "Graph".
- Add support for CREATE GRAPH, which will store graph view definitions
  in the "Graph" dataset.
- Add support for DROP GRAPH, which will remove these definitions in the
  "Graph" dataset.
- Add dependency support for graph views.

Change-Id: I20ca6ea4c782d2fbd548e7340db65cdab5ae693d
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
M asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
A asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
A asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
A asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
64 files changed, 3,264 insertions(+), 75 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/23/13823/1

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index f2bb66d..79dfe70 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -64,6 +64,7 @@
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.statement.ViewDecl;
@@ -189,8 +190,8 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
-            SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+            List<ViewDecl> declaredViews, List<GraphElementDecl> declaredGraphs, MetadataProvider metadataProvider,
+            IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
             Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
         if (q == null) {
             return null;
@@ -201,7 +202,7 @@
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
         LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
-                warningCollector, q.getVarCounter());
+                declaredGraphs, warningCollector, q.getVarCounter());
         rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 70ac386..84b4231 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -87,6 +87,7 @@
 import org.apache.asterix.common.external.IDataSourceAdapter;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.IMetadataLockUtil;
@@ -119,6 +120,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -136,6 +138,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
@@ -157,6 +161,7 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.GraphUtil;
 import org.apache.asterix.lang.common.util.ViewUtil;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.IDatasetDetails;
@@ -179,6 +184,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Library;
@@ -435,6 +441,12 @@
                     case VIEW_DROP:
                         handleViewDropStatement(metadataProvider, stmt);
                         break;
+                    case CREATE_GRAPH:
+                        handleCreateGraphStatement(metadataProvider, stmt);
+                        break;
+                    case GRAPH_DROP:
+                        handleGraphDropStatement(metadataProvider, stmt);
+                        break;
                     case LOAD:
                         handleLoadStatement(metadataProvider, stmt, hcc);
                         break;
@@ -2651,7 +2663,7 @@
             IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
             Query wrappedQuery = queryRewriter.createViewAccessorQuery(viewDecl);
             metadataProvider.setDefaultDataverse(dv);
-            apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), metadataProvider,
+            apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), null, metadataProvider,
                     wrappedQuery, sessionOutput, false, false, Collections.emptyList(), warningCollector);
 
             List<List<Triple<DataverseName, String, String>>> dependencies =
@@ -2744,6 +2756,163 @@
         }
     }
 
+    public void handleCreateGraphStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        CreateGraphStatement cgs = (CreateGraphStatement) stmt;
+        String graphName = cgs.getGraphName();
+        metadataProvider.validateDatabaseObjectName(cgs.getDataverseName(), graphName, stmt.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(cgs.getDataverseName());
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        lockUtil.createGraphBegin(lockManager, metadataProvider.getLocks(), graphIdentifier);
+        try {
+            doCreateGraph(metadataProvider, cgs, graphIdentifier);
+
+        } finally {
+            metadataProvider.getLocks().unlock();
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
+        }
+    }
+
+    protected CreateResult doCreateGraph(MetadataProvider metadataProvider, CreateGraphStatement cgs,
+            GraphIdentifier graphIdentifier) throws Exception {
+        DataverseName dataverseName = graphIdentifier.getDataverseName();
+        String graphName = graphIdentifier.getGraphName();
+        SourceLocation sourceLoc = cgs.getSourceLocation();
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+            }
+
+            Graph existingGraph = MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+            if (existingGraph != null) {
+                if (cgs.isIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return CreateResult.NOOP;
+
+                } else if (!cgs.isReplaceIfExists()) {
+                    throw new CompilationException(ErrorCode.GRAPH_EXISTS, sourceLoc, existingGraph.getGraphName());
+                }
+            }
+
+            // Build the graph schema.
+            Graph.Schema.Builder schemaBuilder = new Graph.Schema.Builder(graphIdentifier);
+            List<GraphElementDecl> graphElementDecls = new ArrayList<>();
+            for (CreateGraphStatement.VertexElement vertex : cgs.getVertexElements()) {
+                Graph.Vertex schemaVertex = schemaBuilder.addVertex(vertex.getLabel(), vertex.getPrimaryKeyFields(),
+                        vertex.getDefinition());
+                if (schemaVertex == null) {
+                    throw new CompilationException(ErrorCode.INVALID_GRAPH_VERTEX_DEFINITION, sourceLoc,
+                            "Conflicting primary keys for vertices with label " + vertex.getLabel());
+                }
+                graphElementDecls.add(new GraphElementDecl(schemaVertex.getIdentifier(), vertex.getExpression()));
+            }
+            for (CreateGraphStatement.EdgeElement edge : cgs.getEdgeElements()) {
+                Graph.Edge schemaEdge;
+
+                if (edge.getDefinition() == null) {
+                    schemaEdge = schemaBuilder.addEdge(edge.getEdgeLabel(), edge.getDestinationLabel(),
+                            edge.getSourceLabel(), edge.getDestinationKeyFields());
+
+                } else {
+                    schemaEdge = schemaBuilder.addEdge(edge.getEdgeLabel(), edge.getDestinationLabel(),
+                            edge.getSourceLabel(), edge.getPrimaryKeyFields(), edge.getDestinationKeyFields(),
+                            edge.getSourceKeyFields(), edge.getDefinition());
+                    graphElementDecls.add(new GraphElementDecl(schemaEdge.getIdentifier(), edge.getExpression()));
+                }
+
+                if (schemaEdge == null) {
+                    throw new CompilationException(ErrorCode.INVALID_GRAPH_EDGE_DEFINITION, sourceLoc,
+                            "Vertex label in edge with label " + edge.getEdgeLabel() + " not found");
+                }
+            }
+
+            // Verify that each element definition is usable.
+            IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
+            metadataProvider.setDefaultDataverse(dv);
+            for (GraphElementDecl elementDecl : graphElementDecls) {
+                Query wrappedQuery = queryRewriter.createGraphElementAccessorQuery(elementDecl);
+                apiFramework.reWriteQuery(declaredFunctions, null, Collections.singletonList(elementDecl),
+                        metadataProvider, wrappedQuery, sessionOutput, false, false, Collections.emptyList(),
+                        warningCollector);
+            }
+
+            // Build our dependencies (collected over all graph element bodies).
+            List<List<Triple<DataverseName, String, String>>> dependencies =
+                    GraphUtil.getGraphDependencies(graphElementDecls, queryRewriter);
+
+            // Add / upsert our graph to our metadata.
+            Graph newGraph = new Graph(graphIdentifier, schemaBuilder.build(), dependencies);
+            if (existingGraph == null) {
+                MetadataManager.INSTANCE.addGraph(mdTxnCtx, newGraph);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return CreateResult.CREATED;
+
+            } else {
+                MetadataManager.INSTANCE.updateGraph(mdTxnCtx, newGraph);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return CreateResult.REPLACED;
+            }
+
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
+    public void handleGraphDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        GraphDropStatement gds = (GraphDropStatement) stmt;
+        String graphName = gds.getGraphName();
+        metadataProvider.validateDatabaseObjectName(gds.getDataverseName(), graphName, stmt.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(gds.getDataverseName());
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        lockUtil.dropGraphBegin(lockManager, metadataProvider.getLocks(), graphIdentifier);
+        try {
+            doDropGraph(metadataProvider, gds, dataverseName, graphName);
+
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected boolean doDropGraph(MetadataProvider metadataProvider, GraphDropStatement gds,
+            DataverseName dataverseName, String graphName) throws Exception {
+        SourceLocation sourceLoc = gds.getSourceLocation();
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dataverse == null) {
+                if (gds.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return false;
+                } else {
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+                }
+            }
+
+            Graph graph = MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+            if (graph == null) {
+                if (gds.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return false;
+                } else {
+                    throw new CompilationException(ErrorCode.UNKNOWN_GRAPH, sourceLoc, graphName);
+                }
+            }
+
+            MetadataManager.INSTANCE.dropGraph(mdTxnCtx, dataverseName, graphName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            return true;
+
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
     protected void handleDeclareFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FunctionDecl fds = (FunctionDecl) stmt;
         FunctionSignature signature = fds.getSignature();
@@ -2920,8 +3089,8 @@
                 fdList.addAll(declaredFunctions);
                 fdList.add(fd);
                 metadataProvider.setDefaultDataverse(dv);
-                apiFramework.reWriteQuery(fdList, null, metadataProvider, wrappedQuery, sessionOutput, false, false,
-                        Collections.emptyList(), warningCollector);
+                apiFramework.reWriteQuery(fdList, null, null, metadataProvider, wrappedQuery, sessionOutput, false,
+                        false, Collections.emptyList(), warningCollector);
 
                 List<List<Triple<DataverseName, String, String>>> dependencies =
                         FunctionUtil.getFunctionDependencies(fd, queryRewriter);
@@ -3638,7 +3807,7 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null, null,
                 metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
@@ -3655,7 +3824,7 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(insertUpsert, stmtParams);
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null, null,
                 metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8a87de7..32efc6e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -202,7 +202,7 @@
                 if (st.getKind() == Statement.Kind.QUERY) {
                     Query query = (Query) st;
                     IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
-                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions, null,
+                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions, null, null,
                             TestUtils.NOOP_WARNING_COLLECTOR, query.getVarCounter());
                     rewrite(rewriter, query, rwContext);
 
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
new file mode 100644
index 0000000..60df6e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+// Verify that the Yelp use case generates the appropriate graph metadata.
+
+DROP DATAVERSE    Yelp IF EXISTS;
+DROP DATAVERSE    Yelp_A IF EXISTS;
+DROP DATAVERSE    Yelp_B IF EXISTS;
+CREATE DATAVERSE  Yelp;
+CREATE DATAVERSE  Yelp_A;
+CREATE DATAVERSE  Yelp_B;
+
+USE               Yelp_A;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+CREATE DATASET    Businesses (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Checkins (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Friends (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Reviews (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Tips (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Users (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE FUNCTION   RelevantBusinesses()
+                  { FROM Businesses B
+                    WHERE B.stars > 3.5
+                    SELECT B.* };
+
+USE               Yelp_B;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+CREATE DATASET    Businesses (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Checkins (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Friends (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Reviews (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Tips (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Users (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE SYNONYM    Yelpers FOR Users;
+
+USE               Yelp;
+CREATE GRAPH      YelpGraph_1 AS
+VERTEX            (:User)
+                  PRIMARY KEY (user_id)
+                  AS Yelp_B.Yelpers,
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:User)-[:FRIENDS_WITH]->(:User)
+                  PRIMARY KEY (user_id, friend)
+                  SOURCE KEY (user_id)
+                  DESTINATION KEY (friend)
+                  AS ( FROM    Yelp_B.Users U
+                       UNNEST  U.friends F
+                       SELECT  U.user_id, F AS friend ),
+EDGE              (:User)-[:FRIENDS_WITH]->(:User)
+                          PRIMARY KEY (user_id, friend)
+                          SOURCE KEY (user_id)
+                          DESTINATION KEY (friend)
+                          AS Yelp_B.Friends,
+EDGE              (:Review)-[:MADE_BY]->(:User)
+                  DESTINATION KEY (user_id),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
+
+CREATE GRAPH      YelpGraph_2 IF NOT EXISTS AS
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
+CREATE GRAPH      YelpGraph_2 IF NOT EXISTS AS
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
new file mode 100644
index 0000000..ca52d3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+DROP GRAPH  Yelp.YelpGraph_1 IF EXISTS;
+DROP GRAPH  Yelp.YelpGraph_1 IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
new file mode 100644
index 0000000..586a718
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+CREATE OR REPLACE GRAPH   Yelp.YelpGraph_2 AS
+VERTEX                    (:User)
+                          PRIMARY KEY (user_id)
+                          AS Yelp_B.Yelpers,
+VERTEX                    (:Review)
+                          PRIMARY KEY (review_id)
+                          AS ( FROM    Yelp_A.Reviews R
+                               SELECT  VALUE R ),
+VERTEX                    (:Business)
+                          PRIMARY KEY (business_id)
+                          AS ( FROM    Yelp_A.RelevantBusinesses() B
+                               SELECT  VALUE B ),
+EDGE                      (:User)-[:FRIENDS_WITH]->(:User)
+                          PRIMARY KEY (user_id, friend)
+                          SOURCE KEY (user_id)
+                          DESTINATION KEY (friend)
+                          AS ( FROM    Yelp_B.Users U
+                               UNNEST  U.friends F
+                               SELECT  U.user_id, F AS friend ),
+EDGE                      (:User)-[:FRIENDS_WITH]->(:User)
+                                  PRIMARY KEY (user_id, friend)
+                                  SOURCE KEY (user_id)
+                                  DESTINATION KEY (friend)
+                                  AS Yelp_B.Friends,
+EDGE                      (:Review)-[:MADE_BY]->(:User)
+                          DESTINATION KEY (user_id),
+EDGE                      (:Review)-[:ABOUT]->(:Business)
+                          DESTINATION KEY (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
new file mode 100644
index 0000000..25cc3da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_1", "Dependencies": [ [ [ "Yelp_A", "Reviews" ], [ "Yelp_B", "Users" ], [ "Yelp_B", "Friends" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ], [ [ "Yelp_B", "Yelpers" ] ] ], "Vertices": [ { "Label": "User", "PrimaryKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Yelpers" }, { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "( FROM    Yelp_B.Users U\n                       UNNEST  U.friends F\n                       SELECT  U.user_id, F AS friend )" }, { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Friends" }, { "Label": "MADE_BY", "DestinationLabel": "User", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "user_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null }, { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ] ], "Vertices": [ { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
new file mode 100644
index 0000000..9ad1dd7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
@@ -0,0 +1 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ] ], "Vertices": [ { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
new file mode 100644
index 0000000..3e2b0d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ], [ "Yelp_B", "Users" ], [ "Yelp_B", "Friends" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ], [ [ "Yelp_B", "Yelpers" ] ] ], "Vertices": [ { "Label": "User", "PrimaryKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Yelpers" }, { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                               SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                               SELECT  VALUE B )" } ], "Edges": [ { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "( FROM    Yelp_B.Users U\n                               UNNEST  U.friends F\n                               SELECT  U.user_id, F AS friend )" }, { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Friends" }, { "Label": "MADE_BY", "DestinationLabel": "User", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "user_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null }, { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index dc6ae3d..89ea5e3 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -324,6 +324,11 @@
         <expected-error>Cannot drop type test.subType being used by type test.superType_superlist</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="yelp-use-case">
+        <output-dir compare="Text">yelp-use-case</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="exception">
     <test-case FilePath="exception">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
new file mode 100644
index 0000000..ff47a77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataset that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP DATASET      GenericDataset;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
new file mode 100644
index 0000000..ff0f01e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+// Verify that we cannot drop a graph that doesn't exist.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP GRAPH        GraphThatDoesntExist1 IF EXISTS;
+DROP GRAPH        GraphThatDoesntExist2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
new file mode 100644
index 0000000..09693af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+// Verify that a function that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE FUNCTION   TestFunction () { SELECT VALUE { "a": 1, "b": 1 } };
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  PRIMARY KEY (a, b)
+                  SOURCE KEY (a)
+                  DESTINATION KEY (b)
+                  AS ( FROM TestFunction() T
+                       SELECT T.* );
+
+DROP FUNCTION     TestFunction();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
new file mode 100644
index 0000000..0cdec73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+// Verify that a synonym that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE SYNONYM    DatasetSynonym FOR GenericDataset;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS DatasetSynonym,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP SYNONYM      DatasetSynonym;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
new file mode 100644
index 0000000..844389f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataverse that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+DROP DATAVERSE    TestDataverse2 IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+CREATE DATAVERSE  TestDataverse2;
+
+USE               TestDataverse;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+USE               TestDataverse2;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+USE               TestDataverse;
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+USE               TestDataverse2;
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+USE               TestDataverse;
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS TestDataverse2.GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP DATAVERSE    TestDataverse2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
new file mode 100644
index 0000000..f6173aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataset variable as an element definition is a valid dataset.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS DatasetThatDoesNotExist,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
new file mode 100644
index 0000000..5191339
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+// Verify that a subquery as an element definition is a valid query.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  PRIMARY KEY (_id, _foreign_id)
+                  SOURCE KEY (_id)
+                  DESTINATION KEY (_foreign_id)
+                  AS ( FROM GenericDataset G,
+                            GenericDataset G2
+                       SELECT V );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
new file mode 100644
index 0000000..eb0fa47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+// Verify that vertices w/ the same label cannot have conflicting primary keys.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_other_id)
+                  AS ( FROM GenericDataset
+                       SELECT VALUE { "_other_id": _other_id } );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
new file mode 100644
index 0000000..7c5269e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+// Verify that edges do not reference a vertex label that does not exist.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex3)
+                  DESTINATION KEY (_foreign_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
new file mode 100644
index 0000000..fe00035
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+// Verify that we cannot create a graph in the same dataverse with the same name.
+
+DROP DATAVERSE    TestDataverse1 IF EXISTS;
+DROP DATAVERSE    TestDataverse2 IF EXISTS;
+CREATE DATAVERSE  TestDataverse1;
+CREATE DATAVERSE  TestDataverse2;
+
+USE               TestDataverse1;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+USE               TestDataverse2;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+USE               TestDataverse1;
+CREATE GRAPH      TestGraph IF NOT EXISTS AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset;
+
+USE               TestDataverse2;
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 27575fb..c8ccc68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -15716,4 +15716,21 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="graph-view">
+    <test-case FilePath="graph-view">
+      <compilation-unit name="error-handling">
+        <output-dir compare="Text">error-handling</output-dir>
+        <expected-error>Cannot drop dataset TestDataverse.GenericDataset being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop function TestDataverse.TestFunction() being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop synonym TestDataverse.DatasetSynonym being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop dataverse: dataset (or view) TestDataverse2.GenericDataset being used by graph TestDataverse.TestGraph</expected-error>
+        <expected-error>Cannot find dataset DatasetThatDoesNotExist in dataverse TestDataverse nor an alias with name DatasetThatDoesNotExist</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier V</expected-error>
+        <expected-error>Invalid graph vertex definition: Conflicting primary keys for vertices with label Vertex1</expected-error>
+        <expected-error>Invalid graph edge definition: Vertex label in edge with label EDGE_1 not found</expected-error>
+        <expected-error>A graph with this name TestGraph already exists (in line 68, at column 1)</expected-error>
+        <expected-error>Cannot find graph with name GraphThatDoesntExist2</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
index 79f13bf..0f8708a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
@@ -508,4 +508,12 @@
      */
     void downgradeDatasetLockToExclusiveModify(LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
+
+    /**
+     * Graph related metadata lock operations.
+     */
+    void acquireGraphReadLock(LockList locks, DataverseName dataverseName, String graphName) throws AlgebricksException;
+
+    void acquireGraphWriteLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index d49dffc..049a02c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -253,6 +253,10 @@
     INVALID_FOREIGN_KEY_DEFINITION_REF_PK_MISMATCH(1166),
     CANNOT_CHANGE_PRIMARY_KEY(1167),
     AMBIGUOUS_PROJECTION(1168),
+    UNKNOWN_GRAPH(1169),
+    GRAPH_EXISTS(1170),
+    INVALID_GRAPH_VERTEX_DEFINITION(1171),
+    INVALID_GRAPH_EDGE_DEFINITION(1172),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
new file mode 100644
index 0000000..9366771
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.common.graph;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+public class GraphElementIdentifier implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final GraphIdentifier graphIdentifier;
+    private final Kind elementKind;
+
+    // A vertex/edge schema instance can have the same label as another vertex/edge schema instance.
+    private final String labelName;
+    private final int labelId;
+
+    public GraphElementIdentifier(GraphIdentifier graphIdentifier, Kind elementKind, String labelName, int labelId) {
+        this.graphIdentifier = graphIdentifier;
+        this.elementKind = elementKind;
+        this.labelName = labelName;
+        this.labelId = labelId;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return graphIdentifier;
+    }
+
+    public Kind getElementKind() {
+        return elementKind;
+    }
+
+    public String getLabelName() {
+        return labelName;
+    }
+
+    public int getLabelId() {
+        return labelId;
+    }
+
+    @Override
+    public String toString() {
+        return graphIdentifier + "#" + labelName + " (" + elementKind + " " + labelId + ")";
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o instanceof GraphElementIdentifier) {
+            GraphElementIdentifier that = (GraphElementIdentifier) o;
+            return graphIdentifier.equals(that.graphIdentifier) && labelName.equals(that.labelName)
+                    && labelId == that.labelId;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(graphIdentifier, labelName, labelId);
+    }
+
+    public enum Kind {
+        VERTEX,
+        EDGE;
+
+        @Override
+        public String toString() {
+            switch (this) {
+                case EDGE:
+                    return "edge";
+                case VERTEX:
+                    return "vertex";
+                default:
+                    throw new IllegalStateException("Unknown graph element kind.");
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
new file mode 100644
index 0000000..877d554
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.common.graph;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.asterix.common.metadata.DataverseName;
+
+public class GraphIdentifier implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final DataverseName dataverseName;
+    private final String graphName;
+
+    public GraphIdentifier(DataverseName dataverseName, String graphName) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    @Override
+    public String toString() {
+        return dataverseName + "." + graphName;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o instanceof GraphIdentifier) {
+            GraphIdentifier that = (GraphIdentifier) o;
+            return dataverseName.equals(that.dataverseName) && graphName.equals(that.graphName);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(dataverseName, graphName);
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..bbc3213 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 import com.google.common.collect.ImmutableList;
@@ -58,6 +59,14 @@
     void insertDeleteUpsertBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException;
 
+    // Graph helpers
+
+    void createGraphBegin(IMetadataLockManager lockManager, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException;
+
+    void dropGraphBegin(IMetadataLockManager lockManager, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException;
+
     // Index helpers
 
     void createIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 136e169..125cfd9e1 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -255,6 +255,10 @@
 1166 = Invalid foreign key definition: foreign key does not match primary key of %1$s %2$s
 1167 = Cannot change primary key of %1$s %2$s
 1168 = Ambiguous projection in SELECT clause
+1169 = Cannot find graph with name %1$s
+1170 = A graph with this name %1$s already exists
+1171 = Invalid graph vertex definition: %1$s
+1172 = Invalid graph edge definition: %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
index bf1b40c..b1534a7 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
@@ -40,6 +40,7 @@
         SELECT_EXPRESSION,
         PRIMARY_EXPRESSION,
         CASE_EXPRESSION,
-        WINDOW_EXPRESSION
+        WINDOW_EXPRESSION,
+        GRAPH_ELEMENT_EXPRESSION
     }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
index 1645d76..50e216d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
@@ -23,8 +23,10 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.Warning;
@@ -42,6 +44,8 @@
 
     ViewDecl parseViewBody(DatasetFullyQualifiedName viewName) throws CompilationException;
 
+    GraphElementDecl parseGraphElementBody(GraphElementIdentifier identifier) throws CompilationException;
+
     /**
      * Gets the warnings generated during parsing
      */
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 6099cb6..bc9dac2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
@@ -65,4 +66,6 @@
     Query createFunctionAccessorQuery(FunctionDecl functionDecl);
 
     Query createViewAccessorQuery(ViewDecl viewDecl);
+
+    Query createGraphElementAccessorQuery(GraphElementDecl graphElementDecl);
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 2594cdd..5adadc5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -85,6 +85,7 @@
         CREATE_INDEX,
         CREATE_DATAVERSE,
         CREATE_VIEW,
+        CREATE_GRAPH,
         CREATE_FULL_TEXT_FILTER,
         CREATE_FULL_TEXT_CONFIG,
         INDEX_DROP,
@@ -107,6 +108,8 @@
         CREATE_SYNONYM,
         SYNONYM_DROP,
         VIEW_DROP,
+        GRAPH_DECL,
+        GRAPH_DROP,
         COMPACT,
         EXTERNAL_DATASET_REFRESH,
         SUBSCRIBE_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 95cccb0..f5da435 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -24,8 +24,10 @@
 import java.util.Map;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -37,16 +39,19 @@
     private final IWarningCollector warningCollector;
     private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
     private final Map<DatasetFullyQualifiedName, ViewDecl> declaredViews;
+    private final Map<GraphElementIdentifier, GraphElementDecl> declaredGraphElements;
     private final Counter varCounter;
     private int systemVarCounter = 1;
     private final Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
 
     public LangRewritingContext(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, IWarningCollector warningCollector, int varCounter) {
+            List<ViewDecl> declaredViews, List<GraphElementDecl> declaredGraphElements,
+            IWarningCollector warningCollector, int varCounter) {
         this.metadataProvider = metadataProvider;
         this.warningCollector = warningCollector;
         this.declaredFunctions = createMap(declaredFunctions, FunctionDecl::getSignature);
         this.declaredViews = createMap(declaredViews, ViewDecl::getViewName);
+        this.declaredGraphElements = createMap(declaredGraphElements, GraphElementDecl::getIdentifier);
         this.varCounter = new Counter(varCounter);
     }
 
@@ -102,6 +107,10 @@
         return declaredViews;
     }
 
+    public Map<GraphElementIdentifier, GraphElementDecl> getDeclaredGraphElements() {
+        return declaredGraphElements;
+    }
+
     private static <K, V> Map<K, V> createMap(List<V> values, java.util.function.Function<V, K> keyMapper) {
         if (values == null || values.isEmpty()) {
             return Collections.emptyMap();
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
new file mode 100644
index 0000000..448fbb3
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
@@ -0,0 +1,213 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class CreateGraphStatement extends AbstractStatement {
+    private final DataverseName dataverseName;
+    private final String graphName;
+    private final boolean replaceIfExists;
+    private final boolean ifNotExists;
+
+    private final List<VertexElement> vertexElements;
+    private final List<EdgeElement> edgeElements;
+
+    public CreateGraphStatement(DataverseName dataverseName, String graphName, boolean replaceIfExists,
+            boolean ifNotExists, List<VertexElement> vertexElements, List<EdgeElement> edgeElements) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+        this.replaceIfExists = replaceIfExists;
+        this.ifNotExists = ifNotExists;
+        this.vertexElements = vertexElements;
+        this.edgeElements = edgeElements;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CREATE_GRAPH;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    public boolean isReplaceIfExists() {
+        return replaceIfExists;
+    }
+
+    public boolean isIfNotExists() {
+        return ifNotExists;
+    }
+
+    public List<VertexElement> getVertexElements() {
+        return vertexElements;
+    }
+
+    public List<EdgeElement> getEdgeElements() {
+        return edgeElements;
+    }
+
+    public static class VertexElement {
+        private final List<Integer> primaryKeySourceIndicators;
+        private final List<List<String>> primaryKeyFields;
+        private final Expression expression;
+        private final String definition;
+        private final String label;
+
+        public VertexElement(String label, List<List<String>> primaryKeyFields,
+                List<Integer> primaryKeySourceIndicators, Expression expression, String definition) {
+            this.primaryKeySourceIndicators = primaryKeySourceIndicators;
+            this.primaryKeyFields = primaryKeyFields;
+            this.expression = expression;
+            this.definition = definition;
+            this.label = label;
+        }
+
+        public List<List<String>> getPrimaryKeyFields() {
+            return primaryKeyFields;
+        }
+
+        public List<Integer> getPrimaryKeySourceIndicators() {
+            return primaryKeySourceIndicators;
+        }
+
+        public Expression getExpression() {
+            return expression;
+        }
+
+        public String getDefinition() {
+            return definition;
+        }
+
+        public String getLabel() {
+            return label;
+        }
+
+        @Override
+        public String toString() {
+            return "(:" + label + ") AS " + definition;
+        }
+    }
+
+    public static class EdgeElement {
+        private final List<Integer> destinationKeySourceIndicators;
+        private final List<Integer> sourceKeySourceIndicators;
+        private final List<Integer> primaryKeySourceIndicators;
+
+        private final List<List<String>> destinationKeyFields;
+        private final List<List<String>> sourceKeyFields;
+        private final List<List<String>> primaryKeyFields;
+
+        private final String destinationLabel, edgeLabel, sourceLabel;
+        private final Expression expression;
+        private final String definition;
+
+        public EdgeElement(String edgeLabel, String destinationLabel, String sourceLabel,
+                List<List<String>> primaryKeyFields, List<Integer> primaryKeySourceIndicators,
+                List<List<String>> destinationKeyFields, List<Integer> destinationKeySourceIndicators,
+                List<List<String>> sourceKeyFields, List<Integer> sourceKeySourceIndicators, Expression expression,
+                String definition) {
+            this.destinationKeySourceIndicators = destinationKeySourceIndicators;
+            this.sourceKeySourceIndicators = sourceKeySourceIndicators;
+            this.primaryKeySourceIndicators = primaryKeySourceIndicators;
+            this.destinationKeyFields = destinationKeyFields;
+            this.sourceKeyFields = sourceKeyFields;
+            this.primaryKeyFields = primaryKeyFields;
+            this.destinationLabel = destinationLabel;
+            this.edgeLabel = edgeLabel;
+            this.sourceLabel = sourceLabel;
+            this.expression = expression;
+            this.definition = definition;
+        }
+
+        public List<Integer> getDestinationKeySourceIndicators() {
+            return destinationKeySourceIndicators;
+        }
+
+        public List<Integer> getSourceKeySourceIndicators() {
+            return sourceKeySourceIndicators;
+        }
+
+        public List<Integer> getPrimaryKeySourceIndicators() {
+            return primaryKeySourceIndicators;
+        }
+
+        public List<List<String>> getDestinationKeyFields() {
+            return destinationKeyFields;
+        }
+
+        public List<List<String>> getSourceKeyFields() {
+            return sourceKeyFields;
+        }
+
+        public List<List<String>> getPrimaryKeyFields() {
+            return primaryKeyFields;
+        }
+
+        public String getDestinationLabel() {
+            return destinationLabel;
+        }
+
+        public String getEdgeLabel() {
+            return edgeLabel;
+        }
+
+        public String getSourceLabel() {
+            return sourceLabel;
+        }
+
+        public Expression getExpression() {
+            return expression;
+        }
+
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            String edgeBodyPattern = "[:" + edgeLabel + "]";
+            String sourceNodePattern = "(:" + sourceLabel + ")";
+            String destinationNodePattern = "(:" + destinationLabel + ")";
+            String edgePattern = sourceNodePattern + "-" + edgeBodyPattern + "->" + destinationNodePattern;
+            return (definition == null) ? edgePattern : (edgePattern + " AS " + definition);
+        }
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
new file mode 100644
index 0000000..5411a85
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class GraphDropStatement extends AbstractStatement {
+    private final DataverseName dataverseName;
+    private final String graphName;
+    private final boolean ifExists;
+
+    public GraphDropStatement(DataverseName dataverseName, String graphName, boolean ifExists) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+        this.ifExists = ifExists;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    public boolean getIfExists() {
+        return ifExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_DROP;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
new file mode 100644
index 0000000..c1e22c2
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public final class GraphElementDecl extends AbstractStatement {
+    private final GraphElementIdentifier identifier;
+    private final Expression body;
+    private Expression normalizedBody;
+
+    public GraphElementDecl(GraphElementIdentifier identifier, Expression body) {
+        this.identifier = Objects.requireNonNull(identifier);
+        this.body = Objects.requireNonNull(body);
+    }
+
+    public GraphElementIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return identifier.getGraphIdentifier();
+    }
+
+    public Expression getBody() {
+        return body;
+    }
+
+    public Expression getNormalizedBody() {
+        return normalizedBody;
+    }
+
+    public void setNormalizedBody(Expression normalizedBody) {
+        this.normalizedBody = normalizedBody;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_DECL;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.QUERY;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
new file mode 100644
index 0000000..edb99ac
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.common.util;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
+import org.apache.asterix.metadata.entities.Graph;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public final class GraphUtil {
+    private GraphUtil() {
+    }
+
+    public static GraphElementDecl parsedStoredGraphElement(Graph.Element element, IParserFactory parserFactory,
+            IWarningCollector warningCollector, SourceLocation sourceLocation) throws CompilationException {
+        IParser parser = parserFactory.createParser(new StringReader(element.getDefinition()));
+        try {
+            GraphElementDecl graphElement = parser.parseGraphElementBody(element.getIdentifier());
+            if (warningCollector != null) {
+                parser.getWarnings(warningCollector);
+            }
+            return graphElement;
+
+        } catch (CompilationException e) {
+            GraphElementIdentifier.Kind kind = element.getIdentifier().getElementKind();
+            throw new CompilationException(
+                    kind.equals(GraphElementIdentifier.Kind.VERTEX) ? ErrorCode.INVALID_GRAPH_VERTEX_DEFINITION
+                            : ErrorCode.INVALID_GRAPH_EDGE_DEFINITION,
+                    e, sourceLocation, element.getIdentifier().toString(), e.getMessage());
+        }
+    }
+
+    public static List<List<Triple<DataverseName, String, String>>> getGraphDependencies(
+            List<GraphElementDecl> elementDecls, IQueryRewriter rewriter) throws CompilationException {
+        List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> synonymDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
+
+        for (GraphElementDecl elementDecl : elementDecls) {
+            Expression normalizedBody = elementDecl.getNormalizedBody();
+            if (normalizedBody == null) {
+                // We should have set the normalized body by calling {@link APIFramework#reWriteQuery} beforehand.
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, elementDecl.getSourceLocation(),
+                        elementDecl.getIdentifier().toString());
+            }
+
+            // Collect the list of used functions and used datasets.
+            ExpressionUtils.collectDependencies(normalizedBody, rewriter, datasetDependencies, synonymDependencies,
+                    functionDependencies);
+        }
+
+        return Graph.createDependencies(datasetDependencies, functionDependencies, synonymDependencies);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..4db2c3e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -74,6 +74,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -91,6 +92,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
@@ -987,6 +990,39 @@
         return null;
     }
 
+    @Override
+    public Void visit(GraphElementDecl ged, Integer arg) throws CompilationException {
+        // Note: this statement is internal.
+        return null;
+    }
+
+    @Override
+    public Void visit(CreateGraphStatement cgs, Integer step) throws CompilationException {
+        out.print(skip(step) + CREATE + generateOrReplace(cgs.isReplaceIfExists()) + " graph ");
+        out.print(generateIfNotExists(cgs.isIfNotExists()));
+        out.print(generateFullName(cgs.getDataverseName(), cgs.getGraphName()));
+        out.println(" as ");
+        for (CreateGraphStatement.VertexElement vertex : cgs.getVertexElements()) {
+            out.print(skip(step) + " vertex ");
+            out.println(vertex.toString());
+        }
+        for (CreateGraphStatement.EdgeElement edge : cgs.getEdgeElements()) {
+            out.print(skip(step) + " edge ");
+            out.println(edge.toString());
+        }
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(GraphDropStatement gds, Integer step) throws CompilationException {
+        out.print(skip(step) + "drop graph ");
+        out.print(generateFullName(gds.getDataverseName(), gds.getGraphName()));
+        out.print(generateIfExists(gds.getIfExists()));
+        out.println(SEMICOLON);
+        return null;
+    }
+
     protected void printConfiguration(Map<String, String> properties) {
         if (properties.size() > 0) {
             out.print("(");
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..4a766d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -50,6 +51,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.LibraryDropStatement;
@@ -309,4 +312,19 @@
     public R visit(ViewDecl vd, T arg) throws CompilationException {
         return null;
     }
+
+    @Override
+    public R visit(CreateGraphStatement cgs, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(GraphDropStatement gds, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(GraphElementDecl ged, T arg) throws CompilationException {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index c2b1311..ed33648 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -51,6 +51,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -67,6 +68,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.LibraryDropStatement;
@@ -216,4 +219,10 @@
     R visit(ViewDropStatement vds, T arg) throws CompilationException;
 
     R visit(ViewDecl vd, T arg) throws CompilationException;
+
+    R visit(CreateGraphStatement cgs, T arg) throws CompilationException;
+
+    R visit(GraphDropStatement gds, T arg) throws CompilationException;
+
+    R visit(GraphElementDecl ged, T arg) throws CompilationException;
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
new file mode 100644
index 0000000..c9a7f3e
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.sqlpp.expression;
+
+import java.util.Collections;
+
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class GraphElementExpression extends CallExpr {
+    public static final FunctionIdentifier GRAPH_ELEMENT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "graph-element", 5);
+
+    private final GraphElementIdentifier identifier;
+
+    public GraphElementExpression(GraphElementIdentifier identifier) {
+        super(new FunctionSignature(GRAPH_ELEMENT), Collections.emptyList(), null);
+        this.identifier = identifier;
+    }
+
+    public GraphElementIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return identifier.getGraphIdentifier();
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_ELEMENT_EXPRESSION;
+    }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index b28f4b9..195ca6a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -31,6 +31,8 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.AbstractExpression;
@@ -46,14 +48,17 @@
 import org.apache.asterix.lang.common.literal.MissingLiteral;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.ExpressionUtils;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.GraphUtil;
 import org.apache.asterix.lang.common.util.ViewUtil;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.lang.sqlpp.expression.GraphElementExpression;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
@@ -82,12 +87,12 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.ViewDetails;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.TypeUtil;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -328,11 +333,12 @@
     }
 
     protected void loadAndInlineUdfsAndViews() throws CompilationException {
-        Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> udfAndViewDecls =
-                loadUdfsAndViews(topStatement);
-        Map<FunctionSignature, FunctionDecl> udfs = udfAndViewDecls.first;
-        Map<DatasetFullyQualifiedName, ViewDecl> views = udfAndViewDecls.second;
-        if (udfs.isEmpty() && views.isEmpty()) {
+        Decls decls = loadDecls(topStatement);
+        Map<FunctionSignature, FunctionDecl> udfs = decls.udfs;
+        Map<DatasetFullyQualifiedName, ViewDecl> views = decls.views;
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements = decls.graphElements;
+
+        if (udfs.isEmpty() && views.isEmpty() && graphElements.isEmpty()) {
             // nothing to do
             return;
         }
@@ -389,10 +395,10 @@
         return SqlppVariableUtil.toUserDefinedName(paramVar.getValue());
     }
 
-    private Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> loadUdfsAndViews(
-            IReturningStatement topExpr) throws CompilationException {
+    private Decls loadDecls(IReturningStatement topExpr) throws CompilationException {
         Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
         Map<DatasetFullyQualifiedName, ViewDecl> views = new LinkedHashMap<>();
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements = new LinkedHashMap<>();
         Deque<AbstractCallExpression> workQueue = new ArrayDeque<>();
         SqlppGatherFunctionCallsVisitor callVisitor = new SqlppGatherFunctionCallsVisitor(workQueue);
         for (Expression expr : topExpr.getDirectlyEnclosedExpressions()) {
@@ -401,6 +407,15 @@
         AbstractCallExpression fnCall;
         while ((fnCall = workQueue.poll()) != null) {
             switch (fnCall.getKind()) {
+                case GRAPH_ELEMENT_EXPRESSION:
+                    GraphElementExpression graphElementCall = (GraphElementExpression) fnCall;
+                    GraphElementIdentifier identifier = graphElementCall.getIdentifier();
+                    if (!graphElements.containsKey(identifier)) {
+                        GraphElementDecl elementDecl = fetchGraphElementDecl(identifier, fnCall.getSourceLocation());
+                        graphElements.put(identifier, elementDecl);
+                        elementDecl.getNormalizedBody().accept(callVisitor, null);
+                    }
+                    break;
                 case CALL_EXPRESSION:
                     FunctionSignature fs = fnCall.getFunctionSignature();
                     DataverseName fsDataverse = fs.getDataverseName();
@@ -439,7 +454,7 @@
                             fnCall.getFunctionSignature().toString(false));
             }
         }
-        return new Pair<>(udfs, views);
+        return new Decls(udfs, views, graphElements);
     }
 
     private FunctionDecl fetchFunctionDecl(FunctionSignature fs, SourceLocation sourceLoc) throws CompilationException {
@@ -511,6 +526,37 @@
         return viewDecl;
     }
 
+    private GraphElementDecl fetchGraphElementDecl(GraphElementIdentifier identifier, SourceLocation sourceLocation)
+            throws CompilationException {
+        GraphElementDecl elementDecl = context.getDeclaredGraphElements().get(identifier);
+
+        // If we cannot find the graph in our context, search our metadata.
+        if (elementDecl == null) {
+            GraphIdentifier graphIdentifier = identifier.getGraphIdentifier();
+            Graph graph;
+            try {
+                graph = metadataProvider.findGraph(graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+            } catch (AlgebricksException e) {
+                throw new CompilationException(ErrorCode.UNKNOWN_GRAPH, e, sourceLocation,
+                        graphIdentifier.getGraphName());
+            }
+            Graph.Element graphElement = graph.getGraphSchema().getElement(identifier);
+            if (graphElement == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLocation,
+                        "Unable to resolve the graph element " + identifier);
+            }
+            elementDecl = GraphUtil.parsedStoredGraphElement(graphElement, parserFactory, context.getWarningCollector(),
+                    sourceLocation);
+        }
+
+        Expression normalizedBody = elementDecl.getNormalizedBody();
+        if (normalizedBody == null) {
+            normalizedBody = rewriteGraphElementBody(elementDecl);
+            elementDecl.setNormalizedBody(normalizedBody);
+        }
+        return elementDecl;
+    }
+
     private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
         FunctionSignature fs = fnDecl.getSignature();
         return rewriteFunctionOrViewBody(fs.getDataverseName(), fs, fnDecl.getFuncBody(), fnDecl.getParamList(),
@@ -534,6 +580,12 @@
         return rewrittenBodyExpr;
     }
 
+    private Expression rewriteGraphElementBody(GraphElementDecl elementDecl) throws CompilationException {
+        GraphIdentifier graphIdentifier = elementDecl.getGraphIdentifier();
+        return rewriteFunctionOrViewBody(graphIdentifier.getDataverseName(), graphIdentifier.getGraphName(),
+                elementDecl.getBody(), Collections.emptyList(), false, elementDecl.getSourceLocation());
+    }
+
     private Expression rewriteFunctionOrViewBody(DataverseName entityDataverseName, Object entityDisplayName,
             Expression bodyExpr, List<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls,
             SourceLocation sourceLoc) throws CompilationException {
@@ -583,6 +635,12 @@
     }
 
     @Override
+    public Query createGraphElementAccessorQuery(GraphElementDecl graphElementDecl) {
+        GraphElementExpression functionCall = new GraphElementExpression(graphElementDecl.getIdentifier());
+        return ExpressionUtils.createWrappedQuery(functionCall, graphElementDecl.getSourceLocation());
+    }
+
+    @Override
     public Query createViewAccessorQuery(ViewDecl viewDecl) {
         // dataverse_name.view_name
         DataverseName dataverseName = viewDecl.getViewName().getDataverseName();
@@ -605,4 +663,17 @@
         resultExpr.setSourceLocation(sourceLoc);
         return resultExpr;
     }
+
+    private static class Decls {
+        Map<FunctionSignature, FunctionDecl> udfs;
+        Map<DatasetFullyQualifiedName, ViewDecl> views;
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements;
+
+        public Decls(Map<FunctionSignature, FunctionDecl> udfs, Map<DatasetFullyQualifiedName, ViewDecl> views,
+                Map<GraphElementIdentifier, GraphElementDecl> graphElements) {
+            this.udfs = udfs;
+            this.views = views;
+            this.graphElements = graphElements;
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
index 38b66e2..e8a0185 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
@@ -52,9 +52,15 @@
 
     @Override
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
-        FunctionSignature fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(),
-                callExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, true,
-                context.getDeclaredFunctions(), allowNonStoredUdfCalls);
+        FunctionSignature fs;
+        if (callExpr.getKind().equals(Expression.Kind.GRAPH_ELEMENT_EXPRESSION)) {
+            // This is a marker function that we cannot resolve. Skip this.
+            fs = callExpr.getFunctionSignature();
+        } else {
+            fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(), callExpr.getSourceLocation(),
+                    context.getMetadataProvider(), callExprResolver, true, context.getDeclaredFunctions(),
+                    allowNonStoredUdfCalls);
+        }
         callExpr.setFunctionSignature(fs);
         return super.visit(callExpr, arg);
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index facdfa7..91fb309 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -74,6 +74,7 @@
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.base.AbstractClause;
@@ -133,6 +134,7 @@
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
@@ -149,6 +151,8 @@
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
@@ -256,6 +260,7 @@
     private static final String REPLACE = "REPLACE";
     private static final String RETURNS = "RETURNS";
     private static final String CONFIG = "CONFIG";
+    private static final String SOURCE = "SOURCE";
 
 
     private static final String INT_TYPE_NAME = "int";
@@ -454,6 +459,22 @@
         });
     }
 
+    @Override
+    public GraphElementDecl parseGraphElementBody(GraphElementIdentifier identifier) throws CompilationException {
+        return parseImpl(new ParseFunction<GraphElementDecl>() {
+            @Override
+            public GraphElementDecl parse() throws ParseException {
+                DataverseName dataverse = defaultDataverse;
+                defaultDataverse = identifier.getGraphIdentifier().getDataverseName();
+                createNewScope();
+                Expression elementBodyExpr = SQLPPParser.this.ViewBody();
+                removeCurrentScope();
+                defaultDataverse = dataverse;
+                return new GraphElementDecl(identifier, elementBodyExpr);
+            }
+        });
+    }
+
     private <T> T parseImpl(ParseFunction<T> parseFunction) throws CompilationException {
         warningCollector.clear();
         hintCollector.clear();
@@ -878,6 +899,7 @@
     | stmt = CreateFeedPolicyStatement(startToken)
     | stmt = CreateFullTextStatement(startToken)
     | stmt = CreateViewStatement(startToken, false)
+    | stmt = CreateGraphStatement(startToken, false)
   )
   {
     return stmt;
@@ -894,6 +916,7 @@
   (
     stmt = CreateFunctionStatement(startStmtToken, true)
     | stmt = CreateViewStatement(startStmtToken, true)
+    | stmt = CreateGraphStatement(startStmtToken, true)
   )
   {
     // check expected token here to make the grammar extension plugin happy
@@ -1521,11 +1544,11 @@
           }
         )*
         (
-          <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+          <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = KeyFields() <RIGHTPAREN>
           <NOT> <ENFORCED>
         )?
         (
-          <IDENTIFIER> { expectToken(FOREIGN); } <KEY> <LEFTPAREN> foreignKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+          <IDENTIFIER> { expectToken(FOREIGN); } <KEY> <LEFTPAREN> foreignKeyFields = KeyFields() <RIGHTPAREN>
           <IDENTIFIER> { expectToken(REFERENCES); } refNameComponents = QualifiedName()
           <NOT> <ENFORCED>
           {
@@ -2019,13 +2042,13 @@
   Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
 }
 {
-  <PRIMARY> <KEY> primaryKeyFields = PrimaryKeyFields()
+  <PRIMARY> <KEY> primaryKeyFields = KeyFields()
   {
     return primaryKeyFields;
   }
 }
 
-Pair<List<Integer>, List<List<String>>> PrimaryKeyFields() throws ParseException:
+Pair<List<Integer>, List<List<String>>> KeyFields() throws ParseException:
 {
   Pair<Integer, List<String>> tmp = null;
   List<Integer> keyFieldSourceIndicators = new ArrayList<Integer>();
@@ -2048,6 +2071,184 @@
     }
 }
 
+CreateGraphStatement CreateGraphStatement(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+  CreateGraphStatement stmt = null;
+}
+{
+  <GRAPH> stmt = GraphSpecification(startStmtToken, orReplace)
+  {
+    return stmt;
+  }
+}
+
+CreateGraphStatement GraphSpecification(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+  Pair<DataverseName, Identifier> nameComponents = null;
+  boolean ifNotExists = false;
+
+  List<CreateGraphStatement.VertexElement> vertexElements = new ArrayList<CreateGraphStatement.VertexElement>();
+  List<CreateGraphStatement.EdgeElement> edgeElements = new ArrayList<CreateGraphStatement.EdgeElement>();
+  CreateGraphStatement.VertexElement vertexElement = null;
+  CreateGraphStatement.EdgeElement edgeElement = null;
+}
+{
+  nameComponents = QualifiedName()
+  ifNotExists = IfNotExists()
+  {
+    if (orReplace && ifNotExists) {
+      throw new SqlppParseException(getSourceLocation(startStmtToken), "Unexpected IF NOT EXISTS");
+    }
+  }
+  <AS>
+  vertexElement = GraphVertexSpecification() { vertexElements.add(vertexElement); }
+  ( <COMMA>
+    (
+      ( vertexElement = GraphVertexSpecification() { vertexElements.add(vertexElement); } )
+      | ( edgeElement = GraphEdgeSpecification(startStmtToken) { edgeElements.add(edgeElement); } )
+    )
+  )*
+  {
+    CreateGraphStatement stmt = new CreateGraphStatement(nameComponents.first, nameComponents.second.getValue(),
+        orReplace, ifNotExists, vertexElements, edgeElements);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
+CreateGraphStatement.VertexElement GraphVertexSpecification() throws ParseException:
+{
+  Pair<List<Integer>, List<List<String>>> primaryKeyFields;
+  Token beginPos = null, endPos = null;
+  Expression vertexDefinitionExpr;
+  String vertexName;
+}
+{
+  <VERTEX>
+  vertexName = GraphVertexDefinitionPattern()
+  <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = KeyFields() <RIGHTPAREN>
+  <AS>
+  {
+    beginPos = token;
+    createNewScope();
+  }
+  (
+    vertexDefinitionExpr = ViewBody()
+    | <LEFTPAREN> vertexDefinitionExpr = ViewBody() <RIGHTPAREN>
+  )
+  {
+    endPos = token;
+    String vDef = extractFragment(beginPos.beginLine, beginPos.beginColumn + 1, endPos.endLine, endPos.endColumn + 1);
+    removeCurrentScope();
+    return new CreateGraphStatement.VertexElement(vertexName, primaryKeyFields.second, primaryKeyFields.first,
+      vertexDefinitionExpr, vDef);
+  }
+}
+
+String GraphVertexDefinitionPattern() throws ParseException:
+{
+  String vertexName;
+}
+{
+  <LEFTPAREN> <COLON> vertexName = Identifier() <RIGHTPAREN>
+  {
+    return vertexName;
+  }
+}
+
+CreateGraphStatement.EdgeElement GraphEdgeSpecification(Token startStmtToken) throws ParseException:
+{
+  Pair<Triple<String, String, String>, Boolean> edgeDefinitionPattern;
+  Pair<List<Integer>, List<List<String>>> keyFields;
+  Token beginPos = null, endPos = null;
+  Expression edgeDefinitionExpr = null;
+
+  List<Integer> destinationKeySourceIndicators = null;
+  List<Integer> sourceKeySourceIndicators = null;
+  List<Integer> primaryKeySourceIndicators = null;
+  List<List<String>> destinationKeyFields = null;
+  List<List<String>> sourceKeyFields = null;
+  List<List<String>> primaryKeyFields = null;
+}
+{
+  <EDGE>
+  edgeDefinitionPattern = GraphEdgeDefinitionPattern()
+  (
+    (
+      <PRIMARY> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        primaryKeyFields = keyFields.second;
+        primaryKeySourceIndicators = keyFields.first;
+      }
+      <IDENTIFIER> { expectToken(SOURCE); } <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        sourceKeyFields = keyFields.second;
+        sourceKeySourceIndicators = keyFields.first;
+      }
+      <DESTINATION> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        destinationKeyFields = keyFields.second;
+        destinationKeySourceIndicators = keyFields.first;
+      }
+      <AS>
+      {
+        beginPos = token;
+        createNewScope();
+      }
+      (
+        edgeDefinitionExpr = ViewBody()
+        | <LEFTPAREN> edgeDefinitionExpr = ViewBody() <RIGHTPAREN>
+      )
+    )
+    |
+    (
+      <DESTINATION> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        destinationKeyFields = keyFields.second;
+        destinationKeySourceIndicators = keyFields.first;
+      }
+    )
+  )
+  {
+    String destinationLabel, edgeLabel, sourceLabel;
+    if (edgeDefinitionPattern.second) { // isDirectedLeft
+      sourceLabel = edgeDefinitionPattern.first.third;
+      edgeLabel = edgeDefinitionPattern.first.second;
+      destinationLabel = edgeDefinitionPattern.first.first;
+    } else {
+      sourceLabel = edgeDefinitionPattern.first.first;
+      edgeLabel = edgeDefinitionPattern.first.second;
+      destinationLabel = edgeDefinitionPattern.first.third;
+    }
+
+    String eDef = null;
+    if (edgeDefinitionExpr != null) {
+      endPos = token;
+      eDef = extractFragment(beginPos.beginLine, beginPos.beginColumn + 1, endPos.endLine, endPos.endColumn + 1);
+      removeCurrentScope();
+    }
+
+    return new CreateGraphStatement.EdgeElement(edgeLabel, destinationLabel, sourceLabel, primaryKeyFields,
+        primaryKeySourceIndicators, destinationKeyFields, destinationKeySourceIndicators, sourceKeyFields,
+        sourceKeySourceIndicators, edgeDefinitionExpr, eDef);
+  }
+}
+
+Pair<Triple<String, String, String>, Boolean> GraphEdgeDefinitionPattern() throws ParseException:
+{
+  String leftVertexName, edgeName, rightVertexName;
+  boolean isDirectedLeft;
+}
+{
+  leftVertexName = GraphVertexDefinitionPattern()
+  ( <MINUS> <LEFTBRACKET> <COLON> edgeName = Identifier() <RIGHTBRACKET> <MINUS> <GT> { isDirectedLeft = false; }
+  | <LT> <MINUS> <LEFTBRACKET> <COLON> edgeName = Identifier() <RIGHTBRACKET> <MINUS> { isDirectedLeft = true; } )
+  rightVertexName = GraphVertexDefinitionPattern()
+  {
+    Triple<String, String, String> t = new Triple<String, String, String>(leftVertexName, edgeName, rightVertexName);
+    return new Pair<Triple<String, String, String>, Boolean>(t, isDirectedLeft);
+  }
+}
+
 Statement DropStatement() throws ParseException:
 {
   Token startToken = null;
@@ -2068,6 +2269,7 @@
     | stmt = DropSynonymStatement(startToken)
     | stmt = DropFullTextStatement(startToken)
     | stmt = DropViewStatement(startToken)
+    | stmt = DropGraphStatement(startToken)
   )
   {
     return stmt;
@@ -2382,6 +2584,30 @@
   }
 }
 
+GraphDropStatement DropGraphStatement(Token startStmtToken) throws ParseException:
+{
+   GraphDropStatement stmt = null;
+}
+{
+  <GRAPH> stmt = DropGraphSpecification(startStmtToken)
+  {
+    return stmt;
+  }
+}
+
+GraphDropStatement DropGraphSpecification(Token startStmtToken) throws ParseException:
+{
+  Pair<DataverseName,Identifier> pairId = null;
+  boolean ifExists = false;
+}
+{
+  pairId = QualifiedName() ifExists = IfExists()
+  {
+    GraphDropStatement stmt = new GraphDropStatement(pairId.first, pairId.second.getValue(), ifExists);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
 boolean IfExists() throws ParseException :
 {
 }
@@ -5305,10 +5531,12 @@
   | <DEFINITION : "definition">
   | <DELETE : "delete">
   | <DESC : "desc">
+  | <DESTINATION : "destination">
   | <DISCONNECT : "disconnect">
   | <DISTINCT : "distinct">
   | <DIV : "div">
   | <DROP : "drop">
+  | <EDGE: "edge">
   | <ELEMENT : "element">
   | <EXPLAIN : "explain">
   | <ELSE : "else">
@@ -5327,6 +5555,7 @@
   | <FULL : "full">
   | <FULLTEXT : "fulltext">
   | <FUNCTION : "function">
+  | <GRAPH : "graph">
   | <GROUP : "group">
   | <HAVING : "having">
   | <HINTS : "hints">
@@ -5397,6 +5626,7 @@
   | <USING : "using">
   | <VALUE : "value">
   | <VALUED : "valued">
+  | <VERTEX : "vertex">
   | <VIEW : "view">
   | <WHEN : "when">
   | <WHERE : "where">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index 6602666..63ccd10 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
@@ -59,6 +60,8 @@
     protected final Map<DataverseName, Dataverse> dataverses = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name.
     protected final Map<DataverseName, Map<String, Dataset>> datasets = new HashMap<>();
+    // Key is dataverse name. Key of value map is graph name.
+    protected final Map<DataverseName, Map<String, Graph>> graphs = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name. Key of value map of value map is index name.
     protected final Map<DataverseName, Map<String, Map<String, Index>>> indexes = new HashMap<>();
     // Key is dataverse name. Key of value map is datatype name.
@@ -120,24 +123,27 @@
                     synchronized (indexes) {
                         synchronized (datatypes) {
                             synchronized (functions) {
-                                synchronized (fullTextConfigs) {
-                                    synchronized (fullTextFilters) {
-                                        synchronized (adapters) {
-                                            synchronized (libraries) {
-                                                synchronized (compactionPolicies) {
-                                                    synchronized (synonyms) {
-                                                        dataverses.clear();
-                                                        nodeGroups.clear();
-                                                        datasets.clear();
-                                                        indexes.clear();
-                                                        datatypes.clear();
-                                                        functions.clear();
-                                                        fullTextConfigs.clear();
-                                                        fullTextFilters.clear();
-                                                        adapters.clear();
-                                                        libraries.clear();
-                                                        compactionPolicies.clear();
-                                                        synonyms.clear();
+                                synchronized (graphs) {
+                                    synchronized (fullTextConfigs) {
+                                        synchronized (fullTextFilters) {
+                                            synchronized (adapters) {
+                                                synchronized (libraries) {
+                                                    synchronized (compactionPolicies) {
+                                                        synchronized (synonyms) {
+                                                            dataverses.clear();
+                                                            nodeGroups.clear();
+                                                            datasets.clear();
+                                                            graphs.clear();
+                                                            indexes.clear();
+                                                            datatypes.clear();
+                                                            functions.clear();
+                                                            fullTextConfigs.clear();
+                                                            fullTextFilters.clear();
+                                                            adapters.clear();
+                                                            libraries.clear();
+                                                            compactionPolicies.clear();
+                                                            synonyms.clear();
+                                                        }
                                                     }
                                                 }
                                             }
@@ -192,6 +198,13 @@
         }
     }
 
+    public Graph addGraphIfNotExists(Graph graph) {
+        synchronized (graphs) {
+            Map<String, Graph> graphMap = graphs.computeIfAbsent(graph.getDataverseName(), k -> new HashMap<>());
+            return (!graphMap.containsKey(graph.getGraphName())) ? graphMap.put(graph.getGraphName(), graph) : null;
+        }
+    }
+
     public Index addIndexIfNotExists(Index index) {
         synchronized (indexes) {
             return addIndexIfNotExistsInternal(index);
@@ -250,35 +263,38 @@
                 synchronized (indexes) {
                     synchronized (datatypes) {
                         synchronized (functions) {
-                            synchronized (fullTextConfigs) {
-                                synchronized (fullTextFilters) {
-                                    synchronized (adapters) {
-                                        synchronized (libraries) {
-                                            synchronized (feeds) {
-                                                synchronized (compactionPolicies) {
-                                                    synchronized (synonyms) {
-                                                        datasets.remove(dataverse.getDataverseName());
-                                                        indexes.remove(dataverse.getDataverseName());
-                                                        datatypes.remove(dataverse.getDataverseName());
-                                                        adapters.remove(dataverse.getDataverseName());
-                                                        compactionPolicies.remove(dataverse.getDataverseName());
-                                                        List<FunctionSignature> markedFunctionsForRemoval =
-                                                                new ArrayList<>();
-                                                        for (FunctionSignature signature : functions.keySet()) {
-                                                            if (signature.getDataverseName()
-                                                                    .equals(dataverse.getDataverseName())) {
-                                                                markedFunctionsForRemoval.add(signature);
+                            synchronized (graphs) {
+                                synchronized (fullTextConfigs) {
+                                    synchronized (fullTextFilters) {
+                                        synchronized (adapters) {
+                                            synchronized (libraries) {
+                                                synchronized (feeds) {
+                                                    synchronized (compactionPolicies) {
+                                                        synchronized (synonyms) {
+                                                            datasets.remove(dataverse.getDataverseName());
+                                                            graphs.remove(dataverse.getDataverseName());
+                                                            indexes.remove(dataverse.getDataverseName());
+                                                            datatypes.remove(dataverse.getDataverseName());
+                                                            adapters.remove(dataverse.getDataverseName());
+                                                            compactionPolicies.remove(dataverse.getDataverseName());
+                                                            List<FunctionSignature> markedFunctionsForRemoval =
+                                                                    new ArrayList<>();
+                                                            for (FunctionSignature signature : functions.keySet()) {
+                                                                if (signature.getDataverseName()
+                                                                        .equals(dataverse.getDataverseName())) {
+                                                                    markedFunctionsForRemoval.add(signature);
+                                                                }
                                                             }
+                                                            for (FunctionSignature signature : markedFunctionsForRemoval) {
+                                                                functions.remove(signature);
+                                                            }
+                                                            fullTextConfigs.remove(dataverse.getDataverseName());
+                                                            fullTextFilters.remove(dataverse.getDataverseName());
+                                                            libraries.remove(dataverse.getDataverseName());
+                                                            feeds.remove(dataverse.getDataverseName());
+                                                            synonyms.remove(dataverse.getDataverseName());
+                                                            return dataverses.remove(dataverse.getDataverseName());
                                                         }
-                                                        for (FunctionSignature signature : markedFunctionsForRemoval) {
-                                                            functions.remove(signature);
-                                                        }
-                                                        fullTextConfigs.remove(dataverse.getDataverseName());
-                                                        fullTextFilters.remove(dataverse.getDataverseName());
-                                                        libraries.remove(dataverse.getDataverseName());
-                                                        feeds.remove(dataverse.getDataverseName());
-                                                        synonyms.remove(dataverse.getDataverseName());
-                                                        return dataverses.remove(dataverse.getDataverseName());
                                                     }
                                                 }
                                             }
@@ -313,6 +329,16 @@
         }
     }
 
+    public Graph dropGraph(Graph graph) {
+        synchronized (graphs) {
+            Map<String, Graph> graphMap = graphs.get(graph.getDataverseName());
+            if (graphMap == null) {
+                return null;
+            }
+            return graphMap.remove(graph.getGraphName());
+        }
+    }
+
     public Index dropIndex(Index index) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
@@ -360,6 +386,16 @@
         }
     }
 
+    public Graph getGraph(DataverseName dataverseName, String graphName) {
+        synchronized (graphs) {
+            Map<String, Graph> g = graphs.get(dataverseName);
+            if (g == null) {
+                return null;
+            }
+            return g.get(graphName);
+        }
+    }
+
     public Index getIndex(DataverseName dataverseName, String datasetName, String indexName) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 0ab5c7b..cc3fa7e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -52,6 +52,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -1092,6 +1093,85 @@
         }
     }
 
+    @Override
+    public void addGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException {
+        try {
+            metadataNode.addGraph(ctx.getTxnId(), graph);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.addGraph(graph);
+    }
+
+    @Override
+    public void updateGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException {
+        try {
+            metadataNode.updateGraph(ctx.getTxnId(), graph);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.dropGraph(graph.getDataverseName(), graph.getGraphName());
+        ctx.addGraph(graph);
+    }
+
+    @Override
+    public void dropGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        try {
+            metadataNode.dropGraph(ctx.getTxnId(), dataverseName, graphName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.dropGraph(dataverseName, graphName);
+    }
+
+    @Override
+    public Graph getGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        Graph graph = ctx.getGraph(dataverseName, graphName);
+        if (graph != null) {
+            // This graph is still uncommitted. Do not add this to the cache.
+            return graph;
+        }
+        if (ctx.graphIsDropped(dataverseName, graphName)) {
+            // Graph has been dropped by this transaction, but could still be in the cache.
+            return null;
+        }
+
+        graph = cache.getGraph(dataverseName, graphName);
+        if (graph != null) {
+            // Graph is already in the cache. Do not add it again.
+            return graph;
+        }
+
+        try {
+            graph = metadataNode.getGraph(ctx.getTxnId(), dataverseName, graphName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+
+        if (graph != null) {
+            // Graph has been fetched from the metadata node. Add this to the cache when the transaction commits.
+            ctx.addGraph(graph);
+        }
+        return graph;
+    }
+
+    @Override
+    public List<Graph> getDataverseGraphs(MetadataTransactionContext ctx, DataverseName dataverseName)
+            throws AlgebricksException {
+        try {
+            return metadataNode.getDataverseGraphs(ctx.getTxnId(), dataverseName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+    }
+
     // TODO: Optimize <-- use keys instead of object -->
     @Override
     public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 87f5129..807c223 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -73,6 +73,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Library;
@@ -92,6 +93,7 @@
 import org.apache.asterix.metadata.entitytupletranslators.FullTextConfigMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FullTextFilterMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.GraphTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
@@ -661,6 +663,12 @@
         try {
             confirmDataverseCanBeDeleted(txnId, dataverseName);
 
+            // Drop all graphs in this dataverse.
+            // Graphs may depend on functions and datasets.
+            for (Graph graph : getDataverseGraphs(txnId, dataverseName)) {
+                dropGraph(txnId, dataverseName, graph.getGraphName());
+            }
+
             // Drop all feeds and connections in this dataverse.
             // Feeds may depend on datatypes and adapters
             List<Feed> dataverseFeeds = getDataverseFeeds(txnId, dataverseName);
@@ -765,7 +773,8 @@
                 || !getDataverseFeeds(txnId, dataverseName).isEmpty()
                 || !getDataverseSynonyms(txnId, dataverseName).isEmpty()
                 || !getDataverseFullTextConfigs(txnId, dataverseName).isEmpty()
-                || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty();
+                || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty()
+                || !getDataverseGraphs(txnId, dataverseName).isEmpty();
     }
 
     @Override
@@ -1131,6 +1140,19 @@
         }
     }
 
+    public List<Graph> getAllGraphs(TxnId txnId) throws AlgebricksException {
+        try {
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            List<Graph> results = new ArrayList<>();
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, null, valueExtractor, results);
+            return results;
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     private void confirmDataverseCanBeDeleted(TxnId txnId, DataverseName dataverseName) throws AlgebricksException {
         // If a dataset from a DIFFERENT dataverse
         // uses a type from this dataverse
@@ -1211,11 +1233,33 @@
                 }
             }
         }
+
+        // If a graph from a DIFFERENT dataverse uses a graph from this dataverse, throw an error.
+        for (Graph graph : getAllGraphs(txnId)) {
+            if (graph.getDataverseName().equals(dataverseName)) {
+                continue;
+            }
+            List<DependencyKind> dependenciesSchema = Graph.DEPENDENCIES_SCHEMA;
+            List<List<Triple<DataverseName, String, String>>> dependencies = graph.getDependencies();
+            for (int i = 0, n = dependencies.size(); i < n; i++) {
+                for (Triple<DataverseName, String, String> dependency : dependencies.get(i)) {
+                    if (dependency.first.equals(dataverseName)) {
+                        DependencyKind dependencyKind = dependenciesSchema.get(i);
+                        throw new AsterixException(
+                                org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS,
+                                dependencyKind, dependencyKind.getDependencyDisplayName(dependency), "graph",
+                                MetadataUtil.getFullyQualifiedDisplayName(graph.getDataverseName(),
+                                        graph.getGraphName()));
+                    }
+                }
+            }
+        }
     }
 
     private void confirmFunctionCanBeDeleted(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
         confirmFunctionIsUnusedByViews(txnId, signature);
         confirmFunctionIsUnusedByFunctions(txnId, signature);
+        confirmFunctionIsUnusedByGraphs(txnId, signature);
 
         // if any other feed connection uses this function, throw an error
         List<FeedConnection> feedConnections = getAllFeedConnections(txnId);
@@ -1229,6 +1273,11 @@
         }
     }
 
+    private void confirmFunctionIsUnusedByGraphs(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
+                signature.getName(), Integer.toString(signature.getArity()));
+    }
+
     private void confirmFunctionIsUnusedByViews(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
         confirmObjectIsUnusedByViews(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
                 signature.getName(), Integer.toString(signature.getArity()));
@@ -1350,6 +1399,7 @@
             String datasetName) throws AlgebricksException {
         confirmDatasetIsUnusedByFunctions(txnId, datasetTypeDisplayName, dataverseName, datasetName);
         confirmDatasetIsUnusedByViews(txnId, datasetTypeDisplayName, dataverseName, datasetName);
+        confirmDatasetIsUnusedByGraphs(txnId, datasetTypeDisplayName, dataverseName, datasetName);
     }
 
     private void confirmDatasetIsUnusedByFunctions(TxnId txnId, String datasetKindDisplayName,
@@ -1364,6 +1414,12 @@
                 null);
     }
 
+    private void confirmDatasetIsUnusedByGraphs(TxnId txnId, String datasetKindDisplayName, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, datasetKindDisplayName, DependencyKind.DATASET, dataverseName, datasetName,
+                null);
+    }
+
     private void confirmLibraryCanBeDeleted(TxnId txnId, DataverseName dataverseName, String libraryName)
             throws AlgebricksException {
         confirmLibraryIsUnusedByFunctions(txnId, dataverseName, libraryName);
@@ -1473,6 +1529,32 @@
         }
     }
 
+    private void confirmObjectIsUnusedByGraphs(TxnId txnId, String objectKindDisplayName, DependencyKind dependencyKind,
+            DataverseName dataverseName, String objectName, String objectArg) throws AlgebricksException {
+        int graphDependencyIdx = Graph.DEPENDENCIES_SCHEMA.indexOf(dependencyKind);
+        if (graphDependencyIdx < 0) {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+        }
+        for (Graph graph : getAllGraphs(txnId)) {
+            List<List<Triple<DataverseName, String, String>>> graphDependencies = graph.getDependencies();
+            if (graphDependencyIdx < graphDependencies.size()) {
+                List<Triple<DataverseName, String, String>> graphObjectDependencies =
+                        graphDependencies.get(graphDependencyIdx);
+                if (graphObjectDependencies != null) {
+                    for (Triple<DataverseName, String, String> dependency : graphObjectDependencies) {
+                        if (dependency.first.equals(dataverseName) && dependency.second.equals(objectName)
+                                && (objectArg == null || objectArg.equals(dependency.third))) {
+                            throw new AsterixException(
+                                    org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS,
+                                    objectKindDisplayName, dependencyKind.getDependencyDisplayName(dependency), "graph",
+                                    graph.getGraphName());
+                        }
+                    }
+                }
+            }
+        }
+    }
+
     private List<String> getNestedComplexDatatypeNamesForThisDatatype(TxnId txnId, DataverseName dataverseName,
             String datatypeName) throws AlgebricksException {
         // Return all field types that aren't builtin types
@@ -2397,6 +2479,7 @@
             throws AlgebricksException {
         confirmSynonymIsUnusedByFunctions(txnId, dataverseName, synonymName);
         confirmSynonymIsUnusedByViews(txnId, dataverseName, synonymName);
+        confirmSynonymIsUnusedByGraphs(txnId, dataverseName, synonymName);
     }
 
     private void confirmSynonymIsUnusedByFunctions(TxnId txnId, DataverseName dataverseName, String synonymName)
@@ -2409,6 +2492,11 @@
         confirmObjectIsUnusedByViews(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
     }
 
+    private void confirmSynonymIsUnusedByGraphs(TxnId txnId, DataverseName dataverseName, String synonymName)
+            throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
+    }
+
     @Override
     public Synonym getSynonym(TxnId txnId, DataverseName dataverseName, String synonymName) throws AlgebricksException {
         try {
@@ -2441,6 +2529,76 @@
     }
 
     @Override
+    public void addGraph(TxnId txnId, Graph graph) throws AlgebricksException {
+        try {
+            GraphTupleTranslator tupleWriter = tupleTranslatorProvider.getGraphTupleTranslator(true);
+            ITupleReference graphTuple = tupleWriter.getTupleFromMetadataEntity(graph);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, graphTuple);
+
+        } catch (HyracksDataException e) {
+            if (e.matches(ErrorCode.DUPLICATE_KEY)) {
+                throw new AsterixException(org.apache.asterix.common.exceptions.ErrorCode.GRAPH_EXISTS, e,
+                        graph.getGraphName());
+            } else {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+
+    @Override
+    public Graph getGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException {
+        List<Graph> results = new ArrayList<>();
+
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, graphName);
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey, valueExtractor, results);
+            if (results.isEmpty()) {
+                return null;
+            }
+            return results.get(0);
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
+    public void dropGraph(TxnId txnId, DataverseName dataverseName, String graphName) throws AlgebricksException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, graphName);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, tuple);
+
+        } catch (HyracksDataException e) {
+            if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                throw new AsterixException(org.apache.asterix.common.exceptions.ErrorCode.UNKNOWN_GRAPH, e, graphName);
+
+            } else {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+
+    @Override
+    public List<Graph> getDataverseGraphs(TxnId txnId, DataverseName dataverseName) throws AlgebricksException {
+        List<Graph> results = new ArrayList<>();
+
+        try {
+            ITupleReference searchKey = createTuple(dataverseName);
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey, valueExtractor, results);
+            return results;
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
     public void updateDataset(TxnId txnId, Dataset dataset) throws AlgebricksException {
         try {
             // This method will delete previous entry of the dataset and insert the new one
@@ -2482,6 +2640,12 @@
         addDatatype(txnId, datatype);
     }
 
+    @Override
+    public void updateGraph(TxnId txnId, Graph graph) throws AlgebricksException {
+        dropGraph(txnId, graph.getDataverseName(), graph.getGraphName());
+        addGraph(txnId, graph);
+    }
+
     public ITxnIdFactory getTxnIdFactory() {
         return txnIdFactory;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 48fb450..ad2e7ea 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -20,9 +20,11 @@
 package org.apache.asterix.metadata;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
@@ -37,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
@@ -96,6 +99,11 @@
         logAndApply(new MetadataLogicalOperation(dataset, true));
     }
 
+    public void addGraph(Graph graph) {
+        droppedCache.dropGraph(graph);
+        logAndApply(new MetadataLogicalOperation(graph, true));
+    }
+
     public void addIndex(Index index) {
         droppedCache.dropIndex(index);
         logAndApply(new MetadataLogicalOperation(index, true));
@@ -143,6 +151,12 @@
         logAndApply(new MetadataLogicalOperation(dataset, false));
     }
 
+    public void dropGraph(DataverseName dataverseName, String graphName) {
+        Graph graph = new Graph(new GraphIdentifier(dataverseName, graphName), null, Collections.emptyList());
+        droppedCache.addGraphIfNotExists(graph);
+        logAndApply(new MetadataLogicalOperation(graph, false));
+    }
+
     public void dropIndex(DataverseName dataverseName, String datasetName, String indexName) {
         Index index =
                 new Index(dataverseName, datasetName, indexName, null, null, false, false, MetadataUtil.PENDING_NO_OP);
@@ -236,6 +250,13 @@
         return droppedCache.getDataset(dataverseName, datasetName) != null;
     }
 
+    public boolean graphIsDropped(DataverseName dataverseName, String graphName) {
+        if (droppedCache.getDataverse(dataverseName) != null) {
+            return true;
+        }
+        return droppedCache.getGraph(dataverseName, graphName) != null;
+    }
+
     public boolean indexIsDropped(DataverseName dataverseName, String datasetName, String indexName) {
         if (droppedCache.getDataverse(dataverseName) != null) {
             return true;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 5ff8a03..182cb62 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -871,6 +872,22 @@
     void rebindMetadataNode();
 
     /**
+     * Graph related metadata operations.
+     */
+    void addGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException;
+
+    void updateGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException;
+
+    void dropGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
+
+    Graph getGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
+
+    List<Graph> getDataverseGraphs(MetadataTransactionContext ctx, DataverseName dataverseName)
+            throws AlgebricksException;
+
+    /**
      * Feed Connection Related Metadata operations
      */
     void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection) throws AlgebricksException;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 561a4fa..f04f03b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -969,6 +970,19 @@
     <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException;
 
+    void addGraph(TxnId txnId, Graph graph) throws AlgebricksException, RemoteException;
+
+    void updateGraph(TxnId txnId, Graph graph) throws AlgebricksException, RemoteException;
+
+    Graph getGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException;
+
+    void dropGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException;
+
+    List<Graph> getDataverseGraphs(TxnId txnId, DataverseName dataverseName)
+            throws AlgebricksException, RemoteException;
+
     void addFeedConnection(TxnId txnId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
 
     FeedConnection getFeedConnection(TxnId txnId, DataverseName dataverseName, String feedName, String datasetName)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index fd0d30b..52ecb21 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -127,7 +127,7 @@
                     MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
                     MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
                     MetadataPrimaryIndexes.SYNONYM_DATASET, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET,
-                    MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET };
+                    MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, MetadataPrimaryIndexes.GRAPH_DATASET };
 
     private MetadataBootstrap() {
     }
@@ -178,6 +178,7 @@
                 insertNewCompactionPoliciesIfNotExist(mdTxnCtx);
                 insertSynonymEntitiesIfNotExist(mdTxnCtx);
                 insertFullTextConfigAndFilterIfNotExist(mdTxnCtx);
+                insertGraphEntitiesIfNotExists(mdTxnCtx);
             }
             // #. initialize datasetIdFactory
             MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
@@ -305,6 +306,22 @@
         }
     }
 
+    private static void insertGraphEntitiesIfNotExists(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+        IAType[] graphEntityTypes = new IAType[] { MetadataPrimaryIndexes.GRAPH_DATASET.getPayloadRecordType(),
+                MetadataRecordTypes.VERTICES_RECORDTYPE, MetadataRecordTypes.EDGES_RECORDTYPE };
+        for (IAType graphEntityType : graphEntityTypes) {
+            if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                    graphEntityType.getTypeName()) == null) {
+                MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+                        graphEntityType.getTypeName(), graphEntityType, false));
+            }
+        }
+        if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                MetadataConstants.GRAPH_DATASET_NAME) == null) {
+            insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.GRAPH_DATASET });
+        }
+    }
+
     private static void insertSynonymEntitiesIfNotExist(MetadataTransactionContext mdTxnCtx)
             throws AlgebricksException {
         IAType synonymDatasetRecordType = MetadataPrimaryIndexes.SYNONYM_DATASET.getPayloadRecordType();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index a191e43..344b423 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -65,6 +65,8 @@
             new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME, 16, 16);
     public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_FILTER =
             new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME, 17, 17);
+    public static final MetadataIndexImmutableProperties PROPERTIES_GRAPH =
+            new MetadataIndexImmutableProperties(MetadataConstants.GRAPH_DATASET_NAME, 18, 18);
 
     public static final IMetadataIndex DATAVERSE_DATASET =
             new MetadataIndex(PROPERTIES_DATAVERSE, 2, new IAType[] { BuiltinType.ASTRING },
@@ -157,6 +159,12 @@
                             Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_FILTER_NAME)),
                     0, MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE, true, new int[] { 0, 1 });
 
+    public static final IMetadataIndex GRAPH_DATASET =
+            new MetadataIndex(PROPERTIES_GRAPH, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+                    Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+                            Arrays.asList(MetadataRecordTypes.FIELD_NAME_GRAPH_NAME)),
+                    0, MetadataRecordTypes.GRAPH_RECORDTYPE, true, new int[] { 0, 1 });
+
     private MetadataPrimaryIndexes() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 48d4fc5..862fc45 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -53,6 +53,9 @@
     public static final String FIELD_NAME_DEPENDENCIES = "Dependencies";
     public static final String FIELD_NAME_DERIVED = "Derived";
     public static final String FIELD_NAME_DESCRIPTION = "Description";
+    public static final String FIELD_NAME_DESTINATION_KEY = "DestinationKey";
+    public static final String FIELD_NAME_DESTINATION_LABEL = "DestinationLabel";
+    public static final String FIELD_NAME_EDGES = "Edges";
     public static final String FIELD_NAME_EXTERNAL_DETAILS = "ExternalDetails";
     public static final String FIELD_NAME_FEED_NAME = "FeedName";
     public static final String FIELD_NAME_FEED_TYPE = "FeedType";
@@ -66,6 +69,7 @@
     public static final String FIELD_NAME_FILE_STRUCTURE = "FileStructure";
     public static final String FIELD_NAME_FOREIGN_KEY = "ForeignKey";
     public static final String FIELD_NAME_FOREIGN_KEYS = "ForeignKeys";
+    public static final String FIELD_NAME_GRAPH_NAME = "GraphName";
     public static final String FIELD_NAME_GROUP_NAME = "GroupName";
     public static final String FIELD_NAME_HINTS = "Hints";
     public static final String FIELD_NAME_INDEX_NAME = "IndexName";
@@ -79,6 +83,7 @@
     public static final String FIELD_NAME_IS_PRIMARY = "IsPrimary";
     public static final String FIELD_NAME_KIND = "Kind";
     public static final String FIELD_NAME_LANGUAGE = "Language";
+    public static final String FIELD_NAME_LABEL = "Label";
     public static final String FIELD_NAME_HASH = "MD5Hash";
     public static final String FIELD_NAME_LIBRARY_DATAVERSE_NAME = "LibraryDataverseName";
     public static final String FIELD_NAME_LIBRARY_NAME = "LibraryName";
@@ -107,6 +112,8 @@
     public static final String FIELD_NAME_RETURN_TYPE_DATAVERSE_NAME = "ReturnTypeDataverseName";
     public static final String FIELD_NAME_SEARCH_KEY = "SearchKey";
     public static final String FIELD_NAME_STATUS = "Status";
+    public static final String FIELD_NAME_SOURCE_KEY = "SourceKey";
+    public static final String FIELD_NAME_SOURCE_LABEL = "SourceLabel";
     public static final String FIELD_NAME_SYNONYM_NAME = "SynonymName";
     public static final String FIELD_NAME_TAG = "Tag";
     public static final String FIELD_NAME_TIMESTAMP = "Timestamp";
@@ -114,6 +121,7 @@
     public static final String FIELD_NAME_TYPE = "Type";
     public static final String FIELD_NAME_UNORDERED_LIST = "UnorderedList";
     public static final String FIELD_NAME_VALUE = "Value";
+    public static final String FIELD_NAME_VERTICES = "Vertices";
     public static final String FIELD_NAME_VIEW_DETAILS = "ViewDetails";
     public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
     public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
@@ -560,6 +568,49 @@
                     FIELD_NAME_FULL_TEXT_FILTER_TYPE },
             new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, true);
 
+    //-------------------------------------- Graph ---------------------------------------//
+    public static final String RECORD_NAME_GRAPH = "GraphRecordType";
+
+    public static final int GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX = 0;
+    public static final int GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX = 1;
+    public static final int GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX = 2;
+    public static final ARecordType VERTICES_RECORDTYPE = createRecordType("VerticesRecordType",
+            new String[] { FIELD_NAME_LABEL, FIELD_NAME_PRIMARY_KEY, FIELD_NAME_DEFINITION },
+            new IAType[] { BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null), BuiltinType.ASTRING },
+            true);
+
+    public static final int GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX = 0;
+    public static final int GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX = 1;
+    public static final int GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX = 2;
+    public static final int GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX = 3;
+    public static final int GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX = 4;
+    public static final int GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX = 5;
+    public static final int GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX = 6;
+    public static final ARecordType EDGES_RECORDTYPE = createRecordType("EdgesRecordType",
+            new String[] { FIELD_NAME_LABEL, FIELD_NAME_DESTINATION_LABEL, FIELD_NAME_SOURCE_LABEL,
+                    FIELD_NAME_PRIMARY_KEY, FIELD_NAME_DESTINATION_KEY, FIELD_NAME_SOURCE_KEY, FIELD_NAME_DEFINITION },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    AUnionType.createNullableType(BuiltinType.ASTRING, null) },
+            true);
+
+    public static final int GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
+    public static final int GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX = 1;
+    public static final int GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX = 2;
+    public static final int GRAPH_ARECORD_VERTICES_FIELD_INDEX = 3;
+    public static final int GRAPH_ARECORD_EDGES_FIELD_INDEX = 4;
+    public static final ARecordType GRAPH_RECORDTYPE = createRecordType(RECORD_NAME_GRAPH,
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_GRAPH_NAME, FIELD_NAME_DEPENDENCIES,
+                    FIELD_NAME_VERTICES, FIELD_NAME_EDGES },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                            null),
+                    new AOrderedListType(VERTICES_RECORDTYPE, null), new AOrderedListType(EDGES_RECORDTYPE, null) },
+            true);
+
     // private members
     private MetadataRecordTypes() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index ee629b1..5353175 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.entities.Synonym;
@@ -163,6 +164,11 @@
         return MetadataManager.INSTANCE.getSynonym(mdTxnCtx, dataverseName, synonymName);
     }
 
+    public static Graph findGraph(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        return MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+    }
+
     public static FullTextConfigMetadataEntity findFullTextConfigDescriptor(MetadataTransactionContext mdTxnCtx,
             DataverseName dataverseName, String ftConfigName) throws AlgebricksException {
         // If the config name is null, then the default config will be returned
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 011d862..7ba5bfa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -88,6 +88,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
@@ -471,6 +472,10 @@
         return MetadataManagerUtil.findFullTextFilterDescriptor(mdTxnCtx, dataverseName, ftFilterName);
     }
 
+    public Graph findGraph(DataverseName dataverseName, String graphName) throws AlgebricksException {
+        return MetadataManagerUtil.findGraph(mdTxnCtx, dataverseName, graphName);
+    }
+
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return BuiltinFunctions.getBuiltinFunctionInfo(fid);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
new file mode 100644
index 0000000..7103cf5
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
@@ -0,0 +1,347 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+
+/**
+ * Metadata describing a graph view, composed of vertices and edges.
+ */
+public class Graph implements IMetadataEntity<Graph> {
+    private static final long serialVersionUID = 1L;
+
+    private final GraphIdentifier identifier;
+    private final Schema graphSchema;
+    private final List<List<Triple<DataverseName, String, String>>> dependencies;
+
+    // Vertices and edges from a graph have dependencies on the datasets, functions, and synonyms used in their bodies.
+    public final static List<DependencyKind> DEPENDENCIES_SCHEMA =
+            Arrays.asList(DependencyKind.DATASET, DependencyKind.FUNCTION, DependencyKind.SYNONYM);
+
+    public Graph(GraphIdentifier identifier, Schema graphSchema,
+            List<List<Triple<DataverseName, String, String>>> dependencies) {
+        this.identifier = Objects.requireNonNull(identifier);
+        this.dependencies = dependencies;
+        this.graphSchema = graphSchema;
+    }
+
+    @Override
+    public Graph addToCache(MetadataCache cache) {
+        return cache.addGraphIfNotExists(this);
+    }
+
+    @Override
+    public Graph dropFromCache(MetadataCache cache) {
+        return cache.dropGraph(this);
+    }
+
+    public DataverseName getDataverseName() {
+        return identifier.getDataverseName();
+    }
+
+    public String getGraphName() {
+        return identifier.getGraphName();
+    }
+
+    public GraphIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public Schema getGraphSchema() {
+        return graphSchema;
+    }
+
+    public List<List<Triple<DataverseName, String, String>>> getDependencies() {
+        return dependencies;
+    }
+
+    public static List<List<Triple<DataverseName, String, String>>> createDependencies(
+            List<Triple<DataverseName, String, String>> datasetDependencies,
+            List<Triple<DataverseName, String, String>> functionDependencies,
+            List<Triple<DataverseName, String, String>> synonymDependencies) {
+        List<List<Triple<DataverseName, String, String>>> depList = new ArrayList<>(DEPENDENCIES_SCHEMA.size());
+        depList.add(datasetDependencies);
+        depList.add(functionDependencies);
+        if (!synonymDependencies.isEmpty()) {
+            depList.add(synonymDependencies);
+        }
+        return depList;
+    }
+
+    public static class Schema {
+        private final Map<GraphElementIdentifier, Element> elementMap = new HashMap<>();
+
+        // The element map above is composed of the vertices and edges below.
+        private final List<Vertex> vertexList = new ArrayList<>();
+        private final List<Edge> edgeList = new ArrayList<>();
+
+        public List<Vertex> getVertices() {
+            return vertexList;
+        }
+
+        public List<Edge> getEdges() {
+            return edgeList;
+        }
+
+        public Element getElement(GraphElementIdentifier identifier) {
+            return elementMap.get(identifier);
+        }
+
+        private Schema() {
+        }
+
+        public static class Builder {
+            private final HashMap<String, List<Vertex>> vertexLabelMap = new HashMap<>();
+            private final HashMap<String, List<Edge>> edgeLabelMap = new HashMap<>();
+            private final Schema workingSchema = new Schema();
+            private final GraphIdentifier graphIdentifier;
+
+            public Builder(GraphIdentifier graphIdentifier) {
+                this.graphIdentifier = graphIdentifier;
+            }
+
+            /**
+             * @return Null if the primary keys of an existing vertex conflict with the vertex to-be-added.
+             *         The vertex to-be-added otherwise.
+             */
+            public Vertex addVertex(String labelName, List<List<String>> primaryKeyFieldNames, String definition) {
+                if (!vertexLabelMap.containsKey(labelName)) {
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.VERTEX, labelName, 0);
+                    Vertex v = new Vertex(identifier, primaryKeyFieldNames, definition);
+                    workingSchema.vertexList.add(v);
+
+                    // Create a new map entry.
+                    List<Vertex> vertexList = new ArrayList<>();
+                    vertexList.add(v);
+                    vertexLabelMap.put(labelName, vertexList);
+                    return v;
+
+                } else {
+                    List<Vertex> sameLabelVertices = vertexLabelMap.get(labelName);
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.VERTEX, labelName, sameLabelVertices.size());
+                    Vertex v = new Vertex(identifier, primaryKeyFieldNames, definition);
+                    workingSchema.vertexList.add(v);
+
+                    // Insert into the existing map entry.
+                    sameLabelVertices.add(v);
+                    List<List<String>> primaryKeyFromOtherVertices = sameLabelVertices.get(0).getPrimaryKeyFieldNames();
+                    return primaryKeyFromOtherVertices.equals(primaryKeyFieldNames) ? v : null;
+                }
+            }
+
+            /**
+             * @return Null if there exists no vertex with the given source label or destination label.
+             *         The edge to-be-added otherwise.
+             */
+            public Edge addEdge(String edgeLabelName, String destinationLabelName, String sourceLabelName,
+                    List<List<String>> destinationKeyFieldNames) {
+                if (!vertexLabelMap.containsKey(sourceLabelName)) {
+                    return null;
+                }
+
+                Vertex representativeSourceVertex = vertexLabelMap.get(sourceLabelName).get(0);
+                return addEdge(edgeLabelName, destinationLabelName, sourceLabelName,
+                        representativeSourceVertex.getPrimaryKeyFieldNames(), destinationKeyFieldNames,
+                        representativeSourceVertex.getPrimaryKeyFieldNames(), null);
+            }
+
+            /**
+             * @return Null if there exists no vertex with the given source label or destination label.
+             *         The edge to-be-added otherwise.
+             */
+            public Edge addEdge(String edgeLabelName, String destinationLabelName, String sourceLabelName,
+                    List<List<String>> primaryKeyFieldNames, List<List<String>> destinationKeyFieldNames,
+                    List<List<String>> sourceKeyFieldNames, String definition) {
+                if (!vertexLabelMap.containsKey(sourceLabelName) || !vertexLabelMap.containsKey(destinationLabelName)) {
+                    return null;
+                }
+                List<Vertex> sourceVertices = vertexLabelMap.get(sourceLabelName);
+                List<Vertex> destinationVertices = vertexLabelMap.get(destinationLabelName);
+
+                if (!edgeLabelMap.containsKey(edgeLabelName)) {
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.EDGE, edgeLabelName, 0);
+                    Edge e = new Edge(identifier, primaryKeyFieldNames, destinationKeyFieldNames, sourceKeyFieldNames,
+                            destinationVertices, sourceVertices, definition);
+                    workingSchema.edgeList.add(e);
+
+                    // Create a new map entry.
+                    List<Edge> edgeList = new ArrayList<>();
+                    edgeList.add(e);
+                    edgeLabelMap.put(edgeLabelName, edgeList);
+                    return e;
+
+                } else {
+                    List<Edge> sameLabelEdges = edgeLabelMap.get(edgeLabelName);
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.EDGE, edgeLabelName, sameLabelEdges.size());
+                    Edge e = new Edge(identifier, primaryKeyFieldNames, destinationKeyFieldNames, sourceKeyFieldNames,
+                            destinationVertices, sourceVertices, definition);
+                    workingSchema.edgeList.add(e);
+
+                    // Insert into the existing map entry.
+                    sameLabelEdges.add(e);
+                    return e;
+                }
+            }
+
+            public Schema build() {
+                // Build the element map, composed of our vertices and edges.
+                workingSchema.elementMap.clear();
+                workingSchema.getEdges().forEach(e -> workingSchema.elementMap.put(e.identifier, e));
+                workingSchema.getVertices().forEach(v -> workingSchema.elementMap.put(v.identifier, v));
+                return workingSchema;
+            }
+        }
+    }
+
+    public static final class Vertex implements Element {
+        private final GraphElementIdentifier identifier;
+        private final List<List<String>> primaryKeyFieldNames;
+        private final String definition;
+
+        private Vertex(GraphElementIdentifier identifier, List<List<String>> primaryKeyFieldNames, String definition) {
+            this.identifier = Objects.requireNonNull(identifier);
+            this.primaryKeyFieldNames = Objects.requireNonNull(primaryKeyFieldNames);
+            this.definition = definition;
+        }
+
+        @Override
+        public GraphElementIdentifier getIdentifier() {
+            return identifier;
+        }
+
+        @Override
+        public String getLabelName() {
+            return identifier.getLabelName();
+        }
+
+        public List<List<String>> getPrimaryKeyFieldNames() {
+            return primaryKeyFieldNames;
+        }
+
+        @Override
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            return "(:" + getLabelName() + ") AS " + definition;
+        }
+    }
+
+    public static final class Edge implements Element {
+        private final List<List<String>> primaryKeyFieldNames;
+        private final List<List<String>> destinationKeyFieldNames;
+        private final List<List<String>> sourceKeyFieldNames;
+
+        private final GraphElementIdentifier identifier;
+        private final List<Vertex> destinationVertices;
+        private final List<Vertex> sourceVertices;
+        private final String definition;
+
+        private Edge(GraphElementIdentifier identifier, List<List<String>> primaryKeyFieldNames,
+                List<List<String>> destinationKeyFieldNames, List<List<String>> sourceKeyFieldNames,
+                List<Vertex> destinationVertices, List<Vertex> sourceVertices, String definition) {
+            this.primaryKeyFieldNames = Objects.requireNonNull(primaryKeyFieldNames);
+            this.destinationKeyFieldNames = Objects.requireNonNull(destinationKeyFieldNames);
+            this.sourceKeyFieldNames = Objects.requireNonNull(sourceKeyFieldNames);
+            this.destinationVertices = Objects.requireNonNull(destinationVertices);
+            this.sourceVertices = Objects.requireNonNull(sourceVertices);
+            this.identifier = Objects.requireNonNull(identifier);
+            this.definition = definition;
+        }
+
+        @Override
+        public GraphElementIdentifier getIdentifier() {
+            return identifier;
+        }
+
+        @Override
+        public String getLabelName() {
+            return identifier.getLabelName();
+        }
+
+        public String getDestinationLabelName() {
+            return destinationVertices.get(0).getLabelName();
+        }
+
+        public String getSourceLabelName() {
+            return sourceVertices.get(0).getLabelName();
+        }
+
+        public List<List<String>> getPrimaryKeyFieldNames() {
+            return primaryKeyFieldNames;
+        }
+
+        public List<List<String>> getDestinationKeyFieldNames() {
+            return destinationKeyFieldNames;
+        }
+
+        public List<List<String>> getSourceKeyFieldNames() {
+            return sourceKeyFieldNames;
+        }
+
+        public List<Vertex> getDestinationVertices() {
+            return destinationVertices;
+        }
+
+        public List<Vertex> getSourceVertices() {
+            return sourceVertices;
+        }
+
+        @Override
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            String edgeBodyPattern = "[:" + getLabelName() + "]";
+            String sourceNodePattern = "(:" + getSourceLabelName() + ")";
+            String destinationNodePattern = "(:" + getDestinationLabelName() + ")";
+            String edgePattern = sourceNodePattern + "-" + edgeBodyPattern + "->" + destinationNodePattern;
+            return (definition == null) ? edgePattern : (edgePattern + " AS " + definition);
+        }
+    }
+
+    public interface Element {
+        GraphElementIdentifier getIdentifier();
+
+        String getLabelName();
+
+        String getDefinition();
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
new file mode 100644
index 0000000..05f0da4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
@@ -0,0 +1,406 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.Graph;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class GraphTupleTranslator extends AbstractTupleTranslator<Graph> {
+    // Payload field containing serialized Graph.
+    private static final int GRAPH_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+    // For constructing our dependency, edge, and vertex lists.
+    protected OrderedListBuilder listBuilder;
+    protected OrderedListBuilder innerListBuilder;
+    protected OrderedListBuilder nameListBuilder;
+    protected IARecordBuilder subRecordBuilder;
+    protected AOrderedListType stringListList;
+    protected AOrderedListType stringList;
+
+    protected GraphTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.GRAPH_DATASET, GRAPH_PAYLOAD_TUPLE_FIELD_INDEX);
+        if (getTuple) {
+            listBuilder = new OrderedListBuilder();
+            innerListBuilder = new OrderedListBuilder();
+            nameListBuilder = new OrderedListBuilder();
+            subRecordBuilder = new RecordBuilder();
+            stringListList = new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null);
+            stringList = new AOrderedListType(BuiltinType.ASTRING, null);
+        }
+    }
+
+    @Override
+    protected Graph createMetadataEntityFromARecord(ARecord graphRecord)
+            throws HyracksDataException, AlgebricksException {
+        // Read in the dataverse name.
+        DataverseName dataverseName = DataverseName.createFromCanonicalForm(
+                ((AString) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX))
+                        .getStringValue());
+
+        // Read in the graph name.
+        String graphName =
+                ((AString) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX))
+                        .getStringValue();
+
+        // Read in the dependencies.
+        IACursor dependenciesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX))
+                        .getCursor();
+        List<List<Triple<DataverseName, String, String>>> dependencies = new ArrayList<>();
+        while (dependenciesCursor.next()) {
+            List<Triple<DataverseName, String, String>> dependencyList = new ArrayList<>();
+            IACursor qualifiedDependencyCursor = ((AOrderedList) dependenciesCursor.get()).getCursor();
+            while (qualifiedDependencyCursor.next()) {
+                Triple<DataverseName, String, String> dependency =
+                        getDependency((AOrderedList) qualifiedDependencyCursor.get());
+                dependencyList.add(dependency);
+            }
+            dependencies.add(dependencyList);
+        }
+
+        // Read in the vertex and edge lists.
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        Graph.Schema graphSchema = readGraphSchema(graphRecord, graphIdentifier);
+        return new Graph(graphIdentifier, graphSchema, dependencies);
+    }
+
+    private Graph.Schema readGraphSchema(ARecord graphRecord, GraphIdentifier graphIdentifier) throws AsterixException {
+        Graph.Schema.Builder schemaBuilder = new Graph.Schema.Builder(graphIdentifier);
+
+        // Read in the vertex list.
+        IACursor verticesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX))
+                        .getCursor();
+        while (verticesCursor.next()) {
+            ARecord vertex = (ARecord) verticesCursor.get();
+
+            // Read in the label name.
+            String labelName =
+                    ((AString) vertex.getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the primary key fields.
+            List<List<String>> primaryKeyFields = new ArrayList<>();
+            IACursor primaryKeyCursor = ((AOrderedList) vertex
+                    .getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX)).getCursor();
+            while (primaryKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) primaryKeyCursor.get()).getCursor();
+                primaryKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the vertex definition.
+            String definition =
+                    ((AString) vertex.getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX))
+                            .getStringValue();
+
+            if (schemaBuilder.addVertex(labelName, primaryKeyFields, definition) == null) {
+                throw new AsterixException(ErrorCode.METADATA_ERROR,
+                        "Conflicting primary key definitions for node of label " + labelName);
+            }
+        }
+
+        // Read in the edge list.
+        IACursor edgesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX))
+                        .getCursor();
+        while (edgesCursor.next()) {
+            ARecord edge = (ARecord) edgesCursor.get();
+
+            // Read in the label name.
+            String labelName = ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX))
+                    .getStringValue();
+
+            // Read in the destination label name.
+            String destinationLabelName =
+                    ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the source label name.
+            String sourceLabelName =
+                    ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the primary key fields.
+            List<List<String>> primaryKeyFields = new ArrayList<>();
+            IACursor primaryKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (primaryKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) primaryKeyCursor.get()).getCursor();
+                primaryKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the destination key fields.
+            List<List<String>> destinationKeyFields = new ArrayList<>();
+            IACursor destinationKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (destinationKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) destinationKeyCursor.get()).getCursor();
+                destinationKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the source key fields.
+            List<List<String>> sourceKeyFields = new ArrayList<>();
+            IACursor sourceKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (sourceKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) sourceKeyCursor.get()).getCursor();
+                sourceKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the edge definition.
+            String definition;
+            IAObject definitionObj = edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX);
+            if (definitionObj.getType().equals(BuiltinType.ANULL)) {
+                definition = null;
+
+            } else {
+                definition = ((AString) definitionObj).getStringValue();
+            }
+
+            if (schemaBuilder.addEdge(labelName, destinationLabelName, sourceLabelName, primaryKeyFields,
+                    destinationKeyFields, sourceKeyFields, definition) == null) {
+                throw new AsterixException(ErrorCode.METADATA_ERROR,
+                        "Vertex label in edge with label " + labelName + " not found");
+            }
+        }
+
+        return schemaBuilder.build();
+    }
+
+    private List<String> readNameList(IACursor nameCursor) {
+        List<String> fieldName = new ArrayList<>();
+        while (nameCursor.next()) {
+            String subName = ((AString) nameCursor.get()).getStringValue();
+            fieldName.add(subName);
+        }
+        return fieldName;
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(Graph graph) throws AlgebricksException, HyracksDataException {
+        // Write our primary key (dataverse name, graph name).
+        String dataverseCanonicalName = graph.getDataverseName().getCanonicalForm();
+        tupleBuilder.reset();
+        aString.setValue(dataverseCanonicalName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        aString.setValue(graph.getGraphName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        // Write the payload in the third field of the tuple.
+        recordBuilder.reset(MetadataRecordTypes.GRAPH_RECORDTYPE);
+
+        // Write the dataverse name.
+        fieldValue.reset();
+        aString.setValue(dataverseCanonicalName);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX, fieldValue);
+
+        // Write the graph name.
+        fieldValue.reset();
+        aString.setValue(graph.getGraphName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX, fieldValue);
+
+        // Write our dependencies.
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX]);
+        for (List<Triple<DataverseName, String, String>> dependencies : graph.getDependencies()) {
+            List<String> subNames = new ArrayList<>();
+            innerListBuilder.reset(stringListList);
+            for (Triple<DataverseName, String, String> dependency : dependencies) {
+                subNames.clear();
+                getDependencySubNames(dependency, subNames);
+                writeNameList(subNames, itemValue);
+                innerListBuilder.addItem(itemValue);
+            }
+            itemValue.reset();
+            innerListBuilder.write(itemValue.getDataOutput(), true);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX, fieldValue);
+
+        // Write our vertex set.
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX]);
+        for (Graph.Vertex vertex : graph.getGraphSchema().getVertices()) {
+            writeVertexRecord(vertex, itemValue);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX, fieldValue);
+
+        // Write our edge set.
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX]);
+        for (Graph.Edge edge : graph.getGraphSchema().getEdges()) {
+            writeEdgeRecord(edge, itemValue);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX, fieldValue);
+
+        // Finally, write our record.
+        recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        tupleBuilder.addFieldEndOffset();
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+
+    private void writeVertexRecord(Graph.Vertex vertex, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        subRecordBuilder.reset(MetadataRecordTypes.VERTICES_RECORDTYPE);
+
+        // Write the label name.
+        fieldValue.reset();
+        aString.setValue(vertex.getLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the primary key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : vertex.getPrimaryKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the vertex definition.
+        fieldValue.reset();
+        aString.setValue(vertex.getDefinition());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX, fieldValue);
+
+        itemValue.reset();
+        subRecordBuilder.write(itemValue.getDataOutput(), true);
+    }
+
+    private void writeEdgeRecord(Graph.Edge edge, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        subRecordBuilder.reset(MetadataRecordTypes.EDGES_RECORDTYPE);
+
+        // Write the label name.
+        fieldValue.reset();
+        aString.setValue(edge.getLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the destination label name.
+        fieldValue.reset();
+        aString.setValue(edge.getDestinationLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the source label name.
+        fieldValue.reset();
+        aString.setValue(edge.getSourceLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the primary key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getPrimaryKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the destination key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getDestinationKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the source key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getSourceKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the edge definition.
+        fieldValue.reset();
+        if (edge.getDefinition() == null) {
+            nullSerde.serialize(ANull.NULL, fieldValue.getDataOutput());
+
+        } else {
+            aString.setValue(edge.getDefinition());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+        }
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX, fieldValue);
+
+        itemValue.reset();
+        subRecordBuilder.write(itemValue.getDataOutput(), true);
+    }
+
+    private void writeNameList(List<String> name, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        nameListBuilder.reset(stringList);
+        for (String subName : name) {
+            itemValue.reset();
+            aString.setValue(subName);
+            stringSerde.serialize(aString, itemValue.getDataOutput());
+            nameListBuilder.addItem(itemValue);
+        }
+        itemValue.reset();
+        nameListBuilder.write(itemValue.getDataOutput(), true);
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
index c2a9ee3..224c0a7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -92,4 +92,8 @@
     public SynonymTupleTranslator getSynonymTupleTranslator(boolean getTuple) {
         return new SynonymTupleTranslator(getTuple);
     }
+
+    public GraphTupleTranslator getGraphTupleTranslator(boolean getTuple) {
+        return new GraphTupleTranslator(getTuple);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
index 5e44e32..fb7fa16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
@@ -40,7 +40,8 @@
         ADAPTER,
         MERGE_POLICY,
         NODE_GROUP,
-        SYNONYM
+        SYNONYM,
+        GRAPH
     }
 
     private final EntityKind entityKind;
@@ -150,4 +151,8 @@
     static MetadataLockKey createMergePolicyLockKey(String mergePolicyName) {
         return new MetadataLockKey(EntityKind.MERGE_POLICY, null, null, mergePolicyName);
     }
+
+    static MetadataLockKey createGraphLockKey(DataverseName dataverseName, String graphName) {
+        return new MetadataLockKey(EntityKind.GRAPH, null, dataverseName, graphName);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
index 48508e1..260d039 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
@@ -297,4 +297,19 @@
         locks.downgrade(IMetadataLock.Mode.EXCLUSIVE_MODIFY, lock);
     }
 
+    @Override
+    public void acquireGraphReadLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createGraphLockKey(dataverseName, graphName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.READ, lock);
+    }
+
+    @Override
+    public void acquireGraphWriteLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createGraphLockKey(dataverseName, graphName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.WRITE, lock);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index b52cddd..9058340 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -56,6 +56,7 @@
     public static final String SYNONYM_DATASET_NAME = "Synonym";
     public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
     public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
+    public static final String GRAPH_DATASET_NAME = "Graph";
 
     private MetadataConstants() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 7bb11e5..35584e5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -24,6 +24,7 @@
 
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.IMetadataLockUtil;
 import org.apache.asterix.common.metadata.LockList;
@@ -95,6 +96,20 @@
     }
 
     @Override
+    public void createGraphBegin(IMetadataLockManager lockMgr, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, graphIdentifier.getDataverseName());
+        lockMgr.acquireGraphWriteLock(locks, graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+    }
+
+    @Override
+    public void dropGraphBegin(IMetadataLockManager lockMgr, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, graphIdentifier.getDataverseName());
+        lockMgr.acquireGraphWriteLock(locks, graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+    }
+
+    @Override
     public void createIndexBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
             String datasetName, String fullTextConfigName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13823
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I20ca6ea4c782d2fbd548e7340db65cdab5ae693d
Gerrit-Change-Number: 13823
Gerrit-PatchSet: 1
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13823 )


Change subject: [ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH
......................................................................

[ASTERIXDB-2979][MTD] Implement CREATE / DROP GRAPH

- user model changes: yes
- storage format changes: yes
- interface changes: no

Details:
- Add grammar support for graph view definitions, which in the future
  will provide support for graph queries on these graph views.
- Create a new Metadata dataset called "Graph".
- Add support for CREATE GRAPH, which will store graph view definitions
  in the "Graph" dataset.
- Add support for DROP GRAPH, which will remove these definitions in the
  "Graph" dataset.
- Add dependency support for graph views.

Change-Id: I20ca6ea4c782d2fbd548e7340db65cdab5ae693d
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
A asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
M asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
A asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
M asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
A asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
M asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
A asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
A asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
64 files changed, 3,264 insertions(+), 75 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/23/13823/1

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index f2bb66d..79dfe70 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -64,6 +64,7 @@
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.statement.ViewDecl;
@@ -189,8 +190,8 @@
     }
 
     public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, MetadataProvider metadataProvider, IReturningStatement q,
-            SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
+            List<ViewDecl> declaredViews, List<GraphElementDecl> declaredGraphs, MetadataProvider metadataProvider,
+            IReturningStatement q, SessionOutput output, boolean allowNonStoredUdfCalls, boolean inlineUdfsAndViews,
             Collection<VarIdentifier> externalVars, IWarningCollector warningCollector) throws CompilationException {
         if (q == null) {
             return null;
@@ -201,7 +202,7 @@
         }
         IQueryRewriter rw = rewriterFactory.createQueryRewriter();
         LangRewritingContext rwCtx = new LangRewritingContext(metadataProvider, declaredFunctions, declaredViews,
-                warningCollector, q.getVarCounter());
+                declaredGraphs, warningCollector, q.getVarCounter());
         rw.rewrite(rwCtx, q, allowNonStoredUdfCalls, inlineUdfsAndViews, externalVars);
         return new Pair<>(q, q.getVarCounter());
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 70ac386..84b4231 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -87,6 +87,7 @@
 import org.apache.asterix.common.external.IDataSourceAdapter;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.IMetadataLockUtil;
@@ -119,6 +120,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -136,6 +138,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
@@ -157,6 +161,7 @@
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.GraphUtil;
 import org.apache.asterix.lang.common.util.ViewUtil;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.IDatasetDetails;
@@ -179,6 +184,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Library;
@@ -435,6 +441,12 @@
                     case VIEW_DROP:
                         handleViewDropStatement(metadataProvider, stmt);
                         break;
+                    case CREATE_GRAPH:
+                        handleCreateGraphStatement(metadataProvider, stmt);
+                        break;
+                    case GRAPH_DROP:
+                        handleGraphDropStatement(metadataProvider, stmt);
+                        break;
                     case LOAD:
                         handleLoadStatement(metadataProvider, stmt, hcc);
                         break;
@@ -2651,7 +2663,7 @@
             IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
             Query wrappedQuery = queryRewriter.createViewAccessorQuery(viewDecl);
             metadataProvider.setDefaultDataverse(dv);
-            apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), metadataProvider,
+            apiFramework.reWriteQuery(declaredFunctions, Collections.singletonList(viewDecl), null, metadataProvider,
                     wrappedQuery, sessionOutput, false, false, Collections.emptyList(), warningCollector);
 
             List<List<Triple<DataverseName, String, String>>> dependencies =
@@ -2744,6 +2756,163 @@
         }
     }
 
+    public void handleCreateGraphStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        CreateGraphStatement cgs = (CreateGraphStatement) stmt;
+        String graphName = cgs.getGraphName();
+        metadataProvider.validateDatabaseObjectName(cgs.getDataverseName(), graphName, stmt.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(cgs.getDataverseName());
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        lockUtil.createGraphBegin(lockManager, metadataProvider.getLocks(), graphIdentifier);
+        try {
+            doCreateGraph(metadataProvider, cgs, graphIdentifier);
+
+        } finally {
+            metadataProvider.getLocks().unlock();
+            ExternalDatasetsRegistry.INSTANCE.releaseAcquiredLocks(metadataProvider);
+        }
+    }
+
+    protected CreateResult doCreateGraph(MetadataProvider metadataProvider, CreateGraphStatement cgs,
+            GraphIdentifier graphIdentifier) throws Exception {
+        DataverseName dataverseName = graphIdentifier.getDataverseName();
+        String graphName = graphIdentifier.getGraphName();
+        SourceLocation sourceLoc = cgs.getSourceLocation();
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+            }
+
+            Graph existingGraph = MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+            if (existingGraph != null) {
+                if (cgs.isIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return CreateResult.NOOP;
+
+                } else if (!cgs.isReplaceIfExists()) {
+                    throw new CompilationException(ErrorCode.GRAPH_EXISTS, sourceLoc, existingGraph.getGraphName());
+                }
+            }
+
+            // Build the graph schema.
+            Graph.Schema.Builder schemaBuilder = new Graph.Schema.Builder(graphIdentifier);
+            List<GraphElementDecl> graphElementDecls = new ArrayList<>();
+            for (CreateGraphStatement.VertexElement vertex : cgs.getVertexElements()) {
+                Graph.Vertex schemaVertex = schemaBuilder.addVertex(vertex.getLabel(), vertex.getPrimaryKeyFields(),
+                        vertex.getDefinition());
+                if (schemaVertex == null) {
+                    throw new CompilationException(ErrorCode.INVALID_GRAPH_VERTEX_DEFINITION, sourceLoc,
+                            "Conflicting primary keys for vertices with label " + vertex.getLabel());
+                }
+                graphElementDecls.add(new GraphElementDecl(schemaVertex.getIdentifier(), vertex.getExpression()));
+            }
+            for (CreateGraphStatement.EdgeElement edge : cgs.getEdgeElements()) {
+                Graph.Edge schemaEdge;
+
+                if (edge.getDefinition() == null) {
+                    schemaEdge = schemaBuilder.addEdge(edge.getEdgeLabel(), edge.getDestinationLabel(),
+                            edge.getSourceLabel(), edge.getDestinationKeyFields());
+
+                } else {
+                    schemaEdge = schemaBuilder.addEdge(edge.getEdgeLabel(), edge.getDestinationLabel(),
+                            edge.getSourceLabel(), edge.getPrimaryKeyFields(), edge.getDestinationKeyFields(),
+                            edge.getSourceKeyFields(), edge.getDefinition());
+                    graphElementDecls.add(new GraphElementDecl(schemaEdge.getIdentifier(), edge.getExpression()));
+                }
+
+                if (schemaEdge == null) {
+                    throw new CompilationException(ErrorCode.INVALID_GRAPH_EDGE_DEFINITION, sourceLoc,
+                            "Vertex label in edge with label " + edge.getEdgeLabel() + " not found");
+                }
+            }
+
+            // Verify that each element definition is usable.
+            IQueryRewriter queryRewriter = rewriterFactory.createQueryRewriter();
+            metadataProvider.setDefaultDataverse(dv);
+            for (GraphElementDecl elementDecl : graphElementDecls) {
+                Query wrappedQuery = queryRewriter.createGraphElementAccessorQuery(elementDecl);
+                apiFramework.reWriteQuery(declaredFunctions, null, Collections.singletonList(elementDecl),
+                        metadataProvider, wrappedQuery, sessionOutput, false, false, Collections.emptyList(),
+                        warningCollector);
+            }
+
+            // Build our dependencies (collected over all graph element bodies).
+            List<List<Triple<DataverseName, String, String>>> dependencies =
+                    GraphUtil.getGraphDependencies(graphElementDecls, queryRewriter);
+
+            // Add / upsert our graph to our metadata.
+            Graph newGraph = new Graph(graphIdentifier, schemaBuilder.build(), dependencies);
+            if (existingGraph == null) {
+                MetadataManager.INSTANCE.addGraph(mdTxnCtx, newGraph);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return CreateResult.CREATED;
+
+            } else {
+                MetadataManager.INSTANCE.updateGraph(mdTxnCtx, newGraph);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return CreateResult.REPLACED;
+            }
+
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
+    public void handleGraphDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
+        GraphDropStatement gds = (GraphDropStatement) stmt;
+        String graphName = gds.getGraphName();
+        metadataProvider.validateDatabaseObjectName(gds.getDataverseName(), graphName, stmt.getSourceLocation());
+        DataverseName dataverseName = getActiveDataverseName(gds.getDataverseName());
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        lockUtil.dropGraphBegin(lockManager, metadataProvider.getLocks(), graphIdentifier);
+        try {
+            doDropGraph(metadataProvider, gds, dataverseName, graphName);
+
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    protected boolean doDropGraph(MetadataProvider metadataProvider, GraphDropStatement gds,
+            DataverseName dataverseName, String graphName) throws Exception {
+        SourceLocation sourceLoc = gds.getSourceLocation();
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dataverse == null) {
+                if (gds.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return false;
+                } else {
+                    throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+                }
+            }
+
+            Graph graph = MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+            if (graph == null) {
+                if (gds.getIfExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return false;
+                } else {
+                    throw new CompilationException(ErrorCode.UNKNOWN_GRAPH, sourceLoc, graphName);
+                }
+            }
+
+            MetadataManager.INSTANCE.dropGraph(mdTxnCtx, dataverseName, graphName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            return true;
+
+        } catch (Exception e) {
+            abort(e, e, mdTxnCtx);
+            throw e;
+        }
+    }
+
     protected void handleDeclareFunctionStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         FunctionDecl fds = (FunctionDecl) stmt;
         FunctionSignature signature = fds.getSignature();
@@ -2920,8 +3089,8 @@
                 fdList.addAll(declaredFunctions);
                 fdList.add(fd);
                 metadataProvider.setDefaultDataverse(dv);
-                apiFramework.reWriteQuery(fdList, null, metadataProvider, wrappedQuery, sessionOutput, false, false,
-                        Collections.emptyList(), warningCollector);
+                apiFramework.reWriteQuery(fdList, null, null, metadataProvider, wrappedQuery, sessionOutput, false,
+                        false, Collections.emptyList(), warningCollector);
 
                 List<List<Triple<DataverseName, String, String>>> dependencies =
                         FunctionUtil.getFunctionDependencies(fd, queryRewriter);
@@ -3638,7 +3807,7 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(query, stmtParams);
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null, null,
                 metadataProvider, query, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
@@ -3655,7 +3824,7 @@
         Map<VarIdentifier, IAObject> externalVars = createExternalVariables(insertUpsert, stmtParams);
 
         // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
-        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null,
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions, null, null,
                 metadataProvider, insertUpsert, sessionOutput, true, true, externalVars.keySet(), warningCollector);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 8a87de7..32efc6e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -202,7 +202,7 @@
                 if (st.getKind() == Statement.Kind.QUERY) {
                     Query query = (Query) st;
                     IQueryRewriter rewriter = sqlppRewriterFactory.createQueryRewriter();
-                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions, null,
+                    LangRewritingContext rwContext = new LangRewritingContext(metadataProvider, functions, null, null,
                             TestUtils.NOOP_WARNING_COLLECTOR, query.getVarCounter());
                     rewrite(rewriter, query, rwContext);
 
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
new file mode 100644
index 0000000..60df6e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.1.ddl.sqlpp
@@ -0,0 +1,117 @@
+/*
+ * 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.
+ */
+
+// Verify that the Yelp use case generates the appropriate graph metadata.
+
+DROP DATAVERSE    Yelp IF EXISTS;
+DROP DATAVERSE    Yelp_A IF EXISTS;
+DROP DATAVERSE    Yelp_B IF EXISTS;
+CREATE DATAVERSE  Yelp;
+CREATE DATAVERSE  Yelp_A;
+CREATE DATAVERSE  Yelp_B;
+
+USE               Yelp_A;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+CREATE DATASET    Businesses (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Checkins (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Friends (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Reviews (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Tips (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Users (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE FUNCTION   RelevantBusinesses()
+                  { FROM Businesses B
+                    WHERE B.stars > 3.5
+                    SELECT B.* };
+
+USE               Yelp_B;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+CREATE DATASET    Businesses (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Checkins (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Friends (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Reviews (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Tips (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE DATASET    Users (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+CREATE SYNONYM    Yelpers FOR Users;
+
+USE               Yelp;
+CREATE GRAPH      YelpGraph_1 AS
+VERTEX            (:User)
+                  PRIMARY KEY (user_id)
+                  AS Yelp_B.Yelpers,
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:User)-[:FRIENDS_WITH]->(:User)
+                  PRIMARY KEY (user_id, friend)
+                  SOURCE KEY (user_id)
+                  DESTINATION KEY (friend)
+                  AS ( FROM    Yelp_B.Users U
+                       UNNEST  U.friends F
+                       SELECT  U.user_id, F AS friend ),
+EDGE              (:User)-[:FRIENDS_WITH]->(:User)
+                          PRIMARY KEY (user_id, friend)
+                          SOURCE KEY (user_id)
+                          DESTINATION KEY (friend)
+                          AS Yelp_B.Friends,
+EDGE              (:Review)-[:MADE_BY]->(:User)
+                  DESTINATION KEY (user_id),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
+
+CREATE GRAPH      YelpGraph_2 IF NOT EXISTS AS
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
+CREATE GRAPH      YelpGraph_2 IF NOT EXISTS AS
+VERTEX            (:Review)
+                  PRIMARY KEY (review_id)
+                  AS ( FROM    Yelp_A.Reviews R
+                       SELECT  VALUE R ),
+VERTEX            (:Business)
+                  PRIMARY KEY (business_id)
+                  AS ( FROM    Yelp_A.RelevantBusinesses() B
+                       SELECT  VALUE B ),
+EDGE              (:Review)-[:ABOUT]->(:Business)
+                  DESTINATION KEY (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
new file mode 100644
index 0000000..ca52d3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.3.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+DROP GRAPH  Yelp.YelpGraph_1 IF EXISTS;
+DROP GRAPH  Yelp.YelpGraph_1 IF EXISTS;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
new file mode 100644
index 0000000..586a718
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.5.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+CREATE OR REPLACE GRAPH   Yelp.YelpGraph_2 AS
+VERTEX                    (:User)
+                          PRIMARY KEY (user_id)
+                          AS Yelp_B.Yelpers,
+VERTEX                    (:Review)
+                          PRIMARY KEY (review_id)
+                          AS ( FROM    Yelp_A.Reviews R
+                               SELECT  VALUE R ),
+VERTEX                    (:Business)
+                          PRIMARY KEY (business_id)
+                          AS ( FROM    Yelp_A.RelevantBusinesses() B
+                               SELECT  VALUE B ),
+EDGE                      (:User)-[:FRIENDS_WITH]->(:User)
+                          PRIMARY KEY (user_id, friend)
+                          SOURCE KEY (user_id)
+                          DESTINATION KEY (friend)
+                          AS ( FROM    Yelp_B.Users U
+                               UNNEST  U.friends F
+                               SELECT  U.user_id, F AS friend ),
+EDGE                      (:User)-[:FRIENDS_WITH]->(:User)
+                                  PRIMARY KEY (user_id, friend)
+                                  SOURCE KEY (user_id)
+                                  DESTINATION KEY (friend)
+                                  AS Yelp_B.Friends,
+EDGE                      (:Review)-[:MADE_BY]->(:User)
+                          DESTINATION KEY (user_id),
+EDGE                      (:Review)-[:ABOUT]->(:Business)
+                          DESTINATION KEY (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
new file mode 100644
index 0000000..7396ce2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/basic/yelp-use-case/yelp-use-case.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+FROM               `Metadata`.`Graph` G
+SELECT             G.DataverseName, G.GraphName, G.Dependencies, G.Vertices, G.Edges
+ORDER BY           G.DataverseName, G.GraphName;
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
new file mode 100644
index 0000000..25cc3da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.1.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_1", "Dependencies": [ [ [ "Yelp_A", "Reviews" ], [ "Yelp_B", "Users" ], [ "Yelp_B", "Friends" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ], [ [ "Yelp_B", "Yelpers" ] ] ], "Vertices": [ { "Label": "User", "PrimaryKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Yelpers" }, { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "( FROM    Yelp_B.Users U\n                       UNNEST  U.friends F\n                       SELECT  U.user_id, F AS friend )" }, { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Friends" }, { "Label": "MADE_BY", "DestinationLabel": "User", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "user_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null }, { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ] ], "Vertices": [ { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
new file mode 100644
index 0000000..9ad1dd7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.2.adm
@@ -0,0 +1 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ] ], "Vertices": [ { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                       SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                       SELECT  VALUE B )" } ], "Edges": [ { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
new file mode 100644
index 0000000..3e2b0d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/yelp-use-case/yelp-use-case.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "Yelp", "GraphName": "YelpGraph_2", "Dependencies": [ [ [ "Yelp_A", "Reviews" ], [ "Yelp_B", "Users" ], [ "Yelp_B", "Friends" ] ], [ [ "Yelp_A", "RelevantBusinesses", "0" ] ], [ [ "Yelp_B", "Yelpers" ] ] ], "Vertices": [ { "Label": "User", "PrimaryKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Yelpers" }, { "Label": "Review", "PrimaryKey": [ [ "review_id" ] ], "Definition": "( FROM    Yelp_A.Reviews R\n                               SELECT  VALUE R )" }, { "Label": "Business", "PrimaryKey": [ [ "business_id" ] ], "Definition": "( FROM    Yelp_A.RelevantBusinesses() B\n                               SELECT  VALUE B )" } ], "Edges": [ { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "( FROM    Yelp_B.Users U\n                               UNNEST  U.friends F\n                               SELECT  U.user_id, F AS friend )" }, { "Label": "FRIENDS_WITH", "DestinationLabel": "User", "SourceLabel": "User", "PrimaryKey": [ [ "user_id" ], [ "friend" ] ], "DestinationKey": [ [ "friend" ] ], "SourceKey": [ [ "user_id" ] ], "Definition": "Yelp_B.Friends" }, { "Label": "MADE_BY", "DestinationLabel": "User", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "user_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null }, { "Label": "ABOUT", "DestinationLabel": "Business", "SourceLabel": "Review", "PrimaryKey": [ [ "review_id" ] ], "DestinationKey": [ [ "business_id" ] ], "SourceKey": [ [ "review_id" ] ], "Definition": null } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index dc6ae3d..89ea5e3 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -324,6 +324,11 @@
         <expected-error>Cannot drop type test.subType being used by type test.superType_superlist</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="yelp-use-case">
+        <output-dir compare="Text">yelp-use-case</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="exception">
     <test-case FilePath="exception">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
new file mode 100644
index 0000000..ff47a77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataset that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP DATASET      GenericDataset;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
new file mode 100644
index 0000000..ff0f01e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.10.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+// Verify that we cannot drop a graph that doesn't exist.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP GRAPH        GraphThatDoesntExist1 IF EXISTS;
+DROP GRAPH        GraphThatDoesntExist2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
new file mode 100644
index 0000000..09693af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.2.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+// Verify that a function that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE FUNCTION   TestFunction () { SELECT VALUE { "a": 1, "b": 1 } };
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  PRIMARY KEY (a, b)
+                  SOURCE KEY (a)
+                  DESTINATION KEY (b)
+                  AS ( FROM TestFunction() T
+                       SELECT T.* );
+
+DROP FUNCTION     TestFunction();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
new file mode 100644
index 0000000..0cdec73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.3.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+// Verify that a synonym that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE SYNONYM    DatasetSynonym FOR GenericDataset;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS DatasetSynonym,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP SYNONYM      DatasetSynonym;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
new file mode 100644
index 0000000..844389f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.4.ddl.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataverse that a graph is dependent on cannot be dropped.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+DROP DATAVERSE    TestDataverse2 IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+CREATE DATAVERSE  TestDataverse2;
+
+USE               TestDataverse;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+USE               TestDataverse2;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+USE               TestDataverse;
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+USE               TestDataverse2;
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+USE               TestDataverse;
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS TestDataverse2.GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+DROP DATAVERSE    TestDataverse2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
new file mode 100644
index 0000000..f6173aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.5.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+// Verify that a dataset variable as an element definition is a valid dataset.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS DatasetThatDoesNotExist,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
new file mode 100644
index 0000000..5191339
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.6.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+// Verify that a subquery as an element definition is a valid query.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  PRIMARY KEY (_id, _foreign_id)
+                  SOURCE KEY (_id)
+                  DESTINATION KEY (_foreign_id)
+                  AS ( FROM GenericDataset G,
+                            GenericDataset G2
+                       SELECT V );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
new file mode 100644
index 0000000..eb0fa47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.7.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+// Verify that vertices w/ the same label cannot have conflicting primary keys.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_other_id)
+                  AS ( FROM GenericDataset
+                       SELECT VALUE { "_other_id": _other_id } );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
new file mode 100644
index 0000000..7c5269e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.8.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+// Verify that edges do not reference a vertex label that does not exist.
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex3)
+                  DESTINATION KEY (_foreign_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
new file mode 100644
index 0000000..fe00035
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/graph-view/error-handling/error-handling.9.ddl.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+// Verify that we cannot create a graph in the same dataverse with the same name.
+
+DROP DATAVERSE    TestDataverse1 IF EXISTS;
+DROP DATAVERSE    TestDataverse2 IF EXISTS;
+CREATE DATAVERSE  TestDataverse1;
+CREATE DATAVERSE  TestDataverse2;
+
+USE               TestDataverse1;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+USE               TestDataverse2;
+CREATE TYPE       GenericType
+AS                { _id: uuid };
+
+CREATE DATASET    GenericDataset (GenericType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+VERTEX            (:Vertex2)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset,
+EDGE              (:Vertex1)-[:EDGE_1]->(:Vertex2)
+                  DESTINATION KEY (_foreign_id);
+
+USE               TestDataverse1;
+CREATE GRAPH      TestGraph IF NOT EXISTS AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset;
+
+USE               TestDataverse2;
+CREATE GRAPH      TestGraph AS
+VERTEX            (:Vertex1)
+                  PRIMARY KEY (_id)
+                  AS GenericDataset;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 27575fb..c8ccc68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -15716,4 +15716,21 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="graph-view">
+    <test-case FilePath="graph-view">
+      <compilation-unit name="error-handling">
+        <output-dir compare="Text">error-handling</output-dir>
+        <expected-error>Cannot drop dataset TestDataverse.GenericDataset being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop function TestDataverse.TestFunction() being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop synonym TestDataverse.DatasetSynonym being used by graph TestGraph</expected-error>
+        <expected-error>Cannot drop dataverse: dataset (or view) TestDataverse2.GenericDataset being used by graph TestDataverse.TestGraph</expected-error>
+        <expected-error>Cannot find dataset DatasetThatDoesNotExist in dataverse TestDataverse nor an alias with name DatasetThatDoesNotExist</expected-error>
+        <expected-error>Cannot resolve ambiguous alias reference for identifier V</expected-error>
+        <expected-error>Invalid graph vertex definition: Conflicting primary keys for vertices with label Vertex1</expected-error>
+        <expected-error>Invalid graph edge definition: Vertex label in edge with label EDGE_1 not found</expected-error>
+        <expected-error>A graph with this name TestGraph already exists (in line 68, at column 1)</expected-error>
+        <expected-error>Cannot find graph with name GraphThatDoesntExist2</expected-error>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
index 79f13bf..0f8708a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
@@ -508,4 +508,12 @@
      */
     void downgradeDatasetLockToExclusiveModify(LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
+
+    /**
+     * Graph related metadata lock operations.
+     */
+    void acquireGraphReadLock(LockList locks, DataverseName dataverseName, String graphName) throws AlgebricksException;
+
+    void acquireGraphWriteLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index d49dffc..049a02c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -253,6 +253,10 @@
     INVALID_FOREIGN_KEY_DEFINITION_REF_PK_MISMATCH(1166),
     CANNOT_CHANGE_PRIMARY_KEY(1167),
     AMBIGUOUS_PROJECTION(1168),
+    UNKNOWN_GRAPH(1169),
+    GRAPH_EXISTS(1170),
+    INVALID_GRAPH_VERTEX_DEFINITION(1171),
+    INVALID_GRAPH_EDGE_DEFINITION(1172),
 
     // Feed errors
     DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
new file mode 100644
index 0000000..9366771
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphElementIdentifier.java
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.common.graph;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+public class GraphElementIdentifier implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final GraphIdentifier graphIdentifier;
+    private final Kind elementKind;
+
+    // A vertex/edge schema instance can have the same label as another vertex/edge schema instance.
+    private final String labelName;
+    private final int labelId;
+
+    public GraphElementIdentifier(GraphIdentifier graphIdentifier, Kind elementKind, String labelName, int labelId) {
+        this.graphIdentifier = graphIdentifier;
+        this.elementKind = elementKind;
+        this.labelName = labelName;
+        this.labelId = labelId;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return graphIdentifier;
+    }
+
+    public Kind getElementKind() {
+        return elementKind;
+    }
+
+    public String getLabelName() {
+        return labelName;
+    }
+
+    public int getLabelId() {
+        return labelId;
+    }
+
+    @Override
+    public String toString() {
+        return graphIdentifier + "#" + labelName + " (" + elementKind + " " + labelId + ")";
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o instanceof GraphElementIdentifier) {
+            GraphElementIdentifier that = (GraphElementIdentifier) o;
+            return graphIdentifier.equals(that.graphIdentifier) && labelName.equals(that.labelName)
+                    && labelId == that.labelId;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(graphIdentifier, labelName, labelId);
+    }
+
+    public enum Kind {
+        VERTEX,
+        EDGE;
+
+        @Override
+        public String toString() {
+            switch (this) {
+                case EDGE:
+                    return "edge";
+                case VERTEX:
+                    return "vertex";
+                default:
+                    throw new IllegalStateException("Unknown graph element kind.");
+            }
+        }
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
new file mode 100644
index 0000000..877d554
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/graph/GraphIdentifier.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.common.graph;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.asterix.common.metadata.DataverseName;
+
+public class GraphIdentifier implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final DataverseName dataverseName;
+    private final String graphName;
+
+    public GraphIdentifier(DataverseName dataverseName, String graphName) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    @Override
+    public String toString() {
+        return dataverseName + "." + graphName;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o instanceof GraphIdentifier) {
+            GraphIdentifier that = (GraphIdentifier) o;
+            return dataverseName.equals(that.dataverseName) && graphName.equals(that.graphName);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(dataverseName, graphName);
+    }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index 28eb553..bbc3213 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 import com.google.common.collect.ImmutableList;
@@ -58,6 +59,14 @@
     void insertDeleteUpsertBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
             String datasetName) throws AlgebricksException;
 
+    // Graph helpers
+
+    void createGraphBegin(IMetadataLockManager lockManager, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException;
+
+    void dropGraphBegin(IMetadataLockManager lockManager, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException;
+
     // Index helpers
 
     void createIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 136e169..125cfd9e1 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -255,6 +255,10 @@
 1166 = Invalid foreign key definition: foreign key does not match primary key of %1$s %2$s
 1167 = Cannot change primary key of %1$s %2$s
 1168 = Ambiguous projection in SELECT clause
+1169 = Cannot find graph with name %1$s
+1170 = A graph with this name %1$s already exists
+1171 = Invalid graph vertex definition: %1$s
+1172 = Invalid graph edge definition: %1$s
 
 # Feed Errors
 3001 = Illegal state.
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
index bf1b40c..b1534a7 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Expression.java
@@ -40,6 +40,7 @@
         SELECT_EXPRESSION,
         PRIMARY_EXPRESSION,
         CASE_EXPRESSION,
-        WINDOW_EXPRESSION
+        WINDOW_EXPRESSION,
+        GRAPH_ELEMENT_EXPRESSION
     }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
index 1645d76..50e216d 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IParser.java
@@ -23,8 +23,10 @@
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.Warning;
@@ -42,6 +44,8 @@
 
     ViewDecl parseViewBody(DatasetFullyQualifiedName viewName) throws CompilationException;
 
+    GraphElementDecl parseGraphElementBody(GraphElementIdentifier identifier) throws CompilationException;
+
     /**
      * Gets the warnings generated during parsing
      */
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 6099cb6..bc9dac2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
@@ -65,4 +66,6 @@
     Query createFunctionAccessorQuery(FunctionDecl functionDecl);
 
     Query createViewAccessorQuery(ViewDecl viewDecl);
+
+    Query createGraphElementAccessorQuery(GraphElementDecl graphElementDecl);
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 2594cdd..5adadc5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -85,6 +85,7 @@
         CREATE_INDEX,
         CREATE_DATAVERSE,
         CREATE_VIEW,
+        CREATE_GRAPH,
         CREATE_FULL_TEXT_FILTER,
         CREATE_FULL_TEXT_CONFIG,
         INDEX_DROP,
@@ -107,6 +108,8 @@
         CREATE_SYNONYM,
         SYNONYM_DROP,
         VIEW_DROP,
+        GRAPH_DECL,
+        GRAPH_DROP,
         COMPACT,
         EXTERNAL_DATASET_REFRESH,
         SUBSCRIBE_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
index 95cccb0..f5da435 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/rewrites/LangRewritingContext.java
@@ -24,8 +24,10 @@
 import java.util.Map;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -37,16 +39,19 @@
     private final IWarningCollector warningCollector;
     private final Map<FunctionSignature, FunctionDecl> declaredFunctions;
     private final Map<DatasetFullyQualifiedName, ViewDecl> declaredViews;
+    private final Map<GraphElementIdentifier, GraphElementDecl> declaredGraphElements;
     private final Counter varCounter;
     private int systemVarCounter = 1;
     private final Map<Integer, VarIdentifier> oldVarIdToNewVarId = new HashMap<>();
 
     public LangRewritingContext(MetadataProvider metadataProvider, List<FunctionDecl> declaredFunctions,
-            List<ViewDecl> declaredViews, IWarningCollector warningCollector, int varCounter) {
+            List<ViewDecl> declaredViews, List<GraphElementDecl> declaredGraphElements,
+            IWarningCollector warningCollector, int varCounter) {
         this.metadataProvider = metadataProvider;
         this.warningCollector = warningCollector;
         this.declaredFunctions = createMap(declaredFunctions, FunctionDecl::getSignature);
         this.declaredViews = createMap(declaredViews, ViewDecl::getViewName);
+        this.declaredGraphElements = createMap(declaredGraphElements, GraphElementDecl::getIdentifier);
         this.varCounter = new Counter(varCounter);
     }
 
@@ -102,6 +107,10 @@
         return declaredViews;
     }
 
+    public Map<GraphElementIdentifier, GraphElementDecl> getDeclaredGraphElements() {
+        return declaredGraphElements;
+    }
+
     private static <K, V> Map<K, V> createMap(List<V> values, java.util.function.Function<V, K> keyMapper) {
         if (values == null || values.isEmpty()) {
             return Collections.emptyMap();
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
new file mode 100644
index 0000000..448fbb3
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateGraphStatement.java
@@ -0,0 +1,213 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class CreateGraphStatement extends AbstractStatement {
+    private final DataverseName dataverseName;
+    private final String graphName;
+    private final boolean replaceIfExists;
+    private final boolean ifNotExists;
+
+    private final List<VertexElement> vertexElements;
+    private final List<EdgeElement> edgeElements;
+
+    public CreateGraphStatement(DataverseName dataverseName, String graphName, boolean replaceIfExists,
+            boolean ifNotExists, List<VertexElement> vertexElements, List<EdgeElement> edgeElements) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+        this.replaceIfExists = replaceIfExists;
+        this.ifNotExists = ifNotExists;
+        this.vertexElements = vertexElements;
+        this.edgeElements = edgeElements;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.CREATE_GRAPH;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    public boolean isReplaceIfExists() {
+        return replaceIfExists;
+    }
+
+    public boolean isIfNotExists() {
+        return ifNotExists;
+    }
+
+    public List<VertexElement> getVertexElements() {
+        return vertexElements;
+    }
+
+    public List<EdgeElement> getEdgeElements() {
+        return edgeElements;
+    }
+
+    public static class VertexElement {
+        private final List<Integer> primaryKeySourceIndicators;
+        private final List<List<String>> primaryKeyFields;
+        private final Expression expression;
+        private final String definition;
+        private final String label;
+
+        public VertexElement(String label, List<List<String>> primaryKeyFields,
+                List<Integer> primaryKeySourceIndicators, Expression expression, String definition) {
+            this.primaryKeySourceIndicators = primaryKeySourceIndicators;
+            this.primaryKeyFields = primaryKeyFields;
+            this.expression = expression;
+            this.definition = definition;
+            this.label = label;
+        }
+
+        public List<List<String>> getPrimaryKeyFields() {
+            return primaryKeyFields;
+        }
+
+        public List<Integer> getPrimaryKeySourceIndicators() {
+            return primaryKeySourceIndicators;
+        }
+
+        public Expression getExpression() {
+            return expression;
+        }
+
+        public String getDefinition() {
+            return definition;
+        }
+
+        public String getLabel() {
+            return label;
+        }
+
+        @Override
+        public String toString() {
+            return "(:" + label + ") AS " + definition;
+        }
+    }
+
+    public static class EdgeElement {
+        private final List<Integer> destinationKeySourceIndicators;
+        private final List<Integer> sourceKeySourceIndicators;
+        private final List<Integer> primaryKeySourceIndicators;
+
+        private final List<List<String>> destinationKeyFields;
+        private final List<List<String>> sourceKeyFields;
+        private final List<List<String>> primaryKeyFields;
+
+        private final String destinationLabel, edgeLabel, sourceLabel;
+        private final Expression expression;
+        private final String definition;
+
+        public EdgeElement(String edgeLabel, String destinationLabel, String sourceLabel,
+                List<List<String>> primaryKeyFields, List<Integer> primaryKeySourceIndicators,
+                List<List<String>> destinationKeyFields, List<Integer> destinationKeySourceIndicators,
+                List<List<String>> sourceKeyFields, List<Integer> sourceKeySourceIndicators, Expression expression,
+                String definition) {
+            this.destinationKeySourceIndicators = destinationKeySourceIndicators;
+            this.sourceKeySourceIndicators = sourceKeySourceIndicators;
+            this.primaryKeySourceIndicators = primaryKeySourceIndicators;
+            this.destinationKeyFields = destinationKeyFields;
+            this.sourceKeyFields = sourceKeyFields;
+            this.primaryKeyFields = primaryKeyFields;
+            this.destinationLabel = destinationLabel;
+            this.edgeLabel = edgeLabel;
+            this.sourceLabel = sourceLabel;
+            this.expression = expression;
+            this.definition = definition;
+        }
+
+        public List<Integer> getDestinationKeySourceIndicators() {
+            return destinationKeySourceIndicators;
+        }
+
+        public List<Integer> getSourceKeySourceIndicators() {
+            return sourceKeySourceIndicators;
+        }
+
+        public List<Integer> getPrimaryKeySourceIndicators() {
+            return primaryKeySourceIndicators;
+        }
+
+        public List<List<String>> getDestinationKeyFields() {
+            return destinationKeyFields;
+        }
+
+        public List<List<String>> getSourceKeyFields() {
+            return sourceKeyFields;
+        }
+
+        public List<List<String>> getPrimaryKeyFields() {
+            return primaryKeyFields;
+        }
+
+        public String getDestinationLabel() {
+            return destinationLabel;
+        }
+
+        public String getEdgeLabel() {
+            return edgeLabel;
+        }
+
+        public String getSourceLabel() {
+            return sourceLabel;
+        }
+
+        public Expression getExpression() {
+            return expression;
+        }
+
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            String edgeBodyPattern = "[:" + edgeLabel + "]";
+            String sourceNodePattern = "(:" + sourceLabel + ")";
+            String destinationNodePattern = "(:" + destinationLabel + ")";
+            String edgePattern = sourceNodePattern + "-" + edgeBodyPattern + "->" + destinationNodePattern;
+            return (definition == null) ? edgePattern : (edgePattern + " AS " + definition);
+        }
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
new file mode 100644
index 0000000..5411a85
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphDropStatement.java
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class GraphDropStatement extends AbstractStatement {
+    private final DataverseName dataverseName;
+    private final String graphName;
+    private final boolean ifExists;
+
+    public GraphDropStatement(DataverseName dataverseName, String graphName, boolean ifExists) {
+        this.dataverseName = dataverseName;
+        this.graphName = graphName;
+        this.ifExists = ifExists;
+    }
+
+    public DataverseName getDataverseName() {
+        return dataverseName;
+    }
+
+    public String getGraphName() {
+        return graphName;
+    }
+
+    public boolean getIfExists() {
+        return ifExists;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_DROP;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.DDL;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
new file mode 100644
index 0000000..c1e22c2
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/GraphElementDecl.java
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public final class GraphElementDecl extends AbstractStatement {
+    private final GraphElementIdentifier identifier;
+    private final Expression body;
+    private Expression normalizedBody;
+
+    public GraphElementDecl(GraphElementIdentifier identifier, Expression body) {
+        this.identifier = Objects.requireNonNull(identifier);
+        this.body = Objects.requireNonNull(body);
+    }
+
+    public GraphElementIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return identifier.getGraphIdentifier();
+    }
+
+    public Expression getBody() {
+        return body;
+    }
+
+    public Expression getNormalizedBody() {
+        return normalizedBody;
+    }
+
+    public void setNormalizedBody(Expression normalizedBody) {
+        this.normalizedBody = normalizedBody;
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_DECL;
+    }
+
+    @Override
+    public byte getCategory() {
+        return Category.QUERY;
+    }
+
+    @Override
+    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+        return visitor.visit(this, arg);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
new file mode 100644
index 0000000..edb99ac
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/GraphUtil.java
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.common.util;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.common.base.IQueryRewriter;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
+import org.apache.asterix.metadata.entities.Graph;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public final class GraphUtil {
+    private GraphUtil() {
+    }
+
+    public static GraphElementDecl parsedStoredGraphElement(Graph.Element element, IParserFactory parserFactory,
+            IWarningCollector warningCollector, SourceLocation sourceLocation) throws CompilationException {
+        IParser parser = parserFactory.createParser(new StringReader(element.getDefinition()));
+        try {
+            GraphElementDecl graphElement = parser.parseGraphElementBody(element.getIdentifier());
+            if (warningCollector != null) {
+                parser.getWarnings(warningCollector);
+            }
+            return graphElement;
+
+        } catch (CompilationException e) {
+            GraphElementIdentifier.Kind kind = element.getIdentifier().getElementKind();
+            throw new CompilationException(
+                    kind.equals(GraphElementIdentifier.Kind.VERTEX) ? ErrorCode.INVALID_GRAPH_VERTEX_DEFINITION
+                            : ErrorCode.INVALID_GRAPH_EDGE_DEFINITION,
+                    e, sourceLocation, element.getIdentifier().toString(), e.getMessage());
+        }
+    }
+
+    public static List<List<Triple<DataverseName, String, String>>> getGraphDependencies(
+            List<GraphElementDecl> elementDecls, IQueryRewriter rewriter) throws CompilationException {
+        List<Triple<DataverseName, String, String>> datasetDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> synonymDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
+
+        for (GraphElementDecl elementDecl : elementDecls) {
+            Expression normalizedBody = elementDecl.getNormalizedBody();
+            if (normalizedBody == null) {
+                // We should have set the normalized body by calling {@link APIFramework#reWriteQuery} beforehand.
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, elementDecl.getSourceLocation(),
+                        elementDecl.getIdentifier().toString());
+            }
+
+            // Collect the list of used functions and used datasets.
+            ExpressionUtils.collectDependencies(normalizedBody, rewriter, datasetDependencies, synonymDependencies,
+                    functionDependencies);
+        }
+
+        return Graph.createDependencies(datasetDependencies, functionDependencies, synonymDependencies);
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0ddbeb4..4db2c3e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -74,6 +74,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -91,6 +92,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
@@ -987,6 +990,39 @@
         return null;
     }
 
+    @Override
+    public Void visit(GraphElementDecl ged, Integer arg) throws CompilationException {
+        // Note: this statement is internal.
+        return null;
+    }
+
+    @Override
+    public Void visit(CreateGraphStatement cgs, Integer step) throws CompilationException {
+        out.print(skip(step) + CREATE + generateOrReplace(cgs.isReplaceIfExists()) + " graph ");
+        out.print(generateIfNotExists(cgs.isIfNotExists()));
+        out.print(generateFullName(cgs.getDataverseName(), cgs.getGraphName()));
+        out.println(" as ");
+        for (CreateGraphStatement.VertexElement vertex : cgs.getVertexElements()) {
+            out.print(skip(step) + " vertex ");
+            out.println(vertex.toString());
+        }
+        for (CreateGraphStatement.EdgeElement edge : cgs.getEdgeElements()) {
+            out.print(skip(step) + " edge ");
+            out.println(edge.toString());
+        }
+        out.println(SEMICOLON);
+        return null;
+    }
+
+    @Override
+    public Void visit(GraphDropStatement gds, Integer step) throws CompilationException {
+        out.print(skip(step) + "drop graph ");
+        out.print(generateFullName(gds.getDataverseName(), gds.getGraphName()));
+        out.print(generateIfExists(gds.getIfExists()));
+        out.println(SEMICOLON);
+        return null;
+    }
+
     protected void printConfiguration(Map<String, String> properties) {
         if (properties.size() > 0) {
             out.print("(");
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 91fe664..4a766d3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -34,6 +34,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -50,6 +51,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.LibraryDropStatement;
@@ -309,4 +312,19 @@
     public R visit(ViewDecl vd, T arg) throws CompilationException {
         return null;
     }
+
+    @Override
+    public R visit(CreateGraphStatement cgs, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(GraphDropStatement gds, T arg) throws CompilationException {
+        return null;
+    }
+
+    @Override
+    public R visit(GraphElementDecl ged, T arg) throws CompilationException {
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index c2b1311..ed33648 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -51,6 +51,7 @@
 import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
@@ -67,6 +68,8 @@
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.LibraryDropStatement;
@@ -216,4 +219,10 @@
     R visit(ViewDropStatement vds, T arg) throws CompilationException;
 
     R visit(ViewDecl vd, T arg) throws CompilationException;
+
+    R visit(CreateGraphStatement cgs, T arg) throws CompilationException;
+
+    R visit(GraphDropStatement gds, T arg) throws CompilationException;
+
+    R visit(GraphElementDecl ged, T arg) throws CompilationException;
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
new file mode 100644
index 0000000..c9a7f3e
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/expression/GraphElementExpression.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.lang.sqlpp.expression;
+
+import java.util.Collections;
+
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class GraphElementExpression extends CallExpr {
+    public static final FunctionIdentifier GRAPH_ELEMENT =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "graph-element", 5);
+
+    private final GraphElementIdentifier identifier;
+
+    public GraphElementExpression(GraphElementIdentifier identifier) {
+        super(new FunctionSignature(GRAPH_ELEMENT), Collections.emptyList(), null);
+        this.identifier = identifier;
+    }
+
+    public GraphElementIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public GraphIdentifier getGraphIdentifier() {
+        return identifier.getGraphIdentifier();
+    }
+
+    @Override
+    public Kind getKind() {
+        return Kind.GRAPH_ELEMENT_EXPRESSION;
+    }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index b28f4b9..195ca6a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -31,6 +31,8 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.AbstractExpression;
@@ -46,14 +48,17 @@
 import org.apache.asterix.lang.common.literal.MissingLiteral;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.ViewDecl;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.ExpressionUtils;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.GraphUtil;
 import org.apache.asterix.lang.common.util.ViewUtil;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.lang.sqlpp.expression.GraphElementExpression;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.GenerateColumnNameVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineColumnAliasVisitor;
 import org.apache.asterix.lang.sqlpp.rewrites.visitor.InlineWithExpressionVisitor;
@@ -82,12 +87,12 @@
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.ViewDetails;
 import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.TypeUtil;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -328,11 +333,12 @@
     }
 
     protected void loadAndInlineUdfsAndViews() throws CompilationException {
-        Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> udfAndViewDecls =
-                loadUdfsAndViews(topStatement);
-        Map<FunctionSignature, FunctionDecl> udfs = udfAndViewDecls.first;
-        Map<DatasetFullyQualifiedName, ViewDecl> views = udfAndViewDecls.second;
-        if (udfs.isEmpty() && views.isEmpty()) {
+        Decls decls = loadDecls(topStatement);
+        Map<FunctionSignature, FunctionDecl> udfs = decls.udfs;
+        Map<DatasetFullyQualifiedName, ViewDecl> views = decls.views;
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements = decls.graphElements;
+
+        if (udfs.isEmpty() && views.isEmpty() && graphElements.isEmpty()) {
             // nothing to do
             return;
         }
@@ -389,10 +395,10 @@
         return SqlppVariableUtil.toUserDefinedName(paramVar.getValue());
     }
 
-    private Pair<Map<FunctionSignature, FunctionDecl>, Map<DatasetFullyQualifiedName, ViewDecl>> loadUdfsAndViews(
-            IReturningStatement topExpr) throws CompilationException {
+    private Decls loadDecls(IReturningStatement topExpr) throws CompilationException {
         Map<FunctionSignature, FunctionDecl> udfs = new LinkedHashMap<>();
         Map<DatasetFullyQualifiedName, ViewDecl> views = new LinkedHashMap<>();
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements = new LinkedHashMap<>();
         Deque<AbstractCallExpression> workQueue = new ArrayDeque<>();
         SqlppGatherFunctionCallsVisitor callVisitor = new SqlppGatherFunctionCallsVisitor(workQueue);
         for (Expression expr : topExpr.getDirectlyEnclosedExpressions()) {
@@ -401,6 +407,15 @@
         AbstractCallExpression fnCall;
         while ((fnCall = workQueue.poll()) != null) {
             switch (fnCall.getKind()) {
+                case GRAPH_ELEMENT_EXPRESSION:
+                    GraphElementExpression graphElementCall = (GraphElementExpression) fnCall;
+                    GraphElementIdentifier identifier = graphElementCall.getIdentifier();
+                    if (!graphElements.containsKey(identifier)) {
+                        GraphElementDecl elementDecl = fetchGraphElementDecl(identifier, fnCall.getSourceLocation());
+                        graphElements.put(identifier, elementDecl);
+                        elementDecl.getNormalizedBody().accept(callVisitor, null);
+                    }
+                    break;
                 case CALL_EXPRESSION:
                     FunctionSignature fs = fnCall.getFunctionSignature();
                     DataverseName fsDataverse = fs.getDataverseName();
@@ -439,7 +454,7 @@
                             fnCall.getFunctionSignature().toString(false));
             }
         }
-        return new Pair<>(udfs, views);
+        return new Decls(udfs, views, graphElements);
     }
 
     private FunctionDecl fetchFunctionDecl(FunctionSignature fs, SourceLocation sourceLoc) throws CompilationException {
@@ -511,6 +526,37 @@
         return viewDecl;
     }
 
+    private GraphElementDecl fetchGraphElementDecl(GraphElementIdentifier identifier, SourceLocation sourceLocation)
+            throws CompilationException {
+        GraphElementDecl elementDecl = context.getDeclaredGraphElements().get(identifier);
+
+        // If we cannot find the graph in our context, search our metadata.
+        if (elementDecl == null) {
+            GraphIdentifier graphIdentifier = identifier.getGraphIdentifier();
+            Graph graph;
+            try {
+                graph = metadataProvider.findGraph(graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+            } catch (AlgebricksException e) {
+                throw new CompilationException(ErrorCode.UNKNOWN_GRAPH, e, sourceLocation,
+                        graphIdentifier.getGraphName());
+            }
+            Graph.Element graphElement = graph.getGraphSchema().getElement(identifier);
+            if (graphElement == null) {
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLocation,
+                        "Unable to resolve the graph element " + identifier);
+            }
+            elementDecl = GraphUtil.parsedStoredGraphElement(graphElement, parserFactory, context.getWarningCollector(),
+                    sourceLocation);
+        }
+
+        Expression normalizedBody = elementDecl.getNormalizedBody();
+        if (normalizedBody == null) {
+            normalizedBody = rewriteGraphElementBody(elementDecl);
+            elementDecl.setNormalizedBody(normalizedBody);
+        }
+        return elementDecl;
+    }
+
     private Expression rewriteFunctionBody(FunctionDecl fnDecl) throws CompilationException {
         FunctionSignature fs = fnDecl.getSignature();
         return rewriteFunctionOrViewBody(fs.getDataverseName(), fs, fnDecl.getFuncBody(), fnDecl.getParamList(),
@@ -534,6 +580,12 @@
         return rewrittenBodyExpr;
     }
 
+    private Expression rewriteGraphElementBody(GraphElementDecl elementDecl) throws CompilationException {
+        GraphIdentifier graphIdentifier = elementDecl.getGraphIdentifier();
+        return rewriteFunctionOrViewBody(graphIdentifier.getDataverseName(), graphIdentifier.getGraphName(),
+                elementDecl.getBody(), Collections.emptyList(), false, elementDecl.getSourceLocation());
+    }
+
     private Expression rewriteFunctionOrViewBody(DataverseName entityDataverseName, Object entityDisplayName,
             Expression bodyExpr, List<VarIdentifier> externalVars, boolean allowNonStoredUdfCalls,
             SourceLocation sourceLoc) throws CompilationException {
@@ -583,6 +635,12 @@
     }
 
     @Override
+    public Query createGraphElementAccessorQuery(GraphElementDecl graphElementDecl) {
+        GraphElementExpression functionCall = new GraphElementExpression(graphElementDecl.getIdentifier());
+        return ExpressionUtils.createWrappedQuery(functionCall, graphElementDecl.getSourceLocation());
+    }
+
+    @Override
     public Query createViewAccessorQuery(ViewDecl viewDecl) {
         // dataverse_name.view_name
         DataverseName dataverseName = viewDecl.getViewName().getDataverseName();
@@ -605,4 +663,17 @@
         resultExpr.setSourceLocation(sourceLoc);
         return resultExpr;
     }
+
+    private static class Decls {
+        Map<FunctionSignature, FunctionDecl> udfs;
+        Map<DatasetFullyQualifiedName, ViewDecl> views;
+        Map<GraphElementIdentifier, GraphElementDecl> graphElements;
+
+        public Decls(Map<FunctionSignature, FunctionDecl> udfs, Map<DatasetFullyQualifiedName, ViewDecl> views,
+                Map<GraphElementIdentifier, GraphElementDecl> graphElements) {
+            this.udfs = udfs;
+            this.views = views;
+            this.graphElements = graphElements;
+        }
+    }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
index 38b66e2..e8a0185 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppFunctionCallResolverVisitor.java
@@ -52,9 +52,15 @@
 
     @Override
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
-        FunctionSignature fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(),
-                callExpr.getSourceLocation(), context.getMetadataProvider(), callExprResolver, true,
-                context.getDeclaredFunctions(), allowNonStoredUdfCalls);
+        FunctionSignature fs;
+        if (callExpr.getKind().equals(Expression.Kind.GRAPH_ELEMENT_EXPRESSION)) {
+            // This is a marker function that we cannot resolve. Skip this.
+            fs = callExpr.getFunctionSignature();
+        } else {
+            fs = FunctionUtil.resolveFunctionCall(callExpr.getFunctionSignature(), callExpr.getSourceLocation(),
+                    context.getMetadataProvider(), callExprResolver, true, context.getDeclaredFunctions(),
+                    allowNonStoredUdfCalls);
+        }
         callExpr.setFunctionSignature(fs);
         return super.visit(callExpr, arg);
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index facdfa7..91fb309 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -74,6 +74,7 @@
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphElementIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.base.AbstractClause;
@@ -133,6 +134,7 @@
 import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
 import org.apache.asterix.lang.common.statement.CreateFeedStatement;
 import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
+import org.apache.asterix.lang.common.statement.CreateGraphStatement;
 import org.apache.asterix.lang.common.statement.CreateIndexStatement;
 import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
 import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
@@ -149,6 +151,8 @@
 import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.FunctionDropStatement;
+import org.apache.asterix.lang.common.statement.GraphDropStatement;
+import org.apache.asterix.lang.common.statement.GraphElementDecl;
 import org.apache.asterix.lang.common.statement.IndexDropStatement;
 import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
 import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
@@ -256,6 +260,7 @@
     private static final String REPLACE = "REPLACE";
     private static final String RETURNS = "RETURNS";
     private static final String CONFIG = "CONFIG";
+    private static final String SOURCE = "SOURCE";
 
 
     private static final String INT_TYPE_NAME = "int";
@@ -454,6 +459,22 @@
         });
     }
 
+    @Override
+    public GraphElementDecl parseGraphElementBody(GraphElementIdentifier identifier) throws CompilationException {
+        return parseImpl(new ParseFunction<GraphElementDecl>() {
+            @Override
+            public GraphElementDecl parse() throws ParseException {
+                DataverseName dataverse = defaultDataverse;
+                defaultDataverse = identifier.getGraphIdentifier().getDataverseName();
+                createNewScope();
+                Expression elementBodyExpr = SQLPPParser.this.ViewBody();
+                removeCurrentScope();
+                defaultDataverse = dataverse;
+                return new GraphElementDecl(identifier, elementBodyExpr);
+            }
+        });
+    }
+
     private <T> T parseImpl(ParseFunction<T> parseFunction) throws CompilationException {
         warningCollector.clear();
         hintCollector.clear();
@@ -878,6 +899,7 @@
     | stmt = CreateFeedPolicyStatement(startToken)
     | stmt = CreateFullTextStatement(startToken)
     | stmt = CreateViewStatement(startToken, false)
+    | stmt = CreateGraphStatement(startToken, false)
   )
   {
     return stmt;
@@ -894,6 +916,7 @@
   (
     stmt = CreateFunctionStatement(startStmtToken, true)
     | stmt = CreateViewStatement(startStmtToken, true)
+    | stmt = CreateGraphStatement(startStmtToken, true)
   )
   {
     // check expected token here to make the grammar extension plugin happy
@@ -1521,11 +1544,11 @@
           }
         )*
         (
-          <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+          <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = KeyFields() <RIGHTPAREN>
           <NOT> <ENFORCED>
         )?
         (
-          <IDENTIFIER> { expectToken(FOREIGN); } <KEY> <LEFTPAREN> foreignKeyFields = PrimaryKeyFields() <RIGHTPAREN>
+          <IDENTIFIER> { expectToken(FOREIGN); } <KEY> <LEFTPAREN> foreignKeyFields = KeyFields() <RIGHTPAREN>
           <IDENTIFIER> { expectToken(REFERENCES); } refNameComponents = QualifiedName()
           <NOT> <ENFORCED>
           {
@@ -2019,13 +2042,13 @@
   Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
 }
 {
-  <PRIMARY> <KEY> primaryKeyFields = PrimaryKeyFields()
+  <PRIMARY> <KEY> primaryKeyFields = KeyFields()
   {
     return primaryKeyFields;
   }
 }
 
-Pair<List<Integer>, List<List<String>>> PrimaryKeyFields() throws ParseException:
+Pair<List<Integer>, List<List<String>>> KeyFields() throws ParseException:
 {
   Pair<Integer, List<String>> tmp = null;
   List<Integer> keyFieldSourceIndicators = new ArrayList<Integer>();
@@ -2048,6 +2071,184 @@
     }
 }
 
+CreateGraphStatement CreateGraphStatement(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+  CreateGraphStatement stmt = null;
+}
+{
+  <GRAPH> stmt = GraphSpecification(startStmtToken, orReplace)
+  {
+    return stmt;
+  }
+}
+
+CreateGraphStatement GraphSpecification(Token startStmtToken, boolean orReplace) throws ParseException:
+{
+  Pair<DataverseName, Identifier> nameComponents = null;
+  boolean ifNotExists = false;
+
+  List<CreateGraphStatement.VertexElement> vertexElements = new ArrayList<CreateGraphStatement.VertexElement>();
+  List<CreateGraphStatement.EdgeElement> edgeElements = new ArrayList<CreateGraphStatement.EdgeElement>();
+  CreateGraphStatement.VertexElement vertexElement = null;
+  CreateGraphStatement.EdgeElement edgeElement = null;
+}
+{
+  nameComponents = QualifiedName()
+  ifNotExists = IfNotExists()
+  {
+    if (orReplace && ifNotExists) {
+      throw new SqlppParseException(getSourceLocation(startStmtToken), "Unexpected IF NOT EXISTS");
+    }
+  }
+  <AS>
+  vertexElement = GraphVertexSpecification() { vertexElements.add(vertexElement); }
+  ( <COMMA>
+    (
+      ( vertexElement = GraphVertexSpecification() { vertexElements.add(vertexElement); } )
+      | ( edgeElement = GraphEdgeSpecification(startStmtToken) { edgeElements.add(edgeElement); } )
+    )
+  )*
+  {
+    CreateGraphStatement stmt = new CreateGraphStatement(nameComponents.first, nameComponents.second.getValue(),
+        orReplace, ifNotExists, vertexElements, edgeElements);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
+CreateGraphStatement.VertexElement GraphVertexSpecification() throws ParseException:
+{
+  Pair<List<Integer>, List<List<String>>> primaryKeyFields;
+  Token beginPos = null, endPos = null;
+  Expression vertexDefinitionExpr;
+  String vertexName;
+}
+{
+  <VERTEX>
+  vertexName = GraphVertexDefinitionPattern()
+  <PRIMARY> <KEY> <LEFTPAREN> primaryKeyFields = KeyFields() <RIGHTPAREN>
+  <AS>
+  {
+    beginPos = token;
+    createNewScope();
+  }
+  (
+    vertexDefinitionExpr = ViewBody()
+    | <LEFTPAREN> vertexDefinitionExpr = ViewBody() <RIGHTPAREN>
+  )
+  {
+    endPos = token;
+    String vDef = extractFragment(beginPos.beginLine, beginPos.beginColumn + 1, endPos.endLine, endPos.endColumn + 1);
+    removeCurrentScope();
+    return new CreateGraphStatement.VertexElement(vertexName, primaryKeyFields.second, primaryKeyFields.first,
+      vertexDefinitionExpr, vDef);
+  }
+}
+
+String GraphVertexDefinitionPattern() throws ParseException:
+{
+  String vertexName;
+}
+{
+  <LEFTPAREN> <COLON> vertexName = Identifier() <RIGHTPAREN>
+  {
+    return vertexName;
+  }
+}
+
+CreateGraphStatement.EdgeElement GraphEdgeSpecification(Token startStmtToken) throws ParseException:
+{
+  Pair<Triple<String, String, String>, Boolean> edgeDefinitionPattern;
+  Pair<List<Integer>, List<List<String>>> keyFields;
+  Token beginPos = null, endPos = null;
+  Expression edgeDefinitionExpr = null;
+
+  List<Integer> destinationKeySourceIndicators = null;
+  List<Integer> sourceKeySourceIndicators = null;
+  List<Integer> primaryKeySourceIndicators = null;
+  List<List<String>> destinationKeyFields = null;
+  List<List<String>> sourceKeyFields = null;
+  List<List<String>> primaryKeyFields = null;
+}
+{
+  <EDGE>
+  edgeDefinitionPattern = GraphEdgeDefinitionPattern()
+  (
+    (
+      <PRIMARY> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        primaryKeyFields = keyFields.second;
+        primaryKeySourceIndicators = keyFields.first;
+      }
+      <IDENTIFIER> { expectToken(SOURCE); } <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        sourceKeyFields = keyFields.second;
+        sourceKeySourceIndicators = keyFields.first;
+      }
+      <DESTINATION> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        destinationKeyFields = keyFields.second;
+        destinationKeySourceIndicators = keyFields.first;
+      }
+      <AS>
+      {
+        beginPos = token;
+        createNewScope();
+      }
+      (
+        edgeDefinitionExpr = ViewBody()
+        | <LEFTPAREN> edgeDefinitionExpr = ViewBody() <RIGHTPAREN>
+      )
+    )
+    |
+    (
+      <DESTINATION> <KEY> <LEFTPAREN> keyFields = KeyFields() <RIGHTPAREN>
+      {
+        destinationKeyFields = keyFields.second;
+        destinationKeySourceIndicators = keyFields.first;
+      }
+    )
+  )
+  {
+    String destinationLabel, edgeLabel, sourceLabel;
+    if (edgeDefinitionPattern.second) { // isDirectedLeft
+      sourceLabel = edgeDefinitionPattern.first.third;
+      edgeLabel = edgeDefinitionPattern.first.second;
+      destinationLabel = edgeDefinitionPattern.first.first;
+    } else {
+      sourceLabel = edgeDefinitionPattern.first.first;
+      edgeLabel = edgeDefinitionPattern.first.second;
+      destinationLabel = edgeDefinitionPattern.first.third;
+    }
+
+    String eDef = null;
+    if (edgeDefinitionExpr != null) {
+      endPos = token;
+      eDef = extractFragment(beginPos.beginLine, beginPos.beginColumn + 1, endPos.endLine, endPos.endColumn + 1);
+      removeCurrentScope();
+    }
+
+    return new CreateGraphStatement.EdgeElement(edgeLabel, destinationLabel, sourceLabel, primaryKeyFields,
+        primaryKeySourceIndicators, destinationKeyFields, destinationKeySourceIndicators, sourceKeyFields,
+        sourceKeySourceIndicators, edgeDefinitionExpr, eDef);
+  }
+}
+
+Pair<Triple<String, String, String>, Boolean> GraphEdgeDefinitionPattern() throws ParseException:
+{
+  String leftVertexName, edgeName, rightVertexName;
+  boolean isDirectedLeft;
+}
+{
+  leftVertexName = GraphVertexDefinitionPattern()
+  ( <MINUS> <LEFTBRACKET> <COLON> edgeName = Identifier() <RIGHTBRACKET> <MINUS> <GT> { isDirectedLeft = false; }
+  | <LT> <MINUS> <LEFTBRACKET> <COLON> edgeName = Identifier() <RIGHTBRACKET> <MINUS> { isDirectedLeft = true; } )
+  rightVertexName = GraphVertexDefinitionPattern()
+  {
+    Triple<String, String, String> t = new Triple<String, String, String>(leftVertexName, edgeName, rightVertexName);
+    return new Pair<Triple<String, String, String>, Boolean>(t, isDirectedLeft);
+  }
+}
+
 Statement DropStatement() throws ParseException:
 {
   Token startToken = null;
@@ -2068,6 +2269,7 @@
     | stmt = DropSynonymStatement(startToken)
     | stmt = DropFullTextStatement(startToken)
     | stmt = DropViewStatement(startToken)
+    | stmt = DropGraphStatement(startToken)
   )
   {
     return stmt;
@@ -2382,6 +2584,30 @@
   }
 }
 
+GraphDropStatement DropGraphStatement(Token startStmtToken) throws ParseException:
+{
+   GraphDropStatement stmt = null;
+}
+{
+  <GRAPH> stmt = DropGraphSpecification(startStmtToken)
+  {
+    return stmt;
+  }
+}
+
+GraphDropStatement DropGraphSpecification(Token startStmtToken) throws ParseException:
+{
+  Pair<DataverseName,Identifier> pairId = null;
+  boolean ifExists = false;
+}
+{
+  pairId = QualifiedName() ifExists = IfExists()
+  {
+    GraphDropStatement stmt = new GraphDropStatement(pairId.first, pairId.second.getValue(), ifExists);
+    return addSourceLocation(stmt, startStmtToken);
+  }
+}
+
 boolean IfExists() throws ParseException :
 {
 }
@@ -5305,10 +5531,12 @@
   | <DEFINITION : "definition">
   | <DELETE : "delete">
   | <DESC : "desc">
+  | <DESTINATION : "destination">
   | <DISCONNECT : "disconnect">
   | <DISTINCT : "distinct">
   | <DIV : "div">
   | <DROP : "drop">
+  | <EDGE: "edge">
   | <ELEMENT : "element">
   | <EXPLAIN : "explain">
   | <ELSE : "else">
@@ -5327,6 +5555,7 @@
   | <FULL : "full">
   | <FULLTEXT : "fulltext">
   | <FUNCTION : "function">
+  | <GRAPH : "graph">
   | <GROUP : "group">
   | <HAVING : "having">
   | <HINTS : "hints">
@@ -5397,6 +5626,7 @@
   | <USING : "using">
   | <VALUE : "value">
   | <VALUED : "valued">
+  | <VERTEX : "vertex">
   | <VIEW : "view">
   | <WHEN : "when">
   | <WHERE : "where">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index 6602666..63ccd10 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
@@ -59,6 +60,8 @@
     protected final Map<DataverseName, Dataverse> dataverses = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name.
     protected final Map<DataverseName, Map<String, Dataset>> datasets = new HashMap<>();
+    // Key is dataverse name. Key of value map is graph name.
+    protected final Map<DataverseName, Map<String, Graph>> graphs = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name. Key of value map of value map is index name.
     protected final Map<DataverseName, Map<String, Map<String, Index>>> indexes = new HashMap<>();
     // Key is dataverse name. Key of value map is datatype name.
@@ -120,24 +123,27 @@
                     synchronized (indexes) {
                         synchronized (datatypes) {
                             synchronized (functions) {
-                                synchronized (fullTextConfigs) {
-                                    synchronized (fullTextFilters) {
-                                        synchronized (adapters) {
-                                            synchronized (libraries) {
-                                                synchronized (compactionPolicies) {
-                                                    synchronized (synonyms) {
-                                                        dataverses.clear();
-                                                        nodeGroups.clear();
-                                                        datasets.clear();
-                                                        indexes.clear();
-                                                        datatypes.clear();
-                                                        functions.clear();
-                                                        fullTextConfigs.clear();
-                                                        fullTextFilters.clear();
-                                                        adapters.clear();
-                                                        libraries.clear();
-                                                        compactionPolicies.clear();
-                                                        synonyms.clear();
+                                synchronized (graphs) {
+                                    synchronized (fullTextConfigs) {
+                                        synchronized (fullTextFilters) {
+                                            synchronized (adapters) {
+                                                synchronized (libraries) {
+                                                    synchronized (compactionPolicies) {
+                                                        synchronized (synonyms) {
+                                                            dataverses.clear();
+                                                            nodeGroups.clear();
+                                                            datasets.clear();
+                                                            graphs.clear();
+                                                            indexes.clear();
+                                                            datatypes.clear();
+                                                            functions.clear();
+                                                            fullTextConfigs.clear();
+                                                            fullTextFilters.clear();
+                                                            adapters.clear();
+                                                            libraries.clear();
+                                                            compactionPolicies.clear();
+                                                            synonyms.clear();
+                                                        }
                                                     }
                                                 }
                                             }
@@ -192,6 +198,13 @@
         }
     }
 
+    public Graph addGraphIfNotExists(Graph graph) {
+        synchronized (graphs) {
+            Map<String, Graph> graphMap = graphs.computeIfAbsent(graph.getDataverseName(), k -> new HashMap<>());
+            return (!graphMap.containsKey(graph.getGraphName())) ? graphMap.put(graph.getGraphName(), graph) : null;
+        }
+    }
+
     public Index addIndexIfNotExists(Index index) {
         synchronized (indexes) {
             return addIndexIfNotExistsInternal(index);
@@ -250,35 +263,38 @@
                 synchronized (indexes) {
                     synchronized (datatypes) {
                         synchronized (functions) {
-                            synchronized (fullTextConfigs) {
-                                synchronized (fullTextFilters) {
-                                    synchronized (adapters) {
-                                        synchronized (libraries) {
-                                            synchronized (feeds) {
-                                                synchronized (compactionPolicies) {
-                                                    synchronized (synonyms) {
-                                                        datasets.remove(dataverse.getDataverseName());
-                                                        indexes.remove(dataverse.getDataverseName());
-                                                        datatypes.remove(dataverse.getDataverseName());
-                                                        adapters.remove(dataverse.getDataverseName());
-                                                        compactionPolicies.remove(dataverse.getDataverseName());
-                                                        List<FunctionSignature> markedFunctionsForRemoval =
-                                                                new ArrayList<>();
-                                                        for (FunctionSignature signature : functions.keySet()) {
-                                                            if (signature.getDataverseName()
-                                                                    .equals(dataverse.getDataverseName())) {
-                                                                markedFunctionsForRemoval.add(signature);
+                            synchronized (graphs) {
+                                synchronized (fullTextConfigs) {
+                                    synchronized (fullTextFilters) {
+                                        synchronized (adapters) {
+                                            synchronized (libraries) {
+                                                synchronized (feeds) {
+                                                    synchronized (compactionPolicies) {
+                                                        synchronized (synonyms) {
+                                                            datasets.remove(dataverse.getDataverseName());
+                                                            graphs.remove(dataverse.getDataverseName());
+                                                            indexes.remove(dataverse.getDataverseName());
+                                                            datatypes.remove(dataverse.getDataverseName());
+                                                            adapters.remove(dataverse.getDataverseName());
+                                                            compactionPolicies.remove(dataverse.getDataverseName());
+                                                            List<FunctionSignature> markedFunctionsForRemoval =
+                                                                    new ArrayList<>();
+                                                            for (FunctionSignature signature : functions.keySet()) {
+                                                                if (signature.getDataverseName()
+                                                                        .equals(dataverse.getDataverseName())) {
+                                                                    markedFunctionsForRemoval.add(signature);
+                                                                }
                                                             }
+                                                            for (FunctionSignature signature : markedFunctionsForRemoval) {
+                                                                functions.remove(signature);
+                                                            }
+                                                            fullTextConfigs.remove(dataverse.getDataverseName());
+                                                            fullTextFilters.remove(dataverse.getDataverseName());
+                                                            libraries.remove(dataverse.getDataverseName());
+                                                            feeds.remove(dataverse.getDataverseName());
+                                                            synonyms.remove(dataverse.getDataverseName());
+                                                            return dataverses.remove(dataverse.getDataverseName());
                                                         }
-                                                        for (FunctionSignature signature : markedFunctionsForRemoval) {
-                                                            functions.remove(signature);
-                                                        }
-                                                        fullTextConfigs.remove(dataverse.getDataverseName());
-                                                        fullTextFilters.remove(dataverse.getDataverseName());
-                                                        libraries.remove(dataverse.getDataverseName());
-                                                        feeds.remove(dataverse.getDataverseName());
-                                                        synonyms.remove(dataverse.getDataverseName());
-                                                        return dataverses.remove(dataverse.getDataverseName());
                                                     }
                                                 }
                                             }
@@ -313,6 +329,16 @@
         }
     }
 
+    public Graph dropGraph(Graph graph) {
+        synchronized (graphs) {
+            Map<String, Graph> graphMap = graphs.get(graph.getDataverseName());
+            if (graphMap == null) {
+                return null;
+            }
+            return graphMap.remove(graph.getGraphName());
+        }
+    }
+
     public Index dropIndex(Index index) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
@@ -360,6 +386,16 @@
         }
     }
 
+    public Graph getGraph(DataverseName dataverseName, String graphName) {
+        synchronized (graphs) {
+            Map<String, Graph> g = graphs.get(dataverseName);
+            if (g == null) {
+                return null;
+            }
+            return g.get(graphName);
+        }
+    }
+
     public Index getIndex(DataverseName dataverseName, String datasetName, String indexName) {
         synchronized (indexes) {
             Map<String, Map<String, Index>> datasetMap = indexes.get(dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 0ab5c7b..cc3fa7e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -52,6 +52,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -1092,6 +1093,85 @@
         }
     }
 
+    @Override
+    public void addGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException {
+        try {
+            metadataNode.addGraph(ctx.getTxnId(), graph);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.addGraph(graph);
+    }
+
+    @Override
+    public void updateGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException {
+        try {
+            metadataNode.updateGraph(ctx.getTxnId(), graph);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.dropGraph(graph.getDataverseName(), graph.getGraphName());
+        ctx.addGraph(graph);
+    }
+
+    @Override
+    public void dropGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        try {
+            metadataNode.dropGraph(ctx.getTxnId(), dataverseName, graphName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+        ctx.dropGraph(dataverseName, graphName);
+    }
+
+    @Override
+    public Graph getGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        Graph graph = ctx.getGraph(dataverseName, graphName);
+        if (graph != null) {
+            // This graph is still uncommitted. Do not add this to the cache.
+            return graph;
+        }
+        if (ctx.graphIsDropped(dataverseName, graphName)) {
+            // Graph has been dropped by this transaction, but could still be in the cache.
+            return null;
+        }
+
+        graph = cache.getGraph(dataverseName, graphName);
+        if (graph != null) {
+            // Graph is already in the cache. Do not add it again.
+            return graph;
+        }
+
+        try {
+            graph = metadataNode.getGraph(ctx.getTxnId(), dataverseName, graphName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+
+        if (graph != null) {
+            // Graph has been fetched from the metadata node. Add this to the cache when the transaction commits.
+            ctx.addGraph(graph);
+        }
+        return graph;
+    }
+
+    @Override
+    public List<Graph> getDataverseGraphs(MetadataTransactionContext ctx, DataverseName dataverseName)
+            throws AlgebricksException {
+        try {
+            return metadataNode.getDataverseGraphs(ctx.getTxnId(), dataverseName);
+
+        } catch (RemoteException e) {
+            throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+        }
+    }
+
     // TODO: Optimize <-- use keys instead of object -->
     @Override
     public void dropDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 87f5129..807c223 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -73,6 +73,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.entities.Library;
@@ -92,6 +93,7 @@
 import org.apache.asterix.metadata.entitytupletranslators.FullTextConfigMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FullTextFilterMetadataEntityTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.GraphTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
 import org.apache.asterix.metadata.entitytupletranslators.MetadataTupleTranslatorProvider;
@@ -661,6 +663,12 @@
         try {
             confirmDataverseCanBeDeleted(txnId, dataverseName);
 
+            // Drop all graphs in this dataverse.
+            // Graphs may depend on functions and datasets.
+            for (Graph graph : getDataverseGraphs(txnId, dataverseName)) {
+                dropGraph(txnId, dataverseName, graph.getGraphName());
+            }
+
             // Drop all feeds and connections in this dataverse.
             // Feeds may depend on datatypes and adapters
             List<Feed> dataverseFeeds = getDataverseFeeds(txnId, dataverseName);
@@ -765,7 +773,8 @@
                 || !getDataverseFeeds(txnId, dataverseName).isEmpty()
                 || !getDataverseSynonyms(txnId, dataverseName).isEmpty()
                 || !getDataverseFullTextConfigs(txnId, dataverseName).isEmpty()
-                || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty();
+                || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty()
+                || !getDataverseGraphs(txnId, dataverseName).isEmpty();
     }
 
     @Override
@@ -1131,6 +1140,19 @@
         }
     }
 
+    public List<Graph> getAllGraphs(TxnId txnId) throws AlgebricksException {
+        try {
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            List<Graph> results = new ArrayList<>();
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, null, valueExtractor, results);
+            return results;
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
     private void confirmDataverseCanBeDeleted(TxnId txnId, DataverseName dataverseName) throws AlgebricksException {
         // If a dataset from a DIFFERENT dataverse
         // uses a type from this dataverse
@@ -1211,11 +1233,33 @@
                 }
             }
         }
+
+        // If a graph from a DIFFERENT dataverse uses a graph from this dataverse, throw an error.
+        for (Graph graph : getAllGraphs(txnId)) {
+            if (graph.getDataverseName().equals(dataverseName)) {
+                continue;
+            }
+            List<DependencyKind> dependenciesSchema = Graph.DEPENDENCIES_SCHEMA;
+            List<List<Triple<DataverseName, String, String>>> dependencies = graph.getDependencies();
+            for (int i = 0, n = dependencies.size(); i < n; i++) {
+                for (Triple<DataverseName, String, String> dependency : dependencies.get(i)) {
+                    if (dependency.first.equals(dataverseName)) {
+                        DependencyKind dependencyKind = dependenciesSchema.get(i);
+                        throw new AsterixException(
+                                org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_DATAVERSE_DEPENDENT_EXISTS,
+                                dependencyKind, dependencyKind.getDependencyDisplayName(dependency), "graph",
+                                MetadataUtil.getFullyQualifiedDisplayName(graph.getDataverseName(),
+                                        graph.getGraphName()));
+                    }
+                }
+            }
+        }
     }
 
     private void confirmFunctionCanBeDeleted(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
         confirmFunctionIsUnusedByViews(txnId, signature);
         confirmFunctionIsUnusedByFunctions(txnId, signature);
+        confirmFunctionIsUnusedByGraphs(txnId, signature);
 
         // if any other feed connection uses this function, throw an error
         List<FeedConnection> feedConnections = getAllFeedConnections(txnId);
@@ -1229,6 +1273,11 @@
         }
     }
 
+    private void confirmFunctionIsUnusedByGraphs(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
+                signature.getName(), Integer.toString(signature.getArity()));
+    }
+
     private void confirmFunctionIsUnusedByViews(TxnId txnId, FunctionSignature signature) throws AlgebricksException {
         confirmObjectIsUnusedByViews(txnId, "function", DependencyKind.FUNCTION, signature.getDataverseName(),
                 signature.getName(), Integer.toString(signature.getArity()));
@@ -1350,6 +1399,7 @@
             String datasetName) throws AlgebricksException {
         confirmDatasetIsUnusedByFunctions(txnId, datasetTypeDisplayName, dataverseName, datasetName);
         confirmDatasetIsUnusedByViews(txnId, datasetTypeDisplayName, dataverseName, datasetName);
+        confirmDatasetIsUnusedByGraphs(txnId, datasetTypeDisplayName, dataverseName, datasetName);
     }
 
     private void confirmDatasetIsUnusedByFunctions(TxnId txnId, String datasetKindDisplayName,
@@ -1364,6 +1414,12 @@
                 null);
     }
 
+    private void confirmDatasetIsUnusedByGraphs(TxnId txnId, String datasetKindDisplayName, DataverseName dataverseName,
+            String datasetName) throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, datasetKindDisplayName, DependencyKind.DATASET, dataverseName, datasetName,
+                null);
+    }
+
     private void confirmLibraryCanBeDeleted(TxnId txnId, DataverseName dataverseName, String libraryName)
             throws AlgebricksException {
         confirmLibraryIsUnusedByFunctions(txnId, dataverseName, libraryName);
@@ -1473,6 +1529,32 @@
         }
     }
 
+    private void confirmObjectIsUnusedByGraphs(TxnId txnId, String objectKindDisplayName, DependencyKind dependencyKind,
+            DataverseName dataverseName, String objectName, String objectArg) throws AlgebricksException {
+        int graphDependencyIdx = Graph.DEPENDENCIES_SCHEMA.indexOf(dependencyKind);
+        if (graphDependencyIdx < 0) {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+        }
+        for (Graph graph : getAllGraphs(txnId)) {
+            List<List<Triple<DataverseName, String, String>>> graphDependencies = graph.getDependencies();
+            if (graphDependencyIdx < graphDependencies.size()) {
+                List<Triple<DataverseName, String, String>> graphObjectDependencies =
+                        graphDependencies.get(graphDependencyIdx);
+                if (graphObjectDependencies != null) {
+                    for (Triple<DataverseName, String, String> dependency : graphObjectDependencies) {
+                        if (dependency.first.equals(dataverseName) && dependency.second.equals(objectName)
+                                && (objectArg == null || objectArg.equals(dependency.third))) {
+                            throw new AsterixException(
+                                    org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS,
+                                    objectKindDisplayName, dependencyKind.getDependencyDisplayName(dependency), "graph",
+                                    graph.getGraphName());
+                        }
+                    }
+                }
+            }
+        }
+    }
+
     private List<String> getNestedComplexDatatypeNamesForThisDatatype(TxnId txnId, DataverseName dataverseName,
             String datatypeName) throws AlgebricksException {
         // Return all field types that aren't builtin types
@@ -2397,6 +2479,7 @@
             throws AlgebricksException {
         confirmSynonymIsUnusedByFunctions(txnId, dataverseName, synonymName);
         confirmSynonymIsUnusedByViews(txnId, dataverseName, synonymName);
+        confirmSynonymIsUnusedByGraphs(txnId, dataverseName, synonymName);
     }
 
     private void confirmSynonymIsUnusedByFunctions(TxnId txnId, DataverseName dataverseName, String synonymName)
@@ -2409,6 +2492,11 @@
         confirmObjectIsUnusedByViews(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
     }
 
+    private void confirmSynonymIsUnusedByGraphs(TxnId txnId, DataverseName dataverseName, String synonymName)
+            throws AlgebricksException {
+        confirmObjectIsUnusedByGraphs(txnId, "synonym", DependencyKind.SYNONYM, dataverseName, synonymName, null);
+    }
+
     @Override
     public Synonym getSynonym(TxnId txnId, DataverseName dataverseName, String synonymName) throws AlgebricksException {
         try {
@@ -2441,6 +2529,76 @@
     }
 
     @Override
+    public void addGraph(TxnId txnId, Graph graph) throws AlgebricksException {
+        try {
+            GraphTupleTranslator tupleWriter = tupleTranslatorProvider.getGraphTupleTranslator(true);
+            ITupleReference graphTuple = tupleWriter.getTupleFromMetadataEntity(graph);
+            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, graphTuple);
+
+        } catch (HyracksDataException e) {
+            if (e.matches(ErrorCode.DUPLICATE_KEY)) {
+                throw new AsterixException(org.apache.asterix.common.exceptions.ErrorCode.GRAPH_EXISTS, e,
+                        graph.getGraphName());
+            } else {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+
+    @Override
+    public Graph getGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException {
+        List<Graph> results = new ArrayList<>();
+
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, graphName);
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey, valueExtractor, results);
+            if (results.isEmpty()) {
+                return null;
+            }
+            return results.get(0);
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
+    public void dropGraph(TxnId txnId, DataverseName dataverseName, String graphName) throws AlgebricksException {
+        try {
+            ITupleReference searchKey = createTuple(dataverseName, graphName);
+            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey);
+            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, tuple);
+
+        } catch (HyracksDataException e) {
+            if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                throw new AsterixException(org.apache.asterix.common.exceptions.ErrorCode.UNKNOWN_GRAPH, e, graphName);
+
+            } else {
+                throw new AlgebricksException(e);
+            }
+        }
+    }
+
+    @Override
+    public List<Graph> getDataverseGraphs(TxnId txnId, DataverseName dataverseName) throws AlgebricksException {
+        List<Graph> results = new ArrayList<>();
+
+        try {
+            ITupleReference searchKey = createTuple(dataverseName);
+            GraphTupleTranslator tupleReader = tupleTranslatorProvider.getGraphTupleTranslator(false);
+            IValueExtractor<Graph> valueExtractor = new MetadataEntityValueExtractor<>(tupleReader);
+            searchIndex(txnId, MetadataPrimaryIndexes.GRAPH_DATASET, searchKey, valueExtractor, results);
+            return results;
+
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    @Override
     public void updateDataset(TxnId txnId, Dataset dataset) throws AlgebricksException {
         try {
             // This method will delete previous entry of the dataset and insert the new one
@@ -2482,6 +2640,12 @@
         addDatatype(txnId, datatype);
     }
 
+    @Override
+    public void updateGraph(TxnId txnId, Graph graph) throws AlgebricksException {
+        dropGraph(txnId, graph.getDataverseName(), graph.getGraphName());
+        addGraph(txnId, graph);
+    }
+
     public ITxnIdFactory getTxnIdFactory() {
         return txnIdFactory;
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 48fb450..ad2e7ea 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -20,9 +20,11 @@
 package org.apache.asterix.metadata;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
@@ -37,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.NodeGroup;
@@ -96,6 +99,11 @@
         logAndApply(new MetadataLogicalOperation(dataset, true));
     }
 
+    public void addGraph(Graph graph) {
+        droppedCache.dropGraph(graph);
+        logAndApply(new MetadataLogicalOperation(graph, true));
+    }
+
     public void addIndex(Index index) {
         droppedCache.dropIndex(index);
         logAndApply(new MetadataLogicalOperation(index, true));
@@ -143,6 +151,12 @@
         logAndApply(new MetadataLogicalOperation(dataset, false));
     }
 
+    public void dropGraph(DataverseName dataverseName, String graphName) {
+        Graph graph = new Graph(new GraphIdentifier(dataverseName, graphName), null, Collections.emptyList());
+        droppedCache.addGraphIfNotExists(graph);
+        logAndApply(new MetadataLogicalOperation(graph, false));
+    }
+
     public void dropIndex(DataverseName dataverseName, String datasetName, String indexName) {
         Index index =
                 new Index(dataverseName, datasetName, indexName, null, null, false, false, MetadataUtil.PENDING_NO_OP);
@@ -236,6 +250,13 @@
         return droppedCache.getDataset(dataverseName, datasetName) != null;
     }
 
+    public boolean graphIsDropped(DataverseName dataverseName, String graphName) {
+        if (droppedCache.getDataverse(dataverseName) != null) {
+            return true;
+        }
+        return droppedCache.getGraph(dataverseName, graphName) != null;
+    }
+
     public boolean indexIsDropped(DataverseName dataverseName, String datasetName, String indexName) {
         if (droppedCache.getDataverse(dataverseName) != null) {
             return true;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 5ff8a03..182cb62 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -871,6 +872,22 @@
     void rebindMetadataNode();
 
     /**
+     * Graph related metadata operations.
+     */
+    void addGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException;
+
+    void updateGraph(MetadataTransactionContext ctx, Graph graph) throws AlgebricksException;
+
+    void dropGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
+
+    Graph getGraph(MetadataTransactionContext ctx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException;
+
+    List<Graph> getDataverseGraphs(MetadataTransactionContext ctx, DataverseName dataverseName)
+            throws AlgebricksException;
+
+    /**
      * Feed Connection Related Metadata operations
      */
     void addFeedConnection(MetadataTransactionContext ctx, FeedConnection feedConnection) throws AlgebricksException;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 561a4fa..f04f03b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Library;
 import org.apache.asterix.metadata.entities.Node;
@@ -969,6 +970,19 @@
     <T extends IExtensionMetadataEntity> List<T> getEntities(TxnId txnId, IExtensionMetadataSearchKey searchKey)
             throws AlgebricksException, RemoteException;
 
+    void addGraph(TxnId txnId, Graph graph) throws AlgebricksException, RemoteException;
+
+    void updateGraph(TxnId txnId, Graph graph) throws AlgebricksException, RemoteException;
+
+    Graph getGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException;
+
+    void dropGraph(TxnId txnId, DataverseName dataverseName, String graphName)
+            throws AlgebricksException, RemoteException;
+
+    List<Graph> getDataverseGraphs(TxnId txnId, DataverseName dataverseName)
+            throws AlgebricksException, RemoteException;
+
     void addFeedConnection(TxnId txnId, FeedConnection feedConnection) throws AlgebricksException, RemoteException;
 
     FeedConnection getFeedConnection(TxnId txnId, DataverseName dataverseName, String feedName, String datasetName)
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index fd0d30b..52ecb21 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -127,7 +127,7 @@
                     MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
                     MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
                     MetadataPrimaryIndexes.SYNONYM_DATASET, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET,
-                    MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET };
+                    MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, MetadataPrimaryIndexes.GRAPH_DATASET };
 
     private MetadataBootstrap() {
     }
@@ -178,6 +178,7 @@
                 insertNewCompactionPoliciesIfNotExist(mdTxnCtx);
                 insertSynonymEntitiesIfNotExist(mdTxnCtx);
                 insertFullTextConfigAndFilterIfNotExist(mdTxnCtx);
+                insertGraphEntitiesIfNotExists(mdTxnCtx);
             }
             // #. initialize datasetIdFactory
             MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
@@ -305,6 +306,22 @@
         }
     }
 
+    private static void insertGraphEntitiesIfNotExists(MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+        IAType[] graphEntityTypes = new IAType[] { MetadataPrimaryIndexes.GRAPH_DATASET.getPayloadRecordType(),
+                MetadataRecordTypes.VERTICES_RECORDTYPE, MetadataRecordTypes.EDGES_RECORDTYPE };
+        for (IAType graphEntityType : graphEntityTypes) {
+            if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                    graphEntityType.getTypeName()) == null) {
+                MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+                        graphEntityType.getTypeName(), graphEntityType, false));
+            }
+        }
+        if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+                MetadataConstants.GRAPH_DATASET_NAME) == null) {
+            insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.GRAPH_DATASET });
+        }
+    }
+
     private static void insertSynonymEntitiesIfNotExist(MetadataTransactionContext mdTxnCtx)
             throws AlgebricksException {
         IAType synonymDatasetRecordType = MetadataPrimaryIndexes.SYNONYM_DATASET.getPayloadRecordType();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index a191e43..344b423 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -65,6 +65,8 @@
             new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME, 16, 16);
     public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_FILTER =
             new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME, 17, 17);
+    public static final MetadataIndexImmutableProperties PROPERTIES_GRAPH =
+            new MetadataIndexImmutableProperties(MetadataConstants.GRAPH_DATASET_NAME, 18, 18);
 
     public static final IMetadataIndex DATAVERSE_DATASET =
             new MetadataIndex(PROPERTIES_DATAVERSE, 2, new IAType[] { BuiltinType.ASTRING },
@@ -157,6 +159,12 @@
                             Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_FILTER_NAME)),
                     0, MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE, true, new int[] { 0, 1 });
 
+    public static final IMetadataIndex GRAPH_DATASET =
+            new MetadataIndex(PROPERTIES_GRAPH, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+                    Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+                            Arrays.asList(MetadataRecordTypes.FIELD_NAME_GRAPH_NAME)),
+                    0, MetadataRecordTypes.GRAPH_RECORDTYPE, true, new int[] { 0, 1 });
+
     private MetadataPrimaryIndexes() {
     }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 48d4fc5..862fc45 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -53,6 +53,9 @@
     public static final String FIELD_NAME_DEPENDENCIES = "Dependencies";
     public static final String FIELD_NAME_DERIVED = "Derived";
     public static final String FIELD_NAME_DESCRIPTION = "Description";
+    public static final String FIELD_NAME_DESTINATION_KEY = "DestinationKey";
+    public static final String FIELD_NAME_DESTINATION_LABEL = "DestinationLabel";
+    public static final String FIELD_NAME_EDGES = "Edges";
     public static final String FIELD_NAME_EXTERNAL_DETAILS = "ExternalDetails";
     public static final String FIELD_NAME_FEED_NAME = "FeedName";
     public static final String FIELD_NAME_FEED_TYPE = "FeedType";
@@ -66,6 +69,7 @@
     public static final String FIELD_NAME_FILE_STRUCTURE = "FileStructure";
     public static final String FIELD_NAME_FOREIGN_KEY = "ForeignKey";
     public static final String FIELD_NAME_FOREIGN_KEYS = "ForeignKeys";
+    public static final String FIELD_NAME_GRAPH_NAME = "GraphName";
     public static final String FIELD_NAME_GROUP_NAME = "GroupName";
     public static final String FIELD_NAME_HINTS = "Hints";
     public static final String FIELD_NAME_INDEX_NAME = "IndexName";
@@ -79,6 +83,7 @@
     public static final String FIELD_NAME_IS_PRIMARY = "IsPrimary";
     public static final String FIELD_NAME_KIND = "Kind";
     public static final String FIELD_NAME_LANGUAGE = "Language";
+    public static final String FIELD_NAME_LABEL = "Label";
     public static final String FIELD_NAME_HASH = "MD5Hash";
     public static final String FIELD_NAME_LIBRARY_DATAVERSE_NAME = "LibraryDataverseName";
     public static final String FIELD_NAME_LIBRARY_NAME = "LibraryName";
@@ -107,6 +112,8 @@
     public static final String FIELD_NAME_RETURN_TYPE_DATAVERSE_NAME = "ReturnTypeDataverseName";
     public static final String FIELD_NAME_SEARCH_KEY = "SearchKey";
     public static final String FIELD_NAME_STATUS = "Status";
+    public static final String FIELD_NAME_SOURCE_KEY = "SourceKey";
+    public static final String FIELD_NAME_SOURCE_LABEL = "SourceLabel";
     public static final String FIELD_NAME_SYNONYM_NAME = "SynonymName";
     public static final String FIELD_NAME_TAG = "Tag";
     public static final String FIELD_NAME_TIMESTAMP = "Timestamp";
@@ -114,6 +121,7 @@
     public static final String FIELD_NAME_TYPE = "Type";
     public static final String FIELD_NAME_UNORDERED_LIST = "UnorderedList";
     public static final String FIELD_NAME_VALUE = "Value";
+    public static final String FIELD_NAME_VERTICES = "Vertices";
     public static final String FIELD_NAME_VIEW_DETAILS = "ViewDetails";
     public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
     public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
@@ -560,6 +568,49 @@
                     FIELD_NAME_FULL_TEXT_FILTER_TYPE },
             new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, true);
 
+    //-------------------------------------- Graph ---------------------------------------//
+    public static final String RECORD_NAME_GRAPH = "GraphRecordType";
+
+    public static final int GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX = 0;
+    public static final int GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX = 1;
+    public static final int GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX = 2;
+    public static final ARecordType VERTICES_RECORDTYPE = createRecordType("VerticesRecordType",
+            new String[] { FIELD_NAME_LABEL, FIELD_NAME_PRIMARY_KEY, FIELD_NAME_DEFINITION },
+            new IAType[] { BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null), BuiltinType.ASTRING },
+            true);
+
+    public static final int GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX = 0;
+    public static final int GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX = 1;
+    public static final int GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX = 2;
+    public static final int GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX = 3;
+    public static final int GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX = 4;
+    public static final int GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX = 5;
+    public static final int GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX = 6;
+    public static final ARecordType EDGES_RECORDTYPE = createRecordType("EdgesRecordType",
+            new String[] { FIELD_NAME_LABEL, FIELD_NAME_DESTINATION_LABEL, FIELD_NAME_SOURCE_LABEL,
+                    FIELD_NAME_PRIMARY_KEY, FIELD_NAME_DESTINATION_KEY, FIELD_NAME_SOURCE_KEY, FIELD_NAME_DEFINITION },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                    AUnionType.createNullableType(BuiltinType.ASTRING, null) },
+            true);
+
+    public static final int GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
+    public static final int GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX = 1;
+    public static final int GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX = 2;
+    public static final int GRAPH_ARECORD_VERTICES_FIELD_INDEX = 3;
+    public static final int GRAPH_ARECORD_EDGES_FIELD_INDEX = 4;
+    public static final ARecordType GRAPH_RECORDTYPE = createRecordType(RECORD_NAME_GRAPH,
+            new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_GRAPH_NAME, FIELD_NAME_DEPENDENCIES,
+                    FIELD_NAME_VERTICES, FIELD_NAME_EDGES },
+            new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
+                    new AOrderedListType(new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null),
+                            null),
+                    new AOrderedListType(VERTICES_RECORDTYPE, null), new AOrderedListType(EDGES_RECORDTYPE, null) },
+            true);
+
     // private members
     private MetadataRecordTypes() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index ee629b1..5353175 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -38,6 +38,7 @@
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.NodeGroup;
 import org.apache.asterix.metadata.entities.Synonym;
@@ -163,6 +164,11 @@
         return MetadataManager.INSTANCE.getSynonym(mdTxnCtx, dataverseName, synonymName);
     }
 
+    public static Graph findGraph(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        return MetadataManager.INSTANCE.getGraph(mdTxnCtx, dataverseName, graphName);
+    }
+
     public static FullTextConfigMetadataEntity findFullTextConfigDescriptor(MetadataTransactionContext mdTxnCtx,
             DataverseName dataverseName, String ftConfigName) throws AlgebricksException {
         // If the config name is null, then the default config will be returned
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 011d862..7ba5bfa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -88,6 +88,7 @@
 import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
 import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
 import org.apache.asterix.metadata.entities.Function;
+import org.apache.asterix.metadata.entities.Graph;
 import org.apache.asterix.metadata.entities.Index;
 import org.apache.asterix.metadata.entities.Synonym;
 import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
@@ -471,6 +472,10 @@
         return MetadataManagerUtil.findFullTextFilterDescriptor(mdTxnCtx, dataverseName, ftFilterName);
     }
 
+    public Graph findGraph(DataverseName dataverseName, String graphName) throws AlgebricksException {
+        return MetadataManagerUtil.findGraph(mdTxnCtx, dataverseName, graphName);
+    }
+
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return BuiltinFunctions.getBuiltinFunctionInfo(fid);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
new file mode 100644
index 0000000..7103cf5
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Graph.java
@@ -0,0 +1,347 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.asterix.common.graph.GraphElementIdentifier;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+
+/**
+ * Metadata describing a graph view, composed of vertices and edges.
+ */
+public class Graph implements IMetadataEntity<Graph> {
+    private static final long serialVersionUID = 1L;
+
+    private final GraphIdentifier identifier;
+    private final Schema graphSchema;
+    private final List<List<Triple<DataverseName, String, String>>> dependencies;
+
+    // Vertices and edges from a graph have dependencies on the datasets, functions, and synonyms used in their bodies.
+    public final static List<DependencyKind> DEPENDENCIES_SCHEMA =
+            Arrays.asList(DependencyKind.DATASET, DependencyKind.FUNCTION, DependencyKind.SYNONYM);
+
+    public Graph(GraphIdentifier identifier, Schema graphSchema,
+            List<List<Triple<DataverseName, String, String>>> dependencies) {
+        this.identifier = Objects.requireNonNull(identifier);
+        this.dependencies = dependencies;
+        this.graphSchema = graphSchema;
+    }
+
+    @Override
+    public Graph addToCache(MetadataCache cache) {
+        return cache.addGraphIfNotExists(this);
+    }
+
+    @Override
+    public Graph dropFromCache(MetadataCache cache) {
+        return cache.dropGraph(this);
+    }
+
+    public DataverseName getDataverseName() {
+        return identifier.getDataverseName();
+    }
+
+    public String getGraphName() {
+        return identifier.getGraphName();
+    }
+
+    public GraphIdentifier getIdentifier() {
+        return identifier;
+    }
+
+    public Schema getGraphSchema() {
+        return graphSchema;
+    }
+
+    public List<List<Triple<DataverseName, String, String>>> getDependencies() {
+        return dependencies;
+    }
+
+    public static List<List<Triple<DataverseName, String, String>>> createDependencies(
+            List<Triple<DataverseName, String, String>> datasetDependencies,
+            List<Triple<DataverseName, String, String>> functionDependencies,
+            List<Triple<DataverseName, String, String>> synonymDependencies) {
+        List<List<Triple<DataverseName, String, String>>> depList = new ArrayList<>(DEPENDENCIES_SCHEMA.size());
+        depList.add(datasetDependencies);
+        depList.add(functionDependencies);
+        if (!synonymDependencies.isEmpty()) {
+            depList.add(synonymDependencies);
+        }
+        return depList;
+    }
+
+    public static class Schema {
+        private final Map<GraphElementIdentifier, Element> elementMap = new HashMap<>();
+
+        // The element map above is composed of the vertices and edges below.
+        private final List<Vertex> vertexList = new ArrayList<>();
+        private final List<Edge> edgeList = new ArrayList<>();
+
+        public List<Vertex> getVertices() {
+            return vertexList;
+        }
+
+        public List<Edge> getEdges() {
+            return edgeList;
+        }
+
+        public Element getElement(GraphElementIdentifier identifier) {
+            return elementMap.get(identifier);
+        }
+
+        private Schema() {
+        }
+
+        public static class Builder {
+            private final HashMap<String, List<Vertex>> vertexLabelMap = new HashMap<>();
+            private final HashMap<String, List<Edge>> edgeLabelMap = new HashMap<>();
+            private final Schema workingSchema = new Schema();
+            private final GraphIdentifier graphIdentifier;
+
+            public Builder(GraphIdentifier graphIdentifier) {
+                this.graphIdentifier = graphIdentifier;
+            }
+
+            /**
+             * @return Null if the primary keys of an existing vertex conflict with the vertex to-be-added.
+             *         The vertex to-be-added otherwise.
+             */
+            public Vertex addVertex(String labelName, List<List<String>> primaryKeyFieldNames, String definition) {
+                if (!vertexLabelMap.containsKey(labelName)) {
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.VERTEX, labelName, 0);
+                    Vertex v = new Vertex(identifier, primaryKeyFieldNames, definition);
+                    workingSchema.vertexList.add(v);
+
+                    // Create a new map entry.
+                    List<Vertex> vertexList = new ArrayList<>();
+                    vertexList.add(v);
+                    vertexLabelMap.put(labelName, vertexList);
+                    return v;
+
+                } else {
+                    List<Vertex> sameLabelVertices = vertexLabelMap.get(labelName);
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.VERTEX, labelName, sameLabelVertices.size());
+                    Vertex v = new Vertex(identifier, primaryKeyFieldNames, definition);
+                    workingSchema.vertexList.add(v);
+
+                    // Insert into the existing map entry.
+                    sameLabelVertices.add(v);
+                    List<List<String>> primaryKeyFromOtherVertices = sameLabelVertices.get(0).getPrimaryKeyFieldNames();
+                    return primaryKeyFromOtherVertices.equals(primaryKeyFieldNames) ? v : null;
+                }
+            }
+
+            /**
+             * @return Null if there exists no vertex with the given source label or destination label.
+             *         The edge to-be-added otherwise.
+             */
+            public Edge addEdge(String edgeLabelName, String destinationLabelName, String sourceLabelName,
+                    List<List<String>> destinationKeyFieldNames) {
+                if (!vertexLabelMap.containsKey(sourceLabelName)) {
+                    return null;
+                }
+
+                Vertex representativeSourceVertex = vertexLabelMap.get(sourceLabelName).get(0);
+                return addEdge(edgeLabelName, destinationLabelName, sourceLabelName,
+                        representativeSourceVertex.getPrimaryKeyFieldNames(), destinationKeyFieldNames,
+                        representativeSourceVertex.getPrimaryKeyFieldNames(), null);
+            }
+
+            /**
+             * @return Null if there exists no vertex with the given source label or destination label.
+             *         The edge to-be-added otherwise.
+             */
+            public Edge addEdge(String edgeLabelName, String destinationLabelName, String sourceLabelName,
+                    List<List<String>> primaryKeyFieldNames, List<List<String>> destinationKeyFieldNames,
+                    List<List<String>> sourceKeyFieldNames, String definition) {
+                if (!vertexLabelMap.containsKey(sourceLabelName) || !vertexLabelMap.containsKey(destinationLabelName)) {
+                    return null;
+                }
+                List<Vertex> sourceVertices = vertexLabelMap.get(sourceLabelName);
+                List<Vertex> destinationVertices = vertexLabelMap.get(destinationLabelName);
+
+                if (!edgeLabelMap.containsKey(edgeLabelName)) {
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.EDGE, edgeLabelName, 0);
+                    Edge e = new Edge(identifier, primaryKeyFieldNames, destinationKeyFieldNames, sourceKeyFieldNames,
+                            destinationVertices, sourceVertices, definition);
+                    workingSchema.edgeList.add(e);
+
+                    // Create a new map entry.
+                    List<Edge> edgeList = new ArrayList<>();
+                    edgeList.add(e);
+                    edgeLabelMap.put(edgeLabelName, edgeList);
+                    return e;
+
+                } else {
+                    List<Edge> sameLabelEdges = edgeLabelMap.get(edgeLabelName);
+                    GraphElementIdentifier identifier = new GraphElementIdentifier(graphIdentifier,
+                            GraphElementIdentifier.Kind.EDGE, edgeLabelName, sameLabelEdges.size());
+                    Edge e = new Edge(identifier, primaryKeyFieldNames, destinationKeyFieldNames, sourceKeyFieldNames,
+                            destinationVertices, sourceVertices, definition);
+                    workingSchema.edgeList.add(e);
+
+                    // Insert into the existing map entry.
+                    sameLabelEdges.add(e);
+                    return e;
+                }
+            }
+
+            public Schema build() {
+                // Build the element map, composed of our vertices and edges.
+                workingSchema.elementMap.clear();
+                workingSchema.getEdges().forEach(e -> workingSchema.elementMap.put(e.identifier, e));
+                workingSchema.getVertices().forEach(v -> workingSchema.elementMap.put(v.identifier, v));
+                return workingSchema;
+            }
+        }
+    }
+
+    public static final class Vertex implements Element {
+        private final GraphElementIdentifier identifier;
+        private final List<List<String>> primaryKeyFieldNames;
+        private final String definition;
+
+        private Vertex(GraphElementIdentifier identifier, List<List<String>> primaryKeyFieldNames, String definition) {
+            this.identifier = Objects.requireNonNull(identifier);
+            this.primaryKeyFieldNames = Objects.requireNonNull(primaryKeyFieldNames);
+            this.definition = definition;
+        }
+
+        @Override
+        public GraphElementIdentifier getIdentifier() {
+            return identifier;
+        }
+
+        @Override
+        public String getLabelName() {
+            return identifier.getLabelName();
+        }
+
+        public List<List<String>> getPrimaryKeyFieldNames() {
+            return primaryKeyFieldNames;
+        }
+
+        @Override
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            return "(:" + getLabelName() + ") AS " + definition;
+        }
+    }
+
+    public static final class Edge implements Element {
+        private final List<List<String>> primaryKeyFieldNames;
+        private final List<List<String>> destinationKeyFieldNames;
+        private final List<List<String>> sourceKeyFieldNames;
+
+        private final GraphElementIdentifier identifier;
+        private final List<Vertex> destinationVertices;
+        private final List<Vertex> sourceVertices;
+        private final String definition;
+
+        private Edge(GraphElementIdentifier identifier, List<List<String>> primaryKeyFieldNames,
+                List<List<String>> destinationKeyFieldNames, List<List<String>> sourceKeyFieldNames,
+                List<Vertex> destinationVertices, List<Vertex> sourceVertices, String definition) {
+            this.primaryKeyFieldNames = Objects.requireNonNull(primaryKeyFieldNames);
+            this.destinationKeyFieldNames = Objects.requireNonNull(destinationKeyFieldNames);
+            this.sourceKeyFieldNames = Objects.requireNonNull(sourceKeyFieldNames);
+            this.destinationVertices = Objects.requireNonNull(destinationVertices);
+            this.sourceVertices = Objects.requireNonNull(sourceVertices);
+            this.identifier = Objects.requireNonNull(identifier);
+            this.definition = definition;
+        }
+
+        @Override
+        public GraphElementIdentifier getIdentifier() {
+            return identifier;
+        }
+
+        @Override
+        public String getLabelName() {
+            return identifier.getLabelName();
+        }
+
+        public String getDestinationLabelName() {
+            return destinationVertices.get(0).getLabelName();
+        }
+
+        public String getSourceLabelName() {
+            return sourceVertices.get(0).getLabelName();
+        }
+
+        public List<List<String>> getPrimaryKeyFieldNames() {
+            return primaryKeyFieldNames;
+        }
+
+        public List<List<String>> getDestinationKeyFieldNames() {
+            return destinationKeyFieldNames;
+        }
+
+        public List<List<String>> getSourceKeyFieldNames() {
+            return sourceKeyFieldNames;
+        }
+
+        public List<Vertex> getDestinationVertices() {
+            return destinationVertices;
+        }
+
+        public List<Vertex> getSourceVertices() {
+            return sourceVertices;
+        }
+
+        @Override
+        public String getDefinition() {
+            return definition;
+        }
+
+        @Override
+        public String toString() {
+            String edgeBodyPattern = "[:" + getLabelName() + "]";
+            String sourceNodePattern = "(:" + getSourceLabelName() + ")";
+            String destinationNodePattern = "(:" + getDestinationLabelName() + ")";
+            String edgePattern = sourceNodePattern + "-" + edgeBodyPattern + "->" + destinationNodePattern;
+            return (definition == null) ? edgePattern : (edgePattern + " AS " + definition);
+        }
+    }
+
+    public interface Element {
+        GraphElementIdentifier getIdentifier();
+
+        String getLabelName();
+
+        String getDefinition();
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
new file mode 100644
index 0000000..05f0da4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/GraphTupleTranslator.java
@@ -0,0 +1,406 @@
+/*
+ * 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.graph.GraphIdentifier;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.Graph;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class GraphTupleTranslator extends AbstractTupleTranslator<Graph> {
+    // Payload field containing serialized Graph.
+    private static final int GRAPH_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+
+    // For constructing our dependency, edge, and vertex lists.
+    protected OrderedListBuilder listBuilder;
+    protected OrderedListBuilder innerListBuilder;
+    protected OrderedListBuilder nameListBuilder;
+    protected IARecordBuilder subRecordBuilder;
+    protected AOrderedListType stringListList;
+    protected AOrderedListType stringList;
+
+    protected GraphTupleTranslator(boolean getTuple) {
+        super(getTuple, MetadataPrimaryIndexes.GRAPH_DATASET, GRAPH_PAYLOAD_TUPLE_FIELD_INDEX);
+        if (getTuple) {
+            listBuilder = new OrderedListBuilder();
+            innerListBuilder = new OrderedListBuilder();
+            nameListBuilder = new OrderedListBuilder();
+            subRecordBuilder = new RecordBuilder();
+            stringListList = new AOrderedListType(new AOrderedListType(BuiltinType.ASTRING, null), null);
+            stringList = new AOrderedListType(BuiltinType.ASTRING, null);
+        }
+    }
+
+    @Override
+    protected Graph createMetadataEntityFromARecord(ARecord graphRecord)
+            throws HyracksDataException, AlgebricksException {
+        // Read in the dataverse name.
+        DataverseName dataverseName = DataverseName.createFromCanonicalForm(
+                ((AString) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX))
+                        .getStringValue());
+
+        // Read in the graph name.
+        String graphName =
+                ((AString) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX))
+                        .getStringValue();
+
+        // Read in the dependencies.
+        IACursor dependenciesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX))
+                        .getCursor();
+        List<List<Triple<DataverseName, String, String>>> dependencies = new ArrayList<>();
+        while (dependenciesCursor.next()) {
+            List<Triple<DataverseName, String, String>> dependencyList = new ArrayList<>();
+            IACursor qualifiedDependencyCursor = ((AOrderedList) dependenciesCursor.get()).getCursor();
+            while (qualifiedDependencyCursor.next()) {
+                Triple<DataverseName, String, String> dependency =
+                        getDependency((AOrderedList) qualifiedDependencyCursor.get());
+                dependencyList.add(dependency);
+            }
+            dependencies.add(dependencyList);
+        }
+
+        // Read in the vertex and edge lists.
+        GraphIdentifier graphIdentifier = new GraphIdentifier(dataverseName, graphName);
+        Graph.Schema graphSchema = readGraphSchema(graphRecord, graphIdentifier);
+        return new Graph(graphIdentifier, graphSchema, dependencies);
+    }
+
+    private Graph.Schema readGraphSchema(ARecord graphRecord, GraphIdentifier graphIdentifier) throws AsterixException {
+        Graph.Schema.Builder schemaBuilder = new Graph.Schema.Builder(graphIdentifier);
+
+        // Read in the vertex list.
+        IACursor verticesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX))
+                        .getCursor();
+        while (verticesCursor.next()) {
+            ARecord vertex = (ARecord) verticesCursor.get();
+
+            // Read in the label name.
+            String labelName =
+                    ((AString) vertex.getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the primary key fields.
+            List<List<String>> primaryKeyFields = new ArrayList<>();
+            IACursor primaryKeyCursor = ((AOrderedList) vertex
+                    .getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX)).getCursor();
+            while (primaryKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) primaryKeyCursor.get()).getCursor();
+                primaryKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the vertex definition.
+            String definition =
+                    ((AString) vertex.getValueByPos(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX))
+                            .getStringValue();
+
+            if (schemaBuilder.addVertex(labelName, primaryKeyFields, definition) == null) {
+                throw new AsterixException(ErrorCode.METADATA_ERROR,
+                        "Conflicting primary key definitions for node of label " + labelName);
+            }
+        }
+
+        // Read in the edge list.
+        IACursor edgesCursor =
+                ((AOrderedList) graphRecord.getValueByPos(MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX))
+                        .getCursor();
+        while (edgesCursor.next()) {
+            ARecord edge = (ARecord) edgesCursor.get();
+
+            // Read in the label name.
+            String labelName = ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX))
+                    .getStringValue();
+
+            // Read in the destination label name.
+            String destinationLabelName =
+                    ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the source label name.
+            String sourceLabelName =
+                    ((AString) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX))
+                            .getStringValue();
+
+            // Read in the primary key fields.
+            List<List<String>> primaryKeyFields = new ArrayList<>();
+            IACursor primaryKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (primaryKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) primaryKeyCursor.get()).getCursor();
+                primaryKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the destination key fields.
+            List<List<String>> destinationKeyFields = new ArrayList<>();
+            IACursor destinationKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (destinationKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) destinationKeyCursor.get()).getCursor();
+                destinationKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the source key fields.
+            List<List<String>> sourceKeyFields = new ArrayList<>();
+            IACursor sourceKeyCursor =
+                    ((AOrderedList) edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX))
+                            .getCursor();
+            while (sourceKeyCursor.next()) {
+                IACursor nameCursor = ((AOrderedList) sourceKeyCursor.get()).getCursor();
+                sourceKeyFields.add(readNameList(nameCursor));
+            }
+
+            // Read in the edge definition.
+            String definition;
+            IAObject definitionObj = edge.getValueByPos(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX);
+            if (definitionObj.getType().equals(BuiltinType.ANULL)) {
+                definition = null;
+
+            } else {
+                definition = ((AString) definitionObj).getStringValue();
+            }
+
+            if (schemaBuilder.addEdge(labelName, destinationLabelName, sourceLabelName, primaryKeyFields,
+                    destinationKeyFields, sourceKeyFields, definition) == null) {
+                throw new AsterixException(ErrorCode.METADATA_ERROR,
+                        "Vertex label in edge with label " + labelName + " not found");
+            }
+        }
+
+        return schemaBuilder.build();
+    }
+
+    private List<String> readNameList(IACursor nameCursor) {
+        List<String> fieldName = new ArrayList<>();
+        while (nameCursor.next()) {
+            String subName = ((AString) nameCursor.get()).getStringValue();
+            fieldName.add(subName);
+        }
+        return fieldName;
+    }
+
+    @Override
+    public ITupleReference getTupleFromMetadataEntity(Graph graph) throws AlgebricksException, HyracksDataException {
+        // Write our primary key (dataverse name, graph name).
+        String dataverseCanonicalName = graph.getDataverseName().getCanonicalForm();
+        tupleBuilder.reset();
+        aString.setValue(dataverseCanonicalName);
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        aString.setValue(graph.getGraphName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+
+        // Write the payload in the third field of the tuple.
+        recordBuilder.reset(MetadataRecordTypes.GRAPH_RECORDTYPE);
+
+        // Write the dataverse name.
+        fieldValue.reset();
+        aString.setValue(dataverseCanonicalName);
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_DATAVERSENAME_FIELD_INDEX, fieldValue);
+
+        // Write the graph name.
+        fieldValue.reset();
+        aString.setValue(graph.getGraphName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_GRAPHNAME_FIELD_INDEX, fieldValue);
+
+        // Write our dependencies.
+        ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX]);
+        for (List<Triple<DataverseName, String, String>> dependencies : graph.getDependencies()) {
+            List<String> subNames = new ArrayList<>();
+            innerListBuilder.reset(stringListList);
+            for (Triple<DataverseName, String, String> dependency : dependencies) {
+                subNames.clear();
+                getDependencySubNames(dependency, subNames);
+                writeNameList(subNames, itemValue);
+                innerListBuilder.addItem(itemValue);
+            }
+            itemValue.reset();
+            innerListBuilder.write(itemValue.getDataOutput(), true);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_DEPENDENCIES_FIELD_INDEX, fieldValue);
+
+        // Write our vertex set.
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX]);
+        for (Graph.Vertex vertex : graph.getGraphSchema().getVertices()) {
+            writeVertexRecord(vertex, itemValue);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_VERTICES_FIELD_INDEX, fieldValue);
+
+        // Write our edge set.
+        listBuilder.reset((AOrderedListType) MetadataRecordTypes.GRAPH_RECORDTYPE
+                .getFieldTypes()[MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX]);
+        for (Graph.Edge edge : graph.getGraphSchema().getEdges()) {
+            writeEdgeRecord(edge, itemValue);
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        recordBuilder.addField(MetadataRecordTypes.GRAPH_ARECORD_EDGES_FIELD_INDEX, fieldValue);
+
+        // Finally, write our record.
+        recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        tupleBuilder.addFieldEndOffset();
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+    }
+
+    private void writeVertexRecord(Graph.Vertex vertex, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        subRecordBuilder.reset(MetadataRecordTypes.VERTICES_RECORDTYPE);
+
+        // Write the label name.
+        fieldValue.reset();
+        aString.setValue(vertex.getLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the primary key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : vertex.getPrimaryKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_PRIMARY_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the vertex definition.
+        fieldValue.reset();
+        aString.setValue(vertex.getDefinition());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_VERTICES_ARECORD_DEFINITION_FIELD_INDEX, fieldValue);
+
+        itemValue.reset();
+        subRecordBuilder.write(itemValue.getDataOutput(), true);
+    }
+
+    private void writeEdgeRecord(Graph.Edge edge, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        subRecordBuilder.reset(MetadataRecordTypes.EDGES_RECORDTYPE);
+
+        // Write the label name.
+        fieldValue.reset();
+        aString.setValue(edge.getLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the destination label name.
+        fieldValue.reset();
+        aString.setValue(edge.getDestinationLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the source label name.
+        fieldValue.reset();
+        aString.setValue(edge.getSourceLabelName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_LABEL_FIELD_INDEX, fieldValue);
+
+        // Write the primary key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getPrimaryKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_PRIMARY_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the destination key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getDestinationKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEST_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the source key fields.
+        fieldValue.reset();
+        innerListBuilder.reset(stringListList);
+        for (List<String> keyField : edge.getSourceKeyFieldNames()) {
+            writeNameList(keyField, itemValue);
+            innerListBuilder.addItem(itemValue);
+        }
+        innerListBuilder.write(fieldValue.getDataOutput(), true);
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_SOURCE_KEY_FIELD_INDEX, fieldValue);
+
+        // Write the edge definition.
+        fieldValue.reset();
+        if (edge.getDefinition() == null) {
+            nullSerde.serialize(ANull.NULL, fieldValue.getDataOutput());
+
+        } else {
+            aString.setValue(edge.getDefinition());
+            stringSerde.serialize(aString, fieldValue.getDataOutput());
+        }
+        subRecordBuilder.addField(MetadataRecordTypes.GRAPH_EDGES_ARECORD_DEFINITION_FIELD_INDEX, fieldValue);
+
+        itemValue.reset();
+        subRecordBuilder.write(itemValue.getDataOutput(), true);
+    }
+
+    private void writeNameList(List<String> name, ArrayBackedValueStorage itemValue) throws HyracksDataException {
+        nameListBuilder.reset(stringList);
+        for (String subName : name) {
+            itemValue.reset();
+            aString.setValue(subName);
+            stringSerde.serialize(aString, itemValue.getDataOutput());
+            nameListBuilder.addItem(itemValue);
+        }
+        itemValue.reset();
+        nameListBuilder.write(itemValue.getDataOutput(), true);
+    }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
index c2a9ee3..224c0a7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -92,4 +92,8 @@
     public SynonymTupleTranslator getSynonymTupleTranslator(boolean getTuple) {
         return new SynonymTupleTranslator(getTuple);
     }
+
+    public GraphTupleTranslator getGraphTupleTranslator(boolean getTuple) {
+        return new GraphTupleTranslator(getTuple);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
index 5e44e32..fb7fa16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
@@ -40,7 +40,8 @@
         ADAPTER,
         MERGE_POLICY,
         NODE_GROUP,
-        SYNONYM
+        SYNONYM,
+        GRAPH
     }
 
     private final EntityKind entityKind;
@@ -150,4 +151,8 @@
     static MetadataLockKey createMergePolicyLockKey(String mergePolicyName) {
         return new MetadataLockKey(EntityKind.MERGE_POLICY, null, null, mergePolicyName);
     }
+
+    static MetadataLockKey createGraphLockKey(DataverseName dataverseName, String graphName) {
+        return new MetadataLockKey(EntityKind.GRAPH, null, dataverseName, graphName);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
index 48508e1..260d039 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
@@ -297,4 +297,19 @@
         locks.downgrade(IMetadataLock.Mode.EXCLUSIVE_MODIFY, lock);
     }
 
+    @Override
+    public void acquireGraphReadLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createGraphLockKey(dataverseName, graphName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.READ, lock);
+    }
+
+    @Override
+    public void acquireGraphWriteLock(LockList locks, DataverseName dataverseName, String graphName)
+            throws AlgebricksException {
+        MetadataLockKey key = MetadataLockKey.createGraphLockKey(dataverseName, graphName);
+        IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+        locks.add(IMetadataLock.Mode.WRITE, lock);
+    }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index b52cddd..9058340 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -56,6 +56,7 @@
     public static final String SYNONYM_DATASET_NAME = "Synonym";
     public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
     public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
+    public static final String GRAPH_DATASET_NAME = "Graph";
 
     private MetadataConstants() {
     }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 7bb11e5..35584e5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -24,6 +24,7 @@
 
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.graph.GraphIdentifier;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.IMetadataLockUtil;
 import org.apache.asterix.common.metadata.LockList;
@@ -95,6 +96,20 @@
     }
 
     @Override
+    public void createGraphBegin(IMetadataLockManager lockMgr, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, graphIdentifier.getDataverseName());
+        lockMgr.acquireGraphWriteLock(locks, graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+    }
+
+    @Override
+    public void dropGraphBegin(IMetadataLockManager lockMgr, LockList locks, GraphIdentifier graphIdentifier)
+            throws AlgebricksException {
+        lockMgr.acquireDataverseReadLock(locks, graphIdentifier.getDataverseName());
+        lockMgr.acquireGraphWriteLock(locks, graphIdentifier.getDataverseName(), graphIdentifier.getGraphName());
+    }
+
+    @Override
     public void createIndexBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
             String datasetName, String fullTextConfigName) throws AlgebricksException {
         lockMgr.acquireDataverseReadLock(locks, dataverseName);

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13823
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I20ca6ea4c782d2fbd548e7340db65cdab5ae693d
Gerrit-Change-Number: 13823
Gerrit-PatchSet: 1
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-MessageType: newchange