You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2019/11/18 21:53:04 UTC

[asterixdb] branch master updated: [NO ISSUE] Support multipart dataverse names

This is an automated email from the ASF dual-hosted git repository.

dlych pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 54c0601  [NO ISSUE] Support multipart dataverse names
54c0601 is described below

commit 54c0601ab6ea2fbf9b638eea40e9ceec47454267
Author: Dmitry Lychagin <dm...@couchbase.com>
AuthorDate: Wed Nov 13 15:54:20 2019 -0800

    [NO ISSUE] Support multipart dataverse names
    
    - user model changes: yes
    - storage format changes: no
    - interface changes: yes
    
    Details:
    - Support dataverse names that consist of multiple parts
    - Introduce DataverseName class and adopt it throughout the system
    - Add testcases for multipart dataverse names
    
    Change-Id: I1313b2dc0e8df6a9b0ded48dea122afc3ba5dbe7
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/4004
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
 .../java/org/apache/asterix/active/Activity.java   |   6 +-
 .../java/org/apache/asterix/active/EntityId.java   |  20 +-
 .../asterix/optimizer/base/AnalysisUtil.java       |   9 +-
 .../IntroduceSecondaryIndexInsertDeleteRule.java   |   3 +-
 .../rules/SetAsterixMemoryRequirementsRule.java    |   4 +-
 .../am/AbstractIntroduceAccessMethodRule.java      |  11 +-
 .../rules/am/AccessMethodJobGenParams.java         |  13 +-
 .../optimizer/rules/am/AccessMethodUtils.java      |   2 +-
 .../optimizer/rules/am/BTreeJobGenParams.java      |   3 +-
 .../rules/am/IntroduceLSMComponentFilterRule.java  |  13 +-
 .../IntroducePrimaryIndexForAggregationRule.java   |  12 +-
 .../rules/am/InvertedIndexJobGenParams.java        |   5 +-
 .../rules/am/OptimizableOperatorSubTree.java       |   5 +-
 .../optimizer/rules/am/RTreeJobGenParams.java      |   3 +-
 .../asterix/translator/AbstractLangTranslator.java |  21 +-
 .../asterix/translator/CompiledStatements.java     |  37 +--
 .../asterix/translator/IStatementExecutor.java     |   5 +-
 .../translator/LangExpressionToPlanTranslator.java |   5 +-
 .../apache/asterix/translator/TypeTranslator.java  |  62 ++--
 .../api/http/server/ConnectorApiServlet.java       |   3 +-
 .../api/http/server/RebalanceApiServlet.java       |  26 +-
 .../asterix/api/http/server/ServletUtil.java       |   8 +
 .../asterix/api/http/server/UdfApiServlet.java     |  26 +-
 .../app/active/ActiveNotificationHandler.java      |   8 +-
 .../apache/asterix/app/active/RecoveryTask.java    |  20 +-
 .../asterix/app/external/ExternalLibraryUtils.java |  28 +-
 .../app/function/DatasetResourcesRewriter.java     |   3 +-
 .../asterix/app/function/DatasetRewriter.java      |  82 +----
 .../asterix/app/function/DumpIndexRewriter.java    |   3 +-
 .../apache/asterix/app/function/FeedRewriter.java  |  60 ++--
 .../asterix/app/function/FunctionRewriter.java     |  49 ++-
 .../app/function/StorageComponentsRewriter.java    |   3 +-
 .../asterix/app/message/AbstractUdfMessage.java    |   7 +-
 .../asterix/app/message/DeleteUdfMessage.java      |   5 +-
 .../apache/asterix/app/message/LoadUdfMessage.java |   5 +-
 .../asterix/app/translator/QueryTranslator.java    | 215 ++++++-------
 .../org/apache/asterix/utils/FeedOperations.java   |   9 +-
 .../org/apache/asterix/utils/FlushDatasetUtil.java |   3 +-
 .../org/apache/asterix/utils/RebalanceUtil.java    |  10 +-
 .../api/http/servlet/ConnectorApiServletTest.java  |  10 +-
 .../org/apache/asterix/common/TestDataUtil.java    |   7 +-
 .../test/active/ActiveEventsListenerTest.java      |   4 +-
 .../asterix/test/active/ActiveStatsTest.java       |   4 +-
 .../asterix/test/active/TestEventsListener.java    |   9 +-
 .../apache/asterix/test/active/TestUserActor.java  |  79 +++--
 .../apache/asterix/test/common/TestExecutor.java   |  16 +-
 .../dataflow/CheckpointInSecondaryIndexTest.java   |   3 +-
 .../test/dataflow/MultiPartitionLSMIndexTest.java  |   3 +-
 .../dataflow/SearchCursorComponentSwitchTest.java  |   3 +-
 .../asterix/test/dataflow/StorageTestUtils.java    |   3 +-
 .../apache/asterix/test/dataflow/TestDataset.java  |   9 +-
 .../asterix/test/sqlpp/ParserTestExecutor.java     |  17 +-
 .../storage/IndexDropOperatorNodePushableTest.java |   5 +-
 .../multipart-dataverse/index/index-01.sqlpp}      |  31 +-
 .../multipart-dataverse/index/index-02.sqlpp}      |  31 +-
 .../primary-equi-join-neg_01.sqlpp                 |   4 +-
 .../primary-equi-join-neg_02.sqlpp                 |   4 +-
 .../btree-index-join/primary-equi-join_01.sqlpp    |   4 +-
 .../btree-index-join/primary-equi-join_02.sqlpp    |   4 +-
 .../btree-index-join/primary-ge-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-ge-join_02.sqlpp      |   4 +-
 .../btree-index-join/primary-gt-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-gt-join_02.sqlpp      |   4 +-
 .../btree-index-join/primary-le-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-le-join_02.sqlpp      |   4 +-
 .../btree-index-join/primary-lt-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-lt-join_02.sqlpp      |   4 +-
 .../primary-equi-join-neg_01.sqlpp                 |   4 +-
 .../btree-index-join/primary-equi-join_01.sqlpp    |   4 +-
 .../btree-index-join/primary-equi-join_02.sqlpp    |   4 +-
 .../btree-index-join/primary-ge-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-gt-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-le-join_01.sqlpp      |   4 +-
 .../btree-index-join/primary-lt-join_01.sqlpp      |   4 +-
 .../multipart-dataverse/index/index-01.plan        |  15 +
 .../multipart-dataverse/index/index-02.plan        |  15 +
 .../results_parser_sqlpp/query-issue550.ast        |   2 +-
 .../resources/parserts/results_parser_sqlpp/1.ast  |   2 +-
 .../resources/parserts/results_parser_sqlpp/2.ast  |   4 +-
 .../resources/parserts/results_parser_sqlpp/4.ast  |   4 +-
 .../resources/parserts/results_parser_sqlpp/5.ast  |   4 +-
 .../resources/parserts/results_parser_sqlpp/6.ast  |   4 +-
 .../results_parser_sqlpp/ANYInFieldAccessor.ast    |   4 +-
 .../parserts/results_parser_sqlpp/LetFor.ast       |   2 +-
 .../parserts/results_parser_sqlpp/columnalias.ast  |   2 +-
 .../parserts/results_parser_sqlpp/columnalias2.ast |   2 +-
 .../parserts/results_parser_sqlpp/columnalias3.ast |   2 +-
 .../results_parser_sqlpp/functionDecl1.ast         |   2 +-
 .../parserts/results_parser_sqlpp/nestedFLWOGR.ast |   2 +-
 .../results_parser_sqlpp/nestedFLWOGR2.ast         |   2 +-
 .../results_parser_sqlpp/nestedFLWOGR3.ast         |   4 +-
 .../parserts/results_parser_sqlpp/nestedFor.ast    |   4 +-
 .../results_parser_sqlpp/numberInFieldAccessor.ast |   4 +-
 .../index_1/index_1.1.ddl.sqlpp}                   |  28 +-
 .../index_1/index_1.2.update.sqlpp}                |   9 +-
 .../index_1/index_1.3.query.sqlpp}                 |  13 +-
 .../resolution_1/resolution_1.1.ddl.sqlpp}         |  27 +-
 .../resolution_1/resolution_1.10.query.sqlpp}      |  13 +-
 .../resolution_1/resolution_1.11.query.sqlpp}      |  14 +-
 .../resolution_1/resolution_1.2.update.sqlpp}      |  32 +-
 .../resolution_1/resolution_1.3.update.sqlpp}      |  13 +-
 .../resolution_1/resolution_1.4.query.sqlpp}       |  13 +-
 .../resolution_1/resolution_1.5.query.sqlpp}       |  13 +-
 .../resolution_1/resolution_1.6.query.sqlpp}       |  13 +-
 .../resolution_1/resolution_1.7.query.sqlpp}       |  12 +-
 .../resolution_1/resolution_1.8.query.sqlpp}       |  12 +-
 .../resolution_1/resolution_1.9.query.sqlpp}       |  13 +-
 .../special_chars_1/special_chars_1.1.ddl.sqlpp}   |  24 +-
 .../special_chars_1.2.update.sqlpp}                |  10 +-
 .../special_chars_1/special_chars_1.3.query.sqlpp} |  13 +-
 .../special_chars_1/special_chars_1.4.query.sqlpp} |  13 +-
 .../multipart-dataverse/udf_1/udf_1.1.ddl.sqlpp}   |  13 +-
 .../multipart-dataverse/udf_1/udf_1.2.query.sqlpp} |  11 +-
 .../multipart-dataverse/udf_1/udf_1.3.query.sqlpp} |  11 +-
 .../check-dependencies-1.3.ddl.sqlpp               |   4 +-
 .../multipart-dataverse/index_1/index_1.3.adm      |   1 +
 .../resolution_1/resolution_1.10.adm               |   2 +
 .../resolution_1/resolution_1.11.adm               |   2 +
 .../resolution_1/resolution_1.4.adm                |   2 +
 .../resolution_1/resolution_1.5.adm                |   2 +
 .../resolution_1/resolution_1.6.adm                |   2 +
 .../resolution_1/resolution_1.7.adm                |  10 +
 .../resolution_1/resolution_1.8.adm                |   5 +
 .../resolution_1/resolution_1.9.adm                |   2 +
 .../special_chars_1/special_chars_1.3.adm          |   1 +
 .../special_chars_1/special_chars_1.4.adm          |   1 +
 .../results/multipart-dataverse/udf_1/udf_1.2.adm  |   2 +
 .../results/multipart-dataverse/udf_1/udf_1.3.adm  |   2 +
 .../check-dependencies-1.1.adm                     |   2 +-
 .../src/test/resources/runtimets/testsuite.xml     |   2 +-
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  24 +-
 asterixdb/asterix-common/pom.xml                   |   5 +
 .../asterix/common/api/IMetadataLockManager.java   | 170 ++++++----
 .../common/functions/FunctionConstants.java        |  12 +-
 .../common/functions/FunctionSignature.java        |  74 ++++-
 .../asterix/common/library/ILibraryManager.java    |  15 +-
 .../asterix/common/metadata/DataverseName.java     | 335 +++++++++++++++++++
 .../apache/asterix/common/metadata/IDataset.java   |   2 +-
 .../asterix/common/metadata/IMetadataLock.java     |  12 +-
 .../apache/asterix/common/metadata/LockList.java   |   3 +-
 .../common/storage/DatasetCopyIdentifier.java      |  14 +-
 .../asterix/common/storage/ResourceReference.java  |  10 +-
 .../asterix/common/utils/StoragePathUtil.java      |   7 +-
 .../asterix/common/metadata/DataverseNameTest.java | 244 ++++++++++++++
 .../dataset/adapter/AdapterIdentifier.java         |  14 +-
 .../apache/asterix/external/feed/api/IFeed.java    |   3 +-
 .../external/feed/management/FeedConnectionId.java |   5 +-
 .../asterix/external/feed/policy/FeedPolicy.java   |  11 +-
 .../asterix/external/indexing/ExternalFile.java    |  11 +-
 .../asterix/external/library/ExternalFunction.java |   4 +-
 .../external/library/ExternalLibraryManager.java   |  15 +-
 .../operators/FeedIntakeOperatorDescriptor.java    |   4 +-
 .../provider/DatasourceFactoryProvider.java        |   3 +-
 .../asterix/external/util/ExternalDataUtils.java   |  26 +-
 .../apache/asterix/external/util/FeedUtils.java    |  13 +-
 .../external/feed/test/InputHandlerTest.java       |   3 +-
 .../lang/aql/rewrites/AqlStatementRewriter.java    |   9 +-
 .../lang/aql/visitor/AQLToSQLPPPrintVisitor.java   |  22 +-
 .../lang/aql/visitor/AqlDeleteRewriteVisitor.java  |  22 +-
 asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj  |  78 ++---
 .../lang/common/base/IStatementRewriter.java       |   5 +-
 .../lang/common/context/FunctionSignatures.java    |   7 +-
 .../apache/asterix/lang/common/context/Scope.java  |   7 +-
 .../common/expression/TypeReferenceExpression.java |   7 +-
 .../asterix/lang/common/parser/ScopeChecker.java   |   9 +-
 .../lang/common/statement/CompactStatement.java    |   7 +-
 .../common/statement/ConnectFeedStatement.java     |  15 +-
 .../common/statement/CreateDataverseStatement.java |   8 +-
 .../lang/common/statement/CreateFeedStatement.java |   7 +-
 .../common/statement/CreateIndexStatement.java     |   7 +-
 .../asterix/lang/common/statement/DatasetDecl.java |  55 +---
 .../lang/common/statement/DataverseDecl.java       |   8 +-
 .../common/statement/DataverseDropStatement.java   |   8 +-
 .../lang/common/statement/DeleteStatement.java     |  11 +-
 .../common/statement/DisconnectFeedStatement.java  |  16 +-
 .../common/statement/DropDatasetStatement.java     |   7 +-
 .../lang/common/statement/FeedDropStatement.java   |   7 +-
 .../common/statement/FeedPolicyDropStatement.java  |   7 +-
 .../lang/common/statement/IndexDropStatement.java  |   7 +-
 .../lang/common/statement/InsertStatement.java     |  11 +-
 .../lang/common/statement/LoadStatement.java       |   9 +-
 .../statement/RefreshExternalDatasetStatement.java |   7 +-
 .../lang/common/statement/StartFeedStatement.java  |   7 +-
 .../lang/common/statement/StopFeedStatement.java   |   7 +-
 .../asterix/lang/common/statement/TypeDecl.java    |   9 +-
 .../lang/common/statement/TypeDropStatement.java   |   7 +-
 .../lang/common/statement/UpsertStatement.java     |   3 +-
 .../lang/common/util/CommonFunctionMapUtil.java    |   4 +-
 .../asterix/lang/common/util/ExpressionUtils.java  |  10 +
 .../asterix/lang/common/util/FunctionUtil.java     | 116 +++++--
 .../common/visitor/AbstractInlineUdfsVisitor.java  |   9 +-
 .../lang/common/visitor/FormatPrintVisitor.java    |  50 ++-
 .../lang/common/visitor/QueryPrintVisitor.java     |  26 +-
 .../sqlpp/rewrites/SqlppStatementRewriter.java     |   9 +-
 .../visitor/OperatorExpressionVisitor.java         |   4 +-
 .../visitor/Sql92AggregateFunctionVisitor.java     |   2 +-
 .../visitor/VariableCheckAndRewriteVisitor.java    | 202 +++++++-----
 .../asterix/lang/sqlpp/util/FunctionMapUtil.java   |   8 +-
 .../lang/sqlpp/util/SqlppStatementUtil.java        |  61 ++--
 .../lang/sqlpp/visitor/SqlppAstPrintVisitor.java   |  19 +-
 .../sqlpp/visitor/SqlppDeleteRewriteVisitor.java   |  22 +-
 .../sqlpp/visitor/SqlppFormatPrintVisitor.java     |   2 +-
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj    | 245 +++++++-------
 .../org/apache/asterix/metadata/MetadataCache.java |  97 +++---
 .../apache/asterix/metadata/MetadataManager.java   | 148 ++++-----
 .../org/apache/asterix/metadata/MetadataNode.java  | 354 ++++++++++-----------
 .../metadata/MetadataTransactionContext.java       |  28 +-
 .../metadata/api/ExtensionMetadataDataset.java     |   4 +-
 .../api/IMetadataEntityTupleTranslator.java        |   9 +-
 .../api/IMetadataEntityTupleTranslatorFactory.java |   2 +-
 .../asterix/metadata/api/IMetadataIndex.java       |   3 +-
 .../asterix/metadata/api/IMetadataManager.java     | 100 +++---
 .../apache/asterix/metadata/api/IMetadataNode.java | 101 +++---
 .../asterix/metadata/api/IValueExtractor.java      |   3 +-
 .../bootstrap/MetadataBuiltinEntities.java         |   3 +-
 .../asterix/metadata/bootstrap/MetadataIndex.java  |   3 +-
 .../metadata/bootstrap/MetadataPrimaryIndexes.java |   2 -
 .../metadata/bootstrap/MetadataRecordTypes.java    |   2 +-
 .../asterix/metadata/declared/DataSourceId.java    |  40 ++-
 .../asterix/metadata/declared/DataSourceIndex.java |  14 +-
 .../metadata/declared/DatasetDataSource.java       |   3 +-
 .../metadata/declared/FunctionDataSource.java      |  10 +-
 .../metadata/declared/LoadableDataSource.java      |   6 +-
 .../metadata/declared/MetadataManagerUtil.java     |  47 +--
 .../metadata/declared/MetadataProvider.java        |  82 ++---
 .../asterix/metadata/entities/BuiltinTypeMap.java  |  13 +-
 .../metadata/entities/CompactionPolicy.java        |   9 +-
 .../apache/asterix/metadata/entities/Dataset.java  |  45 ++-
 .../apache/asterix/metadata/entities/Datatype.java |   9 +-
 .../asterix/metadata/entities/Dataverse.java       |  10 +-
 .../org/apache/asterix/metadata/entities/Feed.java |   9 +-
 .../asterix/metadata/entities/FeedConnection.java  |   9 +-
 .../metadata/entities/FeedPolicyEntity.java        |   5 +-
 .../apache/asterix/metadata/entities/Function.java |  22 +-
 .../apache/asterix/metadata/entities/Index.java    |  13 +-
 .../apache/asterix/metadata/entities/Library.java  |   9 +-
 .../AbstractTupleTranslator.java                   |  63 +++-
 .../CompactionPolicyTupleTranslator.java           |  50 +--
 .../DatasetTupleTranslator.java                    | 109 +++----
 .../DatasourceAdapterTupleTranslator.java          |  54 +---
 .../DatatypeTupleTranslator.java                   |  66 ++--
 .../DataverseTupleTranslator.java                  |  62 ++--
 .../ExternalFileTupleTranslator.java               |  77 ++---
 .../FeedConnectionTupleTranslator.java             | 122 ++++---
 .../FeedPolicyTupleTranslator.java                 |  71 ++---
 .../FeedTupleTranslator.java                       |  66 ++--
 .../FunctionTupleTranslator.java                   | 130 ++++----
 .../IndexTupleTranslator.java                      | 226 ++++++-------
 .../LibraryTupleTranslator.java                    |  48 +--
 .../MetadataTupleTranslatorProvider.java           |   4 +
 .../NodeGroupTupleTranslator.java                  |  36 +--
 .../NodeTupleTranslator.java                       |  31 +-
 .../asterix/metadata/feeds/FeedMetadataUtil.java   |  37 +--
 .../functions/ExternalFunctionCompilerUtil.java    |   5 +-
 .../functions/ExternalScalarFunctionInfo.java      |   7 +-
 .../apache/asterix/metadata/lock/DatasetLock.java  |   8 +-
 .../apache/asterix/metadata/lock/MetadataLock.java |  10 +-
 .../asterix/metadata/lock/MetadataLockKey.java     | 128 ++++++++
 .../asterix/metadata/lock/MetadataLockManager.java | 123 +++----
 .../apache/asterix/metadata/utils/DatasetUtil.java |  33 +-
 .../InvertedIndexResourceFactoryProvider.java      |   7 +-
 .../asterix/metadata/utils/MetadataConstants.java  |   4 +-
 .../asterix/metadata/utils/MetadataLockUtil.java   | 131 ++++----
 .../asterix/metadata/utils/MetadataUtil.java       |   5 -
 ...aryCorrelatedInvertedIndexOperationsHelper.java |   4 +-
 .../SecondaryInvertedIndexOperationsHelper.java    |   4 +-
 .../metadata/utils/SplitsAndConstraintsUtil.java   |  10 +-
 .../MetadataEntityValueExtractor.java              |   5 +-
 .../DatasetTupleTranslatorTest.java                |   8 +-
 .../IndexTupleTranslatorTest.java                  |  13 +-
 .../ExternalFunctionCompilerUtilTest.java          |   3 +-
 .../metadata/lock/MetadataLockManagerTest.java     |  60 ++--
 .../asterix/om/functions/BuiltinFunctions.java     |   8 +-
 .../asterix/om/functions/ExternalFunctionInfo.java |   8 +-
 .../om/functions/FunctionInfoRepository.java       |  12 +-
 .../org/apache/asterix/om/types/TypeSignature.java |  19 +-
 .../org/apache/asterix/om/utils/AdmNodeUtils.java  |  50 +++
 .../functions/DecodeDataverseNameDescriptor.java   | 129 ++++++++
 .../runtime/functions/FunctionCollection.java      |   2 +
 .../src/main/resources/Catalog.xsd                 |  26 +-
 .../tools/translator/ADGenDmlTranslator.java       |  10 +-
 .../hyracks/algebricks/common/utils/Triple.java    |   5 +-
 .../apache/hyracks/http/api/IServletRequest.java   |   9 +
 .../apache/hyracks/http/server/BaseRequest.java    |   6 +
 .../java/org/apache/hyracks/util/JSONUtil.java     |  17 +
 285 files changed, 4146 insertions(+), 2977 deletions(-)

diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
index 9538118..d8a6c48 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/Activity.java
@@ -19,6 +19,8 @@ package org.apache.asterix.active;
 
 import java.util.Map;
 
+import org.apache.asterix.common.metadata.DataverseName;
+
 public class Activity implements Comparable<Activity> {
 
     protected int activityId;
@@ -30,8 +32,8 @@ public class Activity implements Comparable<Activity> {
         this.activityDetails = activityDetails;
     }
 
-    public String getDataverseName() {
-        return activeEntityId.getDataverse();
+    public DataverseName getDataverseName() {
+        return activeEntityId.getDataverseName();
     }
 
     public String getActiveEntityName() {
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
index 13e16f0..1d33961 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/EntityId.java
@@ -21,25 +21,27 @@ package org.apache.asterix.active;
 import java.io.Serializable;
 import java.util.Objects;
 
+import org.apache.asterix.common.metadata.DataverseName;
+
 /**
  * A unique identifier for a data feed.
  */
 public class EntityId implements Serializable {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     private final String extensionName;
-    private final String dataverse;
+    private final DataverseName dataverseName;
     private final String entityName;
 
-    public EntityId(String extentionName, String dataverse, String entityName) {
+    public EntityId(String extentionName, DataverseName dataverseName, String entityName) {
         this.extensionName = extentionName;
-        this.dataverse = dataverse;
+        this.dataverseName = dataverseName;
         this.entityName = entityName;
     }
 
-    public String getDataverse() {
-        return dataverse;
+    public DataverseName getDataverseName() {
+        return dataverseName;
     }
 
     public String getEntityName() {
@@ -55,18 +57,18 @@ public class EntityId implements Serializable {
             return true;
         }
         EntityId other = (EntityId) o;
-        return Objects.equals(other.dataverse, dataverse) && Objects.equals(other.entityName, entityName)
+        return Objects.equals(other.dataverseName, dataverseName) && Objects.equals(other.entityName, entityName)
                 && Objects.equals(other.extensionName, extensionName);
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(dataverse, entityName, extensionName);
+        return Objects.hash(dataverseName, entityName, extensionName);
     }
 
     @Override
     public String toString() {
-        return dataverse + "." + entityName + "(" + extensionName + ")";
+        return dataverseName + "." + entityName + "(" + extensionName + ")";
     }
 
     public String getExtensionName() {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
index 9c476e5..7eea595 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java
@@ -23,11 +23,11 @@ import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
 import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -122,14 +122,15 @@ public class AnalysisUtil {
         return false;
     }
 
-    public static Pair<String, String> getDatasetInfo(AbstractDataSourceOperator op) throws AlgebricksException {
+    public static Pair<DataverseName, String> getDatasetInfo(AbstractDataSourceOperator op) {
         DataSourceId srcId = (DataSourceId) op.getDataSource().getId();
         return new Pair<>(srcId.getDataverseName(), srcId.getDatasourceName());
     }
 
-    public static Pair<String, String> getExternalDatasetInfo(UnnestMapOperator op) throws AlgebricksException {
+    public static Pair<DataverseName, String> getExternalDatasetInfo(UnnestMapOperator op) {
         AbstractFunctionCallExpression unnestExpr = (AbstractFunctionCallExpression) op.getExpressionRef().getValue();
-        String dataverseName = AccessMethodUtils.getStringConstant(unnestExpr.getArguments().get(0));
+        DataverseName dataverseName = DataverseName
+                .createFromCanonicalForm(AccessMethodUtils.getStringConstant(unnestExpr.getArguments().get(0)));
         String datasetName = AccessMethodUtils.getStringConstant(unnestExpr.getArguments().get(1));
         return new Pair<>(dataverseName, datasetName);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 6c258e4..0b75be6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -30,6 +30,7 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceIndex;
@@ -147,7 +148,7 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
 
         DataSource datasetSource = (DataSource) primaryIndexModificationOp.getDataSource();
         MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
-        String dataverseName = datasetSource.getId().getDataverseName();
+        DataverseName dataverseName = datasetSource.getId().getDataverseName();
         String datasetName = datasetSource.getId().getDatasourceName();
         Dataset dataset = mp.findDataset(dataverseName, datasetName);
         if (dataset == null) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
index 91dfe7f..f4f8fa2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixMemoryRequirementsRule.java
@@ -22,13 +22,13 @@ package org.apache.asterix.optimizer.rules;
 import java.util.Set;
 import java.util.function.Predicate;
 
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.base.AsterixOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.algebricks.rewriter.rules.SetMemoryRequirementsRule;
 
@@ -83,7 +83,7 @@ public final class SetAsterixMemoryRequirementsRule extends SetMemoryRequirement
 
     private static boolean isMinMemoryBudgetFunction(DataSourceId dsId) {
         return BuiltinFunctions.builtinFunctionHasProperty(
-                new FunctionIdentifier(dsId.getDataverseName(), dsId.getDatasourceName()),
+                FunctionSignature.createFunctionIdentifier(dsId.getDataverseName(), dsId.getDatasourceName()),
                 BuiltinFunctions.DataSourceFunctionProperty.MIN_MEMORY_BUDGET);
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 6583342..a85ed90 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -32,6 +32,7 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -1095,12 +1096,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
         if (dataSourceScanOp.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
             return null;
         }
-        Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo((DataSourceScanOperator) dataSourceScanOp);
-        String dataverseName = datasetInfo.first;
-        String datasetName = datasetInfo.second;
-
-        Index idxUsedInUnnestMap = metadataProvider.getIndex(dataverseName, datasetName, datasetName);
-        return idxUsedInUnnestMap;
+        Pair<DataverseName, String> datasetInfo =
+                AnalysisUtil.getDatasetInfo((DataSourceScanOperator) dataSourceScanOp);
+        return metadataProvider.getIndex(datasetInfo.first, datasetInfo.second, datasetInfo.second);
     }
-
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index cba4411..a4795ba 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -21,6 +21,7 @@ package org.apache.asterix.optimizer.rules.am;
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -38,7 +39,7 @@ public class AccessMethodJobGenParams {
     private static final int NUM_PARAMS = 6;
     protected String indexName;
     protected IndexType indexType;
-    protected String dataverseName;
+    protected DataverseName dataverseName;
     protected String datasetName;
     protected boolean retainInput;
     protected boolean requiresBroadcast;
@@ -48,8 +49,8 @@ public class AccessMethodJobGenParams {
         // Enable creation of an empty object and fill members using setters
     }
 
-    public AccessMethodJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
-            boolean retainInput, boolean requiresBroadcast) {
+    public AccessMethodJobGenParams(String indexName, IndexType indexType, DataverseName dataverseName,
+            String datasetName, boolean retainInput, boolean requiresBroadcast) {
         this.indexName = indexName;
         this.indexType = indexType;
         this.dataverseName = dataverseName;
@@ -62,7 +63,7 @@ public class AccessMethodJobGenParams {
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(indexName)));
         funcArgs.add(new MutableObject<>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
-        funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(dataverseName)));
+        funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(dataverseName.getCanonicalForm())));
         funcArgs.add(new MutableObject<>(AccessMethodUtils.createStringConstant(datasetName)));
         funcArgs.add(new MutableObject<>(AccessMethodUtils.createBooleanConstant(retainInput)));
         funcArgs.add(new MutableObject<>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
@@ -71,7 +72,7 @@ public class AccessMethodJobGenParams {
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         indexName = AccessMethodUtils.getStringConstant(funcArgs.get(0));
         indexType = IndexType.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
-        dataverseName = AccessMethodUtils.getStringConstant(funcArgs.get(2));
+        dataverseName = DataverseName.createFromCanonicalForm(AccessMethodUtils.getStringConstant(funcArgs.get(2)));
         datasetName = AccessMethodUtils.getStringConstant(funcArgs.get(3));
         retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
         requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(5));
@@ -86,7 +87,7 @@ public class AccessMethodJobGenParams {
         return indexType;
     }
 
-    public String getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 1357fd2..38072f3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -1764,7 +1764,7 @@ public class AccessMethodUtils {
         context.computeAndSetTypeEnvironmentForOperator(order);
         List<Mutable<ILogicalExpression>> externalLookupArgs = new ArrayList<>();
         //Add dataverse to the arguments
-        AccessMethodUtils.addStringArg(dataset.getDataverseName(), externalLookupArgs);
+        AccessMethodUtils.addStringArg(dataset.getDataverseName().getCanonicalForm(), externalLookupArgs);
         //Add dataset to the arguments
         AccessMethodUtils.addStringArg(dataset.getDatasetName(), externalLookupArgs);
         //Add PK vars to the arguments
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeJobGenParams.java
index 26501c2..2eefc8c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeJobGenParams.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -45,7 +46,7 @@ public class BTreeJobGenParams extends AccessMethodJobGenParams {
         super();
     }
 
-    public BTreeJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
+    public BTreeJobGenParams(String indexName, IndexType indexType, DataverseName dataverseName, String datasetName,
             boolean retainInput, boolean requiresBroadcast) {
         super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index 109812b..95b7e17 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -28,6 +28,7 @@ import java.util.Queue;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DatasetDataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -90,10 +91,10 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
         Dataset dataset = getDataset(op, context);
         List<String> filterFieldName = null;
         ARecordType recType = null;
+        MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
         if (dataset != null && dataset.getDatasetType() == DatasetType.INTERNAL) {
             filterFieldName = DatasetUtil.getFilterField(dataset);
-            IAType itemType = ((MetadataProvider) context.getMetadataProvider())
-                    .findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+            IAType itemType = mp.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
             if (itemType.getTypeTag() == ATypeTag.OBJECT) {
                 recType = (ARecordType) itemType;
             }
@@ -111,8 +112,7 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
         List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<>();
 
         if (!analysisCtx.getMatchedFuncExprs().isEmpty()) {
-            List<Index> datasetIndexes = ((MetadataProvider) context.getMetadataProvider())
-                    .getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+            List<Index> datasetIndexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
 
             for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
                 IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
@@ -417,10 +417,11 @@ public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
                 if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                     AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
                     FunctionIdentifier fid = f.getFunctionIdentifier();
-                    String dataverseName;
+                    DataverseName dataverseName;
                     String datasetName;
                     if (BuiltinFunctions.EXTERNAL_LOOKUP.equals(fid)) {
-                        dataverseName = AccessMethodUtils.getStringConstant(f.getArguments().get(0));
+                        dataverseName = DataverseName
+                                .createFromCanonicalForm(AccessMethodUtils.getStringConstant(f.getArguments().get(0)));
                         datasetName = AccessMethodUtils.getStringConstant(f.getArguments().get(1));
                     } else if (fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
                         AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index 5eb0fc6..64f9068 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.asterix.common.config.DatasetConfig;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DatasetDataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -163,7 +164,8 @@ public class IntroducePrimaryIndexForAggregationRule implements IAlgebraicRewrit
                 return null;
             }
             String indexName = ConstantExpressionUtil.getStringArgument(functionCallExpression, 0);
-            String dataverseName = ConstantExpressionUtil.getStringArgument(functionCallExpression, 2);
+            DataverseName dataverseName = DataverseName
+                    .createFromCanonicalForm(ConstantExpressionUtil.getStringArgument(functionCallExpression, 2));
             String datasetName = ConstantExpressionUtil.getStringArgument(functionCallExpression, 3);
             Index index = ((MetadataProvider) metadataProvider).getIndex(dataverseName, datasetName, indexName);
             if (!index.isPrimaryIndex()) {
@@ -257,6 +259,7 @@ public class IntroducePrimaryIndexForAggregationRule implements IAlgebraicRewrit
      */
     private Pair<Dataset, Index> findDatasetAndSecondaryPrimaryIndex(AbstractScanOperator scanOperator,
             BTreeJobGenParams originalBTreeParameters, IOptimizationContext context) throws AlgebricksException {
+        MetadataProvider mp = (MetadataProvider) context.getMetadataProvider();
         // #1. get the dataset
         Dataset dataset;
         // case 1: dataset scan
@@ -275,12 +278,11 @@ public class IntroducePrimaryIndexForAggregationRule implements IAlgebraicRewrit
             if (originalBTreeParameters.isEqCondition()) {
                 return null;
             }
-            dataset = ((MetadataProvider) context.getMetadataProvider())
-                    .findDataset(originalBTreeParameters.getDataverseName(), originalBTreeParameters.getDatasetName());
+            dataset = mp.findDataset(originalBTreeParameters.getDataverseName(),
+                    originalBTreeParameters.getDatasetName());
         }
         // #2. get all indexes and look for the primary one
-        List<Index> indexes = ((MetadataProvider) context.getMetadataProvider())
-                .getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         for (Index index : indexes) {
             if (index.getKeyFieldNames().isEmpty()) {
                 return Pair.of(dataset, index);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index 1ddd1b5..26350d8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -56,8 +57,8 @@ public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
     public InvertedIndexJobGenParams() {
     }
 
-    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
-            boolean retainInput, boolean requiresBroadcast) {
+    public InvertedIndexJobGenParams(String indexName, IndexType indexType, DataverseName dataverseName,
+            String datasetName, boolean retainInput, boolean requiresBroadcast) {
         super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 2f1db8c..3cbba0d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -236,7 +237,7 @@ public class OptimizableOperatorSubTree {
      * Also sets recordType to be the type of that dataset.
      */
     public boolean setDatasetAndTypeMetadata(MetadataProvider metadataProvider) throws AlgebricksException {
-        String dataverseName = null;
+        DataverseName dataverseName = null;
         String datasetName = null;
 
         Dataset ds = null;
@@ -267,7 +268,7 @@ public class OptimizableOperatorSubTree {
                             return false;
                         }
                     }
-                    Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
+                    Pair<DataverseName, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
                     dataverseName = datasetInfo.first;
                     datasetName = datasetInfo.second;
                     break;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeJobGenParams.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeJobGenParams.java
index cccb6ef..d58766e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeJobGenParams.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeJobGenParams.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
@@ -37,7 +38,7 @@ public class RTreeJobGenParams extends AccessMethodJobGenParams {
     public RTreeJobGenParams() {
     }
 
-    public RTreeJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
+    public RTreeJobGenParams(String indexName, IndexType indexType, DataverseName dataverseName, String datasetName,
             boolean retainInput, boolean requiresBroadcast) {
         super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
index 967b3ad..b7b464d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AbstractLangTranslator.java
@@ -32,6 +32,7 @@ import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.statement.DatasetDecl;
 import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -114,14 +115,14 @@ public abstract class AbstractLangTranslator {
 
         boolean invalidOperation = false;
         String message = null;
-        String dataverse = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
+        DataverseName dataverseName = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
         switch (stmt.getKind()) {
             case INSERT:
                 InsertStatement insertStmt = (InsertStatement) stmt;
                 if (insertStmt.getDataverseName() != null) {
-                    dataverse = insertStmt.getDataverseName().getValue();
+                    dataverseName = insertStmt.getDataverseName();
                 }
-                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName);
                 if (invalidOperation) {
                     message = "Insert operation is not permitted in dataverse "
                             + MetadataConstants.METADATA_DATAVERSE_NAME;
@@ -131,9 +132,9 @@ public abstract class AbstractLangTranslator {
             case DELETE:
                 DeleteStatement deleteStmt = (DeleteStatement) stmt;
                 if (deleteStmt.getDataverseName() != null) {
-                    dataverse = deleteStmt.getDataverseName().getValue();
+                    dataverseName = deleteStmt.getDataverseName();
                 }
-                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName);
                 if (invalidOperation) {
                     message = "Delete operation is not permitted in dataverse "
                             + MetadataConstants.METADATA_DATAVERSE_NAME;
@@ -142,19 +143,19 @@ public abstract class AbstractLangTranslator {
 
             case DATAVERSE_DROP:
                 DataverseDropStatement dvDropStmt = (DataverseDropStatement) stmt;
-                invalidOperation =
-                        MetadataConstants.METADATA_DATAVERSE_NAME.equals(dvDropStmt.getDataverseName().getValue());
+                dataverseName = dvDropStmt.getDataverseName();
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName);
                 if (invalidOperation) {
-                    message = "Cannot drop dataverse:" + dvDropStmt.getDataverseName().getValue();
+                    message = "Cannot drop dataverse:" + dataverseName;
                 }
                 break;
 
             case DATASET_DROP:
                 DropDatasetStatement dropStmt = (DropDatasetStatement) stmt;
                 if (dropStmt.getDataverseName() != null) {
-                    dataverse = dropStmt.getDataverseName().getValue();
+                    dataverseName = dropStmt.getDataverseName();
                 }
-                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName);
                 if (invalidOperation) {
                     message = "Cannot drop a dataset belonging to the dataverse:"
                             + MetadataConstants.METADATA_DATAVERSE_NAME;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 5950329..e85fec8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -20,6 +20,7 @@ package org.apache.asterix.translator;
 
 import java.util.Map;
 
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
@@ -189,7 +190,7 @@ public class CompiledStatements {
 
     public interface ICompiledDmlStatement extends ICompiledStatement {
 
-        String getDataverseName();
+        DataverseName getDataverseName();
 
         String getDatasetName();
     }
@@ -210,7 +211,7 @@ public class CompiledStatements {
         }
 
         @Override
-        public String getDataverseName() {
+        public DataverseName getDataverseName() {
             return index.getDataverseName();
         }
 
@@ -230,13 +231,13 @@ public class CompiledStatements {
 
     public static class CompiledLoadFromFileStatement extends AbstractCompiledStatement
             implements ICompiledDmlStatement {
-        private final String dataverseName;
+        private final DataverseName dataverseName;
         private final String datasetName;
         private final boolean alreadySorted;
         private final String adapter;
         private final Map<String, String> properties;
 
-        public CompiledLoadFromFileStatement(String dataverseName, String datasetName, String adapter,
+        public CompiledLoadFromFileStatement(DataverseName dataverseName, String datasetName, String adapter,
                 Map<String, String> properties, boolean alreadySorted) {
             this.dataverseName = dataverseName;
             this.datasetName = datasetName;
@@ -246,7 +247,7 @@ public class CompiledStatements {
         }
 
         @Override
-        public String getDataverseName() {
+        public DataverseName getDataverseName() {
             return dataverseName;
         }
 
@@ -274,14 +275,14 @@ public class CompiledStatements {
     }
 
     public static class CompiledInsertStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
-        private final String dataverseName;
+        private final DataverseName dataverseName;
         private final String datasetName;
         private final Query query;
         private final int varCounter;
         private final VariableExpr var;
         private final Expression returnExpression;
 
-        public CompiledInsertStatement(String dataverseName, String datasetName, Query query, int varCounter,
+        public CompiledInsertStatement(DataverseName dataverseName, String datasetName, Query query, int varCounter,
                 VariableExpr var, Expression returnExpression) {
             this.dataverseName = dataverseName;
             this.datasetName = datasetName;
@@ -292,7 +293,7 @@ public class CompiledStatements {
         }
 
         @Override
-        public String getDataverseName() {
+        public DataverseName getDataverseName() {
             return dataverseName;
         }
 
@@ -325,7 +326,7 @@ public class CompiledStatements {
 
     public static class CompiledUpsertStatement extends CompiledInsertStatement {
 
-        public CompiledUpsertStatement(String dataverseName, String datasetName, Query query, int varCounter,
+        public CompiledUpsertStatement(DataverseName dataverseName, String datasetName, Query query, int varCounter,
                 VariableExpr var, Expression returnExpression) {
             super(dataverseName, datasetName, query, varCounter, var, returnExpression);
         }
@@ -348,8 +349,8 @@ public class CompiledStatements {
         }
 
         @Override
-        public String getDataverseName() {
-            return request.getReceivingFeedId().getDataverse();
+        public DataverseName getDataverseName() {
+            return request.getReceivingFeedId().getDataverseName();
         }
 
         public String getFeedName() {
@@ -372,14 +373,14 @@ public class CompiledStatements {
     }
 
     public static class CompiledDeleteStatement extends AbstractCompiledStatement implements ICompiledDmlStatement {
-        private final String dataverseName;
+        private final DataverseName dataverseName;
         private final String datasetName;
         private final Expression condition;
         private final int varCounter;
         private final Query query;
 
-        public CompiledDeleteStatement(VariableExpr var, String dataverseName, String datasetName, Expression condition,
-                int varCounter, Query query) {
+        public CompiledDeleteStatement(VariableExpr var, DataverseName dataverseName, String datasetName,
+                Expression condition, int varCounter, Query query) {
             this.dataverseName = dataverseName;
             this.datasetName = datasetName;
             this.condition = condition;
@@ -393,7 +394,7 @@ public class CompiledStatements {
         }
 
         @Override
-        public String getDataverseName() {
+        public DataverseName getDataverseName() {
             return dataverseName;
         }
 
@@ -417,15 +418,15 @@ public class CompiledStatements {
     }
 
     public static class CompiledCompactStatement extends AbstractCompiledStatement {
-        private final String dataverseName;
+        private final DataverseName dataverseName;
         private final String datasetName;
 
-        public CompiledCompactStatement(String dataverseName, String datasetName) {
+        public CompiledCompactStatement(DataverseName dataverseName, String datasetName) {
             this.dataverseName = dataverseName;
             this.datasetName = datasetName;
         }
 
-        public String getDataverseName() {
+        public DataverseName getDataverseName() {
             return dataverseName;
         }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index 3fdcc33..09b8c1e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -35,6 +35,7 @@ import java.util.stream.Collectors;
 import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -250,12 +251,12 @@ public interface IStatementExecutor {
     /**
      * returns the active dataverse for an entity or a statement
      *
-     * @param dataverse:
+     * @param dataverseName:
      *            the entity or statement dataverse
      * @return
      *         returns the passed dataverse if not null, the active dataverse otherwise
      */
-    String getActiveDataverseName(String dataverse);
+    DataverseName getActiveDataverseName(DataverseName dataverseName);
 
     /**
      * Gets the execution plans that are generated during query compilation
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 5851467..3940364 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -38,6 +38,7 @@ import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -651,7 +652,7 @@ class LangExpressionToPlanTranslator
         return distResultOperator;
     }
 
-    private DatasetDataSource validateDatasetInfo(MetadataProvider metadataProvider, String dataverseName,
+    private DatasetDataSource validateDatasetInfo(MetadataProvider metadataProvider, DataverseName dataverseName,
             String datasetName, SourceLocation sourceLoc) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
@@ -887,7 +888,7 @@ class LangExpressionToPlanTranslator
     private AbstractFunctionCallExpression lookupUserDefinedFunction(FunctionSignature signature,
             List<Mutable<ILogicalExpression>> args, SourceLocation sourceLoc) throws CompilationException {
         try {
-            if (signature.getNamespace() == null) {
+            if (signature.getDataverseName() == null) {
                 return null;
             }
             Function function =
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
index 9c6e75e..9e01f6a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
@@ -29,6 +29,7 @@ import java.util.Map;
 
 import org.apache.asterix.common.annotations.IRecordFieldDataGen;
 import org.apache.asterix.common.annotations.RecordDataGenAnnotation;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.expression.OrderedListTypeDefinition;
 import org.apache.asterix.lang.common.expression.RecordTypeDefinition;
 import org.apache.asterix.lang.common.expression.RecordTypeDefinition.RecordKind;
@@ -56,13 +57,14 @@ public class TypeTranslator {
     }
 
     public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeExpression typeExpr,
-            String typeName, String typeDataverse) throws AlgebricksException {
+            String typeName, DataverseName typeDataverse) throws AlgebricksException {
         Map<TypeSignature, IAType> typeMap = new HashMap<>();
         return computeTypes(mdTxnCtx, typeExpr, typeName, typeDataverse, typeMap);
     }
 
     public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeExpression typeExpr,
-            String typeName, String typeDataverse, Map<TypeSignature, IAType> typeMap) throws AlgebricksException {
+            String typeName, DataverseName typeDataverse, Map<TypeSignature, IAType> typeMap)
+            throws AlgebricksException {
         Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<>();
         Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<>();
         Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences = new HashMap<>();
@@ -82,7 +84,7 @@ public class TypeTranslator {
     private static void firstPass(TypeExpression typeExpr, String typeName, Map<TypeSignature, IAType> typeMap,
             Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
             Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, DataverseName typeDataverse)
             throws AlgebricksException {
 
         if (BuiltinTypeMap.getBuiltinType(typeName) != null) {
@@ -92,9 +94,10 @@ public class TypeTranslator {
         switch (typeExpr.getTypeKind()) {
             case TYPEREFERENCE: {
                 TypeReferenceExpression tre = (TypeReferenceExpression) typeExpr;
-                IAType t = solveTypeReference(new TypeSignature(
-                        tre.getIdent().first == null ? typeDataverse : tre.getIdent().first.getValue(),
-                        tre.getIdent().second.getValue()), typeMap);
+                IAType t = solveTypeReference(
+                        new TypeSignature(tre.getIdent().first == null ? typeDataverse : tre.getIdent().first,
+                                tre.getIdent().second.getValue()),
+                        typeMap);
                 if (t != null) {
                     typeMap.put(typeSignature, t);
                 } else {
@@ -132,12 +135,12 @@ public class TypeTranslator {
     private static void secondPass(MetadataTransactionContext mdTxnCtx, Map<TypeSignature, IAType> typeMap,
             Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
             Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, DataverseName typeDataverse)
             throws AlgebricksException {
         // solve remaining top level references
         for (TypeSignature typeSignature : incompleteTopLevelTypeReferences.keySet()) {
             IAType t;
-            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getDataverseName(),
                     typeSignature.getName());
             if (dt == null) {
                 throw new AlgebricksException("Could not resolve type " + typeSignature);
@@ -181,7 +184,7 @@ public class TypeTranslator {
             IAType t;
             Datatype dt;
             if (MetadataManager.INSTANCE != null) {
-                dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+                dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getDataverseName(),
                         typeSignature.getName());
                 if (dt == null) {
                     throw new AlgebricksException("Could not resolve type " + typeSignature);
@@ -198,7 +201,7 @@ public class TypeTranslator {
 
     private static AOrderedListType computeOrderedListType(TypeSignature typeSignature, OrderedListTypeDefinition oltd,
             Map<TypeSignature, IAType> typeMap, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaultDataverse)
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, DataverseName defaultDataverse)
             throws AlgebricksException {
         TypeExpression tExpr = oltd.getItemTypeExpression();
         String typeName = typeSignature != null ? typeSignature.getName() : null;
@@ -210,7 +213,7 @@ public class TypeTranslator {
     private static AUnorderedListType computeUnorderedListType(TypeSignature typeSignature,
             UnorderedListTypeDefinition ultd, Map<TypeSignature, IAType> typeMap,
             Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaulDataverse)
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, DataverseName defaulDataverse)
             throws AlgebricksException {
         TypeExpression tExpr = ultd.getItemTypeExpression();
         String typeName = typeSignature != null ? typeSignature.getName() : null;
@@ -222,7 +225,7 @@ public class TypeTranslator {
     private static void setCollectionItemType(TypeExpression tExpr, Map<TypeSignature, IAType> typeMap,
             Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
             Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act,
-            String defaultDataverse) throws AlgebricksException {
+            DataverseName defaultDataverse) throws AlgebricksException {
         switch (tExpr.getTypeKind()) {
             case ORDEREDLIST: {
                 OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
@@ -247,9 +250,9 @@ public class TypeTranslator {
             }
             case TYPEREFERENCE: {
                 TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
-                TypeSignature signature = new TypeSignature(
-                        tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(),
-                        tre.getIdent().second.getValue());
+                TypeSignature signature =
+                        new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first,
+                                tre.getIdent().second.getValue());
                 IAType tref = solveTypeReference(signature, typeMap);
                 if (tref != null) {
                     act.setItemType(tref);
@@ -266,10 +269,10 @@ public class TypeTranslator {
 
     private static void addIncompleteCollectionTypeReference(AbstractCollectionType collType,
             TypeReferenceExpression tre, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
-            String defaultDataverse) {
+            DataverseName defaultDataverse) {
         String typeName = tre.getIdent().second.getValue();
-        TypeSignature typeSignature = new TypeSignature(
-                tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(), typeName);
+        TypeSignature typeSignature =
+                new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first, typeName);
         List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeSignature);
         if (typeList == null) {
             typeList = new LinkedList<>();
@@ -295,17 +298,16 @@ public class TypeTranslator {
     }
 
     private static void addIncompleteTopLevelTypeReference(TypeReferenceExpression tre,
-            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String defaultDataverse) {
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, DataverseName defaultDataverse) {
         String name = tre.getIdent().second.getValue();
-        TypeSignature typeSignature = new TypeSignature(
-                tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(), name);
-        List<TypeSignature> refList = incompleteTopLevelTypeReferences.get(name);
+        TypeSignature typeSignature =
+                new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first, name);
+        List<TypeSignature> refList = incompleteTopLevelTypeReferences.get(typeSignature);
         if (refList == null) {
             refList = new LinkedList<>();
-            incompleteTopLevelTypeReferences.put(
-                    new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(),
-                            tre.getIdent().second.getValue()),
-                    refList);
+            incompleteTopLevelTypeReferences
+                    .put(new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first,
+                            tre.getIdent().second.getValue()), refList);
         }
         refList.add(typeSignature);
     }
@@ -321,7 +323,7 @@ public class TypeTranslator {
 
     private static ARecordType computeRecordType(TypeSignature typeSignature, RecordTypeDefinition rtd,
             Map<TypeSignature, IAType> typeMap, Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes, String defaultDataverse)
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes, DataverseName defaultDataverse)
             throws AlgebricksException {
         List<String> names = rtd.getFieldNames();
         int n = names.size();
@@ -346,9 +348,9 @@ public class TypeTranslator {
             switch (texpr.getTypeKind()) {
                 case TYPEREFERENCE: {
                     TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-                    TypeSignature signature = new TypeSignature(
-                            tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(),
-                            tre.getIdent().second.getValue());
+                    TypeSignature signature =
+                            new TypeSignature(tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first,
+                                    tre.getIdent().second.getValue());
                     IAType tref = solveTypeReference(signature, typeMap);
                     if (tref != null) {
                         if (!rtd.getOptionableFields().get(j)) { // not nullable
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
index e422c24..2a08511 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ConnectorApiServlet.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -79,7 +80,7 @@ public class ConnectorApiServlet extends AbstractServlet {
         PrintWriter out = response.writer();
         try {
             ObjectNode jsonResponse = OBJECT_MAPPER.createObjectNode();
-            String dataverseName = request.getParameter("dataverseName");
+            DataverseName dataverseName = ServletUtil.getDataverseName(request, "dataverseName");
             String datasetName = request.getParameter("datasetName");
             if (dataverseName == null || datasetName == null) {
                 jsonResponse.put("error", "Parameter dataverseName or datasetName is null,");
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index 320c7aa..02d75f9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -37,11 +37,13 @@ import java.util.concurrent.Future;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Dataverse;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.rebalance.NoOpDatasetRebalanceCallback;
 import org.apache.asterix.utils.RebalanceUtil;
 import org.apache.commons.lang3.StringUtils;
@@ -66,7 +68,6 @@ import io.netty.handler.codec.http.HttpResponseStatus;
  */
 public class RebalanceApiServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
-    private static final String METADATA = "Metadata";
     private final ICcApplicationContext appCtx;
 
     // One-at-a-time thread executor, for rebalance tasks.
@@ -103,7 +104,7 @@ public class RebalanceApiServlet extends AbstractServlet {
     protected void post(IServletRequest request, IServletResponse response) {
         try {
             // Gets dataverse, dataset, and target nodes for rebalance.
-            String dataverseName = request.getParameter("dataverseName");
+            DataverseName dataverseName = ServletUtil.getDataverseName(request, "dataverseName");
             String datasetName = request.getParameter("datasetName");
             String nodes = request.getParameter("nodes");
 
@@ -127,7 +128,7 @@ public class RebalanceApiServlet extends AbstractServlet {
             }
 
             // Does not allow rebalancing a metadata dataset.
-            if (METADATA.equals(dataverseName)) {
+            if (MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName)) {
                 sendResponse(response, HttpResponseStatus.BAD_REQUEST, "cannot rebalance a metadata dataset");
                 return;
             }
@@ -154,7 +155,7 @@ public class RebalanceApiServlet extends AbstractServlet {
     }
 
     // Schedules a rebalance task.
-    private synchronized CountDownLatch scheduleRebalance(String dataverseName, String datasetName,
+    private synchronized CountDownLatch scheduleRebalance(DataverseName dataverseName, String datasetName,
             String[] targetNodes, IServletResponse response) {
         CountDownLatch terminated = new CountDownLatch(1);
         Future<Void> task =
@@ -165,8 +166,8 @@ public class RebalanceApiServlet extends AbstractServlet {
     }
 
     // Performs the actual rebalance.
-    private Void doRebalance(String dataverseName, String datasetName, String[] targetNodes, IServletResponse response,
-            CountDownLatch terminated) {
+    private Void doRebalance(DataverseName dataverseName, String datasetName, String[] targetNodes,
+            IServletResponse response, CountDownLatch terminated) {
         try {
             // Sets the content type.
             HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, HttpUtil.Encoding.UTF8);
@@ -204,7 +205,7 @@ public class RebalanceApiServlet extends AbstractServlet {
     }
 
     // Lists all datasets that should be rebalanced in a given datavserse.
-    private List<Dataset> getAllDatasetsForRebalance(String dataverseName) throws Exception {
+    private List<Dataset> getAllDatasetsForRebalance(DataverseName dataverseName) throws Exception {
         List<Dataset> datasets;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         try {
@@ -235,14 +236,15 @@ public class RebalanceApiServlet extends AbstractServlet {
     }
 
     // Gets all datasets in a dataverse for the rebalance operation, with a given metadata transaction context.
-    private List<Dataset> getDatasetsInDataverseForRebalance(String dvName, MetadataTransactionContext mdTxnCtx)
+    private List<Dataset> getDatasetsInDataverseForRebalance(DataverseName dvName, MetadataTransactionContext mdTxnCtx)
             throws Exception {
-        return METADATA.equals(dvName) ? Collections.emptyList()
+        return MetadataConstants.METADATA_DATAVERSE_NAME.equals(dvName) ? Collections.emptyList()
                 : MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
     }
 
     // Rebalances a given dataset.
-    private void rebalanceDataset(String dataverseName, String datasetName, String[] targetNodes) throws Exception {
+    private void rebalanceDataset(DataverseName dataverseName, String datasetName, String[] targetNodes)
+            throws Exception {
         IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
         MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
         try {
@@ -251,8 +253,8 @@ public class RebalanceApiServlet extends AbstractServlet {
             activeNotificationHandler.suspend(metadataProvider);
             try {
                 IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
-                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
-                        dataverseName + '.' + datasetName);
+                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataverseName,
+                        datasetName);
                 RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
                         metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE);
             } finally {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
index 2308ea3..9e85c82 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ServletUtil.java
@@ -20,13 +20,16 @@ package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.RESULTSET_ATTR;
 
+import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.client.result.ResultSet;
+import org.apache.hyracks.http.api.IServletRequest;
 
 public class ServletUtil {
     static IResultSet getResultSet(IHyracksClientConnection hcc, IApplicationContext appCtx,
@@ -46,4 +49,9 @@ public class ServletUtil {
         }
         return resultSet;
     }
+
+    public static DataverseName getDataverseName(IServletRequest request, String dataverseParameterName) {
+        List<String> values = request.getParameterValues(dataverseParameterName);
+        return values != null ? DataverseName.create(values) : null;
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UdfApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UdfApiServlet.java
index 79b78a8..c58651f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UdfApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UdfApiServlet.java
@@ -33,6 +33,8 @@ import org.apache.asterix.app.message.LoadUdfMessage;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.http.api.IServletRequest;
@@ -61,28 +63,28 @@ public class UdfApiServlet extends AbstractServlet {
         this.broker = (ICCMessageBroker) appCtx.getServiceContext().getMessageBroker();
     }
 
-    private String[] getResource(FullHttpRequest req) throws IllegalArgumentException {
+    private Pair<String, DataverseName> getResource(FullHttpRequest req) throws IllegalArgumentException {
         String[] path = new QueryStringDecoder(req.uri()).path().split("/");
         if (path.length != 5) {
             throw new IllegalArgumentException("Invalid resource.");
         }
         String resourceName = path[path.length - 1];
-        String dataverseName = path[path.length - 2];
-        return new String[] { resourceName, dataverseName };
+        DataverseName dataverseName = DataverseName.createSinglePartName(path[path.length - 2]); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
+        return new Pair<>(resourceName, dataverseName);
     }
 
     @Override
     protected void post(IServletRequest request, IServletResponse response) {
         FullHttpRequest req = request.getHttpRequest();
-        String[] resourceNames;
+        Pair<String, DataverseName> resourceNames;
         try {
             resourceNames = getResource(req);
         } catch (IllegalArgumentException e) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-        String resourceName = resourceNames[0];
-        String dataverse = resourceNames[1];
+        String resourceName = resourceNames.first;
+        DataverseName dataverse = resourceNames.second;
         File udf = null;
         try {
             File workingDir = new File(appCtx.getServiceContext().getServerCtx().getBaseDir().getAbsolutePath(),
@@ -101,7 +103,7 @@ public class UdfApiServlet extends AbstractServlet {
                 }
             }
             IHyracksClientConnection hcc = appCtx.getHcc();
-            DeploymentId udfName = new DeploymentId(dataverse + "." + resourceName);
+            DeploymentId udfName = new DeploymentId(dataverse.getCanonicalForm() + "." + resourceName); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
             ClassLoader cl = appCtx.getLibraryManager().getLibraryClassLoader(dataverse, resourceName);
             if (cl != null) {
                 deleteUdf(dataverse, resourceName);
@@ -129,27 +131,27 @@ public class UdfApiServlet extends AbstractServlet {
 
     }
 
-    private void deleteUdf(String dataverse, String resourceName) throws Exception {
+    private void deleteUdf(DataverseName dataverse, String resourceName) throws Exception {
         long reqId = broker.newRequestId();
         List<INcAddressedMessage> requests = new ArrayList<>();
         List<String> ncs = new ArrayList<>(appCtx.getClusterStateManager().getParticipantNodes());
         ncs.forEach(s -> requests.add(new DeleteUdfMessage(dataverse, resourceName, reqId)));
         broker.sendSyncRequestToNCs(reqId, ncs, requests, UDF_RESPONSE_TIMEOUT);
         appCtx.getLibraryManager().deregisterLibraryClassLoader(dataverse, resourceName);
-        appCtx.getHcc().unDeployBinary(new DeploymentId(resourceName));
+        appCtx.getHcc().unDeployBinary(new DeploymentId(resourceName)); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT:why dataverse not used?
     }
 
     @Override
     protected void delete(IServletRequest request, IServletResponse response) {
-        String[] resourceNames;
+        Pair<String, DataverseName> resourceNames;
         try {
             resourceNames = getResource(request.getHttpRequest());
         } catch (IllegalArgumentException e) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-        String resourceName = resourceNames[0];
-        String dataverse = resourceNames[1];
+        String resourceName = resourceNames.first;
+        DataverseName dataverse = resourceNames.second;
         try {
             deleteUdf(dataverse, resourceName);
         } catch (Exception e) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index a572e28..0a7cad6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -32,9 +32,9 @@ import org.apache.asterix.active.message.ActivePartitionMessage;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
@@ -280,13 +280,13 @@ public class ActiveNotificationHandler extends SingleThreadEventProcessor<Active
             // write lock the listener
             // exclusive lock all the datasets (except the target dataset)
             IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
-            String dataverseName = listener.getEntityId().getDataverse();
+            DataverseName dataverseName = listener.getEntityId().getDataverseName();
             String entityName = listener.getEntityId().getEntityName();
             if (LOGGER.isEnabled(level)) {
                 LOGGER.log(level, "Suspending " + listener.getEntityId());
             }
             LOGGER.log(level, "Acquiring locks");
-            lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(), dataverseName + '.' + entityName);
+            lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(), dataverseName, entityName);
             List<Dataset> datasets = ((ActiveEntityEventsListener) listener).getDatasets();
             for (Dataset dataset : datasets) {
                 if (targetDataset != null && targetDataset.equals(dataset)) {
@@ -294,7 +294,7 @@ public class ActiveNotificationHandler extends SingleThreadEventProcessor<Active
                     continue;
                 }
                 lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
-                        DatasetUtil.getFullyQualifiedName(dataset));
+                        dataset.getDataverseName(), dataset.getDatasetName());
             }
             LOGGER.log(level, "locks acquired");
             ((ActiveEntityEventsListener) listener).suspend(metadataProvider);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index a1989fc..6efda9f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -21,6 +21,7 @@ package org.apache.asterix.app.active;
 import java.util.concurrent.Callable;
 
 import org.apache.asterix.active.ActivityState;
+import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.IRetryPolicyFactory;
 import org.apache.asterix.active.NoRetryPolicyFactory;
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -29,7 +30,6 @@ import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.metadata.utils.MetadataLockUtil;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -173,12 +173,13 @@ public class RecoveryTask {
     }
 
     protected void acquireRecoveryLocks(IMetadataLockManager lockManager) throws AlgebricksException {
-        lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
-                listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
+        EntityId entityId = listener.getEntityId();
+        lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(), entityId.getDataverseName(),
+                entityId.getEntityName());
         for (Dataset dataset : listener.getDatasets()) {
             lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dataset.getDataverseName());
-            lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
-                    DatasetUtil.getFullyQualifiedName(dataset));
+            lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataset.getDataverseName(),
+                    dataset.getDatasetName());
         }
     }
 
@@ -187,11 +188,12 @@ public class RecoveryTask {
     }
 
     protected void acquirePostRecoveryLocks(IMetadataLockManager lockManager) throws AlgebricksException {
-        lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(),
-                listener.getEntityId().getDataverse() + '.' + listener.getEntityId().getEntityName());
+        EntityId entityId = listener.getEntityId();
+        lockManager.acquireActiveEntityWriteLock(metadataProvider.getLocks(), entityId.getDataverseName(),
+                entityId.getEntityName());
         for (Dataset dataset : listener.getDatasets()) {
-            MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataset.getDatasetName(),
-                    DatasetUtil.getFullyQualifiedName(dataset));
+            MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataset.getDataverseName(),
+                    dataset.getDatasetName());
         }
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index a989941..6bb9bbd 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -36,6 +36,7 @@ import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.api.IDataSourceAdapter;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
 import org.apache.asterix.external.library.ExternalLibrary;
@@ -64,7 +65,7 @@ public class ExternalLibraryUtils {
     public static void setUpExternaLibrary(ILibraryManager externalLibraryManager, boolean isMetadataNode,
             String libraryPath) throws Exception {
         // start by un-installing removed libraries (Metadata Node only)
-        Map<String, List<String>> uninstalledLibs = null;
+        Map<DataverseName, List<String>> uninstalledLibs = null;
         if (isMetadataNode) {
             uninstalledLibs = uninstallLibraries();
         }
@@ -72,7 +73,7 @@ public class ExternalLibraryUtils {
         // get the directory of the to be installed libraries
         String[] pathSplit = libraryPath.split("\\.");
         String[] dvSplit = pathSplit[pathSplit.length - 2].split("/");
-        String dataverse = dvSplit[dvSplit.length - 1];
+        DataverseName dataverse = DataverseName.createSinglePartName(dvSplit[dvSplit.length - 1]); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
         String name = pathSplit[pathSplit.length - 1].trim();
         File installLibDir = new File(libraryPath);
 
@@ -104,8 +105,8 @@ public class ExternalLibraryUtils {
      * @return a map from dataverse -> list of uninstalled libraries.
      * @throws Exception
      */
-    private static Map<String, List<String>> uninstallLibraries() throws Exception {
-        Map<String, List<String>> uninstalledLibs = new HashMap<>();
+    private static Map<DataverseName, List<String>> uninstallLibraries() throws Exception {
+        Map<DataverseName, List<String>> uninstalledLibs = new HashMap<>();
         // get the directory of the un-install libraries
         File uninstallLibDir = getLibraryUninstallDir();
         String[] uninstallLibNames;
@@ -116,7 +117,7 @@ public class ExternalLibraryUtils {
             for (String uninstallLibName : uninstallLibNames) {
                 // Get the <dataverse name - library name> pair
                 String[] components = uninstallLibName.split("\\.");
-                String dataverse = components[0];
+                DataverseName dataverse = DataverseName.createSinglePartName(components[0]); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
                 String libName = components[1];
                 // un-install
                 uninstallLibrary(dataverse, libName);
@@ -147,7 +148,7 @@ public class ExternalLibraryUtils {
      * @throws RemoteException
      * @throws ACIDException
      */
-    public static boolean uninstallLibrary(String dataverse, String libraryName)
+    public static boolean uninstallLibrary(DataverseName dataverse, String libraryName)
             throws AsterixException, RemoteException, ACIDException {
         MetadataTransactionContext mdTxnCtx = null;
         try {
@@ -194,7 +195,7 @@ public class ExternalLibraryUtils {
         return true;
     }
 
-    private static void addLibraryToMetadata(Map<String, List<String>> uninstalledLibs, String dataverse,
+    private static void addLibraryToMetadata(Map<DataverseName, List<String>> uninstalledLibs, DataverseName dataverse,
             String libraryName, ExternalLibrary library) throws ACIDException, RemoteException {
         // Modify metadata accordingly
         List<String> uninstalledLibsInDv = uninstalledLibs.get(dataverse);
@@ -283,8 +284,9 @@ public class ExternalLibraryUtils {
      * failure in installing an element does not effect installation of other
      * libraries.
      */
-    protected static void configureLibrary(ILibraryManager libraryManager, String dataverse, String libraryName,
-            final File libraryDir, Map<String, List<String>> uninstalledLibs, boolean isMetadataNode) throws Exception {
+    protected static void configureLibrary(ILibraryManager libraryManager, DataverseName dataverse, String libraryName,
+            final File libraryDir, Map<DataverseName, List<String>> uninstalledLibs, boolean isMetadataNode)
+            throws Exception {
 
         String[] libraryDescriptors = libraryDir.list((dir, name) -> name.endsWith(".xml"));
 
@@ -318,8 +320,8 @@ public class ExternalLibraryUtils {
      * @param libraryPath
      * @throws Exception
      */
-    protected static void registerClassLoader(ILibraryManager externalLibraryManager, String dataverse, String name,
-            String libraryPath) throws Exception {
+    protected static void registerClassLoader(ILibraryManager externalLibraryManager, DataverseName dataverse,
+            String name, String libraryPath) throws Exception {
         // get the class loader
         URLClassLoader classLoader = getLibraryClassLoader(dataverse, name, libraryPath);
         // register it with the external library manager
@@ -343,12 +345,12 @@ public class ExternalLibraryUtils {
     /**
      * Get the class loader for the library
      *
-     * @param libraryPath
      * @param dataverse
+     * @param libraryPath
      * @return
      * @throws Exception
      */
-    private static URLClassLoader getLibraryClassLoader(String dataverse, String name, String libraryPath)
+    private static URLClassLoader getLibraryClassLoader(DataverseName dataverse, String name, String libraryPath)
             throws Exception {
         // Get a reference to the library directory
         File installDir = new File(libraryPath);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
index f1687b5..c663617 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetResourcesRewriter.java
@@ -21,6 +21,7 @@ package org.apache.asterix.app.function;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -44,7 +45,7 @@ public class DatasetResourcesRewriter extends FunctionRewriter {
     public DatasetResourcesDatasource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
             throws AlgebricksException {
         final SourceLocation loc = f.getSourceLocation();
-        String dataverseName = getString(loc, f.getArguments(), 0);
+        DataverseName dataverseName = getDataverseName(loc, f.getArguments(), 0);
         String datasetName = getString(loc, f.getArguments(), 1);
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
index 590bf54..3b65f96 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DatasetRewriter.java
@@ -24,21 +24,16 @@ import java.util.List;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
-import org.apache.asterix.metadata.entities.Dataverse;
-import org.apache.asterix.metadata.utils.DatasetUtil;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.IFunctionToDataSourceRewriter;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.rules.UnnestToDataScanRule;
 import org.apache.asterix.optimizer.rules.util.EquivalenceClassUtils;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -47,11 +42,8 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -73,30 +65,18 @@ public class DatasetRewriter implements IFunctionToDataSourceRewriter, IResultTy
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
                     "No positional variables are allowed over datasets.");
         }
-        ILogicalExpression expr = f.getArguments().get(0).getValue();
-        if (expr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            return false;
-        }
-        ConstantExpression ce = (ConstantExpression) expr;
-        IAlgebricksConstantValue acv = ce.getValue();
-        if (!(acv instanceof AsterixConstantValue)) {
-            return false;
-        }
-        AsterixConstantValue acv2 = (AsterixConstantValue) acv;
-        if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
-            return false;
-        }
-        String datasetArg = ((AString) acv2.getObject()).getStringValue();
+
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
-        Pair<String, String> datasetReference = parseDatasetReference(metadataProvider, datasetArg);
-        String dataverseName = datasetReference.first;
+        Pair<DataverseName, String> datasetReference = FunctionUtil.parseDatasetFunctionArguments(f.getArguments(),
+                metadataProvider.getDefaultDataverseName(), unnest.getSourceLocation());
+        DataverseName dataverseName = datasetReference.first;
         String datasetName = datasetReference.second;
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
             throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, unnest.getSourceLocation(),
                     datasetName, dataverseName);
         }
-        DataSourceId asid = new DataSourceId(dataverseName, datasetName);
+        DataSourceId dsid = new DataSourceId(dataset.getDataverseName(), dataset.getDatasetName());
         List<LogicalVariable> variables = new ArrayList<>();
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             int numPrimaryKeys = dataset.getPrimaryKeys().size();
@@ -105,7 +85,7 @@ public class DatasetRewriter implements IFunctionToDataSourceRewriter, IResultTy
             }
         }
         variables.add(unnest.getVariable());
-        DataSource dataSource = metadataProvider.findDataSource(asid);
+        DataSource dataSource = metadataProvider.findDataSource(dsid);
         boolean hasMeta = dataSource.hasMeta();
         if (hasMeta) {
             variables.add(context.newVar());
@@ -128,25 +108,6 @@ public class DatasetRewriter implements IFunctionToDataSourceRewriter, IResultTy
         return true;
     }
 
-    private Pair<String, String> parseDatasetReference(MetadataProvider metadataProvider, String datasetArg)
-            throws AlgebricksException {
-        String[] datasetNameComponents = datasetArg.split("\\.");
-        String dataverseName;
-        String datasetName;
-        if (datasetNameComponents.length == 1) {
-            Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
-            if (defaultDataverse == null) {
-                throw new AlgebricksException("Unresolved dataset " + datasetArg + " Dataverse not specified.");
-            }
-            dataverseName = defaultDataverse.getDataverseName();
-            datasetName = datasetNameComponents[0];
-        } else {
-            dataverseName = datasetNameComponents[0];
-            datasetName = datasetNameComponents[1];
-        }
-        return new Pair<>(dataverseName, datasetName);
-    }
-
     private void addPrimaryKey(List<LogicalVariable> scanVariables, DataSource dataSource,
             IOptimizationContext context) {
         List<LogicalVariable> primaryKey = dataSource.getPrimaryKeyVariables(scanVariables);
@@ -160,31 +121,15 @@ public class DatasetRewriter implements IFunctionToDataSourceRewriter, IResultTy
     public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env, IMetadataProvider<?, ?> mp)
             throws AlgebricksException {
         AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
-        if (f.getArguments().size() != 1) {
-            throw new AlgebricksException("dataset arity is 1, not " + f.getArguments().size());
-        }
-        ILogicalExpression a1 = f.getArguments().get(0).getValue();
-        IAType t1 = (IAType) env.getType(a1);
-        if (t1.getTypeTag() == ATypeTag.ANY) {
-            return BuiltinType.ANY;
-        }
-        if (t1.getTypeTag() != ATypeTag.STRING) {
-            throw new AlgebricksException("Illegal type " + t1 + " for dataset() argument.");
-        }
-        String datasetArg = ConstantExpressionUtil.getStringConstant(a1);
-        if (datasetArg == null) {
-            return BuiltinType.ANY;
-        }
         MetadataProvider metadata = (MetadataProvider) mp;
-        Pair<String, String> datasetInfo = DatasetUtil.getDatasetInfo(metadata, datasetArg);
-        String dataverseName = datasetInfo.first;
+        Pair<DataverseName, String> datasetInfo = FunctionUtil.parseDatasetFunctionArguments(f.getArguments(),
+                metadata.getDefaultDataverseName(), f.getSourceLocation());
+        DataverseName dataverseName = datasetInfo.first;
         String datasetName = datasetInfo.second;
-        if (dataverseName == null) {
-            throw new AlgebricksException("Unspecified dataverse!");
-        }
         Dataset dataset = metadata.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
+            throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, f.getSourceLocation(), datasetName,
+                    dataverseName);
         }
         String tn = dataset.getItemTypeName();
         IAType t2 = metadata.findType(dataset.getItemTypeDataverseName(), tn);
@@ -192,6 +137,5 @@ public class DatasetRewriter implements IFunctionToDataSourceRewriter, IResultTy
             throw new AlgebricksException("No type for dataset " + datasetName);
         }
         return t2;
-
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
index 08e5d51..000e910 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/DumpIndexRewriter.java
@@ -21,6 +21,7 @@ package org.apache.asterix.app.function;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -46,7 +47,7 @@ public class DumpIndexRewriter extends FunctionRewriter {
     public DumpIndexDatasource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
             throws AlgebricksException {
         final SourceLocation loc = f.getSourceLocation();
-        String dataverseName = getString(loc, f.getArguments(), 0);
+        DataverseName dataverseName = getDataverseName(loc, f.getArguments(), 0);
         String datasetName = getString(loc, f.getArguments(), 1);
         String indexName = getString(loc, f.getArguments(), 2);
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
index bb80406..90374c3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FeedRewriter.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.util.ExternalDataUtils;
 import org.apache.asterix.external.util.FeedUtils;
@@ -36,12 +37,10 @@ import org.apache.asterix.metadata.entities.FeedConnection;
 import org.apache.asterix.metadata.entities.FeedPolicyEntity;
 import org.apache.asterix.metadata.entities.InternalDatasetDetails;
 import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
-import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionToDataSourceRewriter;
 import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
 import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
@@ -49,7 +48,6 @@ import org.apache.asterix.optimizer.rules.UnnestToDataScanRule;
 import org.apache.asterix.translator.util.PlanTranslationUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -75,16 +73,17 @@ public class FeedRewriter implements IFunctionToDataSourceRewriter, IResultTypeC
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, unnest.getSourceLocation(),
                     "No positional variables are allowed over feeds.");
         }
-        String dataverse = ConstantExpressionUtil.getStringArgument(f, 0);
+        DataverseName dataverseName =
+                DataverseName.createFromCanonicalForm(ConstantExpressionUtil.getStringArgument(f, 0));
         String sourceFeedName = ConstantExpressionUtil.getStringArgument(f, 1);
         String getTargetFeed = ConstantExpressionUtil.getStringArgument(f, 2);
         String subscriptionLocation = ConstantExpressionUtil.getStringArgument(f, 3);
         String targetDataset = ConstantExpressionUtil.getStringArgument(f, 4);
         String outputType = ConstantExpressionUtil.getStringArgument(f, 5);
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
-        DataSourceId asid = new DataSourceId(dataverse, getTargetFeed);
+        DataSourceId asid = new DataSourceId(dataverseName, getTargetFeed);
         String policyName = (String) metadataProvider.getConfig().get(FeedActivityDetails.FEED_POLICY_NAME);
-        FeedPolicyEntity policy = metadataProvider.findFeedPolicy(dataverse, policyName);
+        FeedPolicyEntity policy = metadataProvider.findFeedPolicy(dataverseName, policyName);
         if (policy == null) {
             policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
             if (policy == null) {
@@ -116,15 +115,15 @@ public class FeedRewriter implements IFunctionToDataSourceRewriter, IResultTypeC
         return true;
     }
 
-    private FeedDataSource createFeedDataSource(DataSourceId aqlId, String targetDataset, String sourceFeedName,
+    private FeedDataSource createFeedDataSource(DataSourceId id, String targetDataset, String sourceFeedName,
             String subscriptionLocation, MetadataProvider metadataProvider, FeedPolicyEntity feedPolicy,
             String outputType, String locations, LogicalVariable recordVar, IOptimizationContext context,
             List<LogicalVariable> pkVars) throws AlgebricksException {
-        Dataset dataset = metadataProvider.findDataset(aqlId.getDataverseName(), targetDataset);
-        ARecordType feedOutputType = (ARecordType) metadataProvider.findType(aqlId.getDataverseName(), outputType);
-        Feed sourceFeed = metadataProvider.findFeed(aqlId.getDataverseName(), sourceFeedName);
+        Dataset dataset = metadataProvider.findDataset(id.getDataverseName(), targetDataset);
+        ARecordType feedOutputType = (ARecordType) metadataProvider.findType(id.getDataverseName(), outputType);
+        Feed sourceFeed = metadataProvider.findFeed(id.getDataverseName(), sourceFeedName);
         FeedConnection feedConnection =
-                metadataProvider.findFeedConnection(aqlId.getDataverseName(), sourceFeedName, targetDataset);
+                metadataProvider.findFeedConnection(id.getDataverseName(), sourceFeedName, targetDataset);
         ARecordType metaType = null;
         // Does dataset have meta?
         if (dataset.hasMetaPart()) {
@@ -132,12 +131,7 @@ public class FeedRewriter implements IFunctionToDataSourceRewriter, IResultTypeC
             if (metaTypeName == null) {
                 throw new AlgebricksException("Feed to a dataset with metadata doesn't have meta type specified");
             }
-            String dataverseName = aqlId.getDataverseName();
-            if (metaTypeName.contains(".")) {
-                dataverseName = metaTypeName.substring(0, metaTypeName.indexOf('.'));
-                metaTypeName = metaTypeName.substring(metaTypeName.indexOf('.') + 1);
-            }
-            metaType = (ARecordType) metadataProvider.findType(dataverseName, metaTypeName);
+            metaType = (ARecordType) metadataProvider.findType(id.getDataverseName(), metaTypeName);
         }
         // Is a change feed?
         List<IAType> pkTypes = null;
@@ -168,7 +162,7 @@ public class FeedRewriter implements IFunctionToDataSourceRewriter, IResultTypeC
         } else {
             keyAccessScalarFunctionCallExpression = null;
         }
-        FeedDataSource feedDataSource = new FeedDataSource(sourceFeed, aqlId, targetDataset, feedOutputType, metaType,
+        FeedDataSource feedDataSource = new FeedDataSource(sourceFeed, id, targetDataset, feedOutputType, metaType,
                 pkTypes, keyAccessScalarFunctionCallExpression, sourceFeed.getFeedId(),
                 FeedRuntimeType.valueOf(subscriptionLocation), locations.split(","), context.getComputationNodeDomain(),
                 feedConnection);
@@ -184,31 +178,17 @@ public class FeedRewriter implements IFunctionToDataSourceRewriter, IResultTypeC
             throw new AlgebricksException("Incorrect number of arguments -> arity is "
                     + BuiltinFunctions.FEED_COLLECT.getArity() + ", not " + f.getArguments().size());
         }
-        ILogicalExpression a1 = f.getArguments().get(5).getValue();
-        IAType t1 = (IAType) env.getType(a1);
-        if (t1.getTypeTag() == ATypeTag.ANY) {
-            return BuiltinType.ANY;
-        }
-        if (t1.getTypeTag() != ATypeTag.STRING) {
-            throw new AlgebricksException("Illegal type " + t1 + " for feed-ingest argument.");
-        }
-        String typeArg = ConstantExpressionUtil.getStringConstant(a1);
-        if (typeArg == null) {
+        DataverseName dataverseName =
+                DataverseName.createFromCanonicalForm(ConstantExpressionUtil.getStringArgument(f, 0));
+        String outputTypeName = ConstantExpressionUtil.getStringArgument(f, 5);
+        if (outputTypeName == null) {
             return BuiltinType.ANY;
         }
         MetadataProvider metadata = (MetadataProvider) mp;
-        Pair<String, String> argInfo = DatasetUtil.getDatasetInfo(metadata, typeArg);
-        String dataverseName = argInfo.first;
-        String typeName = argInfo.second;
-        if (dataverseName == null) {
-            throw new AlgebricksException("Unspecified dataverse!");
+        IAType outputType = metadata.findType(dataverseName, outputTypeName);
+        if (outputType == null) {
+            throw new AlgebricksException("Unknown type " + outputTypeName);
         }
-        IAType t2 = metadata.findType(dataverseName, typeName);
-        if (t2 == null) {
-            throw new AlgebricksException("Unknown type  " + typeName);
-        }
-        return t2;
-
+        return outputType;
     }
-
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
index 1cc0a94..dfe29c2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/FunctionRewriter.java
@@ -25,12 +25,15 @@ import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.FunctionDataSource;
+import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.exceptions.UnsupportedTypeException;
 import org.apache.asterix.om.functions.IFunctionToDataSourceRewriter;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.optimizer.rules.UnnestToDataScanRule;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -40,8 +43,6 @@ import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -96,17 +97,43 @@ public abstract class FunctionRewriter implements IFunctionToDataSourceRewriter
 
     protected String getString(SourceLocation loc, List<Mutable<ILogicalExpression>> args, int i)
             throws AlgebricksException {
-        ConstantExpression ce = (ConstantExpression) args.get(i).getValue();
-        IAlgebricksConstantValue acv = ce.getValue();
-        if (!(acv instanceof AsterixConstantValue)) {
+        IAObject iaObject = ConstantExpressionUtil.getConstantIaObject(args.get(i).getValue(), null);
+        if (iaObject == null) {
             throw new CompilationException(EXPECTED_CONSTANT_VALUE, loc);
         }
-        AsterixConstantValue acv2 = (AsterixConstantValue) acv;
-        final ATypeTag typeTag = acv2.getObject().getType().getTypeTag();
-        if (typeTag != ATypeTag.STRING) {
-            throw new UnsupportedTypeException(loc, functionId, typeTag);
+        ATypeTag tt = iaObject.getType().getTypeTag();
+        if (tt != ATypeTag.STRING) {
+            throw new UnsupportedTypeException(loc, functionId, tt);
         }
-        return ((AString) acv2.getObject()).getStringValue();
+        return ((AString) iaObject).getStringValue();
     }
 
+    protected DataverseName getDataverseName(SourceLocation loc, List<Mutable<ILogicalExpression>> args, int i)
+            throws AlgebricksException {
+        IAObject iaObject = ConstantExpressionUtil.getConstantIaObject(args.get(i).getValue(), null);
+        if (iaObject == null) {
+            throw new CompilationException(EXPECTED_CONSTANT_VALUE, loc);
+        }
+        ATypeTag tt = iaObject.getType().getTypeTag();
+        switch (tt) {
+            case STRING:
+                AString str = (AString) iaObject;
+                return DataverseName.createSinglePartName(str.getStringValue());
+            case ARRAY:
+                AOrderedList list = ((AOrderedList) iaObject);
+                int ln = list.size();
+                List<String> parts = new ArrayList<>(ln);
+                for (int j = 0; j < ln; j++) {
+                    IAObject item = list.getItem(j);
+                    ATypeTag itt = item.getType().getTypeTag();
+                    if (itt != ATypeTag.STRING) {
+                        throw new UnsupportedTypeException(loc, functionId, itt);
+                    }
+                    parts.add(((AString) item).getStringValue());
+                }
+                return DataverseName.create(parts);
+            default:
+                throw new UnsupportedTypeException(loc, functionId, tt);
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
index 3b89d50..fb0a6d4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/function/StorageComponentsRewriter.java
@@ -21,6 +21,7 @@ package org.apache.asterix.app.function;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -44,7 +45,7 @@ public class StorageComponentsRewriter extends FunctionRewriter {
     public StorageComponentsDatasource toDatasource(IOptimizationContext context, AbstractFunctionCallExpression f)
             throws AlgebricksException {
         SourceLocation loc = f.getSourceLocation();
-        String dataverseName = getString(loc, f.getArguments(), 0);
+        DataverseName dataverseName = getDataverseName(loc, f.getArguments(), 0);
         String datasetName = getString(loc, f.getArguments(), 1);
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractUdfMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractUdfMessage.java
index 90bd2a6..1c212fe 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractUdfMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/AbstractUdfMessage.java
@@ -23,20 +23,21 @@ import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.messaging.CcIdentifiedMessage;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.INcAddressedMessage;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 public abstract class AbstractUdfMessage extends CcIdentifiedMessage implements INcAddressedMessage {
 
-    protected final String dataverseName;
+    protected final DataverseName dataverseName;
     protected final String libraryName;
     protected static final Logger LOGGER = LogManager.getLogger();
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
 
     private final long reqId;
 
-    public AbstractUdfMessage(String dataverseName, String libraryName, long reqId) {
+    public AbstractUdfMessage(DataverseName dataverseName, String libraryName, long reqId) {
         this.dataverseName = dataverseName;
         this.libraryName = libraryName;
         this.reqId = reqId;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/DeleteUdfMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/DeleteUdfMessage.java
index efbc9c1..3fb58fa 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/DeleteUdfMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/DeleteUdfMessage.java
@@ -21,12 +21,13 @@ package org.apache.asterix.app.message;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 
 public class DeleteUdfMessage extends AbstractUdfMessage {
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
 
-    public DeleteUdfMessage(String dataverseName, String libraryName, long reqId) {
+    public DeleteUdfMessage(DataverseName dataverseName, String libraryName, long reqId) {
         super(dataverseName, libraryName, reqId);
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/LoadUdfMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/LoadUdfMessage.java
index 600603b..6dffb28 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/LoadUdfMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/LoadUdfMessage.java
@@ -21,13 +21,14 @@ package org.apache.asterix.app.message;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.common.api.INcApplicationContext;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.util.file.FileUtil;
 
 public class LoadUdfMessage extends AbstractUdfMessage {
 
     private static final long serialVersionUID = 2L;
 
-    public LoadUdfMessage(String dataverseName, String libraryName, long reqId) {
+    public LoadUdfMessage(DataverseName dataverseName, String libraryName, long reqId) {
         super(dataverseName, libraryName, reqId);
     }
 
@@ -35,6 +36,6 @@ public class LoadUdfMessage extends AbstractUdfMessage {
     protected void handleAction(ILibraryManager mgr, boolean isMdNode, INcApplicationContext appCtx) throws Exception {
         ExternalLibraryUtils.setUpExternaLibrary(mgr, isMdNode,
                 FileUtil.joinPath(appCtx.getServiceContext().getServerCtx().getBaseDir().getAbsolutePath(),
-                        "applications", dataverseName + "." + libraryName));
+                        "applications", dataverseName.getCanonicalForm() + "." + libraryName)); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
     }
 }
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 8dd8d02..6351d0d 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
@@ -77,6 +77,7 @@ import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.common.exceptions.WarningUtil;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.common.utils.JobUtils.ProgressState;
 import org.apache.asterix.common.utils.StorageConstants;
@@ -297,13 +298,13 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                     sessionOutput.out().println(ApiServlet.HTML_STATEMENT_SEPARATOR);
                 }
                 validateOperation(appCtx, activeDataverse, stmt);
-                IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
-                rewriteStatement(stmt, stmtRewriter); // Rewrite the statement's AST.
                 MetadataProvider metadataProvider = new MetadataProvider(appCtx, activeDataverse);
                 metadataProvider.getConfig().putAll(config);
                 metadataProvider.setWriterFactory(writerFactory);
                 metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
                 metadataProvider.setOutputFile(outputFile);
+                IStatementRewriter stmtRewriter = rewriterFactory.createStatementRewriter();
+                rewriteStatement(stmt, stmtRewriter, metadataProvider); // Rewrite the statement's AST.
                 switch (stmt.getKind()) {
                     case SET:
                         handleSetStatement(stmt, config);
@@ -457,7 +458,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             throws Exception {
         DataverseDecl dvd = (DataverseDecl) stmt;
         SourceLocation sourceLoc = dvd.getSourceLocation();
-        String dvName = dvd.getDataverseName().getValue();
+        DataverseName dvName = dvd.getDataverseName();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
@@ -479,7 +480,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
             IRequestParameters requestParameters) throws Exception {
         CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
-        String dvName = stmtCreateDataverse.getDataverseName().getValue();
+        DataverseName dvName = stmtCreateDataverse.getDataverseName();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
@@ -496,7 +497,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     @SuppressWarnings("squid:S00112")
     protected boolean doCreateDataverseStatement(MetadataTransactionContext mdTxnCtx, MetadataProvider metadataProvider,
             CreateDataverseStatement stmtCreateDataverse) throws Exception {
-        String dvName = stmtCreateDataverse.getDataverseName().getValue();
+        DataverseName dvName = stmtCreateDataverse.getDataverseName();
         Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
         if (dv != null) {
             if (stmtCreateDataverse.getIfNotExists()) {
@@ -555,21 +556,17 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
         DatasetDecl dd = (DatasetDecl) stmt;
         SourceLocation sourceLoc = dd.getSourceLocation();
-        String dataverseName = getActiveDataverse(dd.getDataverse());
+        DataverseName dataverseName = getActiveDataverseName(dd.getDataverse());
         String datasetName = dd.getName().getValue();
-        String datasetFullyQualifiedName = dataverseName + "." + datasetName;
         DatasetType dsType = dd.getDatasetType();
-        String itemTypeDataverseName = getActiveDataverse(dd.getItemTypeDataverse());
+        DataverseName itemTypeDataverseName = getActiveDataverseName(dd.getItemTypeDataverse());
         String itemTypeName = dd.getItemTypeName().getValue();
-        String itemTypeFullyQualifiedName = itemTypeDataverseName + "." + itemTypeName;
-        String metaItemTypeDataverseName = null;
+        DataverseName metaItemTypeDataverseName = null;
         String metaItemTypeName = null;
-        String metaItemTypeFullyQualifiedName = null;
         Identifier metaItemTypeId = dd.getMetaItemTypeName();
         if (metaItemTypeId != null) {
             metaItemTypeName = metaItemTypeId.getValue();
-            metaItemTypeDataverseName = getActiveDataverse(dd.getMetaItemTypeDataverse());
-            metaItemTypeFullyQualifiedName = metaItemTypeDataverseName + "." + metaItemTypeName;
+            metaItemTypeDataverseName = getActiveDataverseName(dd.getMetaItemTypeDataverse());
         }
         Identifier ngNameId = dd.getNodegroupName();
         String nodegroupName = ngNameId == null ? null : ngNameId.getValue();
@@ -581,10 +578,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                itemTypeDataverseName, itemTypeFullyQualifiedName, metaItemTypeDataverseName,
-                metaItemTypeFullyQualifiedName, nodegroupName, compactionPolicy, datasetFullyQualifiedName,
-                defaultCompactionPolicy);
+        MetadataLockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
+                itemTypeDataverseName, itemTypeName, metaItemTypeDataverseName, metaItemTypeName, nodegroupName,
+                compactionPolicy, defaultCompactionPolicy);
         Dataset dataset = null;
         try {
             IDatasetDetails datasetDetails;
@@ -756,14 +752,14 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         for (IActiveEntityEventsListener listener : listeners) {
             if (listener.isEntityUsingDataset(dataset) && listener.isActive()) {
                 throw new CompilationException(ErrorCode.COMPILATION_CANT_DROP_ACTIVE_DATASET, sourceLoc,
-                        dataset.getFullyQualifiedName(), listener.getEntityId().toString());
+                        DatasetUtil.getFullyQualifiedDisplayName(dataset), listener.getEntityId().toString());
             }
         }
     }
 
     protected static String configureNodegroupForDataset(ICcApplicationContext appCtx, Map<String, String> hints,
-            String dataverseName, String datasetName, MetadataProvider metadataProvider, SourceLocation sourceLoc)
-            throws Exception {
+            DataverseName dataverseName, String datasetName, MetadataProvider metadataProvider,
+            SourceLocation sourceLoc) throws Exception {
         IClusterStateManager csm = appCtx.getClusterStateManager();
         Set<String> allNodes = csm.getParticipantNodes(true);
         Set<String> selectedNodes = new LinkedHashSet<>();
@@ -793,26 +789,22 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
         SourceLocation sourceLoc = stmtCreateIndex.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
         String datasetName = stmtCreateIndex.getDatasetName().getValue();
         String indexName = stmtCreateIndex.getIndexName().getValue();
         List<Integer> keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        String datasetFullyQualifiedName = dataverseName + "." + datasetName;
         boolean isSecondaryPrimary = stmtCreateIndex.getFieldExprs().isEmpty();
-        Dataset ds = null;
-        Index index = null;
-        MetadataLockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                datasetFullyQualifiedName);
+        MetadataLockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
-            ds = metadataProvider.findDataset(dataverseName, datasetName);
+            Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
                 throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
                         dataverseName);
             }
 
-            index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+            Index index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
                     datasetName, indexName);
             if (index != null) {
                 if (stmtCreateIndex.getIfNotExists()) {
@@ -1213,12 +1205,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleCreateTypeStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         TypeDecl stmtCreateType = (TypeDecl) stmt;
         SourceLocation sourceLoc = stmtCreateType.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
         String typeName = stmtCreateType.getIdent().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + typeName);
+        MetadataLockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
         try {
             Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
@@ -1254,7 +1245,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
         SourceLocation sourceLoc = stmtDelete.getSourceLocation();
-        String dataverseName = stmtDelete.getDataverseName().getValue();
+        DataverseName dataverseName = stmtDelete.getDataverseName();
         if (dataverseName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME + " dataverse can't be dropped");
@@ -1270,7 +1261,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
 
     protected boolean doDropDataverse(DataverseDropStatement stmtDelete, SourceLocation sourceLoc,
             MetadataProvider metadataProvider, IHyracksClientConnection hcc) throws Exception {
-        String dataverseName = stmtDelete.getDataverseName().getValue();
+        DataverseName dataverseName = stmtDelete.getDataverseName();
         ProgressState progress = ProgressState.NO_PROGRESS;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
@@ -1304,7 +1295,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             for (IActiveEntityEventsListener listener : activeListeners) {
                 EntityId activeEntityId = listener.getEntityId();
                 if (activeEntityId.getExtensionName().equals(Feed.EXTENSION_NAME)
-                        && activeEntityId.getDataverse().equals(dataverseName)) {
+                        && activeEntityId.getDataverseName().equals(dataverseName)) {
                     if (listener.getState() != ActivityState.STOPPED) {
                         ((ActiveEntityEventsListener) listener).stop(metadataProvider);
                     }
@@ -1427,10 +1418,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
         DropDatasetStatement stmtDelete = (DropDatasetStatement) stmt;
         SourceLocation sourceLoc = stmtDelete.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
         String datasetName = stmtDelete.getDatasetName().getValue();
-        MetadataLockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
+        MetadataLockUtil.dropDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             doDropDataset(dataverseName, datasetName, metadataProvider, stmtDelete.getIfExists(), hcc, true, sourceLoc);
         } finally {
@@ -1438,7 +1428,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
-    public void doDropDataset(String dataverseName, String datasetName, MetadataProvider metadataProvider,
+    public void doDropDataset(DataverseName dataverseName, String datasetName, MetadataProvider metadataProvider,
             boolean ifExists, IHyracksClientConnection hcc, boolean dropCorrespondingNodeGroup,
             SourceLocation sourceLoc) throws Exception {
         MutableObject<ProgressState> progress = new MutableObject<>(ProgressState.NO_PROGRESS);
@@ -1505,15 +1495,14 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
         SourceLocation sourceLoc = stmtIndexDrop.getSourceLocation();
         String datasetName = stmtIndexDrop.getDatasetName().getValue();
-        String dataverseName = getActiveDataverse(stmtIndexDrop.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
         String indexName = stmtIndexDrop.getIndexName().getValue();
         ProgressState progress = ProgressState.NO_PROGRESS;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        String dsFullyQualifiedName = dataverseName + "." + datasetName;
-        MetadataLockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, dsFullyQualifiedName);
+        MetadataLockUtil.dropIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         // For external index
         boolean dropFilesIndex = false;
         try {
@@ -1682,13 +1671,12 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleTypeDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
         SourceLocation sourceLoc = stmtTypeDrop.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtTypeDrop.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
         String typeName = stmtTypeDrop.getTypeName().getValue();
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + typeName);
+        MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName, typeName);
         try {
             Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
             if (dt == null) {
@@ -1737,17 +1725,17 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
         SourceLocation sourceLoc = cfs.getSourceLocation();
         FunctionSignature signature = cfs.getFunctionSignature();
-        String dataverse = getActiveDataverseName(signature.getNamespace());
-        signature.setNamespace(dataverse);
+        DataverseName dataverseName = getActiveDataverseName(signature.getDataverseName());
+        signature.setDataverseName(dataverseName);
 
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.functionStatementBegin(lockManager, metadataProvider.getLocks(), dataverse,
-                dataverse + "." + signature.getName());
+        MetadataLockUtil.createFunctionBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+                signature.getName());
         try {
-            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
             if (dv == null) {
-                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverse);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
             }
 
             //Check whether the function is use-able
@@ -1763,8 +1751,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             apiFramework.reWriteQuery(declaredFunctions, metadataProvider, wrappedQuery, sessionOutput, false,
                     paramVars, warningCollector);
 
-            List<List<List<String>>> dependencies = FunctionUtil.getFunctionDependencies(
-                    rewriterFactory.createQueryRewriter(), cfs.getFunctionBodyExpression(), metadataProvider);
+            List<List<org.apache.hyracks.algebricks.common.utils.Triple<DataverseName, String, String>>> dependencies =
+                    FunctionUtil.getFunctionDependencies(rewriterFactory.createQueryRewriter(),
+                            cfs.getFunctionBodyExpression(), metadataProvider);
 
             Function function = new Function(signature, cfs.getParamList(), Function.RETURNTYPE_VOID,
                     cfs.getFunctionBody(), getFunctionLanguage(), FunctionKind.SCALAR.toString(), dependencies);
@@ -1792,7 +1781,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     }
 
     protected boolean isFunctionUsed(MetadataTransactionContext ctx, FunctionSignature signature,
-            String currentDataverse) throws AlgebricksException {
+            DataverseName currentDataverse) throws AlgebricksException {
         List<Dataverse> allDataverses = MetadataManager.INSTANCE.getDataverses(ctx);
         for (Dataverse dataverse : allDataverses) {
             if (dataverse.getDataverseName().equals(currentDataverse)) {
@@ -1816,11 +1805,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
         SourceLocation sourceLoc = stmtDropFunction.getSourceLocation();
         FunctionSignature signature = stmtDropFunction.getFunctionSignature();
-        signature.setNamespace(getActiveDataverseName(signature.getNamespace()));
+        signature.setDataverseName(getActiveDataverseName(signature.getDataverseName()));
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.functionStatementBegin(lockManager, metadataProvider.getLocks(), signature.getNamespace(),
-                signature.getNamespace() + "." + signature.getName());
+        MetadataLockUtil.dropFunctionBegin(lockManager, metadataProvider.getLocks(), signature.getDataverseName(),
+                signature.getName());
         try {
             Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
             // If function == null && stmtDropFunction.getIfExists() == true, commit txn directly.
@@ -1845,13 +1834,12 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleLoadStatement(MetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
             throws Exception {
         LoadStatement loadStmt = (LoadStatement) stmt;
-        String dataverseName = getActiveDataverse(loadStmt.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(loadStmt.getDataverseName());
         String datasetName = loadStmt.getDatasetName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
+        MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             CompiledLoadFromFileStatement cls =
                     new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
@@ -1880,12 +1868,12 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             ResultMetadata outMetadata, Stats stats, boolean compileOnly, IRequestParameters requestParameters,
             Map<String, IAObject> stmtParams, IStatementRewriter stmtRewriter) throws Exception {
         InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
-        String dataverseName = getActiveDataverse(stmtInsertUpsert.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtInsertUpsert.getDataverseName());
         final IMetadataLocker locker = new IMetadataLocker() {
             @Override
             public void lock() throws AlgebricksException {
-                MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(),
-                        dataverseName + "." + stmtInsertUpsert.getDatasetName());
+                MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+                        stmtInsertUpsert.getDatasetName());
             }
 
             @Override
@@ -1942,16 +1930,16 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc, boolean compileOnly, Map<String, IAObject> stmtParams,
             IStatementRewriter stmtRewriter) throws Exception {
         DeleteStatement stmtDelete = (DeleteStatement) stmt;
-        String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(),
-                dataverseName + "." + stmtDelete.getDatasetName());
+        MetadataLockUtil.insertDeleteUpsertBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+                stmtDelete.getDatasetName());
         try {
             metadataProvider.setWriteTransaction(true);
             CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
-                    stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
+                    stmtDelete.getDatasetName(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
                     stmtDelete.getQuery());
             clfrqs.setSourceLocation(stmt.getSourceLocation());
             JobSpecification jobSpec =
@@ -2005,8 +1993,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 metadataProvider, insertUpsert, sessionOutput, true, externalVars.keySet(), warningCollector);
 
         InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
-        String dataverseName = getActiveDataverse(rewrittenInsertUpsert.getDataverseName());
-        String datasetName = rewrittenInsertUpsert.getDatasetName().getValue();
+        DataverseName dataverseName = getActiveDataverseName(rewrittenInsertUpsert.getDataverseName());
+        String datasetName = rewrittenInsertUpsert.getDatasetName();
         CompiledInsertStatement clfrqs;
         switch (insertUpsert.getKind()) {
             case INSERT:
@@ -2035,12 +2023,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         CreateFeedStatement cfs = (CreateFeedStatement) stmt;
         SourceLocation sourceLoc = cfs.getSourceLocation();
-        String dataverseName = getActiveDataverse(cfs.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + feedName);
+        MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             Feed feed =
                     MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName, feedName);
@@ -2067,35 +2054,32 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
 
     protected void handleCreateFeedPolicyStatement(MetadataProvider metadataProvider, Statement stmt)
             throws AlgebricksException, HyracksDataException {
-        String dataverse;
-        String policy;
-        FeedPolicyEntity newPolicy = null;
+        FeedPolicyEntity newPolicy;
         MetadataTransactionContext mdTxnCtx = null;
         CreateFeedPolicyStatement cfps = (CreateFeedPolicyStatement) stmt;
         SourceLocation sourceLoc = cfps.getSourceLocation();
-        dataverse = getActiveDataverse(null);
-        policy = cfps.getPolicyName();
-        MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
-                dataverse + "." + policy);
+        DataverseName dataverseName = getActiveDataverseName(null);
+        String policyName = cfps.getPolicyName();
+        MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
         try {
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
-            FeedPolicyEntity feedPolicy =
-                    MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, policy);
+            FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE
+                    .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverseName, policyName);
             if (feedPolicy != null) {
                 if (cfps.getIfNotExists()) {
                     MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
                     return;
                 } else {
                     throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
-                            "A policy with this name " + policy + " already exists.");
+                            "A policy with this name " + policyName + " already exists.");
                 }
             }
             boolean extendingExisting = cfps.getSourcePolicyName() != null;
             String description = cfps.getDescription() == null ? "" : cfps.getDescription();
             if (extendingExisting) {
-                FeedPolicyEntity sourceFeedPolicy = MetadataManager.INSTANCE
-                        .getFeedPolicy(metadataProvider.getMetadataTxnContext(), dataverse, cfps.getSourcePolicyName());
+                FeedPolicyEntity sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(
+                        metadataProvider.getMetadataTxnContext(), dataverseName, cfps.getSourcePolicyName());
                 if (sourceFeedPolicy == null) {
                     sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
                             MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
@@ -2106,7 +2090,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 }
                 Map<String, String> policyProperties = sourceFeedPolicy.getProperties();
                 policyProperties.putAll(cfps.getProperties());
-                newPolicy = new FeedPolicyEntity(dataverse, policy, description, policyProperties);
+                newPolicy = new FeedPolicyEntity(dataverseName, policyName, description, policyProperties);
             } else {
                 Properties prop = new Properties();
                 try {
@@ -2118,7 +2102,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 }
                 Map<String, String> policyProperties = new HashMap<>();
                 prop.forEach((key, value) -> policyProperties.put((String) key, (String) value));
-                newPolicy = new FeedPolicyEntity(dataverse, policy, description, policyProperties);
+                newPolicy = new FeedPolicyEntity(dataverseName, policyName, description, policyProperties);
             }
             MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, newPolicy);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2134,12 +2118,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc) throws Exception {
         FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
         SourceLocation sourceLoc = stmtFeedDrop.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtFeedDrop.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
         String feedName = stmtFeedDrop.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + feedName);
+        MetadataLockUtil.dropFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverseName, feedName);
             if (feed == null) {
@@ -2175,7 +2158,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             listener.unregister();
         }
         JobSpecification spec = FeedOperations.buildRemoveFeedStorageJob(metadataProvider,
-                MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName()));
+                MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverseName(), feedId.getEntityName()));
         runJob(hcc, spec);
         MetadataManager.INSTANCE.dropFeed(mdTxnCtx, feed.getDataverseName(), feed.getFeedName());
         if (LOGGER.isInfoEnabled()) {
@@ -2188,10 +2171,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
         SourceLocation sourceLoc = stmtFeedPolicyDrop.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtFeedPolicyDrop.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtFeedPolicyDrop.getDataverseName());
         String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
-        MetadataLockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + policyName);
+        MetadataLockUtil.dropFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverseName, policyName);
         try {
             FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName, policyName);
             if (feedPolicy == null) {
@@ -2216,12 +2198,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc) throws Exception {
         StartFeedStatement sfs = (StartFeedStatement) stmt;
         SourceLocation sourceLoc = sfs.getSourceLocation();
-        String dataverseName = getActiveDataverse(sfs.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(sfs.getDataverseName());
         String feedName = sfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean committed = false;
-        MetadataLockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + feedName);
+        MetadataLockUtil.startFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
             // Runtime handler
@@ -2236,8 +2217,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             }
             for (FeedConnection feedConnection : feedConnections) {
                 // what if the dataset is in a different dataverse
-                String fqName = feedConnection.getDataverseName() + "." + feedConnection.getDatasetName();
-                lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), fqName);
+                lockManager.acquireDatasetReadLock(metadataProvider.getLocks(), feedConnection.getDataverseName(),
+                        feedConnection.getDatasetName());
             }
             ActiveNotificationHandler activeEventHandler =
                     (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -2270,7 +2251,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     private void handleStopFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         StopFeedStatement sfst = (StopFeedStatement) stmt;
         SourceLocation sourceLoc = sfst.getSourceLocation();
-        String dataverseName = getActiveDataverse(sfst.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(sfst.getDataverseName());
         String feedName = sfst.getFeedName().getValue();
         EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, feedName);
         ActiveNotificationHandler activeEventHandler =
@@ -2281,8 +2262,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                     "Feed " + feedName + " is not started.");
         }
-        MetadataLockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), entityId.getDataverse(),
-                entityId.getEntityName());
+        MetadataLockUtil.stopFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, feedName);
         try {
             listener.stop(metadataProvider);
         } finally {
@@ -2294,7 +2274,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         FeedConnection fc;
         ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
         SourceLocation sourceLoc = cfs.getSourceLocation();
-        String dataverseName = getActiveDataverse(cfs.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
         String feedName = cfs.getFeedName();
         String datasetName = cfs.getDatasetName().getValue();
         String policyName = cfs.getPolicy();
@@ -2306,8 +2286,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         ActiveNotificationHandler activeEventHandler =
                 (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         // Transaction handling
-        MetadataLockUtil.connectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName, dataverseName + "." + feedName);
+        MetadataLockUtil.connectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
+                feedName);
         try {
             // validation
             Dataset dataset = FeedMetadataUtil.validateIfDatasetExists(metadataProvider, dataverseName, datasetName);
@@ -2352,13 +2332,13 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     protected void handleDisconnectFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
         DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
         SourceLocation sourceLoc = cfs.getSourceLocation();
-        String dataverseName = getActiveDataverse(cfs.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(cfs.getDataverseName());
         String datasetName = cfs.getDatasetName().getValue();
         String feedName = cfs.getFeedName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        MetadataLockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName, dataverseName + "." + cfs.getFeedName());
+        MetadataLockUtil.disconnectFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
+                feedName);
         try {
             ActiveNotificationHandler activeEventHandler =
                     (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -2399,14 +2379,13 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc) throws Exception {
         CompactStatement compactStatement = (CompactStatement) stmt;
         SourceLocation sourceLoc = compactStatement.getSourceLocation();
-        String dataverseName = getActiveDataverse(compactStatement.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
         String datasetName = compactStatement.getDatasetName().getValue();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         List<JobSpecification> jobsToExecute = new ArrayList<>();
-        MetadataLockUtil.compactBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
+        MetadataLockUtil.compactBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
             if (ds == null) {
@@ -2703,7 +2682,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             IHyracksClientConnection hcc) throws Exception {
         RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
         SourceLocation sourceLoc = stmtRefresh.getSourceLocation();
-        String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
+        DataverseName dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
         String datasetName = stmtRefresh.getDatasetName().getValue();
         TransactionState transactionState = TransactionState.COMMIT;
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2719,8 +2698,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         Dataset transactionDataset = null;
         boolean lockAquired = false;
         boolean success = false;
-        MetadataLockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
-                dataverseName + "." + datasetName);
+        MetadataLockUtil.refreshDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
         try {
             ds = metadataProvider.findDataset(dataverseName, datasetName);
             // Dataset exists ?
@@ -2925,8 +2903,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     }
 
     @Override
-    public String getActiveDataverseName(String dataverse) {
-        return (dataverse != null && !dataverse.isEmpty()) ? dataverse : activeDataverse.getDataverseName();
+    public DataverseName getActiveDataverseName(DataverseName dataverseName) {
+        return dataverseName != null ? dataverseName : activeDataverse.getDataverseName();
     }
 
     @Override
@@ -2939,10 +2917,6 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         return responsePrinter;
     }
 
-    public String getActiveDataverse(Identifier dataverse) {
-        return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
-    }
-
     @Override
     public void getWarnings(Collection<? super Warning> outWarnings, long maxWarnings) {
         warningCollector.getWarnings(outWarnings, maxWarnings);
@@ -2974,8 +2948,9 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
-    protected void rewriteStatement(Statement stmt, IStatementRewriter rewriter) throws CompilationException {
-        rewriter.rewrite(stmt);
+    protected void rewriteStatement(Statement stmt, IStatementRewriter rewriter, MetadataProvider metadataProvider)
+            throws CompilationException {
+        rewriter.rewrite(stmt, metadataProvider);
     }
 
     private void ensureNonPrimaryIndexDrop(Index index, SourceLocation sourceLoc) throws AlgebricksException {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index b74f4c6..447fdff 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -39,6 +39,7 @@ import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
@@ -186,6 +187,10 @@ public class FeedOperations {
                 argExprs.add(new LiteralExpr(new StringLiteral((String) arg)));
             } else if (arg instanceof Expression) {
                 argExprs.add((Expression) arg);
+            } else if (arg instanceof DataverseName) {
+                argExprs.add(new LiteralExpr(new StringLiteral(((DataverseName) arg).getCanonicalForm())));
+            } else {
+                throw new IllegalArgumentException();
             }
         }
         return argExprs;
@@ -240,12 +245,12 @@ public class FeedOperations {
         Query feedConnQuery = makeConnectionQuery(feedConn);
         CompiledStatements.ICompiledDmlStatement clfrqs;
         if (insertFeed) {
-            InsertStatement stmtUpsert = new InsertStatement(new Identifier(feedConn.getDataverseName()),
+            InsertStatement stmtUpsert = new InsertStatement(feedConn.getDataverseName(),
                     new Identifier(feedConn.getDatasetName()), feedConnQuery, -1, null, null);
             clfrqs = new CompiledStatements.CompiledInsertStatement(feedConn.getDataverseName(),
                     feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
         } else {
-            UpsertStatement stmtUpsert = new UpsertStatement(new Identifier(feedConn.getDataverseName()),
+            UpsertStatement stmtUpsert = new UpsertStatement(feedConn.getDataverseName(),
                     new Identifier(feedConn.getDatasetName()), feedConnQuery, -1, null, null);
             clfrqs = new CompiledStatements.CompiledUpsertStatement(feedConn.getDataverseName(),
                     feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
index c37d8cc..ff99327 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FlushDatasetUtil.java
@@ -20,6 +20,7 @@
 package org.apache.asterix.utils;
 
 import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -44,7 +45,7 @@ public class FlushDatasetUtil {
     }
 
     public static void flushDataset(IHyracksClientConnection hcc, MetadataProvider metadataProvider,
-            String dataverseName, String datasetName) throws Exception {
+            DataverseName dataverseName, String datasetName) throws Exception {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         flushDataset(hcc, metadataProvider, dataset);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 483987c..81f749e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -34,6 +34,7 @@ import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.common.utils.JobUtils;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -90,7 +91,7 @@ public class RebalanceUtil {
      *            the reusable hyracks connection.
      * @throws Exception
      */
-    public static void rebalance(String dataverseName, String datasetName, Set<String> targetNcNames,
+    public static void rebalance(DataverseName dataverseName, String datasetName, Set<String> targetNcNames,
             MetadataProvider metadataProvider, IHyracksClientConnection hcc,
             IDatasetRebalanceCallback datasetRebalanceCallback) throws Exception {
         Dataset sourceDataset;
@@ -241,7 +242,8 @@ public class RebalanceUtil {
         ActiveNotificationHandler activeNotificationHandler =
                 (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
         IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
-        lockManager.upgradeDatasetLockToWrite(metadataProvider.getLocks(), DatasetUtil.getFullyQualifiedName(source));
+        lockManager.upgradeDatasetLockToWrite(metadataProvider.getLocks(), source.getDataverseName(),
+                source.getDatasetName());
         LOGGER.info("Updating dataset {} node group from {} to {}", source.getDatasetName(), source.getNodeGroupName(),
                 target.getNodeGroupName());
         try {
@@ -256,8 +258,8 @@ public class RebalanceUtil {
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             LOGGER.info("dataset {} node group updated to {}", target.getDatasetName(), target.getNodeGroupName());
         } finally {
-            lockManager.downgradeDatasetLockToExclusiveModify(metadataProvider.getLocks(),
-                    DatasetUtil.getFullyQualifiedName(target));
+            lockManager.downgradeDatasetLockToExclusiveModify(metadataProvider.getLocks(), target.getDataverseName(),
+                    target.getDatasetName());
         }
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
index 06d92b7..5c6f0b6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorApiServletTest.java
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -31,10 +32,12 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
@@ -97,7 +100,7 @@ public class ConnectorApiServletTest {
         // Sets up mock returns.
         when(mockRequest.getHttpRequest()).thenReturn(mockHttpRequest);
         when(mockHttpRequest.method()).thenReturn(HttpMethod.GET);
-        when(mockRequest.getParameter("dataverseName")).thenReturn("Metadata");
+        when(mockRequest.getParameterValues("dataverseName")).thenReturn(Collections.singletonList("Metadata"));
         when(mockRequest.getParameter("datasetName")).thenReturn("Dataset");
         when(mockResponse.writer()).thenReturn(outputWriter);
         when(mockHcc.getNodeControllerInfos()).thenReturn(nodeMap);
@@ -117,7 +120,8 @@ public class ConnectorApiServletTest {
         String primaryKey = actualResponse.get("keys").asText();
         Assert.assertEquals("DataverseName,DatasetName", primaryKey);
         ARecordType recordType = (ARecordType) JSONDeserializerForTypes.convertFromJSON(actualResponse.get("type"));
-        Assert.assertEquals(getMetadataRecordType("Metadata", "Dataset"), recordType);
+        Assert.assertEquals(getMetadataRecordType(MetadataConstants.METADATA_DATAVERSE_NAME,
+                MetadataConstants.DATASET_DATASET_NAME), recordType);
 
         // Checks the correctness of results.
         ArrayNode splits = (ArrayNode) actualResponse.get("splits");
@@ -174,7 +178,7 @@ public class ConnectorApiServletTest {
         Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
     }
 
-    private ARecordType getMetadataRecordType(String dataverseName, String datasetName) throws Exception {
+    private ARecordType getMetadataRecordType(DataverseName dataverseName, String datasetName) throws Exception {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         // Retrieves file splits of the dataset.
         MetadataProvider metadataProvider = new MetadataProvider(
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
index d453824..b184053 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
@@ -30,6 +30,7 @@ import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -153,7 +154,7 @@ public class TestDataUtil {
      * @param targetNodes
      * @throws Exception
      */
-    public static void rebalanceDataset(AsterixHyracksIntegrationUtil integrationUtil, String dataverseName,
+    public static void rebalanceDataset(AsterixHyracksIntegrationUtil integrationUtil, DataverseName dataverseName,
             String datasetName, String[] targetNodes) throws Exception {
         ICcApplicationContext ccAppCtx =
                 (ICcApplicationContext) integrationUtil.getClusterControllerService().getApplicationContext();
@@ -164,8 +165,8 @@ public class TestDataUtil {
             activeNotificationHandler.suspend(metadataProvider);
             try {
                 IMetadataLockManager lockManager = ccAppCtx.getMetadataLockManager();
-                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
-                        dataverseName + '.' + datasetName);
+                lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataverseName,
+                        datasetName);
                 RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
                         metadataProvider, ccAppCtx.getHcc(), NoOpDatasetRebalanceCallback.INSTANCE);
             } finally {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index 64520a4..bc38254 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -40,8 +40,10 @@ import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.ActiveProperties;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Feed;
@@ -76,7 +78,7 @@ public class ActiveEventsListenerTest {
     static TestUserActor[] users;
     static String[] nodes = { "node1", "node2" };
     static ActiveNotificationHandler handler;
-    static String dataverseName = "Default";
+    static DataverseName dataverseName = MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME;
     static String entityName = "entityName";
     static EntityId entityId = new EntityId(Feed.EXTENSION_NAME, dataverseName, entityName);
     static Dataset firstDataset;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index 80dde8a..bc40304 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -38,6 +38,7 @@ import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -71,7 +72,8 @@ public class ActiveStatsTest {
     @Test
     public void refreshStatsTest() throws Exception {
         // Entities to be used
-        EntityId entityId = new EntityId("MockExtension", "MockDataverse", "MockEntity");
+        EntityId entityId =
+                new EntityId("MockExtension", DataverseName.createSinglePartName("MockDataverse"), "MockEntity");
         ActiveRuntimeId activeRuntimeId =
                 new ActiveRuntimeId(entityId, FeedIntakeOperatorNodePushable.class.getSimpleName(), 0);
         List<Dataset> datasetList = new ArrayList<>();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
index 2143404..f87bfe2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestEventsListener.java
@@ -31,6 +31,7 @@ import org.apache.asterix.app.active.ActiveEntityEventsListener;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.translator.IStatementExecutor;
@@ -112,8 +113,8 @@ public class TestEventsListener extends ActiveEntityEventsListener {
     protected JobId compileAndStartJob(MetadataProvider metadataProvider) throws HyracksDataException {
         step(onStart);
         try {
-            metadataProvider.getApplicationContext().getMetadataLockManager()
-                    .acquireDatasetReadLock(metadataProvider.getLocks(), "Default.type");
+            metadataProvider.getApplicationContext().getMetadataLockManager().acquireDatasetReadLock(
+                    metadataProvider.getLocks(), MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, "type");
         } catch (AlgebricksException e) {
             throw HyracksDataException.create(e);
         }
@@ -207,8 +208,8 @@ public class TestEventsListener extends ActiveEntityEventsListener {
         try {
             IMetadataLockManager lockManager = metadataProvider.getApplicationContext().getMetadataLockManager();
             LockList locks = metadataProvider.getLocks();
-            lockManager.acquireDataverseReadLock(locks, entityId.getDataverse());
-            lockManager.acquireActiveEntityWriteLock(locks, entityId.getDataverse() + '.' + entityId.getEntityName());
+            lockManager.acquireDataverseReadLock(locks, entityId.getDataverseName());
+            lockManager.acquireActiveEntityWriteLock(locks, entityId.getDataverseName(), entityId.getEntityName());
             // persist entity
         } catch (Throwable th) {
             // This failure puts the system in a bad state.
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
index 5f715af..36f704c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
@@ -25,6 +25,7 @@ import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.api.IActiveEntityController;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -47,14 +48,14 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName dataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 try {
-                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName, entityName);
                     List<Dataset> datasets = actionListener.getDatasets();
                     for (Dataset dataset : datasets) {
-                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(), dataverseName,
-                                DatasetUtil.getFullyQualifiedName(dataset));
+                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(),
+                                dataset.getDataverseName(), dataset.getDatasetName());
                     }
                     actionListener.start(mdProvider);
                 } finally {
@@ -70,14 +71,14 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName dataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 try {
-                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName, entityName);
                     List<Dataset> datasets = actionListener.getDatasets();
                     for (Dataset dataset : datasets) {
-                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(), dataverseName,
-                                DatasetUtil.getFullyQualifiedName(dataset));
+                        MetadataLockUtil.modifyDatasetBegin(lockManager, mdProvider.getLocks(),
+                                dataset.getDataverseName(), dataset.getDatasetName());
                     }
                     actionListener.stop(mdProvider);
                 } finally {
@@ -93,14 +94,14 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName dataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 List<Dataset> datasets = actionListener.getDatasets();
                 try {
-                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName, entityName);
                     for (Dataset dataset : datasets) {
                         lockManager.acquireDatasetExclusiveModificationLock(mdProvider.getLocks(),
-                                DatasetUtil.getFullyQualifiedName(dataset));
+                                dataset.getDataverseName(), dataset.getDatasetName());
                     }
                     actionListener.suspend(mdProvider);
                 } catch (Exception e) {
@@ -118,16 +119,16 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName dataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 try {
-                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName + '.' + entityName);
+                    lockManager.acquireActiveEntityWriteLock(mdProvider.getLocks(), dataverseName, entityName);
                     List<Dataset> datasets = actionListener.getDatasets();
                     for (Dataset dataset : datasets) {
-                        lockManager.upgradeDatasetLockToWrite(mdProvider.getLocks(),
-                                DatasetUtil.getFullyQualifiedName(dataset));
+                        lockManager.upgradeDatasetLockToWrite(mdProvider.getLocks(), dataset.getDataverseName(),
+                                dataset.getDatasetName());
                         lockManager.downgradeDatasetLockToExclusiveModify(mdProvider.getLocks(),
-                                DatasetUtil.getFullyQualifiedName(dataset));
+                                dataset.getDataverseName(), dataset.getDatasetName());
                     }
                     actionListener.resume(mdProvider);
                 } finally {
@@ -143,13 +144,12 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String entityDataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName entityDataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 try {
-                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(),
-                            entityDataverseName + '.' + entityName);
-                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(),
-                            DatasetUtil.getFullyQualifiedName(dataset));
+                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(), entityDataverseName, entityName);
+                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(), dataset.getDataverseName(),
+                            dataset.getDatasetName());
                     List<Dataset> datasets = clusterController.getAllDatasets();
                     if (datasets.contains(dataset)) {
                         throw new HyracksDataException("Dataset " + dataset + " already exists");
@@ -169,13 +169,12 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String entityDataverseName = actionListener.getEntityId().getDataverse();
+                DataverseName entityDataverseName = actionListener.getEntityId().getDataverseName();
                 String entityName = actionListener.getEntityId().getEntityName();
                 try {
-                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(),
-                            entityDataverseName + '.' + entityName); // we have to first read lock all active entities before deleting a dataset
-                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(),
-                            DatasetUtil.getFullyQualifiedName(dataset));
+                    lockManager.acquireActiveEntityReadLock(mdProvider.getLocks(), entityDataverseName, entityName); // we have to first read lock all active entities before deleting a dataset
+                    lockManager.acquireDatasetWriteLock(mdProvider.getLocks(), dataset.getDataverseName(),
+                            dataset.getDatasetName());
                     List<Dataset> datasets = clusterController.getAllDatasets();
                     if (!datasets.contains(dataset)) {
                         throw new HyracksDataException("Dataset " + dataset + " does not exist");
@@ -195,15 +194,14 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = dataset.getDataverseName();
-                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
-                String indexFullyQualifiedName = datasetFullyQualifiedName + ".index";
+                DataverseName dataverseName = dataset.getDataverseName();
+                String datasetName = dataset.getDatasetName();
                 try {
-                    MetadataLockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName,
-                            datasetFullyQualifiedName);
+                    MetadataLockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName);
                     if (actionListener.isActive()) {
                         throw new RuntimeDataException(ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
-                                indexFullyQualifiedName, actionListener.getEntityId(), actionListener.getState());
+                                DatasetUtil.getFullyQualifiedDisplayName(dataverseName, datasetName) + ".index",
+                                actionListener.getEntityId(), actionListener.getState());
                     }
                 } finally {
                     mdProvider.getLocks().reset();
@@ -218,16 +216,15 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = dataset.getDataverseName();
-                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
+                DataverseName dataverseName = dataset.getDataverseName();
+                String datasetName = dataset.getDatasetName();
                 try {
-                    MetadataLockUtil.dropIndexBegin(lockManager, mdProvider.getLocks(), dataverseName,
-                            datasetFullyQualifiedName);
+                    MetadataLockUtil.dropIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName);
                     if (actionListener.isActive()) {
                         throw new RuntimeDataException(
                                 ErrorCode.CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
-                                datasetFullyQualifiedName + ".index", actionListener.getEntityId(),
-                                actionListener.getState());
+                                DatasetUtil.getFullyQualifiedDisplayName(dataverseName, datasetName) + ".index",
+                                actionListener.getEntityId(), actionListener.getState());
                     }
                 } finally {
                     mdProvider.getLocks().reset();
@@ -242,11 +239,11 @@ public class TestUserActor extends Actor {
         Action action = new Action() {
             @Override
             protected void doExecute(MetadataProvider mdProvider) throws Exception {
-                String dataverseName = dataset.getDataverseName();
-                String datasetFullyQualifiedName = dataverseName + '.' + dataset.getDatasetName();
+                DataverseName dataverseName = dataset.getDataverseName();
+                String datasetName = dataset.getDatasetName();
                 try {
                     lockManager.acquireDataverseReadLock(mdProvider.getLocks(), dataverseName);
-                    lockManager.acquireDatasetReadLock(mdProvider.getLocks(), datasetFullyQualifiedName);
+                    lockManager.acquireDatasetReadLock(mdProvider.getLocks(), dataverseName, datasetName);
                     if (!semaphore.tryAcquire()) {
                         semaphore.acquire();
                     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 7a63685..7c1b1f3 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -70,8 +70,11 @@ import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
 import org.apache.asterix.app.external.IExternalUDFLibrarian;
 import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.Servlets;
 import org.apache.asterix.lang.sqlpp.util.SqlppStatementUtil;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.asterix.runtime.evaluators.common.NumberUtils;
 import org.apache.asterix.test.server.ITestServer;
 import org.apache.asterix.test.server.TestServerProvider;
@@ -1997,7 +2000,7 @@ public class TestExecutor {
 
     public void cleanup(String testCase, List<String> badtestcases) throws Exception {
         try {
-            ArrayList<String> toBeDropped = new ArrayList<>();
+            List<DataverseName> toBeDropped = new ArrayList<>();
             InputStream resultStream = executeQueryService(
                     "select dv.DataverseName from Metadata.`Dataverse` as dv order by dv.DataverseName;",
                     getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.CLEAN_JSON);
@@ -2005,9 +2008,10 @@ public class TestExecutor {
             for (int i = 0; i < result.size(); i++) {
                 JsonNode json = result.get(i);
                 if (json != null) {
-                    String dvName = json.get("DataverseName").asText();
-                    if (!dvName.equals("Metadata") && !dvName.equals("Default")) {
-                        toBeDropped.add(SqlppStatementUtil.enclose(dvName));
+                    DataverseName dvName = DataverseName.createFromCanonicalForm(json.get("DataverseName").asText());
+                    if (!dvName.equals(MetadataConstants.METADATA_DATAVERSE_NAME)
+                            && !dvName.equals(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME)) {
+                        toBeDropped.add(dvName);
                     }
                 }
             }
@@ -2015,10 +2019,10 @@ public class TestExecutor {
                 badtestcases.add(testCase);
                 LOGGER.info("Last test left some garbage. Dropping dataverses: " + StringUtils.join(toBeDropped, ','));
                 StringBuilder dropStatement = new StringBuilder();
-                for (String dv : toBeDropped) {
+                for (DataverseName dv : toBeDropped) {
                     dropStatement.setLength(0);
                     dropStatement.append("drop dataverse ");
-                    dropStatement.append(dv);
+                    SqlppStatementUtil.encloseDataverseName(dropStatement, dv);
                     dropStatement.append(";\n");
                     resultStream = executeQueryService(dropStatement.toString(), getEndpoint(Servlets.QUERY_SERVICE),
                             OutputFormat.CLEAN_JSON, UTF_8);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
index 54f44b1..b43c445 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -36,6 +36,7 @@ import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.storage.IIndexCheckpointManager;
 import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
 import org.apache.asterix.common.storage.IndexCheckpoint;
@@ -107,7 +108,7 @@ public class CheckpointInSecondaryIndexTest {
     private static final List<Integer> KEY_INDICATORS_LIST = Collections.singletonList(Index.RECORD_INDICATOR);
     private static final int RECORDS_PER_COMPONENT = 500;
     private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
+    private static final DataverseName DATAVERSE_NAME = DataverseName.createSinglePartName("TestDV");
     private static final String DATASET_NAME = "TestDS";
     private static final String INDEX_NAME = "TestIdx";
     private static final String DATA_TYPE_NAME = "DUMMY";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index a1b251f..1e1df54 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -35,6 +35,7 @@ import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TransactionOptions;
@@ -98,7 +99,7 @@ public class MultiPartitionLSMIndexTest {
     private static final int TOTAL_NUM_OF_RECORDS = 5000;
     private static final int RECORDS_PER_COMPONENT = 500;
     private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
+    private static final DataverseName DATAVERSE_NAME = DataverseName.createSinglePartName("TestDV");
     private static final String DATASET_NAME = "TestDS";
     private static final String INDEX_NAME = "TestIdx";
     private static final String DATA_TYPE_NAME = "DUMMY";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index 37b40bb..28da85c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -31,6 +31,7 @@ import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
 import org.apache.asterix.common.api.IDatasetLifecycleManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionManager;
 import org.apache.asterix.common.transactions.TransactionOptions;
@@ -87,7 +88,7 @@ public class SearchCursorComponentSwitchTest {
     private static final int TOTAL_NUM_OF_RECORDS = 2000;
     private static final int RECORDS_PER_COMPONENT = 1000;
     private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
+    private static final DataverseName DATAVERSE_NAME = DataverseName.createSinglePartName("TestDV");
     private static final String DATASET_NAME = "TestDS";
     private static final String DATA_TYPE_NAME = "DUMMY";
     private static final String NODE_GROUP_NAME = "DEFAULT";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
index 99aee07..e41c193 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
@@ -42,6 +42,7 @@ import org.apache.asterix.common.context.DatasetInfo;
 import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -84,7 +85,7 @@ public class StorageTestUtils {
     public static final int TOTAL_NUM_OF_RECORDS = 10000;
     public static final int RECORDS_PER_COMPONENT = 1000;
     public static final int DATASET_ID = 101;
-    public static final String DATAVERSE_NAME = "TestDV";
+    public static final DataverseName DATAVERSE_NAME = DataverseName.createSinglePartName("TestDV");
     public static final String DATASET_NAME = "TestDS";
     public static final String DATA_TYPE_NAME = "DUMMY";
     public static final String NODE_GROUP_NAME = "DEFAULT";
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index bcf68b5..a6a53cb 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -21,6 +21,7 @@ package org.apache.asterix.test.dataflow;
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.transactions.TxnId;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -45,10 +46,10 @@ public class TestDataset extends Dataset {
 
     private static final long serialVersionUID = 1L;
 
-    public TestDataset(String dataverseName, String datasetName, String recordTypeDataverseName, String recordTypeName,
-            String nodeGroupName, String compactionPolicy, Map<String, String> compactionPolicyProperties,
-            IDatasetDetails datasetDetails, Map<String, String> hints, DatasetType datasetType, int datasetId,
-            int pendingOp) {
+    public TestDataset(DataverseName dataverseName, String datasetName, DataverseName recordTypeDataverseName,
+            String recordTypeName, String nodeGroupName, String compactionPolicy,
+            Map<String, String> compactionPolicyProperties, IDatasetDetails datasetDetails, Map<String, String> hints,
+            DatasetType datasetType, int datasetId, int pendingOp) {
         super(dataverseName, datasetName, recordTypeDataverseName, recordTypeName, nodeGroupName, compactionPolicy,
                 compactionPolicyProperties, datasetDetails, hints, datasetType, datasetId, pendingOp);
     }
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 970cbc0..4826a99 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
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.test.sqlpp;
 
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -32,6 +33,7 @@ import java.util.Map;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
@@ -46,6 +48,7 @@ import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
 import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.lang.sqlpp.util.SqlppAstPrintUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppRewriteUtil;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.test.common.ComparisonException;
@@ -127,7 +130,7 @@ public class ParserTestExecutor extends TestExecutor {
         try {
             List<Statement> statements = parser.parse();
             List<FunctionDecl> functions = getDeclaredFunctions(statements);
-            String dvName = getDefaultDataverse(statements);
+            DataverseName dvName = getDefaultDataverse(statements);
             MetadataProvider metadataProvider = mock(MetadataProvider.class);
 
             @SuppressWarnings("unchecked")
@@ -135,13 +138,13 @@ public class ParserTestExecutor extends TestExecutor {
             when(metadataProvider.getDefaultDataverseName()).thenReturn(dvName);
             when(metadataProvider.getConfig()).thenReturn(config);
             when(config.get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS)).thenReturn("true");
-            when(metadataProvider.findDataset(anyString(), anyString())).thenAnswer(new Answer<Dataset>() {
+            when(metadataProvider.findDataset(any(DataverseName.class), anyString())).thenAnswer(new Answer<Dataset>() {
                 @Override
                 public Dataset answer(InvocationOnMock invocation) {
                     Object[] args = invocation.getArguments();
                     final Dataset mockDataset = mock(Dataset.class);
-                    String fullyQualifiedName = args[0] != null ? args[0] + "." + args[1] : (String) args[1];
-                    when(mockDataset.getFullyQualifiedName()).thenReturn(fullyQualifiedName);
+                    when(mockDataset.getDataverseName()).thenReturn((DataverseName) args[0]);
+                    when(mockDataset.getDatasetName()).thenReturn((String) args[1]);
                     return mockDataset;
                 }
             });
@@ -184,14 +187,14 @@ public class ParserTestExecutor extends TestExecutor {
     }
 
     // Gets the default dataverse for the input statements.
-    private String getDefaultDataverse(List<Statement> statements) {
+    private DataverseName getDefaultDataverse(List<Statement> statements) {
         for (Statement st : statements) {
             if (st.getKind() == Statement.Kind.DATAVERSE_DECL) {
                 DataverseDecl dv = (DataverseDecl) st;
-                return dv.getDataverseName().getValue();
+                return dv.getDataverseName();
             }
         }
-        return null;
+        return MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME;
     }
 
     // Rewrite queries.
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
index 31a6004..aea32f5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.asterix.app.bootstrap.TestNodeController;
 import org.apache.asterix.common.config.DatasetConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -70,7 +71,7 @@ public class IndexDropOperatorNodePushableTest {
     private static final int[] KEY_INDEXES = { 0 };
     private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
     private static final int DATASET_ID = 101;
-    private static final String DATAVERSE_NAME = "TestDV";
+    private static final DataverseName DATAVERSE_NAME = DataverseName.createSinglePartName("TestDV");
     private static final String DATASET_NAME = "TestDS";
     private static final String DATA_TYPE_NAME = "DUMMY";
     private static final String NODE_GROUP_NAME = "DEFAULT";
@@ -148,7 +149,7 @@ public class IndexDropOperatorNodePushableTest {
                     .getClusterControllerService().getApplicationContext();
             MetadataProvider metadataProver = new MetadataProvider(appCtx, null);
             metadataProver.setMetadataTxnContext(mdTxn);
-            final String defaultDv = MetadataBuiltinEntities.DEFAULT_DATAVERSE.getDataverseName();
+            final DataverseName defaultDv = MetadataBuiltinEntities.DEFAULT_DATAVERSE.getDataverseName();
             final Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxn, defaultDv, datasetName);
             MetadataManager.INSTANCE.commitTransaction(mdTxn);
             FileSplit[] splits = SplitsAndConstraintsUtil.getIndexSplits(appCtx.getClusterStateManager(), dataset,
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-01.sqlpp
similarity index 64%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-01.sqlpp
index 2acf394..bc811a2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-01.sqlpp
@@ -16,21 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-import java.io.Serializable;
-import java.util.Map;
+/*
+ * Description: index in a dataverse with a multipart name
+ */
 
-import org.apache.asterix.active.EntityId;
+drop  dataverse x.y if exists;
+create  dataverse x.y;
 
-public interface IFeed extends Serializable {
+use x.y;
 
-    public String getFeedName();
+create type Emp as
+ closed {
+  id : bigint,
+  fname : string,
+  lname : string,
+  age : bigint,
+  dept : string
+};
 
-    public String getDataverseName();
+create dataset employee(Emp) primary key id;
 
-    public EntityId getFeedId();
+create index idx_employee_f_l_name on employee (fname,lname) type btree;
 
-    public Map<String, String> getConfiguration();
+select element l
+from  employee as l
+where ((l.fname = 'Julio') and (l.lname = 'Isa'))
+;
 
-}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-02.sqlpp
similarity index 63%
copy from asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
copy to asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-02.sqlpp
index 0584665..cf8da32 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/multipart-dataverse/index/index-02.sqlpp
@@ -16,17 +16,28 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.lang.common.base;
+/*
+ * Description: index in a dataverse with a multipart name
+ */
+
+drop  dataverse x.y if exists;
+create  dataverse x.y;
+
+create type x.y.Emp as
+ closed {
+  id : bigint,
+  fname : string,
+  lname : string,
+  age : bigint,
+  dept : string
+};
 
-import org.apache.asterix.common.exceptions.CompilationException;
+create dataset x.y.employee(Emp) primary key id;
 
-public interface IStatementRewriter {
+create index idx_employee_f_l_name on x.y.employee (fname,lname) type btree;
 
-    /**
-     * @param statement,
-     *            a non-query statement.
-     */
-    void rewrite(Statement statement) throws CompilationException;
+select element l
+from  x.y.employee as l
+where ((l.fname = 'Julio') and (l.lname = 'Isa'))
+;
 
-    String toExternalVariableName(String statementParameterName);
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
index ab8d14a..37fc73e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 = y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
index bd7d7b2..31126b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 = y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
index 5313576..3822746 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 /*+ indexnl */  = y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
index a3740b6..fb1bf93 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 /*+ indexnl */  = y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
index 6c1a30a..64433e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 /*+ indexnl */  >= y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
index 954b3a6..30c5406 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 /*+ indexnl */  <= y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
index 8763a0d..67e8208 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 /*+ indexnl */  > y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
index 748bfb0..84a84c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 /*+ indexnl */  < y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
index 0e99c88..a80c2ca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 /*+ indexnl */  <= y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
index 2eb1dde..66ac893 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 /*+ indexnl */  >= y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
index f717a64..3c719d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key1 /*+ indexnl */  < y.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
index e574877..d32d8a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
@@ -40,7 +40,7 @@ create  dataset test1.DsTwo(TestType) primary key key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.key2 /*+ indexnl */  > y.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
index 3debb01..427eaef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 = y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
index 7ed0a41..5969055 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 /*+ indexnl */  = y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
index 421b65d..8bc2165 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_02.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key2 /*+ indexnl */  = y.nested.key1)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
index 434c47d..4e13cc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 /*+ indexnl */  >= y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
index 45827bf..c006a0d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 /*+ indexnl */  > y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
index a2029f3..c36ed5b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 /*+ indexnl */  <= y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
index 20e535f..cb4241d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
@@ -45,7 +45,7 @@ create  dataset test1.DsTwo(TestType) primary key nested.key1;
 
 write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_01.adm";
 select element x
-from  `test1.DsOne` as x,
-      `test1.DsTwo` as y
+from  test1.DsOne as x,
+      test1.DsTwo as y
 where (x.nested.key1 /*+ indexnl */  < y.nested.key2)
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-01.plan
new file mode 100644
index 0000000..394213e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-01.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-02.plan
new file mode 100644
index 0000000..394213e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/multipart-dataverse/index/index-02.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$25(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue550.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue550.ast
index 042bd3d..d1ebcac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue550.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue550.ast
@@ -162,7 +162,7 @@ RecordConstructor [
       Variable [ Name=$srec ]
       ]
       FROM [        FunctionCall asterix.dataset@1[
-          LiteralExpr [STRING] [s]
+          LiteralExpr [STRING] [Default.s]
         ]
         AS Variable [ Name=$srec ]
       ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/1.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/1.ast
index f26febf..9dcc461 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/1.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/1.ast
@@ -12,7 +12,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [User]
+    LiteralExpr [STRING] [Default.User]
   ]
   AS Variable [ Name=$user ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/2.ast
index a605028..9b6b992 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/2.ast
@@ -19,7 +19,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Event]
+    LiteralExpr [STRING] [Default.Event]
   ]
   AS Variable [ Name=$event ]
 ,
@@ -105,7 +105,7 @@ Let Variable [ Name=$by_chapter ]
     ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [es]
+        LiteralExpr [STRING] [Default.es]
       ]
       AS Variable [ Name=$e ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/4.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/4.ast
index 9599255..48ad773 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/4.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/4.ast
@@ -9,7 +9,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [SIGroup]
+    LiteralExpr [STRING] [Default.SIGroup]
   ]
   AS Variable [ Name=$sig ]
 ]
@@ -29,7 +29,7 @@ Let Variable [ Name=$similar_sigs ]
     ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [SIGroup]
+        LiteralExpr [STRING] [Default.SIGroup]
       ]
       AS Variable [ Name=$similar_sig ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/5.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/5.ast
index 9eb7cf8..b629e2e 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/5.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/5.ast
@@ -9,7 +9,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Event]
+    LiteralExpr [STRING] [Default.Event]
   ]
   AS Variable [ Name=$event ]
 ]
@@ -29,7 +29,7 @@ Let Variable [ Name=$collocated_events ]
     ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [Events]
+        LiteralExpr [STRING] [Default.Events]
       ]
       AS Variable [ Name=$collocated_event ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/6.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/6.ast
index 854c7d1..b784f99 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/6.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/6.ast
@@ -17,7 +17,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Users]
+    LiteralExpr [STRING] [Default.Users]
   ]
   AS Variable [ Name=$user ]
 ]
@@ -42,7 +42,7 @@ Let Variable [ Name=$similar_users ]
     ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [Users]
+        LiteralExpr [STRING] [Default.Users]
       ]
       AS Variable [ Name=$similar_user ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
index 3e3575d..1ea056d 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
@@ -20,12 +20,12 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [User]
+    LiteralExpr [STRING] [Default.User]
   ]
   AS Variable [ Name=$user ]
 ,
   FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Movie]
+    LiteralExpr [STRING] [Default.Movie]
   ]
   AS Variable [ Name=$mv ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/LetFor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/LetFor.ast
index 4623444..275455a 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/LetFor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/LetFor.ast
@@ -6,7 +6,7 @@ Let Variable [ Name=$users ]
     Variable [ Name=$u ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [User]
+        LiteralExpr [STRING] [Default.User]
       ]
       AS Variable [ Name=$u ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
index b27701c..7d76691 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias.ast
@@ -4,7 +4,7 @@ Variable [ Name=#1 ]
 root
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [tbl_name]
+    LiteralExpr [STRING] [Default.tbl_name]
   ]
   AS Variable [ Name=$t ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
index 563b883..cd6ce26 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias2.ast
@@ -35,7 +35,7 @@ FunctionCall asterix.sql-sum@1[
 root
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [tbl_name]
+    LiteralExpr [STRING] [Default.tbl_name]
   ]
   AS Variable [ Name=$root ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
index 65cedd3..f72802f 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/columnalias3.ast
@@ -9,7 +9,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [tbl_name]
+    LiteralExpr [STRING] [Default.tbl_name]
   ]
   AS Variable [ Name=$t ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/functionDecl1.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/functionDecl1.ast
index 2443f90..60fc518 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/functionDecl1.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/functionDecl1.ast
@@ -107,7 +107,7 @@ Let Variable [ Name=$result ]
       Variable [ Name=$e ]
       ]
       FROM [        FunctionCall asterix.dataset@1[
-          LiteralExpr [STRING] [Events]
+          LiteralExpr [STRING] [Default.Events]
         ]
         AS Variable [ Name=$e ]
       ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR.ast
index aa25b42..3f27a49 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR.ast
@@ -19,7 +19,7 @@ FROM [  (
         Variable [ Name=$u ]
         ]
         FROM [          FunctionCall asterix.dataset@1[
-            LiteralExpr [STRING] [User]
+            LiteralExpr [STRING] [Default.User]
           ]
           AS Variable [ Name=$u ]
         ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
index 463ddc4..3434965 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR2.ast
@@ -35,7 +35,7 @@ FROM [  OrderedListConstructor [
       Variable [ Name=$tmp ]
       ]
       FROM [        FunctionCall asterix.dataset@1[
-          LiteralExpr [STRING] [number]
+          LiteralExpr [STRING] [Default.number]
         ]
         AS Variable [ Name=$tmp ]
       ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR3.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR3.ast
index f4d8694..64b580f 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR3.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFLWOGR3.ast
@@ -19,7 +19,7 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Event]
+    LiteralExpr [STRING] [Default.Event]
   ]
   AS Variable [ Name=$event ]
 ,
@@ -105,7 +105,7 @@ Let Variable [ Name=$by_chapter ]
     ]
     ]
     FROM [      FunctionCall asterix.dataset@1[
-        LiteralExpr [STRING] [es]
+        LiteralExpr [STRING] [Default.es]
       ]
       AS Variable [ Name=$e ]
     ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFor.ast
index 52a9319..1fe5d95 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/nestedFor.ast
@@ -20,12 +20,12 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [User]
+    LiteralExpr [STRING] [Default.User]
   ]
   AS Variable [ Name=$user ]
 ,
   FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Movie]
+    LiteralExpr [STRING] [Default.Movie]
   ]
   AS Variable [ Name=$mv ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/numberInFieldAccessor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/numberInFieldAccessor.ast
index 605113f..bc58ab8 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/numberInFieldAccessor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/numberInFieldAccessor.ast
@@ -20,12 +20,12 @@ RecordConstructor [
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [User]
+    LiteralExpr [STRING] [Default.User]
   ]
   AS Variable [ Name=$user ]
 ,
   FunctionCall asterix.dataset@1[
-    LiteralExpr [STRING] [Movie]
+    LiteralExpr [STRING] [Default.Movie]
   ]
   AS Variable [ Name=$mv ]
 ]
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.1.ddl.sqlpp
similarity index 69%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.1.ddl.sqlpp
index 2acf394..910b6df 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.1.ddl.sqlpp
@@ -16,21 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.asterix.active.EntityId;
+/*
+ * Description: index in a dataverse with a multipart name
+ */
 
-public interface IFeed extends Serializable {
+drop  dataverse x.y if exists;
+create  dataverse x.y;
 
-    public String getFeedName();
+use x.y;
 
-    public String getDataverseName();
+create type Emp as
+ closed {
+  id : bigint,
+  fname : string,
+  lname : string,
+  age : bigint,
+  dept : string
+};
 
-    public EntityId getFeedId();
+create dataset employee(Emp) primary key id;
 
-    public Map<String, String> getConfiguration();
+create index idx_employee_f_l_name on employee (fname,lname) type btree;
 
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.2.update.sqlpp
similarity index 80%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.2.update.sqlpp
index 9e7859c..72db572 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.2.update.sqlpp
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
+/*
+ * Description: index in a dataverse with a multipart name
+ */
 
-public interface FunctionConstants {
+use x.y;
 
-    public static final String ASTERIX_NS = "asterix";
+load dataset employee using localfs ((`path`=`asterix_nc1://data/names.adm`),(`format`=`delimited-text`),(`delimiter`=`|`));
 
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.3.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.3.query.sqlpp
index 9e7859c..ee3a680 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/index_1/index_1.3.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: index in a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+use x.y;
 
-}
+select element l
+from  employee as l
+where ((l.fname = 'Julio') and (l.lname = 'Isa'))
+;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.1.ddl.sqlpp
similarity index 63%
copy from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.1.ddl.sqlpp
index 2acf394..1089bc3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.1.ddl.sqlpp
@@ -16,21 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.external.feed.api;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.asterix.active.EntityId;
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-public interface IFeed extends Serializable {
+drop dataverse sales.east if exists;
+create dataverse sales.east;
 
-    public String getFeedName();
+create type sales.east.orderType as {
+  oid: bigint
+};
 
-    public String getDataverseName();
+create dataset sales.east.orders(sales.east.orderType) primary key oid;
 
-    public EntityId getFeedId();
+drop dataverse sales.west if exists;
+create dataverse sales.west;
 
-    public Map<String, String> getConfiguration();
+create type sales.west.orderType as {
+  oid: bigint
+};
 
-}
+create dataset sales.west.orders(sales.west.orderType) primary key oid;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.10.query.sqlpp
similarity index 81%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.10.query.sqlpp
index 9e7859c..ca18151 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.10.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* Single name resolution rule with a nested field */
 
-}
+select x.y.z
+from sales.east.orders
+order by oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.11.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.11.query.sqlpp
index 9e7859c..c2143ce 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.11.query.sqlpp
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* Join */
 
-}
+select east.east, west.west
+from sales.east.orders east, sales.west.orders west
+where east.oid = west.oid
+order by east.oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.2.update.sqlpp
similarity index 56%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.2.update.sqlpp
index 276e294..09104e4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.2.update.sqlpp
@@ -16,29 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.metadata;
-
-import java.util.List;
-
-public interface IDataset {
-
-    /**
-     * @return the dataverse name
-     */
-    String getDataverseName();
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    /**
-     * @return the dataset name
-     */
-    String getDatasetName();
+use sales.east;
 
-    /**
-     * @return the list of primary keys for the dataset
-     */
-    List<List<String>> getPrimaryKeys();
+insert into orders ({'oid' : 100, 'pid': 0,  'amount': 1, 'east': [11, 111], 'x': { 'y': {'z': 11 } } });
+insert into orders ({'oid' : 101, 'pid': 1,  'amount': 2, 'east': [22, 222], 'x': { 'y': {'z': 22 } } });
+insert into orders ({'oid' : 102, 'pid': 2,  'amount': 3, 'east': [33, 333], 'x': { 'y': {'z': 33 } } });
+insert into orders ({'oid' : 103, 'pid': 1,  'amount': 4, 'east': [44, 444], 'x': { 'y': {'z': 44 } } });
+insert into orders ({'oid' : 104, 'pid': 0,  'amount': 6, 'east': [66, 666], 'x': { 'y': {'z': 66 } } });
 
-    /**
-     * @return the bloom filter fields indexes for the primary index of the dataset
-     */
-    int[] getPrimaryBloomFilterFields();
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.3.update.sqlpp
similarity index 55%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.3.update.sqlpp
index 9e7859c..77f6f30 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.3.update.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-public interface FunctionConstants {
+use sales.west;
 
-    public static final String ASTERIX_NS = "asterix";
+insert into orders ({'oid' : 100, 'pid': 0,  'amount': 1, 'west': [1111, 11111], 'x': { 'y': {'z': 1111 } } });
+insert into orders ({'oid' : 101, 'pid': 1,  'amount': 2, 'west': [2222, 22222], 'x': { 'y': {'z': 2222 } } });
+insert into orders ({'oid' : 102, 'pid': 2,  'amount': 3, 'west': [3333, 33333], 'x': { 'y': {'z': 3333 } } });
+insert into orders ({'oid' : 103, 'pid': 1,  'amount': 4, 'west': [4444, 44444], 'x': { 'y': {'z': 4444 } } });
+insert into orders ({'oid' : 104, 'pid': 0,  'amount': 6, 'west': [6666, 66666], 'x': { 'y': {'z': 6666 } } });
 
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.4.query.sqlpp
similarity index 85%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.4.query.sqlpp
index 9e7859c..6e589a6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.4.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+use sales.east;
 
-}
+select value o
+from orders o
+order by oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.5.query.sqlpp
similarity index 83%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.5.query.sqlpp
index 9e7859c..e3adbad 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.5.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-public interface FunctionConstants {
+/* Use backticks */
 
-    public static final String ASTERIX_NS = "asterix";
+use `sales`.`east`;
 
-}
+select value o
+from orders o
+order by oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.6.query.sqlpp
similarity index 82%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.6.query.sqlpp
index 9e7859c..0905b35 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.6.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* FROM fully qualified dataset name */
 
-}
+select value o
+from sales.east.orders o
+order by oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.7.query.sqlpp
similarity index 80%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.7.query.sqlpp
index 9e7859c..7103abc 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.7.query.sqlpp
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* Local variables first in FROM clause */
 
-}
+select value v
+from sales.east.orders as sales, sales.east as v
+order by v
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.8.query.sqlpp
similarity index 81%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.8.query.sqlpp
index 9e7859c..97e6263 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.8.query.sqlpp
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* Local variables first in SELECT clause */
 
-}
+select sales.east[0] v
+from sales.east.orders as sales
+order by v
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.9.query.sqlpp
similarity index 82%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.9.query.sqlpp
index 9e7859c..d6d1320 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/resolution_1/resolution_1.9.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: name resolution for a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+/* Test auto-generated variable name */
 
-}
+select orders
+from sales.east.orders
+order by oid
+limit 2
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.1.ddl.sqlpp
similarity index 71%
copy from asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.1.ddl.sqlpp
index e472383..471cd01 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataEntityTupleTranslatorFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.1.ddl.sqlpp
@@ -16,14 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.metadata.api;
+/*
+ * Description: special characters in multipart dataverse name
+ */
+
+drop dataverse `a.b`.`c@d` if exists;
+create dataverse `a.b`.`c@d`;
 
-import java.io.Serializable;
+use `a.b`.`c@d`;
 
-public interface IMetadataEntityTupleTranslatorFactory<T> extends Serializable {
+create type Emp as
+ closed {
+  id : bigint,
+  fname : string,
+  lname : string,
+  age : bigint,
+  dept : string
+};
 
-    /**
-     * @return an instance of IMetadataEntityTupleTranslator
-     */
-    IMetadataEntityTupleTranslator<T> createTupleTranslator();
-}
+create dataset employee(Emp) primary key id;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.2.update.sqlpp
similarity index 79%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.2.update.sqlpp
index 9e7859c..73a003d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.2.update.sqlpp
@@ -16,10 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
-
-    public static final String ASTERIX_NS = "asterix";
+/*
+ * Description: special characters in multipart dataverse name
+ */
 
-}
+load dataset `a.b`.`c@d`.employee using localfs ((`path`=`asterix_nc1://data/names.adm`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.3.query.sqlpp
similarity index 82%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.3.query.sqlpp
index 9e7859c..84b899d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.3.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: special characters in multipart dataverse name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+use `a.b`.`c@d`;
 
-}
+select element l
+from  employee as l
+where ((l.fname = 'Julio') and (l.lname = 'Isa'))
+;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.4.query.sqlpp
similarity index 82%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.4.query.sqlpp
index 9e7859c..86b6f68 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/special_chars_1/special_chars_1.4.query.sqlpp
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
-
-    public static final String ASTERIX_NS = "asterix";
+/*
+ * Description: special characters in multipart dataverse name
+ */
 
-}
+select element l
+from  `a.b`.`c@d`.employee as l
+where ((l.fname = 'Julio') and (l.lname = 'Isa'))
+;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.1.ddl.sqlpp
similarity index 82%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.1.ddl.sqlpp
index 9e7859c..6b01a11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.1.ddl.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: user-defined function in a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+drop  dataverse x.y if exists;
+create  dataverse x.y;
 
-}
+create function x.y.echo(v) {
+  v
+};
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.2.query.sqlpp
similarity index 85%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.2.query.sqlpp
index 9e7859c..76263f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.2.query.sqlpp
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
-
-public interface FunctionConstants {
+/*
+ * Description: user-defined function in a dataverse with a multipart name
+ */
 
-    public static final String ASTERIX_NS = "asterix";
+select value x.y.echo(r)
+from range(1,2) r
+order by r
 
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.3.query.sqlpp
similarity index 85%
copy from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.3.query.sqlpp
index 9e7859c..1e5bb2e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/multipart-dataverse/udf_1/udf_1.3.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.common.functions;
+/*
+ * Description: user-defined function in a dataverse with a multipart name
+ */
 
-public interface FunctionConstants {
+use x.y;
 
-    public static final String ASTERIX_NS = "asterix";
+select value echo(r)
+from range(1,2) r
+order by r
 
-}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp
index 215118b..85e966a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/check-dependencies-1/check-dependencies-1.3.ddl.sqlpp
@@ -26,9 +26,7 @@ drop function B.f5@2;
 drop function C.f3@2;
 drop function C.f2@2;
 
-drop dataset B.TweetMessages2;
-
-drop dataverse C;
 drop dataverse B;
+drop dataverse C;
 
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/index_1/index_1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/index_1/index_1.3.adm
new file mode 100644
index 0000000..99238f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/index_1/index_1.3.adm
@@ -0,0 +1 @@
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.10.adm
new file mode 100644
index 0000000..cdd804a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.10.adm
@@ -0,0 +1,2 @@
+{ "z": 11 }
+{ "z": 22 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.11.adm
new file mode 100644
index 0000000..3f84b21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.11.adm
@@ -0,0 +1,2 @@
+{ "east": [ 11, 111 ], "west": [ 1111, 11111 ] }
+{ "east": [ 22, 222 ], "west": [ 2222, 22222 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.4.adm
new file mode 100644
index 0000000..438a875
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.4.adm
@@ -0,0 +1,2 @@
+{ "oid": 100, "pid": 0, "amount": 1, "east": [ 11, 111 ], "x": { "y": { "z": 11 } } }
+{ "oid": 101, "pid": 1, "amount": 2, "east": [ 22, 222 ], "x": { "y": { "z": 22 } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.5.adm
new file mode 100644
index 0000000..438a875
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.5.adm
@@ -0,0 +1,2 @@
+{ "oid": 100, "pid": 0, "amount": 1, "east": [ 11, 111 ], "x": { "y": { "z": 11 } } }
+{ "oid": 101, "pid": 1, "amount": 2, "east": [ 22, 222 ], "x": { "y": { "z": 22 } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.6.adm
new file mode 100644
index 0000000..438a875
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.6.adm
@@ -0,0 +1,2 @@
+{ "oid": 100, "pid": 0, "amount": 1, "east": [ 11, 111 ], "x": { "y": { "z": 11 } } }
+{ "oid": 101, "pid": 1, "amount": 2, "east": [ 22, 222 ], "x": { "y": { "z": 22 } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.7.adm
new file mode 100644
index 0000000..d2f3f89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.7.adm
@@ -0,0 +1,10 @@
+11
+22
+33
+44
+66
+111
+222
+333
+444
+666
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.8.adm
new file mode 100644
index 0000000..85a0da8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.8.adm
@@ -0,0 +1,5 @@
+{ "v": 11 }
+{ "v": 22 }
+{ "v": 33 }
+{ "v": 44 }
+{ "v": 66 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.9.adm
new file mode 100644
index 0000000..9dd18c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/resolution_1/resolution_1.9.adm
@@ -0,0 +1,2 @@
+{ "orders": { "oid": 100, "pid": 0, "amount": 1, "east": [ 11, 111 ], "x": { "y": { "z": 11 } } } }
+{ "orders": { "oid": 101, "pid": 1, "amount": 2, "east": [ 22, 222 ], "x": { "y": { "z": 22 } } } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.3.adm
new file mode 100644
index 0000000..99238f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.3.adm
@@ -0,0 +1 @@
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.4.adm
new file mode 100644
index 0000000..99238f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/special_chars_1/special_chars_1.4.adm
@@ -0,0 +1 @@
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.2.adm
new file mode 100644
index 0000000..7a754f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.2.adm
@@ -0,0 +1,2 @@
+1
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.3.adm
new file mode 100644
index 0000000..7a754f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/multipart-dataverse/udf_1/udf_1.3.adm
@@ -0,0 +1,2 @@
+1
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm
index 50f19bb..50ba196 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/check-dependencies-1/check-dependencies-1.1.adm
@@ -1,5 +1,5 @@
 { "DataverseName": "B", "Name": "f0", "Dependencies": [ [  ], [  ] ] }
-{ "DataverseName": "B", "Name": "f5", "Dependencies": [ [ [ "C", "TweetMessages" ], [ "B", "TweetMessages2" ] ], [ [ "C", "f1", "2" ], [ "B", "f0", "2" ] ] ] }
+{ "DataverseName": "B", "Name": "f5", "Dependencies": [ [ [ "B", "TweetMessages2" ], [ "C", "TweetMessages" ] ], [ [ "C", "f1", "2" ], [ "B", "f0", "2" ] ] ] }
 { "DataverseName": "C", "Name": "f1", "Dependencies": [ [  ], [  ] ] }
 { "DataverseName": "C", "Name": "f2", "Dependencies": [ [ [ "C", "TweetMessages" ] ], [ [ "C", "f1", "2" ], [ "B", "f0", "2" ] ] ] }
 { "DataverseName": "C", "Name": "f3", "Dependencies": [ [  ], [ [ "C", "f2", "2" ] ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index f16b187..3fcce1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -793,7 +793,7 @@
         <output-dir compare="Text">partition-by-nonexistent-field</output-dir>
         <expected-error>Field "id" is not found</expected-error>
         <expected-error>Cannot find dataset</expected-error>
-        <expected-error>Could not find dataset</expected-error>
+        <expected-error>Cannot find dataset</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
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 d301f29..3a0d50b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -45,7 +45,6 @@
     <test-case FilePath="api">
       <compilation-unit name="request-param-validation">
         <output-dir compare="Text">request-param-validation</output-dir>
-        <source-location>false</source-location>
         <expected-error>Invalid value for parameter "format": foo</expected-error>
         <expected-error>Invalid value for parameter "pretty": bar</expected-error>
         <expected-error>Invalid value for parameter "plan-format": blah</expected-error>
@@ -71,6 +70,7 @@
         <expected-error>Invalid value for parameter "profile": true</expected-error>
         <expected-error>Invalid value for parameter "profile": foo</expected-error>
         <expected-error>Invalid value for parameter "profile": foo</expected-error>
+        <source-location>false</source-location>
       </compilation-unit>
     </test-case>
     <test-case FilePath="api">
@@ -6452,6 +6452,28 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="multipart-dataverse">
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="index_1">
+        <output-dir compare="Text">index_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="resolution_1">
+        <output-dir compare="Text">resolution_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="special_chars_1">
+        <output-dir compare="Text">special_chars_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="multipart-dataverse">
+      <compilation-unit name="udf_1">
+        <output-dir compare="Text">udf_1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
   <test-group name="index">
     <test-group name="index/validations">
       <test-case FilePath="index/validations">
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 760e027..2d354a7 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -184,6 +184,11 @@
       <artifactId>commons-lang3</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-collections4</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hyracks</groupId>
       <artifactId>algebricks-common</artifactId>
     </dependency>
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 bed9869..1b92394 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
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.common.api;
 
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.metadata.LockList;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
@@ -33,7 +34,7 @@ public interface IMetadataLockManager {
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDataverseReadLock(LockList locks, String dataverseName) throws AlgebricksException;
+    void acquireDataverseReadLock(LockList locks, DataverseName dataverseName) throws AlgebricksException;
 
     /**
      * Acquire write lock on the dataverse
@@ -45,55 +46,67 @@ public interface IMetadataLockManager {
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDataverseWriteLock(LockList locks, String dataverseName) throws AlgebricksException;
+    void acquireDataverseWriteLock(LockList locks, DataverseName dataverseName) throws AlgebricksException;
 
     /**
      * Acquire read lock on the dataset (for queries)
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDatasetReadLock(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+    void acquireDatasetReadLock(LockList locks, DataverseName dataverseName, String datasetName)
+            throws AlgebricksException;
 
     /**
      * Acquire write lock on the dataset (for dataset create, dataset drop, and index drop)
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDatasetWriteLock(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+    void acquireDatasetWriteLock(LockList locks, DataverseName dataverseName, String datasetName)
+            throws AlgebricksException;
 
     /**
      * Acquire modify lock on the dataset (for inserts, upserts, deletes) Mutually exclusive with create index lock
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDatasetModifyLock(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+    void acquireDatasetModifyLock(LockList locks, DataverseName dataverseName, String datasetName)
+            throws AlgebricksException;
 
     /**
      * Acquire create index lock on the dataset (for index creation) Mutually exclusive with modify lock
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDatasetCreateIndexLock(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+    void acquireDatasetCreateIndexLock(LockList locks, DataverseName dataverseName, String datasetName)
+            throws AlgebricksException;
 
     /**
      * Acquire exclusive modify lock on the dataset. only a single thread can acquire this lock and it is mutually
@@ -101,12 +114,14 @@ public interface IMetadataLockManager {
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDatasetExclusiveModificationLock(LockList locks, String datasetFullyQualifiedName)
+    void acquireDatasetExclusiveModificationLock(LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
 
     /**
@@ -114,24 +129,30 @@ public interface IMetadataLockManager {
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param functionFullyQualifiedName
-     *            the fully qualified name of the function
+     * @param dataverseName
+     *            the dataverse name
+     * @param functionName
+     *            the name of the function in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireFunctionReadLock(LockList locks, String functionFullyQualifiedName) throws AlgebricksException;
+    void acquireFunctionReadLock(LockList locks, DataverseName dataverseName, String functionName)
+            throws AlgebricksException;
 
     /**
      * Acquire write lock on the function
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param functionFullyQualifiedName
-     *            the fully qualified name of the function
+     * @param dataverseName
+     *            the dataverse name
+     * @param functionName
+     *            the name of the function in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireFunctionWriteLock(LockList locks, String functionFullyQualifiedName) throws AlgebricksException;
+    void acquireFunctionWriteLock(LockList locks, DataverseName dataverseName, String functionName)
+            throws AlgebricksException;
 
     /**
      * Acquire read lock on the node group
@@ -162,96 +183,114 @@ public interface IMetadataLockManager {
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param entityFullyQualifiedName
-     *            the fully qualified name of the active entity
+     * @param dataverseName
+     *            the dataverse name
+     * @param entityName
+     *            the name of the active entity in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireActiveEntityReadLock(LockList locks, String entityFullyQualifiedName) throws AlgebricksException;
+    void acquireActiveEntityReadLock(LockList locks, DataverseName dataverseName, String entityName)
+            throws AlgebricksException;
 
     /**
      * Acquire write lock on the active entity
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param entityFullyQualifiedName
-     *            the fully qualified name of the active entity
+     * @param dataverseName
+     *            the dataverse name
+     * @param entityName
+     *            the name of the active entity in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireActiveEntityWriteLock(LockList locks, String entityFullyQualifiedName) throws AlgebricksException;
+    void acquireActiveEntityWriteLock(LockList locks, DataverseName dataverseName, String entityName)
+            throws AlgebricksException;
 
     /**
      * Acquire read lock on the feed policy
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param feedPolicyFullyQualifiedName
-     *            the fully qualified name of the feed policy
+     * @param dataverseName
+     *            the dataverse name
+     * @param feedPolicyName
+     *            the name of the feed policy in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireFeedPolicyWriteLock(LockList locks, String feedPolicyFullyQualifiedName) throws AlgebricksException;
+    void acquireFeedPolicyWriteLock(LockList locks, DataverseName dataverseName, String feedPolicyName)
+            throws AlgebricksException;
 
     /**
      * Acquire write lock on the feed policy
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param feedPolicyFullyQualifiedName
-     *            the fully qualified name of the feed policy
+     * @param dataverseName
+     *            the dataverse name
+     * @param feedPolicyName
+     *            the name of the feed policy in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireFeedPolicyReadLock(LockList locks, String feedPolicyFullyQualifiedName) throws AlgebricksException;
+    void acquireFeedPolicyReadLock(LockList locks, DataverseName dataverseName, String feedPolicyName)
+            throws AlgebricksException;
 
     /**
      * Acquire read lock on the merge policy
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param mergePolicyFullyQualifiedName
-     *            the fully qualified name of the merge policy
+     * @param mergePolicyName
+     *            the name of the merge policy in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireMergePolicyReadLock(LockList locks, String mergePolicyFullyQualifiedName) throws AlgebricksException;
+    void acquireMergePolicyReadLock(LockList locks, String mergePolicyName) throws AlgebricksException;
 
     /**
      * Acquire write lock on the merge policy
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param mergePolicyFullyQualifiedName
-     *            the fully qualified name of the merge policy
+     * @param mergePolicyName
+     *            the name of the merge policy in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireMergePolicyWriteLock(LockList locks, String mergePolicyFullyQualifiedName) throws AlgebricksException;
+    void acquireMergePolicyWriteLock(LockList locks, String mergePolicyName) throws AlgebricksException;
 
     /**
      * Acquire read lock on the data type
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datatypeFullyQualifiedName
-     *            the fully qualified name of the data type
+     * @param dataverseName
+     *            the dataverse name
+     * @param datatypeName
+     *            the name of the data type in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDataTypeReadLock(LockList locks, String datatypeFullyQualifiedName) throws AlgebricksException;
+    void acquireDataTypeReadLock(LockList locks, DataverseName dataverseName, String datatypeName)
+            throws AlgebricksException;
 
     /**
      * Acquire write lock on the data type
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datatypeFullyQualifiedName
-     *            the fully qualified name of the data type
+     * @param dataverseName
+     *            the dataverse name
+     * @param datatypeName
+     *            the name of the data type in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireDataTypeWriteLock(LockList locks, String datatypeFullyQualifiedName) throws AlgebricksException;
+    void acquireDataTypeWriteLock(LockList locks, DataverseName dataverseName, String datatypeName)
+            throws AlgebricksException;
 
     /**
      * Acquire read lock on the extension entity
@@ -260,13 +299,15 @@ public interface IMetadataLockManager {
      *            the lock list to add the new lock to
      * @param extension
      *            the extension key
-     * @param extensionEntityFullyQualifiedName
-     *            the fully qualified name of the extension entity
+     * @param dataverseName
+     *            the dataverse name
+     * @param extensionEntityName
+     *            the name of the extension entity in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireExtensionReadLock(LockList locks, String extension, String extensionEntityFullyQualifiedName)
-            throws AlgebricksException;
+    void acquireExtensionEntityReadLock(LockList locks, String extension, DataverseName dataverseName,
+            String extensionEntityName) throws AlgebricksException;
 
     /**
      * Acquire write lock on the extension entity
@@ -275,36 +316,43 @@ public interface IMetadataLockManager {
      *            the lock list to add the new lock to
      * @param extension
      *            the extension key
-     * @param extensionEntityFullyQualifiedName
-     *            the fully qualified name of the extension entity
+     * @param dataverseName
+     *            the dataverse name
+     * @param extensionEntityName
+     *            the name of the extension entity in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be acquired
      */
-    void acquireExtensionWriteLock(LockList locks, String extension, String extensionEntityFullyQualifiedName)
-            throws AlgebricksException;
+    void acquireExtensionEntityWriteLock(LockList locks, String extension, DataverseName dataverseName,
+            String extensionEntityName) throws AlgebricksException;
 
     /**
      * Upgrade a previously acquired exclusive modification lock on the dataset to a write lock
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be upgraded
      */
-    void upgradeDatasetLockToWrite(LockList locks, String datasetFullyQualifiedName) throws AlgebricksException;
+    void upgradeDatasetLockToWrite(LockList locks, DataverseName dataverseName, String datasetName)
+            throws AlgebricksException;
 
     /**
      * Downgrade an upgraded dataset write lock to an exclusive modification lock
      *
      * @param locks
      *            the lock list to add the new lock to
-     * @param datasetFullyQualifiedName
-     *            the fully qualified name of the dataset
+     * @param dataverseName
+     *            the dataverse name
+     * @param datasetName
+     *            the name of the dataset in the given dataverse
      * @throws AlgebricksException
      *             if lock couldn't be downgraded
      */
-    void downgradeDatasetLockToExclusiveModify(LockList locks, String datasetFullyQualifiedName)
+    void downgradeDatasetLockToExclusiveModify(LockList locks, DataverseName dataverseName, String datasetName)
             throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
index 9e7859c..7343a11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionConstants.java
@@ -18,8 +18,18 @@
  */
 package org.apache.asterix.common.functions;
 
-public interface FunctionConstants {
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+
+public final class FunctionConstants {
 
     public static final String ASTERIX_NS = "asterix";
 
+    public static final DataverseName ASTERIX_DV = DataverseName.createBuiltinDataverseName(ASTERIX_NS);
+
+    public static final DataverseName ALGEBRICKS_DV =
+            DataverseName.createBuiltinDataverseName(AlgebricksBuiltinFunctions.ALGEBRICKS_NS);
+
+    private FunctionConstants() {
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionSignature.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionSignature.java
index d7b054d..9bbf2e7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionSignature.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/functions/FunctionSignature.java
@@ -19,21 +19,28 @@
 package org.apache.asterix.common.functions;
 
 import java.io.Serializable;
+import java.util.Objects;
 
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class FunctionSignature implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private String namespace;
+
+    private static final long serialVersionUID = 2L;
+
+    private DataverseName dataverseName;
+
     private String name;
+
     private int arity;
 
     public FunctionSignature(FunctionIdentifier fi) {
-        this(fi.getNamespace(), fi.getName(), fi.getArity());
+        this(getDataverseName(fi), fi.getName(), fi.getArity());
     }
 
-    public FunctionSignature(String namespace, String name, int arity) {
-        this.namespace = namespace;
+    public FunctionSignature(DataverseName dataverseName, String name, int arity) {
+        this.dataverseName = dataverseName;
         this.name = name;
         this.arity = arity;
     }
@@ -42,27 +49,36 @@ public class FunctionSignature implements Serializable {
     public boolean equals(Object o) {
         if (!(o instanceof FunctionSignature)) {
             return false;
-        } else {
-            FunctionSignature f = ((FunctionSignature) o);
-            return ((namespace != null && namespace.equals(f.getNamespace())
-                    || (namespace == null && f.getNamespace() == null))) && name.equals(f.getName())
-                    && (arity == f.getArity() || arity == FunctionIdentifier.VARARGS
-                            || f.getArity() == FunctionIdentifier.VARARGS);
         }
+        FunctionSignature f = ((FunctionSignature) o);
+        return Objects.equals(dataverseName, f.dataverseName) && name.equals(f.name)
+                && (arity == f.arity || arity == FunctionIdentifier.VARARGS || f.arity == FunctionIdentifier.VARARGS);
     }
 
     @Override
     public String toString() {
-        return namespace + "." + name + "@" + arity;
+        return toString(true);
+    }
+
+    public String toString(boolean includeArity) {
+        String namespaceCanonicalForm = dataverseName != null ? dataverseName.getCanonicalForm() : null;
+        int len = (namespaceCanonicalForm != null ? namespaceCanonicalForm.length() : 4) + 1 + name.length()
+                + (includeArity ? 3 : 0);
+        StringBuilder sb = new StringBuilder(len);
+        sb.append(namespaceCanonicalForm).append('.').append(name);
+        if (includeArity) {
+            sb.append('@').append(arity);
+        }
+        return sb.toString();
     }
 
     @Override
     public int hashCode() {
-        return (namespace + "." + name).hashCode();
+        return Objects.hash(dataverseName, name);
     }
 
-    public String getNamespace() {
-        return namespace;
+    public DataverseName getDataverseName() {
+        return dataverseName;
     }
 
     public String getName() {
@@ -73,8 +89,8 @@ public class FunctionSignature implements Serializable {
         return arity;
     }
 
-    public void setNamespace(String namespace) {
-        this.namespace = namespace;
+    public void setDataverseName(DataverseName dataverseName) {
+        this.dataverseName = dataverseName;
     }
 
     public void setName(String name) {
@@ -85,4 +101,28 @@ public class FunctionSignature implements Serializable {
         this.arity = arity;
     }
 
+    public FunctionIdentifier createFunctionIdentifier() {
+        return createFunctionIdentifier(dataverseName, name, arity);
+    }
+
+    public static FunctionIdentifier createFunctionIdentifier(DataverseName dataverseName, String functionName,
+            int arity) {
+        return new FunctionIdentifier(dataverseName.getCanonicalForm(), functionName, arity);
+    }
+
+    public static FunctionIdentifier createFunctionIdentifier(DataverseName dataverseName, String functionName) {
+        return new FunctionIdentifier(dataverseName.getCanonicalForm(), functionName);
+    }
+
+    public static DataverseName getDataverseName(FunctionIdentifier fi) {
+        String dataverseCanonicalName = fi.getNamespace();
+        switch (dataverseCanonicalName) {
+            case FunctionConstants.ASTERIX_NS:
+                return FunctionConstants.ASTERIX_DV;
+            case AlgebricksBuiltinFunctions.ALGEBRICKS_NS:
+                return FunctionConstants.ALGEBRICKS_DV;
+            default:
+                return DataverseName.createFromCanonicalForm(dataverseCanonicalName);
+        }
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index c1598d9..d7f4309 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -22,6 +22,7 @@ package org.apache.asterix.common.library;
 import java.net.URLClassLoader;
 import java.util.List;
 
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -30,10 +31,11 @@ public interface ILibraryManager {
     /**
      * Registers the library class loader with the external library manager.
      * <code>dataverseName</code> and <code>libraryName</code> uniquely identifies a class loader.
+     * @param dataverseName
      * @param libraryName
      * @param classLoader
      */
-    void registerLibraryClassLoader(String dataverseName, String libraryName, URLClassLoader classLoader)
+    void registerLibraryClassLoader(DataverseName dataverseName, String libraryName, URLClassLoader classLoader)
             throws HyracksDataException;
 
     /**
@@ -43,11 +45,10 @@ public interface ILibraryManager {
 
     /**
      * De-registers a library class loader.
-     *
-     * @param dataverseName
+     *  @param dataverseName
      * @param libraryName
      */
-    void deregisterLibraryClassLoader(String dataverseName, String libraryName);
+    void deregisterLibraryClassLoader(DataverseName dataverseName, String libraryName);
 
     /**
      * Finds a class loader for a given pair of dataverse name and library name.
@@ -56,7 +57,7 @@ public interface ILibraryManager {
      * @param libraryName
      * @return the library class loader associated with the dataverse and library.
      */
-    ClassLoader getLibraryClassLoader(String dataverseName, String libraryName);
+    ClassLoader getLibraryClassLoader(DataverseName dataverseName, String libraryName);
 
     /**
      * Add function parameters  to library manager if it exists.
@@ -65,7 +66,7 @@ public interface ILibraryManager {
      * @param parameters
      */
 
-    void addFunctionParameters(String dataverseName, String fullFunctionName, List<String> parameters);
+    void addFunctionParameters(DataverseName dataverseName, String fullFunctionName, List<String> parameters);
 
     /**
      * Get a list of parameters.
@@ -73,5 +74,5 @@ public interface ILibraryManager {
      * @param fullFunctionName
      * @return A list contains all pre-specified function parameters.
      */
-    List<String> getFunctionParameters(String dataverseName, String fullFunctionName);
+    List<String> getFunctionParameters(DataverseName dataverseName, String fullFunctionName);
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java
new file mode 100644
index 0000000..5ec3f1a
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java
@@ -0,0 +1,335 @@
+/*
+ * 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.metadata;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.function.BiConsumer;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * This class represents a dataverse name.
+ * The logical model is an ordered list of strings (name parts).
+ * Use {@link #create(List)} to create a dataverse name from its parts
+ * and {@link #getParts()} to obtain a list of parts from given dataverse name.
+ * <p>
+ * Each dataverse name can be encoded into a single string (called a canonical form) by
+ * {@link #getCanonicalForm()} and decoded back from it with {@link #createFromCanonicalForm(String)}.
+ * The canonical form encoding concatenates name parts together with {@link #SEPARATOR_CHAR '.'} character.
+ * The {@link #ESCAPE_CHAR '@'} character is used to escape {@link #SEPARATOR_CHAR '.'} and itself in each name part
+ * prior to concatenation.
+ * <p>
+ * E.g. the canonical form for a dataverse name {@code ["a", "b", "c"]} is {@code "a.b.c"}
+ * <p>
+ * {@link #toString()} returns a display form which is a {@link #SEPARATOR_CHAR '.'} separated concatenation
+ * of name parts without escaping. In general it's impossible to reconstruct a dataverse name from its display form.
+ * <p>
+ * Notes:
+ * <li>
+ * <ul>
+ * {@link #getCanonicalForm()} is faster than {@link #getParts()} because this class stores the canonical form,
+ * so {@link #getCanonicalForm()} just returns it while {@link #getParts()} performs parsing and string construction
+ * for each name part.
+ * </ul>
+ * <ul>
+ * {@link #toString()} result is cached, subsequent invocations just return the cached value.
+ * </ul>
+ * </li>
+ */
+public final class DataverseName implements Serializable, Comparable<DataverseName> {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final char SEPARATOR_CHAR = '.';
+
+    private static final char ESCAPE_CHAR = '@';
+
+    private static final char[] SEPARATOR_AND_ESCAPE_CHARS = new char[] { SEPARATOR_CHAR, ESCAPE_CHAR };
+
+    private final boolean isMultiPart;
+
+    private final String canonicalForm;
+
+    private transient volatile String displayForm;
+
+    private DataverseName(String canonicalForm, boolean isMultiPart) {
+        this.canonicalForm = Objects.requireNonNull(canonicalForm);
+        this.isMultiPart = isMultiPart;
+    }
+
+    /**
+     * Returns whether this dataverse name contains multiple name parts or not.
+     */
+    public boolean isMultiPart() {
+        return isMultiPart;
+    }
+
+    /**
+     * Returns a scalar encoding of this dataverse name.
+     * The returned value can be used to reconstruct this name by calling {@link #createFromCanonicalForm(String)}.
+     * <p>
+     * Warning: changing the canonical form encoding will impact backwards compatibility because it's stored in the
+     * metadata datasets and might be returned to users through public APIs.
+     */
+    public String getCanonicalForm() {
+        return canonicalForm;
+    }
+
+    /**
+     * Returns a new list containing dataverse name parts
+     */
+    public List<String> getParts() {
+        List<String> parts = new ArrayList<>(isMultiPart ? 4 : 1);
+        getParts(parts);
+        return parts;
+    }
+
+    /**
+     * Appends dataverse name parts into a given list
+     */
+    public void getParts(Collection<? super String> outParts) {
+        if (isMultiPart) {
+            decodeCanonicalForm(canonicalForm, DataverseName::addPartToCollection, outParts);
+        } else {
+            outParts.add(decodeSinglePartNameFromCanonicalForm(canonicalForm));
+        }
+    }
+
+    /**
+     * Returns a display form which is a {@link #SEPARATOR_CHAR '.'} separated concatenation of name parts without
+     * escaping. In general it's impossible to reconstruct a dataverse name from its display form, so this method
+     * should not be used when roundtripability is required.
+     */
+    @Override
+    public String toString() {
+        return getDisplayForm();
+    }
+
+    private String getDisplayForm() {
+        String result = displayForm;
+        if (result == null) {
+            displayForm = result = createDisplayForm();
+        }
+        return result;
+    }
+
+    private String createDisplayForm() {
+        if (isMultiPart) {
+            StringBuilder displayForm = new StringBuilder(canonicalForm.length() + 1);
+            decodeCanonicalForm(canonicalForm, DataverseName::addPartToDisplayForm, displayForm);
+            return displayForm.substring(0, displayForm.length() - 1); // remove last separator char
+        } else {
+            return decodeSinglePartNameFromCanonicalForm(canonicalForm);
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return canonicalForm.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof DataverseName)) {
+            return false;
+        }
+        DataverseName that = (DataverseName) obj;
+        return canonicalForm.equals(that.canonicalForm);
+    }
+
+    @Override
+    public int compareTo(DataverseName that) {
+        return canonicalForm.compareTo(that.canonicalForm);
+    }
+
+    /**
+     * Creates a new dataverse name from a given list of name parts.
+     * Equivalent to {@code create(parts, 0, parts.size())}.
+     */
+    public static DataverseName create(List<String> parts) {
+        return create(parts, 0, parts.size());
+    }
+
+    /**
+     * Creates a new dataverse name from a given list of name parts.
+     *
+     * @param parts
+     *            list of name parts
+     * @param fromIndex
+     *            index to start from
+     * @param toIndex
+     *            index to stop at (exclusive, value at that index is not used)
+     */
+    public static DataverseName create(List<String> parts, int fromIndex, int toIndex) {
+        int partCount = toIndex - fromIndex;
+        return partCount == 1 ? createSinglePartName(parts.get(fromIndex))
+                : createMultiPartName(parts, fromIndex, toIndex);
+    }
+
+    /**
+     * Creates a new dataverse name from its scalar encoding (canonical form) returned by {@link #getCanonicalForm()}
+     */
+    public static DataverseName createFromCanonicalForm(String canonicalForm) {
+        boolean isMultiPart = isMultiPartCanonicalForm(canonicalForm);
+        return new DataverseName(canonicalForm, isMultiPart);
+    }
+
+    /**
+     * Creates a single-part dataverse name.
+     * Equivalent to {@code create(Collections.singletonList(singlePart))}, but performs faster.
+     */
+    public static DataverseName createSinglePartName(String singlePart) {
+        String canonicalForm = encodeSinglePartNamePartIntoCanonicalForm(singlePart);
+        return new DataverseName(canonicalForm, false);
+    }
+
+    /**
+     * Creates a new dataverse name for a built-in dataverse.
+     * Validates that the canonical form of the created dataverse name is the same as its given single name part.
+     */
+    public static DataverseName createBuiltinDataverseName(String singlePart) {
+        if (StringUtils.containsAny(singlePart, SEPARATOR_AND_ESCAPE_CHARS)) {
+            throw new IllegalArgumentException(singlePart);
+        }
+        DataverseName dataverseName = createSinglePartName(singlePart); // 1-part name
+        String canonicalForm = dataverseName.getCanonicalForm();
+        if (!canonicalForm.equals(singlePart)) {
+            throw new IllegalStateException(canonicalForm + "!=" + singlePart);
+        }
+        return dataverseName;
+    }
+
+    private static DataverseName createMultiPartName(List<String> parts, int fromIndex, int toIndex) {
+        String canonicalForm = encodeMultiPartNameIntoCanonicalForm(parts, fromIndex, toIndex);
+        return new DataverseName(canonicalForm, true);
+    }
+
+    private static String encodeMultiPartNameIntoCanonicalForm(List<String> parts, int fromIndex, int toIndex) {
+        Objects.requireNonNull(parts);
+        int partCount = toIndex - fromIndex;
+        if (partCount <= 0) {
+            throw new IllegalArgumentException(fromIndex + "," + toIndex);
+        }
+        StringBuilder sb = new StringBuilder(32);
+        for (int i = 0; i < partCount; i++) {
+            if (i > 0) {
+                sb.append(SEPARATOR_CHAR);
+            }
+            encodePartIntoCanonicalForm(parts.get(fromIndex + i), sb);
+        }
+        return sb.toString();
+    }
+
+    private static String encodeSinglePartNamePartIntoCanonicalForm(String singlePart) {
+        if (StringUtils.indexOfAny(singlePart, SEPARATOR_AND_ESCAPE_CHARS) < 0) {
+            // no escaping needed
+            return singlePart;
+        }
+        StringBuilder sb = new StringBuilder(singlePart.length() + 4);
+        encodePartIntoCanonicalForm(singlePart, sb);
+        return sb.toString();
+    }
+
+    private static void encodePartIntoCanonicalForm(String part, StringBuilder out) {
+        for (int i = 0, ln = part.length(); i < ln; i++) {
+            char c = part.charAt(i);
+            if (c == SEPARATOR_CHAR || c == ESCAPE_CHAR) {
+                out.append(ESCAPE_CHAR);
+            }
+            out.append(c);
+        }
+    }
+
+    private static <T> void decodeCanonicalForm(String canonicalForm, BiConsumer<CharSequence, T> partConsumer,
+            T partConsumerArg) {
+        int ln = canonicalForm.length();
+        StringBuilder sb = new StringBuilder(ln);
+        for (int i = 0; i < ln; i++) {
+            char c = canonicalForm.charAt(i);
+            switch (c) {
+                case SEPARATOR_CHAR:
+                    partConsumer.accept(sb, partConsumerArg);
+                    sb.setLength(0);
+                    break;
+                case ESCAPE_CHAR:
+                    i++;
+                    c = canonicalForm.charAt(i);
+                    // fall through to 'default'
+                default:
+                    sb.append(c);
+                    break;
+            }
+        }
+        if (sb.length() > 0) {
+            partConsumer.accept(sb, partConsumerArg);
+        }
+    }
+
+    // optimization for a single part name
+    private String decodeSinglePartNameFromCanonicalForm(String canonicalForm) {
+        if (canonicalForm.indexOf(ESCAPE_CHAR) < 0) {
+            // no escaping was done
+            return canonicalForm;
+        }
+
+        StringBuilder singlePart = new StringBuilder(canonicalForm.length());
+        for (int i = 0, ln = canonicalForm.length(); i < ln; i++) {
+            char c = canonicalForm.charAt(i);
+            switch (c) {
+                case SEPARATOR_CHAR:
+                    throw new IllegalStateException(canonicalForm); // should never happen
+                case ESCAPE_CHAR:
+                    i++;
+                    c = canonicalForm.charAt(i);
+                    // fall through to 'default'
+                default:
+                    singlePart.append(c);
+                    break;
+            }
+        }
+        return singlePart.toString();
+    }
+
+    private static boolean isMultiPartCanonicalForm(String canonicalForm) {
+        for (int i = 0, ln = canonicalForm.length(); i < ln; i++) {
+            char c = canonicalForm.charAt(i);
+            switch (c) {
+                case SEPARATOR_CHAR:
+                    return true;
+                case ESCAPE_CHAR:
+                    i++;
+                    break;
+            }
+        }
+        return false;
+    }
+
+    private static void addPartToCollection(CharSequence part, Collection<? super String> out) {
+        out.add(part.toString());
+    }
+
+    private static void addPartToDisplayForm(CharSequence part, StringBuilder out) {
+        out.append(part).append(SEPARATOR_CHAR);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
index 276e294..55c5b97 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IDataset.java
@@ -25,7 +25,7 @@ public interface IDataset {
     /**
      * @return the dataverse name
      */
-    String getDataverseName();
+    DataverseName getDataverseName();
 
     /**
      * @return the dataset name
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java
index ebae47e..1f77aa0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLock.java
@@ -49,6 +49,14 @@ public interface IMetadataLock {
         }
     }
 
+    interface LockKey {
+        @Override
+        boolean equals(Object o);
+
+        @Override
+        int hashCode();
+    }
+
     /**
      * Acquire a lock
      *
@@ -68,9 +76,9 @@ public interface IMetadataLock {
     /**
      * Get the lock's key
      *
-     * @return the key identiying the lock
+     * @return the key identifying the lock
      */
-    String getKey();
+    LockKey getKey();
 
     /**
      * upgrade the lock
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java
index 6f2bc39..43a1849 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/LockList.java
@@ -21,6 +21,7 @@ package org.apache.asterix.common.metadata;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
@@ -33,7 +34,7 @@ import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
  */
 public class LockList {
     private final List<MutablePair<IMetadataLock, IMetadataLock.Mode>> locks = new ArrayList<>();
-    private final HashMap<String, Integer> indexes = new HashMap<>();
+    private final Map<IMetadataLock.LockKey, Integer> indexes = new HashMap<>();
     private boolean lockPhase = true;
 
     /**
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetCopyIdentifier.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetCopyIdentifier.java
index 6500c8a..bf72c19 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetCopyIdentifier.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetCopyIdentifier.java
@@ -21,20 +21,22 @@ package org.apache.asterix.common.storage;
 import java.io.Serializable;
 import java.util.Objects;
 
+import org.apache.asterix.common.metadata.DataverseName;
+
 public class DatasetCopyIdentifier implements Serializable {
 
     private static final long serialVersionUID = 1L;
+    private final DataverseName dataverse;
     private final String dataset;
-    private final String dataverse;
     private final String rebalance;
 
-    private DatasetCopyIdentifier(String dataverse, String datasetName, String rebalance) {
+    private DatasetCopyIdentifier(DataverseName dataverse, String datasetName, String rebalance) {
         this.dataverse = dataverse;
         this.dataset = datasetName;
         this.rebalance = rebalance;
     }
 
-    public static DatasetCopyIdentifier of(String dataverse, String datasetName, String rebalance) {
+    public static DatasetCopyIdentifier of(DataverseName dataverse, String datasetName, String rebalance) {
         return new DatasetCopyIdentifier(dataverse, datasetName, rebalance);
     }
 
@@ -64,13 +66,13 @@ public class DatasetCopyIdentifier implements Serializable {
         return Objects.hash(dataverse, dataset, rebalance);
     }
 
-    public String getDataverse() {
+    public DataverseName getDataverse() {
         return dataverse;
     }
 
     public boolean isMatch(ResourceReference resourceReference) {
-        return resourceReference.getDataverse().equals(dataverse) && resourceReference.getDataset().equals(dataset)
-                && resourceReference.getRebalance().equals(rebalance);
+        return resourceReference.getDataverse().equals(dataverse.getCanonicalForm())
+                && resourceReference.getDataset().equals(dataset) && resourceReference.getRebalance().equals(rebalance);
     }
 
     @Override
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
index ae949fe..9d934e4 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
@@ -29,7 +29,7 @@ public class ResourceReference {
 
     protected String root;
     protected String partition;
-    protected String dataverse;
+    protected String dataverse; // == DataverseName.getCanonicalForm()
     protected String dataset;
     protected String rebalance;
     protected String index;
@@ -52,7 +52,7 @@ public class ResourceReference {
         return partition;
     }
 
-    public String getDataverse() {
+    public String getDataverse() { //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
         return dataverse;
     }
 
@@ -96,7 +96,7 @@ public class ResourceReference {
         ref.index = tokens[--offset];
         ref.rebalance = tokens[--offset];
         ref.dataset = tokens[--offset];
-        ref.dataverse = tokens[--offset];
+        ref.dataverse = tokens[--offset]; //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
         ref.partition = tokens[--offset];
         ref.root = tokens[--offset];
     }
@@ -105,10 +105,6 @@ public class ResourceReference {
         return Integer.parseInt(partition.substring(StorageConstants.PARTITION_DIR_PREFIX.length()));
     }
 
-    public String getDatasetFullyQualifiedName() {
-        return dataverse + '.' + dataset;
-    }
-
     @Override
     public boolean equals(Object o) {
         if (this == o) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index aa2c7af..587b8b3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -23,6 +23,7 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 
 import org.apache.asterix.common.cluster.ClusterPartition;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.storage.ResourceReference;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -63,13 +64,13 @@ public class StoragePathUtil {
                 .toString();
     }
 
-    public static String prepareDataverseIndexName(String dataverseName, String datasetName, String idxName,
+    public static String prepareDataverseIndexName(DataverseName dataverseName, String datasetName, String idxName,
             long rebalanceCount) {
         return prepareDataverseIndexName(dataverseName, prepareFullIndexName(datasetName, idxName, rebalanceCount));
     }
 
-    public static String prepareDataverseIndexName(String dataverseName, String fullIndexName) {
-        return dataverseName + File.separator + fullIndexName;
+    public static String prepareDataverseIndexName(DataverseName dataverseName, String fullIndexName) {
+        return dataverseName.getCanonicalForm() + File.separator + fullIndexName; //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
     }
 
     private static String prepareFullIndexName(String datasetName, String idxName, long rebalanceCount) {
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/common/metadata/DataverseNameTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/common/metadata/DataverseNameTest.java
new file mode 100644
index 0000000..1a47a3f
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/common/metadata/DataverseNameTest.java
@@ -0,0 +1,244 @@
+/*
+ * 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.metadata;
+
+import static org.apache.asterix.common.functions.FunctionConstants.ASTERIX_NS;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Supplier;
+
+import org.apache.commons.collections4.ListUtils;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Unit test for {@link DataverseName}
+ */
+public class DataverseNameTest {
+
+    private static final List<String> TEST_BUILTIN_DATAVERSE_NAME_PARAMS = Arrays.asList(
+            // 1-part-name
+            // default dataverse
+            "Default",
+            // metadata dataverse
+            "Metadata",
+            // dataverse for Algebricks functions
+            AlgebricksBuiltinFunctions.ALGEBRICKS_NS,
+            // dataverse for Asterix functions
+            ASTERIX_NS);
+
+    private static final List<String> TEST_BUILTIN_DATAVERSE_INVALID_NAME_PARAMS = Arrays.asList(
+            // separator character is not allowed
+            "a.b",
+            // escape character is not allowed
+            "c@d");
+
+    private static final List<Pair<String, String>> TEST_SINGLE_PART_NAME_PARAMS = Arrays.asList(
+            // <1-part-name, canonical-form>
+            new Pair<>("abc", "abc"),
+            // with escape character
+            new Pair<>("a@b", "a@@b"),
+            // with separator character
+            new Pair<>("a.b", "a@.b"),
+            // with both escape and separator characters
+            new Pair<>("a@.b", "a@@@.b"));
+
+    private static final List<Pair<List<String>, String>> TEST_MULTI_PART_NAME_PARAMS = Arrays.asList(
+            // <multi-part-name, canonical-form>
+            new Pair<>(Arrays.asList("aa", "bb", "cc"), "aa.bb.cc"),
+            // with escape character
+            new Pair<>(Arrays.asList("a@a@", "@b@b", "@c@c"), "a@@a@@.@@b@@b.@@c@@c"),
+            // with separator character
+            new Pair<>(Arrays.asList("a.a.", ".b.b.", ".c.c"), "a@.a@..@.b@.b@..@.c@.c"),
+            // with both escape and separator characters
+            new Pair<>(Arrays.asList("a@a.", "@b.b@", ".c@c"), "a@@a@..@@b@.b@@.@.c@@c"),
+            // with both escape and separator characters repeated
+            new Pair<>(Arrays.asList("a@@a..", "@@b..b@@", "..c@@c"), "a@@@@a@.@..@@@@b@.@.b@@@@.@.@.c@@@@c"));
+
+    @Test
+    public void testBuiltinDataverseName() {
+        for (String p : TEST_BUILTIN_DATAVERSE_NAME_PARAMS) {
+            testBuiltinDataverseNameImpl(p);
+        }
+    }
+
+    @Test
+    public void testSinglePartName() {
+        for (Pair<String, String> p : TEST_SINGLE_PART_NAME_PARAMS) {
+            String singlePart = p.first;
+            String expectedCanonicalForm = p.second;
+            testSinglePartNameImpl(singlePart, expectedCanonicalForm);
+        }
+    }
+
+    @Test
+    public void testMultiPartName() {
+        // test single part names
+        for (Pair<String, String> p : TEST_SINGLE_PART_NAME_PARAMS) {
+            List<String> parts = Collections.singletonList(p.first);
+            String expectedCanonicalForm = p.second;
+            testMultiPartNameImpl(parts, expectedCanonicalForm);
+        }
+        // test multi part names
+        for (Pair<List<String>, String> p : TEST_MULTI_PART_NAME_PARAMS) {
+            List<String> parts = p.first;
+            String expectedCanonicalForm = p.second;
+            testMultiPartNameImpl(parts, expectedCanonicalForm);
+        }
+    }
+
+    private void testBuiltinDataverseNameImpl(String singlePart) {
+        DataverseName dvBuiltin = DataverseName.createBuiltinDataverseName(singlePart);
+        DataverseName dv = DataverseName.createSinglePartName(singlePart);
+        Assert.assertEquals("same-builtin", dv, dvBuiltin);
+        // part = canonical-form = persistent-form for builtins
+        testSinglePartNameImpl(singlePart, singlePart);
+    }
+
+    private void testSinglePartNameImpl(String singlePart, String expectedCanonicalForm) {
+        List<String> parts = Collections.singletonList(singlePart);
+
+        // construction using createSinglePartName()
+        DataverseName dvConstr1 = DataverseName.createSinglePartName(singlePart);
+        testDataverseNameImpl(dvConstr1, parts, expectedCanonicalForm);
+
+        // construction using create(list)
+        DataverseName dvConstr2 = DataverseName.create(Collections.singletonList(singlePart));
+        testDataverseNameImpl(dvConstr2, parts, expectedCanonicalForm);
+
+        // construction using create(list, from, to)
+        DataverseName dvConstr3 = DataverseName.create(Arrays.asList(null, null, singlePart, null, null), 2, 3);
+        testDataverseNameImpl(dvConstr3, parts, expectedCanonicalForm);
+    }
+
+    private void testMultiPartNameImpl(List<String> parts, String expectedCanonicalForm) {
+        // construction using create(list)
+        DataverseName dvConstr1 = DataverseName.create(parts);
+        testDataverseNameImpl(dvConstr1, parts, expectedCanonicalForm);
+
+        // construction using create(list, from, to)
+        List<String> dv2InputParts =
+                ListUtils.union(ListUtils.union(Arrays.asList(null, null), parts), Arrays.asList(null, null));
+        DataverseName dvConstr2 = DataverseName.create(dv2InputParts, 2, 2 + parts.size());
+        testDataverseNameImpl(dvConstr2, parts, expectedCanonicalForm);
+    }
+
+    private void testDataverseNameImpl(DataverseName dataverseName, List<String> parts, String expectedCanonicalForm) {
+        boolean isMultiPart = parts.size() > 1;
+        Assert.assertEquals("is-multipart", isMultiPart, dataverseName.isMultiPart());
+
+        // test getParts()
+        Assert.assertArrayEquals("get-parts-0", parts.toArray(), dataverseName.getParts().toArray());
+        List<String> outParts = new ArrayList<>();
+        dataverseName.getParts(outParts);
+        Assert.assertArrayEquals("get-parts-1", parts.toArray(), outParts.toArray());
+
+        // test canonical form
+        Assert.assertEquals("canonical-form", expectedCanonicalForm, dataverseName.getCanonicalForm());
+        DataverseName dvFromCanonical = DataverseName.createFromCanonicalForm(expectedCanonicalForm);
+        Assert.assertEquals("canonical-form-round-trip", dataverseName, dvFromCanonical);
+        Assert.assertEquals("canonical-form-round-trip-cmp", 0, dataverseName.compareTo(dvFromCanonical));
+        Assert.assertEquals("canonical-form-round-trip-hash", dataverseName.hashCode(), dvFromCanonical.hashCode());
+
+        // test display form
+        String expectedDisplayForm = String.join(".", parts);
+        Assert.assertEquals("display-form", expectedDisplayForm, dataverseName.toString());
+    }
+
+    @Test
+    public void testCompare() {
+        List<DataverseName> dvList =
+                Arrays.asList(DataverseName.createSinglePartName("a"), DataverseName.create(Arrays.asList("a", "a")),
+                        DataverseName.createSinglePartName("aa"), DataverseName.createSinglePartName("b"));
+
+        for (int i = 0; i < dvList.size() - 1; i++) {
+            for (int j = i + 1; j < dvList.size(); j++) {
+                testCompareImpl(dvList.get(i), dvList.get(j));
+            }
+        }
+    }
+
+    private void testCompareImpl(DataverseName left, DataverseName right) {
+        String label = left.getCanonicalForm() + " ? " + right.getCanonicalForm();
+        Assert.assertNotEquals(left, right);
+        Assert.assertTrue(label, left.compareTo(right) < 0);
+        Assert.assertTrue(label, right.compareTo(left) > 0);
+    }
+
+    @Test
+    public void testExceptions() {
+        // 1. Invalid names for builtin dataverses
+        for (String p : TEST_BUILTIN_DATAVERSE_INVALID_NAME_PARAMS) {
+            testInvalidBuiltinDataverseNameImpl(p);
+        }
+        // 2. NullPointerException
+        testRuntimeException(() -> DataverseName.create(null), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.create(null, 0, 0), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.create(null, 0, 1), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.create(null, 0, 2), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.createSinglePartName(null), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.createBuiltinDataverseName(null), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.createFromCanonicalForm(null), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.create(Collections.singletonList(null)), NullPointerException.class);
+        testRuntimeException(() -> DataverseName.create(Arrays.asList(null, null)), NullPointerException.class);
+        // 3. IndexOutOfBoundsException
+        testRuntimeException(() -> DataverseName.create(Collections.emptyList(), 0, 1),
+                IndexOutOfBoundsException.class);
+        testRuntimeException(() -> DataverseName.create(Collections.emptyList(), 0, 2),
+                IndexOutOfBoundsException.class);
+        // 4. IllegalArgumentException
+        testRuntimeException(() -> DataverseName.create(Collections.emptyList()), IllegalArgumentException.class);
+        testRuntimeException(() -> DataverseName.create(Collections.emptyList(), 0, 0), IllegalArgumentException.class);
+        testRuntimeException(() -> DataverseName.create(Arrays.asList("a", "b", "c"), 2, 1),
+                IllegalArgumentException.class);
+    }
+
+    private <E extends RuntimeException> void testRuntimeException(Supplier<DataverseName> supplier,
+            Class<E> exceptionClass) {
+        try {
+            supplier.get();
+            Assert.fail("Did not get expected exception " + exceptionClass.getName());
+        } catch (RuntimeException e) {
+            if (!exceptionClass.isInstance(e)) {
+                try {
+                    Assert.fail(
+                            "Expected to catch " + exceptionClass.getName() + ", but caught " + e.getClass().getName());
+                } catch (AssertionError ae) {
+                    ae.initCause(e);
+                    throw ae;
+                }
+            }
+        }
+    }
+
+    private void testInvalidBuiltinDataverseNameImpl(String singlePart) {
+        try {
+            DataverseName.createBuiltinDataverseName(singlePart);
+            Assert.fail(singlePart);
+        } catch (IllegalArgumentException e) {
+            // this error is expected
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AdapterIdentifier.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AdapterIdentifier.java
index 9a466d6..9f27b41 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AdapterIdentifier.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/AdapterIdentifier.java
@@ -20,22 +20,24 @@ package org.apache.asterix.external.dataset.adapter;
 
 import java.io.Serializable;
 
+import org.apache.asterix.common.metadata.DataverseName;
+
 /**
  * A unique identifier for a data source adapter.
  */
 public class AdapterIdentifier implements Serializable {
 
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
-    private final String dataverseName;
+    private final DataverseName dataverseName;
     private final String adapterName;
 
-    public AdapterIdentifier(String namespace, String name) {
-        this.dataverseName = namespace;
+    public AdapterIdentifier(DataverseName dataverse, String name) {
+        this.dataverseName = dataverse;
         this.adapterName = name;
     }
 
-    public String getNamespace() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
@@ -61,6 +63,6 @@ public class AdapterIdentifier implements Serializable {
             return false;
         }
         AdapterIdentifier a = (AdapterIdentifier) o;
-        return dataverseName.equals(a.getNamespace()) && adapterName.equals(a.getName());
+        return dataverseName.equals(a.dataverseName) && adapterName.equals(a.adapterName);
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
index 2acf394..dd1d1b7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeed.java
@@ -22,12 +22,13 @@ import java.io.Serializable;
 import java.util.Map;
 
 import org.apache.asterix.active.EntityId;
+import org.apache.asterix.common.metadata.DataverseName;
 
 public interface IFeed extends Serializable {
 
     public String getFeedName();
 
-    public String getDataverseName();
+    public DataverseName getDataverseName();
 
     public EntityId getFeedId();
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
index fab5a8f..73edc6e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
@@ -21,6 +21,7 @@ package org.apache.asterix.external.feed.management;
 import java.io.Serializable;
 
 import org.apache.asterix.active.EntityId;
+import org.apache.asterix.common.metadata.DataverseName;
 
 /**
  * A unique identifier for a feed connection. A feed connection is an instance of a data feed that is flowing into a
@@ -41,8 +42,8 @@ public class FeedConnectionId implements Serializable {
         this.hash = toString().hashCode();
     }
 
-    public FeedConnectionId(String dataverse, String feedName, String datasetName) {
-        this(new EntityId(FEED_EXTENSION_NAME, dataverse, feedName), datasetName);
+    public FeedConnectionId(DataverseName dataverseName, String feedName, String datasetName) {
+        this(new EntityId(FEED_EXTENSION_NAME, dataverseName, feedName), datasetName);
     }
 
     public EntityId getFeedId() {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/policy/FeedPolicy.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/policy/FeedPolicy.java
index da5907c..79c5281 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/policy/FeedPolicy.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/policy/FeedPolicy.java
@@ -21,10 +21,12 @@ package org.apache.asterix.external.feed.policy;
 import java.io.Serializable;
 import java.util.Map;
 
+import org.apache.asterix.common.metadata.DataverseName;
+
 public class FeedPolicy implements Serializable {
 
-    private static final long serialVersionUID = 1L;
-    private final String dataverseName;
+    private static final long serialVersionUID = 2L;
+    private final DataverseName dataverseName;
     // Enforced to be unique within a dataverse.
     private final String policyName;
     // A description of the policy
@@ -32,14 +34,15 @@ public class FeedPolicy implements Serializable {
     // The policy properties associated with the feed dataset
     private Map<String, String> properties;
 
-    public FeedPolicy(String dataverseName, String policyName, String description, Map<String, String> properties) {
+    public FeedPolicy(DataverseName dataverseName, String policyName, String description,
+            Map<String, String> properties) {
         this.dataverseName = dataverseName;
         this.policyName = policyName;
         this.description = description;
         this.properties = properties;
     }
 
-    public String getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
index 8fbab3c..26a3a76 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFile.java
@@ -24,6 +24,7 @@ import java.util.Date;
 import java.util.Objects;
 
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import org.apache.asterix.common.metadata.DataverseName;
 
 public class ExternalFile implements Serializable, Comparable<ExternalFile> {
 
@@ -33,7 +34,7 @@ public class ExternalFile implements Serializable, Comparable<ExternalFile> {
      */
     private static final long serialVersionUID = 1L;
 
-    private String dataverseName;
+    private DataverseName dataverseName;
     private String datasetName;
     private Date lastModefiedTime;
     private long size;
@@ -42,7 +43,7 @@ public class ExternalFile implements Serializable, Comparable<ExternalFile> {
     private ExternalFilePendingOp pendingOp;
 
     public ExternalFile() {
-        this.dataverseName = "";
+        this.dataverseName = null;
         this.datasetName = "";
         this.fileNumber = -1;
         this.fileName = "";
@@ -51,7 +52,7 @@ public class ExternalFile implements Serializable, Comparable<ExternalFile> {
         this.pendingOp = ExternalFilePendingOp.NO_OP;
     }
 
-    public ExternalFile(String dataverseName, String datasetName, int fileNumber, String fileName,
+    public ExternalFile(DataverseName dataverseName, String datasetName, int fileNumber, String fileName,
             Date lastModefiedTime, long size, ExternalFilePendingOp pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
@@ -62,11 +63,11 @@ public class ExternalFile implements Serializable, Comparable<ExternalFile> {
         this.setPendingOp(pendingOp);
     }
 
-    public String getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public void setDataverseName(String dataverseName) {
+    public void setDataverseName(DataverseName dataverseName) {
         this.dataverseName = dataverseName;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
index d8196b4..f3dc0fe 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalFunction.java
@@ -23,7 +23,9 @@ import java.io.IOException;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.api.IExternalFunction;
 import org.apache.asterix.external.api.IFunctionFactory;
 import org.apache.asterix.external.api.IFunctionHelper;
@@ -61,7 +63,7 @@ public abstract class ExternalFunction implements IExternalFunction {
         ILibraryManager libraryManager = appCtx.getLibraryManager();
         String[] fnameComponents = finfo.getFunctionIdentifier().getName().split("#");
         String functionLibary = fnameComponents[0];
-        String dataverse = finfo.getFunctionIdentifier().getNamespace();
+        DataverseName dataverse = FunctionSignature.getDataverseName(finfo.getFunctionIdentifier());
 
         functionHelper = new JavaFunctionHelper(finfo, resultBuffer,
                 libraryManager.getFunctionParameters(dataverse, finfo.getFunctionIdentifier().getName()));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index 60c8bfd..3425b13 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -40,7 +41,7 @@ public class ExternalLibraryManager implements ILibraryManager {
     private static final Logger LOGGER = LogManager.getLogger();
 
     @Override
-    public void registerLibraryClassLoader(String dataverseName, String libraryName, URLClassLoader classLoader)
+    public void registerLibraryClassLoader(DataverseName dataverseName, String libraryName, URLClassLoader classLoader)
             throws RuntimeDataException {
         String key = getKey(dataverseName, libraryName);
         synchronized (libraryClassLoaders) {
@@ -61,7 +62,7 @@ public class ExternalLibraryManager implements ILibraryManager {
     }
 
     @Override
-    public void deregisterLibraryClassLoader(String dataverseName, String libraryName) {
+    public void deregisterLibraryClassLoader(DataverseName dataverseName, String libraryName) {
         String key = getKey(dataverseName, libraryName);
         synchronized (libraryClassLoaders) {
             URLClassLoader cl = libraryClassLoaders.get(key);
@@ -77,23 +78,23 @@ public class ExternalLibraryManager implements ILibraryManager {
     }
 
     @Override
-    public ClassLoader getLibraryClassLoader(String dataverseName, String libraryName) {
+    public ClassLoader getLibraryClassLoader(DataverseName dataverseName, String libraryName) {
         String key = getKey(dataverseName, libraryName);
         return libraryClassLoaders.get(key);
     }
 
     @Override
-    public void addFunctionParameters(String dataverseName, String fullFunctionName, List<String> parameters) {
+    public void addFunctionParameters(DataverseName dataverseName, String fullFunctionName, List<String> parameters) {
         externalFunctionParameters.put(dataverseName + "." + fullFunctionName, parameters);
     }
 
     @Override
-    public List<String> getFunctionParameters(String dataverseName, String fullFunctionName) {
+    public List<String> getFunctionParameters(DataverseName dataverseName, String fullFunctionName) {
         return externalFunctionParameters.getOrDefault(dataverseName + "." + fullFunctionName, Collections.emptyList());
     }
 
-    private static String getKey(String dataverseName, String libraryName) {
-        return dataverseName + "." + libraryName;
+    private static String getKey(DataverseName dataverseName, String libraryName) {
+        return dataverseName + "." + libraryName; //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
     }
 
     private static Pair<String, String> getDataverseAndLibararyName(String key) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
index 7a0341a..7c61653 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
@@ -105,7 +105,7 @@ public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperator
         INcApplicationContext runtimeCtx =
                 (INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext();
         ILibraryManager libraryManager = runtimeCtx.getLibraryManager();
-        ClassLoader classLoader = libraryManager.getLibraryClassLoader(feedId.getDataverse(), adaptorLibraryName);
+        ClassLoader classLoader = libraryManager.getLibraryClassLoader(feedId.getDataverseName(), adaptorLibraryName);
         if (classLoader != null) {
             try {
                 adapterFactory = (IAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance());
@@ -117,7 +117,7 @@ public class FeedIntakeOperatorDescriptor extends AbstractSingleActivityOperator
         } else {
             RuntimeDataException err = new RuntimeDataException(
                     ErrorCode.OPERATORS_FEED_INTAKE_OPERATOR_DESCRIPTOR_CLASSLOADER_NOT_CONFIGURED, adaptorLibraryName,
-                    feedId.getDataverse());
+                    feedId.getDataverseName());
             LOGGER.error(err.getMessage());
             throw err;
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
index 8024dc4..ef8597a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
@@ -32,6 +32,7 @@ import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.api.IExternalDataSourceFactory;
 import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
 import org.apache.asterix.external.api.IInputStreamFactory;
@@ -67,7 +68,7 @@ public class DatasourceFactoryProvider {
             Map<String, String> configuration) throws HyracksDataException {
         IInputStreamFactory streamSourceFactory;
         if (ExternalDataUtils.isExternal(streamSource)) {
-            String dataverse = ExternalDataUtils.getDataverse(configuration);
+            DataverseName dataverse = ExternalDataUtils.getDataverse(configuration);
             streamSourceFactory =
                     ExternalDataUtils.createExternalInputStreamFactory(libraryManager, dataverse, streamSource);
         } else {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index a418cbf..93844d1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -25,6 +25,7 @@ import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.api.IDataParserFactory;
 import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
 import org.apache.asterix.external.api.IInputStreamFactory;
@@ -117,7 +118,7 @@ public class ExternalDataUtils {
                 && (aString.trim().length() > 1));
     }
 
-    public static ClassLoader getClassLoader(ILibraryManager libraryManager, String dataverse, String library) {
+    public static ClassLoader getClassLoader(ILibraryManager libraryManager, DataverseName dataverse, String library) {
         return libraryManager.getLibraryClassLoader(dataverse, library);
     }
 
@@ -129,8 +130,8 @@ public class ExternalDataUtils {
         return aString.trim().split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[1];
     }
 
-    public static IInputStreamFactory createExternalInputStreamFactory(ILibraryManager libraryManager, String dataverse,
-            String stream) throws HyracksDataException {
+    public static IInputStreamFactory createExternalInputStreamFactory(ILibraryManager libraryManager,
+            DataverseName dataverse, String stream) throws HyracksDataException {
         try {
             String libraryName = getLibraryName(stream);
             String className = getExternalClassName(stream);
@@ -141,8 +142,8 @@ public class ExternalDataUtils {
         }
     }
 
-    public static String getDataverse(Map<String, String> configuration) {
-        return configuration.get(ExternalDataConstants.KEY_DATAVERSE);
+    public static DataverseName getDataverse(Map<String, String> configuration) {
+        return DataverseName.createFromCanonicalForm(configuration.get(ExternalDataConstants.KEY_DATAVERSE));
     }
 
     public static String getRecordFormat(Map<String, String> configuration) {
@@ -220,7 +221,7 @@ public class ExternalDataUtils {
             throw new AsterixException("to use " + ExternalDataConstants.EXTERNAL + " reader, the parameter "
                     + ExternalDataConstants.KEY_READER_FACTORY + " must be specified.");
         }
-        String[] libraryAndFactory = readerFactory.split(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR);
+        String[] libraryAndFactory = readerFactory.split(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
         if (libraryAndFactory.length != 2) {
             throw new AsterixException("The parameter " + ExternalDataConstants.KEY_READER_FACTORY
                     + " must follow the format \"DataverseName.LibraryName#ReaderFactoryFullyQualifiedName\"");
@@ -230,8 +231,9 @@ public class ExternalDataUtils {
             throw new AsterixException("The parameter " + ExternalDataConstants.KEY_READER_FACTORY
                     + " must follow the format \"DataverseName.LibraryName#ReaderFactoryFullyQualifiedName\"");
         }
-
-        ClassLoader classLoader = libraryManager.getLibraryClassLoader(dataverseAndLibrary[0], dataverseAndLibrary[1]);
+        DataverseName dataverseName = DataverseName.createSinglePartName(dataverseAndLibrary[0]); //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
+        String libraryName = dataverseAndLibrary[1];
+        ClassLoader classLoader = libraryManager.getLibraryClassLoader(dataverseName, libraryName);
         try {
             return (IRecordReaderFactory<?>) classLoader.loadClass(libraryAndFactory[1]).newInstance();
         } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
@@ -239,8 +241,8 @@ public class ExternalDataUtils {
         }
     }
 
-    public static IDataParserFactory createExternalParserFactory(ILibraryManager libraryManager, String dataverse,
-            String parserFactoryName) throws AsterixException {
+    public static IDataParserFactory createExternalParserFactory(ILibraryManager libraryManager,
+            DataverseName dataverse, String parserFactoryName) throws AsterixException {
         try {
             String library = parserFactoryName.substring(0,
                     parserFactoryName.indexOf(ExternalDataConstants.EXTERNAL_LIBRARY_SEPARATOR));
@@ -262,11 +264,11 @@ public class ExternalDataUtils {
         }
     }
 
-    public static void prepareFeed(Map<String, String> configuration, String dataverseName, String feedName) {
+    public static void prepareFeed(Map<String, String> configuration, DataverseName dataverseName, String feedName) {
         if (!configuration.containsKey(ExternalDataConstants.KEY_IS_FEED)) {
             configuration.put(ExternalDataConstants.KEY_IS_FEED, ExternalDataConstants.TRUE);
         }
-        configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataverseName);
+        configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataverseName.getCanonicalForm());
         configuration.put(ExternalDataConstants.KEY_FEED_NAME, feedName);
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
index ecaced6..2110dee 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedUtils.java
@@ -33,6 +33,7 @@ import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.runtime.utils.RuntimeUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -83,13 +84,9 @@ public class FeedUtils {
     private FeedUtils() {
     }
 
-    private static String prepareDataverseFeedName(String dataverseName, String feedName) {
-        return dataverseName + File.separator + feedName;
-    }
-
-    public static FileSplit splitsForAdapter(String dataverseName, String feedName, String nodeName,
+    public static FileSplit splitsForAdapter(DataverseName dataverseName, String feedName, String nodeName,
             ClusterPartition partition) {
-        File relPathFile = new File(prepareDataverseFeedName(dataverseName, feedName));
+        String relPathFile = dataverseName.getCanonicalForm() + File.separator + feedName; //TODO(MULTI_PART_DATAVERSE_NAME):REVISIT
         String storagePartitionPath = StoragePathUtil.prepareStoragePartitionPath(partition.getPartitionId());
         // Note: feed adapter instances in a single node share the feed logger
         // format: 'storage dir name'/partition_#/dataverse/feed/node
@@ -97,8 +94,8 @@ public class FeedUtils {
         return StoragePathUtil.getFileSplitForClusterPartition(partition, f.getPath());
     }
 
-    public static FileSplit[] splitsForAdapter(ICcApplicationContext appCtx, String dataverseName, String feedName,
-            AlgebricksPartitionConstraint partitionConstraints) throws AsterixException {
+    public static FileSplit[] splitsForAdapter(ICcApplicationContext appCtx, DataverseName dataverseName,
+            String feedName, AlgebricksPartitionConstraint partitionConstraints) throws AsterixException {
         if (partitionConstraints.getPartitionConstraintType() == PartitionConstraintType.COUNT) {
             throw new AsterixException("Can't create file splits for adapter with count partitioning constraints");
         }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
index b6343b9..2c64ce3 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/InputHandlerTest.java
@@ -32,6 +32,7 @@ import java.util.concurrent.Future;
 import org.apache.asterix.active.ActiveRuntimeId;
 import org.apache.asterix.active.EntityId;
 import org.apache.asterix.common.memory.ConcurrentFramePool;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
@@ -59,7 +60,7 @@ public class InputHandlerTest {
     private static final int DEFAULT_FRAME_SIZE = 32768;
     private static final int NUM_FRAMES = 128;
     private static final long FEED_MEM_BUDGET = DEFAULT_FRAME_SIZE * NUM_FRAMES;
-    private static final String DATAVERSE = "dataverse";
+    private static final DataverseName DATAVERSE = DataverseName.createSinglePartName("dataverse");
     private static final String DATASET = "dataset";
     private static final String FEED = "feed";
     private static final String NODE_ID = "NodeId";
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
index 5ca91b6..af015b7 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlStatementRewriter.java
@@ -22,17 +22,18 @@ import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.aql.visitor.AqlDeleteRewriteVisitor;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class AqlStatementRewriter implements IStatementRewriter {
 
     @Override
-    public void rewrite(Statement stmt) throws CompilationException {
-        rewriteDeleteStatement(stmt);
+    public void rewrite(Statement stmt, MetadataProvider metadataProvider) throws CompilationException {
+        rewriteDeleteStatement(stmt, metadataProvider);
     }
 
-    private void rewriteDeleteStatement(Statement stmt) throws CompilationException {
+    private void rewriteDeleteStatement(Statement stmt, MetadataProvider metadataProvider) throws CompilationException {
         if (stmt != null) {
-            AqlDeleteRewriteVisitor visitor = new AqlDeleteRewriteVisitor();
+            AqlDeleteRewriteVisitor visitor = new AqlDeleteRewriteVisitor(metadataProvider);
             stmt.accept(visitor, null);
         }
     }
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AQLToSQLPPPrintVisitor.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AQLToSQLPPPrintVisitor.java
index 59dfbb5..35a3b45 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AQLToSQLPPPrintVisitor.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AQLToSQLPPPrintVisitor.java
@@ -30,7 +30,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.aql.clause.DistinctClause;
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
@@ -47,7 +47,6 @@ import org.apache.asterix.lang.common.clause.WhereClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.FieldAccessor;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
-import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.OperatorExpr;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.DataverseDecl;
@@ -57,6 +56,8 @@ import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.OperatorType;
 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.visitor.FormatPrintVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
@@ -264,7 +265,7 @@ public class AQLToSQLPPPrintVisitor extends FormatPrintVisitor implements IAQLVi
 
     @Override
     public Void visit(DataverseDecl dv, Integer step) throws CompilationException {
-        out.println(skip(step) + "use " + normalize(dv.getDataverseName().getValue()) + ";\n\n");
+        out.println(skip(step) + "use " + generateDataverseName(dv.getDataverseName()) + ";\n\n");
         return null;
     }
 
@@ -302,14 +303,17 @@ public class AQLToSQLPPPrintVisitor extends FormatPrintVisitor implements IAQLVi
 
     @Override
     public Void visit(CallExpr callExpr, Integer step) throws CompilationException {
-        FunctionSignature signature = callExpr.getFunctionSignature();
-        if (signature.getNamespace() != null && signature.getNamespace().equals("Metadata")
-                && signature.getName().equals("dataset") && signature.getArity() == 1) {
-            LiteralExpr expr = (LiteralExpr) callExpr.getExprList().get(0);
-            out.print(normalize(expr.getValue().getStringValue()));
+        if (FunctionUtil.isBuiltinDatasetFunction(callExpr.getFunctionSignature())) {
+            Pair<DataverseName, String> dataset = FunctionUtil.parseDatasetFunctionArguments(callExpr.getExprList(),
+                    null, callExpr.getSourceLocation(), ExpressionUtils::getStringLiteral);
+            if (dataset.first != null) {
+                out.print(generateDataverseName(dataset.first));
+                out.print(".");
+            }
+            out.print(normalize(dataset.second));
         } else {
             printHints(callExpr.getHints(), step);
-            out.print(generateFullName(callExpr.getFunctionSignature().getNamespace(),
+            out.print(generateFullName(callExpr.getFunctionSignature().getDataverseName(),
                     callExpr.getFunctionSignature().getName()) + "(");
             printDelimitedExpressions(callExpr.getExprList(), COMMA, step);
             out.print(")");
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
index 22ef0e3..e53caa2 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/visitor/AqlDeleteRewriteVisitor.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
 import org.apache.asterix.lang.aql.visitor.base.AbstractAqlAstVisitor;
@@ -34,20 +35,27 @@ import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.statement.DeleteStatement;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 
 public class AqlDeleteRewriteVisitor extends AbstractAqlAstVisitor<Void, Void> {
 
+    private final MetadataProvider metadataProvider;
+
+    public AqlDeleteRewriteVisitor(MetadataProvider metadataProvider) {
+        this.metadataProvider = metadataProvider;
+    }
+
     @Override
     public Void visit(DeleteStatement deleteStmt, Void visitArg) {
         List<Expression> arguments = new ArrayList<>();
-        Identifier dataverseName = deleteStmt.getDataverseName();
-        Identifier datasetName = deleteStmt.getDatasetName();
-        String arg = dataverseName == null ? datasetName.getValue()
-                : dataverseName.getValue() + "." + datasetName.getValue();
-        LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
-        arguments.add(argumentLiteral);
+        DataverseName dataverseName = deleteStmt.getDataverseName();
+        if (dataverseName == null) {
+            dataverseName = metadataProvider.getDefaultDataverseName();
+        }
+        String datasetName = deleteStmt.getDatasetName();
+        arguments.add(new LiteralExpr(new StringLiteral(dataverseName.getCanonicalForm())));
+        arguments.add(new LiteralExpr(new StringLiteral(datasetName)));
         CallExpr callExpression = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), arguments);
 
         List<Clause> clauseList = new ArrayList<>();
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index f8aa57c..87d0b93 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -63,6 +63,7 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.aql.clause.DistinctClause;
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
@@ -188,6 +189,8 @@ class AQLParser extends ScopeChecker implements IParser {
 
     private static final String INT_TYPE_NAME = "int";
 
+    private DataverseName defaultDataverse;
+
     private static class IndexParams {
       public IndexType type;
       public int gramLength;
@@ -199,7 +202,7 @@ class AQLParser extends ScopeChecker implements IParser {
     };
 
     private static class FunctionName {
-       public String dataverse = null;
+       public DataverseName dataverse = null;
        public String library = null;
        public String function = null;
        public String hint = null;
@@ -381,8 +384,8 @@ DataverseDecl DataverseDeclaration() throws ParseException:
 {
   <USE> <DATAVERSE> dvName = Identifier()
     {
-      defaultDataverse = dvName;
-      return new DataverseDecl(new Identifier(dvName));
+      defaultDataverse = DataverseName.createSinglePartName(dvName);
+      return new DataverseDecl(defaultDataverse);
     }
 }
 
@@ -417,7 +420,7 @@ Statement CreateStatement() throws ParseException:
 
 TypeDecl TypeSpecification(String hint, boolean dgen) throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   TypeExpression typeExpr = null;
 }
@@ -467,9 +470,9 @@ NodegroupDecl NodegroupSpecification() throws ParseException:
 
 DatasetDecl DatasetSpecification() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
-  Pair<Identifier,Identifier> typeComponents = null;
+  Pair<DataverseName,Identifier> typeComponents = null;
   String adapterName = null;
   Map<String,String> properties = null;
   FunctionSignature appliedFunction = null;
@@ -479,7 +482,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
   DatasetDecl dsetDecl = null;
   boolean autogenerated = false;
   Pair<Integer, List<String>> filterField = null;
-  Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
+  Pair<DataverseName,Identifier> metaTypeComponents = new Pair<DataverseName, Identifier>(null, null);
   RecordConstructor withRecord = null;
 }
 {
@@ -568,7 +571,7 @@ DatasetDecl DatasetSpecification() throws ParseException:
 RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
 {
   RefreshExternalDatasetStatement redss = new RefreshExternalDatasetStatement();
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   String datasetName = null;
 }
 {
@@ -587,7 +590,7 @@ CreateIndexStatement IndexSpecification() throws ParseException:
   CreateIndexStatement cis = new CreateIndexStatement();
   String indexName = null;
   boolean ifNotExists = false;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   Pair<Integer, Pair<List<String>, IndexedTypeExpression>> fieldPair = null;
   IndexParams indexType = null;
   boolean enforced = false;
@@ -702,7 +705,7 @@ CreateDataverseStatement DataverseSpecification() throws ParseException :
   ifNotExists = IfNotExists()
   ( <WITH> <FORMAT> format = StringLiteral() )?
     {
-      return new CreateDataverseStatement(new Identifier(dvName), format, ifNotExists);
+      return new CreateDataverseStatement(DataverseName.createSinglePartName(dvName), format, ifNotExists);
     }
 }
 
@@ -742,10 +745,9 @@ CreateFunctionStatement FunctionSpecification() throws ParseException:
 
 CreateFeedStatement FeedSpecification() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   CreateFeedStatement cfs = null;
-  Pair<Identifier,Identifier> sourceNameComponents = null;
   RecordConstructor withRecord = null;
 }
 {
@@ -920,8 +922,8 @@ Pair<List<Integer>, List<List<String>>> PrimaryKey() throws ParseException:
 Statement DropStatement() throws ParseException:
 {
   String id = null;
-  Pair<Identifier,Identifier> pairId = null;
-  Triple<Identifier,Identifier,Identifier> tripleId = null;
+  Pair<DataverseName,Identifier> pairId = null;
+  Triple<DataverseName,Identifier,Identifier> tripleId = null;
   FunctionSignature funcSig = null;
   boolean ifExists = false;
   Statement stmt = null;
@@ -947,7 +949,7 @@ Statement DropStatement() throws ParseException:
       }
     | <DATAVERSE> id = Identifier() ifExists = IfExists()
       {
-        stmt = new DataverseDropStatement(new Identifier(id), ifExists);
+        stmt = new DataverseDropStatement(DataverseName.createSinglePartName(id), ifExists);
       }
     | <FUNCTION> funcSig = FunctionSignature() ifExists = IfExists()
       {
@@ -985,7 +987,7 @@ boolean IfExists() throws ParseException :
 
 InsertStatement InsertStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query;
   Expression returnExpression = null;
@@ -1009,7 +1011,7 @@ InsertStatement InsertStatement() throws ParseException:
 
 UpsertStatement UpsertStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query;
   Expression returnExpression = null;
@@ -1035,7 +1037,7 @@ DeleteStatement DeleteStatement() throws ParseException:
 {
   VariableExpr var = null;
   Expression condition = null;
-  Pair<Identifier, Identifier> nameComponents;
+  Pair<DataverseName, Identifier> nameComponents;
 }
 {
   <DELETE> var = Variable()
@@ -1131,12 +1133,12 @@ Statement WriteStatement() throws ParseException:
 
 LoadStatement LoadStatement() throws ParseException:
 {
-  Identifier dataverseName = null;
+  DataverseName dataverseName = null;
   Identifier datasetName = null;
   boolean alreadySorted = false;
   String adapterName;
   Map<String,String> properties;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
 }
 {
   <LOAD> <DATASET> nameComponents = QualifiedName()
@@ -1169,7 +1171,7 @@ String AdapterName() throws ParseException :
 
 Statement CompactStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   Statement stmt = null;
 }
 {
@@ -1184,8 +1186,8 @@ Statement CompactStatement() throws ParseException:
 
 Statement FeedStatement() throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
-  Pair<Identifier,Identifier> datasetNameComponents = null;
+  Pair<DataverseName,Identifier> feedNameComponents = null;
+  Pair<DataverseName,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
   List<FunctionSignature> appliedFunctions = new ArrayList<FunctionSignature>();
@@ -1383,7 +1385,7 @@ void RecordField(RecordTypeDefinition recType) throws ParseException:
 
 TypeReferenceExpression TypeReference() throws ParseException:
 {
-  Pair<Identifier,Identifier> id = null;
+  Pair<DataverseName,Identifier> id = null;
 }
 {
  id = QualifiedName()
@@ -1448,7 +1450,7 @@ FunctionName FunctionName() throws ParseException:
         result.function = first;
       } else if (third == null) {
         if (secondAfterDot) {
-          result.dataverse = first;
+          result.dataverse = DataverseName.createSinglePartName(first);
           result.library   = null;
           result.function = second;
         } else {
@@ -1457,7 +1459,7 @@ FunctionName FunctionName() throws ParseException:
           result.function = second;
         }
       } else {
-        result.dataverse = first;
+        result.dataverse = DataverseName.createSinglePartName(first);
         result.library   = second;
         result.function  = third;
       }
@@ -1470,15 +1472,15 @@ FunctionName FunctionName() throws ParseException:
 }
 
 
-Pair<Identifier,Identifier> TypeName() throws ParseException:
+Pair<DataverseName,Identifier> TypeName() throws ParseException:
 {
-  Pair<Identifier,Identifier> name = null;
+  Pair<DataverseName,Identifier> name = null;
 }
 {
   name = QualifiedName()
     {
       if (name.first == null) {
-        name.first = new Identifier(defaultDataverse);
+        name.first = defaultDataverse;
       }
       return name;
     }
@@ -1564,7 +1566,7 @@ String StringLiteral() throws ParseException:
     }
 }
 
-Pair<Identifier,Identifier> QualifiedName() throws ParseException:
+Pair<DataverseName,Identifier> QualifiedName() throws ParseException:
 {
   String first = null;
   String second = null;
@@ -1572,20 +1574,20 @@ Pair<Identifier,Identifier> QualifiedName() throws ParseException:
 {
   first = Identifier() (<DOT> second = Identifier())?
   {
-    Identifier id1 = null;
+    DataverseName id1 = null;
     Identifier id2 = null;
     if (second == null) {
       id2 = new Identifier(first);
     } else
     {
-      id1 = new Identifier(first);
+      id1 = DataverseName.createSinglePartName(first);
       id2 = new Identifier(second);
     }
-    return new Pair<Identifier,Identifier>(id1, id2);
+    return new Pair<DataverseName,Identifier>(id1, id2);
   }
 }
 
-Triple<Identifier,Identifier,Identifier> DoubleQualifiedName() throws ParseException:
+Triple<DataverseName,Identifier,Identifier> DoubleQualifiedName() throws ParseException:
 {
   String first = null;
   String second = null;
@@ -1594,18 +1596,18 @@ Triple<Identifier,Identifier,Identifier> DoubleQualifiedName() throws ParseExcep
 {
   first = Identifier() <DOT> second = Identifier() (<DOT> third = Identifier())?
   {
-    Identifier id1 = null;
+    DataverseName id1 = null;
     Identifier id2 = null;
     Identifier id3 = null;
     if (third == null) {
       id2 = new Identifier(first);
       id3 = new Identifier(second);
     } else {
-      id1 = new Identifier(first);
+      id1 = DataverseName.createSinglePartName(first);
       id2 = new Identifier(second);
       id3 = new Identifier(third);
     }
-    return new Triple<Identifier,Identifier,Identifier>(id1, id2, id3);
+    return new Triple<DataverseName,Identifier,Identifier>(id1, id2, id3);
   }
 }
 
@@ -2278,7 +2280,7 @@ Expression DatasetAccessExpression() throws ParseException:
     }
   | ( <LEFTPAREN> nameArg = Expression() <RIGHTPAREN> ) )
     {
-      String dataverse = MetadataConstants.METADATA_DATAVERSE_NAME;
+      DataverseName dataverse = MetadataConstants.METADATA_DATAVERSE_NAME;
       FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, 1);
       if (signature == null) {
         signature = new FunctionSignature(dataverse, funcName, 1);
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
index 0584665..e5fd574 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IStatementRewriter.java
@@ -19,14 +19,17 @@
 package org.apache.asterix.lang.common.base;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 
 public interface IStatementRewriter {
 
     /**
      * @param statement,
      *            a non-query statement.
+     * @param metadataProvider
+     *            a metadata provider
      */
-    void rewrite(Statement statement) throws CompilationException;
+    void rewrite(Statement statement, MetadataProvider metadataProvider) throws CompilationException;
 
     String toExternalVariableName(String statementParameterName);
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/FunctionSignatures.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/FunctionSignatures.java
index 376ab94..5a68180 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/FunctionSignatures.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/FunctionSignatures.java
@@ -22,16 +22,17 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 
 public class FunctionSignatures {
     private final Map<FunctionSignature, FunctionExpressionMap> functionMap;
 
     public FunctionSignatures() {
-        functionMap = new HashMap<FunctionSignature, FunctionExpressionMap>();
+        functionMap = new HashMap<>();
     }
 
-    public FunctionSignature get(String dataverse, String name, int arity) {
-        FunctionSignature fid = new FunctionSignature(dataverse, name, arity);
+    public FunctionSignature get(DataverseName dataverseName, String name, int arity) {
+        FunctionSignature fid = new FunctionSignature(dataverseName, name, arity);
         FunctionExpressionMap possibleFD = functionMap.get(fid);
         if (possibleFD == null) {
             return null;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
index 92be7c9..5037b44 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/context/Scope.java
@@ -29,6 +29,7 @@ import java.util.Set;
 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.metadata.DataverseName;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.parser.ScopeChecker;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -127,13 +128,13 @@ public final class Scope {
      *            # of arguments
      * @return FunctionDescriptor of the function found; otherwise null
      */
-    public FunctionSignature findFunctionSignature(String dataverse, String name, int arity) {
+    public FunctionSignature findFunctionSignature(DataverseName dataverseName, String name, int arity) {
         FunctionSignature fd = null;
         if (functionSignatures != null) {
-            fd = functionSignatures.get(dataverse, name, arity);
+            fd = functionSignatures.get(dataverseName, name, arity);
         }
         if (fd == null && parent != null) {
-            fd = parent.findFunctionSignature(dataverse, name, arity);
+            fd = parent.findFunctionSignature(dataverseName, name, arity);
         }
         return fd;
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
index a76e5bb..ea0b1b1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/TypeReferenceExpression.java
@@ -21,19 +21,20 @@ package org.apache.asterix.lang.common.expression;
 import java.util.Objects;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public class TypeReferenceExpression extends AbstractTypeExpression {
 
-    private final Pair<Identifier, Identifier> ident;
+    private final Pair<DataverseName, Identifier> ident;
 
-    public TypeReferenceExpression(Pair<Identifier, Identifier> ident) {
+    public TypeReferenceExpression(Pair<DataverseName, Identifier> ident) {
         this.ident = ident;
     }
 
-    public Pair<Identifier, Identifier> getIdent() {
+    public Pair<DataverseName, Identifier> getIdent() {
         return ident;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
index 58470e3..f5aa489 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/parser/ScopeChecker.java
@@ -22,6 +22,7 @@ import java.util.Set;
 import java.util.Stack;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.context.RootScopeFactory;
 import org.apache.asterix.lang.common.context.Scope;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -42,8 +43,6 @@ public class ScopeChecker {
 
     protected String[] inputLines;
 
-    protected String defaultDataverse;
-
     public ScopeChecker() {
         scopeStack.push(RootScopeFactory.createRootScope(this));
     }
@@ -139,9 +138,9 @@ public class ScopeChecker {
      *
      * @return functionDescriptor
      */
-    public final FunctionSignature lookupFunctionSignature(String dataverse, String name, int arity) {
-        if (dataverse != null) {
-            return getCurrentScope().findFunctionSignature(dataverse, name, arity);
+    public final FunctionSignature lookupFunctionSignature(DataverseName dataverseName, String name, int arity) {
+        if (dataverseName != null) {
+            return getCurrentScope().findFunctionSignature(dataverseName, name, arity);
         } else {
             return null;
         }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
index d35516b..4c79029 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CompactStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,10 +27,10 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class CompactStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier datasetName;
 
-    public CompactStatement(Identifier dataverseName, Identifier datasetName) {
+    public CompactStatement(DataverseName dataverseName, Identifier datasetName) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
     }
@@ -39,7 +40,7 @@ public class CompactStatement extends AbstractStatement {
         return Statement.Kind.COMPACT;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
index eaa95cd..90892d2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/ConnectFeedStatement.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.AbstractStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -31,7 +32,7 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public class ConnectFeedStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier datasetName;
     private final String feedName;
     private final String policy;
@@ -39,14 +40,14 @@ public class ConnectFeedStatement extends AbstractStatement {
     private int varCounter;
     private final List<FunctionSignature> appliedFunctions;
 
-    public ConnectFeedStatement(Pair<Identifier, Identifier> feedNameCmp, Pair<Identifier, Identifier> datasetNameCmp,
-            List<FunctionSignature> appliedFunctions, String policy, String whereClauseBody, int varCounter) {
+    public ConnectFeedStatement(Pair<DataverseName, Identifier> feedNameCmp,
+            Pair<DataverseName, Identifier> datasetNameCmp, List<FunctionSignature> appliedFunctions, String policy,
+            String whereClauseBody, int varCounter) {
         if (feedNameCmp.first != null && datasetNameCmp.first != null
-                && !feedNameCmp.first.getValue().equals(datasetNameCmp.first.getValue())) {
+                && !feedNameCmp.first.equals(datasetNameCmp.first)) {
             throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
         }
-        this.dataverseName = feedNameCmp.first != null ? feedNameCmp.first
-                : datasetNameCmp.first != null ? datasetNameCmp.first : null;
+        this.dataverseName = feedNameCmp.first != null ? feedNameCmp.first : datasetNameCmp.first;
         this.datasetName = datasetNameCmp.second;
         this.feedName = feedNameCmp.second.getValue();
         this.policy = policy != null ? policy : BuiltinFeedPolicies.DEFAULT_POLICY.getPolicyName();
@@ -55,7 +56,7 @@ public class ConnectFeedStatement extends AbstractStatement {
         this.appliedFunctions = appliedFunctions;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
index 9881f83..2276755 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateDataverseStatement.java
@@ -19,25 +19,25 @@
 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.base.Statement;
-import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
 
 public class CreateDataverseStatement extends AbstractStatement {
 
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private String format;
     private boolean ifNotExists;
 
-    public CreateDataverseStatement(Identifier dataverseName, String format, boolean ifNotExists) {
+    public CreateDataverseStatement(DataverseName dataverseName, String format, boolean ifNotExists) {
         this.dataverseName = dataverseName;
         this.format = (format == null) ? NonTaggedDataFormat.class.getName() : format;
         this.ifNotExists = ifNotExists;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
index 9f01b1c..f12dfce 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFeedStatement.java
@@ -21,6 +21,7 @@ package org.apache.asterix.lang.common.statement;
 import java.util.Map;
 
 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.expression.RecordConstructor;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -37,18 +38,18 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
  */
 public class CreateFeedStatement extends AbstractStatement {
 
-    private final Pair<Identifier, Identifier> qName;
+    private final Pair<DataverseName, Identifier> qName;
     private final boolean ifNotExists;
     private final AdmObjectNode withObjectNode;
 
-    public CreateFeedStatement(Pair<Identifier, Identifier> qName, RecordConstructor withRecord, boolean ifNotExists)
+    public CreateFeedStatement(Pair<DataverseName, Identifier> qName, RecordConstructor withRecord, boolean ifNotExists)
             throws AlgebricksException {
         this.qName = qName;
         this.ifNotExists = ifNotExists;
         this.withObjectNode = withRecord == null ? null : ExpressionUtils.toNode(withRecord);
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return qName.first;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 2e9f1f9..05fc055 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 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.Statement;
 import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
@@ -33,7 +34,7 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 public class CreateIndexStatement extends AbstractStatement {
 
     private Identifier indexName;
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private Identifier datasetName;
     private List<Pair<List<String>, IndexedTypeExpression>> fieldExprs = new ArrayList<>();
     private List<Integer> fieldIndexIndicators = new ArrayList<>();
@@ -63,11 +64,11 @@ public class CreateIndexStatement extends AbstractStatement {
         this.indexName = indexName;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public void setDataverseName(Identifier dataverseName) {
+    public void setDataverseName(DataverseName dataverseName) {
         this.dataverseName = dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index ac019fc..b6a3d66 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 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.Statement;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
@@ -35,10 +36,10 @@ import org.apache.asterix.runtime.compression.CompressionManager;
 
 public class DatasetDecl extends AbstractStatement {
     protected final Identifier name;
-    protected final Identifier dataverse;
-    protected final Identifier itemTypeDataverse;
+    protected final DataverseName dataverse;
+    protected final DataverseName itemTypeDataverse;
     protected final Identifier itemTypeName;
-    protected final Identifier metaItemTypeDataverse;
+    protected final DataverseName metaItemTypeDataverse;
     protected final Identifier metaItemTypeName;
     protected final Identifier nodegroupName;
     protected final DatasetType datasetType;
@@ -47,24 +48,16 @@ public class DatasetDecl extends AbstractStatement {
     private final AdmObjectNode withObjectNode;
     protected final boolean ifNotExists;
 
-    public DatasetDecl(Identifier dataverse, Identifier name, Identifier itemTypeDataverse, Identifier itemTypeName,
-            Identifier metaItemTypeDataverse, Identifier metaItemTypeName, Identifier nodeGroupName,
-            Map<String, String> hints, DatasetType datasetType, IDatasetDetailsDecl idd, RecordConstructor withRecord,
-            boolean ifNotExists) throws CompilationException {
+    public DatasetDecl(DataverseName dataverse, Identifier name, DataverseName itemTypeDataverse,
+            Identifier itemTypeName, DataverseName metaItemTypeDataverse, Identifier metaItemTypeName,
+            Identifier nodeGroupName, Map<String, String> hints, DatasetType datasetType, IDatasetDetailsDecl idd,
+            RecordConstructor withRecord, boolean ifNotExists) throws CompilationException {
         this.dataverse = dataverse;
         this.name = name;
         this.itemTypeName = itemTypeName;
-        if (itemTypeDataverse.getValue() == null) {
-            this.itemTypeDataverse = dataverse;
-        } else {
-            this.itemTypeDataverse = itemTypeDataverse;
-        }
+        this.itemTypeDataverse = itemTypeDataverse == null ? dataverse : itemTypeDataverse;
         this.metaItemTypeName = metaItemTypeName;
-        if (metaItemTypeDataverse == null || metaItemTypeDataverse.getValue() == null) {
-            this.metaItemTypeDataverse = dataverse;
-        } else {
-            this.metaItemTypeDataverse = metaItemTypeDataverse;
-        }
+        this.metaItemTypeDataverse = metaItemTypeDataverse == null ? dataverse : metaItemTypeDataverse;
         this.nodegroupName = nodeGroupName;
         this.hints = hints;
         this.withObjectNode = DatasetDeclParametersUtil.validateAndGetWithObjectNode(withRecord);
@@ -85,38 +78,26 @@ public class DatasetDecl extends AbstractStatement {
         return name;
     }
 
+    public DataverseName getDataverse() {
+        return dataverse;
+    }
+
     public Identifier getItemTypeName() {
         return itemTypeName;
     }
 
-    public Identifier getItemTypeDataverse() {
+    public DataverseName getItemTypeDataverse() {
         return itemTypeDataverse;
     }
 
-    public String getQualifiedTypeName() {
-        if (itemTypeDataverse == dataverse) {
-            return itemTypeName.getValue();
-        } else {
-            return itemTypeDataverse.getValue() + "." + itemTypeName.getValue();
-        }
-    }
-
     public Identifier getMetaItemTypeName() {
         return metaItemTypeName;
     }
 
-    public Identifier getMetaItemTypeDataverse() {
+    public DataverseName getMetaItemTypeDataverse() {
         return metaItemTypeDataverse;
     }
 
-    public String getQualifiedMetaTypeName() {
-        if (metaItemTypeDataverse == dataverse) {
-            return metaItemTypeName.getValue();
-        } else {
-            return metaItemTypeDataverse.getValue() + "." + metaItemTypeName.getValue();
-        }
-    }
-
     public Identifier getNodegroupName() {
         return nodegroupName;
     }
@@ -179,10 +160,6 @@ public class DatasetDecl extends AbstractStatement {
         return datasetDetailsDecl;
     }
 
-    public Identifier getDataverse() {
-        return dataverse;
-    }
-
     @Override
     public byte getCategory() {
         return Category.DDL;
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
index 99303ce..829c407 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDecl.java
@@ -19,20 +19,20 @@
 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.base.Statement;
-import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class DataverseDecl extends AbstractStatement {
 
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
 
-    public DataverseDecl(Identifier dataverseName) {
+    public DataverseDecl(DataverseName dataverseName) {
         this.dataverseName = dataverseName;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
index 9e184c2..c20bd32 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DataverseDropStatement.java
@@ -19,17 +19,17 @@
 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.base.Statement;
-import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class DataverseDropStatement extends AbstractStatement {
 
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private boolean ifExists;
 
-    public DataverseDropStatement(Identifier dataverseName, boolean ifExists) {
+    public DataverseDropStatement(DataverseName dataverseName, boolean ifExists) {
         this.dataverseName = dataverseName;
         this.ifExists = ifExists;
     }
@@ -39,7 +39,7 @@ public class DataverseDropStatement extends AbstractStatement {
         return Statement.Kind.DATAVERSE_DROP;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
index f05efbc..67180bc 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DeleteStatement.java
@@ -21,6 +21,7 @@ package org.apache.asterix.lang.common.statement;
 import java.util.Objects;
 
 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.base.Statement;
@@ -31,13 +32,13 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 public class DeleteStatement extends AbstractStatement {
 
     private VariableExpr vars;
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private Identifier datasetName;
     private Expression condition;
     private int varCounter;
     private Query rewrittenQuery;
 
-    public DeleteStatement(VariableExpr vars, Identifier dataverseName, Identifier datasetName, Expression condition,
+    public DeleteStatement(VariableExpr vars, DataverseName dataverseName, Identifier datasetName, Expression condition,
             int varCounter) {
         this.vars = vars;
         this.dataverseName = dataverseName;
@@ -55,12 +56,12 @@ public class DeleteStatement extends AbstractStatement {
         return vars;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public Identifier getDatasetName() {
-        return datasetName;
+    public String getDatasetName() {
+        return datasetName.getValue();
     }
 
     public Expression getCondition() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
index cb33452..20306c5 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DisconnectFeedStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -27,29 +28,28 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public class DisconnectFeedStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier feedName;
     private final Identifier datasetName;
 
-    public DisconnectFeedStatement(Identifier dataverseName, Identifier feedName, Identifier datasetName) {
+    public DisconnectFeedStatement(DataverseName dataverseName, Identifier feedName, Identifier datasetName) {
         this.feedName = feedName;
         this.datasetName = datasetName;
         this.dataverseName = dataverseName;
     }
 
-    public DisconnectFeedStatement(Pair<Identifier, Identifier> feedNameComponent,
-            Pair<Identifier, Identifier> datasetNameComponent) {
+    public DisconnectFeedStatement(Pair<DataverseName, Identifier> feedNameComponent,
+            Pair<DataverseName, Identifier> datasetNameComponent) {
         if (feedNameComponent.first != null && datasetNameComponent.first != null
-                && !feedNameComponent.first.getValue().equals(datasetNameComponent.first.getValue())) {
+                && !feedNameComponent.first.equals(datasetNameComponent.first)) {
             throw new IllegalArgumentException("Dataverse for source feed and target dataset do not match");
         }
-        this.dataverseName = feedNameComponent.first != null ? feedNameComponent.first
-                : datasetNameComponent.first != null ? datasetNameComponent.first : null;
+        this.dataverseName = feedNameComponent.first != null ? feedNameComponent.first : datasetNameComponent.first;
         this.datasetName = datasetNameComponent.second;
         this.feedName = feedNameComponent.second;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
index 1434620..557a647 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DropDatasetStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,11 +27,11 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class DropDatasetStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier datasetName;
     private boolean ifExists;
 
-    public DropDatasetStatement(Identifier dataverseName, Identifier datasetName, boolean ifExists) {
+    public DropDatasetStatement(DataverseName dataverseName, Identifier datasetName, boolean ifExists) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.ifExists = ifExists;
@@ -41,7 +42,7 @@ public class DropDatasetStatement extends AbstractStatement {
         return Statement.Kind.DATASET_DROP;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
index ef6c096..c91d1a1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDropStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,11 +27,11 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class FeedDropStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier feedName;
     private boolean ifExists;
 
-    public FeedDropStatement(Identifier dataverseName, Identifier feedName, boolean ifExists) {
+    public FeedDropStatement(DataverseName dataverseName, Identifier feedName, boolean ifExists) {
         this.dataverseName = dataverseName;
         this.feedName = feedName;
         this.ifExists = ifExists;
@@ -41,7 +42,7 @@ public class FeedDropStatement extends AbstractStatement {
         return Statement.Kind.DROP_FEED;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
index a95254b..21a6a39 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedPolicyDropStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,11 +27,11 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class FeedPolicyDropStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier policyName;
     private boolean ifExists;
 
-    public FeedPolicyDropStatement(Identifier dataverseName, Identifier policyName, boolean ifExists) {
+    public FeedPolicyDropStatement(DataverseName dataverseName, Identifier policyName, boolean ifExists) {
         this.dataverseName = dataverseName;
         this.policyName = policyName;
         this.ifExists = ifExists;
@@ -41,7 +42,7 @@ public class FeedPolicyDropStatement extends AbstractStatement {
         return Statement.Kind.DROP_FEED_POLICY;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
index 39e1d31..c3583698 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/IndexDropStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,12 +27,12 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class IndexDropStatement extends AbstractStatement {
 
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private Identifier datasetName;
     private Identifier indexName;
     private boolean ifExists;
 
-    public IndexDropStatement(Identifier dataverseName, Identifier datasetName, Identifier indexName,
+    public IndexDropStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
             boolean ifExists) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
@@ -44,7 +45,7 @@ public class IndexDropStatement extends AbstractStatement {
         return Statement.Kind.INDEX_DROP;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
index efa58fc..5290bfd 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Objects;
 
 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.base.IReturningStatement;
@@ -33,14 +34,14 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class InsertStatement extends AbstractStatement implements IReturningStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier datasetName;
     private final Query query;
     private final VariableExpr var;
     private Expression returnExpression;
     private int varCounter;
 
-    public InsertStatement(Identifier dataverseName, Identifier datasetName, Query query, int varCounter,
+    public InsertStatement(DataverseName dataverseName, Identifier datasetName, Query query, int varCounter,
             VariableExpr var, Expression returnExpression) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
@@ -55,12 +56,12 @@ public class InsertStatement extends AbstractStatement implements IReturningStat
         return Statement.Kind.INSERT;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public Identifier getDatasetName() {
-        return datasetName;
+    public String getDatasetName() {
+        return datasetName.getValue();
     }
 
     public Query getQuery() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
index 5366bc0..42801f0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/LoadStatement.java
@@ -21,6 +21,7 @@ package org.apache.asterix.lang.common.statement;
 import java.util.Map;
 
 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.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -29,12 +30,12 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 public class LoadStatement extends AbstractStatement {
 
     private Identifier datasetName;
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private String adapter;
     private Map<String, String> properties;
     private boolean dataIsLocallySorted;
 
-    public LoadStatement(Identifier dataverseName, Identifier datasetName, String adapter,
+    public LoadStatement(DataverseName dataverseName, Identifier datasetName, String adapter,
             Map<String, String> propertiees, boolean dataIsLocallySorted) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
@@ -59,11 +60,11 @@ public class LoadStatement extends AbstractStatement {
         this.properties = properties;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public void setDataverseName(Identifier dataverseName) {
+    public void setDataverseName(DataverseName dataverseName) {
         this.dataverseName = dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
index 4ca9d97..30a02fb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/RefreshExternalDatasetStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,7 +27,7 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class RefreshExternalDatasetStatement extends AbstractStatement {
 
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private Identifier datasetName;
 
     public Identifier getDatasetName() {
@@ -37,11 +38,11 @@ public class RefreshExternalDatasetStatement extends AbstractStatement {
         this.datasetName = datasetName;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
-    public void setDataverseName(Identifier dataverseName) {
+    public void setDataverseName(DataverseName dataverseName) {
         this.dataverseName = dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
index 3ea791d..7dc652e 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StartFeedStatement.java
@@ -20,6 +20,7 @@
 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.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
@@ -28,10 +29,10 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 public class StartFeedStatement extends AbstractStatement {
 
     public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
-    private Identifier dataverseName;
+    private DataverseName dataverseName;
     private Identifier feedName;
 
-    public StartFeedStatement(Pair<Identifier, Identifier> feedNameComp) {
+    public StartFeedStatement(Pair<DataverseName, Identifier> feedNameComp) {
         dataverseName = feedNameComp.first;
         feedName = feedNameComp.second;
     }
@@ -51,7 +52,7 @@ public class StartFeedStatement extends AbstractStatement {
         return Category.UPDATE;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
index 99e5069..4cfcc0a 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/StopFeedStatement.java
@@ -19,6 +19,7 @@
 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.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
@@ -26,10 +27,10 @@ import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public class StopFeedStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier feedName;
 
-    public StopFeedStatement(Pair<Identifier, Identifier> feedNameComp) {
+    public StopFeedStatement(Pair<DataverseName, Identifier> feedNameComp) {
         this.dataverseName = feedNameComp.first;
         this.feedName = feedNameComp.second;
     }
@@ -49,7 +50,7 @@ public class StopFeedStatement extends AbstractStatement {
         return visitor.visit(this, arg);
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
index 5430945..a840fea 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDecl.java
@@ -20,6 +20,7 @@ package org.apache.asterix.lang.common.statement;
 
 import org.apache.asterix.common.annotations.TypeDataGen;
 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.Statement;
 import org.apache.asterix.lang.common.expression.TypeExpression;
@@ -28,13 +29,13 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class TypeDecl extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private final Identifier ident;
     private final TypeExpression typeDef;
     private final TypeDataGen datagenAnnotation;
     private final boolean ifNotExists;
 
-    public TypeDecl(Identifier dataverseName, Identifier ident, TypeExpression typeDef, TypeDataGen datagen,
+    public TypeDecl(DataverseName dataverseName, Identifier ident, TypeExpression typeDef, TypeDataGen datagen,
             boolean ifNotExists) {
         this.dataverseName = dataverseName;
         this.ident = ident;
@@ -43,7 +44,7 @@ public class TypeDecl extends AbstractStatement {
         this.ifNotExists = ifNotExists;
     }
 
-    public TypeDecl(Identifier dataverse, Identifier ident, TypeExpression typeDef) {
+    public TypeDecl(DataverseName dataverse, Identifier ident, TypeExpression typeDef) {
         this(dataverse, ident, typeDef, null, false);
     }
 
@@ -51,7 +52,7 @@ public class TypeDecl extends AbstractStatement {
         return ident;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
index 9f59f98..73aef54 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/TypeDropStatement.java
@@ -19,6 +19,7 @@
 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.base.Statement;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -26,11 +27,11 @@ import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 
 public class TypeDropStatement extends AbstractStatement {
 
-    private final Identifier dataverseName;
+    private final DataverseName dataverseName;
     private Identifier typeName;
     private boolean ifExists;
 
-    public TypeDropStatement(Identifier dataverseName, Identifier typeName, boolean ifExists) {
+    public TypeDropStatement(DataverseName dataverseName, Identifier typeName, boolean ifExists) {
         this.dataverseName = dataverseName;
         this.typeName = typeName;
         this.ifExists = ifExists;
@@ -41,7 +42,7 @@ public class TypeDropStatement extends AbstractStatement {
         return Statement.Kind.TYPE_DROP;
     }
 
-    public Identifier getDataverseName() {
+    public DataverseName getDataverseName() {
         return dataverseName;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
index 178a8f9..f4f48f7 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.lang.common.statement;
 
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
@@ -25,7 +26,7 @@ import org.apache.asterix.lang.common.struct.Identifier;
 
 public class UpsertStatement extends InsertStatement {
 
-    public UpsertStatement(Identifier dataverseName, Identifier datasetName, Query query, int varCounter,
+    public UpsertStatement(DataverseName dataverseName, Identifier datasetName, Query query, int varCounter,
             VariableExpr var, Expression returnExpression) {
         super(dataverseName, datasetName, query, varCounter, var, returnExpression);
     }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
index 7701502..a7ec834 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
@@ -165,10 +165,10 @@ public class CommonFunctionMapUtil {
         String lowerCaseName = name.toLowerCase();
         String mappedName = getFunctionMapping(lowerCaseName);
         if (mappedName != null) {
-            return new FunctionSignature(fs.getNamespace(), mappedName, fs.getArity());
+            return new FunctionSignature(fs.getDataverseName(), mappedName, fs.getArity());
         }
         String understoreName = lowerCaseName.replace('_', '-');
-        FunctionSignature newFs = new FunctionSignature(fs.getNamespace(), understoreName, fs.getArity());
+        FunctionSignature newFs = new FunctionSignature(fs.getDataverseName(), understoreName, fs.getArity());
         return BuiltinFunctions.isBuiltinCompilerFunction(newFs, true) ? newFs : fs;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index de5e931..dd21152 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -103,4 +103,14 @@ public class ExpressionUtils {
     public static <T> Collection<T> emptyIfNull(Collection<T> coll) {
         return coll == null ? Collections.emptyList() : coll;
     }
+
+    public static String getStringLiteral(Expression arg) {
+        if (arg.getKind() == Expression.Kind.LITERAL_EXPRESSION) {
+            Literal item = ((LiteralExpr) arg).getValue();
+            if (item.getLiteralType() == Literal.Type.STRING) {
+                return item.getStringValue();
+            }
+        }
+        return null;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 24384f2..3bd2504 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -20,27 +20,30 @@
 package org.apache.asterix.lang.common.util;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.expression.CallExpr;
-import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Function;
-import org.apache.asterix.metadata.utils.DatasetUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
 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.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
@@ -50,12 +53,17 @@ public class FunctionUtil {
 
     public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
 
+    private static final DataverseName FN_DATASET_DATAVERSE_NAME =
+            FunctionSignature.getDataverseName(BuiltinFunctions.DATASET);
+
+    private static final String FN_DATASET_NAME = BuiltinFunctions.DATASET.getName();
+
     public static IFunctionInfo getFunctionInfo(FunctionIdentifier fi) {
         return BuiltinFunctions.getAsterixFunctionInfo(fi);
     }
 
     public static IFunctionInfo getFunctionInfo(FunctionSignature fs) {
-        return getFunctionInfo(new FunctionIdentifier(fs.getNamespace(), fs.getName(), fs.getArity()));
+        return getFunctionInfo(fs.createFunctionIdentifier());
     }
 
     public static IFunctionInfo getBuiltinFunctionInfo(String functionName, int arity) {
@@ -112,21 +120,21 @@ public class FunctionUtil {
             return functionDecls;
         }
         String value = (String) metadataProvider.getConfig().get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS);
-        boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
+        boolean includePrivateFunctions = (value != null) && Boolean.parseBoolean(value.toLowerCase());
         Set<CallExpr> functionCalls = functionCollector.getFunctionCalls(expression);
         for (CallExpr functionCall : functionCalls) {
             FunctionSignature signature = functionCall.getFunctionSignature();
             if (declaredFunctions != null && declaredFunctions.contains(signature)) {
                 continue;
             }
-            if (signature.getNamespace() == null) {
-                signature.setNamespace(metadataProvider.getDefaultDataverseName());
+            if (signature.getDataverseName() == null) {
+                signature.setDataverseName(metadataProvider.getDefaultDataverseName());
             }
-            String namespace = signature.getNamespace();
+            DataverseName namespace = signature.getDataverseName();
             // Checks the existence of the referred dataverse.
             try {
-                if (!namespace.equals(FunctionConstants.ASTERIX_NS)
-                        && !namespace.equals(AlgebricksBuiltinFunctions.ALGEBRICKS_NS)
+                if (!namespace.equals(FunctionConstants.ASTERIX_DV)
+                        && !namespace.equals(FunctionConstants.ALGEBRICKS_DV)
                         && metadataProvider.findDataverse(namespace) == null) {
                     throw new CompilationException(ErrorCode.COMPILATION_ERROR, functionCall.getSourceLocation(),
                             "In function call \"" + namespace + "." + signature.getName() + "(...)\", the dataverse \""
@@ -177,30 +185,25 @@ public class FunctionUtil {
         return functionDecls;
     }
 
-    public static List<List<List<String>>> getFunctionDependencies(IQueryRewriter rewriter, Expression expression,
-            MetadataProvider metadataProvider) throws CompilationException {
+    public static List<List<Triple<DataverseName, String, String>>> getFunctionDependencies(IQueryRewriter rewriter,
+            Expression expression, MetadataProvider metadataProvider) throws CompilationException {
         Set<CallExpr> functionCalls = rewriter.getFunctionCalls(expression);
         //Get the List of used functions and used datasets
-        List<List<String>> datasourceDependencies = new ArrayList<>();
-        List<List<String>> functionDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> datasourceDependencies = new ArrayList<>();
+        List<Triple<DataverseName, String, String>> functionDependencies = new ArrayList<>();
         for (CallExpr functionCall : functionCalls) {
             FunctionSignature signature = functionCall.getFunctionSignature();
-            FunctionIdentifier fid =
-                    new FunctionIdentifier(signature.getNamespace(), signature.getName(), signature.getArity());
-            if (fid.equals(BuiltinFunctions.DATASET)) {
-                Pair<String, String> path = DatasetUtil.getDatasetInfo(metadataProvider,
-                        ((LiteralExpr) functionCall.getExprList().get(0)).getValue().getStringValue());
-                datasourceDependencies.add(Arrays.asList(path.first, path.second));
-            }
-
-            else if (BuiltinFunctions.isBuiltinCompilerFunction(signature, false)) {
-                continue;
-            } else {
-                functionDependencies.add(Arrays.asList(signature.getNamespace(), signature.getName(),
+            if (isBuiltinDatasetFunction(signature)) {
+                Pair<DataverseName, String> datasetReference = parseDatasetFunctionArguments(functionCall.getExprList(),
+                        metadataProvider.getDefaultDataverseName(), functionCall.getSourceLocation(),
+                        ExpressionUtils::getStringLiteral);
+                datasourceDependencies.add(new Triple<>(datasetReference.first, datasetReference.second, null));
+            } else if (!BuiltinFunctions.isBuiltinCompilerFunction(signature, false)) {
+                functionDependencies.add(new Triple<>(signature.getDataverseName(), signature.getName(),
                         Integer.toString(signature.getArity())));
             }
         }
-        List<List<List<String>>> dependencies = new ArrayList<>();
+        List<List<Triple<DataverseName, String, String>>> dependencies = new ArrayList<>(2);
         dependencies.add(datasourceDependencies);
         dependencies.add(functionDependencies);
         return dependencies;
@@ -208,10 +211,67 @@ public class FunctionUtil {
 
     private static Function lookupUserDefinedFunctionDecl(MetadataTransactionContext mdTxnCtx,
             FunctionSignature signature) throws AlgebricksException {
-        if (signature.getNamespace() == null) {
+        if (signature.getDataverseName() == null) {
             return null;
         }
         return MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
     }
 
+    public static boolean isBuiltinDatasetFunction(FunctionSignature fs) {
+        return Objects.equals(FN_DATASET_DATAVERSE_NAME, fs.getDataverseName())
+                && Objects.equals(FN_DATASET_NAME, fs.getName());
+    }
+
+    public static Pair<DataverseName, String> parseDatasetFunctionArguments(
+            List<Mutable<ILogicalExpression>> datasetFnArgs, DataverseName defaultDataverseName,
+            SourceLocation sourceLoc) throws CompilationException {
+        return parseDatasetFunctionArguments(datasetFnArgs, defaultDataverseName, sourceLoc,
+                FunctionUtil::getStringConstant);
+    }
+
+    public static <T> Pair<DataverseName, String> parseDatasetFunctionArguments(List<T> datasetFnArgs,
+            DataverseName defaultDataverseName, SourceLocation sourceLoc,
+            java.util.function.Function<T, String> argExtractFunction) throws CompilationException {
+        DataverseName dataverseName;
+        String datasetName;
+        switch (datasetFnArgs.size()) {
+            case 1: // AQL BACK-COMPAT case
+                String datasetArgBackCompat = argExtractFunction.apply(datasetFnArgs.get(0));
+                if (datasetArgBackCompat == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Invalid argument to dataset()");
+                }
+                int pos = datasetArgBackCompat.indexOf('.');
+                if (pos > 0 && pos < datasetArgBackCompat.length() - 1) {
+                    dataverseName = DataverseName.createSinglePartName(datasetArgBackCompat.substring(0, pos)); // AQL BACK-COMPAT
+                    datasetName = datasetArgBackCompat.substring(pos + 1);
+                } else {
+                    dataverseName = defaultDataverseName;
+                    datasetName = datasetArgBackCompat;
+                }
+                break;
+            case 2:
+                String dataverseNameArg = argExtractFunction.apply(datasetFnArgs.get(0));
+                if (dataverseNameArg == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Invalid argument to dataset()");
+                }
+                dataverseName = DataverseName.createFromCanonicalForm(dataverseNameArg);
+
+                datasetName = argExtractFunction.apply(datasetFnArgs.get(1));
+                if (datasetName == null) {
+                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                            "Invalid argument to dataset()");
+                }
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+                        "Invalid number of arguments to dataset()");
+        }
+        return new Pair<>(dataverseName, datasetName);
+    }
+
+    private static String getStringConstant(Mutable<ILogicalExpression> arg) {
+        return ConstantExpressionUtil.getStringConstant(arg.getValue());
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index 1aab7f7..f934d60 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -27,6 +27,7 @@ import java.util.Map;
 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.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -378,17 +379,17 @@ public abstract class AbstractInlineUdfsVisitor extends AbstractQueryExpressionV
         wrappedQuery.setBody(fnDecl.getFuncBody());
         wrappedQuery.setTopLevel(false);
 
-        String fnNamespace = fnDecl.getSignature().getNamespace();
+        DataverseName fnDataverseName = fnDecl.getSignature().getDataverseName();
         Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
 
         Dataverse fnDataverse;
-        if (fnNamespace == null || fnNamespace.equals(defaultDataverse.getDataverseName())) {
+        if (fnDataverseName == null || fnDataverseName.equals(defaultDataverse.getDataverseName())) {
             fnDataverse = defaultDataverse;
         } else {
             try {
-                fnDataverse = metadataProvider.findDataverse(fnNamespace);
+                fnDataverse = metadataProvider.findDataverse(fnDataverseName);
             } catch (AlgebricksException e) {
-                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnNamespace);
+                throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, e, sourceLoc, fnDataverseName);
             }
         }
 
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 6b734dd..d241a71 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
@@ -32,6 +32,7 @@ import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -99,6 +100,7 @@ import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.UnaryExprType;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 
@@ -115,6 +117,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     protected String dataverseSymbol = " dataverse ";
     protected String datasetSymbol = " dataset ";
     protected String assignSymbol = ":=";
+    private final List<String> dataverseNameParts = new ArrayList<>();
 
     public FormatPrintVisitor(PrintWriter out) {
         this.out = out;
@@ -214,7 +217,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     @Override
     public Void visit(CallExpr callExpr, Integer step) throws CompilationException {
         printHints(callExpr.getHints(), step);
-        out.print(generateFullName(callExpr.getFunctionSignature().getNamespace(),
+        out.print(generateFullName(callExpr.getFunctionSignature().getDataverseName(),
                 callExpr.getFunctionSignature().getName()) + "(");
         printDelimitedExpressions(callExpr.getExprList(), COMMA, step);
         out.print(")");
@@ -398,8 +401,8 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
 
     @Override
     public Void visit(TypeReferenceExpression t, Integer arg) throws CompilationException {
-        if (t.getIdent().first != null && t.getIdent().first.getValue() != null) {
-            out.print(normalize(t.getIdent().first.getValue()));
+        if (t.getIdent().first != null && t.getIdent().first != null) {
+            out.print(generateDataverseName(t.getIdent().first));
             out.print('.');
         }
         out.print(normalize(t.getIdent().second.getValue()));
@@ -457,8 +460,8 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     public Void visit(DatasetDecl dd, Integer step) throws CompilationException {
         if (dd.getDatasetType() == DatasetType.INTERNAL) {
             out.print(skip(step) + "create " + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
-                    + generateIfNotExists(dd.getIfNotExists()) + "(" + dd.getQualifiedTypeName() + ")"
-                    + " primary key ");
+                    + generateIfNotExists(dd.getIfNotExists()) + "("
+                    + generateFullName(dd.getItemTypeDataverse(), dd.getItemTypeName()) + ")" + " primary key ");
             printDelimitedKeys(((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs(), ",");
             if (((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated()) {
                 out.print(" autogenerated ");
@@ -466,7 +469,8 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
         } else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
             out.print(
                     skip(step) + "create external " + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
-                            + "(" + dd.getQualifiedTypeName() + ")" + generateIfNotExists(dd.getIfNotExists()));
+                            + "(" + generateFullName(dd.getItemTypeDataverse(), dd.getItemTypeName()) + ")"
+                            + generateIfNotExists(dd.getIfNotExists()));
             ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
             out.print(" using " + revertStringToQuoted(externalDetails.getAdapter()));
             printConfiguration(externalDetails.getProperties());
@@ -498,7 +502,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
 
     @Override
     public Void visit(DataverseDecl dv, Integer step) throws CompilationException {
-        out.println(skip(step) + "use " + dataverseSymbol + normalize(dv.getDataverseName().getValue()) + ";\n\n");
+        out.println(skip(step) + "use " + dataverseSymbol + generateDataverseName(dv.getDataverseName()) + ";\n\n");
         return null;
     }
 
@@ -676,7 +680,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     @Override
     public Void visit(CreateDataverseStatement del, Integer step) throws CompilationException {
         out.print(CREATE + dataverseSymbol);
-        out.print(normalize(del.getDataverseName().getValue()));
+        out.print(generateDataverseName(del.getDataverseName()));
         out.print(generateIfNotExists(del.getIfNotExists()));
         String format = del.getFormat();
         if (format != null && !format.equals(DEFAULT_DATAVERSE_FORMAT)) {
@@ -708,7 +712,7 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     @Override
     public Void visit(DataverseDropStatement del, Integer step) throws CompilationException {
         out.print(skip(step) + "drop " + dataverseSymbol);
-        out.print(normalize(del.getDataverseName().getValue()));
+        out.print(generateDataverseName(del.getDataverseName()));
         out.println(generateIfExists(del.getIfExists()) + SEMICOLON);
         return null;
     }
@@ -809,8 +813,8 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
     public Void visit(CreateFunctionStatement cfs, Integer step) throws CompilationException {
         out.print(skip(step) + CREATE + " function ");
         out.print(generateIfNotExists(cfs.getIfNotExists()));
-        out.print(
-                this.generateFullName(cfs.getFunctionSignature().getNamespace(), cfs.getFunctionSignature().getName()));
+        out.print(this.generateFullName(cfs.getFunctionSignature().getDataverseName(),
+                cfs.getFunctionSignature().getName()));
         out.print("(");
         printDelimitedStrings(cfs.getParamList(), COMMA);
         out.println(") {");
@@ -984,14 +988,28 @@ public class FormatPrintVisitor implements ILangVisitor<Void, Integer> {
         return str;
     }
 
-    protected String generateFullName(String namespace, String identifier) {
-        String dataversePrefix = namespace != null && !namespace.equals("Metadata") ? normalize(namespace) + "." : "";
+    protected String generateDataverseName(DataverseName dataverseName) {
+        StringBuilder sb = new StringBuilder();
+        dataverseNameParts.clear();
+        dataverseName.getParts(dataverseNameParts);
+        for (int i = 0, ln = dataverseNameParts.size(); i < ln; i++) {
+            if (i > 0) {
+                sb.append(DataverseName.SEPARATOR_CHAR);
+            }
+            sb.append(normalize(dataverseNameParts.get(i)));
+        }
+        return sb.toString();
+    }
+
+    protected String generateFullName(DataverseName dataverseName, String identifier) {
+        String dataversePrefix =
+                dataverseName != null && !dataverseName.equals(MetadataConstants.METADATA_DATAVERSE_NAME)
+                        ? generateDataverseName(dataverseName) + "." : "";
         return dataversePrefix + normalize(identifier);
     }
 
-    protected String generateFullName(Identifier dv, Identifier ds) {
-        String dataverse = dv != null ? dv.getValue() : null;
-        return generateFullName(dataverse, ds.getValue());
+    protected String generateFullName(DataverseName dataverseName, Identifier ds) {
+        return generateFullName(dataverseName, ds.getValue());
     }
 
     protected String generateIfNotExists(boolean ifNotExits) {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
index ff55880..6afe4e0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
@@ -25,6 +25,7 @@ import java.util.Map.Entry;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -64,6 +65,7 @@ import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public abstract class QueryPrintVisitor extends AbstractQueryExpressionVisitor<Void, Integer> {
     protected final PrintWriter out;
@@ -142,14 +144,30 @@ public abstract class QueryPrintVisitor extends AbstractQueryExpressionVisitor<V
 
     @Override
     public Void visit(CallExpr pf, Integer step) throws CompilationException {
-        out.println(skip(step) + "FunctionCall " + pf.getFunctionSignature().toString() + "[");
-        for (Expression expr : pf.getExprList()) {
+        return printFunctionCall(pf.getFunctionSignature(), pf.getFunctionSignature().getArity(), pf.getExprList(),
+                step);
+    }
+
+    protected Void printFunctionCall(FunctionSignature fs, int arity, List<Expression> argList, Integer step)
+            throws CompilationException {
+        out.print(skip(step) + "FunctionCall ");
+        printFunctionSignature(out, fs, arity);
+        out.println("[");
+        for (Expression expr : argList) {
             expr.accept(this, step + 1);
         }
         out.println(skip(step) + "]");
         return null;
     }
 
+    private static void printFunctionSignature(PrintWriter out, FunctionSignature fs, int arity) {
+        out.print(fs.toString(false));
+        if (arity != FunctionIdentifier.VARARGS) {
+            out.print("@");
+            out.print(arity);
+        }
+    }
+
     @Override
     public Void visit(OperatorExpr ifbo, Integer step) throws CompilationException {
         List<Expression> exprList = ifbo.getExprList();
@@ -329,8 +347,8 @@ public abstract class QueryPrintVisitor extends AbstractQueryExpressionVisitor<V
 
     @Override
     public Void visit(TypeReferenceExpression t, Integer arg) throws CompilationException {
-        if (t.getIdent().first != null && t.getIdent().first.getValue() != null) {
-            out.print(t.getIdent().first.getValue());
+        if (t.getIdent().first != null && t.getIdent().first != null) {
+            out.print(t.getIdent().first);
             out.print('.');
         }
         out.print(t.getIdent().second.getValue());
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
index 7908636..4a08874 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppStatementRewriter.java
@@ -23,17 +23,18 @@ import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.lang.sqlpp.visitor.SqlppDeleteRewriteVisitor;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class SqlppStatementRewriter implements IStatementRewriter {
 
     @Override
-    public void rewrite(Statement stmt) throws CompilationException {
-        rewriteDeleteStatement(stmt);
+    public void rewrite(Statement stmt, MetadataProvider metadataProvider) throws CompilationException {
+        rewriteDeleteStatement(stmt, metadataProvider);
     }
 
-    private void rewriteDeleteStatement(Statement stmt) throws CompilationException {
+    private void rewriteDeleteStatement(Statement stmt, MetadataProvider metadataProvider) throws CompilationException {
         if (stmt != null) {
-            SqlppDeleteRewriteVisitor visitor = new SqlppDeleteRewriteVisitor();
+            SqlppDeleteRewriteVisitor visitor = new SqlppDeleteRewriteVisitor(metadataProvider);
             stmt.accept(visitor, null);
         }
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 23b4d60..6d07a7b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -94,7 +94,7 @@ public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVis
         }
     }
 
-    private Expression processInOperator(OperatorExpr operatorExpr, OperatorType opType) throws CompilationException {
+    private Expression processInOperator(OperatorExpr operatorExpr, OperatorType opType) {
         VariableExpr bindingVar = new VariableExpr(context.newVariable());
         bindingVar.setSourceLocation(operatorExpr.getSourceLocation());
         Expression itemExpr = operatorExpr.getExprList().get(0);
@@ -123,7 +123,7 @@ public class OperatorExpressionVisitor extends AbstractSqlppExpressionScopingVis
 
     private Expression processConcatOperator(OperatorExpr operatorExpr) {
         // All operators have to be "||"s (according to the grammar).
-        CallExpr callExpr = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, FunctionMapUtil.CONCAT, 1),
+        CallExpr callExpr = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_DV, FunctionMapUtil.CONCAT, 1),
                 operatorExpr.getExprList());
         callExpr.setSourceLocation(operatorExpr.getSourceLocation());
         return callExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
index 6d2dec7..cb6b396 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/Sql92AggregateFunctionVisitor.java
@@ -144,7 +144,7 @@ class Sql92AggregateFunctionVisitor extends AbstractSqlppSimpleExpressionVisitor
                 }
                 FieldAccessor faInner = new FieldAccessor(fromBindingVar, groupVarField);
                 faInner.setSourceLocation(usedVar.getSourceLocation());
-                Expression faOuter = VariableCheckAndRewriteVisitor.resolveAsFieldAccess(faInner, usedVar.getVar(),
+                Expression faOuter = VariableCheckAndRewriteVisitor.generateFieldAccess(faInner, usedVar.getVar(),
                         usedVar.getSourceLocation());
                 varExprMap.put(usedVar, faOuter);
             }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
index aaf8feb..70739ed 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/VariableCheckAndRewriteVisitor.java
@@ -28,6 +28,7 @@ import java.util.Set;
 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.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -48,15 +49,13 @@ import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppExpressionScoping
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopingVisitor {
 
-    private static final FunctionSignature FN_DATASET = new FunctionSignature(BuiltinFunctions.DATASET);
-
     protected final MetadataProvider metadataProvider;
 
     /**
@@ -70,105 +69,151 @@ public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopi
     }
 
     @Override
+    public Expression visit(VariableExpr varExpr, ILangExpression parent) throws CompilationException {
+        if (resolveAsVariableReference(varExpr)) {
+            return varExpr;
+        }
+        DataverseName dataverseName = metadataProvider.getDefaultDataverseName();
+        String datasetName = SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue();
+        CallExpr datasetExpr = resolveAsDataset(dataverseName, datasetName, parent, varExpr);
+        return datasetExpr != null ? datasetExpr : resolveAsFieldAccessOverContextVar(varExpr);
+    }
+
+    @Override
     public Expression visit(FieldAccessor fa, ILangExpression parent) throws CompilationException {
         Expression leadingExpr = fa.getExpr();
-        if (leadingExpr.getKind() != Kind.VARIABLE_EXPRESSION) {
-            fa.setExpr(leadingExpr.accept(this, parent));
-            return fa;
+        if (leadingExpr.getKind() == Kind.VARIABLE_EXPRESSION) {
+            // resolving a.b
+            VariableExpr leadingVarExpr = (VariableExpr) leadingExpr;
+            if (resolveAsVariableReference(leadingVarExpr)) {
+                return fa;
+            } else {
+                String dataverseNamePart =
+                        SqlppVariableUtil.toUserDefinedVariableName(leadingVarExpr.getVar().getValue()).getValue();
+                DataverseName dataverseName = DataverseName.createSinglePartName(dataverseNamePart); // 1-part name
+                String datasetName = fa.getIdent().getValue();
+                CallExpr datasetExpr = resolveAsDataset(dataverseName, datasetName, parent, leadingVarExpr);
+                if (datasetExpr != null) {
+                    return datasetExpr;
+                } else {
+                    fa.setExpr(resolveAsFieldAccessOverContextVar(leadingVarExpr));
+                    return fa;
+                }
+            }
         } else {
-            VariableExpr varExpr = (VariableExpr) leadingExpr;
-            String lastIdentifier = fa.getIdent().getValue();
-            Expression resolvedExpr = resolve(varExpr,
-                    /* Resolves within the dataverse that has the same name as the variable name. */
-                    SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue(), lastIdentifier,
-                    parent);
-            if (resolvedExpr.getKind() == Kind.CALL_EXPRESSION) {
-                CallExpr callExpr = (CallExpr) resolvedExpr;
-                if (callExpr.getFunctionSignature().equals(FN_DATASET)) {
-                    // The field access is resolved to be a dataset access in the form of "dataverse.dataset".
-                    return resolvedExpr;
+            List<String> dataverseNameParts = new ArrayList<>(4);
+            Pair<VariableExpr, FieldAccessor> topExprs = new Pair<>(null, null);
+            if (extractDataverseName(fa.getExpr(), dataverseNameParts, topExprs)) {
+                // resolving a.b.c(.x)*
+                VariableExpr topVarExpr = topExprs.getFirst(); // = a
+                if (resolveAsVariableReference(topVarExpr)) {
+                    return fa;
+                } else {
+                    DataverseName dataverseName = DataverseName.create(dataverseNameParts);
+                    String datasetName = fa.getIdent().getValue();
+                    CallExpr datasetExpr = resolveAsDataset(dataverseName, datasetName, parent, topVarExpr);
+                    if (datasetExpr != null) {
+                        return datasetExpr;
+                    }
+                    FieldAccessor topFaExpr = topExprs.getSecond(); // = a.b
+                    topFaExpr.setExpr(resolveAsFieldAccessOverContextVar(topVarExpr));
+                    return fa;
                 }
+            } else {
+                fa.setExpr(leadingExpr.accept(this, parent));
+                return fa;
             }
-            fa.setExpr(resolvedExpr);
-            return fa;
         }
     }
 
-    @Override
-    public Expression visit(VariableExpr varExpr, ILangExpression parent) throws CompilationException {
-        return resolve(varExpr, metadataProvider.getDefaultDataverseName(),
-                SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue(), parent);
-    }
-
-    // Resolve a variable expression with dataverse name and dataset name.
-    private Expression resolve(VariableExpr varExpr, String dataverseName, String datasetName, ILangExpression parent)
-            throws CompilationException {
-
+    private boolean resolveAsVariableReference(VariableExpr varExpr) throws CompilationException {
         VarIdentifier varId = varExpr.getVar();
         String varName = varId.getValue();
-        SourceLocation sourceLoc = varExpr.getSourceLocation();
-        VarIdentifier var = lookupVariable(varName, sourceLoc);
-        if (var != null) {
-            // Exists such an identifier
-            varExpr.setIsNewVar(false);
-            varExpr.setVar(var);
-            return varExpr;
-        }
-
-        if (SqlppVariableUtil.isExternalVariableIdentifier(varId)) {
-            throw new CompilationException(ErrorCode.PARAMETER_NO_VALUE, sourceLoc,
-                    SqlppVariableUtil.variableNameToDisplayedFieldName(varId.getValue()));
-        }
-
-        boolean resolveToDataset = parent.accept(CheckDatasetOnlyResolutionVisitor.INSTANCE, varExpr);
-        if (resolveToDataset) {
-            // resolve the undefined identifier reference as a dataset access.
-            // for a From/Join/UNNEST/Quantifiers binding expression
-            return resolveAsDataset(dataverseName, datasetName, sourceLoc);
-        } else {
-            // resolve the undefined identifier reference as a field access on a context variable
-            Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> localVars =
-                    scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
-            Set<VariableExpr> contextVars = Scope.findVariablesAnnotatedBy(localVars.keySet(),
-                    SqlppVariableAnnotation.CONTEXT_VARIABLE, localVars, sourceLoc);
-            VariableExpr contextVar = pickContextVar(contextVars, varExpr);
-            return resolveAsFieldAccess(contextVar, varId, sourceLoc);
-        }
-    }
-
-    private VarIdentifier lookupVariable(String varName, SourceLocation sourceLoc) throws CompilationException {
         if (scopeChecker.isInForbiddenScopes(varName)) {
-            throw new CompilationException(ErrorCode.FORBIDDEN_SCOPE, sourceLoc);
+            throw new CompilationException(ErrorCode.FORBIDDEN_SCOPE, varExpr.getSourceLocation());
         }
         Identifier ident = scopeChecker.lookupSymbol(varName);
-        return ident != null ? (VarIdentifier) ident : null;
+        if (ident == null) {
+            if (SqlppVariableUtil.isExternalVariableIdentifier(varId)) {
+                throw new CompilationException(ErrorCode.PARAMETER_NO_VALUE, varExpr.getSourceLocation(),
+                        SqlppVariableUtil.variableNameToDisplayedFieldName(varId.getValue()));
+            } else {
+                return false;
+            }
+        }
+        // Exists such an identifier
+        varExpr.setIsNewVar(false);
+        varExpr.setVar((VarIdentifier) ident);
+        return true;
     }
 
-    private Expression resolveAsDataset(String dataverseName, String datasetName, SourceLocation sourceLoc)
-            throws CompilationException {
+    // try resolving the undefined identifier reference as a dataset access.
+    // for a From/Join/UNNEST/Quantifiers binding expression
+    private CallExpr resolveAsDataset(DataverseName dataverseName, String datasetName, ILangExpression parent,
+            VariableExpr varExpr) throws CompilationException {
+        if (!parent.accept(CheckDatasetOnlyResolutionVisitor.INSTANCE, varExpr)) {
+            return null;
+        }
+        SourceLocation sourceLoc = varExpr.getSourceLocation();
         Dataset dataset = findDataset(dataverseName, datasetName, sourceLoc);
         if (dataset == null) {
             throw createUnresolvableError(dataverseName, datasetName, sourceLoc);
         }
         metadataProvider.addAccessedDataset(dataset);
-        List<Expression> argList = new ArrayList<>(1);
-        argList.add(new LiteralExpr(new StringLiteral(dataset.getFullyQualifiedName())));
+        List<Expression> argList = new ArrayList<>(2);
+        argList.add(new LiteralExpr(new StringLiteral(dataset.getDataverseName().getCanonicalForm())));
+        argList.add(new LiteralExpr(new StringLiteral(dataset.getDatasetName())));
         CallExpr callExpr = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), argList);
         callExpr.setSourceLocation(sourceLoc);
         return callExpr;
     }
 
+    // resolve the undefined identifier reference as a field access on a context variable
+    private FieldAccessor resolveAsFieldAccessOverContextVar(VariableExpr varExpr) throws CompilationException {
+        Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> localVars =
+                scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
+        Set<VariableExpr> contextVars = Scope.findVariablesAnnotatedBy(localVars.keySet(),
+                SqlppVariableAnnotation.CONTEXT_VARIABLE, localVars, varExpr.getSourceLocation());
+        VariableExpr contextVar = pickContextVar(contextVars, varExpr);
+        return generateFieldAccess(contextVar, varExpr.getVar(), varExpr.getSourceLocation());
+    }
+
     // Rewrites for an field access by name
-    static Expression resolveAsFieldAccess(Expression sourceExpr, VarIdentifier fieldVar, SourceLocation sourceLoc) {
+    static FieldAccessor generateFieldAccess(Expression sourceExpr, VarIdentifier fieldVar, SourceLocation sourceLoc) {
         VarIdentifier fieldName = SqlppVariableUtil.toUserDefinedVariableName(fieldVar.getValue());
         FieldAccessor fa = new FieldAccessor(sourceExpr, fieldName);
         fa.setSourceLocation(sourceLoc);
         return fa;
     }
 
-    private CompilationException createUnresolvableError(String dataverseName, String datasetName,
+    private static boolean extractDataverseName(Expression expr, List<String> outDataverseName,
+            Pair<VariableExpr, FieldAccessor> outTopExprs) {
+        switch (expr.getKind()) {
+            case VARIABLE_EXPRESSION:
+                VariableExpr varExpr = (VariableExpr) expr;
+                String varName = SqlppVariableUtil.toUserDefinedVariableName(varExpr.getVar().getValue()).getValue();
+                outDataverseName.add(varName);
+                outTopExprs.setFirst(varExpr);
+                return true;
+            case FIELD_ACCESSOR_EXPRESSION:
+                FieldAccessor faExpr = (FieldAccessor) expr;
+                if (extractDataverseName(faExpr.getExpr(), outDataverseName, outTopExprs)) {
+                    outDataverseName.add(faExpr.getIdent().getValue());
+                    if (outTopExprs.getSecond() == null) {
+                        outTopExprs.setSecond(faExpr);
+                    }
+                    return true;
+                } else {
+                    return false;
+                }
+            default:
+                return false;
+        }
+    }
+
+    private CompilationException createUnresolvableError(DataverseName dataverseName, String datasetName,
             SourceLocation sourceLoc) {
-        String defaultDataverseName = metadataProvider.getDefaultDataverseName();
+        DataverseName defaultDataverseName = metadataProvider.getDefaultDataverseName();
         if (dataverseName == null && defaultDataverseName == null) {
             return new CompilationException(ErrorCode.NAME_RESOLVE_UNKNOWN_DATASET, sourceLoc, datasetName);
         }
@@ -177,30 +222,15 @@ public class VariableCheckAndRewriteVisitor extends AbstractSqlppExpressionScopi
                 dataverseName == null ? defaultDataverseName : dataverseName);
     }
 
-    private Dataset findDataset(String dataverseName, String datasetName, SourceLocation sourceLoc)
+    private Dataset findDataset(DataverseName dataverseName, String datasetName, SourceLocation sourceLoc)
             throws CompilationException {
         try {
-            Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
-            if (dataset != null) {
-                return dataset;
-            }
-            return findDatasetByFullyQualifiedName(datasetName);
+            return metadataProvider.findDataset(dataverseName, datasetName);
         } catch (AlgebricksException e) {
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, e, sourceLoc, e.getMessage());
         }
     }
 
-    private Dataset findDatasetByFullyQualifiedName(String name) throws AlgebricksException {
-        if (name.indexOf('.') < 0) {
-            return null;
-        }
-        String[] path = StringUtils.split(name, '.');
-        if (path.length != 2) {
-            return null;
-        }
-        return metadataProvider.findDataset(path[0], path[1]);
-    }
-
     @Override
     public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
         // skip variables inside SQL-92 aggregates (they will be resolved by SqlppGroupByAggregationSugarVisitor)
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
index 8df2616..b092f4a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/FunctionMapUtil.java
@@ -100,7 +100,7 @@ public class FunctionMapUtil {
         String name = applySql92AggregateNameMapping(fs.getName().toLowerCase());
         String prefix =
                 CORE_AGGREGATE_PREFIX_FUNCTIONS.contains(name) ? CORE_AGGREGATE_PREFIX : CORE_SQL_AGGREGATE_PREFIX;
-        return new FunctionSignature(FunctionConstants.ASTERIX_NS, prefix + name, fs.getArity());
+        return new FunctionSignature(FunctionConstants.ASTERIX_DV, prefix + name, fs.getArity());
     }
 
     /**
@@ -142,7 +142,7 @@ public class FunctionMapUtil {
             FunctionIdentifier fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, internalName, ns.getArity());
             IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
             if (finfo != null && BuiltinFunctions.getAggregateFunction(finfo.getFunctionIdentifier()) != null) {
-                return new FunctionSignature(FunctionConstants.ASTERIX_NS, internalName, ns.getArity());
+                return new FunctionSignature(FunctionConstants.ASTERIX_DV, internalName, ns.getArity());
             }
         } else if (checkSql92Aggregate) {
             if (isSql92AggregateFunction(ns)) {
@@ -154,7 +154,7 @@ public class FunctionMapUtil {
                 throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_EXPRESSION, sourceLoc);
             }
         }
-        return new FunctionSignature(ns.getNamespace(), ns.getName(), ns.getArity());
+        return new FunctionSignature(ns.getDataverseName(), ns.getName(), ns.getArity());
     }
 
     /**
@@ -170,7 +170,7 @@ public class FunctionMapUtil {
         if (internalFuncName == null) {
             return callExpr;
         }
-        callExpr.setFunctionSignature(new FunctionSignature(FunctionConstants.ASTERIX_NS, internalFuncName, 1));
+        callExpr.setFunctionSignature(new FunctionSignature(FunctionConstants.ASTERIX_DV, internalFuncName, 1));
         ListConstructor listConstr =
                 new ListConstructor(ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR, callExpr.getExprList());
         listConstr.setSourceLocation(callExpr.getSourceLocation());
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
index e41b9ac..42f62ba 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/util/SqlppStatementUtil.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.lang.sqlpp.util;
 
+import java.util.List;
+
+import org.apache.asterix.common.metadata.DataverseName;
+
 public class SqlppStatementUtil {
 
     private static final String IF_EXISTS = " IF EXISTS ";
@@ -45,15 +49,15 @@ public class SqlppStatementUtil {
     }
 
     @SuppressWarnings("squid:S1172") // unused variable
-    public static StringBuilder getCreateDataverseStatement(StringBuilder stringBuilder, String dataverseName,
+    public static StringBuilder getCreateDataverseStatement(StringBuilder stringBuilder, DataverseName dataverseName,
             boolean ifNotExists, int version) {
         stringBuilder.append(CREATE_DATAVERSE);
-        enclose(stringBuilder, dataverseName);
+        encloseDataverseName(stringBuilder, dataverseName);
         return ifNotExists(stringBuilder, ifNotExists).append(SEMI_COLON);
     }
 
     @SuppressWarnings("squid:S1172") // unused variable
-    public static StringBuilder getDropDatasetStatement(StringBuilder stringBuilder, String dataverseName,
+    public static StringBuilder getDropDatasetStatement(StringBuilder stringBuilder, DataverseName dataverseName,
             String datasetName, boolean ifExists, int version) {
         stringBuilder.append(DROP_DATASET);
         enclose(stringBuilder, dataverseName, datasetName);
@@ -61,7 +65,7 @@ public class SqlppStatementUtil {
     }
 
     @SuppressWarnings("squid:S1172") // unused variable
-    public static StringBuilder getCreateIndexStatement(StringBuilder stringBuilder, String dataverseName,
+    public static StringBuilder getCreateIndexStatement(StringBuilder stringBuilder, DataverseName dataverseName,
             String datasetName, String indexName, String fields, int version) {
         stringBuilder.append(CREATE_INDEX);
         enclose(stringBuilder, indexName).append(ON);
@@ -69,7 +73,7 @@ public class SqlppStatementUtil {
     }
 
     @SuppressWarnings("squid:S1172") // unused variable
-    public static StringBuilder getCreatePrimaryIndexStatement(StringBuilder stringBuilder, String dataverseName,
+    public static StringBuilder getCreatePrimaryIndexStatement(StringBuilder stringBuilder, DataverseName dataverseName,
             String datasetName, String indexName, int version) {
         stringBuilder.append(CREATE_PRIMARY_INDEX);
         enclose(stringBuilder, indexName).append(ON);
@@ -77,7 +81,7 @@ public class SqlppStatementUtil {
     }
 
     @SuppressWarnings("squid:S1172") // unused variable
-    public static StringBuilder getDropIndexStatement(StringBuilder stringBuilder, String dataverseName,
+    public static StringBuilder getDropIndexStatement(StringBuilder stringBuilder, DataverseName dataverseName,
             String datasetName, String indexName, boolean ifExists, int version) {
         stringBuilder.append(DROP_INDEX);
         enclose(stringBuilder, dataverseName, datasetName, indexName);
@@ -103,30 +107,47 @@ public class SqlppStatementUtil {
     }
 
     /**
-     * Same as {@link SqlppStatementUtil#enclose(StringBuilder, String)} but for a qualified identifier.
+     * Encloses each part of the {@param dataverseName} in back-ticks and concatenates them with
+     * {@link DataverseName#SEPARATOR_CHAR} separator
+     * @param stringBuilder where the dataverse name will be appended
+     * @param dataverseName a dataverse name which could be a valid one or one that needs to be delimited
+     * @return {@param stringBuilder} with the <i>delimited</i> dataverseName appended
+     */
+    public static StringBuilder encloseDataverseName(StringBuilder stringBuilder, DataverseName dataverseName) {
+        List<String> parts = dataverseName.getParts();
+        for (int i = 0, ln = parts.size(); i < ln; i++) {
+            if (i > 0) {
+                stringBuilder.append(DataverseName.SEPARATOR_CHAR);
+            }
+            enclose(stringBuilder, parts.get(i));
+        }
+        return stringBuilder;
+    }
+
+    /**
+     * Encloses a dataverse name and a given idenfitier.
      * @param stringBuilder where the identifier will be appended
-     * @param identifier1 the qualifying identifier
-     * @param identifier2 the qualified identifier
+     * @param dataverseName the dataverse name
+     * @param identifier the identifier
      * @return {@param stringBuilder} with the <i>delimited</i> qualified identifier appended
      */
-    public static StringBuilder enclose(StringBuilder stringBuilder, String identifier1, String identifier2) {
-        return stringBuilder.append(BACK_TICK).append(identifier1).append(BACK_TICK).append(DOT).append(BACK_TICK)
-                .append(identifier2).append(BACK_TICK);
+    public static StringBuilder enclose(StringBuilder stringBuilder, DataverseName dataverseName, String identifier) {
+        encloseDataverseName(stringBuilder, dataverseName).append(DOT);
+        return enclose(stringBuilder, identifier);
     }
 
     /**
      * Same as {@link SqlppStatementUtil#enclose(StringBuilder, String)} but for a double qualified identifier.
      * @param stringBuilder where the identifier will be appended
-     * @param identifier1 the 1st qualifying identifier
-     * @param identifier2 the 2nd qualifying identifier
-     * @param identifier3 the qualified identifier
+     * @param dataverseName the 1st qualifying identifier
+     * @param identifier1 the 2nd qualifying identifier
+     * @param identifier2 the qualified identifier
      * @return {@param stringBuilder} with the <i>delimited</i> qualified identifier appended
      */
-    public static StringBuilder enclose(StringBuilder stringBuilder, String identifier1, String identifier2,
-            String identifier3) {
-        return stringBuilder.append(BACK_TICK).append(identifier1).append(BACK_TICK).append(DOT).append(BACK_TICK)
-                .append(identifier2).append(BACK_TICK).append(DOT).append(BACK_TICK).append(identifier3)
-                .append(BACK_TICK);
+    public static StringBuilder enclose(StringBuilder stringBuilder, DataverseName dataverseName, String identifier1,
+            String identifier2) {
+        enclose(stringBuilder, dataverseName, identifier1).append(DOT);
+        return enclose(stringBuilder, identifier2);
     }
 
     public static String enclose(String identifier) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index a3b752b..33446de 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -19,8 +19,10 @@
 package org.apache.asterix.lang.sqlpp.visitor;
 
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.functions.FunctionSignature;
@@ -32,8 +34,11 @@ import org.apache.asterix.lang.common.clause.OrderbyClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
 import org.apache.asterix.lang.common.expression.ListSliceExpression;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.QueryPrintVisitor;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -84,7 +89,7 @@ public class SqlppAstPrintVisitor extends QueryPrintVisitor implements ISqlppVis
         out.print(skip(step) + "AS ");
         fromTerm.getLeftVariable().accept(this, 0);
         if (fromTerm.hasPositionalVariable()) {
-            out.println(" AT ");
+            out.println(" AT");
             fromTerm.getPositionalVariable().accept(this, 0);
         }
         if (fromTerm.hasCorrelateClauses()) {
@@ -252,11 +257,15 @@ public class SqlppAstPrintVisitor extends QueryPrintVisitor implements ISqlppVis
         if (BuiltinFunctions.isBuiltinCompilerFunction(normalizedFunctionSignature, true)) {
             functionSignature = normalizedFunctionSignature;
         }
-        out.println(skip(step) + "FunctionCall " + functionSignature.toString() + "[");
-        for (Expression expr : pf.getExprList()) {
-            expr.accept(this, step + 1);
+        //TODO(MULTI_PART_DATAVERSE_NAME):temporary workaround to preserve AST reference results
+        if (FunctionUtil.isBuiltinDatasetFunction(functionSignature)) {
+            String singleArg = pf.getExprList().stream().map(LiteralExpr.class::cast).map(LiteralExpr::getValue)
+                    .map(StringLiteral.class::cast).map(StringLiteral::getValue).collect(Collectors.joining("."));
+            printFunctionCall(functionSignature, 1,
+                    Collections.singletonList(new LiteralExpr(new StringLiteral(singleArg))), step);
+        } else {
+            printFunctionCall(functionSignature, functionSignature.getArity(), pf.getExprList(), step);
         }
-        out.println(skip(step) + "]");
         return null;
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
index 8cfd04b..966d153 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppDeleteRewriteVisitor.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.clause.WhereClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -31,7 +32,6 @@ import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.statement.DeleteStatement;
 import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
 import org.apache.asterix.lang.sqlpp.clause.FromTerm;
 import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
@@ -41,6 +41,7 @@ import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
 import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
 import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
 import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppAstVisitor;
+import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 
 /**
@@ -49,15 +50,22 @@ import org.apache.asterix.om.functions.BuiltinFunctions;
  */
 public class SqlppDeleteRewriteVisitor extends AbstractSqlppAstVisitor<Void, Void> {
 
+    private final MetadataProvider metadataProvider;
+
+    public SqlppDeleteRewriteVisitor(MetadataProvider metadataProvider) {
+        this.metadataProvider = metadataProvider;
+    }
+
     @Override
     public Void visit(DeleteStatement deleteStmt, Void visitArg) {
         List<Expression> arguments = new ArrayList<>();
-        Identifier dataverseName = deleteStmt.getDataverseName();
-        Identifier datasetName = deleteStmt.getDatasetName();
-        String arg = dataverseName == null ? datasetName.getValue()
-                : dataverseName.getValue() + "." + datasetName.getValue();
-        LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
-        arguments.add(argumentLiteral);
+        DataverseName dataverseName = deleteStmt.getDataverseName();
+        if (dataverseName == null) {
+            dataverseName = metadataProvider.getDefaultDataverseName();
+        }
+        String datasetName = deleteStmt.getDatasetName();
+        arguments.add(new LiteralExpr(new StringLiteral(dataverseName.getCanonicalForm())));
+        arguments.add(new LiteralExpr(new StringLiteral(datasetName)));
         CallExpr callExpression = new CallExpr(new FunctionSignature(BuiltinFunctions.DATASET), arguments);
         callExpression.setSourceLocation(deleteStmt.getSourceLocation());
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
index e541363..9901c8c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppFormatPrintVisitor.java
@@ -330,7 +330,7 @@ public class SqlppFormatPrintVisitor extends FormatPrintVisitor implements ISqlp
     @Override
     public Void visit(WindowExpression windowExpr, Integer step) throws CompilationException {
         out.print(skip(step) + "window ");
-        out.print(generateFullName(windowExpr.getFunctionSignature().getNamespace(),
+        out.print(generateFullName(windowExpr.getFunctionSignature().getDataverseName(),
                 windowExpr.getFunctionSignature().getName()) + "(");
         printDelimitedExpressions(windowExpr.getExprList(), COMMA, step);
         out.print(")");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 38ace29..13cebb6 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -70,6 +70,7 @@ import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.common.exceptions.WarningUtil;
 import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.AbstractLangExpression;
 import org.apache.asterix.lang.common.base.AbstractStatement;
@@ -225,6 +226,8 @@ class SQLPPParser extends ScopeChecker implements IParser {
 
     private int externalVarCounter;
 
+    private DataverseName defaultDataverse;
+
     private final WarningCollector warningCollector = new WarningCollector();
 
     private final Map<SourceLocation, String> hintCollector = new HashMap<SourceLocation, String>();
@@ -240,7 +243,7 @@ class SQLPPParser extends ScopeChecker implements IParser {
     };
 
     private static class FunctionName {
-       public String dataverse;
+       public DataverseName dataverse;
        public String library;
        public String function;
        public SqlppHint hint;
@@ -556,13 +559,13 @@ Statement SingleStatement() throws ParseException:
 DataverseDecl DataverseDeclaration() throws ParseException:
 {
   Token startToken = null;
-  String dvName = null;
+  List<String> dvName = null;
 }
 {
-  <USE> { startToken = token; } dvName = Identifier()
+  <USE> { startToken = token; } dvName = MultipartIdentifier()
     {
-      defaultDataverse = dvName;
-      DataverseDecl dvDecl = new DataverseDecl(new Identifier(dvName));
+      defaultDataverse = DataverseName.create(dvName);
+      DataverseDecl dvDecl = new DataverseDecl(defaultDataverse);
       return addSourceLocation(dvDecl, startToken);
     }
 }
@@ -591,7 +594,7 @@ Statement CreateStatement() throws ParseException:
 
 TypeDecl TypeSpecification(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   TypeExpression typeExpr = null;
 }
@@ -647,9 +650,9 @@ NodegroupDecl NodegroupSpecification(Token startStmtToken) throws ParseException
 
 DatasetDecl DatasetSpecification(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
-  Pair<Identifier,Identifier> typeComponents = null;
+  Pair<DataverseName,Identifier> typeComponents = null;
   String adapterName = null;
   Map<String,String> properties = null;
   FunctionSignature appliedFunction = null;
@@ -659,7 +662,7 @@ DatasetDecl DatasetSpecification(Token startStmtToken) throws ParseException:
   DatasetDecl stmt = null;
   boolean autogenerated = false;
   Pair<Integer, List<String>> filterField = null;
-  Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
+  Pair<DataverseName,Identifier> metaTypeComponents = new Pair<DataverseName, Identifier>(null, null);
   RecordConstructor withRecord = null;
 }
 {
@@ -750,7 +753,7 @@ DatasetDecl DatasetSpecification(Token startStmtToken) throws ParseException:
 RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
 {
   Token startToken = null;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   String datasetName = null;
 }
 {
@@ -768,7 +771,7 @@ CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseExcept
   CreateIndexStatement stmt = new CreateIndexStatement();
   String indexName = null;
   boolean ifNotExists = false;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   Pair<Integer, Pair<List<String>, IndexedTypeExpression>> fieldPair = null;
   IndexParams indexType = null;
   boolean enforced = false;
@@ -872,14 +875,14 @@ IndexParams IndexType() throws ParseException:
 
 CreateDataverseStatement DataverseSpecification(Token startStmtToken) throws ParseException :
 {
-  String dvName = null;
+  List<String> dvName = null;
   boolean ifNotExists = false;
 }
 {
-  <DATAVERSE> dvName = Identifier()
+  <DATAVERSE> dvName = MultipartIdentifier()
   ifNotExists = IfNotExists()
     {
-      CreateDataverseStatement stmt = new CreateDataverseStatement(new Identifier(dvName), null, ifNotExists);
+      CreateDataverseStatement stmt = new CreateDataverseStatement(DataverseName.create(dvName), null, ifNotExists);
       return addSourceLocation(stmt, startStmtToken);
     }
 }
@@ -895,7 +898,7 @@ CreateFunctionStatement FunctionSpecification(Token startStmtToken) throws Parse
   Token beginPos;
   Token endPos;
   FunctionName fctName = null;
-  String currentDataverse = defaultDataverse;
+  DataverseName currentDataverse = defaultDataverse;
 
   createNewScope();
 }
@@ -927,7 +930,7 @@ CreateFunctionStatement FunctionSpecification(Token startStmtToken) throws Parse
 
 CreateFeedStatement FeedSpecification(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   boolean ifNotExists = false;
   String adapterName = null;
   Map<String,String> properties = null;
@@ -1016,20 +1019,20 @@ boolean IfNotExists() throws ParseException:
 
 void ApplyFunction(List<FunctionSignature> funcSigs) throws ParseException:
 {
-  FunctionName functioName = null;
+  FunctionName functionName = null;
   String fqFunctionName = null;
 }
 {
-  <APPLY> <FUNCTION> functioName = FunctionName()
+  <APPLY> <FUNCTION> functionName = FunctionName()
     {
-       fqFunctionName = functioName.library == null ? functioName.function : functioName.library + "#" + functioName.function;
-       funcSigs.add(new FunctionSignature(functioName.dataverse, fqFunctionName, 1));
+       fqFunctionName = functionName.library == null ? functionName.function : functionName.library + "#" + functionName.function;
+       funcSigs.add(new FunctionSignature(functionName.dataverse, fqFunctionName, 1));
     }
   (
-      <COMMA> functioName = FunctionName()
+      <COMMA> functionName = FunctionName()
       {
-        fqFunctionName = functioName.library == null ? functioName.function : functioName.library + "#" + functioName.function;
-        funcSigs.add(new FunctionSignature(functioName.dataverse, fqFunctionName, 1));
+        fqFunctionName = functionName.library == null ? functionName.function : functionName.library + "#" + functionName.function;
+        funcSigs.add(new FunctionSignature(functionName.dataverse, fqFunctionName, 1));
       }
   )*
 }
@@ -1092,8 +1095,9 @@ Statement DropStatement() throws ParseException:
 {
   Token startToken = null;
   String id = null;
-  Pair<Identifier,Identifier> pairId = null;
-  Triple<Identifier,Identifier,Identifier> tripleId = null;
+  List<String> multipartId = null;
+  Pair<DataverseName,Identifier> pairId = null;
+  Triple<DataverseName,Identifier,Identifier> tripleId = null;
   FunctionSignature funcSig = null;
   boolean ifExists = false;
   AbstractStatement stmt = null;
@@ -1117,9 +1121,9 @@ Statement DropStatement() throws ParseException:
       {
         stmt = new TypeDropStatement(pairId.first, pairId.second, ifExists);
       }
-    | <DATAVERSE> id = Identifier() ifExists = IfExists()
+    | <DATAVERSE> multipartId = MultipartIdentifier() ifExists = IfExists()
       {
-        stmt = new DataverseDropStatement(new Identifier(id), ifExists);
+        stmt = new DataverseDropStatement(DataverseName.create(multipartId), ifExists);
       }
     | <FUNCTION> funcSig = FunctionSignature() ifExists = IfExists()
       {
@@ -1156,7 +1160,7 @@ boolean IfExists() throws ParseException :
 InsertStatement InsertStatement() throws ParseException:
 {
   Token startToken = null;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
@@ -1180,7 +1184,7 @@ InsertStatement InsertStatement() throws ParseException:
 UpsertStatement UpsertStatement() throws ParseException:
 {
   Token startToken = null;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
   VariableExpr var = null;
   Query query = null;
   Expression returnExpression = null;
@@ -1206,7 +1210,7 @@ DeleteStatement DeleteStatement() throws ParseException:
   Token startToken = null;
   VariableExpr var = null;
   Expression condition = null;
-  Pair<Identifier, Identifier> nameComponents;
+  Pair<DataverseName, Identifier> nameComponents;
 }
 {
   <DELETE> { startToken = token; }
@@ -1310,12 +1314,12 @@ Statement WriteStatement() throws ParseException:
 LoadStatement LoadStatement() throws ParseException:
 {
   Token startToken = null;
-  Identifier dataverseName = null;
+  DataverseName dataverseName = null;
   Identifier datasetName = null;
   boolean alreadySorted = false;
   String adapterName;
   Map<String,String> properties;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
 }
 {
   <LOAD> { startToken = token; } Dataset() nameComponents = QualifiedName()
@@ -1350,7 +1354,7 @@ String AdapterName() throws ParseException :
 Statement CompactStatement() throws ParseException:
 {
   Token startToken = null;
-  Pair<Identifier,Identifier> nameComponents = null;
+  Pair<DataverseName,Identifier> nameComponents = null;
 }
 {
   <COMPACT> { startToken = token; } Dataset() nameComponents = QualifiedName()
@@ -1379,7 +1383,7 @@ Statement ConnectionStatement() throws ParseException:
 
 Statement StartStatement(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
+  Pair<DataverseName,Identifier> feedNameComponents = null;
 
   AbstractStatement stmt = null;
 }
@@ -1393,7 +1397,7 @@ Statement StartStatement(Token startStmtToken) throws ParseException:
 
 AbstractStatement StopStatement(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
+  Pair<DataverseName,Identifier> feedNameComponents = null;
 
   AbstractStatement stmt = null;
 }
@@ -1407,8 +1411,8 @@ AbstractStatement StopStatement(Token startStmtToken) throws ParseException:
 
 AbstractStatement DisconnectStatement(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
-  Pair<Identifier,Identifier> datasetNameComponents = null;
+  Pair<DataverseName,Identifier> feedNameComponents = null;
+  Pair<DataverseName,Identifier> datasetNameComponents = null;
 
   AbstractStatement stmt = null;
 }
@@ -1426,8 +1430,8 @@ AbstractStatement DisconnectStatement(Token startStmtToken) throws ParseExceptio
 
 AbstractStatement ConnectStatement(Token startStmtToken) throws ParseException:
 {
-  Pair<Identifier,Identifier> feedNameComponents = null;
-  Pair<Identifier,Identifier> datasetNameComponents = null;
+  Pair<DataverseName,Identifier> feedNameComponents = null;
+  Pair<DataverseName,Identifier> datasetNameComponents = null;
 
   Map<String,String> configuration = null;
   List<FunctionSignature> appliedFunctions = new ArrayList<FunctionSignature>();
@@ -1643,7 +1647,7 @@ void RecordField(RecordTypeDefinition recType) throws ParseException:
 
 TypeReferenceExpression TypeReference() throws ParseException:
 {
-  Pair<Identifier,Identifier> id = null;
+  Pair<DataverseName,Identifier> id = null;
 }
 {
   id = QualifiedName()
@@ -1689,64 +1693,54 @@ UnorderedListTypeDefinition UnorderedListTypeDef() throws ParseException:
 
 FunctionName FunctionName() throws ParseException:
 {
-  String first = null;
-  String second = null;
-  String third = null;
-  boolean secondAfterDot = false;
+  Triple<List<String>, SourceLocation, SqlppHint> prefix = null;
+  String suffix = null;
 }
 {
-  first = Identifier()
+  // Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
+  //       that copy must be kept in sync with this code
+  prefix = MultipartIdentifierWithHints(SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
+    SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT)
+  (<SHARP> suffix = Identifier())?
   {
     FunctionName result = new FunctionName();
-    result.sourceLoc = getSourceLocation(token);
-    Token hintToken = fetchHint(token, SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
-      SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT);
-    if (hintToken != null) {
-      result.hint = hintToken.hint;
+    result.sourceLoc = prefix.second;
+    result.hint = prefix.third;
+    List<String> list = prefix.first;
+    int ln = list.size();
+    String last = list.get(ln - 1);
+    if (suffix == null) {
+      // prefix = (dv_part1.dv_part2...dv_partN.)?func_name
+      // no library name
+      result.function = last;
+    } else {
+      // prefix = (dv_part1.dv_part2...dv_partN.)?lib_name
+      // suffix = func_name
+      result.library = last;
+      result.function = suffix;
     }
-  }
-  ( <DOT> second = Identifier()
-    {
-      secondAfterDot = true;
+    if (ln > 1) {
+      result.dataverse = DataverseName.create(list, 0, ln - 1);
+    } else {
+      result.dataverse = defaultDataverse;
     }
-  (<SHARP> third = Identifier())? | <SHARP> second = Identifier() )?
-    {
-      if (second == null) {
-        result.dataverse = defaultDataverse;
-        result.library = null;
-        result.function = first;
-      } else if (third == null) {
-        if (secondAfterDot) {
-          result.dataverse = first;
-          result.library   = null;
-          result.function = second;
-        } else {
-          result.dataverse = defaultDataverse;
-          result.library   = first;
-          result.function = second;
-        }
-      } else {
-        result.dataverse = first;
-        result.library   = second;
-        result.function  = third;
-      }
 
-      if (result.function.equalsIgnoreCase(INT_TYPE_NAME)) {
-         result.function = BuiltinType.AINT64.getTypeName();
-      }
-      return result;
+    if (result.function.equalsIgnoreCase(INT_TYPE_NAME)) {
+       result.function = BuiltinType.AINT64.getTypeName();
     }
+    return result;
+  }
 }
 
-Pair<Identifier,Identifier> TypeName() throws ParseException:
+Pair<DataverseName,Identifier> TypeName() throws ParseException:
 {
-  Pair<Identifier,Identifier> name = null;
+  Pair<DataverseName,Identifier> name = null;
 }
 {
   name = QualifiedName()
     {
       if (name.first == null) {
-        name.first = new Identifier(defaultDataverse);
+        name.first = defaultDataverse;
       }
       return name;
     }
@@ -1860,48 +1854,71 @@ String StringLiteral() throws ParseException:
     }
 }
 
-Pair<Identifier,Identifier> QualifiedName() throws ParseException:
+List<String> MultipartIdentifier() throws ParseException:
 {
-  String first = null;
-  String second = null;
+  Triple<List<String>, SourceLocation, SqlppHint> result = null;
 }
 {
-  first = Identifier() (<DOT> second = Identifier())?
+  result = MultipartIdentifierWithHints(null)
   {
-    Identifier id1 = null;
-    Identifier id2 = null;
-    if (second == null) {
-      id2 = new Identifier(first);
-    } else
-    {
-      id1 = new Identifier(first);
-      id2 = new Identifier(second);
-    }
-    return new Pair<Identifier,Identifier>(id1, id2);
+    return result.first;
   }
 }
 
-Triple<Identifier,Identifier,Identifier> DoubleQualifiedName() throws ParseException:
+Triple<List<String>, SourceLocation, SqlppHint> MultipartIdentifierWithHints(SqlppHint... expectedHints)
+  throws ParseException:
 {
-  String first = null;
-  String second = null;
-  String third = null;
+  List<String> list = new ArrayList<String>();
+  SourceLocation sourceLoc = null;
+  SqlppHint hint = null;
+  String item = null;
 }
 {
-  first = Identifier() <DOT> second = Identifier() (<DOT> third = Identifier())?
+  item = Identifier()
   {
-    Identifier id1 = null;
-    Identifier id2 = null;
-    Identifier id3 = null;
-    if (third == null) {
-      id2 = new Identifier(first);
-      id3 = new Identifier(second);
-    } else {
-      id1 = new Identifier(first);
-      id2 = new Identifier(second);
-      id3 = new Identifier(third);
+    list.add(item);
+    sourceLoc = getSourceLocation(token);
+    if (expectedHints != null && expectedHints.length > 0) {
+      Token hintToken = fetchHint(token, expectedHints);
+      if (hintToken != null) {
+        hint = hintToken.hint;
+      }
     }
-    return new Triple<Identifier,Identifier,Identifier>(id1, id2, id3);
+  }
+  (<DOT> item = Identifier() { list.add(item); } )*
+  {
+    return new Triple<List<String>, SourceLocation, SqlppHint>(list, sourceLoc, hint);
+  }
+}
+
+Pair<DataverseName,Identifier> QualifiedName() throws ParseException:
+{
+  List<String> list = null;
+}
+{
+  list = MultipartIdentifier()
+  {
+    int len = list.size();
+    DataverseName id1 = len > 1 ? DataverseName.create(list, 0, len - 1) : null;
+    Identifier id2 = new Identifier(list.get(len - 1));
+    return new Pair<DataverseName,Identifier>(id1, id2);
+  }
+}
+
+Triple<DataverseName, Identifier, Identifier> DoubleQualifiedName() throws ParseException:
+{
+  List<String> list = new ArrayList<String>();
+  String item = null;
+}
+{
+  item = Identifier() { list.add(item); }
+  (<DOT> item = Identifier() { list.add(item); } )+
+  {
+    int len = list.size();
+    DataverseName id1 = len > 2 ? DataverseName.create(list, 0, len - 2) : null;
+    Identifier id2 = new Identifier(list.get(len - 2));
+    Identifier id3 = new Identifier(list.get(len - 1));
+    return new Triple<DataverseName,Identifier,Identifier>(id1, id2, id3);
   }
 }
 
@@ -2495,8 +2512,8 @@ Expression PrimaryExpr() throws ParseException:
   Expression expr = null;
 }
 {
-  ( LOOKAHEAD(4)
-    expr = FunctionCallExpr()
+  (
+    LOOKAHEAD(Identifier() (<DOT> Identifier())* (<SHARP> Identifier())? <LEFTPAREN>) expr = FunctionCallExpr()
   | expr = CaseExpr()
   | expr = Literal()
   | expr = VariableRef()
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 0cd03d7..9d738ed 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
@@ -20,12 +20,14 @@
 package org.apache.asterix.metadata;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.functions.FunctionSignature;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.metadata.api.IMetadataEntity;
 import org.apache.asterix.metadata.entities.CompactionPolicy;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -50,36 +52,36 @@ import org.apache.asterix.metadata.utils.IndexUtil;
 public class MetadataCache {
 
     // Key is dataverse name.
-    protected final Map<String, Dataverse> dataverses = new HashMap<>();
+    protected final Map<DataverseName, Dataverse> dataverses = new HashMap<>();
     // Key is dataverse name. Key of value map is dataset name.
-    protected final Map<String, Map<String, Dataset>> datasets = new HashMap<>();
+    protected final Map<DataverseName, Map<String, Dataset>> datasets = 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<String, Map<String, Map<String, Index>>> indexes = new HashMap<>();
+    protected final Map<DataverseName, Map<String, Map<String, Index>>> indexes = new HashMap<>();
     // Key is dataverse name. Key of value map is datatype name.
-    protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<>();
-    // Key is dataverse name.
+    protected final Map<DataverseName, Map<String, Datatype>> datatypes = new HashMap<>();
+    // Key is node group name.
     protected final Map<String, NodeGroup> nodeGroups = new HashMap<>();
     // Key is function Identifier . Key of value map is function name.
     protected final Map<FunctionSignature, Function> functions = new HashMap<>();
-    // Key is adapter dataverse. Key of value map is the adapter name
-    protected final Map<String, Map<String, DatasourceAdapter>> adapters = new HashMap<>();
+    // Key is adapter dataverse name. Key of value map is the adapter name
+    protected final Map<DataverseName, Map<String, DatasourceAdapter>> adapters = new HashMap<>();
 
     // Key is DataverseName, Key of the value map is the Policy name
-    protected final Map<String, Map<String, FeedPolicyEntity>> feedPolicies = new HashMap<>();
+    protected final Map<DataverseName, Map<String, FeedPolicyEntity>> feedPolicies = new HashMap<>();
     // Key is library dataverse. Key of value map is the library name
-    protected final Map<String, Map<String, Library>> libraries = new HashMap<>();
+    protected final Map<DataverseName, Map<String, Library>> libraries = new HashMap<>();
     // Key is library dataverse. Key of value map is the feed name
-    protected final Map<String, Map<String, Feed>> feeds = new HashMap<>();
+    protected final Map<DataverseName, Map<String, Feed>> feeds = new HashMap<>();
     // Key is DataverseName, Key of the value map is the Policy name
-    protected final Map<String, Map<String, CompactionPolicy>> compactionPolicies = new HashMap<>();
+    protected final Map<DataverseName, Map<String, CompactionPolicy>> compactionPolicies = new HashMap<>();
     // Key is DataverseName, Key of value map is feedConnectionId
-    protected final Map<String, Map<String, FeedConnection>> feedConnections = new HashMap<>();
+    protected final Map<DataverseName, Map<String, FeedConnection>> feedConnections = new HashMap<>();
 
     // Atomically executes all metadata operations in ctx's log.
     public void commit(MetadataTransactionContext ctx) {
         // Forward roll the operations written in ctx's log.
         int logIx = 0;
-        ArrayList<MetadataLogicalOperation> opLog = ctx.getOpLog();
+        List<MetadataLogicalOperation> opLog = ctx.getOpLog();
         try {
             for (logIx = 0; logIx < opLog.size(); logIx++) {
                 doOperation(opLog.get(logIx));
@@ -135,11 +137,12 @@ public class MetadataCache {
         synchronized (dataverses) {
... 8001 lines suppressed ...