You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/02/27 07:22:05 UTC

[4/4] incubator-asterixdb git commit: Add DDL support for optionally associating meta record with dataset record.

Add DDL support for optionally associating meta record with dataset record.

-Fixed the TestExecutor.runScriptAndCompareWithResult() to compare fields
 after "Time"-prefixed fields.

Change-Id: I613316985dbd7f553d946f95c5b1a1df56914902
Reviewed-on: https://asterix-gerrit.ics.uci.edu/666
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/commit/b9169b66
Tree: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/tree/b9169b66
Diff: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/diff/b9169b66

Branch: refs/heads/master
Commit: b9169b66c48b57bb33df917468848427ef72b418
Parents: 89dfcee
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Fri Feb 26 21:21:49 2016 -0800
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Fri Feb 26 22:17:08 2016 -0800

----------------------------------------------------------------------
 .../physical/ExternalDataLookupPOperator.java   |   2 +-
 .../optimizer/rules/UnnestToDataScanRule.java   |  45 +++--
 .../optimizer/rules/am/BTreeAccessMethod.java   |  22 +--
 .../rules/am/OptimizableOperatorSubTree.java    |   6 +
 .../rules/util/EquivalenceClassUtils.java       |  27 ++-
 .../LangExpressionToPlanTranslator.java         |  11 +-
 .../asterix/translator/util/ValidateUtil.java   |  21 +-
 .../asterix/aql/translator/QueryTranslator.java |  38 +++-
 .../dataset_with_meta-1.1.ddl.aql               |  35 ++++
 .../dataset_with_meta-1.2.update.aql            |  19 ++
 .../dataset_with_meta-1.3.query.aql             |  22 +++
 .../dataset_with_meta-1.4.ddl.aql               |  20 ++
 .../dataset_with_meta-2.1.ddl.aql               |  39 ++++
 .../dataset_with_meta-2.2.update.aql            |  19 ++
 .../dataset_with_meta-2.3.query.aql             |  22 +++
 .../dataset_with_meta-2.4.ddl.aql               |  21 ++
 .../dataset_with_meta-3.1.ddl.aql               |  35 ++++
 .../dataset_with_meta-3.2.update.aql            |  19 ++
 .../dataset_with_meta-3.3.query.aql             |  22 +++
 .../dataset_with_meta-3.4.ddl.aql               |  20 ++
 .../dataset_with_meta-4.1.ddl.aql               |  32 +++
 .../dataset_with_meta-4.2.update.aql            |  19 ++
 .../dataset_with_meta-4.3.query.aql             |  22 +++
 .../dataset_with_meta-4.4.ddl.aql               |  20 ++
 .../dataset_with_meta-5.1.ddl.aql               |  36 ++++
 .../dataset_with_meta-5.2.update.aql            |  19 ++
 .../dataset_with_meta-5.3.query.aql             |  22 +++
 .../dataset_with_meta-5.4.ddl.aql               |  20 ++
 .../dataset_with_meta-1.1.adm                   |   1 +
 .../dataset_with_meta-2.1.adm                   |   1 +
 .../dataset_with_meta-2.3.adm                   |   1 +
 .../dataset_with_meta-4.3.adm                   |   1 +
 .../dataset_with_meta-5.3.adm                   |   1 +
 .../metadata/results/basic/meta01/meta01.1.adm  |   4 +-
 .../metadata/results/basic/meta05/meta05.1.adm  |   4 +-
 .../metadata/results/basic/meta12/meta12.1.adm  |   2 +-
 .../metadata/results/basic/meta16/meta16.1.adm  |   4 +-
 .../metadata/results/basic/meta17/meta17.1.adm  |   4 +-
 .../metadata_dataverse/metadata_dataverse.1.adm |   6 +-
 .../verify_failure_previous_success.adm         |  13 --
 .../verify_failure_previous_success.1.adm       |   2 +-
 .../verify_failure_subsequent_no_execution.adm  |   1 -
 ...verify_failure_subsequent_no_execution.1.adm |   2 +-
 .../src/test/resources/metadata/testsuite.xml   |  25 +++
 .../query_dataset_with_meta-5-1.1.ddl.aql       |  34 ++++
 .../query_dataset_with_meta-5-1.2.update.aql    |  19 ++
 .../query_dataset_with_meta-5-1.3.query.aql     |  23 +++
 .../query_dataset_with_meta-5-1.4.ddl.aql       |  20 ++
 .../query_dataset_with_meta-5-2.1.ddl.aql       |  34 ++++
 .../query_dataset_with_meta-5-2.2.update.aql    |  19 ++
 .../query_dataset_with_meta-5-2.3.query.aql     |  23 +++
 .../query_dataset_with_meta-5-2.4.ddl.aql       |  20 ++
 .../feed-with-external-parser.1.adm             | 198 +++++++++----------
 .../query_dataset_with_meta-1.1.adm             |   0
 .../query_dataset_with_meta-2.1.adm             |   0
 .../src/test/resources/runtimets/testsuite.xml  |  12 ++
 .../apache/asterix/test/aql/TestExecutor.java   |  46 ++++-
 asterix-lang-aql/src/main/javacc/AQL.html       |   6 +-
 asterix-lang-aql/src/main/javacc/AQL.jj         |  36 +++-
 .../lang/common/statement/DatasetDecl.java      |  36 +++-
 asterix-lang-sqlpp/src/main/javacc/SQLPP.html   |   4 +-
 asterix-lang-sqlpp/src/main/javacc/SQLPP.jj     |  36 +++-
 .../metadata/bootstrap/MetadataRecordTypes.java |   2 +
 .../metadata/declared/AqlDataSource.java        |  13 +-
 .../metadata/declared/AqlMetadataProvider.java  |  15 +-
 .../metadata/declared/DatasetDataSource.java    |  19 +-
 .../metadata/declared/FeedDataSource.java       |   8 +-
 .../metadata/declared/LoadableDataSource.java   |  13 +-
 .../asterix/metadata/entities/Dataset.java      |  27 +++
 .../DatasetTupleTranslator.java                 |  41 +++-
 .../asterix/metadata/utils/DatasetUtils.java    |   3 +-
 .../metadata/utils/MetadataLockManager.java     |  25 ++-
 .../DatasetTupleTranslatorTest.java             |  63 ++++++
 73 files changed, 1283 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 7272033..4c0818b 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -114,7 +114,7 @@ public class ExternalDataLookupPOperator extends AbstractScanPOperator {
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
             throws AlgebricksException {
         AqlDataSource ds = new DatasetDataSource(datasetId, datasetId.getDataverseName(), datasetId.getDatasourceName(),
-                recordType, AqlDataSourceType.EXTERNAL_DATASET);
+                recordType, null /*external dataset doesn't have meta records.*/, AqlDataSourceType.EXTERNAL_DATASET);
         IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
         AbstractScanOperator as = (AbstractScanOperator) op;
         deliveredProperties = dspp.computePropertiesVector(as.getVariables());

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
index 1945be3..ab28e28 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -63,7 +63,8 @@ import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -107,34 +108,40 @@ public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
                 String datasetName = datasetReference.second;
                 Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
                 if (dataset == null) {
-                    throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse "
-                            + dataverseName);
+                    throw new AlgebricksException(
+                            "Could not find dataset " + datasetName + " in dataverse " + dataverseName);
                 }
 
                 AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
