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/03/01 21:52:45 UTC

[2/2] incubator-asterixdb git commit: Add an integer ordered list to dataset "Dataset" and "Index" optinally.

Add an integer ordered list to dataset "Dataset" and "Index" optinally.

For the case the primary or secondary index key contains fields from the meta
record associated with a dataset record, we need an integer ordered
list to indicate which record a key field comes from.

Change-Id: I979c642ebd60e53213369f1e2070146b4c26e805
Reviewed-on: https://asterix-gerrit.ics.uci.edu/683
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>
Reviewed-by: Michael Blow <mi...@couchbase.com>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: c9bfe259ed0f9b41d2e4c7d67db9fb2d8342f6e2
Parents: 06e06b0
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Tue Mar 1 00:02:40 2016 -0800
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Tue Mar 1 12:47:47 2016 -0800

----------------------------------------------------------------------
 .../optimizer/rules/UnnestToDataScanRule.java   |  2 +-
 .../rules/util/EquivalenceClassUtils.java       | 11 ++-
 .../asterix/translator/util/ValidateUtil.java   | 13 +--
 .../asterix/aql/translator/QueryTranslator.java | 29 +++---
 .../dataset_with_meta-6.1.ddl.aql               | 37 ++++++++
 .../dataset_with_meta-6.2.update.aql            | 19 ++++
 .../dataset_with_meta-6.3.query.aql             | 22 +++++
 .../dataset_with_meta-6.4.ddl.aql               | 20 +++++
 .../dataset_with_meta-6.1.ddl.aql               | 37 ++++++++
 .../dataset_with_meta-6.2.update.aql            | 19 ++++
 .../dataset_with_meta-6.3.query.aql             | 22 +++++
 .../dataset_with_meta-6.4.ddl.aql               | 20 +++++
 .../queries/basic/meta01/meta01.1.ddl.aql       |  1 +
 .../dataset_with_meta-4.3.adm                   |  2 +-
 .../dataset_with_meta-5.3.adm                   |  2 +-
 .../dataset_with_meta-6.3.adm                   |  2 +
 .../dataset_with_meta-7.3.adm                   |  2 +
 .../src/test/resources/metadata/testsuite.xml   | 10 +++
 .../query_dataset_with_meta-1.1.ddl.aql         | 34 +++++++
 .../query_dataset_with_meta-1.2.update.aql      | 19 ++++
 .../query_dataset_with_meta-1.3.query.aql       | 23 +++++
 .../query_dataset_with_meta-1.4.ddl.aql         | 20 +++++
 .../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-2.1.ddl.aql         | 34 +++++++
 .../query_dataset_with_meta-2.2.update.aql      | 19 ++++
 .../query_dataset_with_meta-2.3.query.aql       | 23 +++++
 .../query_dataset_with_meta-2.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 -----
 asterix-lang-aql/src/main/javacc/AQL.html       |  4 +-
 asterix-lang-aql/src/main/javacc/AQL.jj         | 53 ++++++-----
 .../common/statement/CreateIndexStatement.java  | 11 ++-
 .../lang/common/statement/FeedDetailsDecl.java  | 54 -----------
 .../common/statement/InternalDetailsDecl.java   | 10 ++-
 asterix-lang-sqlpp/src/main/javacc/SQLPP.html   |  4 +-
 asterix-lang-sqlpp/src/main/javacc/SQLPP.jj     | 51 ++++++-----
 .../apache/asterix/metadata/MetadataCache.java  |  4 +-
 .../apache/asterix/metadata/MetadataNode.java   | 19 ++--
 .../metadata/MetadataTransactionContext.java    |  2 +-
 .../metadata/bootstrap/MetadataBootstrap.java   |  2 +-
 .../apache/asterix/metadata/entities/Index.java | 26 +++---
 .../entities/InternalDatasetDetails.java        | 66 +++++++++++---
 .../DatasetTupleTranslator.java                 | 20 ++++-
 .../IndexTupleTranslator.java                   | 55 +++++++++++-
 .../DatasetTupleTranslatorTest.java             | 45 ++++++----
 .../IndexTupleTranslatorTest.java               | 95 ++++++++++++++++++++
 51 files changed, 795 insertions(+), 380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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 ab28e28..5af6ccc 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
