Added filters to AsterixDB's primary and secondary indexes.

Change-Id: If16b2758bb25392c8dd0405d7d59ae503efad3f0
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/82
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index d70d9a8..d767eb2 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -104,6 +104,10 @@
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
         int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
         int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
+
+        int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
+        int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
+
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
@@ -115,7 +119,9 @@
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(),
                 jobGenParams.getRetainNull(), dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
-                jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), implConfig);
+                jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), implConfig, minFilterFieldIndexes,
+                maxFilterFieldIndexes);
+
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 706c734..209957f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -125,12 +125,15 @@
         }
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
 
+        int[] minFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMinFilterVars(), inputSchemas);
+        int[] maxFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMaxFilterVars(), inputSchemas);
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(
                 metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
                 jobGenParams.getRetainNull(), jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
                 jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
-                jobGenParams.getSimilarityThreshold());
+                jobGenParams.getSimilarityThreshold(), minFilterFieldIndexes, maxFilterFieldIndexes);
+
         // Contribute operator in hyracks job.
         builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
         builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
@@ -142,8 +145,9 @@
             AqlMetadataProvider metadataProvider, JobGenContext context, JobSpecification jobSpec,
             UnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainNull,
             String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
-            SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold)
-            throws AlgebricksException {
+            SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+
         try {
             IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
             IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
@@ -205,6 +209,30 @@
             ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys,
                     typeEnv, context);
 
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recordType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, recordType, context.getBinaryComparatorFactoryProvider());
+
+            int[] filterFields = null;
+            int[] invertedIndexFields = null;
+            int[] filterFieldsForNonBulkLoadOps = null;
+            int[] invertedIndexFieldsForNonBulkLoadOps = null;
+            if (filterTypeTraits != null) {
+                filterFields = new int[1];
+                filterFields[0] = numTokenKeys + numPrimaryKeys;
+                invertedIndexFields = new int[numTokenKeys + numPrimaryKeys];
+                for (int k = 0; k < invertedIndexFields.length; k++) {
+                    invertedIndexFields[k] = k;
+                }
+
+                filterFieldsForNonBulkLoadOps = new int[1];
+                filterFieldsForNonBulkLoadOps[0] = numPrimaryKeys + numSecondaryKeys;
+                invertedIndexFieldsForNonBulkLoadOps = new int[numPrimaryKeys + numSecondaryKeys];
+                for (int k = 0; k < invertedIndexFieldsForNonBulkLoadOps.length; k++) {
+                    invertedIndexFieldsForNonBulkLoadOps[k] = k;
+                }
+            }
+
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
                     .splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(),
@@ -227,21 +255,26 @@
                         compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate());
+                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps);
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
                         compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate());
+                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps);
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
                     jobSpec, queryField, appContext.getStorageManagerInterface(), secondarySplitsAndConstraint.first,
                     appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                     invListsTypeTraits, invListsComparatorFactories, dataflowHelperFactory, queryTokenizerFactory,
                     searchModifierFactory, outputRecDesc, retainInput, retainNull, context.getNullWriterFactory(),
-                    NoOpOperationCallbackFactory.INSTANCE);
+                    NoOpOperationCallbackFactory.INSTANCE, minFilterFieldIndexes, maxFilterFieldIndexes);
+
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
                     secondarySplitsAndConstraint.second);
         } catch (MetadataException e) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 33ad11b..cf6b24f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -73,6 +73,10 @@
         RTreeJobGenParams jobGenParams = new RTreeJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
+
+        int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
+        int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
+
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         Dataset dataset = mp.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
@@ -83,11 +87,12 @@
         }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
                 builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(),
-                jobGenParams.getRetainNull(), dataset, jobGenParams.getIndexName(), keyIndexes);
+                jobGenParams.getRetainNull(), dataset, jobGenParams.getIndexName(), keyIndexes, minFilterFieldIndexes,
+                maxFilterFieldIndexes);
+
         builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
-
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index d4ff954..0e2aabd 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -61,6 +61,7 @@
 import edu.uci.ics.asterix.optimizer.rules.SweepIllegalNonfunctionalFunctions;
 import edu.uci.ics.asterix.optimizer.rules.UnnestToDataScanRule;
 import edu.uci.ics.asterix.optimizer.rules.am.IntroduceJoinAccessMethodRule;
+import edu.uci.ics.asterix.optimizer.rules.am.IntroduceLSMComponentFilterRule;
 import edu.uci.ics.asterix.optimizer.rules.am.IntroduceSelectAccessMethodRule;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -227,6 +228,7 @@
         List<IAlgebraicRewriteRule> accessMethod = new LinkedList<IAlgebraicRewriteRule>();
         accessMethod.add(new IntroduceSelectAccessMethodRule());
         accessMethod.add(new IntroduceJoinAccessMethodRule());
+        accessMethod.add(new IntroduceLSMComponentFilterRule());
         accessMethod.add(new IntroduceSecondaryIndexInsertDeleteRule());
         accessMethod.add(new RemoveUnusedOneToOneEquiJoinRule());
         accessMethod.add(new PushSimilarityFunctionsBelowJoin());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
index a2b8ef5..4187136 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
@@ -25,7 +25,6 @@
 import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataImplConfig;
-import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index f2ca7d3..6735ffd 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -145,32 +146,45 @@
             List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
             List<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+
             for (String secondaryKey : secondaryKeyFields) {
-                Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(recordVar.get(0)));
-                String[] fieldNames = recType.getFieldNames();
-                int pos = -1;
-                for (int j = 0; j < fieldNames.length; j++) {
-                    if (fieldNames[j].equals(secondaryKey)) {
-                        pos = j;
-                        break;
-                    }
+                prepareVarAndExpression(secondaryKey, recType.getFieldNames(), recordVar, expressions,
+                        secondaryKeyVars, context);
+            }
+            String additionalFilteringField = ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterField();
+            List<LogicalVariable> additionalFilteringVars = null;
+            List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions = null;
+            List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
+            AssignOperator additionalFilteringAssign = null;
+
+            if (additionalFilteringField != null) {
+                additionalFilteringVars = new ArrayList<LogicalVariable>();
+                additionalFilteringAssignExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+                additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+                prepareVarAndExpression(additionalFilteringField, recType.getFieldNames(), recordVar,
+                        additionalFilteringAssignExpressions, additionalFilteringVars, context);
+                additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
+                        additionalFilteringAssignExpressions);
+                for (LogicalVariable var : additionalFilteringVars) {
+                    additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(
+                            new VariableReferenceExpression(var)));
                 }
-                // Assumes the indexed field is in the closed portion of the type.
-                Mutable<ILogicalExpression> indexRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
-                        new AsterixConstantValue(new AInt32(pos))));
-                AbstractFunctionCallExpression func = new ScalarFunctionCallExpression(
-                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
-                expressions.add(new MutableObject<ILogicalExpression>(func));
-                LogicalVariable newVar = context.newVar();
-                secondaryKeyVars.add(newVar);
             }
 
             AssignOperator assign = new AssignOperator(secondaryKeyVars, expressions);
             ProjectOperator project = new ProjectOperator(projectVars);
-            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+
+            if (additionalFilteringAssign != null) {
+                additionalFilteringAssign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+                assign.getInputs().add(new MutableObject<ILogicalOperator>(additionalFilteringAssign));
+            } else {
+                assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+            }
             project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             context.computeAndSetTypeEnvironmentForOperator(project);
+            if (additionalFilteringAssign != null) {
+                context.computeAndSetTypeEnvironmentForOperator(additionalFilteringAssign);
+            }
             context.computeAndSetTypeEnvironmentForOperator(assign);
             if (index.getIndexType() == IndexType.BTREE
                     || index.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
@@ -187,6 +201,7 @@
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
+                indexUpdate.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                 indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                 currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
@@ -229,6 +244,7 @@
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
+                indexUpdate.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                 indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
                 currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
@@ -243,6 +259,28 @@
     }
 
     @SuppressWarnings("unchecked")
+    private void prepareVarAndExpression(String field, String[] fieldNames, List<LogicalVariable> recordVar,
+            List<Mutable<ILogicalExpression>> expressions, List<LogicalVariable> vars, IOptimizationContext context) {
+        Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                recordVar.get(0)));
+        int pos = -1;
+        for (int j = 0; j < fieldNames.length; j++) {
+            if (fieldNames[j].equals(field)) {
+                pos = j;
+                break;
+            }
+        }
+        // Assumes the indexed field is in the closed portion of the type.
+        Mutable<ILogicalExpression> indexRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
+                new AsterixConstantValue(new AInt32(pos))));
+        AbstractFunctionCallExpression func = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
+        expressions.add(new MutableObject<ILogicalExpression>(func));
+        LogicalVariable newVar = context.newVar();
+        vars.add(newVar);
+    }
+
+    @SuppressWarnings("unchecked")
     private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
             IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
         List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<Mutable<ILogicalExpression>>();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
