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);
}