@@ -176,7 +176,7 @@ public class UnnestToDataScanRule implements IAlgebraicRewriteRule {
                 DataSourceScanOperator scan = new DataSourceScanOperator(v,
                         createFeedDataSource(asid, targetDataset, sourceFeedName, subscriptionLocation,
                                 metadataProvider, policy, outputType,
-                                null /* TODO(Adbullah): to figure out the meta type name*/, csLocations));
+                                null /* TODO(Abdullah): to figure out the meta type name*/, csLocations));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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 5ede3bb..e85fecf 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
@@ -96,16 +96,21 @@ public class EquivalenceClassUtils {
                 metaFieldNameToIndexMap.put(metaFieldNames[metaFieldIndex], metaFieldIndex);
             }
         }
+        List<Integer> keySourceIndicators = datasetDetails.getKeySourceIndicator();
         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);
-            Integer fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
-            if (fieldIndexInRecord == null && hasMeta) {
+            int source = keySourceIndicators == null ? 0 : keySourceIndicators.get(pkIndex);
+            Integer fieldIndexInRecord;
+            if (source == 0) {
+                // The field is from the main record.
+                fieldIndexInRecord = fieldNameToIndexMap.get(pkFieldName);
+            } else {
+                // The field is from the auxiliary meta record.
                 referredRecordVar = metaRecordVar;
-                pkFieldName = primaryKey.get(pkIndex).get(1);
                 fieldIndexInRecord = metaFieldNameToIndexMap.get(pkFieldName);
             }
             LogicalVariable var = indexSearchVars.get(pkIndex);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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 f4691f1..95a8802 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,8 +35,6 @@ 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.
      *
@@ -89,7 +87,8 @@ public class ValidateUtil {
      *             (if the validation failed), IOException
      */
     public static List<IAType> validatePartitioningExpressions(ARecordType recType, ARecordType metaRecType,
-            List<List<String>> partitioningExprs, boolean autogenerated) throws AsterixException, IOException {
+            List<List<String>> partitioningExprs, List<Integer> keySourceIndicators, boolean autogenerated)
+                    throws AsterixException, IOException {
         List<IAType> partitioningExprTypes = new ArrayList<IAType>(partitioningExprs.size());
         if (autogenerated) {
             if (partitioningExprs.size() > 1) {
@@ -107,12 +106,8 @@ public class ValidateUtil {
         } else {
             for (int i = 0; i < partitioningExprs.size(); i++) {
                 List<String> fieldName = partitioningExprs.get(i);
-                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)
+                boolean useMeta = keySourceIndicators.get(i) > 0;
+                IAType fieldType = useMeta ? metaRecType.getSubFieldType(fieldName)
                         : recType.getSubFieldType(fieldName);
                 switch (fieldType.getTypeTag()) {
                     case INT8:

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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 87859dc..75f325b 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
@@ -595,10 +595,12 @@ public class QueryTranslator extends AbstractLangTranslator {
 
                     List<List<String>> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
                             .getPartitioningExprs();
+                    List<Integer> keySourceIndicators = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+                            .getKeySourceIndicators();
                     boolean autogenerated = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated();
                     ARecordType aRecordType = (ARecordType) itemType;
                     List<IAType> partitioningTypes = ValidateUtil.validatePartitioningExpressions(aRecordType,
-                            metaRecType, partitioningExprs, autogenerated);
+                            metaRecType, partitioningExprs, keySourceIndicators, autogenerated);
 
                     List<String> filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
                     if (filterField != null) {
@@ -615,7 +617,7 @@ public class QueryTranslator extends AbstractLangTranslator {
                     }
                     datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
                             InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
-                            partitioningTypes, autogenerated, filterField, temp);
+                            keySourceIndicators, partitioningTypes, autogenerated, filterField, temp);
                     break;
                 }
                 case EXTERNAL: {
@@ -957,7 +959,7 @@ public class QueryTranslator extends AbstractLangTranslator {
                     //Add an entry for the files index
                     filesIndex = new Index(dataverseName, datasetName,
                             ExternalIndexingOperations.getFilesIndexName(datasetName), IndexType.BTREE,
-                            ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES,
+                            ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
                             ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, false,
                             IMetadataEntity.PENDING_ADD_OP);
                     MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
@@ -993,8 +995,8 @@ public class QueryTranslator extends AbstractLangTranslator {
 
             //#. add a new index with PendingAddOp
             Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(), indexFields,
-                    indexFieldTypes, stmtCreateIndex.getGramLength(), stmtCreateIndex.isEnforced(), false,
-                    IMetadataEntity.PENDING_ADD_OP);
+                    stmtCreateIndex.getFieldSourceIndicators(), indexFieldTypes, stmtCreateIndex.getGramLength(),
+                    stmtCreateIndex.isEnforced(), false, IMetadataEntity.PENDING_ADD_OP);
             MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
 
             ARecordType enforcedType = null;
@@ -1580,8 +1582,8 @@ public class QueryTranslator extends AbstractLangTranslator {
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
                         new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldTypes(), index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
-                                IMetadataEntity.PENDING_DROP_OP));
+                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
+                                index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
 
                 //#. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1633,8 +1635,9 @@ public class QueryTranslator extends AbstractLangTranslator {
                             MetadataManager.INSTANCE.addIndex(mdTxnCtx,
                                     new Index(dataverseName, datasetName, externalIndex.getIndexName(),
                                             externalIndex.getIndexType(), externalIndex.getKeyFieldNames(),
-                                            index.getKeyFieldTypes(), index.isEnforcingKeyFileds(),
-                                            externalIndex.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+                                            externalIndex.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
+                                            index.isEnforcingKeyFileds(), externalIndex.isPrimaryIndex(),
+                                            IMetadataEntity.PENDING_DROP_OP));
                         }
                     }
                 }
@@ -1643,8 +1646,8 @@ public class QueryTranslator extends AbstractLangTranslator {
                 MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 MetadataManager.INSTANCE.addIndex(mdTxnCtx,
                         new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
-                                index.getKeyFieldTypes(), index.isEnforcingKeyFileds(), index.isPrimaryIndex(),
-                                IMetadataEntity.PENDING_DROP_OP));
+                                index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
+                                index.isEnforcingKeyFileds(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
 
                 //#. commit the existing transaction before calling runJob.
                 MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -2947,8 +2950,8 @@ public class QueryTranslator extends AbstractLangTranslator {
             DropStatement dropStmt = new DropStatement(new Identifier(dataverseNameTo), pregelixStmt.getDatasetNameTo(),
                     true);
             this.handleDatasetDropStatement(metadataProvider, dropStmt, hcc);
-            IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(), false, null,
-                    toDataset.getDatasetDetails().isTemp());
+            IDatasetDetailsDecl idd = new InternalDetailsDecl(toIndex.getKeyFieldNames(),
+                    toIndex.getKeyFieldSourceIndicators(), false, null, toDataset.getDatasetDetails().isTemp());
             DatasetDecl createToDataset = new DatasetDecl(new Identifier(dataverseNameTo),
                     pregelixStmt.getDatasetNameTo(), new Identifier(toDataset.getItemTypeDataverseName()),
                     new Identifier(toDataset.getItemTypeName()),

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql
new file mode 100644
index 0000000..510137f
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.1.ddl.aql
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+create index MetaIndex on Book(meta().id) type btree;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.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/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.3.query.aql
new file mode 100644
index 0000000..615cb49
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.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.Index')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-6/dataset_with_meta-6.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/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql
new file mode 100644
index 0000000..41b50d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.1.ddl.aql
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+create index MetaIndex on Book(meta().id, id) type btree;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.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/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.3.query.aql
new file mode 100644
index 0000000..615cb49
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.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.Index')
+where $x.DataverseName='test'
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/dataset_with_meta-7/dataset_with_meta-6.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/c9bfe259/asterix-app/src/test/resources/metadata/queries/basic/meta01/meta01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/meta01/meta01.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/meta01/meta01.1.ddl.aql
index 5df40a8..ccb5b3e 100644
--- a/asterix-app/src/test/resources/metadata/queries/basic/meta01/meta01.1.ddl.aql
+++ b/asterix-app/src/test/resources/metadata/queries/basic/meta01/meta01.1.ddl.aql
@@ -22,6 +22,7 @@
  * Date         : 15 Sep 2012
  */
 
+drop dataverse test if exists;
 drop dataverse testdv if exists;
 create dataverse testdv;
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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
index ec8c3bf..581328f 100644
--- 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
@@ -1 +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" }
+{ "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, "KeySourceIndicator": [ 1i8 ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Mon Feb 29 15:58:41 PST 2016", "DatasetId": 104i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "LineType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/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
index 722f3a5..485ca4e 100644
--- 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
@@ -1 +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" }
+{ "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": [ [ "key" ] ], "PrimaryKey": [ [ "key" ] ], "Autogenerated": false, "KeySourceIndicator": [ 1i8 ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Mon Feb 29 15:58:41 PST 2016", "DatasetId": 105i32, "PendingOp": 0i32, "MetatypeDataverseName": "test", "MetatypeName": "EmptyType" }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm
new file mode 100644
index 0000000..1c33fc9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-6/dataset_with_meta-6.3.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "Book", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Feb 29 18:41:47 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "MetaIndex", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": false, "Timestamp": "Mon Feb 29 18:42:53 PST 2016", "PendingOp": 0i32, "SearchKeySourceIndicator": [ 1i8 ] }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm
new file mode 100644
index 0000000..52dc45a
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/dataset_with_meta-7/dataset_with_meta-7.3.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "Book", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ] ], "IsPrimary": true, "Timestamp": "Mon Feb 29 18:45:56 PST 2016", "PendingOp": 0i32 }
+{ "DataverseName": "test", "DatasetName": "Book", "IndexName": "MetaIndex", "IndexStructure": "BTREE", "SearchKey": [ [ "id" ], [ "id" ] ], "IsPrimary": false, "Timestamp": "Mon Feb 29 18:45:57 PST 2016", "PendingOp": 0i32, "SearchKeySourceIndicator": [ 1i8, 0i8 ] }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/metadata/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/metadata/testsuite.xml b/asterix-app/src/test/resources/metadata/testsuite.xml
index 550d93d..595eb84 100644
--- a/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -44,6 +44,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-6">
+        <output-dir compare="Text">dataset_with_meta-6</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="dataset_with_meta-7">
+        <output-dir compare="Text">dataset_with_meta-7</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
       <compilation-unit name="meta01">
         <output-dir compare="Text">meta01</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.ddl.aql
new file mode 100644
index 0000000..ca5a534
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.1.ddl.aql
@@ -0,0 +1,34 @@
+/*
+ * 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 id;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.3.query.aql
new file mode 100644
index 0000000..43150da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use dataverse test;
+
+for $x in dataset Book
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-1.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql
deleted file mode 100644
index ca5a534..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.1.ddl.aql
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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 id;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql
deleted file mode 100644
index bd244d0..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.2.update.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * 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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql
deleted file mode 100644
index 43150da..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.3.query.aql
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-
-use dataverse test;
-
-for $x in dataset Book
-return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql
deleted file mode 100644
index dc10acd..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-1/query_dataset_with_meta-5-1.4.ddl.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.ddl.aql
new file mode 100644
index 0000000..a341b29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.1.ddl.aql
@@ -0,0 +1,34 @@
+/*
+ * 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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.2.update.aql
new file mode 100644
index 0000000..bd244d0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.3.query.aql
new file mode 100644
index 0000000..43150da
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use dataverse test;
+
+for $x in dataset Book
+return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.4.ddl.aql
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-2.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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql
deleted file mode 100644
index a341b29..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.1.ddl.aql
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql
deleted file mode 100644
index bd244d0..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.2.update.aql
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * 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/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql
deleted file mode 100644
index 43150da..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.3.query.aql
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.
- */
-
-use dataverse test;
-
-for $x in dataset Book
-return $x

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql
deleted file mode 100644
index dc10acd..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/meta/query_dataset_with_meta-2/query_dataset_with_meta-5-2.4.ddl.aql
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * 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/c9bfe259/asterix-lang-aql/src/main/javacc/AQL.html
----------------------------------------------------------------------
diff --git a/asterix-lang-aql/src/main/javacc/AQL.html b/asterix-lang-aql/src/main/javacc/AQL.html
index 3da5e77..d3670d6 100644
--- a/asterix-lang-aql/src/main/javacc/AQL.html
+++ b/asterix-lang-aql/src/main/javacc/AQL.html
@@ -342,7 +342,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod21">DatasetSpecification</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( "external" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> "using" <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( "internal" | "temporary" )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> ( &lt;COMMA&gt; <A HREF="#prod27">TypeName</A> )? &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( "autogenerated" )? ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( "with filter on" <A H
 REF="#prod36">NestedField</A> )? )</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( "external" &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> "using" <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( "internal" | "temporary" )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; ( &lt;WITH&gt; <A HREF="#prod18">Identifier</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; )? <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( "autogenerated" )? ( "on" <A HREF="#prod18">Identifier</A> )? ( "hints" <A HREF="#prod33">Properties</A> )? ( "using" "compaction" "policy" <A HREF="#prod34">CompactionPolicy</A
 > ( <A HREF="#prod32">Configuration</A> )? )? ( "with filter on" <A HREF="#prod36">NestedField</A> )? )</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod16">RefreshExternalDatasetStatement</A></TD>
@@ -562,7 +562,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod36">NestedField</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( "@" )? <A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod18">Identifier</A> ( &lt;LEFTPAREN&gt; &lt;RIGHTPAREN&gt; )? ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod40">StringLiteral</A></TD>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-lang-aql/src/main/javacc/AQL.jj
----------------------------------------------------------------------
diff --git a/asterix-lang-aql/src/main/javacc/AQL.jj b/asterix-lang-aql/src/main/javacc/AQL.jj
index ea95871..8205021 100644
--- a/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -408,14 +408,14 @@ DatasetDecl DatasetSpecification() throws ParseException:
   Map<String,String> properties = null;
   Map<String,String> compactionPolicyProperties = null;
   FunctionSignature appliedFunction = null;
-  List<List<String>> primaryKeyFields = null;
+  Pair<List<Integer>, List<List<String>>> primaryKeyFields = null;
   String nodeGroupName = null;
   Map<String,String> hints = new HashMap<String,String>();
   DatasetDecl dsetDecl = null;
   boolean autogenerated = false;
   String compactionPolicy = null;
   boolean temp = false;
-  List<String> filterField = null;
+  Pair<Integer, List<String>> filterField = null;
   Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
 }
 {
@@ -471,9 +471,13 @@ DatasetDecl DatasetSpecification() throws ParseException:
     ( "using" "compaction" "policy" compactionPolicy = CompactionPolicy() (compactionPolicyProperties = Configuration())? )?
     ( "with filter on" filterField = NestedField() )?
       {
-        InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields,
+        if(filterField!=null && filterField.first!=0){
+          throw new ParseException("A filter field can only be a field in the main record of the dataset.");
+        }
+        InternalDetailsDecl idd = new InternalDetailsDecl(primaryKeyFields.second,
+                                                          primaryKeyFields.first,
                                                           autogenerated,
-                                                          filterField,
+                                                          filterField == null? null : filterField.second,
                                                           temp);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
@@ -537,7 +541,7 @@ CreateIndexStatement IndexSpecification() throws ParseException:
   String indexName = null;
   boolean ifNotExists = false;
   Pair<Identifier,Identifier> nameComponents = null;
-  Pair<List<String>, TypeExpression> fieldPair = null;
+  Pair<Integer, Pair<List<String>, TypeExpression>> fieldPair = null;
   IndexParams indexType = null;
   boolean enforced = false;
 }
@@ -547,11 +551,13 @@ CreateIndexStatement IndexSpecification() throws ParseException:
   "on" nameComponents = QualifiedName()
   <LEFTPAREN> ( fieldPair = OpenField()
     {
-      cis.addFieldExprPair(fieldPair);
+       cis.addFieldExprPair(fieldPair.second);
+       cis.addFieldIndexIndicator(fieldPair.first);
     }
   ) (<COMMA> fieldPair = OpenField()
     {
-      cis.addFieldExprPair(fieldPair);
+       cis.addFieldExprPair(fieldPair.second);
+       cis.addFieldIndexIndicator(fieldPair.first);
     }
   )* <RIGHTPAREN> ( "type" indexType = IndexType() )? ( "enforced" { enforced = true; } )?
     {
@@ -817,23 +823,26 @@ FunctionSignature FunctionSignature() throws ParseException:
     }
 }
 
-List<List<String>> PrimaryKey() throws ParseException:
+Pair<List<Integer>, List<List<String>>> PrimaryKey() throws ParseException:
 {
-  List<String> tmp = null;
-  List<List<String>> primaryKeyFields = new ArrayList<List<String>>();
+   Pair<Integer, List<String>> tmp = null;
+   List<Integer> keyFieldSourceIndicators = new ArrayList<Integer>();
+   List<List<String>> primaryKeyFields = new ArrayList<List<String>>();
 }
 {
   "primary" "key" tmp = NestedField()
     {
-      primaryKeyFields.add(tmp);
+      keyFieldSourceIndicators.add(tmp.first);
+      primaryKeyFields.add(tmp.second);
     }
   ( <COMMA> tmp = NestedField()
     {
-      primaryKeyFields.add(tmp);
+      keyFieldSourceIndicators.add(tmp.first);
+      primaryKeyFields.add(tmp.second);
     }
   )*
     {
-      return primaryKeyFields;
+      return new Pair<List<Integer>, List<List<String>>> (keyFieldSourceIndicators, primaryKeyFields);
     }
 }
 
@@ -1381,23 +1390,25 @@ String Identifier() throws ParseException:
   )
 }
 