new file mode 100644
index 0000000..f90a72e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -0,0 +1,409 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ *     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 edu.uci.ics.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.DatasetDataSource;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceLSMComponentFilterRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        if (!checkIfRuleIsApplicable(opRef, context)) {
+            return false;
+        }
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        ILogicalExpression condExpr = ((SelectOperator) op).getCondition().getValue();
+        AccessMethodAnalysisContext analysisCtx = analyzeCondition(condExpr);
+        if (analysisCtx.matchedFuncExprs.isEmpty()) {
+            return false;
+        }
+
+        Dataset dataset = getDataset(op, context);
+        String filterFieldName = null;
+        ARecordType recType = null;
+        if (dataset != null && dataset.getDatasetType() == DatasetType.INTERNAL) {
+            filterFieldName = DatasetUtils.getFilterField(dataset);
+            IAType itemType = ((AqlMetadataProvider) context.getMetadataProvider()).findType(
+                    dataset.getDataverseName(), dataset.getItemTypeName());
+            if (itemType.getTypeTag() == ATypeTag.RECORD) {
+                recType = (ARecordType) itemType;
+            }
+        }
+        if (filterFieldName == null || recType == null) {
+            return false;
+        }
+        List<Index> datasetIndexes = ((AqlMetadataProvider) context.getMetadataProvider()).getDatasetIndexes(
+                dataset.getDataverseName(), dataset.getDatasetName());
+
+        List<IOptimizableFuncExpr> optFuncExprs = new ArrayList<IOptimizableFuncExpr>();
+
+        for (int i = 0; i < analysisCtx.matchedFuncExprs.size(); i++) {
+            IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(i);
+            boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, recType, datasetIndexes);
+            if (found && optFuncExpr.getFieldName(0).compareTo(filterFieldName) == 0) {
+                optFuncExprs.add(optFuncExpr);
+            }
+        }
+        if (optFuncExprs.isEmpty()) {
+            return false;
+        }
+        changePlan(optFuncExprs, op, dataset, context);
+
+        OperatorPropertiesUtil.typeOpRec(opRef, context);
+        context.addToDontApplySet(this, op);
+        return true;
+    }
+
+    private AssignOperator createAssignOperator(List<IOptimizableFuncExpr> optFuncExprs,
+            List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOptimizationContext context) {
+        List<LogicalVariable> assignKeyVarList = new ArrayList<LogicalVariable>();
+        List<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+
+        for (IOptimizableFuncExpr optFuncExpr : optFuncExprs) {
+            ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr()
+                    .getFunctionIdentifier());
+            ILogicalExpression searchKeyExpr = new ConstantExpression(optFuncExpr.getConstantVal(0));
+            LogicalVariable var = context.newVar();
+            assignKeyExprList.add(new MutableObject<ILogicalExpression>(searchKeyExpr));
+            assignKeyVarList.add(var);
+            if (ck == ComparisonKind.GE || ck == ComparisonKind.GT) {
+                minFilterVars.add(var);
+            } else if (ck == ComparisonKind.LE || ck == ComparisonKind.LT) {
+                maxFilterVars.add(var);
+            } else if (ck == ComparisonKind.EQ) {
+                minFilterVars.add(var);
+                maxFilterVars.add(var);
+            }
+        }
+        return new AssignOperator(assignKeyVarList, assignKeyExprList);
+    }
+
+    private void changePlan(List<IOptimizableFuncExpr> optFuncExprs, AbstractLogicalOperator op, Dataset dataset,
+            IOptimizationContext context) throws AlgebricksException {
+
+        AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        while (descendantOp != null) {
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
+                AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
+                if (dataset.getDatasetName().compareTo(((DatasetDataSource) ds).getDataset().getDatasetName()) == 0) {
+                    List<LogicalVariable> minFilterVars = new ArrayList<LogicalVariable>();
+                    List<LogicalVariable> maxFilterVars = new ArrayList<LogicalVariable>();
+
+                    AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars, context);
+
+                    dataSourceScanOp.setMinFilterVars(minFilterVars);
+                    dataSourceScanOp.setMaxFilterVars(maxFilterVars);
+
+                    List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();;
+                    for (LogicalVariable var : assignOp.getVariables()) {
+                        additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(
+                                new VariableReferenceExpression(var)));
+                    }
+
+                    dataSourceScanOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+
+                    assignOp.getInputs().add(
+                            new MutableObject<ILogicalOperator>(dataSourceScanOp.getInputs().get(0).getValue()));
+                    dataSourceScanOp.getInputs().get(0).setValue(assignOp);
+                }
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    FunctionIdentifier fid = f.getFunctionIdentifier();
+                    if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                        throw new IllegalStateException();
+                    }
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    if (dataset.getDatasetName().compareTo(jobGenParams.datasetName) == 0) {
+                        List<LogicalVariable> minFilterVars = new ArrayList<LogicalVariable>();
+                        List<LogicalVariable> maxFilterVars = new ArrayList<LogicalVariable>();
+
+                        AssignOperator assignOp = createAssignOperator(optFuncExprs, minFilterVars, maxFilterVars,
+                                context);
+
+                        unnestMapOp.setMinFilterVars(minFilterVars);
+                        unnestMapOp.setMaxFilterVars(maxFilterVars);
+
+                        List<Mutable<ILogicalExpression>> additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();;
+                        for (LogicalVariable var : assignOp.getVariables()) {
+                            additionalFilteringExpressions.add(new MutableObject<ILogicalExpression>(
+                                    new VariableReferenceExpression(var)));
+                        }
+                        unnestMapOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+                        assignOp.getInputs().add(
+                                new MutableObject<ILogicalOperator>(unnestMapOp.getInputs().get(0).getValue()));
+                        unnestMapOp.getInputs().get(0).setValue(assignOp);
+                    }
+                }
+            }
+            if (descendantOp.getInputs().isEmpty()) {
+                break;
+            }
+            descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+        }
+    }
+
+    private Dataset getDataset(AbstractLogicalOperator op, IOptimizationContext context) throws AlgebricksException {
+        AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        while (descendantOp != null) {
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
+                AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
+                return ((DatasetDataSource) ds).getDataset();
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    FunctionIdentifier fid = f.getFunctionIdentifier();
+                    if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                        throw new IllegalStateException();
+                    }
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    return ((AqlMetadataProvider) context.getMetadataProvider()).findDataset(
+                            jobGenParams.dataverseName, jobGenParams.datasetName);
+                }
+            }
+            if (descendantOp.getInputs().isEmpty()) {
+                break;
+            }
+            descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+        }
+        return null;
+    }
+
+    private boolean checkIfRuleIsApplicable(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+        // First check that the operator is a select and its condition is a function call.
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+
+        ILogicalExpression condExpr = ((SelectOperator) op).getCondition().getValue();
+        if (condExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        return true;
+    }
+
+    private AccessMethodAnalysisContext analyzeCondition(ILogicalExpression cond) {
+        AccessMethodAnalysisContext analysisCtx = new AccessMethodAnalysisContext();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        if (funcIdent != AlgebricksBuiltinFunctions.OR) {
+            analyzeFunctionExpr(funcExpr, analysisCtx);
+            for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                ILogicalExpression argExpr = arg.getValue();
+                if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                    continue;
+                }
+                analyzeFunctionExpr((AbstractFunctionCallExpression) argExpr, analysisCtx);
+            }
+        }
+        return analysisCtx;
+    }
+
+    private void analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, AccessMethodAnalysisContext analysisCtx) {
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        if (funcIdent == AlgebricksBuiltinFunctions.LE || funcIdent == AlgebricksBuiltinFunctions.GE
+                || funcIdent == AlgebricksBuiltinFunctions.LT || funcIdent == AlgebricksBuiltinFunctions.GT
+                || funcIdent == AlgebricksBuiltinFunctions.EQ) {
+            AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+        }
+    }
+
+    private boolean findMacthedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
+            Dataset dataset, ARecordType recType, List<Index> datasetIndexes) throws AlgebricksException {
+        AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        while (descendantOp != null) {
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                AssignOperator assignOp = (AssignOperator) descendantOp;
+                List<LogicalVariable> varList = assignOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    int funcVarIndex = optFuncExpr.findLogicalVar(var);
+                    if (funcVarIndex == -1) {
+                        continue;
+                    }
+                    String fieldName = getFieldNameFromSubAssignTree(optFuncExpr, descendantOp, varIndex, recType);
+                    if (fieldName == null) {
+                        return false;
+                    }
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    return true;
+                }
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator scanOp = (DataSourceScanOperator) descendantOp;
+                List<LogicalVariable> varList = scanOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    int funcVarIndex = optFuncExpr.findLogicalVar(var);
+                    if (funcVarIndex == -1) {
+                        continue;
+                    }
+                    // The variable value is one of the partitioning fields.
+                    String fieldName = DatasetUtils.getPartitioningKeys(dataset).get(varIndex);
+                    if (fieldName == null) {
+                        return false;
+                    }
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    return true;
+                }
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                List<LogicalVariable> varList = unnestMapOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    int funcVarIndex = optFuncExpr.findLogicalVar(var);
+                    if (funcVarIndex == -1) {
+                        continue;
+                    }
+
+                    String indexName = null;
+                    Index index = null;
+                    ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                    if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                        FunctionIdentifier fid = f.getFunctionIdentifier();
+                        if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                            throw new IllegalStateException();
+                        }
+                        AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                        jobGenParams.readFromFuncArgs(f.getArguments());
+                        indexName = jobGenParams.indexName;
+                        for (Index idx : datasetIndexes) {
+                            if (idx.getIndexName().compareTo(indexName) == 0) {
+                                index = idx;
+                                break;
+                            }
+                        }
+                    }
+
+                    int numSecondaryKeys = AccessMethodUtils.getNumSecondaryKeys(index, recType);
+                    String fieldName;
+                    if (varIndex >= numSecondaryKeys) {
+                        fieldName = DatasetUtils.getPartitioningKeys(dataset).get(varIndex - numSecondaryKeys);
+                    } else {
+                        fieldName = index.getKeyFieldNames().get(varIndex);
+                    }
+                    if (fieldName == null) {
+                        return false;
+                    }
+                    optFuncExpr.setFieldName(funcVarIndex, fieldName);
+                    return true;
+                }
+            }
+
+            if (descendantOp.getInputs().isEmpty()) {
+                break;
+            }
+            descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+        }
+        return false;
+    }
+
+    private String getFieldNameFromSubAssignTree(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
+            int varIndex, ARecordType recType) {
+        AbstractLogicalExpression expr = null;
+        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AssignOperator assignOp = (AssignOperator) op;
+            expr = (AbstractLogicalExpression) assignOp.getExpressions().get(varIndex).getValue();
+        }
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return null;
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        if (funcIdent == AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME) {
+            ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
+            if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return null;
+            }
+            ConstantExpression constExpr = (ConstantExpression) nameArg;
+            return ((AString) ((AsterixConstantValue) constExpr.getValue()).getObject()).getStringValue();
+        } else if (funcIdent == AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
+            ILogicalExpression idxArg = funcExpr.getArguments().get(1).getValue();
+            if (idxArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                return null;
+            }
+            ConstantExpression constExpr = (ConstantExpression) idxArg;
+            int fieldIndex = ((AInt32) ((AsterixConstantValue) constExpr.getValue()).getObject()).getIntegerValue();
+            return recType.getFieldNames()[fieldIndex];
+        }
+
+        ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
+        if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return null;
+        }
+
+        return null;
+    }
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index 580e53b..6922909 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -240,44 +240,63 @@
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
             List<String> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
             for (String keyFieldName : partitionKeys) {
-                IFunctionInfo finfoAccess = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
-                @SuppressWarnings("unchecked")
-                ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                                new AString(keyFieldName)))));
-                f.substituteVar(METADATA_DUMMY_VAR, resVar);
-                exprs.add(new MutableObject<ILogicalExpression>(f));
-                LogicalVariable v = context.newVar();
-                vars.add(v);
-                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+                prepareVarAndExpression(keyFieldName, resVar, vars, exprs, varRefsForLoading);
             }
+
+            String additionalFilteringField = DatasetUtils.getFilterField(targetDatasource.getDataset());
+            List<LogicalVariable> additionalFilteringVars = null;
+            List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions = null;
+            List<Mutable<ILogicalExpression>> additionalFilteringExpressions = null;
+            AssignOperator additionalFilteringAssign = null;
+            if (additionalFilteringField != null) {
+                additionalFilteringVars = new ArrayList<LogicalVariable>();
+                additionalFilteringAssignExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+                additionalFilteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+
+                prepareVarAndExpression(additionalFilteringField, resVar, additionalFilteringVars,
+                        additionalFilteringAssignExpressions, additionalFilteringExpressions);
+
+                additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
+                        additionalFilteringAssignExpressions);
+
+            }
+
             AssignOperator assign = new AssignOperator(vars, exprs);
-            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+
+            if (additionalFilteringAssign != null) {
+                additionalFilteringAssign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+                assign.getInputs().add(new MutableObject<ILogicalOperator>(additionalFilteringAssign));
+            } else {
+                assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+            }
+
             Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
                     resVar));
             ILogicalOperator leafOperator = null;
 
             switch (stmt.getKind()) {
                 case INSERT: {
-                    ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.INSERT);
+                    InsertDeleteOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef,
+                            varRefsForLoading, InsertDeleteOperator.Kind.INSERT);
+                    insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                     insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                     leafOperator = new SinkOperator();
                     leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
                     break;
                 }
                 case DELETE: {
-                    ILogicalOperator deleteOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.DELETE);
+                    InsertDeleteOperator deleteOp = new InsertDeleteOperator(targetDatasource, varRef,
+                            varRefsForLoading, InsertDeleteOperator.Kind.DELETE);
+                    deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                     deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                     leafOperator = new SinkOperator();
                     leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
                     break;
                 }
                 case CONNECT_FEED: {
-                    ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
-                            InsertDeleteOperator.Kind.INSERT);
+                    InsertDeleteOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef,
+                            varRefsForLoading, InsertDeleteOperator.Kind.INSERT);
+                    insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
                     insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                     leafOperator = new SinkOperator();
                     leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
@@ -291,6 +310,24 @@
         return plan;
     }
 
+    @SuppressWarnings("unchecked")
+    private void prepareVarAndExpression(String field, LogicalVariable resVar,
+            List<LogicalVariable> additionalFilteringVars,
+            List<Mutable<ILogicalExpression>> additionalFilteringAssignExpressions,
+            List<Mutable<ILogicalExpression>> varRefs) {
+        IFunctionInfo finfoAccess = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
+
+        ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                        new AString(field)))));
+        f.substituteVar(METADATA_DUMMY_VAR, resVar);
+        additionalFilteringAssignExpressions.add(new MutableObject<ILogicalExpression>(f));
+        LogicalVariable v = context.newVar();
+        additionalFilteringVars.add(v);
+        varRefs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+    }
+
     private DatasetDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
             String datasetName) throws AlgebricksException {
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
diff --git a/asterix-app/data/fbm-with-send-time.adm b/asterix-app/data/fbm-with-send-time.adm
new file mode 100644
index 0000000..c5c018f
--- /dev/null
+++ b/asterix-app/data/fbm-with-send-time.adm
@@ -0,0 +1,15 @@
+{"message-id":1,"author-id":3,"in-response-to":2,"sender-location":point("47.16,77.75"),"message":" love sprint its shortcut-menu is awesome:)","send-time":datetime("2012-01-20T10:10:00")}
+{"message-id":2,"author-id":1,"in-response-to":4,"sender-location":point("41.66,80.87"),"message":" dislike iphone its touch-screen is horrible","send-time":datetime("2012-02-20T10:10:00")}
+{"message-id":3,"author-id":2,"in-response-to":4,"sender-location":point("48.09,81.01"),"message":" like samsung the plan is amazing","send-time":datetime("2012-03-20T10:10:00")}
+{"message-id":4,"author-id":1,"in-response-to":2,"sender-location":point("37.73,97.04"),"message":" can't stand at&t the network is horrible:(","send-time":datetime("2012-04-20T10:10:00")}
+{"message-id":5,"author-id":6,"in-response-to":2,"sender-location":point("34.7,90.76"),"message":" love sprint the customization is mind-blowing","send-time":datetime("2012-05-20T10:10:00")}
+{"message-id":6,"author-id":2,"in-response-to":1,"sender-location":point("31.5,75.56"),"message":" like t-mobile its platform is mind-blowing","send-time":datetime("2012-06-20T10:10:00")}
+{"message-id":7,"author-id":5,"in-response-to":15,"sender-location":point("32.91,85.05"),"message":" dislike sprint the speed is horrible","send-time":datetime("2012-07-20T10:10:00")}
+{"message-id":8,"author-id":1,"in-response-to":11,"sender-location":point("40.33,80.87"),"message":" like verizon the 3G is awesome:)","send-time":datetime("2012-08-20T10:10:00")}
+{"message-id":9,"author-id":3,"in-response-to":12,"sender-location":point("34.45,96.48"),"message":" love verizon its wireless is good","send-time":datetime("2012-09-20T10:10:00")}
+{"message-id":10,"author-id":1,"in-response-to":12,"sender-location":point("42.5,70.01"),"message":" can't stand motorola the touch-screen is terrible","send-time":datetime("2012-10-20T10:10:00")}
+{"message-id":11,"author-id":1,"in-response-to":1,"sender-location":point("38.97,77.49"),"message":" can't stand at&t its plan is terrible","send-time":datetime("2012-11-20T10:10:00")}
+{"message-id":12,"author-id":10,"in-response-to":6,"sender-location":point("42.26,77.76"),"message":" can't stand t-mobile its voicemail-service is OMG:(","send-time":datetime("2012-12-20T10:10:00")}
+{"message-id":13,"author-id":10,"in-response-to":4,"sender-location":point("42.77,78.92"),"message":" dislike iphone the voice-command is bad:(","send-time":datetime("2013-08-20T10:10:00")}
+{"message-id":14,"author-id":9,"in-response-to":12,"sender-location":point("41.33,85.28"),"message":" love at&t its 3G is good:)","send-time":datetime("2013-09-20T10:10:00")}
+{"message-id":15,"author-id":7,"in-response-to":11,"sender-location":point("44.47,67.11"),"message":" like iphone the voicemail-service is awesome","send-time":datetime("2014-01-20T10:10:00")}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 7014c92..b6d09b1 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -493,9 +493,13 @@
                     } else {
                         validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx);
                     }
+                    String filterField = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getFilterField();
+                    if (filterField != null) {
+                        aRecordType.validateFilterField(filterField);
+                    }
                     datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
                             InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
-                            ngName, autogenerated, compactionPolicy, compactionPolicyProperties);
+                            ngName, autogenerated, compactionPolicy, compactionPolicyProperties, filterField);
                     break;
                 }
                 case EXTERNAL: {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 19283ff..49a6408 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -121,6 +121,22 @@
             return JobSpecificationUtils.createJobSpecification();
         }
 
+        Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
+                dataverseName);
+        IDataFormat format;
+        try {
+            format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+
+        ARecordType itemType = (ARecordType) metadataProvider.findType(dataverseName, dataset.getItemTypeName());
+
+        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+                itemType, format.getBinaryComparatorFactoryProvider());
+        int[] filterFields = DatasetUtils.createFilterFields(dataset);
+        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
         JobSpecification specPrimary = JobSpecificationUtils.createJobSpecification();
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
@@ -135,7 +151,8 @@
                         dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
                         new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true));
+                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
+                        filterCmpFactories, btreeFields, filterFields));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
 
@@ -162,7 +179,13 @@
         IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                 itemType, format.getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
-        int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
+        int[] bloomFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
+
+        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+                itemType, format.getBinaryComparatorFactoryProvider());
+        int[] filterFields = DatasetUtils.createFilterFields(dataset);
+        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
@@ -178,19 +201,20 @@
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                comparatorFactories, blooFilterKeyFields, true, dataset.getDatasetId(), compactionInfo.first,
-                compactionInfo.second);
+                comparatorFactories, bloomFilterKeyFields, true, dataset.getDatasetId(), compactionInfo.first,
+                compactionInfo.second, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(dataset
                                 .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                .getBloomFilterFalsePositiveRate(), true), localResourceFactoryProvider,