-
-                ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
+                List<LogicalVariable> variables = new ArrayList<LogicalVariable>();
 
                 if (dataset.getDatasetType() == DatasetType.INTERNAL) {
                     int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
                     for (int i = 0; i < numPrimaryKeys; i++) {
-                        v.add(context.newVar());
+                        variables.add(context.newVar());
                     }
                 }
-                v.add(unnest.getVariable());
+                variables.add(unnest.getVariable());
                 AqlDataSource dataSource = metadataProvider.findDataSource(asid);
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, dataSource);
+                boolean hasMeta = dataSource.hasMeta();
+                if (hasMeta) {
+                    variables.add(context.newVar());
+                }
+                DataSourceScanOperator scan = new DataSourceScanOperator(variables, dataSource);
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
                 opRef.setValue(scan);
-                addPrimaryKey(v, context);
+                addPrimaryKey(variables, context);
                 context.computeAndSetTypeEnvironmentForOperator(scan);
 
                 // Adds equivalence classes --- one equivalent class between a primary key
                 // variable and a record field-access expression.
                 IAType[] schemaTypes = dataSource.getSchemaTypes();
-                ARecordType recordType = (ARecordType) schemaTypes[schemaTypes.length - 1];
-                EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(scan, v, recordType, dataset, context);
+                ARecordType recordType = (ARecordType) (hasMeta ? schemaTypes[schemaTypes.length - 2]
+                        : schemaTypes[schemaTypes.length - 1]);
+                ARecordType metaRecordType = (ARecordType) (hasMeta ? schemaTypes[schemaTypes.length - 1] : null);
+                EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(scan, variables, recordType,
+                        metaRecordType, dataset, context);
                 return true;
             }
 