-Pair<List<String>, TypeExpression> OpenField() throws ParseException:
+Pair<Integer, Pair<List<String>, TypeExpression>> OpenField() throws ParseException:
 {
   TypeExpression fieldType = null;
-  List<String> fieldList = null;
+  Pair<Integer, List<String>> fieldList = null;
 }
 {
   fieldList = NestedField()
   ( <COLON> fieldType =  IndexedTypeExpr() )?
   {
-    return new Pair<List<String>, TypeExpression>(fieldList, fieldType);
+    return new Pair<Integer, Pair<List<String>, TypeExpression>>
+           (fieldList.first, new Pair<List<String>, TypeExpression>(fieldList.second, fieldType));
   }
 }
 
-List<String> NestedField() throws ParseException:
+Pair<Integer, List<String>> NestedField() throws ParseException:
 {
   List<String> exprList = new ArrayList<String>();
   String lit = null;
+  int source = 0;
 }
 {
   lit = Identifier()
@@ -1405,14 +1416,14 @@ List<String> NestedField() throws ParseException:
     boolean meetParens = false;
   }
   (
+    LOOKAHEAD(1)
     <LEFTPAREN><RIGHTPAREN>
     {
-        if(lit.equals("meta")){
-            exprList.add(lit+"()");
-        }else{
+        if(!lit.equals("meta")){
             throw new ParseException("The string before () has to be \"meta\".");
         }
         meetParens = true;
+        source = 1;
     }
   )?
   {
@@ -1427,7 +1438,7 @@ List<String> NestedField() throws ParseException:
     }
   )*
   {
-    return exprList;
+    return new Pair<Integer, List<String>>(source, exprList);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 6e8f41e..d3d681a 100644
--- a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -34,7 +34,8 @@ public class CreateIndexStatement implements Statement {
     private Identifier indexName;
     private Identifier dataverseName;
     private Identifier datasetName;
-    private List<Pair<List<String>, TypeExpression>> fieldExprs = new ArrayList<Pair<List<String>, TypeExpression>>();
+    private List<Pair<List<String>, TypeExpression>> fieldExprs = new ArrayList<>();
+    private List<Integer> fieldIndexIndicators = new ArrayList<>();
     private IndexType indexType = IndexType.BTREE;
     private boolean enforced;
     private boolean ifNotExists;
@@ -85,6 +86,14 @@ public class CreateIndexStatement implements Statement {
         this.fieldExprs.add(fp);
     }
 
+    public List<Integer> getFieldSourceIndicators() {
+        return fieldIndexIndicators;
+    }
+
+    public void addFieldIndexIndicator(Integer index) {
+        fieldIndexIndicators.add(index);
+    }
+
     public IndexType getIndexType() {
         return indexType;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDetailsDecl.java
----------------------------------------------------------------------
diff --git a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDetailsDecl.java b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDetailsDecl.java
deleted file mode 100644
index 1a42aa3..0000000
--- a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FeedDetailsDecl.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.lang.common.statement;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.common.functions.FunctionSignature;
-
-public class FeedDetailsDecl extends InternalDetailsDecl {
-    private final Map<String, String> configuration;
-    private final String adapterFactoryClassname;
-    private final FunctionSignature functionSignature;
-
-    public FeedDetailsDecl(String adapterFactoryClassname, Map<String, String> configuration,
-            FunctionSignature signature, List<List<String>> partitioningExpr, List<String> filterField) {
-        super(partitioningExpr, false, filterField, false);
-        this.adapterFactoryClassname = adapterFactoryClassname;
-        this.configuration = configuration;
-        this.functionSignature = signature;
-    }
-
-    public Map<String, String> getConfiguration() {
-        return configuration;
-    }
-
-    public String getAdapterFactoryClassname() {
-        return adapterFactoryClassname;
-    }
-
-    public FunctionSignature getSignature() {
-        return functionSignature;
-    }
-
-    public FunctionSignature getFunctionSignature() {
-        return functionSignature;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
----------------------------------------------------------------------
diff --git a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
index 431f9fb..41d97a8 100644
--- a/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
+++ b/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InternalDetailsDecl.java
@@ -22,13 +22,15 @@ import java.util.List;
 
 public class InternalDetailsDecl implements IDatasetDetailsDecl {
     private final List<List<String>> partitioningExprs;
+    private final List<Integer> keySourceIndicators;
     private final boolean autogenerated;
     private final boolean temp;
     private final List<String> filterField;
 
-    public InternalDetailsDecl(List<List<String>> partitioningExpr, boolean autogenerated, List<String> filterField,
-            boolean temp) {
+    public InternalDetailsDecl(List<List<String>> partitioningExpr, List<Integer> keySourceIndicators,
+            boolean autogenerated, List<String> filterField, boolean temp) {
         this.partitioningExprs = partitioningExpr;
+        this.keySourceIndicators = keySourceIndicators;
         this.autogenerated = autogenerated;
         this.filterField = filterField;
         this.temp = temp;
@@ -38,6 +40,10 @@ public class InternalDetailsDecl implements IDatasetDetailsDecl {
         return partitioningExprs;
     }
 
+    public List<Integer> getKeySourceIndicators() {
+        return keySourceIndicators;
+    }
+
     public boolean isAutogenerated() {
         return autogenerated;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/c9bfe259/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
----------------------------------------------------------------------
diff --git a/asterix-lang-sqlpp/src/main/javacc/SQLPP.html b/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
index 42aaa06..ebe7926 100644
--- a/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
+++ b/asterix-lang-sqlpp/src/main/javacc/SQLPP.html
@@ -403,7 +403,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod21">DatasetSpecification</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;EXTERNAL&gt; &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> &lt;USING&gt; <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( &lt;INTERNAL&gt; | &lt;TEMPORARY&gt; )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> ( &lt;COMMA&gt; <A HREF="#prod27">TypeName</A> )? &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( &lt;AUTOGENERATED&gt; )? ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34
 ">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( &lt;WITH&gt; &lt;FILTER&gt; &lt;ON&gt; <A HREF="#prod36">NestedField</A> )? )</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;EXTERNAL&gt; &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; <A HREF="#prod28">IfNotExists</A> &lt;USING&gt; <A HREF="#prod31">AdapterName</A> <A HREF="#prod32">Configuration</A> ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? | ( &lt;INTERNAL&gt; | &lt;TEMPORARY&gt; )? &lt;DATASET&gt; <A HREF="#prod30">QualifiedName</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; ( &lt;WITH&gt; <A HREF="#prod18">Identifier</A> &lt;LEFTPAREN&gt; <A HREF="#prod27">TypeName</A> &lt;RIGHTPAREN&gt; )? <A HREF="#prod28">IfNotExists</A> <A HREF="#prod35">PrimaryKey</A> ( &lt;AUTOGENERATED&gt; )? ( &lt;ON&gt; <A HREF="#prod18">Identifier</A> )? ( &lt;HINTS&gt; <A HREF="#prod33">Properties</A> 
 )? ( &lt;USING&gt; &lt;COMPACTION&gt; &lt;POLICY&gt; <A HREF="#prod34">CompactionPolicy</A> ( <A HREF="#prod32">Configuration</A> )? )? ( &lt;WITH&gt; &lt;FILTER&gt; &lt;ON&gt; <A HREF="#prod36">NestedField</A> )? )</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod16">RefreshExternalDatasetStatement</A></TD>
@@ -623,7 +623,7 @@
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod36">NestedField</A></TD>
 <TD ALIGN=CENTER VALIGN=BASELINE>::=</TD>
-<TD ALIGN=LEFT VALIGN=BASELINE>( &lt;ATT&gt; )? <A HREF="#prod18">Identifier</A> ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
+<TD ALIGN=LEFT VALIGN=BASELINE><A HREF="#prod18">Identifier</A> ( &lt;LEFTPAREN&gt; &lt;RIGHTPAREN&gt; )? ( &lt;DOT&gt; <A HREF="#prod18">Identifier</A> )*</TD>
 </TR>
 <TR>
 <TD ALIGN=RIGHT VALIGN=BASELINE><A NAME="prod40">QuotedString</A></TD>