+                                .getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
+                        btreeFields, filterFields), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -223,7 +247,13 @@
         IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                 itemType, format.getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
-        int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
+        int[] bloomFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
+
+        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+                itemType, format.getBinaryComparatorFactoryProvider());
+        int[] filterFields = DatasetUtils.createFilterFields(dataset);
+        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
 
         ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
                 loadStmt.getProperties(), null, null, ExternalDatasetTransactionState.COMMIT, null, null);
@@ -232,10 +262,12 @@
                 spec, itemType, externalDatasetDetails, format, dataset);
         IOperatorDescriptor scanner = p.first;
         AlgebricksPartitionConstraint scannerPc = p.second;
-        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde, format);
+        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde, format,
+                filterTypeTraits == null ? false : true);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, scanner, scannerPc);
 
-        AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, format);
+        AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, format,
+                filterTypeTraits == null ? false : true);
         AlgebricksMetaOperatorDescriptor asterixOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { recDesc });
 
@@ -246,12 +278,20 @@
         for (int i = 0; i < numKeys; i++) {
             keys[i] = i + 1;
         }
+        int numFilterFields = 0;
+        if (filterTypeTraits != null) {
+            numFilterFields++;
+        }
+
         // Move key fields to front.
-        int[] fieldPermutation = new int[numKeys + 1];
+        int[] fieldPermutation = new int[numKeys + 1 + numFilterFields];
         for (int i = 0; i < numKeys; i++) {
             fieldPermutation[i] = i + 1;
         }
         fieldPermutation[numKeys] = 0;
+        if (numFilterFields > 0) {
+            fieldPermutation[numKeys + 1] = numKeys + 1;
+        }
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
@@ -261,6 +301,7 @@
         for (int i = 0; i < fs.length; i++) {
             sb.append(stringOf(fs[i]) + " ");
         }
+
         LOGGER.info("LOAD into File Splits: " + sb.toString());
 
         long numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
@@ -272,13 +313,14 @@
             btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
-                    comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
+                    comparatorFactories, bloomFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_TREE_FILL_FACTOR,
                     true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
                             compactionInfo.second, new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), true), NoOpOperationCallbackFactory.INSTANCE);
+                            storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
+                            filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
             AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                     splitsAndConstraint.second);
 
@@ -297,13 +339,14 @@
             btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
-                    comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
+                    comparatorFactories, bloomFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_TREE_FILL_FACTOR,
                     true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
                             compactionInfo.second, new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                            storageProperties.getBloomFilterFalsePositiveRate(), true), NoOpOperationCallbackFactory.INSTANCE);
+                            storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
+                            filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
             AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                     splitsAndConstraint.second);
 
@@ -324,20 +367,31 @@
     }
 
     private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType,
-            IDataFormat format) throws AlgebricksException {
+            IDataFormat format, boolean filterExist) throws AlgebricksException {
+        int numFilterFields = 0;
+        if (filterExist) {
+            numFilterFields++;
+        }
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         int numKeys = partitioningKeys.size();
-        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
+        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys + numFilterFields];
         for (int i = 0; i < numKeys; i++) {
             Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
                     .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
             evalFactories[i] = evalFactoryAndType.first;
         }
-        int[] outColumns = new int[numKeys];
-        int[] projectionList = new int[numKeys + 1];
+        if (numFilterFields > 0) {
+            String filterField = DatasetUtils.getFilterField(dataset);
+            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
+                    .partitioningEvaluatorFactory(itemType, filterField);
+            evalFactories[numKeys] = evalFactoryAndType.first;
+        }
+
+        int[] outColumns = new int[numKeys + numFilterFields];
+        int[] projectionList = new int[numKeys + 1 + numFilterFields];
         projectionList[0] = 0;
 
-        for (int i = 0; i < numKeys; i++) {
+        for (int i = 0; i < numKeys + numFilterFields; i++) {
             outColumns[i] = i + 1;
             projectionList[i + 1] = i + 1;
         }
@@ -351,10 +405,15 @@
 
     @SuppressWarnings("rawtypes")
     private static RecordDescriptor computePayloadKeyRecordDescriptor(Dataset dataset, ARecordType itemType,
-            ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
+            ISerializerDeserializer payloadSerde, IDataFormat dataFormat, boolean filterExist)
+            throws AlgebricksException {
+        int numFilterFields = 0;
+        if (filterExist) {
+            numFilterFields++;
+        }
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         int numKeys = partitioningKeys.size();
-        ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
+        ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys + numFilterFields];
         recordFields[0] = payloadSerde;
         for (int i = 0; i < numKeys; i++) {
             IAType keyType;
@@ -366,6 +425,17 @@
             ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
             recordFields[i + 1] = keySerde;
         }
+        if (numFilterFields > 0) {
+            String filterField = DatasetUtils.getFilterField(dataset);
+            IAType type;
+            try {
+                type = itemType.getFieldType(filterField);
+            } catch (IOException e) {
+                throw new AlgebricksException(e);
+            }
+            ISerializerDeserializer serde = dataFormat.getSerdeProvider().getSerializerDeserializer(type);
+            recordFields[numKeys + 1] = serde;
+        }
         return new RecordDescriptor(recordFields);
     }
 
@@ -389,6 +459,12 @@
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
         int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
 
+        ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
+                itemType, format.getBinaryComparatorFactoryProvider());
+        int[] filterFields = DatasetUtils.createFilterFields(dataset);
+        int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
+
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
 
@@ -403,7 +479,8 @@
                         compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(
                                 dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), true), NoOpOperationCallbackFactory.INSTANCE);
+                        storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
+                        filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, compactOp, splitsAndConstraint.second);
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/ExternalIndexingOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/ExternalIndexingOperations.java
index bcd9ff9..7b45992 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/ExternalIndexingOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/ExternalIndexingOperations.java
@@ -26,13 +26,14 @@
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
 import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
@@ -87,9 +88,9 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexCompactOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.ExternalRTreeDataflowHelperFactory;
@@ -415,7 +416,7 @@
                         dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
                         new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), false));
+                        storageProperties.getBloomFilterFalsePositiveRate(), false, null, null, null, null));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 859e39c..17b0b57 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -99,7 +99,7 @@
                         dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
                         new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate(), false));
+                        storageProperties.getBloomFilterFalsePositiveRate(), false, null, null, null, null));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeOperationsHelper.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeOperationsHelper.java
index 597e835..076a844 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeOperationsHelper.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeOperationsHelper.java
@@ -17,6 +17,7 @@
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -25,8 +26,8 @@
 import edu.uci.ics.asterix.metadata.feeds.ExternalDataScanOperatorDescriptor;
 import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
-import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.ExternalBTreeWithBuddyLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -67,21 +68,23 @@
         IIndexDataflowHelperFactory indexDataflowHelperFactory;
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
-            ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                    true, dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
+            ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(secondaryTypeTraits,
+                    secondaryComparatorFactories, secondaryBloomFilterKeyFields, true, dataset.getDatasetId(),
+                    mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits, filterCmpFactories,
+                    secondaryBTreeFields, secondaryFilterFields);
             localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
                     LocalResource.LSMBTreeResource);
             indexDataflowHelperFactory = new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                     dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
                     new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate(), false);
+                    storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
+                    secondaryBTreeFields, secondaryFilterFields);
         } else {
             // External dataset local resource and dataflow helper
             int[] buddyBreeFields = new int[] { numSecondaryKeys };
             ILocalResourceMetadata localResourceMetadata = new ExternalBTreeWithBuddyLocalResourceMetadata(
-                    dataset.getDatasetId(), secondaryComparatorFactories, secondaryRecDesc.getTypeTraits(),
+                    dataset.getDatasetId(), secondaryComparatorFactories, secondaryTypeTraits,
                     mergePolicyFactory, mergePolicyFactoryProperties, buddyBreeFields);
             localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
                     LocalResource.ExternalBTreeWithBuddyResource);