@@ -166,8 +173,10 @@ public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
                 v.add(unnest.getVariable());
 
                 String csLocations = metadataProvider.getConfig().get(FeedActivityDetails.COLLECT_LOCATIONS);
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid, targetDataset,
-                        sourceFeedName, subscriptionLocation, metadataProvider, policy, outputType, csLocations));
+                DataSourceScanOperator scan = new DataSourceScanOperator(v,
+                        createFeedDataSource(asid, targetDataset, sourceFeedName, subscriptionLocation,
+                                metadataProvider, policy, outputType,
+                                null /* TODO(Adbullah): to figure out the meta type name*/, csLocations));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
@@ -194,16 +203,16 @@ public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
 
     private AqlDataSource createFeedDataSource(AqlSourceId aqlId, String targetDataset, String sourceFeedName,
             String subscriptionLocation, AqlMetadataProvider metadataProvider, FeedPolicyEntity feedPolicy,
-            String outputType, String locations) throws AlgebricksException {
-        if (!aqlId.getDataverseName().equals(
-                metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
-                        .getDataverseName())) {
+            String outputType, String outputMetaType, String locations) throws AlgebricksException {
+        if (!aqlId.getDataverseName().equals(metadataProvider.getDefaultDataverse() == null ? null
+                : metadataProvider.getDefaultDataverse().getDataverseName())) {
             return null;
         }
         IAType feedOutputType = metadataProvider.findType(aqlId.getDataverseName(), outputType);
+        IAType feedOutputMetaType = metadataProvider.findType(aqlId.getDataverseName(), outputMetaType);
         Feed sourceFeed = metadataProvider.findFeed(aqlId.getDataverseName(), sourceFeedName);
 
-        FeedDataSource feedDataSource = new FeedDataSource(aqlId, targetDataset, feedOutputType,
+        FeedDataSource feedDataSource = new FeedDataSource(aqlId, targetDataset, feedOutputType, feedOutputMetaType,
                 AqlDataSource.AqlDataSourceType.FEED, sourceFeed.getFeedId(), sourceFeed.getFeedType(),
                 ConnectionLocation.valueOf(subscriptionLocation), locations.split(","));
         feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 3fdbc5c..1bbb13b 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -59,7 +59,6 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractData
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
@@ -124,8 +123,8 @@ public class BTreeAccessMethod implements IAccessMethod {
             IOptimizationContext context) throws AlgebricksException {
         SelectOperator select = (SelectOperator) selectRef.getValue();
         Mutable<ILogicalExpression> conditionRef = select.getCondition();
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, subTree, null,
-                chosenIndex, analysisCtx, false, false, false, context);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, subTree, null, chosenIndex,
+                analysisCtx, false, false, false, context);
         if (primaryIndexUnnestOp == null) {
             return false;
         }
@@ -185,8 +184,8 @@ public class BTreeAccessMethod implements IAccessMethod {
             newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
         }
 
-        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, indexSubTree,
-                probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(conditionRef, indexSubTree, probeSubTree,
+                chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context);
         if (primaryIndexUnnestOp == null) {
             return false;
         }
@@ -212,12 +211,13 @@ public class BTreeAccessMethod implements IAccessMethod {
     }
 
     @Override
-    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef, OptimizableOperatorSubTree indexSubTree,
-            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context)
-                    throws AlgebricksException {
+    public ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalExpression> conditionRef,
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, boolean retainInput, boolean retainNull, boolean requiresBroadcast,
+            IOptimizationContext context) throws AlgebricksException {
         Dataset dataset = indexSubTree.dataset;
         ARecordType recordType = indexSubTree.recordType;
+        ARecordType metaRecordType = indexSubTree.metaRecordType;
         // we made sure indexSubTree has datasource scan
         AbstractDataSourceOperator dataSourceOp = (AbstractDataSourceOperator) indexSubTree.dataSourceRef.getValue();
         List<Pair<Integer, Integer>> exprAndVarList = analysisCtx.indexExprsAndVars.get(chosenIndex);
@@ -499,7 +499,7 @@ public class BTreeAccessMethod implements IAccessMethod {
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
             EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp,
-                    dataSourceOp.getVariables(), recordType, dataset, context);
+                    dataSourceOp.getVariables(), recordType, metaRecordType, dataset, context);
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
             try {
@@ -527,7 +527,7 @@ public class BTreeAccessMethod implements IAccessMethod {
             // Adds equivalence classes --- one equivalent class between a primary key
             // variable and a record field-access expression.
             EquivalenceClassUtils.addEquivalenceClassesForPrimaryIndexAccess(primaryIndexUnnestOp, scanVariables,
-                    recordType, dataset, context);
+                    recordType, metaRecordType, dataset, context);
         }
 
         return primaryIndexUnnestOp;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 39a9ab8..be3dad8 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -68,6 +68,7 @@ public class OptimizableOperatorSubTree {
     // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
     public Dataset dataset = null;
     public ARecordType recordType = null;
+    public ARecordType metaRecordType = null;
 
     // Additional datasources can exist if IntroduceJoinAccessMethodRule has been applied.
     // (E.g. There are index-nested-loop-joins in the plan.)
@@ -264,10 +265,15 @@ public class OptimizableOperatorSubTree {
             }
             rType = (ARecordType) itemType;
 
+            // Get the meta record type for that dataset.
+            IAType metaItemType = metadataProvider.findType(ds.getMetaItemTypeDataverseName(),
+                    ds.getMetaItemTypeName());
+
             // First index is always the primary datasource in this subtree.
             if (i == 0) {
                 dataset = ds;
                 recordType = rType;
+                metaRecordType = (ARecordType) metaItemType;
             } else {
                 ixJoinOuterAdditionalDatasets.add(ds);
                 ixJoinOuterAdditionalRecordTypes.add(rType);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
index 9f98da0..5ede3bb 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/EquivalenceClassUtils.java
@@ -66,6 +66,8 @@ public class EquivalenceClassUtils {
      *            is the record variable.
      * @param recordType
      *            , the record type of an index payload record.
+     * @param metaRecordType
+     *            , the type of a meta record associated with an index payload record.
      * @param dataset
      *            , the accessed dataset.
      * @param context
@@ -74,7 +76,7 @@ public class EquivalenceClassUtils {
      */
     @SuppressWarnings("unchecked")
     public static void addEquivalenceClassesForPrimaryIndexAccess(ILogicalOperator operator,
-            List<LogicalVariable> indexSearchVars, ARecordType recordType, Dataset dataset,
+            List<LogicalVariable> indexSearchVars, ARecordType recordType, ARecordType metaRecordType, Dataset dataset,
             IOptimizationContext context) throws AlgebricksException {
         if (dataset.getDatasetDetails().getDatasetType() != DatasetType.INTERNAL) {
             return;
@@ -86,15 +88,30 @@ public class EquivalenceClassUtils {
         for (int fieldIndex = 0; fieldIndex < fieldNames.length; ++fieldIndex) {
             fieldNameToIndexMap.put(fieldNames[fieldIndex], fieldIndex);
         }
-
-        LogicalVariable recordVar = indexSearchVars.get(indexSearchVars.size() - 1);
+        boolean hasMeta = dataset.hasMetaPart();
+        Map<String, Integer> metaFieldNameToIndexMap = new HashMap<>();
+        if (hasMeta) {
+            String[] metaFieldNames = metaRecordType.getFieldNames();
+            for (int metaFieldIndex = 0; metaFieldIndex < metaFieldNames.length; ++metaFieldIndex) {
+                metaFieldNameToIndexMap.put(metaFieldNames[metaFieldIndex], metaFieldIndex);
+            }
+        }
+        LogicalVariable recordVar = hasMeta ? indexSearchVars.get(indexSearchVars.size() - 2)
+                : indexSearchVars.get(indexSearchVars.size() - 1);
+        LogicalVariable metaRecordVar = hasMeta ? indexSearchVars.get(indexSearchVars.size() - 1) : null;
         for (int pkIndex = 0; pkIndex < primaryKey.size(); ++pkIndex) {
+            LogicalVariable referredRecordVar = recordVar;
             String pkFieldName = primaryKey.get(pkIndex).get(0);
-            int fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
+            Integer fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
+            if (fieldIndexInRecord == null && hasMeta) {
+                referredRecordVar = metaRecordVar;
+                pkFieldName = primaryKey.get(pkIndex).get(1);
+                fieldIndexInRecord = metaFieldNameToIndexMap.get(pkFieldName);
+            }
             LogicalVariable var = indexSearchVars.get(pkIndex);
             ILogicalExpression expr = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX),
-                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(recordVar)),
+                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(referredRecordVar)),
                     new MutableObject<ILogicalExpression>(
                             new ConstantExpression(new AsterixConstantValue(new AInt32(fieldIndexInRecord)))));
             EquivalenceClass equivClass = new EquivalenceClass(SingletonList.newSingletonList(var), var,

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index dd4d6db..c635125 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -183,13 +183,15 @@ class LangExpressionToPlanTranslator
                     "Unable to load dataset " + clffs.getDatasetName() + " since it does not exist");
         }
         IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                dataset.getMetaItemTypeName());
         DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                 stmt.getDatasetName());
         List<List<String>> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
 
         LoadableDataSource lds;
         try {
-            lds = new LoadableDataSource(dataset, itemType, clffs.getAdapter(), clffs.getProperties());
+            lds = new LoadableDataSource(dataset, itemType, metaItemType, clffs.getAdapter(), clffs.getProperties());
         } catch (IOException e) {
             throw new AlgebricksException(e);
         }
@@ -451,10 +453,11 @@ class LangExpressionToPlanTranslator
             throw new AlgebricksException("Cannot write output to an external dataset.");
         }
         AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
-        String itemTypeName = dataset.getItemTypeName();
-        IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), itemTypeName);
+        IAType itemType = metadataProvider.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
+        IAType metaItemType = metadataProvider.findType(dataset.getMetaItemTypeDataverseName(),
+                dataset.getMetaItemTypeName());
         DatasetDataSource dataSource = new DatasetDataSource(sourceId, dataset.getDataverseName(),
-                dataset.getDatasetName(), itemType, AqlDataSourceType.INTERNAL_DATASET);
+                dataset.getDatasetName(), itemType, metaItemType, AqlDataSourceType.INTERNAL_DATASET);
 
         return dataSource;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