@@ -94,7 +97,7 @@
         }
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
                 secondaryBloomFilterKeyFields, indexDataflowHelperFactory, localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -107,6 +110,7 @@
     @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             /*
              * In case of external data, this method is used to build loading jobs for both initial load on index creation
@@ -185,7 +189,8 @@
                             new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate(), false), BTree.DEFAULT_FILL_FACTOR);
+                                    .getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
+                            secondaryBTreeFields, secondaryFilterFields), GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
 
             // Connect the operators.
             spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
@@ -203,6 +208,10 @@
         }
     }
 
+    protected int getNumSecondaryKeys() {
+        return numSecondaryKeys;
+    }
+
     @Override
     public JobSpecification buildCompactJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
@@ -213,19 +222,20 @@
             compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                    secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             mergePolicyFactory, mergePolicyFactoryProperties,
                             new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate(), false), NoOpOperationCallbackFactory.INSTANCE);
+                                    .getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
+                            secondaryBTreeFields, secondaryFilterFields), NoOpOperationCallbackFactory.INSTANCE);
         } else {
             // External dataset
             compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                    secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
                     new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory, mergePolicyFactoryProperties,
                             new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexOperationsHelper.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexOperationsHelper.java
index 851de4a..890253a 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexOperationsHelper.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexOperationsHelper.java
@@ -120,6 +120,7 @@
     protected int[] primaryBloomFilterKeyFields;
     protected RecordDescriptor primaryRecDesc;
     protected IBinaryComparatorFactory[] secondaryComparatorFactories;
+    protected ITypeTraits[] secondaryTypeTraits;
     protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
@@ -128,6 +129,14 @@
     protected ILSMMergePolicyFactory mergePolicyFactory;
     protected Map<String, String> mergePolicyFactoryProperties;
 
+    protected int numFilterFields;
+    protected String filterFieldName;
+    protected ITypeTraits[] filterTypeTraits;
+    protected IBinaryComparatorFactory[] filterCmpFactories;
+    protected int[] secondaryFilterFields;
+    protected int[] primaryFilterFields;
+    protected int[] primaryBTreeFields;
+    protected int[] secondaryBTreeFields;
     protected List<ExternalFile> externalFiles;
 
     // Prevent public construction. Should be created via createIndexCreator().
@@ -197,6 +206,13 @@
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             numPrimaryKeys = ExternalIndexingOperations.getRIDSize(dataset);
         } else {
+            filterFieldName = DatasetUtils.getFilterField(dataset);
+            if (filterFieldName != null) {
+                numFilterFields = 1;
+            } else {
+                numFilterFields = 0;
+            }
+            
             numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
                     .splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
@@ -210,8 +226,40 @@
                 metadataProvider.getMetadataTxnContext());
         mergePolicyFactory = compactionInfo.first;
         mergePolicyFactoryProperties = compactionInfo.second;
+
+        if (numFilterFields > 0) {
+            setFilterTypeTraitsAndComparators();
+        }
     }
 
+    protected void setFilterTypeTraitsAndComparators() throws AlgebricksException {
+        filterTypeTraits = new ITypeTraits[numFilterFields];
+        filterCmpFactories = new IBinaryComparatorFactory[numFilterFields];
+        secondaryFilterFields = new int[numFilterFields];
+        primaryFilterFields = new int[numFilterFields];
+        primaryBTreeFields = new int[numPrimaryKeys + 1];
+        secondaryBTreeFields = new int[numSecondaryKeys + numPrimaryKeys];
+        for (int i = 0; i < primaryBTreeFields.length; i++) {
+            primaryBTreeFields[i] = i;
+        }
+        for (int i = 0; i < secondaryBTreeFields.length; i++) {
+            secondaryBTreeFields[i] = i;
+        }
+
+        IAType type;
+        try {
+            type = itemType.getFieldType(filterFieldName);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+        filterCmpFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true);
+        filterTypeTraits[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
+        secondaryFilterFields[0] = getNumSecondaryKeys() + numPrimaryKeys;
+        primaryFilterFields[0] = numPrimaryKeys + 1;
+    }
+
+    protected abstract int getNumSecondaryKeys();
+
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         int numPrimaryKeys = partitioningKeys.size();
@@ -240,15 +288,16 @@
 
     protected void setSecondaryRecDescAndComparators(IndexType indexType, List<String> secondaryKeyFields,
             int gramLength, AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
-        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
+        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys + numFilterFields];
         if (indexType == IndexType.RTREE) {
             secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
         } else {
             secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
         }
         secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
-        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys
+                + numFilterFields];
+        secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
         ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
         IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
@@ -286,7 +335,15 @@
                 }
             }
         }
-        secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        if (numFilterFields > 0) {
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat()
+                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
+            IAType type = keyTypePair.first;
+            ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+            secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
+        }
+        secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
     }
 
     protected AbstractOperatorDescriptor createDummyKeyProviderOp(JobSpecification spec) throws AsterixException,
@@ -336,8 +393,10 @@
                         mergePolicyFactory, mergePolicyFactoryProperties, new PrimaryIndexOperationTrackerProvider(
                                 dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                .getBloomFilterFalsePositiveRate(), true), false, false, null,
-                searchCallbackFactory);
+                                .getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
+                        primaryBTreeFields, primaryFilterFields), false, false, null,
+                searchCallbackFactory, null, null);
+
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -345,18 +404,22 @@
 
     protected AlgebricksMetaOperatorDescriptor createAssignOp(JobSpecification spec,
             BTreeSearchOperatorDescriptor primaryScanOp, int numSecondaryKeyFields) throws AlgebricksException {
-        int[] outColumns = new int[numSecondaryKeyFields];
-        int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys];
-        for (int i = 0; i < numSecondaryKeyFields; i++) {
-            outColumns[i] = numPrimaryKeys + i + 1;
+        int[] outColumns = new int[numSecondaryKeyFields + numFilterFields];
+        int[] projectionList = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
+        for (int i = 0; i < numSecondaryKeyFields + numFilterFields; i++) {
+            outColumns[i] = numPrimaryKeys + i;
         }
         int projCount = 0;
         for (int i = 0; i < numSecondaryKeyFields; i++) {
-            projectionList[projCount++] = numPrimaryKeys + i + 1;
+            projectionList[projCount++] = numPrimaryKeys + i;
         }
         for (int i = 0; i < numPrimaryKeys; i++) {
             projectionList[projCount++] = i;
         }
+        if (numFilterFields > 0) {
+            projectionList[projCount++] = numPrimaryKeys + numSecondaryKeyFields;
+        }
+
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
         for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
             sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
@@ -385,13 +448,13 @@
     protected TreeIndexBulkLoadOperatorDescriptor createTreeIndexBulkLoadOp(JobSpecification spec,
             int numSecondaryKeyFields, IIndexDataflowHelperFactory dataflowHelperFactory, float fillFactor)
             throws MetadataException, AlgebricksException {
-        int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys];
-        for (int i = 0; i < numSecondaryKeyFields + numPrimaryKeys; i++) {
+        int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
+        for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
         }
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
                 secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint, false,
                 dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
@@ -512,7 +575,7 @@
         ExternalIndexBulkModifyOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkModifyOperatorDescriptor(
                 spec, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
                 dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE, deletedFiles, fieldPermutation,
                 fillFactor, numElementsHint);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexOperationsHelper.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexOperationsHelper.java
index e6202b7..d7ff27c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexOperationsHelper.java
@@ -77,6 +77,9 @@
     private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
     private RecordDescriptor tokenKeyPairRecDesc;
     private boolean isPartitioned;
+    private int[] invertedIndexFields;
+    private int[] invertedIndexFieldsForNonBulkLoadOps;
+    private int[] secondaryFilterFieldsForNonBulkLoadOps;
 
     protected SecondaryInvertedIndexOperationsHelper(PhysicalOptimizationConfig physOptConf,
             IAsterixPropertiesProvider propertiesProvider) {
@@ -102,9 +105,10 @@
         }
         // Prepare record descriptor used in the assign op, and the optional
         // select op.
-        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
-        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+        secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys + numFilterFields];
+        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys
+                + numFilterFields];
+        secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
         ITypeTraitProvider typeTraitProvider = FormatUtils.getDefaultFormat().getTypeTraitProvider();
         for (int i = 0; i < numSecondaryKeys; i++) {
@@ -117,7 +121,15 @@
             secondaryRecFields[i] = keySerde;
             secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(secondaryKeyType);
         }
-        secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        if (numFilterFields > 0) {
+            secondaryFieldAccessEvalFactories[numSecondaryKeys] = FormatUtils.getDefaultFormat()
+                    .getFieldAccessEvaluatorFactory(itemType, filterFieldName, numPrimaryKeys);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
+            IAType type = keyTypePair.first;
+            ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+            secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
+        }
+        secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
         // Comparators and type traits for tokens.
         int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
         tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
@@ -143,7 +155,8 @@
         // For tokenization, sorting and loading.
         // One token (+ optional partitioning field) + primary keys.
         numTokenKeyPairFields = (!isPartitioned) ? 1 + numPrimaryKeys : 2 + numPrimaryKeys;
-        ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
+        ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields
+                + numFilterFields];
         ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
         tokenKeyPairComparatorFactories = new IBinaryComparatorFactory[numTokenKeyPairFields];
         tokenKeyPairFields[0] = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -161,7 +174,27 @@
             tokenKeyPairTypeTraits[i + pkOff] = primaryRecDesc.getTypeTraits()[i];
             tokenKeyPairComparatorFactories[i + pkOff] = primaryComparatorFactories[i];
         }
+        if (numFilterFields > 0) {
+            tokenKeyPairFields[numPrimaryKeys + pkOff] = secondaryRecFields[numPrimaryKeys + numSecondaryKeys];
+        }
         tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields, tokenKeyPairTypeTraits);
+        if (filterFieldName != null) {
+            invertedIndexFields = new int[numTokenKeyPairFields];
+            for (int i = 0; i < invertedIndexFields.length; i++) {
+                invertedIndexFields[i] = i;
+            }
+            secondaryFilterFieldsForNonBulkLoadOps = new int[numFilterFields];
+            secondaryFilterFieldsForNonBulkLoadOps[0] = numSecondaryKeys + numPrimaryKeys;
+            invertedIndexFieldsForNonBulkLoadOps = new int[numSecondaryKeys + numPrimaryKeys];
+            for (int i = 0; i < invertedIndexFieldsForNonBulkLoadOps.length; i++) {
+                invertedIndexFieldsForNonBulkLoadOps[i] = i;
+            }
+        }
+
+    }
+
+    protected int getNumSecondaryKeys() {
+        return numTokenKeyPairFields - numPrimaryKeys;
     }
 
     @Override
@@ -171,7 +204,9 @@
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMInvertedIndexLocalResourceMetadata(invListsTypeTraits,
                 primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory, isPartitioned,
-                dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties);
+                dataset.getDatasetId(), mergePolicyFactory, mergePolicyFactoryProperties, filterTypeTraits,
+                filterCmpFactories, invertedIndexFields, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
+                invertedIndexFieldsForNonBulkLoadOps);
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMInvertedIndexResource);
 
@@ -235,8 +270,8 @@
 
     private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
         int docField = 0;
-        int[] primaryKeyFields = new int[numPrimaryKeys];
-        for (int i = 0; i < numPrimaryKeys; i++) {
+        int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
+        for (int i = 0; i < primaryKeyFields.length; i++) {
             primaryKeyFields[i] = numSecondaryKeys + i;
         }
         BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
@@ -261,8 +296,8 @@
     }
 
     private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
-        int[] fieldPermutation = new int[numTokenKeyPairFields];
-        for (int i = 0; i < numTokenKeyPairFields; i++) {
+        int[] fieldPermutation = new int[numTokenKeyPairFields + numFilterFields];
+        for (int i = 0; i < fieldPermutation.length; i++) {
             fieldPermutation[i] = i;
         }
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
@@ -285,14 +320,18 @@
                     new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate());
+                    storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                    filterCmpFactories, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
+                    invertedIndexFieldsForNonBulkLoadOps);
         } else {
             return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                     dataset.getDatasetId()), mergePolicyFactory, mergePolicyFactoryProperties,
                     new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                    storageProperties.getBloomFilterFalsePositiveRate());
+                    storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                    filterCmpFactories, secondaryFilterFields, secondaryFilterFieldsForNonBulkLoadOps,
+                    invertedIndexFieldsForNonBulkLoadOps);
         }
     }
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeOperationsHelper.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeOperationsHelper.java
index 37aa427..38e6a4f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeOperationsHelper.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeOperationsHelper.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -37,8 +38,8 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
-import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.ExternalRTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -77,6 +78,7 @@
     protected int numNestedSecondaryKeyFields;
     protected ATypeTag keyType;
     protected int[] primaryKeyFields;
+    protected int[] rtreeFields;
 
     protected SecondaryRTreeOperationsHelper(PhysicalOptimizationConfig physOptConf,
             IAsterixPropertiesProvider propertiesProvider) {
@@ -88,15 +90,17 @@
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+
         IIndexDataflowHelperFactory indexDataflowHelperFactory;
         ILocalResourceFactoryProvider localResourceFactoryProvider;
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
-            ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
-                    valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
+            ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(secondaryTypeTraits,
+                    secondaryComparatorFactories, primaryComparatorFactories, valueProviderFactories,
+                    RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
                             secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
-                    mergePolicyFactoryProperties, primaryKeyFields);
+                    mergePolicyFactoryProperties, filterTypeTraits, filterCmpFactories, rtreeFields, primaryKeyFields,
+                    secondaryFilterFields);
             localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
                     LocalResource.LSMRTreeResource);
             indexDataflowHelperFactory = new LSMRTreeDataflowHelperFactory(valueProviderFactories,
@@ -105,12 +109,13 @@
                     new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
                     AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
-                    storageProperties.getBloomFilterFalsePositiveRate(), primaryKeyFields);
+                    storageProperties.getBloomFilterFalsePositiveRate(), rtreeFields, primaryKeyFields,
+                    filterTypeTraits, filterCmpFactories, secondaryFilterFields);
         } else {
             // External dataset
             // Prepare a LocalResourceMetadata which will be stored in NC's local resource repository
             ILocalResourceMetadata localResourceMetadata = new ExternalRTreeLocalResourceMetadata(
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+                    secondaryTypeTraits, secondaryComparatorFactories,
                     ExternalIndexingOperations.getBuddyBtreeComparatorFactories(), valueProviderFactories,
                     RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
                             secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
@@ -130,8 +135,9 @@
 
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
+                secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, null,
                 indexDataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -139,6 +145,10 @@
         return spec;
     }
 
+    protected int getNumSecondaryKeys() {
+        return numNestedSecondaryKeyFields;
+    }
+
     @Override
     protected void setSecondaryRecDescAndComparators(IndexType indexType, List<String> secondaryKeyFields,
             int gramLength, AqlMetadataProvider metadata) throws AlgebricksException, AsterixException {
@@ -159,12 +169,13 @@
         numNestedSecondaryKeyFields = numDimensions * 2;
         int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
         secondaryFieldAccessEvalFactories = metadata.getFormat().createMBRFactory(itemType, secondaryKeyFields.get(0),
-                recordColumn, numDimensions);
+                recordColumn, numDimensions, filterFieldName);
+
         secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
         valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
-                + numNestedSecondaryKeyFields];
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
+                + numNestedSecondaryKeyFields + numFilterFields];
+        secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
         keyType = nestedKeyType.getTypeTag();
         for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
@@ -175,6 +186,7 @@
                     nestedKeyType, true);
             secondaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
             valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+
         }
         // Add serializers and comparators for primary index fields.
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
@@ -188,7 +200,18 @@
                 secondaryTypeTraits[numNestedSecondaryKeyFields + i] = IndexingConstants.getTypeTraits(i);
             }
         }
-        secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        if (numFilterFields > 0) {
+            rtreeFields = new int[numNestedSecondaryKeyFields + numPrimaryKeys];
+            for (int i = 0; i < rtreeFields.length; i++) {
+                rtreeFields[i] = i;
+            }
+
+            Pair<IAType, Boolean> typePair = Index.getNonNullableKeyFieldType(filterFieldName, itemType);
+            IAType type = typePair.first;
+            ISerializerDeserializer serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(type);
+            secondaryRecFields[numPrimaryKeys + numNestedSecondaryKeyFields] = serde;
+        }
+        secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
         primaryKeyFields = new int[numPrimaryKeys];
         for (int i = 0; i < primaryKeyFields.length; i++) {
             primaryKeyFields[i] = i + numNestedSecondaryKeyFields;
@@ -232,7 +255,9 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
                                     secondaryComparatorFactories.length), storageProperties
-                                    .getBloomFilterFalsePositiveRate(), primaryKeyFields), BTree.DEFAULT_FILL_FACTOR);
+                                    .getBloomFilterFalsePositiveRate(), rtreeFields, primaryKeyFields,
+                            filterTypeTraits, filterCmpFactories, secondaryFilterFields),
+                    GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
             // Connect the operators.
             spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
             spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
@@ -311,23 +336,23 @@
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                    new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                            primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            mergePolicyFactory, mergePolicyFactoryProperties,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
+                    secondaryComparatorFactories, secondaryBloomFilterKeyFields, new LSMRTreeDataflowHelperFactory(
+                            valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
+                            mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
-                                    secondaryComparatorFactories.length), storageProperties
-                                    .getBloomFilterFalsePositiveRate(), primaryKeyFields),
+                                    secondaryComparatorFactories.length),
+                            storageProperties.getBloomFilterFalsePositiveRate(), rtreeFields, primaryKeyFields,
+                            filterTypeTraits, filterCmpFactories, secondaryFilterFields),
                     NoOpOperationCallbackFactory.INSTANCE);
         } else {
             // External dataset
             compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
-                    secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+                    secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
                     new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
                             primaryComparatorFactories, mergePolicyFactory, mergePolicyFactoryProperties,
                             new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -338,6 +363,7 @@
                             ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset)),
                     NoOpOperationCallbackFactory.INSTANCE);
         }
+
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,
                 secondaryPartitionConstraint);
         spec.addRoot(compactOp);
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
index 7b5391b..abac277 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_2/issue_251_dataset_hint_2.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Thu Sep 26 03:03:21 PDT 2013", "DatasetId": 103, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Wed Jun 25 19:37:16 PDT 2014", "DatasetId": 106, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
index f877136..ff7a1e9 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_3/issue_251_dataset_hint_3.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Thu Sep 26 03:05:13 PDT 2013", "DatasetId": 104, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{ { "Name": "CARDINALITY", "Value": "2000" } }}, "Timestamp": "Wed Jun 25 19:37:17 PDT 2014", "DatasetId": 107, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
index bdbd9f0b..d3221a8 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_251_dataset_hint_4/issue_251_dataset_hint_4.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Thu Sep 26 03:07:19 PDT 2013", "DatasetId": 105, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test", "DatasetName": "Book", "DataTypeName": "LineType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:37:17 PDT 2014", "DatasetId": 108, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
index be969a1..2833c78 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta02/meta02.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Thu Sep 26 02:41:09 PDT 2013", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "testdv", "DatasetName": "dst01", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:24:17 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
index 9f736a5..7acc199 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta09/meta09.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Thu Sep 26 02:43:46 PDT 2013", "DatasetId": 102, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:27 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
\ No newline at end of file
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 0750a58..e5bc93c 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,14 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
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 4580e90..e6dca7a 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,73 +1,74 @@
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FilterField_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "FilterField", "FieldType": "Field_FilterField_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 0750a58..e5bc93c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,14 +1,14 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName" ], "PrimaryKey": [ "DataverseName", "FeedName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 10, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedActivity", "DataTypeName": "FeedActivityRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "PrimaryKey": [ "DataverseName", "FeedName", "DatasetName", "ActivityId" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 11, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "PolicyName" ], "PrimaryKey": [ "DataverseName", "PolicyName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014", "DatasetId": 6, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 4580e90..e6dca7a 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,73 +1,74 @@
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedActivityRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "ActivityId", "FieldType": "int32" }, { "FieldName": "ActivityType", "FieldType": "string" }, { "FieldName": "Details", "FieldType": "Field_Details_in_FeedActivityRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "PolicyName", "FieldType": "string" }, { "FieldName": "Description", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_FeedPolicyRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdaptorName", "FieldType": "string" }, { "FieldName": "AdaptorConfiguration", "FieldType": "Field_AdaptorConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_AdaptorConfiguration_in_FeedRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Details_in_FeedActivityRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Details_in_FeedActivityRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FilterField_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_FeedPolicyRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_FeedPolicyRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "FilterField", "FieldType": "Field_FilterField_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Jun 25 19:33:20 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Jun 25 19:33:19 PDT 2014" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.1.ddl.aql
new file mode 100644
index 0000000..e3bc95d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+ * Description  : Test filters with equality predicate
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.3.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.3.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.5.query.aql
new file mode 100644
index 0000000..d0ddd7d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/equality-predicate/equality-predicate.5.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where $m.send-time = datetime("2014-01-20T10:10:00")
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.ddl.aql
new file mode 100644
index 0000000..1d4cb8d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Description  : Test filters with insert pipeline in the existence of a secondary b-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+
+create dataset FacebookMessages2(FacebookMessageType)
+primary key message-id with filter on send-time;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.3.ddl.aql
new file mode 100644
index 0000000..92e1112
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index fbAuthorIdx on FacebookMessages2(author-id) type btree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.4.update.aql
new file mode 100644
index 0000000..73972ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.4.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset FacebookMessages2 (
+for $m in dataset('FacebookMessages')
+	return $m
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.5.query.aql
new file mode 100644
index 0000000..08425d4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-btree/insert-with-secondary-btree.5.query.aql
@@ -0,0 +1,7 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages2')
+where $m.author-id = 1
+and $m.send-time > datetime("2012-08-20T10:10:00")
+and $m.send-time < datetime("2012-11-20T10:10:00")
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.ddl.aql
new file mode 100644
index 0000000..223c35b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : Test filters with insert pipeline in the existence of a secondary ngram index
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+  
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+
+create dataset FacebookMessages2(FacebookMessageType)
+primary key message-id with filter on send-time;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.3.ddl.aql
new file mode 100644
index 0000000..43aa9d9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.3.ddl.aql
@@ -0,0 +1,4 @@
+ 
+use dataverse test;
+
+create index fbMessageIdx on FacebookMessages2(message) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.4.update.aql
new file mode 100644
index 0000000..73972ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.4.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset FacebookMessages2 (
+for $m in dataset('FacebookMessages')
+	return $m
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.5.query.aql
new file mode 100644
index 0000000..d9d6e2f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.5.query.aql
@@ -0,0 +1,8 @@
+
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where contains($m.message, "love")
+and $m.send-time < datetime("2012-12-20T10:10:00")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.ddl.aql
new file mode 100644
index 0000000..2d918e0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Description  : Test filters with insert pipeline in the existence of a secondary word index
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+
+create dataset FacebookMessages2(FacebookMessageType)
+primary key message-id with filter on send-time;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.3.ddl.aql
new file mode 100644
index 0000000..f2e15d9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.3.ddl.aql
@@ -0,0 +1,4 @@
+ 
+use dataverse test;
+
+create index fbMessageIdx on FacebookMessages2(message) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.4.update.aql
new file mode 100644
index 0000000..73972ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.4.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset FacebookMessages2 (
+for $m in dataset('FacebookMessages')
+	return $m
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.5.query.aql
new file mode 100644
index 0000000..a82b86a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.5.query.aql
@@ -0,0 +1,8 @@
+
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where similarity-jaccard(word-tokens($m.message), word-tokens("love sprint at&t verizon")) >= 0.2f
+and $m.send-time < datetime("2012-12-20T10:10:00")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.ddl.aql
new file mode 100644
index 0000000..60f8490
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Description  : Test filters with insert pipeline in the existence of a secondary r-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+
+create dataset FacebookMessages2(FacebookMessageType)
+primary key message-id with filter on send-time;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.3.ddl.aql
new file mode 100644
index 0000000..b0d0999
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index fbSenderLocIndex on FacebookMessages2(sender-location) type rtree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.4.update.aql
new file mode 100644
index 0000000..73972ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.4.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset FacebookMessages2 (
+for $m in dataset('FacebookMessages')
+	return $m
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.5.query.aql
new file mode 100644
index 0000000..e41178e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.5.query.aql
@@ -0,0 +1,7 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where spatial-intersect($m.sender-location, create-polygon([40.0,79.87,30.0,75.0,50.0,80.0,10.0,10.0]))
+and $m.send-time < datetime("2012-11-20T10:10:00.000Z")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.1.ddl.aql
new file mode 100644
index 0000000..60e8c6a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.1.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * Description  : Test filters with insert pipeline
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+create dataset FacebookMessages2(FacebookMessageType)
+primary key message-id with filter on send-time;
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.3.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.3.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.4.update.aql
new file mode 100644
index 0000000..73972ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.4.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset FacebookMessages2 (
+for $m in dataset('FacebookMessages')
+	return $m
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.5.query.aql
new file mode 100644
index 0000000..7cf718c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/insert/insert.5.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages2')
+where $m.send-time > datetime("2012-08-20T10:10:00")
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.1.ddl.aql
new file mode 100644
index 0000000..fde599e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.1.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : Test filters with loading and in the existence of a secondary b-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.3.ddl.aql
new file mode 100644
index 0000000..9aac25d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index fbAuthorIdx on FacebookMessages(author-id) type btree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.5.query.aql
new file mode 100644
index 0000000..f85397e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-btree/load-with-secondary-btree.5.query.aql
@@ -0,0 +1,7 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where $m.author-id = 1
+and $m.send-time > datetime("2012-08-20T10:10:00")
+and $m.send-time < datetime("2012-11-20T10:10:00")
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.ddl.aql
new file mode 100644
index 0000000..964ff36
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : Test filters with loading and in the existence of a secondary ngram index
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */  
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
+
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.2.update.aql
new file mode 100644
index 0000000..730e70d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.2.update.aql
@@ -0,0 +1,5 @@
+ 
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.3.ddl.aql
new file mode 100644
index 0000000..88ae544
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.3.ddl.aql
@@ -0,0 +1,4 @@
+ 
+use dataverse test;
+
+create index fbMessageIdx on FacebookMessages(message) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.5.query.aql
new file mode 100644
index 0000000..d9d6e2f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.5.query.aql
@@ -0,0 +1,8 @@
+
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where contains($m.message, "love")
+and $m.send-time < datetime("2012-12-20T10:10:00")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.ddl.aql
new file mode 100644
index 0000000..f5a962b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+ * Description  : Test filters with loading and in the existence of a secondary word index
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.2.update.aql
new file mode 100644
index 0000000..66fb3b7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.2.update.aql
@@ -0,0 +1,5 @@
+
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.3.ddl.aql
new file mode 100644
index 0000000..091c3d5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.3.ddl.aql
@@ -0,0 +1,4 @@
+ 
+use dataverse test;
+
+create index fbMessageIdx on FacebookMessages(message) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.5.query.aql
new file mode 100644
index 0000000..a82b86a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.5.query.aql
@@ -0,0 +1,8 @@
+
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where similarity-jaccard(word-tokens($m.message), word-tokens("love sprint at&t verizon")) >= 0.2f
+and $m.send-time < datetime("2012-12-20T10:10:00")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.ddl.aql
new file mode 100644
index 0000000..979048b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.ddl.aql
@@ -0,0 +1,24 @@
+/*
+ * Description  : Test filters with loading and in the existence of a secondary r-tree
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.3.ddl.aql
new file mode 100644
index 0000000..bd042e1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.5.query.aql
new file mode 100644
index 0000000..e41178e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load-with-secondary-rtree/load-with-secondary-rtree.5.query.aql
@@ -0,0 +1,7 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where spatial-intersect($m.sender-location, create-polygon([40.0,79.87,30.0,75.0,50.0,80.0,10.0,10.0]))
+and $m.send-time < datetime("2012-11-20T10:10:00.000Z")
+order by $m.send-time
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load/load.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.1.ddl.aql
new file mode 100644
index 0000000..35abfa3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.1.ddl.aql
@@ -0,0 +1,22 @@
+/*
+ * Description  : Test filters with loading
+ * Expected Res : Success
+ * Date         : 25th Jun 2014
+ */ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type FacebookMessageType as closed {
+        message-id: int32,
+        author-id: int32,
+        in-response-to: int32?,
+        sender-location: point?,
+        message: string,
+        send-time: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id with filter on send-time;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load/load.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.2.update.aql
new file mode 100644
index 0000000..f9073ae
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset FacebookMessages using localfs
+(("path"="nc1://data/fbm-with-send-time.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load/load.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.3.ddl.aql
new file mode 100644
index 0000000..8b13789
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.3.ddl.aql
@@ -0,0 +1 @@
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load/load.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.4.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.4.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/filters/load/load.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.5.query.aql
new file mode 100644
index 0000000..9769a7b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/filters/load/load.5.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $m in dataset('FacebookMessages')
+where $m.send-time > datetime("2012-08-20T10:10:00")
+return $m
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
index aac07d7..6f83465 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv02/cross-dv02.1.adm
@@ -1,4 +1,4 @@
-{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:31 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
-{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:31 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
index aac07d7..6f83465 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv04/cross-dv04.1.adm
@@ -1,4 +1,4 @@
-{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
-{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:31 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
-{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "student", "DatasetName": "gdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "prof", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
+{ "DataverseName": "teacher", "DatasetName": "pstdoc", "DataTypeName": "tchrType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:31 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
+{ "DataverseName": "student", "DatasetName": "ugdstd", "DataTypeName": "stdType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Wed Apr 30 14:23:30 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 660e2a8..8069a14 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES", "LastRefreshTime": datetime("2014-06-08T20:30:43.724Z"), "TransactionState": 0, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 107, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 101, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 104, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 105, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 102, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:42 PDT 2014", "DatasetId": 103, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:30:43 PDT 2014", "DatasetId": 106, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES", "LastRefreshTime": datetime("2014-06-25T06:52:12.170Z"), "TransactionState": 0, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 746, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 740, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 743, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 744, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 741, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 742, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:52:12 PDT 2014", "DatasetId": 745, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm
new file mode 100644
index 0000000..3dfab1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/equality-predicate/equality-predicate.1.adm
@@ -0,0 +1 @@
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm
new file mode 100644
index 0000000..2bb88cc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-btree/insert-with-secondary-btree.1.adm
@@ -0,0 +1 @@
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm
new file mode 100644
index 0000000..2810527
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-ngram/insert-with-secondary-inverted-ngram.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm
new file mode 100644
index 0000000..2810527
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-inverted-word/insert-with-secondary-inverted-word.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm
new file mode 100644
index 0000000..0075709
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/insert-with-secondary-rtree/insert-with-secondary-rtree.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm
new file mode 100644
index 0000000..f84c8cb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/insert/insert.1.adm
@@ -0,0 +1,7 @@
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm
new file mode 100644
index 0000000..2bb88cc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-btree/load-with-secondary-btree.1.adm
@@ -0,0 +1 @@
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm
new file mode 100644
index 0000000..2810527
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-ngram/load-with-secondary-inverted-ngram.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm
new file mode 100644
index 0000000..2810527
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-inverted-word/load-with-secondary-inverted-word.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 5, "author-id": 6, "in-response-to": 2, "sender-location": point("34.7,90.76"), "message": " love sprint the customization is mind-blowing", "send-time": datetime("2012-05-20T10:10:00.000Z") }
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm
new file mode 100644
index 0000000..0075709
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/load-with-secondary-rtree/load-with-secondary-rtree.1.adm
@@ -0,0 +1,3 @@
+{ "message-id": 1, "author-id": 3, "in-response-to": 2, "sender-location": point("47.16,77.75"), "message": " love sprint its shortcut-menu is awesome:)", "send-time": datetime("2012-01-20T10:10:00.000Z") }
+{ "message-id": 6, "author-id": 2, "in-response-to": 1, "sender-location": point("31.5,75.56"), "message": " like t-mobile its platform is mind-blowing", "send-time": datetime("2012-06-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm b/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm
new file mode 100644
index 0000000..f84c8cb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/filters/load/load.1.adm
@@ -0,0 +1,7 @@
+{ "message-id": 9, "author-id": 3, "in-response-to": 12, "sender-location": point("34.45,96.48"), "message": " love verizon its wireless is good", "send-time": datetime("2012-09-20T10:10:00.000Z") }
+{ "message-id": 10, "author-id": 1, "in-response-to": 12, "sender-location": point("42.5,70.01"), "message": " can't stand motorola the touch-screen is terrible", "send-time": datetime("2012-10-20T10:10:00.000Z") }
+{ "message-id": 13, "author-id": 10, "in-response-to": 4, "sender-location": point("42.77,78.92"), "message": " dislike iphone the voice-command is bad:(", "send-time": datetime("2013-08-20T10:10:00.000Z") }
+{ "message-id": 12, "author-id": 10, "in-response-to": 6, "sender-location": point("42.26,77.76"), "message": " can't stand t-mobile its voicemail-service is OMG:(", "send-time": datetime("2012-12-20T10:10:00.000Z") }
+{ "message-id": 11, "author-id": 1, "in-response-to": 1, "sender-location": point("38.97,77.49"), "message": " can't stand at&t its plan is terrible", "send-time": datetime("2012-11-20T10:10:00.000Z") }
+{ "message-id": 14, "author-id": 9, "in-response-to": 12, "sender-location": point("41.33,85.28"), "message": " love at&t its 3G is good:)", "send-time": datetime("2013-09-20T10:10:00.000Z") }
+{ "message-id": 15, "author-id": 7, "in-response-to": 11, "sender-location": point("44.47,67.11"), "message": " like iphone the voicemail-service is awesome", "send-time": datetime("2014-01-20T10:10:00.000Z") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
index 1b4a1f6..4a25884 100644
--- a/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf23/udf23.1.adm
@@ -1,6 +1,6 @@
-{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "CompactionPolicy" ], "PrimaryKey": [ "DataverseName", "CompactionPolicy" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ], "FilterField": [  ] }, "ExternalDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jun 24 23:48:37 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 50f5ce2..fa187d0 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -5117,11 +5117,68 @@
       </compilation-unit>
     </test-case>
   </test-group>
-  <test-group name="json">
-    <test-case FilePath="json">
-      <compilation-unit name="int01">
-        <output-dir compare="JSON">int01</output-dir>
-      </compilation-unit>
-    </test-case>
-  </test-group>
+  <test-group name="filters">
+  		<test-case FilePath="filters">
+			<compilation-unit name="equality-predicate">
+				<output-dir compare="Text">equality-predicate</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="load">
+				<output-dir compare="Text">load</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="load-with-secondary-btree">
+				<output-dir compare="Text">load-with-secondary-btree</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="load-with-secondary-inverted-ngram">
+				<output-dir compare="Text">load-with-secondary-inverted-ngram</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="load-with-secondary-inverted-word">
+				<output-dir compare="Text">load-with-secondary-inverted-word</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="load-with-secondary-rtree">
+				<output-dir compare="Text">load-with-secondary-rtree</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="insert">
+				<output-dir compare="Text">insert</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="insert-with-secondary-btree">
+				<output-dir compare="Text">insert-with-secondary-btree</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="insert-with-secondary-inverted-ngram">
+				<output-dir compare="Text">insert-with-secondary-inverted-ngram</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="insert-with-secondary-inverted-word">
+				<output-dir compare="Text">insert-with-secondary-inverted-word</output-dir>
+			</compilation-unit>
+		</test-case>
+		<test-case FilePath="filters">
+			<compilation-unit name="insert-with-secondary-rtree">
+				<output-dir compare="Text">insert-with-secondary-rtree</output-dir>
+			</compilation-unit>
+		</test-case>
+	</test-group>
+    <test-group name="json">
+		<test-case FilePath="json">
+			<compilation-unit name="int01">
+				<output-dir compare="JSON">int01</output-dir>
+			</compilation-unit>
+		</test-case>
+    </test-group>
 </test-suite>
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDetailsDecl.java
index 4028340..552bafd 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDetailsDecl.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedDetailsDecl.java
@@ -26,8 +26,8 @@
 
     public FeedDetailsDecl(String adapterFactoryClassname, Map<String, String> configuration,
             FunctionSignature signature, Identifier nodeGroupName, List<String> partitioningExpr,
-            String compactionPolicy, Map<String, String> compactionPolicyProperties) {
-        super(nodeGroupName, partitioningExpr, false, compactionPolicy, compactionPolicyProperties);
+            String compactionPolicy, Map<String, String> compactionPolicyProperties, String filterField) {
+        super(nodeGroupName, partitioningExpr, false, compactionPolicy, compactionPolicyProperties, filterField);
         this.adapterFactoryClassname = adapterFactoryClassname;
         this.configuration = configuration;
         this.functionSignature = signature;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/InternalDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/InternalDetailsDecl.java
index d9fa886..3be08b2 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/InternalDetailsDecl.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/InternalDetailsDecl.java
@@ -25,15 +25,17 @@
     private final boolean autogenerated;
     private final String compactionPolicy;
     private final Map<String, String> compactionPolicyProperties;
+    private final String filterField;
 
     public InternalDetailsDecl(Identifier nodeGroupName, List<String> partitioningExpr, boolean autogenerated,
-            String compactionPolicy, Map<String, String> compactionPolicyProperties) {
+            String compactionPolicy, Map<String, String> compactionPolicyProperties, String filterField) {
         this.nodegroupName = nodeGroupName == null ? new Identifier(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)
                 : nodeGroupName;
         this.partitioningExprs = partitioningExpr;
         this.autogenerated = autogenerated;
         this.compactionPolicy = compactionPolicy;
         this.compactionPolicyProperties = compactionPolicyProperties;
+        this.filterField = filterField;
     }
 
     public List<String> getPartitioningExprs() {
@@ -55,4 +57,9 @@
     public Map<String, String> getCompactionPolicyProperties() {
         return compactionPolicyProperties;
     }
+
+    public String getFilterField() {
+        return filterField;
+    }
+
 }
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index ad86862..23eecd4 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -314,6 +314,7 @@
   DatasetDecl dsetDecl = null;
   boolean autogenerated = false;
   String compactionPolicy = null;
+  String filterField = null;
 }
 {
   (
@@ -348,6 +349,7 @@
     ("on" nodeGroupName = Identifier() )?
     ( "hints" hints = Properties() )?
     ( "using" "compaction" "policy" compactionPolicy = CompactionPolicy() compactionPolicyProperties = Configuration() )?
+    ( "with filter on" filterField = FilterField() )?
       {
         InternalDetailsDecl idd = new InternalDetailsDecl(nodeGroupName != null
                                                             ? new Identifier(nodeGroupName)
@@ -355,7 +357,8 @@
                                                           primaryKeyFields,
                                                           autogenerated,
                                                           compactionPolicy,
-                                                          compactionPolicyProperties);
+                                                          compactionPolicyProperties,
+                                                          filterField);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    new Identifier(typeName),
@@ -431,6 +434,17 @@
     }
 }
 
+String FilterField() throws ParseException :
+{
+  String filterField = null;
+}
+{
+  filterField = Identifier()
+    {
+	  return filterField;
+    }
+}
+
 IndexParams IndexType() throws ParseException:
 {
   IndexType type = null;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
index eb11425..ab1c8ef 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
@@ -33,7 +33,7 @@
 
     public static final String FRAME_SIZE_PROPERTY = "FrameSize";
 
-    public static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
+    public static final float DEFAULT_TREE_FILL_FACTOR = 1.00f;
 
     public static int DEFAULT_INPUT_DATA_COLUMN = 0;
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index a299d0ae..1ad86c2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -238,7 +238,7 @@
             IDatasetDetails id = new InternalDatasetDetails(FileStructure.BTREE, PartitioningStrategy.HASH,
                     primaryIndexes[i].getPartitioningExpr(), primaryIndexes[i].getPartitioningExpr(),
                     primaryIndexes[i].getNodeGroupName(), false, GlobalConfig.DEFAULT_COMPACTION_POLICY_NAME,
-                    GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES);
+                    GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES, null);
             MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(primaryIndexes[i].getDataverseName(),
                     primaryIndexes[i].getIndexedDatasetName(), primaryIndexes[i].getPayloadRecordType().getTypeName(),
                     id, new HashMap<String, String>(), DatasetType.INTERNAL, primaryIndexes[i].getDatasetId().getId(),
@@ -400,12 +400,14 @@
                     runtimeContext.getMetadataMergePolicyFactory().createMergePolicy(
                             GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES), opTracker,
                     runtimeContext.getLSMIOScheduler(),
-                    LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), index.isPrimaryIndex());
+                    LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(), index.isPrimaryIndex(),
+                    null, null, null, null);
             lsmBtree.create();
             resourceID = runtimeContext.getResourceIdFactory().createId();
             ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
                     comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(), index.getDatasetId().getId(),
-                    runtimeContext.getMetadataMergePolicyFactory(), GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES);
+                    runtimeContext.getMetadataMergePolicyFactory(), GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES,
+                    null, null, null, null);
             ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
@@ -421,7 +423,7 @@
                                 .getBloomFilterFalsePositiveRate(), runtimeContext.getMetadataMergePolicyFactory()
                                 .createMergePolicy(GlobalConfig.DEFAULT_COMPACTION_POLICY_PROPERTIES), opTracker,
                         runtimeContext.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
-                                .createIOOperationCallback(), index.isPrimaryIndex());
+                                .createIOOperationCallback(), index.isPrimaryIndex(), null, null, null, null);
                 indexLifecycleManager.register(resourceID, lsmBtree);
             }
         }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 11c5370..edbe047 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -152,15 +152,16 @@
     public static final int INTERNAL_DETAILS_ARECORD_AUTOGENERATED_FIELD_INDEX = 5;
     public static final int INTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_FIELD_INDEX = 6;
     public static final int INTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX = 7;
+    public static final int INTERNAL_DETAILS_ARECORD_FILTER_FIELD_FIELD_INDEX = 8;
 
     private static final ARecordType createInternalDetailsRecordType() throws AsterixException {
         AOrderedListType olType = new AOrderedListType(BuiltinType.ASTRING, null);
         AOrderedListType compactionPolicyPropertyListType = new AOrderedListType(
                 COMPACTION_POLICY_PROPERTIES_RECORDTYPE, null);
         String[] fieldNames = { "FileStructure", "PartitioningStrategy", "PartitioningKey", "PrimaryKey", "GroupName",
-                "Autogenerated", "CompactionPolicy", "CompactionPolicyProperties" };
+                "Autogenerated", "CompactionPolicy", "CompactionPolicyProperties", "FilterField" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, olType, olType, BuiltinType.ASTRING,
-                BuiltinType.ABOOLEAN, BuiltinType.ASTRING, compactionPolicyPropertyListType };
+                BuiltinType.ABOOLEAN, BuiltinType.ASTRING, compactionPolicyPropertyListType, olType };
         return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 50d9ab2..c739506 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -291,16 +291,16 @@
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
             IDataSource<AqlSourceId> dataSource, List<LogicalVariable> scanVariables,
-            List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+            List<LogicalVariable> projectVariables, boolean projectPushed, List<LogicalVariable> minFilterVars,
+            List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+            JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
         try {
             switch (((AqlDataSource) dataSource).getDatasourceType()) {
                 case FEED:
                     return buildFeedIntakeRuntime(jobSpec, dataSource);
                 case INTERNAL_DATASET:
-                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataSource, context,
-                            implConfig);
+                    return buildInternalDatasetScan(jobSpec, scanVariables, minFilterVars, maxFilterVars, opSchema,
+                            typeEnv, dataSource, context, implConfig);
 
                 case EXTERNAL_DATASET:
                     return buildExternalDatasetScan(jobSpec, dataSource);
@@ -315,16 +315,36 @@
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
-            List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig) throws AlgebricksException,
-            MetadataException {
+            List<LogicalVariable> outputVars, List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, IDataSource<AqlSourceId> dataSource,
+            JobGenContext context, Object implConfig) throws AlgebricksException, MetadataException {
         AqlSourceId asid = dataSource.getId();
         String dataverseName = asid.getDataverseName();
         String datasetName = asid.getDatasetName();
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
+
+        int[] minFilterFieldIndexes = null;
+        if (minFilterVars != null && !minFilterVars.isEmpty()) {
+            minFilterFieldIndexes = new int[minFilterVars.size()];
+            int i = 0;
+            for (LogicalVariable v : minFilterVars) {
+                minFilterFieldIndexes[i] = opSchema.findVariable(v);
+                i++;
+            }
+        }
+        int[] maxFilterFieldIndexes = null;
+        if (maxFilterVars != null && !maxFilterVars.isEmpty()) {
+            maxFilterFieldIndexes = new int[maxFilterVars.size()];
+            int i = 0;
+            for (LogicalVariable v : maxFilterVars) {
+                maxFilterFieldIndexes[i] = opSchema.findVariable(v);
+                i++;
+            }
+        }
+
         return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, true, false,
                 ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true, true,
-                implConfig);
+                implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
@@ -519,7 +539,8 @@
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, boolean retainNull, Dataset dataset, String indexName,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
-            Object implConfig) throws AlgebricksException {
+            Object implConfig, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+
         boolean isSecondary = true;
         int numSecondaryKeys = 0;
         try {
@@ -533,6 +554,16 @@
             int[] bloomFilterKeyFields;
             ITypeTraits[] typeTraits;
             IBinaryComparatorFactory[] comparatorFactories;
+
+            String itemTypeName = dataset.getItemTypeName();
+            ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                    dataset.getDataverseName(), itemTypeName).getDatatype();
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, itemType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = null;
+            int[] btreeFields = null;
+
             if (isSecondary) {
                 Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                         dataset.getDatasetName(), indexName);
@@ -544,18 +575,28 @@
                 typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, 0, outputVars.size(), typeEnv, context);
                 comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(outputVars, 0,
                         outputVars.size(), typeEnv, context);
+
+                if (filterTypeTraits != null) {
+                    filterFields = new int[1];
+                    filterFields[0] = numSecondaryKeys + numPrimaryKeys;
+                    btreeFields = new int[numSecondaryKeys + numPrimaryKeys];
+                    for (int k = 0; k < btreeFields.length; k++) {
+                        btreeFields[k] = k;
+                    }
+                }
+
             } else {
                 bloomFilterKeyFields = new int[numPrimaryKeys];
                 for (int i = 0; i < numPrimaryKeys; i++) {
                     bloomFilterKeyFields[i] = i;
                 }
-                String itemTypeName = dataset.getItemTypeName();
-                ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-                        dataset.getDataverseName(), itemTypeName).getDatatype();
 
                 typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
                 comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset, itemType,
                         context.getBinaryComparatorFactoryProvider());
+
+                filterFields = DatasetUtils.createFilterFields(dataset);
+                btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
             }
 
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
@@ -601,9 +642,11 @@
                                 compactionInfo.second, isSecondary ? new SecondaryIndexOperationTrackerProvider(
                                         dataset.getDatasetId()) : new PrimaryIndexOperationTrackerProvider(
                                         dataset.getDatasetId()), rtcProvider,
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE, getStorageProperties()
-                                        .getBloomFilterFalsePositiveRate(), !isSecondary), retainInput, retainNull,
-                        context.getNullWriterFactory(), searchCallbackFactory);
+                                LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                                storageProperties.getBloomFilterFalsePositiveRate(), !isSecondary, filterTypeTraits,
+                                filterCmpFactories, btreeFields, filterFields), retainInput, retainNull,
+                        context.getNullWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
+                        maxFilterFieldIndexes);
             } else {
                 // External dataset <- use the btree with buddy btree->
                 // Be Careful of Key Start Index ?
@@ -648,7 +691,8 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, boolean retainNull, Dataset dataset, String indexName,
-            int[] keyFields) throws AlgebricksException {
+            int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws AlgebricksException {
+
         try {
             ARecordType recType = (ARecordType) findType(dataset.getDataverseName(), dataset.getItemTypeName());
             int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
@@ -697,7 +741,21 @@
                     dataset, recType, context.getBinaryComparatorFactoryProvider());
             int[] btreeFields = new int[primaryComparatorFactories.length];
             for (int i = 0; i < btreeFields.length; i++) {
-                btreeFields[i] = i + numSecondaryKeys;
+                btreeFields[i] = i + numNestedSecondaryKeyFields;
+            }
+
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = null;
+            int[] rtreeFields = null;
+            if (filterTypeTraits != null) {
+                filterFields = new int[1];
+                filterFields[0] = numNestedSecondaryKeyFields + numPrimaryKeys;
+                rtreeFields = new int[numNestedSecondaryKeyFields + numPrimaryKeys];
+                for (int i = 0; i < rtreeFields.length; i++) {
+                    rtreeFields[i] = i;
+                }
             }
 
             IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
@@ -715,9 +773,11 @@
                                 compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
                                         dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                                 LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(
-                                        nestedKeyType.getTypeTag(), comparatorFactories.length), getStorageProperties()
-                                        .getBloomFilterFalsePositiveRate(), btreeFields), retainInput, retainNull,
-                        context.getNullWriterFactory(), searchCallbackFactory);
+                                        nestedKeyType.getTypeTag(), comparatorFactories.length),
+                                storageProperties.getBloomFilterFalsePositiveRate(), rtreeFields, btreeFields,
+                                filterTypeTraits, filterCmpFactories, filterFields), retainInput, retainNull,
+                        context.getNullWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
+                        maxFilterFieldIndexes);
 
             } else {
                 // External Dataset
@@ -736,6 +796,7 @@
                         spPc.first, typeTraits, comparatorFactories, keyFields, indexDataflowHelperFactory,
                         retainInput, retainNull, context.getNullWriterFactory(), searchCallbackFactory);
             }