----------------------------------------------------------------------
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index 1c8db4f..f4691f1 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -35,6 +35,8 @@ import org.apache.asterix.om.types.IAType;
  */
 public class ValidateUtil {
 
+    private static final String META = "meta()";
+
     /**
      * Validates the field that will be used as filter for the components of an LSM index.
      *
@@ -86,7 +88,7 @@ public class ValidateUtil {
      * @throws AsterixException
      *             (if the validation failed), IOException
      */
-    public static List<IAType> validatePartitioningExpressions(ARecordType recType,
+    public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
             List<List<String>> partitioningExprs, boolean autogenerated) throws AsterixException, IOException {
         List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
         if (autogenerated) {
@@ -105,7 +107,13 @@ public class ValidateUtil {
         } else {
             for (int i = 0; i < partitioningExprs.size(); i++) {
                 List<String> fieldName = partitioningExprs.get(i);
-                IAType fieldType = recType.getSubFieldType(fieldName);
+                List<String> metaFieldName = null;
+                boolean useMeta = fieldName.get(0).equals(META);
+                if (useMeta) {
+                    metaFieldName = fieldName.subList(1, fieldName.size());
+                }
+                IAType fieldType = useMeta ? metaRecType.getSubFieldType(metaFieldName)
+                        : recType.getSubFieldType(fieldName);
                 switch (fieldType.getTypeTag()) {
                     case INT8:
                     case INT16:
@@ -156,15 +164,18 @@ public class ValidateUtil {
             IAType fieldType = recType.getSubFieldType(fieldName);
             if (fieldType == null) {
                 fieldType = keyFieldTypes.get(pos);
-                if (keyFieldTypes.get(pos) == BuiltinType.ANULL)
+                if (keyFieldTypes.get(pos) == BuiltinType.ANULL) {
                     throw new AsterixException("A field with this name  \"" + fieldName + "\" could not be found.");
-            } else if (openFieldCompositeIdx)
+                }
+            } else if (openFieldCompositeIdx) {
                 throw new AsterixException("A closed field \"" + fieldName
                         + "\" could be only in a prefix part of the composite index, containing opened field.");
+            }
             if (keyFieldTypes.get(pos) != BuiltinType.ANULL
-                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag())
+                    && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag()) {
                 throw new AsterixException(
                         "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
+            }
             switch (indexType) {
                 case BTREE:
                     switch (fieldType.getTypeTag()) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
index ea50221..87859dc 100644
--- a/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
@@ -532,6 +532,8 @@ public class QueryTranslator extends AbstractLangTranslator {
         DatasetType dsType = dd.getDatasetType();
         String itemTypeDataverseName = dd.getItemTypeDataverse().getValue();
         String itemTypeName = dd.getItemTypeName().getValue();
+        String metaItemTypeDataverseName = dd.getMetaItemTypeDataverse().getValue();
+        String metaItemTypeName = dd.getMetaItemTypeName().getValue();
         Identifier ngNameId = dd.getNodegroupName();
         String nodegroupName = getNodeGroupName(ngNameId, dd, dataverseName);
         String compactionPolicy = dd.getCompactionPolicy();
@@ -544,7 +546,8 @@ public class QueryTranslator extends AbstractLangTranslator {
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
 
         MetadataLockManager.INSTANCE.createDatasetBegin(dataverseName, itemTypeDataverseName,
-                itemTypeDataverseName + "." + itemTypeName, nodegroupName, compactionPolicy,
+                itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
+                metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                 dataverseName + "." + datasetName, defaultCompactionPolicy);
         Dataset dataset = null;
         try {
@@ -578,14 +581,24 @@ public class QueryTranslator extends AbstractLangTranslator {
                 case INTERNAL: {
                     IAType itemType = dt.getDatatype();
                     if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                        throw new AlgebricksException("Can only partition ARecord's.");
+                        throw new AlgebricksException("Dataset type has to be a record type.");
                     }
+
+                    IAType metaItemType = null;
+                    if (metaItemTypeDataverseName != null && metaItemTypeName != null) {
+                        metaItemType = metadataProvider.findType(metaItemTypeDataverseName, metaItemTypeName);
+                    }
+                    if (metaItemType != null && metaItemType.getTypeTag() != ATypeTag.RECORD) {
+                        throw new AlgebricksException("Dataset meta type has to be a record type.");
+                    }
+                    ARecordType metaRecType = (ARecordType) metaItemType;
+
                     List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
                             .getPartitioningExprs();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
-                            partitioningExprs, autogenerated);
+                            metaRecType, partitioningExprs, autogenerated);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
@@ -622,9 +635,10 @@ public class QueryTranslator extends AbstractLangTranslator {
             }
 
             //#. add a new dataset with PendingAddOp
-            dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName, ngName,
-                    compactionPolicy, compactionPolicyProperties, datasetDetails, dd.getHints(), dsType,
-                    DatasetIdFactory.generateDatasetId(), IMetadataEntity.PENDING_ADD_OP);
+            dataset = new Dataset(dataverseName, datasetName, itemTypeDataverseName, itemTypeName,
+                    metaItemTypeDataverseName, metaItemTypeName, ngName, compactionPolicy, compactionPolicyProperties,
+                    datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
+                    IMetadataEntity.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
 
             if (dd.getDatasetType() == DatasetType.INTERNAL) {
@@ -698,7 +712,8 @@ public class QueryTranslator extends AbstractLangTranslator {
             throw e;
         } finally {
             MetadataLockManager.INSTANCE.createDatasetEnd(dataverseName, itemTypeDataverseName,
-                    itemTypeDataverseName + "." + itemTypeName, nodegroupName, compactionPolicy,
+                    itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
+                    metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
                     dataverseName + "." + datasetName, defaultCompactionPolicy);
         }
     }
@@ -1393,8 +1408,9 @@ public class QueryTranslator extends AbstractLangTranslator {
                 MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
                 MetadataManager.INSTANCE.addDataset(mdTxnCtx,
                         new Dataset(dataverseName, datasetName, ds.getItemTypeDataverseName(), ds.getItemTypeName(),
-                                ds.getNodeGroupName(), ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(),
-                                ds.getDatasetDetails(), ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
+                                ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName(), ds.getNodeGroupName(),
+                                ds.getCompactionPolicy(), ds.getCompactionPolicyProperties(), ds.getDatasetDetails(),
+                                ds.getHints(), ds.getDatasetType(), ds.getDatasetId(),
                                 IMetadataEntity.PENDING_DROP_OP));
 
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2935,7 +2951,9 @@ public class QueryTranslator extends AbstractLangTranslator {
                     toDataset.getDatasetDetails().isTemp());
             DatasetDecl createToDataset = new DatasetDecl(new Identifier(dataverseNameTo),
                     pregelixStmt.getDatasetNameTo(), new Identifier(toDataset.getItemTypeDataverseName()),
-                    new Identifier(toDataset.getItemTypeName()), new Identifier(toDataset.getNodeGroupName()),
+                    new Identifier(toDataset.getItemTypeName()),
+                    new Identifier(toDataset.getMetaItemTypeDataverseName()),
+                    new Identifier(toDataset.getMetaItemTypeName()), new Identifier(toDataset.getNodeGroupName()),
                     toDataset.getCompactionPolicy(), toDataset.getCompactionPolicyProperties(), toDataset.getHints(),
                     toDataset.getDatasetType(), idd, false);
             this.handleCreateDatasetStatement(metadataProvider, createToDataset, hcc);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql
new file mode 100644
index 0000000..6539dbc
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.1.ddl.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+    id: string
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-1/dataset_with_meta-1.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql
new file mode 100644
index 0000000..cec01d2
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.1.ddl.aql
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse meta if exists;
+create dataverse meta;
+use dataverse meta;
+create type EmptyType as open {
+    id: string
+}
+
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(meta.EmptyType)
+primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql
new file mode 100644
index 0000000..c7a256c
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-2/dataset_with_meta-2.4.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+drop dataverse meta if exists;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql
new file mode 100644
index 0000000..78bd7ab
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.1.ddl.aql
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+    id: string
+}
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(LineType)
+primary key id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-3/dataset_with_meta-3.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql
new file mode 100644
index 0000000..869bc1e
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.1.ddl.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(LineType)
+primary key meta().id;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-4/dataset_with_meta-4.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql
new file mode 100644
index 0000000..ddb42c8
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.1.ddl.aql
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type EmptyType as open {
+  "key":int32
+}
+
+
+create type LineType as open {
+  id:int32,
+  text: string
+}
+
+create dataset Book(LineType) with meta(EmptyType)
+primary key meta()."key";
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.2.update.aql
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql
new file mode 100644
index 0000000..079d075
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+for $x in dataset('Metadata.Dataset')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-5/dataset_with_meta-5.4.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm
new file mode 100644
index 0000000..776e07f
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-1/dataset_with_meta-1.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "EmptyType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm
new file mode 100644
index 0000000..f3f00e9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-2/dataset_with_meta-2.1.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "meta", "MetatypeName": "EmptyType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm
new file mode 100644
index 0000000..085aaa4
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-3/dataset_with_meta-2.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "id" ] ], "PrimaryKey": [ [ "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 17:32:57 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "LineType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm
new file mode 100644
index 0000000..ec8c3bf
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-4/dataset_with_meta-4.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "meta()", "id" ] ], "PrimaryKey": [ [ "meta()", "id" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 22:29:39 PST 2016", "DatasetId": 101i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "LineType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm
new file mode 100644
index 0000000..722f3a5
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-5/dataset_with_meta-5.3.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test", "DatasetName": "Book", "DatatypeDataverseName": "test", "DatatypeName": "LineType", "DatasetType": "INTERNAL", "GroupName": "DEFAULT_NG_ALL_NODES", "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "meta()", "key" ] ], "PrimaryKey": [ [ "meta()", "key" ] ], "Autogenerated": false }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Feb 24 22:55:29 PST 2016", "DatasetId": 105i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "EmptyType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
index 98f0152..a4c2eaa 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta01/meta01.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Sat Nov 24 14:44:45 PST 2012" }
-{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Sat Nov 24 14:45:14 PST 2012" }
+{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:38:04 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:38:09 PST 2016", "PendingOp": 0i32 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
index 82675b2..c92c3da 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta05/meta05.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:42:44 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:42:44 PST 2016", "PendingOp": 0i32 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
index aa1464a..b5f779e 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta12/meta12.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "dst01", "IndexName": "dst01", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:40:44 PDT 2012" }
+{ "DataverseName": "test", "DatasetName": "dst01", "IndexName": "dst01", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:42:45 PST 2016", "PendingOp": 0i32 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index 117526a..9d7b9ce 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012", "SearchKeyType": [ "string" ] }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "BTREE", "SearchKey": [ [ "name" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32, "SearchKeyType": [ "string" ], "IsEnforced": true }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index 9685606..bfd033c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,2 +1,2 @@
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012", "SearchKeyType": [ "point" ] }
-{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Sep 17 23:21:46 PDT 2012" }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "idx1", "IndexStructure": "RTREE", "SearchKey": [ [ "location" ] ], "IsPrimary": false, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32, "SearchKeyType": [ "point" ], "IsEnforced": true }
+{ "DataverseName": "testdv", "DatasetName": "t1", "IndexName": "t1", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/b9169b66/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
index fc4acca..d5046f4 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataverse/metadata_dataverse.1.adm
@@ -1,3 +1,3 @@
-{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:48 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "test", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:55 PDT 2013", "PendingOp": 0 }
-{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Thu Apr 04 21:10:52 PDT 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:46 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "test", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "testdv", "DataFormat": "org.apache.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Wed Feb 24 18:43:52 PST 2016", "PendingOp": 0i32 }