+
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
 
         } catch (MetadataException me) {
@@ -835,12 +896,21 @@
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
-            LogicalVariable payload, JobGenContext context, JobSpecification spec) throws AlgebricksException {
+            LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields, JobGenContext context,
+            JobSpecification spec) throws AlgebricksException {
         String dataverseName = dataSource.getId().getDataverseName();
         String datasetName = dataSource.getId().getDatasetName();
+
+        Dataset dataset = findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        }
+
         int numKeys = keys.size();
+        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+
         // move key fields to front
-        int[] fieldPermutation = new int[numKeys + 1];
+        int[] fieldPermutation = new int[numKeys + 1 + numFilterFields];
         int[] bloomFilterKeyFields = new int[numKeys];
         // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
         int i = 0;
@@ -851,10 +921,9 @@
             i++;
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
-
-        Dataset dataset = findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        if (numFilterFields > 0) {
+            int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+            fieldPermutation[numKeys + 1] = idx;
         }
 
         try {
@@ -875,6 +944,12 @@
 
             long numElementsHint = getCardinalityPerPartitionHint(dataset);
 
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, itemType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = DatasetUtils.createFilterFields(dataset);
+            int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
+
             // TODO
             // figure out the right behavior of the bulkload and then give the
             // right callback
@@ -885,13 +960,13 @@
             TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
                     splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
-                    GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, true,
+                    GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(
                                     dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate(), true),
-                    NoOpOperationCallbackFactory.INSTANCE);
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                            storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
+                            filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException me) {
@@ -901,12 +976,20 @@
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
-            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> keys, LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
         String datasetName = dataSource.getId().getDatasetName();
+
+        Dataset dataset = findDataset(dataSource.getId().getDataverseName(), datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse "
+                    + dataSource.getId().getDataverseName());
+        }
+
         int numKeys = keys.size();
+        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
         // Move key fields to front.
-        int[] fieldPermutation = new int[numKeys + 1];
+        int[] fieldPermutation = new int[numKeys + 1 + numFilterFields];
         int[] bloomFilterKeyFields = new int[numKeys];
         int i = 0;
         for (LogicalVariable varKey : keys) {
@@ -916,12 +999,11 @@
             i++;
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
-
-        Dataset dataset = findDataset(dataSource.getId().getDataverseName(), datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse "
-                    + dataSource.getId().getDataverseName());
+        if (numFilterFields > 0) {
+            int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+            fieldPermutation[numKeys + 1] = idx;
         }
+
         try {
             Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), dataset.getDatasetName());
@@ -946,6 +1028,13 @@
             for (i = 0; i < numKeys; i++) {
                 primaryKeyFields[i] = i;
             }
+
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, itemType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = DatasetUtils.createFilterFields(dataset);
+            int[] btreeFields = DatasetUtils.createBTreeFieldsWhenThereisAFilter(dataset);
+
             TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
             PrimaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
                     jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
@@ -960,8 +1049,8 @@
                             compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(
                                     dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate(), true), null, modificationCallbackFactory, true,
-                    indexName);
+                                    .getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
+                            btreeFields, filterFields), null, modificationCallbackFactory, true, indexName);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
@@ -974,26 +1063,27 @@
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
-            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> keys, LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
         return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, typeEnv, keys, payload,
-                recordDesc, context, spec);
+                additionalNonKeyFields, recordDesc, context, spec);
     }
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
-            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> keys, LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
         return getInsertOrDeleteRuntime(IndexOperation.DELETE, dataSource, propagatedSchema, typeEnv, keys, payload,
-                recordDesc, context, spec);
+                additionalNonKeyFields, recordDesc, context, spec);
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(
             IndexOperation indexOp, IDataSourceIndex<String, AqlSourceId> dataSourceIndex,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
-            List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr,
-            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+            List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+            JobGenContext context, JobSpecification spec) throws AlgebricksException {
         String indexName = dataSourceIndex.getId();
         String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
         String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
@@ -1013,19 +1103,21 @@
         switch (secondaryIndex.getIndexType()) {
             case BTREE: {
                 return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
-                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+                        primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
+                        indexOp);
             }
             case RTREE: {
                 return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
-                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+                        primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
+                        indexOp);
             }
             case SINGLE_PARTITION_WORD_INVIX:
             case SINGLE_PARTITION_NGRAM_INVIX:
             case LENGTH_PARTITIONED_WORD_INVIX:
             case LENGTH_PARTITIONED_NGRAM_INVIX: {
                 return getInvertedIndexDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
-                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp,
-                        secondaryIndex.getIndexType());
+                        primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
+                        indexOp, secondaryIndex.getIndexType());
             }
             default: {
                 throw new AlgebricksException("Insert and delete not implemented for index type: "
@@ -1038,20 +1130,22 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
             IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
+            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+            throws AlgebricksException {
         return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas,
-                typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+                typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec);
     }
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexDeleteRuntime(
             IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec) throws AlgebricksException {
+            List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
+            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+            throws AlgebricksException {
         return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas,
-                typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+                typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec);
     }
 
     private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -1070,12 +1164,20 @@
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String dataverseName,
             String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
             List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
+            List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp)
+            throws AlgebricksException {
+
+        Dataset dataset = findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        }
 
         int numKeys = primaryKeys.size() + secondaryKeys.size();
+        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+
         // generate field permutations
-        int[] fieldPermutation = new int[numKeys];
+        int[] fieldPermutation = new int[numKeys + numFilterFields];
         int[] bloomFilterKeyFields = new int[secondaryKeys.size()];
         int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
         int i = 0;
@@ -1093,11 +1195,11 @@
             i++;
             j++;
         }
-
-        Dataset dataset = findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        if (numFilterFields > 0) {
+            int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+            fieldPermutation[numKeys] = idx;
         }
+
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType;
         try {
@@ -1114,6 +1216,20 @@
             Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                     dataset.getDatasetName(), indexName);
 
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = null;
+            int[] btreeFields = null;
+            if (filterTypeTraits != null) {
+                filterFields = new int[1];
+                filterFields[0] = numKeys;
+                btreeFields = new int[numKeys];
+                for (int k = 0; k < btreeFields.length; k++) {
+                    btreeFields[k] = k;
+                }
+            }
+
             List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
             ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
             IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
@@ -1156,8 +1272,8 @@
                             compactionInfo.first, compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
                                     dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate(), false), filterFactory,
-                    modificationCallbackFactory, false, indexName);
+                                    .getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
+                            btreeFields, filterFields), filterFactory, modificationCallbackFactory, false, indexName);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1170,8 +1286,9 @@
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInvertedIndexDmlRuntime(String dataverseName,
             String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
             List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec, IndexOperation indexOp, IndexType indexType) throws AlgebricksException {
+            List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
+            IndexType indexType) throws AlgebricksException {
 
         // Sanity checks.
         if (primaryKeys.size() > 1) {
@@ -1181,9 +1298,15 @@
             throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
         }
 
+        Dataset dataset = findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        }
+
         int numKeys = primaryKeys.size() + secondaryKeys.size();
+        int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
         // generate field permutations
-        int[] fieldPermutation = new int[numKeys];
+        int[] fieldPermutation = new int[numKeys + numFilterFields];
         int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
         int i = 0;
         int j = 0;
@@ -1199,6 +1322,10 @@
             i++;
             j++;
         }
+        if (numFilterFields > 0) {
+            int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+            fieldPermutation[numKeys] = idx;
+        }
 
         boolean isPartitioned;
         if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
@@ -1208,10 +1335,6 @@
             isPartitioned = false;
         }
 
-        Dataset dataset = findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
-        }
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType;
         try {
@@ -1262,6 +1385,30 @@
             IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
                     secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
 
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
+
+            int[] filterFields = null;
+            int[] invertedIndexFields = null;
+            int[] filterFieldsForNonBulkLoadOps = null;
+            int[] invertedIndexFieldsForNonBulkLoadOps = null;
+            if (filterTypeTraits != null) {
+                filterFields = new int[1];
+                filterFields[0] = numTokenFields + primaryKeys.size();
+                invertedIndexFields = new int[numTokenFields + primaryKeys.size()];
+                for (int k = 0; k < invertedIndexFields.length; k++) {
+                    invertedIndexFields[k] = k;
+                }
+
+                filterFieldsForNonBulkLoadOps = new int[numFilterFields];
+                filterFieldsForNonBulkLoadOps[0] = numKeys;
+                invertedIndexFieldsForNonBulkLoadOps = new int[numKeys];
+                for (int k = 0; k < invertedIndexFieldsForNonBulkLoadOps.length; k++) {
+                    invertedIndexFieldsForNonBulkLoadOps[k] = k;
+                }
+            }
+
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
                     dataverseName, datasetName, indexName);
@@ -1283,14 +1430,18 @@
                         new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate());
+                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps);
             } else {
                 indexDataFlowFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
                         new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                        storageProperties.getBloomFilterFalsePositiveRate());
+                        storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
+                        filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
+                        invertedIndexFieldsForNonBulkLoadOps);
             }
             AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor insertDeleteOp = new AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor(
                     spec, recordDesc, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
@@ -1309,8 +1460,9 @@
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String dataverseName,
             String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
             List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
+            List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
+            RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp)
+            throws AlgebricksException {
         try {
             Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
             String itemTypeName = dataset.getItemTypeName();
@@ -1330,7 +1482,9 @@
             int numKeys = numSecondaryKeys + numPrimaryKeys;
             ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
             IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
-            int[] fieldPermutation = new int[numKeys];
+
+            int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+            int[] fieldPermutation = new int[numKeys + numFilterFields];
             int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
             int i = 0;
             int j = 0;
@@ -1347,6 +1501,11 @@
                 i++;
                 j++;
             }
+
+            if (numFilterFields > 0) {
+                int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+                fieldPermutation[numKeys] = idx;
+            }
             IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
             IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
             for (i = 0; i < numSecondaryKeys; i++) {
@@ -1372,6 +1531,20 @@
                 btreeFields[k] = k + numSecondaryKeys;
             }
 
+            ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, recType);
+            IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
+            int[] filterFields = null;
+            int[] rtreeFields = null;
+            if (filterTypeTraits != null) {
+                filterFields = new int[1];
+                filterFields[0] = numSecondaryKeys + numPrimaryKeys;
+                rtreeFields = new int[numSecondaryKeys + numPrimaryKeys];
+                for (int k = 0; k < rtreeFields.length; k++) {
+                    rtreeFields[k] = k;
+                }
+            }
+
             // prepare callback
             JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
             int datasetId = dataset.getDatasetId();
@@ -1392,7 +1565,8 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
                                     comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate(),
-                            btreeFields), filterFactory, modificationCallbackFactory, false, indexName);
+                            rtreeFields, btreeFields, filterTypeTraits, filterCmpFactories, filterFields),
+                    filterFactory, modificationCallbackFactory, false, indexName);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
         } catch (MetadataException | IOException e) {
             throw new AlgebricksException(e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
index b1ef68d..c1bd4ef 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
@@ -58,10 +58,11 @@
     protected final boolean autogenerated;
     protected final String compactionPolicy;
     protected final Map<String, String> compactionPolicyProperties;
+    protected final String filterField;
 
     public InternalDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
             List<String> partitioningKey, List<String> primaryKey, String groupName, boolean autogenerated,
-            String compactionPolicy, Map<String, String> compactionPolicyProperties) {
+            String compactionPolicy, Map<String, String> compactionPolicyProperties, String filterField) {
         this.fileStructure = fileStructure;
         this.partitioningStrategy = partitioningStrategy;
         this.partitioningKeys = partitioningKey;
@@ -70,6 +71,7 @@
         this.nodeGroupName = groupName;
         this.compactionPolicy = compactionPolicy;
         this.compactionPolicyProperties = compactionPolicyProperties;
+        this.filterField = filterField;
     }
 
     @Override
@@ -107,6 +109,10 @@
         return compactionPolicyProperties;
     }
 
+    public String getFilterField() {
+        return filterField;
+    }
+
     @Override
     public DatasetType getDatasetType() {
         return DatasetType.INTERNAL;
@@ -203,6 +209,21 @@
         internalRecordBuilder.addField(
                 MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX, fieldValue);
 
+        // write field 8
+        listBuilder
+                .reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_FILTER_FIELD_FIELD_INDEX]);
+        String filterField = getFilterField();
+        if (filterField != null) {
+            itemValue.reset();
+            aString.setValue(filterField);
+            stringSerde.serialize(aString, itemValue.getDataOutput());
+            listBuilder.addItem(itemValue);
+        }
+        fieldValue.reset();
+        listBuilder.write(fieldValue.getDataOutput(), true);
+        internalRecordBuilder.addField(MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_FILTER_FIELD_FIELD_INDEX,
+                fieldValue);
+
         try {
             internalRecordBuilder.write(out, true);
         } catch (IOException | AsterixException e) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 8d8ff25..8666ccf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -152,9 +152,16 @@
                             .getStringValue();
                     compactionPolicyProperties.put(key, value);
                 }
-
+                cursor = ((AOrderedList) datasetDetailsRecord
+                        .getValueByPos(MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_FILTER_FIELD_FIELD_INDEX))
+                        .getCursor();
+                String filterField = null;
+                if (cursor.next()) {
+                    filterField = ((AString) cursor.get()).getStringValue();
+                }
                 datasetDetails = new InternalDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
-                        partitioningKey, groupName, autogenerated, compactionPolicy, compactionPolicyProperties);
+                        partitioningKey, groupName, autogenerated, compactionPolicy, compactionPolicyProperties,
+                        filterField);
 
                 break;
             }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
index b0de324..998d322 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
@@ -20,7 +20,6 @@
 import java.util.Map;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -29,7 +28,6 @@
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
 import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.external.IndexingConstants;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -135,6 +133,87 @@
         return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
     }
 
+    public static String getFilterField(Dataset dataset) {
+        return (((InternalDatasetDetails) dataset.getDatasetDetails())).getFilterField();
+    }
+
+    public static IBinaryComparatorFactory[] computeFilterBinaryComparatorFactories(Dataset dataset,
+            ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+        String filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[1];
+        IAType type;
+        try {
+            type = itemType.getFieldType(filterField);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+        bcfs[0] = comparatorFactoryProvider.getBinaryComparatorFactory(type, true);
+        return bcfs;
+    }
+
+    public static ITypeTraits[] computeFilterTypeTraits(Dataset dataset, ARecordType itemType)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+        String filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        ITypeTraits[] typeTraits = new ITypeTraits[1];
+
+        IAType type;
+        try {
+            type = itemType.getFieldType(filterField);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+        typeTraits[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
+        return typeTraits;
+    }
+
+    public static int[] createFilterFields(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+
+        String filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
+
+        int[] filterFields = new int[1];
+        filterFields[0] = numKeys + 1;
+        return filterFields;
+    }
+
+    public static int[] createBTreeFieldsWhenThereisAFilter(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            return null;
+        }
+
+        String filterField = getFilterField(dataset);
+        if (filterField == null) {
+            return null;
+        }
+
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        int[] btreeFields = new int[partitioningKeys.size() + 1];
+        for (int i = 0; i < btreeFields.length; ++i) {
+            btreeFields[i] = i;
+        }
+        return btreeFields;
+    }
+
     public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         for (int i = 0; i < partitioningKeys.size(); i++) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
index 3ebedde..bb602a4 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/base/IDataFormat.java
@@ -75,8 +75,8 @@
 
     public ICopyEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException;
 
-    public ICopyEvaluatorFactory[] createMBRFactory(ARecordType recType, String fldName, int recordColumn, int dimension)
-            throws AlgebricksException;
+    public ICopyEvaluatorFactory[] createMBRFactory(ARecordType recType, String fldName, int recordColumn,
+            int dimension, String filterFieldName) throws AlgebricksException;
 
     public IExpressionEvalSizeComputer getExpressionEvalSizeComputer();
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index ac62a5d..abc62fa 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -409,6 +409,43 @@
         }
     }
 
+    /**
+     * Validates the field that will be used as filter for the components of an LSM index.
+     * 
+     * @param keyFieldNames
+     *            a list of key fields that will be validated
+     * @param indexType
+     *            the type of the index that its key fields is being validated
+     * @throws AlgebricksException
+     *             (if the validation failed), IOException
+     */
+    public void validateFilterField(String filterField) throws AlgebricksException, IOException {
+        IAType fieldType = getFieldType(filterField);
+        if (fieldType == null) {
+            throw new AlgebricksException("A field with this name  \"" + filterField + "\" could not be found.");
+        }
+        switch (fieldType.getTypeTag()) {
+            case INT8:
+            case INT16:
+            case INT32:
+            case INT64:
+            case FLOAT:
+            case DOUBLE:
+            case STRING:
+            case DATE:
+            case TIME:
+            case DATETIME:
+            case UNION:
+            case UUID:
+            case YEARMONTHDURATION:
+            case DAYTIMEDURATION:
+                break;
+            default:
+                throw new AlgebricksException("The field \"" + filterField + "\" which is of type "
+                        + fieldType.getTypeTag() + " cannot be used as a filter for a dataset.");
+        }
+    }
+
     public boolean doesFieldExist(String fieldName) {
         for (String f : fieldNames) {
             if (f.compareTo(fieldName) == 0) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
index 2c8c901..3c0efc2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorDescriptor.java
@@ -43,7 +43,7 @@
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
                 highKeyInclusive, dataflowHelperFactory, retainInput, retainNull, iNullWriterFactory,
-                searchOpCallbackProvider);
+                searchOpCallbackProvider, null, null);
     }
 
     @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
index 258c739..e89a27d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalBTreeSearchOperatorNodePushable.java
@@ -29,14 +29,15 @@
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.ExternalBTreeWithBuddyDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.ExternalBTreeWithBuddy;
 
-public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable{
+public class ExternalBTreeSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
 
-    public ExternalBTreeSearchOperatorNodePushable(ExternalBTreeSearchOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive) {
-        super(opDesc, ctx, partition, recordDescProvider, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive);
+    public ExternalBTreeSearchOperatorNodePushable(ExternalBTreeSearchOperatorDescriptor opDesc,
+            IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields,
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
+        super(opDesc, ctx, partition, recordDescProvider, lowKeyFields, highKeyFields, lowKeyInclusive,
+                highKeyInclusive, null, null);
     }
-    
+
     // We override the open function to search a specific version of the index
     @Override
     public void open() throws HyracksDataException {
@@ -45,7 +46,7 @@
         writer.open();
         dataFlowHelper.open();
         index = indexHelper.getIndexInstance();
-        
+
         if (retainNull) {
             int fieldCount = getFieldCount();
             nullTupleBuild = new ArrayTupleBuilder(fieldCount);
@@ -61,7 +62,7 @@
         } else {
             nullTupleBuild = null;
         }
-        
+
         ExternalBTreeWithBuddy externalIndex = (ExternalBTreeWithBuddy) index;
         try {
             searchPred = createSearchPredicate();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
index fcaaa5a..36ab7b9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
             INullWriterFactory iNullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory) {
         super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, keyFields, dataflowHelperFactory, retainInput, retainNull, iNullWriterFactory,
-                searchOpCallbackFactory);
+                searchOpCallbackFactory, null, null);
     }
 
     @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
index d9352f4..fbd490e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/external/ExternalRTreeSearchOperatorNodePushable.java
@@ -30,13 +30,13 @@
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.ExternalRTree;
 import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorNodePushable;
 
-public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperatorNodePushable{
+public class ExternalRTreeSearchOperatorNodePushable extends RTreeSearchOperatorNodePushable {
 
     public ExternalRTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
-        super(opDesc, ctx, partition, recordDescProvider, keyFields);
+        super(opDesc, ctx, partition, recordDescProvider, keyFields, null, null);
     }
-    
+
     // We override this method to specify the searched version of the index
     @Override
     public void open() throws HyracksDataException {
@@ -45,7 +45,7 @@
         indexHelper.open();
         ExternalRTreeDataflowHelper rTreeDataflowHelper = (ExternalRTreeDataflowHelper) indexHelper;
         index = indexHelper.getIndexInstance();
-        
+
         if (retainNull) {
             int fieldCount = getFieldCount();
             nullTupleBuild = new ArrayTupleBuilder(fieldCount);
@@ -61,7 +61,7 @@
         } else {
             nullTupleBuild = null;
         }
-        
+
         ExternalRTree rTreeIndex = (ExternalRTree) index;
         try {
             searchPred = createSearchPredicate();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 5c2d62d..8c5ae1f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -690,11 +690,12 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public ICopyEvaluatorFactory[] createMBRFactory(ARecordType recType, String fldName, int recordColumn, int dimension)
-            throws AlgebricksException {
+    public ICopyEvaluatorFactory[] createMBRFactory(ARecordType recType, String fldName, int recordColumn,
+            int dimension, String filterFieldName) throws AlgebricksException {
         ICopyEvaluatorFactory evalFactory = getFieldAccessEvaluatorFactory(recType, fldName, recordColumn);
         int numOfFields = dimension * 2;
-        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numOfFields];
+        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numOfFields
+                + (filterFieldName == null ? 0 : 1)];
 
         ArrayBackedValueStorage abvs1 = new ArrayBackedValueStorage();
         DataOutput dos1 = abvs1.getDataOutput();
@@ -721,6 +722,9 @@
 
             evalFactories[i] = new CreateMBREvalFactory(evalFactory, dimensionEvalFactory, coordinateEvalFactory);
         }
+        if (filterFieldName != null) {
+            evalFactories[numOfFields] = getFieldAccessEvaluatorFactory(recType, filterFieldName, recordColumn);
+        }
         return evalFactories;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
index e62e558..b7e00dd 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/AbstractLSMLocalResourceMetadata.java
@@ -15,15 +15,24 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public abstract class AbstractLSMLocalResourceMetadata implements ILocalResourceMetadata {
 
     private static final long serialVersionUID = 1L;
 
     protected final int datasetID;
+    protected final ITypeTraits[] filterTypeTraits;
+    protected final IBinaryComparatorFactory[] filterCmpFactories;
+    protected final int[] filterFields;
 
-    public AbstractLSMLocalResourceMetadata(int datasetID) {
+    public AbstractLSMLocalResourceMetadata(int datasetID, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) {
         this.datasetID = datasetID;
+        this.filterTypeTraits = filterTypeTraits;
+        this.filterCmpFactories = filterCmpFactories;
+        this.filterFields = filterFields;
     }
 
     public int getDatasetID() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
index f4a50fc..4ba6b0a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeLocalResourceMetadata.java
@@ -37,7 +37,7 @@
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties) {
         super(typeTraits, cmpFactories, bloomFilterKeyFields, isPrimary, datasetID, mergePolicyFactory,
-                mergePolicyProperties);
+                mergePolicyProperties, null, null, null, null);
     }
 
     @Override
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
index e86da4a..8cb3eeb 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalBTreeWithBuddyLocalResourceMetadata.java
@@ -45,7 +45,7 @@
     public ExternalBTreeWithBuddyLocalResourceMetadata(int datasetID, IBinaryComparatorFactory[] btreeCmpFactories,
             ITypeTraits[] typeTraits, ILSMMergePolicyFactory mergePolicyFactory,
             Map<String, String> mergePolicyProperties, int[] buddyBtreeFields) {
-        super(datasetID);
+        super(datasetID, null, null, null);
         this.btreeCmpFactories = btreeCmpFactories;
         this.typeTraits = typeTraits;
         this.mergePolicyFactory = mergePolicyFactory;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
index e88ec08..333852b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ExternalRTreeLocalResourceMetadata.java
@@ -40,13 +40,13 @@
 
     private static final long serialVersionUID = 1L;
 
-    public ExternalRTreeLocalResourceMetadata(ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILinearizeComparatorFactory linearizeCmpFactory, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties, int[] btreeFields) {
+    public ExternalRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] btreeFields) {
         super(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                linearizeCmpFactory, datasetID, mergePolicyFactory, mergePolicyProperties, btreeFields);
+                linearizeCmpFactory, datasetID, mergePolicyFactory, mergePolicyProperties, null, null, null,
+                btreeFields, null);
     }
 
     @Override
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index e6cc42e..e1700e3 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -41,17 +41,20 @@
     protected final boolean isPrimary;
     protected final ILSMMergePolicyFactory mergePolicyFactory;
     protected final Map<String, String> mergePolicyProperties;
+    protected final int[] btreeFields;
 
     public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, boolean isPrimary, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties) {
-        super(datasetID);
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
         this.typeTraits = typeTraits;
         this.cmpFactories = cmpFactories;
         this.bloomFilterKeyFields = bloomFilterKeyFields;
         this.isPrimary = isPrimary;
         this.mergePolicyFactory = mergePolicyFactory;
         this.mergePolicyProperties = mergePolicyProperties;
+        this.btreeFields = btreeFields;
     }
 
     @Override
@@ -66,7 +69,8 @@
                 isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID) : new BaseOperationTracker(
                         (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
                 runtimeContextProvider.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
-                        .createIOOperationCallback(), isPrimary);
+                        .createIOOperationCallback(), isPrimary, filterTypeTraits, filterCmpFactories, btreeFields,
+                filterFields);
         return lsmBTree;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index ded7d35..f4662fa 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -43,13 +43,18 @@
     private final boolean isPartitioned;
     private final ILSMMergePolicyFactory mergePolicyFactory;
     private final Map<String, String> mergePolicyProperties;
+    private final int[] invertedIndexFields;
+    private final int[] filterFieldsForNonBulkLoadOps;
+    private final int[] invertedIndexFieldsForNonBulkLoadOps;
 
     public LSMInvertedIndexLocalResourceMetadata(ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             boolean isPartitioned, int datasetID, ILSMMergePolicyFactory mergePolicyFactory,
-            Map<String, String> mergePolicyProperties) {
-        super(datasetID);
+            Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] invertedIndexFields, int[] filterFields,
+            int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
         this.invListTypeTraits = invListTypeTraits;
         this.invListCmpFactories = invListCmpFactories;
         this.tokenTypeTraits = tokenTypeTraits;
@@ -58,6 +63,9 @@
         this.isPartitioned = isPartitioned;
         this.mergePolicyFactory = mergePolicyFactory;
         this.mergePolicyProperties = mergePolicyProperties;
+        this.invertedIndexFields = invertedIndexFields;
+        this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
+        this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
     }
 
     @Override
@@ -80,7 +88,9 @@
                         mergePolicyFactory.createMergePolicy(mergePolicyProperties),
                         new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
                                 .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(),
+                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps);
             } else {
                 return InvertedIndexUtils.createLSMInvertedIndex(
                         virtualBufferCaches,
@@ -96,7 +106,9 @@
                         mergePolicyFactory.createMergePolicy(mergePolicyProperties),
                         new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
                                 .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
-                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback(),
+                        invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+                        filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps);
             }
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index 06e92c8..db9730c 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -47,13 +47,17 @@
     protected final ILinearizeComparatorFactory linearizeCmpFactory;
     protected final ILSMMergePolicyFactory mergePolicyFactory;
     protected final Map<String, String> mergePolicyProperties;
+    protected final int[] rtreeFields;
     protected final int[] btreeFields;
 
+
     public LSMRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
             IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int datasetID,
-            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties, int[] btreeFields) {
-        super(datasetID);
+            ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties,
+            ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] rtreeFields,
+            int[] btreeFields, int[] filterFields) {
+        super(datasetID, filterTypeTraits, filterCmpFactories, filterFields);
         this.typeTraits = typeTraits;
         this.rtreeCmpFactories = rtreeCmpFactories;
         this.btreeCmpFactories = btreeCmpFactories;
@@ -62,6 +66,7 @@
         this.linearizeCmpFactory = linearizeCmpFactory;
         this.mergePolicyFactory = mergePolicyFactory;
         this.mergePolicyProperties = mergePolicyProperties;
+        this.rtreeFields = rtreeFields;
         this.btreeFields = btreeFields;
     }
 
@@ -77,7 +82,8 @@
                     mergePolicyFactory.createMergePolicy(mergePolicyProperties), new BaseOperationTracker(
                             (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
                     runtimeContextProvider.getLSMIOScheduler(), LSMRTreeIOOperationCallbackFactory.INSTANCE
-                            .createIOOperationCallback(), linearizeCmpFactory, btreeFields);
+                            .createIOOperationCallback(), linearizeCmpFactory, rtreeFields, btreeFields,
+                    filterTypeTraits, filterCmpFactories, filterFields);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }