Implement TokenizeOperator in addition to the changes made by Zachary Heilbron.
Now, the compiler builds the bulk-load plan.
For other details, please refer to the corresponding Hyracks changes.
Change-Id: I646539d88ea2bdc6da4fbf2b6e9460a6189125ff
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/79
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Young-Seok Kim <kisskys@gmail.com>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index 78e6f74..15b64db 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -3,9 +3,9 @@
* 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.
@@ -57,6 +57,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -434,6 +435,11 @@
}
@Override
+ public ILogicalOperator visitTokenizeOperator(TokenizeOperator op, ILogicalOperator arg) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public ILogicalOperator visitSinkOperator(SinkOperator op, ILogicalOperator arg) {
throw new UnsupportedOperationException();
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
index 9f0fc3c..c53e49a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -105,7 +105,7 @@
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
- if (ds.getDatasourceType() != AqlDataSourceType.FEED) {
+ if (ds.getDatasourceType() != AqlDataSourceType.FEED && ds.getDatasourceType() != AqlDataSourceType.ADAPTED_LOADABLE) {
if (((DatasetDataSource) ds).getDataset().getDatasetName().compareTo(insertDatasetName) == 0) {
return true;
}
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 6735ffd..64fb132 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
@@ -3,9 +3,9 @@
* 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.
@@ -15,14 +15,18 @@
package edu.uci.ics.asterix.optimizer.rules;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hadoop.mapred.lib.TotalOrderPartitioner;
import edu.uci.ics.asterix.aql.util.FunctionUtils;
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.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlIndex;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -35,10 +39,12 @@
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
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;
@@ -52,12 +58,19 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
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.AbstractLogicalOperator.ExecutionMode;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewriteRule {
@@ -86,13 +99,17 @@
/** assume the payload is always a single variable expression */
recordExpr.getUsedVariables(recordVar);
- /** op2 is the assign operator which extract primary keys from the record variable */
+ /**
+ * op2 is the assign operator which extract primary keys from the record
+ * variable
+ */
AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
if (recordVar.size() == 0) {
/**
- * For the case primary key-assignment expressions are constant expressions,
- * find assign op that creates record to be inserted/deleted.
+ * For the case primary key-assignment expressions are constant
+ * expressions, find assign op that creates record to be
+ * inserted/deleted.
*/
while (fid != AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR) {
if (op2.getInputs().size() == 0) {
@@ -137,10 +154,69 @@
List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
ILogicalOperator currentTop = op1;
boolean hasSecondaryIndex = false;
+
+ // Put an n-gram or a keyword index in the later stage of index-update,
+ // since TokenizeOperator needs to be involved.
+ Collections.sort(indexes, new Comparator<Index>() {
+ @Override
+ public int compare(Index o1, Index o2) {
+ return o1.getIndexType().ordinal() - o2.getIndexType().ordinal();
+ }
+
+ });
+
+ // Check whether multiple keyword or n-gram indexes exist
+ int secondaryIndexTotalCnt = 0;
+ for (Index index : indexes) {
+ if (index.isSecondaryIndex())
+ secondaryIndexTotalCnt++;
+ }
+
+ // Initialize inputs to the SINK operator
+ if (secondaryIndexTotalCnt > 0) {
+ op0.getInputs().clear();
+ }
+
+ // Replicate Operator is applied only when doing the bulk-load.
+ AbstractLogicalOperator replicateOp = null;
+
+ if (secondaryIndexTotalCnt > 1 && insertOp.isBulkload()) {
+ // Split the logical plan into "each secondary index update branch"
+ // to replicate each <PK,RECORD> pair.
+ replicateOp = new ReplicateOperator(secondaryIndexTotalCnt);
+ replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+ replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
+ context.computeAndSetTypeEnvironmentForOperator(replicateOp);
+ currentTop = replicateOp;
+ }
+
+ // Prepare filtering field information
+ 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)));
+ }
+ }
+
+ // Iterate each secondary index and applying Index Update operations.
for (Index index : indexes) {
if (!index.isSecondaryIndex()) {
continue;
}
+
hasSecondaryIndex = true;
List<String> secondaryKeyFields = index.getKeyFieldNames();
List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
@@ -151,25 +227,6 @@
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)));
- }
- }
AssignOperator assign = new AssignOperator(secondaryKeyVars, expressions);
ProjectOperator project = new ProjectOperator(projectVars);
@@ -180,12 +237,22 @@
} else {
assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
}
- project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+ // Only apply replicate operator when doing bulk-load
+ if (secondaryIndexTotalCnt > 1 && insertOp.isBulkload())
+ project.getInputs().add(new MutableObject<ILogicalOperator>(replicateOp));
+ else
+ project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
context.computeAndSetTypeEnvironmentForOperator(project);
+
if (additionalFilteringAssign != null) {
context.computeAndSetTypeEnvironmentForOperator(additionalFilteringAssign);
}
+
context.computeAndSetTypeEnvironmentForOperator(assign);
+
+ // BTree, Keyword, or n-gram index case
if (index.getIndexType() == IndexType.BTREE
|| index.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
|| index.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX
@@ -198,13 +265,87 @@
Mutable<ILogicalExpression> filterExpression = createFilterExpression(secondaryKeyVars,
context.getOutputTypeEnvironment(assign), false);
AqlIndex dataSourceIndex = new AqlIndex(index, dataverseName, datasetName, mp);
- 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);
+
+ // Introduce the TokenizeOperator only when doing bulk-load,
+ // and index type is keyword or n-gram.
+ if (index.getIndexType() != IndexType.BTREE && insertOp.isBulkload()) {
+
+ // Check whether the index is length-partitioned or not.
+ // If partitioned, [input variables to TokenizeOperator,
+ // token, number of token] pairs will be generated and
+ // fed into the IndexInsertDeleteOperator.
+ // If not, [input variables, token] pairs will be generated
+ // and fed into the IndexInsertDeleteOperator.
+ // Input variables are passed since TokenizeOperator is not an
+ // filtering operator.
+ boolean isPartitioned = false;
+ if (index.getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || index.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX)
+ isPartitioned = true;
+
+ // Create a new logical variable - token
+ List<LogicalVariable> tokenizeKeyVars = new ArrayList<LogicalVariable>();
+ List<Mutable<ILogicalExpression>> tokenizeKeyExprs = new ArrayList<Mutable<ILogicalExpression>>();
+ LogicalVariable tokenVar = context.newVar();
+ tokenizeKeyVars.add(tokenVar);
+ tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(
+ new VariableReferenceExpression(tokenVar)));
+
+ // Check the field type of the secondary key.
+ IAType secondaryKeyType = null;
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0)
+ .toString(), recType);
+ secondaryKeyType = keyPairType.first;
+
+ List<Object> varTypes = new ArrayList<Object>();
+ varTypes.add(NonTaggedFormatUtil.getTokenType(secondaryKeyType));
+
+ // If the index is a length-partitioned, then create
+ // additional variable - number of token.
+ // We use a special type for the length-partitioned index.
+ // The type is short, and this does not contain type info.
+ if (isPartitioned) {
+ LogicalVariable lengthVar = context.newVar();
+ tokenizeKeyVars.add(lengthVar);
+ tokenizeKeyExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+ lengthVar)));
+ varTypes.add(BuiltinType.SHORTWITHOUTTYPEINFO);
+ }
+
+ // TokenizeOperator to tokenize [SK, PK] pairs
+ TokenizeOperator tokenUpdate = new TokenizeOperator(dataSourceIndex,
+ insertOp.getPrimaryKeyExpressions(), secondaryExpressions, tokenizeKeyVars,
+ filterExpression, insertOp.getOperation(), insertOp.isBulkload(), isPartitioned, varTypes);
+ tokenUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ context.computeAndSetTypeEnvironmentForOperator(tokenUpdate);
+
+ IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
+ insertOp.getPrimaryKeyExpressions(), tokenizeKeyExprs, filterExpression,
+ insertOp.getOperation(), insertOp.isBulkload());
+ indexUpdate.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
+
+ context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
+
+ currentTop = indexUpdate;
+ op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+ } else {
+ // When TokenizeOperator is not needed
+ IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
+ insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
+ insertOp.getOperation(), insertOp.isBulkload());
+ indexUpdate.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+ indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+
+ currentTop = indexUpdate;
+ context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
+
+ if (insertOp.isBulkload())
+ op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+ }
+
} else if (index.getIndexType() == IndexType.RTREE) {
Pair<IAType, Boolean> keyPairType = Index
.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
@@ -236,31 +377,40 @@
AssignOperator assignCoordinates = new AssignOperator(keyVarList, keyExprList);
assignCoordinates.getInputs().add(new MutableObject<ILogicalOperator>(assign));
context.computeAndSetTypeEnvironmentForOperator(assignCoordinates);
- // We must enforce the filter if the originating spatial type is nullable.
+ // We must enforce the filter if the originating spatial type is
+ // nullable.
boolean forceFilter = keyPairType.second;
Mutable<ILogicalExpression> filterExpression = createFilterExpression(keyVarList,
context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
AqlIndex dataSourceIndex = new AqlIndex(index, dataverseName, datasetName, mp);
IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
- insertOp.getOperation());
+ insertOp.getOperation(), insertOp.isBulkload());
indexUpdate.setAdditionalFilteringExpressions(additionalFilteringExpressions);
indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
currentTop = indexUpdate;
context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
+
+ if (insertOp.isBulkload())
+ op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
}
+
}
if (!hasSecondaryIndex) {
return false;
}
- op0.getInputs().clear();
- op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+ if (!insertOp.isBulkload()) {
+ op0.getInputs().clear();
+ op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+ }
return true;
}
@SuppressWarnings("unchecked")
private void prepareVarAndExpression(String field, String[] fieldNames, List<LogicalVariable> recordVar,
- List<Mutable<ILogicalExpression>> expressions, List<LogicalVariable> vars, IOptimizationContext context) {
+ List<Mutable<ILogicalExpression>> expressions, List<LogicalVariable> vars, IOptimizationContext context) throws AlgebricksException {
Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
recordVar.get(0)));
int pos = -1;
@@ -270,6 +420,9 @@
break;
}
}
+ if (pos == -1) {
+ throw new AlgebricksException("An exception occurred when finding the position of the indexed field -" + field);
+ }
// 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))));
@@ -284,7 +437,8 @@
private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<Mutable<ILogicalExpression>>();
- // Add 'is not null' to all nullable secondary index keys as a filtering condition.
+ // Add 'is not null' to all nullable secondary index keys as a filtering
+ // condition.
for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
IAType secondaryKeyType = (IAType) typeEnv.getVarType(secondaryKeyVar);
if (!isNullableType(secondaryKeyType) && !forceFilter) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index 4b7fd44..8239eca 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -300,7 +300,8 @@
ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
if (e1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
IDataSource<AqlSourceId> dataSource = (IDataSource<AqlSourceId>) scan.getDataSource();
- if (((AqlDataSource) dataSource).getDatasourceType().equals(AqlDataSourceType.FEED)) {
+ AqlDataSourceType dsType = ((AqlDataSource) dataSource).getDatasourceType();
+ if (dsType == AqlDataSourceType.FEED || dsType == AqlDataSourceType.ADAPTED_LOADABLE) {
return false;
}
AqlSourceId asid = dataSource.getId();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index 2550a34..ef9d6cd 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -3,9 +3,9 @@
* 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.
@@ -22,7 +22,6 @@
import edu.uci.ics.asterix.algebra.operators.CommitOperator;
import edu.uci.ics.asterix.algebra.operators.physical.CommitPOperator;
import edu.uci.ics.asterix.common.transactions.JobId;
-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.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -63,48 +62,55 @@
while (descendantOp != null) {
if (descendantOp.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE) {
IndexInsertDeleteOperator indexInsertDeleteOperator = (IndexInsertDeleteOperator) descendantOp;
- primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
- datasetId = ((DatasetDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource())
- .getDataset().getDatasetId();
- break;
+ if (!indexInsertDeleteOperator.isBulkload()) {
+ primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
+ datasetId = ((DatasetDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource())
+ .getDataset().getDatasetId();
+ break;
+ }
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) descendantOp;
- primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
- datasetId = ((DatasetDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+ if (!insertDeleteOperator.isBulkload()) {
+ primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
+ datasetId = ((DatasetDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+ break;
+ }
+ }
+ if (descendantOp.getInputs().size() < 1) {
break;
}
descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
}
- if (primaryKeyExprs != null) {
-
- //copy primaryKeyExprs
- List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
- for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
- VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr.getValue();
- primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
- }
-
- //get JobId(TransactorId)
- AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
- JobId jobId = mp.getJobId();
-
- //create the logical and physical operator
- CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
- CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars,
- mp.isWriteTransaction());
- commitOperator.setPhysicalOperator(commitPOperator);
-
- //create ExtensionOperator and put the commitOperator in it.
- ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
- extensionOperator.setPhysicalOperator(commitPOperator);
-
- //update plan link
- extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
- context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
- opRef.setValue(extensionOperator);
+ if (primaryKeyExprs == null) {
+ return false;
}
+ //copy primaryKeyExprs
+ List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
+ for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
+ VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr.getValue();
+ primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
+ }
+
+ //get JobId(TransactorId)
+ AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+ JobId jobId = mp.getJobId();
+
+ //create the logical and physical operator
+ CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
+ CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars,
+ mp.isWriteTransaction());
+ commitOperator.setPhysicalOperator(commitPOperator);
+
+ //create ExtensionOperator and put the commitOperator in it.
+ ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
+ extensionOperator.setPhysicalOperator(commitPOperator);
+
+ //update plan link
+ extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
+ context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+ opRef.setValue(extensionOperator);
return true;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
index 6744471..27908f3 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
@@ -207,7 +207,8 @@
case POINT:
case POINT3D:
case POLYGON:
- case RECTANGLE: {
+ case RECTANGLE:
+ case SHORTWITHOUTTYPEINFO: {
return true;
}
case RECORD: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index d7e3ceb..cfd25c0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -38,6 +38,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -261,6 +262,11 @@
}
@Override
+ public Void visitTokenizeOperator(TokenizeOperator op, Void tag) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitExternalDataLookupOperator(ExternalDataLookupOperator op, Void arg) throws AlgebricksException {
return null;
}
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 6922909..aab2664 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
@@ -3,9 +3,9 @@
* 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.
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.translator;
import java.io.File;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -92,9 +93,9 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.functions.FunctionConstants;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.declared.AdaptedLoadableDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
@@ -105,7 +106,6 @@
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.metadata.functions.ExternalFunctionCompilerUtil;
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;
@@ -114,6 +114,7 @@
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -142,6 +143,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
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.DistinctOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -182,24 +184,94 @@
private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
public AqlExpressionToPlanTranslator(AqlMetadataProvider metadataProvider, int currentVarCounter,
- String outputDatasetName, ICompiledDmlStatement stmt) {
+ String outputDatasetName, ICompiledDmlStatement stmt) throws AlgebricksException {
this.context = new TranslationContext(new Counter(currentVarCounter));
this.outputDatasetName = outputDatasetName;
this.stmt = stmt;
this.metadataProvider = metadataProvider;
+ FormatUtils.getDefaultFormat().registerRuntimeFunctions();
}
public int getVarCounter() {
return context.getVarCounter();
}
- public ILogicalPlan translate(Query expr) throws AlgebricksException, AsterixException {
- IDataFormat format = FormatUtils.getDefaultFormat();
- format.registerRuntimeFunctions();
+ public ILogicalPlan translateLoad() throws AlgebricksException {
+ CompiledLoadFromFileStatement clffs = (CompiledLoadFromFileStatement) stmt;
+ Dataset dataset = metadataProvider.findDataset(clffs.getDataverseName(), clffs.getDatasetName());
+ if (dataset == null) {
+ throw new AlgebricksException("Unable to load dataset " + clffs.getDatasetName()
+ + " since it does not exist");
+ }
+ IAType itemType = metadataProvider.findType(clffs.getDataverseName(), dataset.getItemTypeName());
+ DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+ stmt.getDatasetName());
+ List<String> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
+ AdaptedLoadableDataSource lds;
+ try {
+ lds = new AdaptedLoadableDataSource(dataset, itemType, clffs.getAdapter(), clffs.getProperties(),
+ clffs.alreadySorted());
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ ILogicalOperator etsOp = new EmptyTupleSourceOperator();
+ List<LogicalVariable> loadVars = new ArrayList<>();
+ // Add a logical variable for PK.
+ for (int i = 0; i < partitionKeys.size(); ++i) {
+ loadVars.add(context.newVar());
+ }
+ // Add a logical variable for the record.
+ loadVars.add(context.newVar());
+ DataSourceScanOperator dssOp = new DataSourceScanOperator(loadVars, lds);
+ dssOp.getInputs().add(new MutableObject<ILogicalOperator>(etsOp));
+ ILogicalExpression payloadExpr = new VariableReferenceExpression(loadVars.get(loadVars.size() - 1));
+ Mutable<ILogicalExpression> payloadRef = new MutableObject<ILogicalExpression>(payloadExpr);
+ // Set the expression for PK
+ List<Mutable<ILogicalExpression>> pkRefs = new ArrayList<>();
+ for (int i = 0; i < loadVars.size() - 1; ++i) {
+ ILogicalExpression pkExpr = new VariableReferenceExpression(loadVars.get(i));
+ Mutable<ILogicalExpression> pkRef = new MutableObject<ILogicalExpression>(pkExpr);
+ pkRefs.add(pkRef);
+ }
+
+ // Set the Filter variable if it was specified
+ 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, loadVars.get(loadVars.size() - 1), additionalFilteringVars,
+ additionalFilteringAssignExpressions, additionalFilteringExpressions);
+
+ additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
+ additionalFilteringAssignExpressions);
+ }
+
+ InsertDeleteOperator insertOp = new InsertDeleteOperator(targetDatasource, payloadRef, pkRefs,
+ InsertDeleteOperator.Kind.INSERT, true);
+ insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
+
+ if (additionalFilteringAssign != null) {
+ additionalFilteringAssign.getInputs().add(new MutableObject<ILogicalOperator>(dssOp));
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(additionalFilteringAssign));
+ } else {
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(dssOp));
+ }
+
+ ILogicalOperator leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+ return new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(leafOperator));
+ }
+
+ public ILogicalPlan translate(Query expr) throws AlgebricksException, AsterixException {
Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
new EmptyTupleSourceOperator()));
-
ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
ILogicalOperator topOp = p.first;
ProjectOperator project = (ProjectOperator) topOp;
@@ -277,7 +349,7 @@
switch (stmt.getKind()) {
case INSERT: {
InsertDeleteOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteOperator.Kind.INSERT);
+ varRefsForLoading, InsertDeleteOperator.Kind.INSERT, false);
insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
leafOperator = new SinkOperator();
@@ -286,7 +358,7 @@
}
case DELETE: {
InsertDeleteOperator deleteOp = new InsertDeleteOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteOperator.Kind.DELETE);
+ varRefsForLoading, InsertDeleteOperator.Kind.DELETE, false);
deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
leafOperator = new SinkOperator();
@@ -295,7 +367,7 @@
}
case CONNECT_FEED: {
InsertDeleteOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteOperator.Kind.INSERT);
+ varRefsForLoading, InsertDeleteOperator.Kind.INSERT, false);
insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
leafOperator = new SinkOperator();
diff --git a/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.adm b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.adm
new file mode 100644
index 0000000..5a5cec0
--- /dev/null
+++ b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.adm
@@ -0,0 +1,10 @@
+{"dblpid": "books/acm/kim95/Blakeley95", "title": "OQL[C++] Extending C++ with an Object Query Capability.", "authors": "José A. Blakeley", "misc": "2002-01-03 69-88 Modern Database Systems db/books/collections/kim95.html#Blakeley95 1995" }
+{"dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{"dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{"dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{"dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
+{"dblpid": "books/acm/kim95/Kim95", "title": "Introduction to Part 1 Next-Generation Database Technology.", "authors": "Won Kim", "misc": "2002-01-03 5-17 1995 Modern Database Systems db/books/collections/kim95.html#Kim95" }
+{"dblpid": "books/acm/kim95/Kim95b", "title": "Introduction to Part 2 Technology for Interoperating Legacy Databases.", "authors": "Won Kim", "misc": "2002-01-03 515-520 1995 Modern Database Systems db/books/collections/kim95.html#Kim95b" }
+{"dblpid": "books/acm/kim95/KimG95", "title": "Requirements for a Performance Benchmark for Object-Oriented Database Systems.", "authors": "Won Kim Jorge F. Garza", "misc": "2002-01-03 203-215 1995 Modern Database Systems db/books/collections/kim95.html#KimG95" }
+{"dblpid": "books/acm/kim95/Kowalski95", "title": "The POSC Solution to Managing E&P Data.", "authors": "Vincent J. Kowalski", "misc": "2002-01-03 281-301 1995 Modern Database Systems db/books/collections/kim95.html#Kowalski95" }
+{"dblpid": "books/acm/kim95/Lunt95", "title": "Authorization in Object-Oriented Databases.", "authors": "Teresa F. Lunt", "misc": "2002-01-03 130-145 1995 Modern Database Systems db/books/collections/kim95.html#Lunt95" }
diff --git a/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.csv b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.csv
new file mode 100644
index 0000000..d7a9491
--- /dev/null
+++ b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.csv
@@ -0,0 +1,20 @@
+Object SQL - A Language for the Design and Implementation of Object Databases.,Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent
+OQL[C++] Extending C++ with an Object Query Capability.,José A. Blakeley
+Transaction Management in Multidatabase Systems.,Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz
+Multimedia Information Systems Issues and Approaches.,Stavros Christodoulakis Leonidas Koveos
+Active Database Systems.,Umeshwar Dayal Eric N. Hanson Jennifer Widom
+Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.,Test Person
+Distributed Databases.,Hector Garcia-Molina Meichun Hsu
+An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.,Nathan Goodman
+Cooperative Transactions for Multiuser Environments.,Gail E. Kaiser
+Schema Architecture of the UniSQL/M Multidatabase System,William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham
+Physical Object Management.,Alfons Kemper Guido Moerkotte
+Introduction to Part 1 Next-Generation Database Technology.,Won Kim
+Object-Oriented Database Systems Promises Reality and Future.,Won Kim
+Introduction to Part 2 Technology for Interoperating Legacy Databases.,Won Kim
+On Resolving Schematic Heterogeneity in Multidatabase Systems.,Won Kim Injun Choi Sunit K. Gala Mark Scheevel
+Requirements for a Performance Benchmark for Object-Oriented Database Systems.,Won Kim Jorge F. Garza
+On View Support in Object-Oriented Databases Systems.,Won Kim William Kelley
+The POSC Solution to Managing E&P Data.,Vincent J. Kowalski
+C++ Bindings to an Object Database.,David Krieger Tim Andrews
+Authorization in Object-Oriented Databases.,Teresa F. Lunt
diff --git a/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.txt b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.txt
new file mode 100644
index 0000000..bb9d82a
--- /dev/null
+++ b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk.txt
@@ -0,0 +1,20 @@
+Object SQL - A Language for the Design and Implementation of Object Databases.:Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent
+OQL[C++] Extending C++ with an Object Query Capability.:José A. Blakeley
+Transaction Management in Multidatabase Systems.:Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz
+Multimedia Information Systems Issues and Approaches.:Stavros Christodoulakis Leonidas Koveos
+Active Database Systems.:Umeshwar Dayal Eric N. Hanson Jennifer Widom
+Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.:Test Person
+Distributed Databases.:Hector Garcia-Molina Meichun Hsu
+An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.:Nathan Goodman
+Cooperative Transactions for Multiuser Environments.:Gail E. Kaiser
+Schema Architecture of the UniSQL/M Multidatabase System:William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham
+Physical Object Management.:Alfons Kemper Guido Moerkotte
+Introduction to Part 1 Next-Generation Database Technology.:Won Kim
+Object-Oriented Database Systems Promises, Reality, and Future.:Won Kim
+Introduction to Part 2 Technology for Interoperating Legacy Databases.:Won Kim
+On Resolving Schematic Heterogeneity in Multidatabase Systems.:Won Kim Injun Choi Sunit K. Gala Mark Scheevel
+Requirements for a Performance Benchmark for Object-Oriented Database Systems.:Won Kim Jorge F. Garza
+On View Support in Object-Oriented Databases Systems.:Won Kim William Kelley
+The POSC Solution to Managing E&P Data.:Vincent J. Kowalski
+C++ Bindings to an Object Database.:David Krieger Tim Andrews
+Authorization in Object-Oriented Databases.:Teresa F. Lunt
diff --git a/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.adm b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.adm
new file mode 100644
index 0000000..174e408
--- /dev/null
+++ b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.adm
@@ -0,0 +1,10 @@
+{"id": "89fb1567-70f8-4e59-87d9-ace64f73daf1", "dblpid": "books/acm/kim95/Blakeley95", "title": "OQL[C++] Extending C++ with an Object Query Capability.", "authors": "José A. Blakeley", "misc": "2002-01-03 69-88 Modern Database Systems db/books/collections/kim95.html#Blakeley95 1995" }
+{"id": "8cea25ab-55f8-467e-929d-94888f754832", "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{"id": "8d59a28c-dfde-404b-a6a4-de2acefbcd88", "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{"id": "8d68c3e2-5101-479e-9dbb-5dc59d3d465c", "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{"id": "9171b105-e2e0-4ebd-b7ae-d1c2914b3233", "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
+{"id": "93e4d3ec-a6c9-4a71-a8a1-d8a827250c72", "dblpid": "books/acm/kim95/Kim95", "title": "Introduction to Part 1 Next-Generation Database Technology.", "authors": "Won Kim", "misc": "2002-01-03 5-17 1995 Modern Database Systems db/books/collections/kim95.html#Kim95" }
+{"id": "9816c851-7232-498b-9eb3-5fed80f46307", "dblpid": "books/acm/kim95/Kim95b", "title": "Introduction to Part 2 Technology for Interoperating Legacy Databases.", "authors": "Won Kim", "misc": "2002-01-03 515-520 1995 Modern Database Systems db/books/collections/kim95.html#Kim95b" }
+{"id": "00f3969c-8427-47cd-9bd3-201cd931af9c", "dblpid": "books/acm/kim95/KimG95", "title": "Requirements for a Performance Benchmark for Object-Oriented Database Systems.", "authors": "Won Kim Jorge F. Garza", "misc": "2002-01-03 203-215 1995 Modern Database Systems db/books/collections/kim95.html#KimG95" }
+{"id": "01d75219-1d25-4a52-a651-d81dca2f9330", "dblpid": "books/acm/kim95/Kowalski95", "title": "The POSC Solution to Managing E&P Data.", "authors": "Vincent J. Kowalski", "misc": "2002-01-03 281-301 1995 Modern Database Systems db/books/collections/kim95.html#Kowalski95" }
+{"id": "02a199ca-bf58-412e-bd9f-60a0c975a8ac", "dblpid": "books/acm/kim95/Lunt95", "title": "Authorization in Object-Oriented Databases.", "authors": "Teresa F. Lunt", "misc": "2002-01-03 130-145 1995 Modern Database Systems db/books/collections/kim95.html#Lunt95" }
diff --git a/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.csv b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.csv
new file mode 100644
index 0000000..3ef4fdd
--- /dev/null
+++ b/asterix-app/data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.csv
@@ -0,0 +1,20 @@
+00f3969c-8427-47cd-9bd3-201cd931af9c,Object SQL - A Language for the Design and Implementation of Object Databases.,Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent
+01d75219-1d25-4a52-a651-d81dca2f9330,OQL[C++] Extending C++ with an Object Query Capability.,José A. Blakeley
+02a199ca-bf58-412e-bd9f-60a0c975a8ac,Transaction Management in Multidatabase Systems.,Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz
+03465c48-9215-47d1-9dcc-26b919db3879,Multimedia Information Systems Issues and Approaches.,Stavros Christodoulakis Leonidas Koveos
+044caf96-82c6-49d9-9369-b0aaa7d79e6d,Active Database Systems.,Umeshwar Dayal Eric N. Hanson Jennifer Widom
+05b7846c-a59f-4de6-a1fc-2d5bb74118c8,Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.,Test Person
+06f68dbc-3e76-41cb-a27b-dc63464594ae,Distributed Databases.,Hector Garcia-Molina Meichun Hsu
+0b500ae8-f539-4f27-91dc-ba7974f6b4ca,An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.,Nathan Goodman
+0b9f1f30-f1f1-4c4b-a87e-b92e811bfabf,Cooperative Transactions for Multiuser Environments.,Gail E. Kaiser
+0c1cc0d4-af8e-410e-ac81-c37bf99d28b4,Schema Architecture of the UniSQL/M Multidatabase System,William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham
+1183e948-2cb0-4155-93fb-6aeeb652b4b2,Physical Object Management.,Alfons Kemper Guido Moerkotte
+11ce8c22-f7cb-4c1c-9b97-5e2119c1c01d,Introduction to Part 1 Next-Generation Database Technology.,Won Kim
+11e1f9f0-41cd-46fd-b8af-5791ecf6e2dd,Object-Oriented Database Systems Promises Reality and Future.,Won Kim
+16231f8c-dcd1-49b6-b1e7-0715b1a46388,Introduction to Part 2 Technology for Interoperating Legacy Databases.,Won Kim
+20a4b67b-496c-4bd9-af81-2de94ca79fe4,On Resolving Schematic Heterogeneity in Multidatabase Systems.,Won Kim Injun Choi Sunit K. Gala Mark Scheevel
+2623fde7-dd96-4100-93c6-8cf0f421dee1,Requirements for a Performance Benchmark for Object-Oriented Database Systems.,Won Kim Jorge F. Garza
+27712335-9540-4b08-b423-69887e4614cc,On View Support in Object-Oriented Databases Systems.,Won Kim William Kelley
+2ad34903-838a-473d-a35f-b8b1b63f688c,The POSC Solution to Managing E&P Data.,Vincent J. Kowalski
+2c045409-db9e-408b-a2dc-b95ed5ec9702,C++ Bindings to an Object Database.,David Krieger Tim Andrews
+2c824d67-f302-4afe-84df-cdc6841723ba,Authorization in Object-Oriented Databases.,Teresa F. Lunt
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index 486c52b..72a4085 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -22,6 +22,7 @@
import org.json.JSONException;
import edu.uci.ics.asterix.api.common.Job.SubmissionMode;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
import edu.uci.ics.asterix.aql.expression.FunctionDecl;
import edu.uci.ics.asterix.aql.expression.Query;
import edu.uci.ics.asterix.aql.expression.visitor.AQLPrintVisitor;
@@ -229,7 +230,13 @@
AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(queryMetadataProvider, varCounter,
outputDatasetName, statement);
- ILogicalPlan plan = t.translate(rwQ);
+ ILogicalPlan plan;
+ // statement = null when it's a query
+ if (statement == null || statement.getKind() != Kind.LOAD) {
+ plan = t.translate(rwQ);
+ } else {
+ plan = t.translateLoad();
+ }
boolean isWriteTransaction = queryMetadataProvider.isWriteTransaction();
LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
@@ -247,7 +254,7 @@
}
}
- if (rwQ != null) {
+ if (rwQ != null || statement.getKind() == Kind.LOAD) {
StringBuilder buffer = new StringBuilder();
PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
out.print(buffer);
@@ -315,7 +322,7 @@
break;
}
}
- if (rwQ != null) {
+ if (rwQ != null || statement.getKind() == Kind.LOAD) {
StringBuilder buffer = new StringBuilder();
PlanPrettyPrinter.printPlan(plan, buffer, pvisitor, 0);
out.print(buffer);
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 4100d44..25d9884 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
@@ -3,9 +3,9 @@
* 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.
@@ -83,7 +83,6 @@
import edu.uci.ics.asterix.file.ExternalIndexingOperations;
import edu.uci.ics.asterix.file.FeedOperations;
import edu.uci.ics.asterix.file.IndexOperations;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.IDatasetDetails;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -193,7 +192,7 @@
/**
* Compiles and submits for execution a list of AQL statements.
- *
+ *
* @param hcc
* A Hyracks client connection that is used to submit a jobspec to Hyracks.
* @param hdc
@@ -577,7 +576,7 @@
//#. execute compensation operations
// remove the index in NC
// [Notice]
- // As long as we updated(and committed) metadata, we should remove any effect of the job
+ // As long as we updated(and committed) metadata, we should remove any effect of the job
// because an exception occurs during runJob.
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
bActiveTxn = true;
@@ -1034,7 +1033,7 @@
}
}
- //#. prepare jobs which will drop corresponding datasets with indexes.
+ //#. prepare jobs which will drop corresponding datasets with indexes.
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
for (int j = 0; j < datasets.size(); j++) {
String datasetName = datasets.get(j).getDatasetName();
@@ -1075,7 +1074,7 @@
}
jobsToExecute.add(DataverseOperations.createDropDataverseJobSpec(dv, metadataProvider));
- //#. mark PendingDropOp on the dataverse record by
+ //#. mark PendingDropOp on the dataverse record by
// first, deleting the dataverse record from the DATAVERSE_DATASET
// second, inserting the dataverse record with the PendingDropOp value into the DATAVERSE_DATASET
MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverseName);
@@ -1117,7 +1116,7 @@
runJob(hcc, jobSpec, true);
}
} catch (Exception e2) {
- //do no throw exception since still the metadata needs to be compensated.
+ //do no throw exception since still the metadata needs to be compensated.
e.addSuppressed(e2);
}
@@ -1282,7 +1281,7 @@
runJob(hcc, jobSpec, true);
}
} catch (Exception e2) {
- //do no throw exception since still the metadata needs to be compensated.
+ //do no throw exception since still the metadata needs to be compensated.
e.addSuppressed(e2);
}
@@ -1366,7 +1365,7 @@
new Index(dataverseName, datasetName, indexName, index.getIndexType(),
index.getKeyFieldNames(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
- //#. commit the existing transaction before calling runJob.
+ //#. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
@@ -1428,7 +1427,7 @@
new Index(dataverseName, datasetName, indexName, index.getIndexType(),
index.getKeyFieldNames(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
- //#. commit the existing transaction before calling runJob.
+ //#. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
@@ -1604,49 +1603,30 @@
private void handleLoadStatement(AqlMetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
throws Exception {
-
- LoadStatement loadStmt = (LoadStatement) stmt;
- String dataverseName = getActiveDataverseName(loadStmt.getDataverseName());
- String datasetName = loadStmt.getDatasetName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
- MetadataLockManager.INSTANCE.modifyDatasetBegin(dataverseName, dataverseName + "." + datasetName);
- List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+ acquireReadLatch();
try {
- CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName, datasetName,
- loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
-
- IDataFormat format = getDataFormat(metadataProvider.getMetadataTxnContext(), dataverseName);
- Job job = DatasetOperations.createLoadDatasetJobSpec(metadataProvider, cls, format);
- jobsToExecute.add(job.getJobSpec());
- // Also load the dataset's secondary indexes.
- List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, loadStmt
- .getDatasetName().getValue());
- for (Index index : datasetIndexes) {
- if (!index.isSecondaryIndex()) {
- continue;
- }
- // Create CompiledCreateIndexStatement from metadata entity 'index'.
- CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(),
- dataverseName, index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(),
- index.getIndexType());
- jobsToExecute.add(IndexOperations.buildSecondaryIndexLoadingJobSpec(cis, metadataProvider));
- }
+ LoadStatement loadStmt = (LoadStatement) stmt;
+ String dataverseName = getActiveDataverseName(loadStmt.getDataverseName());
+ CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName, loadStmt
+ .getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
+ loadStmt.dataIsAlreadySorted());
+ JobSpecification spec = APIFramework.compileQuery(null, metadataProvider, null, 0, null, sessionConfig,
+ out, pdf, cls);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
-
- for (JobSpecification jobspec : jobsToExecute) {
- runJob(hcc, jobspec, true);
+ if (spec != null) {
+ runJob(hcc, spec, true);
}
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
}
-
throw e;
} finally {
- MetadataLockManager.INSTANCE.modifyDatasetEnd(dataverseName, dataverseName + "." + datasetName);
+ releaseReadLatch();
}
}
@@ -2204,7 +2184,7 @@
return;
}
- // At this point, we know data has changed in the external file system, record transaction in metadata and start
+ // At this point, we know data has changed in the external file system, record transaction in metadata and start
transactionDataset = ExternalIndexingOperations.createTransactionDataset(ds);
/*
* Remove old dataset record and replace it with a new one
@@ -2384,18 +2364,6 @@
return startedJobIds;
}
- private static IDataFormat getDataFormat(MetadataTransactionContext mdTxnCtx, String dataverseName)
- throws AsterixException {
- Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
- IDataFormat format;
- try {
- format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
- } catch (Exception e) {
- throw new AsterixException(e);
- }
- return format;
- }
-
private String getActiveDataverseName(String dataverse) throws AlgebricksException {
if (dataverse != null) {
return dataverse;
@@ -2410,6 +2378,14 @@
return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
}
+ private void acquireReadLatch() {
+ MetadataManager.INSTANCE.acquireReadLatch();
+ }
+
+ private void releaseReadLatch() {
+ MetadataManager.INSTANCE.releaseReadLatch();
+ }
+
private void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
try {
if (IS_DEBUG_MODE) {
@@ -2421,4 +2397,5 @@
throw new IllegalStateException(rootE);
}
}
+
}
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 49a6408..7a9e1b2 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
@@ -16,72 +16,41 @@
package edu.uci.ics.asterix.file;
import java.io.File;
-import java.io.IOException;
import java.rmi.RemoteException;
-import java.util.List;
import java.util.Map;
import java.util.logging.Logger;
-import edu.uci.ics.asterix.api.common.Job;
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.ExternalDatasetTransactionState;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
-import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.formats.base.IDataFormat;
import edu.uci.ics.asterix.metadata.MetadataManager;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
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.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
-import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
@@ -92,9 +61,6 @@
public class DatasetOperations {
- private static final PhysicalOptimizationConfig physicalOptimizationConfig = OptimizationConfUtil
- .getPhysicalOptimizationConfig();
-
private static Logger LOGGER = Logger.getLogger(DatasetOperations.class.getName());
public static JobSpecification createDropDatasetJobSpec(CompiledDatasetDropStatement datasetDropStmt,
@@ -222,223 +188,10 @@
return spec;
}
- @SuppressWarnings("rawtypes")
- public static Job createLoadDatasetJobSpec(AqlMetadataProvider metadataProvider,
- CompiledLoadFromFileStatement loadStmt, IDataFormat format) throws AsterixException, AlgebricksException {
- MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
- String dataverseName = loadStmt.getDataverseName();
- String datasetName = loadStmt.getDatasetName();
- Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
- if (dataset == null) {
- throw new AsterixException("Could not find dataset " + datasetName + " in dataverse " + dataverseName);
- }
- if (dataset.getDatasetType() != DatasetType.INTERNAL) {
- throw new AsterixException("Cannot load data into dataset (" + datasetName + ")" + "of type "
- + dataset.getDatasetType());
- }
- JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
- ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName,
- dataset.getItemTypeName()).getDatatype();
- ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
-
- IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(dataset, itemType,
- format.getBinaryHashFunctionFactoryProvider());
- IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
- itemType, format.getBinaryComparatorFactoryProvider());
- ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
- 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);
-
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider.buildExternalDataScannerRuntime(
- spec, itemType, externalDatasetDetails, format, dataset);
- IOperatorDescriptor scanner = p.first;
- AlgebricksPartitionConstraint scannerPc = p.second;
- RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde, format,
- filterTypeTraits == null ? false : true);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, scanner, scannerPc);
-
- AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, format,
- filterTypeTraits == null ? false : true);
- AlgebricksMetaOperatorDescriptor asterixOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
- new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { recDesc });
-
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixOp, scannerPc);
-
- int numKeys = DatasetUtils.getPartitioningKeys(dataset).size();
- int[] keys = new int[numKeys];
- 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 + 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);
-
- FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
- StringBuilder sb = new StringBuilder();
- 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);
- AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
- TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad;
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(dataset,
- metadataProvider.getMetadataTxnContext());
- if (!loadStmt.alreadySorted()) {
- btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
- 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, filterTypeTraits,
- filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
- splitsAndConstraint.second);
-
- spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, asterixOp, 0);
-
- int framesLimit = physicalOptimizationConfig.getMaxFramesExternalSort();
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, framesLimit, keys,
- comparatorFactories, recDesc);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sorter,
- splitsAndConstraint.second);
- IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
- new FieldHashPartitionComputerFactory(keys, hashFactories));
- spec.connect(hashConn, asterixOp, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
- } else {
- btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
- 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, filterTypeTraits,
- filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
- splitsAndConstraint.second);
-
- spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, asterixOp, 0);
-
- IConnectorDescriptor sortMergeConn = new MToNPartitioningMergingConnectorDescriptor(spec,
- new FieldHashPartitionComputerFactory(keys, hashFactories), keys, comparatorFactories, null);
- spec.connect(sortMergeConn, asterixOp, 0, btreeBulkLoad, 0);
- }
- spec.addRoot(btreeBulkLoad);
- spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-
- return new Job(spec);
- }
-
private static String stringOf(FileSplit fs) {
return fs.getNodeName() + ":" + fs.getLocalFile().toString();
}
- private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType,
- 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 + numFilterFields];
- for (int i = 0; i < numKeys; i++) {
- Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
- .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
- evalFactories[i] = evalFactoryAndType.first;
- }
- 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 + numFilterFields; i++) {
- outColumns[i] = i + 1;
- projectionList[i + 1] = i + 1;
- }
- IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[evalFactories.length];
- for (int i = 0; i < evalFactories.length; ++i) {
- sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
- evalFactories[i]);
- }
- return new AssignRuntimeFactory(outColumns, sefs, projectionList);
- }
-
- @SuppressWarnings("rawtypes")
- private static RecordDescriptor computePayloadKeyRecordDescriptor(Dataset dataset, ARecordType itemType,
- 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 + numFilterFields];
- recordFields[0] = payloadSerde;
- for (int i = 0; i < numKeys; i++) {
- IAType keyType;
- try {
- keyType = itemType.getFieldType(partitioningKeys.get(i));
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
- 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);
- }
-
public static JobSpecification compactDatasetJobSpec(Dataverse dataverse, String datasetName,
AqlMetadataProvider metadata) throws AsterixException, AlgebricksException {
String dataverseName = dataverse.getDataverseName();
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 7b45992..707b7de 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
@@ -267,7 +267,7 @@
HDFSIndexingAdapterFactory adapterFactory = new HDFSIndexingAdapterFactory();
adapterFactory.setFiles(files);
adapterFactory.configure(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties(),
- (ARecordType) itemType);
+ (ARecordType) itemType, false, null);
return new Pair<ExternalDataScanOperatorDescriptor, AlgebricksPartitionConstraint>(
new ExternalDataScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory),
adapterFactory.getPartitionConstraint());
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 076a844..5459a80 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
@@ -34,13 +34,18 @@
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
@@ -84,8 +89,8 @@
// External dataset local resource and dataflow helper
int[] buddyBreeFields = new int[] { numSecondaryKeys };
ILocalResourceMetadata localResourceMetadata = new ExternalBTreeWithBuddyLocalResourceMetadata(
- dataset.getDatasetId(), secondaryComparatorFactories, secondaryTypeTraits,
- mergePolicyFactory, mergePolicyFactoryProperties, buddyBreeFields);
+ dataset.getDatasetId(), secondaryComparatorFactories, secondaryTypeTraits, mergePolicyFactory,
+ mergePolicyFactoryProperties, buddyBreeFields);
localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
LocalResource.ExternalBTreeWithBuddyResource);
indexDataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(mergePolicyFactory,
@@ -116,8 +121,13 @@
* In case of external data, this method is used to build loading jobs for both initial load on index creation
* and transaction load on dataset referesh
*/
+
+ RecordDescriptor[] rDescs = new RecordDescriptor[] { new RecordDescriptor(new ISerializerDeserializer[] {}) };
+ AlgebricksMetaOperatorDescriptor etsOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 1,
+ new IPushRuntimeFactory[] { new EmptyTupleSourceRuntimeFactory() }, rDescs);
// Create external indexing scan operator
ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
+
// Assign op.
AlgebricksMetaOperatorDescriptor asterixAssignOp = createExternalAssignOp(spec, numSecondaryKeys);
@@ -139,16 +149,26 @@
LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numSecondaryKeys },
ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
+ IOperatorDescriptor root;
if (externalFiles != null) {
// Transaction load
secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, numSecondaryKeys, dataflowHelperFactory,
- BTree.DEFAULT_FILL_FACTOR);
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ root = secondaryBulkLoadOp;
} else {
// Initial load
secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys, dataflowHelperFactory,
- BTree.DEFAULT_FILL_FACTOR);
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
+ new RecordDescriptor[] { secondaryRecDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
+ root = metaOp;
}
+ AlgebricksPartitionConstraintHelper
+ .setPartitionConstraintInJobSpec(spec, etsOp, primaryPartitionConstraint);
// Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), etsOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
if (anySecondaryKeyIsNullable) {
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
@@ -157,7 +177,7 @@
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
}
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
+ spec.addRoot(root);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
return spec;
} else {
@@ -192,6 +212,8 @@
.getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
secondaryBTreeFields, secondaryFilterFields), GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] { secondaryRecDesc });
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
@@ -202,7 +224,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
}
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
+ spec.addRoot(metaOp);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
return spec;
}
@@ -221,21 +244,21 @@
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
- mergePolicyFactory, mergePolicyFactoryProperties,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
- secondaryBTreeFields, secondaryFilterFields), NoOpOperationCallbackFactory.INSTANCE);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), mergePolicyFactory,
+ mergePolicyFactoryProperties, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits,
+ filterCmpFactories, secondaryBTreeFields, secondaryFilterFields),
+ NoOpOperationCallbackFactory.INSTANCE);
} else {
// External dataset
compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, 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 890253a..63c7411 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
@@ -3,9 +3,9 @@
* 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.
@@ -212,7 +212,7 @@
} else {
numFilterFields = 0;
}
-
+
numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
.splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, datasetName);
@@ -453,10 +453,10 @@
fieldPermutation[i] = i;
}
TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- secondaryFileSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
- secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint, false,
- dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+ secondaryRecDesc, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+ secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
secondaryPartitionConstraint);
return treeIndexBulkLoadOp;
@@ -528,7 +528,8 @@
return indexingOpAndConstraints.first;
}
- protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec, int numSecondaryKeys) throws AlgebricksException {
+ protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec, int numSecondaryKeys)
+ throws AlgebricksException {
int[] outColumns = new int[numSecondaryKeys];
int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
@@ -574,10 +575,9 @@
}
ExternalIndexBulkModifyOperatorDescriptor treeIndexBulkLoadOp = new ExternalIndexBulkModifyOperatorDescriptor(
spec, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
- dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE, deletedFiles, fieldPermutation,
- fillFactor, numElementsHint);
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, secondaryBloomFilterKeyFields, dataflowHelperFactory,
+ NoOpOperationCallbackFactory.INSTANCE, deletedFiles, fieldPermutation, fillFactor, numElementsHint);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
secondaryPartitionConstraint);
return 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 d7ff27c..79ab7a2 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
@@ -3,9 +3,9 @@
* 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.
@@ -40,6 +40,8 @@
import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -252,6 +254,8 @@
// Create secondary inverted index bulk load op.
LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
+ AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
@@ -263,7 +267,8 @@
}
spec.connect(new OneToOneConnectorDescriptor(spec), tokenizerOp, 0, sortOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, invIndexBulkLoadOp, 0);
- spec.addRoot(invIndexBulkLoadOp);
+ spec.connect(new OneToOneConnectorDescriptor(spec), invIndexBulkLoadOp, 0, metaOp, 0);
+ spec.addRoot(metaOp);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
return spec;
}
@@ -275,7 +280,7 @@
primaryKeyFields[i] = numSecondaryKeys + i;
}
BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
- tokenKeyPairRecDesc, tokenizerFactory, docField, primaryKeyFields, isPartitioned);
+ tokenKeyPairRecDesc, tokenizerFactory, docField, primaryKeyFields, isPartitioned, false);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
primaryPartitionConstraint);
return tokenizerOp;
@@ -302,11 +307,10 @@
}
IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
- spec, fieldPermutation, false, numElementsHint, false,
+ spec, secondaryRecDesc, fieldPermutation, false, numElementsHint, false,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
- invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
- NoOpOperationCallbackFactory.INSTANCE);
+ invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
secondaryPartitionConstraint);
return invIndexBulkLoadOp;
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 38e6a4f..1f48509 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
@@ -47,7 +47,11 @@
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -57,7 +61,6 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -114,10 +117,9 @@
} else {
// External dataset
// Prepare a LocalResourceMetadata which will be stored in NC's local resource repository
- ILocalResourceMetadata localResourceMetadata = new ExternalRTreeLocalResourceMetadata(
- secondaryTypeTraits, secondaryComparatorFactories,
- ExternalIndexingOperations.getBuddyBtreeComparatorFactories(), valueProviderFactories,
- RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
+ ILocalResourceMetadata localResourceMetadata = new ExternalRTreeLocalResourceMetadata(secondaryTypeTraits,
+ secondaryComparatorFactories, ExternalIndexingOperations.getBuddyBtreeComparatorFactories(),
+ valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
secondaryComparatorFactories.length), dataset.getDatasetId(), mergePolicyFactory,
mergePolicyFactoryProperties, primaryKeyFields);
localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(localResourceMetadata,
@@ -258,6 +260,8 @@
.getBloomFilterFalsePositiveRate(), rtreeFields, primaryKeyFields,
filterTypeTraits, filterCmpFactories, secondaryFilterFields),
GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { new SinkRuntimeFactory() }, new RecordDescriptor[] {});
// Connect the operators.
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
@@ -268,7 +272,8 @@
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
}
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
+ spec.addRoot(metaOp);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
} else {
// External dataset
@@ -276,6 +281,9 @@
* In case of external data, this method is used to build loading jobs for both initial load on index creation
* and transaction load on dataset referesh
*/
+ RecordDescriptor[] rDescs = new RecordDescriptor[] { new RecordDescriptor(new ISerializerDeserializer[] {}) };
+ AlgebricksMetaOperatorDescriptor etsOp = new AlgebricksMetaOperatorDescriptor(spec, 0, 1,
+ new IPushRuntimeFactory[] { new EmptyTupleSourceRuntimeFactory() }, rDescs);
// Create external indexing scan operator
ExternalDataScanOperatorDescriptor primaryScanOp = createExternalIndexingOp(spec);
// Assign op.
@@ -302,17 +310,27 @@
storageProperties.getBloomFilterFalsePositiveRate(), new int[] { numNestedSecondaryKeyFields },
ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset));
// Create secondary RTree bulk load op.
+ IOperatorDescriptor root;
AbstractTreeIndexOperatorDescriptor secondaryBulkLoadOp;
if (externalFiles != null) {
// Transaction load
secondaryBulkLoadOp = createExternalIndexBulkModifyOp(spec, numNestedSecondaryKeyFields,
- dataflowHelperFactory, BTree.DEFAULT_FILL_FACTOR);
+ dataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ root = secondaryBulkLoadOp;
} else {
// Initial load
secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numNestedSecondaryKeyFields,
- dataflowHelperFactory, BTree.DEFAULT_FILL_FACTOR);
+ dataflowHelperFactory, GlobalConfig.DEFAULT_TREE_FILL_FACTOR);
+ AlgebricksMetaOperatorDescriptor metaOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { new SinkRuntimeFactory() },
+ new RecordDescriptor[] { secondaryRecDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBulkLoadOp, 0, metaOp, 0);
+ root = metaOp;
}
+ AlgebricksPartitionConstraintHelper
+ .setPartitionConstraintInJobSpec(spec, etsOp, primaryPartitionConstraint);
// Connect the operators.
+ spec.connect(new OneToOneConnectorDescriptor(spec), etsOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
if (anySecondaryKeyIsNullable) {
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
@@ -321,7 +339,7 @@
spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
}
spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
- spec.addRoot(secondaryBulkLoadOp);
+ spec.addRoot(root);
spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
}
return spec;
@@ -351,8 +369,8 @@
// External dataset
compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, secondaryBloomFilterKeyFields,
new ExternalRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
primaryComparatorFactories, mergePolicyFactory, mergePolicyFactoryProperties,
new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
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..d5cb506 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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": "Wed Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
\ 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..d5cb506 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 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": "Wed Aug 20 14:03:26 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 Aug 20 14:03:26 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 Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
index 10b5853..f8bd22d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index 1976c17..9d2d60b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index d0b4cd1..8cad02f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index 1976c17..9d2d60b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index d0b4cd1..8cad02f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan b/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan
index c4f9bd8..97c7b80 100644
--- a/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC), $$2(ASC)] HASH:[$$1, $$2] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
index c288790..d86e4f5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
index c288790..d86e4f5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index b66d07a..83a29cc 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- COMMIT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-rtree-secondary-index.plan
index 8e0e202..25b65de 100644
--- a/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-rtree-secondary-index.plan
@@ -1,3 +1,12 @@
+-- SINK |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BULKLOAD |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$1(ASC)] HASH:[$$1] |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -7,4 +16,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.ddl.aql
new file mode 100644
index 0000000..5e9b6ea
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.ddl.aql
@@ -0,0 +1,16 @@
+// Bulk-Load test case: load a ADM file to a dataset that has an autogenerated-PK
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+ id: uuid,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id autogenerated;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.2.update.aql
new file mode 100644
index 0000000..d58640a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset DBLP using localfs
+(("path"="nc1://data/pub-small/dblp-small-id-autogenerated-pk.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.3.query.aql
new file mode 100644
index 0000000..2590ca1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title,"Authorization in Object-Oriented Databases.")
+return $o.title;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.1.ddl.aql
new file mode 100644
index 0000000..04398bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.1.ddl.aql
@@ -0,0 +1,18 @@
+// Bulk-Load test case: load a ADM file to a dataset that has an auto-generated-PK
+// This test should fail since we load every field data including the auto-genereated-PK field - id,
+// since Asterix creates random UUID for each record
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+ id: uuid,
+ dblpid: string,
+ title: string,
+ authors: string,
+ misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id autogenerated;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.2.update.aql
new file mode 100644
index 0000000..69411c7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset DBLP using localfs
+(("path"="nc1://data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.3.query.aql
new file mode 100644
index 0000000..2590ca1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_01.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title,"Authorization in Object-Oriented Databases.")
+return $o.title;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.ddl.aql
new file mode 100644
index 0000000..4225494
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.ddl.aql
@@ -0,0 +1,14 @@
+// Bulk-Load test case: load a delimited-text file to a dataset that has an autogenerated-PK
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+ id: uuid,
+ title: string,
+ authors: string
+}
+
+create dataset DBLP(DBLPType) primary key id autogenerated;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.2.update.aql
new file mode 100644
index 0000000..7b23a09
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset DBLP using localfs
+(("path"="nc1://data/pub-small/dblp-small-id-autogenerated-pk.csv"),("format"="delimited-text"),("delimiter"=","));
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.3.query.aql
new file mode 100644
index 0000000..2590ca1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title,"Authorization in Object-Oriented Databases.")
+return $o.title;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.ddl.aql
new file mode 100644
index 0000000..3d74d04
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.ddl.aql
@@ -0,0 +1,16 @@
+// Bulk-Load test case: load a delimited-text file to a dataset that has an autogenerated-PK
+// This test should fail since we load every field data including the auto-genereated-PK field - id,
+// since Asterix creates random UUID for each record
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+ id: uuid,
+ title: string,
+ authors: string
+}
+
+create dataset DBLP(DBLPType) primary key id autogenerated;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.2.update.aql
new file mode 100644
index 0000000..13e10ea
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset DBLP using localfs
+(("path"="nc1://data/pub-small/dblp-small-id-autogenerated-pk_including_uuid.csv"),("format"="delimited-text"),("delimiter"=","));
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.3.query.aql
new file mode 100644
index 0000000..2590ca1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title,"Authorization in Object-Oriented Databases.")
+return $o.title;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.ddl.aql
new file mode 100644
index 0000000..d424e16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.ddl.aql
@@ -0,0 +1,14 @@
+// Bulk-Load test case: load a delimited-text file to a dataset that has an autogenerated-PK
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+ id: uuid,
+ title: string,
+ authors: string
+}
+
+create dataset DBLP(DBLPType) primary key id autogenerated;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.2.update.aql
new file mode 100644
index 0000000..3503381
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.2.update.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id-autogenerated-pk.txt"),("format"="delimited-text"),("delimiter"=":"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.3.query.aql
new file mode 100644
index 0000000..3be37b0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.3.query.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title,"Physical Object Management.")
+return $o.title;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.adm
new file mode 100644
index 0000000..1244966
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_01/load-with-autogenerated-pk_adm_01.1.adm
@@ -0,0 +1 @@
+"Authorization in Object-Oriented Databases."
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_02.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_02.1.adm
new file mode 100644
index 0000000..1244966
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_adm_02/load-with-autogenerated-pk_adm_02.1.adm
@@ -0,0 +1 @@
+"Authorization in Object-Oriented Databases."
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.adm
new file mode 100644
index 0000000..1244966
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_01/load-with-autogenerated-pk_csv_01.1.adm
@@ -0,0 +1 @@
+"Authorization in Object-Oriented Databases."
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.adm
new file mode 100644
index 0000000..1244966
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_csv_02/load-with-autogenerated-pk_csv_02.1.adm
@@ -0,0 +1 @@
+"Authorization in Object-Oriented Databases."
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.adm
new file mode 100644
index 0000000..9c7eccb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/load-with-autogenerated-pk_txt_01/load-with-autogenerated-pk_txt_01.1.adm
@@ -0,0 +1 @@
+"Physical Object Management."
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 9a24645..3464a29 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -3,9 +3,9 @@
! 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.
@@ -1464,6 +1464,33 @@
</test-case>
-->
<test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_txt_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_txt_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_adm_02">
+ <output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
+ <expected-error>edu.uci.ics.asterix.runtime.operators.file.ADMDataParser.ParseException</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_csv_01">
+ <output-dir compare="Text">load-with-autogenerated-pk_csv_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
+ <compilation-unit name="load-with-autogenerated-pk_csv_02">
+ <output-dir compare="Text">load-with-autogenerated-pk_csv_02</output-dir>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="dml">
<compilation-unit name="load-with-index">
<output-dir compare="Text">load-with-index</output-dir>
</compilation-unit>
@@ -1483,7 +1510,6 @@
<output-dir compare="Text">load-with-word-index</output-dir>
</compilation-unit>
</test-case>
-
<test-case FilePath="dml">
<compilation-unit name="opentype-c2o-recursive">
<output-dir compare="Text">opentype-c2o-recursive</output-dir>
@@ -3289,12 +3315,12 @@
</compilation-unit>
</test-case>
<test-case FilePath="open-closed">
- <compilation-unit name="query-issue456">
+ <compilation-unit name="query-issue456">
<output-dir compare="Text">query-issue456</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="open-closed">
- <compilation-unit name="query-issue465">
+ <compilation-unit name="query-issue465">
<output-dir compare="Text">query-issue465</output-dir>
</compilation-unit>
</test-case>
@@ -4492,7 +4518,7 @@
<output-dir compare="Text">cross-dv07</output-dir>
</compilation-unit>
</test-case>
- <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
+ <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
<test-case FilePath="cross-dataverse">
<compilation-unit name="cross-dv08">
<output-dir compare="Text">cross-dv08</output-dir>
@@ -4536,7 +4562,7 @@
<expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
- <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
+ <!--NotImplementedException: No binary comparator factory implemented for type RECORD.
<test-case FilePath="cross-dataverse">
<compilation-unit name="cross-dv17">
<output-dir compare="Text">cross-dv17</output-dir>
@@ -4572,7 +4598,7 @@
</test-case>
</test-group>
<test-group name="user-defined-functions">
- <test-case FilePath="user-defined-functions">
+ <test-case FilePath="user-defined-functions">
<compilation-unit name="query-issue218-2">
<output-dir compare="Text">query-issue218-2</output-dir>
</compilation-unit>
@@ -4766,13 +4792,13 @@
<test-case FilePath="user-defined-functions">
<compilation-unit name="f01">
<output-dir compare="Text">f01</output-dir>
- <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="user-defined-functions">
<compilation-unit name="invoke-private-function">
<output-dir compare="Text">invoke-private-function</output-dir>
- <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -4790,13 +4816,13 @@
<test-case FilePath="load">
<compilation-unit name="issue14_query">
<output-dir compare="Text">none</output-dir>
- <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="issue315_query">
<output-dir compare="Text">none</output-dir>
- <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
@@ -4807,7 +4833,7 @@
<test-case FilePath="load">
<compilation-unit name="issue650_query">
<output-dir compare="Text">none</output-dir>
- <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+ <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
@@ -4839,7 +4865,7 @@
<test-case FilePath="load">
<compilation-unit name="duplicate-key-error">
<output-dir compare="Text">none</output-dir>
- <expected-error>edu.uci.ics.hyracks.api.exceptions.HyracksException</expected-error>
+ <expected-error>edu.uci.ics.hyracks.api.exceptions.HyracksException</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
@@ -4937,7 +4963,7 @@
</test-case>
</test-group>
<test-group name="external-indexing">
- <test-case FilePath="external-indexing">
+ <test-case FilePath="external-indexing">
<compilation-unit name="text-format">
<output-dir compare="Text">text-format</output-dir>
</compilation-unit>
@@ -4947,7 +4973,7 @@
<output-dir compare="Text">sequence-format</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="external-indexing">
+ <test-case FilePath="external-indexing">
<compilation-unit name="rc-format">
<output-dir compare="Text">rc-format</output-dir>
</compilation-unit>
@@ -5000,7 +5026,7 @@
</compilation-unit>
</test-case>
<test-case FilePath="temporal">
- <compilation-unit name="accessors">
+ <compilation-unit name="accessors">
<output-dir compare="Text">accessors</output-dir>
</compilation-unit>
</test-case>
@@ -5014,63 +5040,63 @@
<output-dir compare="Text">accessors_interval_null</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="adjust_timezone">
+ <test-case FilePath="temporal">
+ <compilation-unit name="adjust_timezone">
<output-dir compare="Text">adjust_timezone</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="calendar_duration">
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="calendar_duration">
<output-dir compare="Text">calendar_duration</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="date_functions">
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="date_functions">
<output-dir compare="Text">date_functions</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="datetime_functions">
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="datetime_functions">
<output-dir compare="Text">datetime_functions</output-dir>
</compilation-unit>
- </test-case>
+ </test-case>
<!--
- <test-case FilePath="temporal">
+ <test-case FilePath="temporal">
<compilation-unit name="insert_from_delimited_ds">
<output-dir compare="Text">insert_from_delimited_ds</output-dir>
</compilation-unit>
- </test-case>
+ </test-case>
-->
- <test-case FilePath="temporal">
- <compilation-unit name="insert_from_ext_ds">
+ <test-case FilePath="temporal">
+ <compilation-unit name="insert_from_ext_ds">
<output-dir compare="Text">insert_from_ext_ds</output-dir>
</compilation-unit>
- </test-case>
+ </test-case>
<test-case FilePath="temporal">
<compilation-unit name="insert_from_ext_ds_2">
<output-dir compare="Text">insert_from_ext_ds_2</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="interval_functions">
+ <test-case FilePath="temporal">
+ <compilation-unit name="interval_functions">
<output-dir compare="Text">interval_functions</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="time_functions">
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="time_functions">
<output-dir compare="Text">time_functions</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="constructor">
- <compilation-unit name="interval">
+ </test-case>
+ <test-case FilePath="constructor">
+ <compilation-unit name="interval">
<output-dir compare="Text">interval</output-dir>
</compilation-unit>
- </test-case>
- <test-case FilePath="temporal">
- <compilation-unit name="duration_comps">
+ </test-case>
+ <test-case FilePath="temporal">
+ <compilation-unit name="duration_comps">
<output-dir compare="Text">duration_comps</output-dir>
</compilation-unit>
- </test-case>
+ </test-case>
</test-group>
<test-group name="leftouterjoin">
<test-case FilePath="leftouterjoin">
@@ -5119,10 +5145,10 @@
<compilation-unit name="probe-pidx-with-join-invidx-sidx2">
<output-dir compare="Text">probe-pidx-with-join-invidx-sidx2</output-dir>
</compilation-unit>
- </test-case>
+ </test-case>
</test-group>
<test-group name="distinct">
- <test-case FilePath="distinct">
+ <test-case FilePath="distinct">
<compilation-unit name="query-issue443">
<output-dir compare="Text">query-issue443</output-dir>
</compilation-unit>
@@ -5165,67 +5191,67 @@
</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-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-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-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index cdab917..e69c903 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -151,7 +151,7 @@
}
@Override
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
if (!initialized) {
hdfsScheduler = initializeHDFSScheduler();
initialized = true;
@@ -206,7 +206,7 @@
/*
* This method is overridden to do the following:
- * if data is text data (adm or delimited text), it will use a text tuple parser,
+ * if data is text data (adm or delimited text), it will use a text tuple parser,
* otherwise it will use hdfs record object parser
*/
protected void configureFormat(IAType sourceDatatype) throws Exception {
@@ -214,9 +214,9 @@
if (specifiedFormat == null) {
throw new IllegalArgumentException(" Unspecified data format");
} else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
- parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, false);
+ parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, false, false, -1, null);
} else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
- parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, false);
+ parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, false, false, -1, null);
} else if (FORMAT_BINARY.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
parserFactory = new HDFSObjectTupleParserFactory((ARecordType) atype, this, configuration);
} else {
@@ -232,7 +232,7 @@
* 1. NoOp means appended file
* 2. AddOp means new file
* 3. UpdateOp means the delta of a file
- *
+ *
* @return
* @throws IOException
*/
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
index 00104e0..5e573cb 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
@@ -104,13 +104,13 @@
((HDFSIndexingParserFactory) parserFactory).setJobConf(conf);
((HDFSIndexingParserFactory) parserFactory).setArguments(configuration);
HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits,
- conf, clusterLocations, files, parserFactory, ctx, nodeName, (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
+ conf, clusterLocations, files, parserFactory, ctx, nodeName, (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
(String) configuration.get(KEY_FORMAT));
return hdfsIndexingAdapter;
}
@Override
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
if (!initialized) {
hdfsScheduler = initializeHDFSScheduler();
initialized = true;
@@ -129,13 +129,13 @@
// The function below is overwritten to create indexing adapter factory instead of regular adapter factory
configureFormat(atype);
}
-
+
protected void configureFormat(IAType sourceDatatype) throws Exception {
parserFactory = new HDFSIndexingParserFactory((ARecordType)atype,
- (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
- (String) configuration.get(KEY_FORMAT),
- (String) configuration.get(KEY_DELIMITER),
+ (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
+ (String) configuration.get(KEY_FORMAT),
+ (String) configuration.get(KEY_DELIMITER),
(String) configuration.get(HDFSAdapterFactory.KEY_PARSER));
}
@@ -168,9 +168,9 @@
}
fieldParserFactories[i] = vpf;
}
- return new DelimitedDataParser(recordType, fieldParserFactories, delimiter);
+ return new DelimitedDataParser(recordType, fieldParserFactories, delimiter, false, -1, null);
}
-
+
public static AlgebricksPartitionConstraint getClusterLocations() {
ArrayList<String> locs = new ArrayList<String>();
Map<String, String[]> stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index ab59241..1db713d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -72,10 +72,10 @@
}
@Override
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
if (!configured) {
populateConfiguration(configuration);
- hdfsAdapterFactory.configure(configuration, outputType);
+ hdfsAdapterFactory.configure(configuration, outputType, isPKAutoGenerated, primaryKeys);
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index 8c37841..2da3803 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -50,10 +50,12 @@
private IAType sourceDatatype;
private FileSplit[] fileSplits;
+ private boolean isPKAutoGenerated = false;
+ private int primaryKeyPosition = -1;
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(fileSplits, parserFactory, sourceDatatype, ctx);
+ NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(fileSplits, parserFactory, sourceDatatype, ctx, isPKAutoGenerated, primaryKeyPosition);
return fsAdapter;
}
@@ -73,12 +75,47 @@
}
@Override
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
this.configuration = configuration;
String[] splits = ((String) configuration.get(KEY_PATH)).split(",");
- IAType sourceDatatype = (IAType) outputType;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+
+ // Configuration for a Dataset that has an auto-generated PK
+ ARecordType sourceRecordType = null;
+ IAType sourceDatatype = null;
+
+ if (isPKAutoGenerated) {
+ // Currently, support for a single field Primary Key only
+ String primaryKey = primaryKeys.get(0);
+
+ String typeName = outputType.getTypeName();
+ String[] targetFieldNames = outputType.getFieldNames();
+ String[] sourceFieldNames = new String[targetFieldNames.length - 1];
+ IAType[] targetFieldTypes = outputType.getFieldTypes();
+ IAType[] sourceFieldTypes = new IAType[targetFieldTypes.length - 1];
+ boolean isOpen = outputType.isOpen();
+
+ int idx = 0;
+ for (int i = 0; i < outputType.getFieldNames().length; i++) {
+ if (!targetFieldNames[i].equals(primaryKey)) {
+ sourceFieldNames[idx] = targetFieldNames[i];
+ sourceFieldTypes[idx] = targetFieldTypes[i];
+ idx++;
+ } else {
+ this.primaryKeyPosition = i;
+ }
+ }
+ sourceRecordType = new ARecordType(typeName, sourceFieldNames,
+ sourceFieldTypes, isOpen);
+
+ sourceDatatype = (IAType) sourceRecordType;
+ } else {
+ sourceDatatype = (IAType) outputType;
+ }
+
configureFileSplits(splits);
- configureFormat(sourceDatatype);
+ configureFormat(sourceDatatype, isPKAutoGenerated, primaryKeyPosition, outputType);
+
}
@Override
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
index 1dcd351..b424b96 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/StreamBasedAdapterFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -61,7 +61,7 @@
protected ITupleParserFactory parserFactory;
protected ITupleParser parser;
-
+
protected List<ExternalFile> files;
protected static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
@@ -73,7 +73,9 @@
typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
}
- protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+ // Used when a data set has an auto-generated PK field
+ protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType, boolean conditionalPush,
+ boolean isPKAutoGenerated, int primaryKeyPosition, ARecordType origSourceDataTypeForAutoGeneratedPK)
throws AsterixException {
int n = recordType.getFieldTypes().length;
IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
@@ -105,21 +107,25 @@
Character delimiter = delimiterValue.charAt(0);
return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, fieldParserFactories, delimiter,
- configuration) : new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
+ configuration) : new NtDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter,
+ isPKAutoGenerated, primaryKeyPosition, origSourceDataTypeForAutoGeneratedPK);
}
- protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType, boolean conditionalPush)
+ protected ITupleParserFactory getADMDataTupleParserFactory(ARecordType recordType, boolean conditionalPush,
+ boolean isPKAutoGenerated, int primaryKeyPosition, ARecordType origSourceDataTypeForAutoGeneratedPK)
throws AsterixException {
try {
return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, configuration)
- : new AdmSchemafullRecordParserFactory(recordType);
+ : new AdmSchemafullRecordParserFactory(recordType, isPKAutoGenerated,
+ primaryKeyPosition, origSourceDataTypeForAutoGeneratedPK);
} catch (Exception e) {
throw new AsterixException(e);
}
}
- protected void configureFormat(IAType sourceDatatype) throws Exception {
+ protected void configureFormat(IAType sourceDatatype, boolean isPKAutoGenerated,
+ int primaryKeyPosition, IAType origSourceDataTypeForAutoGeneratedPK) throws Exception {
String propValue = (String) configuration.get(BATCH_SIZE);
int batchSize = propValue != null ? Integer.parseInt(propValue) : -1;
propValue = (String) configuration.get(BATCH_INTERVAL);
@@ -132,9 +138,11 @@
if (specifiedFormat == null) {
throw new IllegalArgumentException(" Unspecified data format");
} else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
- parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+ parserFactory = getDelimitedDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush,
+ isPKAutoGenerated, primaryKeyPosition, (ARecordType) origSourceDataTypeForAutoGeneratedPK);
} else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
- parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush);
+ parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, conditionalPush,
+ isPKAutoGenerated, primaryKeyPosition, (ARecordType) origSourceDataTypeForAutoGeneratedPK);
} else {
throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
}
@@ -144,4 +152,6 @@
}
+
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
index 7d754a2..d3c5716 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FileSystemBasedAdapter.java
@@ -37,12 +37,17 @@
protected final ITupleParser tupleParser;
protected final IAType sourceDatatype;
protected IHyracksTaskContext ctx;
+ protected boolean isPKAutoGenerated;
+ protected int primaryKeyPosition;
- public FileSystemBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx)
+ public FileSystemBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx,
+ boolean isPKAutoGenerated, int primaryKeyPosition)
throws HyracksDataException {
this.tupleParser = parserFactory.createTupleParser(ctx);
this.sourceDatatype = sourceDatatype;
this.ctx = ctx;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyPosition;
}
@Override
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index 7e7c6c0..91e86dc 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -3,9 +3,9 @@
* 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.
@@ -53,7 +53,7 @@
public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
String nodeName, ITupleParserFactory parserFactory, IHyracksTaskContext ctx,
Map<String, String> configuration, List<ExternalFile> files) throws HyracksDataException {
- super(parserFactory, atype, ctx);
+ super(parserFactory, atype, ctx, false, -1);
this.readSchedule = readSchedule;
this.executed = executed;
this.inputSplits = inputSplits;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
index 0c2d3e5..2864efd 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
@@ -45,12 +45,12 @@
private String inputFormat;
// content format <adm, delimited-text, binary>
private String format;
-
+
public HDFSIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits,
JobConf conf, AlgebricksPartitionConstraint clusterLocations, List<ExternalFile> files,
ITupleParserFactory parserFactory, IHyracksTaskContext ctx, String nodeName,
String inputFormat, String format) throws IOException {
- super(parserFactory, atype, ctx);
+ super(parserFactory, atype, ctx, false, -1);
this.nodeName = nodeName;
this.readSchedule = readSchedule;
this.executed = executed;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
index 6280635..7f7a48d 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveAdapter.java
@@ -3,9 +3,9 @@
* 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.
@@ -33,7 +33,7 @@
public HiveAdapter(IAType atype, HDFSAdapter hdfsAdapter, ITupleParserFactory parserFactory, IHyracksTaskContext ctx)
throws HyracksDataException {
- super(parserFactory, atype, ctx);
+ super(parserFactory, atype, ctx, false, -1);
this.hdfsAdapter = hdfsAdapter;
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
index 3663e52..d98c0ff 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/NCFileSystemAdapter.java
@@ -21,6 +21,8 @@
import java.io.InputStream;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -39,7 +41,13 @@
public NCFileSystemAdapter(FileSplit[] fileSplits, ITupleParserFactory parserFactory, IAType atype,
IHyracksTaskContext ctx) throws HyracksDataException {
- super(parserFactory, atype, ctx);
+ super(parserFactory, atype, ctx, false, -1);
+ this.fileSplits = fileSplits;
+ }
+
+ public NCFileSystemAdapter(FileSplit[] fileSplits, ITupleParserFactory parserFactory, IAType atype,
+ IHyracksTaskContext ctx, boolean isPKAutoGenerated, int primaryKeyPosition) throws HyracksDataException {
+ super(parserFactory, atype, ctx, isPKAutoGenerated, primaryKeyPosition);
this.fileSplits = fileSplits;
}
@@ -57,6 +65,16 @@
}
@Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ InputStream in = getInputStream(partition);
+ if (tupleParser instanceof AbstractTupleParser) {
+ ((AbstractTupleParser)tupleParser).setFilename(getFilename(partition));
+ }
+ tupleParser.parse(in, writer);
+
+ }
+
+ @Override
public String getFilename(int partition) {
final FileSplit fileSplit = fileSplits[partition];
return fileSplit.getNodeName() + ":" + fileSplit.getLocalFile().getFile().getPath();
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
index a95cabc..eb9adcc 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParser.java
@@ -32,12 +32,12 @@
* This class is used with hdfs objects instead of hdfs
*/
public class HDFSObjectTupleParser implements ITupleParser{
-
+
private ArrayTupleBuilder tb;
private final FrameTupleAppender appender;
private final ByteBuffer frame;
private IAsterixHDFSRecordParser deserializer;
-
+
public HDFSObjectTupleParser(IHyracksTaskContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer) throws HyracksDataException {
appender = new FrameTupleAppender(ctx.getFrameSize());
frame = ctx.allocateFrame();
@@ -69,7 +69,7 @@
throw new HyracksDataException(e);
}
}
-
+
protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
FrameUtils.flushFrame(frame, writer);
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
index af78902..366236b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSObjectTupleParserFactory.java
@@ -33,14 +33,14 @@
private HDFSAdapterFactory adapterFactory;
// adapter arguments
private Map<String,String> arguments;
-
+
public HDFSObjectTupleParserFactory(ARecordType atype, HDFSAdapterFactory adapterFactory, Map<String,String> arguments){
this.parserClassName = (String) arguments.get(HDFSAdapterFactory.KEY_PARSER);
this.atype = atype;
this.arguments = arguments;
this.adapterFactory = adapterFactory;
}
-
+
@Override
public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
IAsterixHDFSRecordParser objectParser;
@@ -58,9 +58,8 @@
} catch (Exception e) {
throw new HyracksDataException("Unable to initialize object parser", e);
}
-
+
return new HDFSObjectTupleParser(ctx, atype, objectParser);
}
-
-
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
index 92e07f4..3d360e7 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/RCFileIndexingTupleParser.java
@@ -25,7 +25,7 @@
private Integer rowNumber = -1;
private Integer lastFileNumber = -1;
private long lastByteLocation = -1;
-
+
public RCFileIndexingTupleParser(IHyracksTaskContext ctx, ARecordType recType, IAsterixHDFSRecordParser deserializer)
throws HyracksDataException {
super(ctx, recType, deserializer);
@@ -54,4 +54,5 @@
aMutableInt.setValue(rowNumber);
tb.addField(intSerde, aMutableInt);
}
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
index 3127a68..db30213 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/TextOrSeqIndexingTupleParser.java
@@ -35,8 +35,9 @@
AbstractHDFSReader inReader) throws Exception {
aMutableInt.setValue(inReader.getFileNumber());
aMutableLong.setValue(inReader.getReaderPosition());
-
+
tb.addField(intSerde, aMutableInt);
tb.addField(longSerde, aMutableLong);
}
+
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adaptor/TestTypedAdaptorFactory.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adaptor/TestTypedAdaptorFactory.java
index 39f7ab2..365b6dd 100644
--- a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adaptor/TestTypedAdaptorFactory.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adaptor/TestTypedAdaptorFactory.java
@@ -72,7 +72,7 @@
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- ITupleParserFactory tupleParserFactory = new AdmSchemafullRecordParserFactory(adapterOutputType);
+ ITupleParserFactory tupleParserFactory = new AdmSchemafullRecordParserFactory(adapterOutputType, false, -1, null);
return new TestTypedAdaptor(tupleParserFactory, adapterOutputType, ctx, configuration);
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AdaptedLoadableDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AdaptedLoadableDataSource.java
new file mode 100644
index 0000000..cfe0e05
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AdaptedLoadableDataSource.java
@@ -0,0 +1,94 @@
+package edu.uci.ics.asterix.metadata.declared;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+
+public class AdaptedLoadableDataSource extends AqlDataSource {
+
+ private final INodeDomain domain;
+
+ private final IAType[] schemaTypes;
+
+ private final Dataset targetDataset;
+
+ private final List<String> partitioningKeys;
+
+ private final String adapter;
+
+ private final Map<String, String> adapterProperties;
+
+ private final boolean alreadySorted;
+
+ public AdaptedLoadableDataSource(Dataset targetDataset, IAType itemType, String adapter,
+ Map<String, String> properties, boolean alreadySorted) throws AlgebricksException, IOException {
+ super(new AqlSourceId("loadable_dv", "loadable_ds"), "loadable_dv", "loadable_source",
+ AqlDataSourceType.ADAPTED_LOADABLE);
+ this.targetDataset = targetDataset;
+ this.adapter = adapter;
+ this.adapterProperties = properties;
+ this.alreadySorted = alreadySorted;
+ partitioningKeys = DatasetUtils.getPartitioningKeys(targetDataset);
+ domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(targetDataset));
+ ARecordType recType = (ARecordType) itemType;
+ schemaTypes = new IAType[partitioningKeys.size() + 1];
+ for (int i = 0; i < partitioningKeys.size(); ++i) {
+ schemaTypes[i] = recType.getFieldType(partitioningKeys.get(i));
+ }
+ schemaTypes[schemaTypes.length - 1] = itemType;
+ }
+
+ @Override
+ public IAType[] getSchemaTypes() {
+ return schemaTypes;
+ }
+
+ @Override
+ public INodeDomain getDomain() {
+ return domain;
+ }
+
+ @Override
+ public void computeLocalStructuralProperties(List<ILocalStructuralProperty> localProps,
+ List<LogicalVariable> variables) {
+ if (alreadySorted) {
+ for (int i = 0; i < partitioningKeys.size(); ++i) {
+ localProps.add(new LocalOrderProperty(new OrderColumn(variables.get(i), OrderKind.ASC)));
+ }
+ }
+ }
+
+ public List<String> getPartitioningKeys() {
+ return partitioningKeys;
+ }
+
+ public String getAdapter() {
+ return adapter;
+ }
+
+ public Map<String, String> getAdapterProperties() {
+ return adapterProperties;
+ }
+
+ public IAType getTargetDatasetType() {
+ return schemaTypes[schemaTypes.length - 1];
+ }
+
+ public Dataset getTargetDataset() {
+ return targetDataset;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 6d1a00b..acbfc83 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -3,9 +3,9 @@
* 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.
@@ -53,10 +53,11 @@
public enum AqlDataSourceType {
INTERNAL_DATASET,
EXTERNAL_DATASET,
- FEED
+ FEED,
+ ADAPTED_LOADABLE
}
- public AqlDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
+ public AqlDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName,
AqlDataSourceType datasourceType) throws AlgebricksException {
this.id = id;
this.datasourceDataverse = datasourceDataverse;
@@ -76,6 +77,11 @@
public abstract INodeDomain getDomain();
+ public void computeLocalStructuralProperties(List<ILocalStructuralProperty> localProps,
+ List<LogicalVariable> variables) {
+ // do nothing
+ }
+
@Override
public AqlSourceId getId() {
return id;
@@ -88,9 +94,9 @@
@Override
public IDataSourcePropertiesProvider getPropertiesProvider() {
- return new AqlDataSourcePartitioningProvider(datasourceType, domain);
+ return new AqlDataSourcePartitioningProvider(this, domain);
}
-
+
@Override
public void computeFDs(List<LogicalVariable> scanVariables, List<FunctionalDependency> fdList) {
int n = scanVariables.size();
@@ -102,55 +108,35 @@
fdList.add(fd);
}
}
-
-
+
private static class AqlDataSourcePartitioningProvider implements IDataSourcePropertiesProvider {
- private INodeDomain domain;
+ private final AqlDataSource ds;
- private AqlDataSourceType aqlDataSourceType;
+ private final INodeDomain domain;
- public AqlDataSourcePartitioningProvider(AqlDataSourceType datasetSourceType, INodeDomain domain) {
- this.aqlDataSourceType = datasetSourceType;
+ public AqlDataSourcePartitioningProvider(AqlDataSource dataSource, INodeDomain domain) {
+ this.ds = dataSource;
this.domain = domain;
}
@Override
public IPhysicalPropertiesVector computePropertiesVector(List<LogicalVariable> scanVariables) {
IPhysicalPropertiesVector propsVector = null;
-
- switch (aqlDataSourceType) {
- case EXTERNAL_DATASET: {
- IPartitioningProperty pp = new RandomPartitioningProperty(domain);
- List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
- propsVector = new StructuralPropertiesVector(pp, propsLocal);
- }
-
- case FEED: {
- int n = scanVariables.size();
- IPartitioningProperty pp;
- if (n < 2) {
- pp = new RandomPartitioningProperty(domain);
- } else {
- Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
- int i = 0;
- for (LogicalVariable v : scanVariables) {
- pvars.add(v);
- ++i;
- if (i >= n - 1) {
- break;
- }
- }
- pp = new UnorderedPartitionedProperty(pvars, domain);
- }
- List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+ IPartitioningProperty pp;
+ List<ILocalStructuralProperty> propsLocal;
+ int n;
+ switch (ds.getDatasourceType()) {
+ case ADAPTED_LOADABLE:
+ case EXTERNAL_DATASET:
+ pp = new RandomPartitioningProperty(domain);
+ propsLocal = new ArrayList<ILocalStructuralProperty>();
+ ds.computeLocalStructuralProperties(propsLocal, scanVariables);
propsVector = new StructuralPropertiesVector(pp, propsLocal);
break;
- }
- case INTERNAL_DATASET: {
- int n = scanVariables.size();
- IPartitioningProperty pp;
+ case FEED:
+ n = scanVariables.size();
if (n < 2) {
pp = new RandomPartitioningProperty(domain);
} else {
@@ -165,17 +151,35 @@
}
pp = new UnorderedPartitionedProperty(pvars, domain);
}
- List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+ propsLocal = new ArrayList<ILocalStructuralProperty>();
+ propsVector = new StructuralPropertiesVector(pp, propsLocal);
+ break;
+
+ case INTERNAL_DATASET:
+ n = scanVariables.size();
+ if (n < 2) {
+ pp = new RandomPartitioningProperty(domain);
+ } else {
+ Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
+ int i = 0;
+ for (LogicalVariable v : scanVariables) {
+ pvars.add(v);
+ ++i;
+ if (i >= n - 1) {
+ break;
+ }
+ }
+ pp = new UnorderedPartitionedProperty(pvars, domain);
+ }
+ propsLocal = new ArrayList<ILocalStructuralProperty>();
for (int i = 0; i < n - 1; i++) {
propsLocal.add(new LocalOrderProperty(new OrderColumn(scanVariables.get(i), OrderKind.ASC)));
}
propsVector = new StructuralPropertiesVector(pp, propsLocal);
- }
break;
- default: {
+ default:
throw new IllegalArgumentException();
- }
}
return propsVector;
}
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 c739506..f8a43d0 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
@@ -3,9 +3,9 @@
* 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.
@@ -36,6 +36,7 @@
import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
@@ -64,6 +65,7 @@
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.entities.Feed;
import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.FeedActivity.FeedActivityDetails;
import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
import edu.uci.ics.asterix.metadata.entities.Index;
@@ -120,9 +122,11 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.OperatorSchemaImpl;
import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
import edu.uci.ics.hyracks.algebricks.data.IResultSerializerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -138,6 +142,7 @@
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -151,12 +156,13 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
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.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -296,24 +302,90 @@
JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
try {
switch (((AqlDataSource) dataSource).getDatasourceType()) {
- case FEED:
+ case FEED: {
return buildFeedIntakeRuntime(jobSpec, dataSource);
- case INTERNAL_DATASET:
+ }
+ case INTERNAL_DATASET: {
return buildInternalDatasetScan(jobSpec, scanVariables, minFilterVars, maxFilterVars, opSchema,
typeEnv, dataSource, context, implConfig);
+ }
+ case EXTERNAL_DATASET: {
+ Dataset dataset = ((DatasetDataSource) dataSource).getDataset();
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(),
+ itemTypeName).getDatatype();
+ ExternalDatasetDetails edd = (ExternalDatasetDetails) dataset.getDatasetDetails();
+ IAdapterFactory adapterFactory = getConfiguredAdapterFactory(dataset, edd.getAdapter(),
+ edd.getProperties(), itemType, false, null);
+ return buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory,
+ NonTaggedDataFormat.INSTANCE);
+ }
+ case ADAPTED_LOADABLE: {
+ AdaptedLoadableDataSource alds = (AdaptedLoadableDataSource) dataSource;
- case EXTERNAL_DATASET:
- return buildExternalDatasetScan(jobSpec, dataSource);
+ List<String> partitioningKeys = alds.getPartitioningKeys();
+ boolean isPKAutoGenerated = ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails())
+ .isAutogenerated();
- default:
+ IAdapterFactory wrappedAdapterFactory = getConfiguredAdapterFactory(alds.getTargetDataset(),
+ alds.getAdapter(), alds.getAdapterProperties(), alds.getTargetDatasetType(),
+ isPKAutoGenerated, partitioningKeys);
+ RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+ return buildAdaptedLoadableDatasetScan(jobSpec, alds, wrappedAdapterFactory, rDesc);
+ }
+ default: {
throw new IllegalArgumentException();
+ }
}
- } catch (MetadataException e) {
+ } catch (AsterixException e) {
throw new AlgebricksException(e);
}
}
+ private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildAdaptedLoadableDatasetScan(
+ JobSpecification jobSpec, AdaptedLoadableDataSource alds, IAdapterFactory wrappedAdapterFactory,
+ RecordDescriptor rDesc) throws AlgebricksException {
+ if (!(wrappedAdapterFactory.getSupportedOperations().equals(SupportedOperation.READ) || wrappedAdapterFactory
+ .getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
+ throw new AlgebricksException(" External dataset adapter does not support read operation");
+ }
+ ISerializerDeserializer recSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider().getSerializerDeserializer(
+ alds.getTargetDatasetType());
+ ARecordType recType = (ARecordType) alds.getTargetDatasetType();
+ int[] extractFields = new int[rDesc.getFieldCount() - 1];
+ for (int i = 0; i < extractFields.length; ++i) {
+ try {
+ extractFields[i] = recType.findFieldPosition(alds.getPartitioningKeys().get(i));
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+ IAdapterFactory fieldExtractingAdapterFactory = new FieldExtractingAdapterFactory(wrappedAdapterFactory,
+ new RecordDescriptor(new ISerializerDeserializer[] { recSerde }), rDesc, extractFields,
+ (ARecordType) alds.getTargetDatasetType());
+ ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, rDesc,
+ fieldExtractingAdapterFactory);
+ AlgebricksPartitionConstraint constraint;
+ try {
+ constraint = fieldExtractingAdapterFactory.getPartitionConstraint();
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
+ }
+
+ public IDataFormat getDataFormat(String dataverseName) throws AsterixException {
+ Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ IDataFormat format;
+ try {
+ format = (IDataFormat) Class.forName(dataverse.getDataFormat()).newInstance();
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ return format;
+ }
+
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
List<LogicalVariable> outputVars, List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, IDataSource<AqlSourceId> dataSource,
@@ -347,31 +419,13 @@
implConfig, minFilterFieldIndexes, maxFilterFieldIndexes);
}
- private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
- IDataSource<AqlSourceId> dataSource) throws AlgebricksException, MetadataException {
- Dataset dataset = ((DatasetDataSource) dataSource).getDataset();
- String itemTypeName = dataset.getItemTypeName();
- IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
- .getDatatype();
- return buildExternalDataScannerRuntime(jobSpec, itemType, (ExternalDatasetDetails) dataset.getDatasetDetails(),
- NonTaggedDataFormat.INSTANCE, dataset);
-
- }
-
- @SuppressWarnings("rawtypes")
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataScannerRuntime(
- JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format,
- Dataset dataset) throws AlgebricksException {
- if (itemType.getTypeTag() != ATypeTag.RECORD) {
- throw new AlgebricksException("Can only scan datasets of records.");
- }
-
+ private IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
+ Map<String, String> configuration, IAType itemType, boolean isPKAutoGenerated, List<String> primaryKeys)
+ throws AlgebricksException {
IAdapterFactory adapterFactory;
- String adapterName;
DatasourceAdapter adapterEntity;
String adapterFactoryClassname;
try {
- adapterName = datasetDetails.getAdapter();
adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
adapterName);
if (adapterEntity != null) {
@@ -385,8 +439,6 @@
adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
}
- Map<String, String> configuration = datasetDetails.getProperties();
-
// check to see if dataset is indexed
Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(),
@@ -406,17 +458,25 @@
switch (adapterFactory.getAdapterType()) {
case GENERIC:
- ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType);
+ ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType,
+ isPKAutoGenerated, primaryKeys);
break;
case TYPED:
((ITypedAdapterFactory) adapterFactory).configure(configuration);
break;
}
- } catch (AlgebricksException ae) {
- throw ae;
+ return adapterFactory;
} catch (Exception e) {
throw new AlgebricksException("Unable to create adapter " + e);
}
+ }
+
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
+ JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory, IDataFormat format)
+ throws AlgebricksException {
+ if (itemType.getTypeTag() != ATypeTag.RECORD) {
+ throw new AlgebricksException("Can only scan datasets of records.");
+ }
if (!(adapterFactory.getSupportedOperations().equals(SupportedOperation.READ) || adapterFactory
.getSupportedOperations().equals(SupportedOperation.READ_WRITE))) {
@@ -884,7 +944,7 @@
switch (((AqlDataSource) dataSource).getDatasourceType()) {
case INTERNAL_DATASET:
case EXTERNAL_DATASET:
- result = ((DatasetDataSource) dataSource).getDataset().getDatasetType() == DatasetType.EXTERNAL;
+ result = false;
break;
case FEED:
result = true;
@@ -957,7 +1017,7 @@
// bulkload?)
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
- TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, null,
appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, true,
@@ -966,7 +1026,7 @@
dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
LSMBTreeIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate(), true, filterTypeTraits,
- filterCmpFactories, btreeFields, filterFields), NoOpOperationCallbackFactory.INSTANCE);
+ filterCmpFactories, btreeFields, filterFields));
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
splitsAndConstraint.second);
} catch (MetadataException me) {
@@ -977,9 +1037,10 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> keys, LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields,
- RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
- String datasetName = dataSource.getId().getDatasetName();
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, boolean bulkload)
+ 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 "
@@ -1041,19 +1102,25 @@
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
- AsterixLSMTreeInsertDeleteOperatorDescriptor insertDeleteOp = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(),
- appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
- comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate(), true, filterTypeTraits, filterCmpFactories,
- btreeFields, filterFields), null, modificationCallbackFactory, true, indexName);
-
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
- splitsAndConstraint.second);
+ IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ datasetId), compactionInfo.first, compactionInfo.second, new PrimaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
+ true, filterTypeTraits, filterCmpFactories, btreeFields, filterFields);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, bloomFilterKeyFields, fieldPermutation,
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR, true, numElementsHint, true, idfh);
+ } else {
+ op = new AsterixLSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
+ fieldPermutation, indexOp, idfh, null, modificationCallbackFactory, true, indexName);
+ }
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
} catch (MetadataException me) {
throw new AlgebricksException(me);
@@ -1064,9 +1131,10 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> keys, LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields,
- RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, boolean bulkload)
+ throws AlgebricksException {
return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, typeEnv, keys, payload,
- additionalNonKeyFields, recordDesc, context, spec);
+ additionalNonKeyFields, recordDesc, context, spec, bulkload);
}
@Override
@@ -1075,7 +1143,7 @@
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,
- additionalNonKeyFields, recordDesc, context, spec);
+ additionalNonKeyFields, recordDesc, context, spec, false);
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(
@@ -1083,7 +1151,7 @@
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec) throws AlgebricksException {
+ JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException {
String indexName = dataSourceIndex.getId();
String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
@@ -1104,12 +1172,12 @@
case BTREE: {
return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
- indexOp);
+ indexOp, bulkload);
}
case RTREE: {
return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
- indexOp);
+ indexOp, bulkload);
}
case SINGLE_PARTITION_WORD_INVIX:
case SINGLE_PARTITION_NGRAM_INVIX:
@@ -1117,7 +1185,7 @@
case LENGTH_PARTITIONED_NGRAM_INVIX: {
return getInvertedIndexDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
primaryKeys, secondaryKeys, additionalNonKeyFields, filterFactory, recordDesc, context, spec,
- indexOp, secondaryIndex.getIndexType());
+ indexOp, secondaryIndex.getIndexType(), bulkload);
}
default: {
throw new AlgebricksException("Insert and delete not implemented for index type: "
@@ -1131,10 +1199,245 @@
IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
- ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
- throws AlgebricksException {
+ ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
+ boolean bulkload) throws AlgebricksException {
return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas,
- typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec);
+ typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec,
+ bulkload);
+ }
+
+ @Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getTokenizerRuntime(
+ IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
+ IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+ List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+ JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException {
+
+ String indexName = dataSourceIndex.getId();
+ String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
+ String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+
+ IOperatorSchema inputSchema = new OperatorSchemaImpl();
+ if (inputSchemas.length > 0) {
+ inputSchema = inputSchemas[0];
+ } else {
+ throw new AlgebricksException("TokenizeOperator can not operate without any input variable.");
+ }
+
+ Dataset dataset = findDataset(dataverseName, datasetName);
+ if (dataset == null) {
+ throw new AlgebricksException("Unknown dataset " + datasetName);
+ }
+ Index secondaryIndex;
+ try {
+ secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ }
+ AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+ // TokenizeOperator only supports a keyword or n-gram index.
+ switch (secondaryIndex.getIndexType()) {
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX: {
+ return getBinaryTokenizerRuntime(dataverseName, datasetName, indexName, inputSchema, propagatedSchema,
+ typeEnv, primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec,
+ IndexOperation.INSERT, secondaryIndex.getIndexType(), bulkload);
+ }
+ default: {
+ throw new AlgebricksException("Currently, we do not support TokenizeOperator for the index type: "
+ + secondaryIndex.getIndexType());
+ }
+ }
+
+ }
+
+ // Get a Tokenizer for the bulk-loading data into a n-gram or keyword index.
+ private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBinaryTokenizerRuntime(String dataverseName,
+ String datasetName, String indexName, IOperatorSchema inputSchema, IOperatorSchema propagatedSchema,
+ IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+ AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+ JobSpecification spec, IndexOperation indexOp, IndexType indexType, boolean bulkload)
+ throws AlgebricksException {
+
+ // Sanity checks.
+ if (primaryKeys.size() > 1) {
+ throw new AlgebricksException("Cannot tokenize composite primary key.");
+ }
+ if (secondaryKeys.size() > 1) {
+ throw new AlgebricksException("Cannot tokenize composite secondary key fields.");
+ }
+
+ boolean isPartitioned;
+ if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
+ isPartitioned = true;
+ } else {
+ isPartitioned = false;
+ }
+
+ // Number of Keys that needs to be propagated
+ int numKeys = inputSchema.getSize();
+
+ // Get the rest of Logical Variables that are not (PK or SK) and each variable's positions.
+ // These variables will be propagated through TokenizeOperator.
+ List<LogicalVariable> otherKeys = new ArrayList<LogicalVariable>();
+ if (inputSchema.getSize() > 0) {
+ for (int k = 0; k < inputSchema.getSize(); k++) {
+ boolean found = false;
+ for (LogicalVariable varKey : primaryKeys) {
+ if (varKey.equals(inputSchema.getVariable(k))) {
+ found = true;
+ break;
+ } else {
+ found = false;
+ }
+ }
+ if (!found) {
+ for (LogicalVariable varKey : secondaryKeys) {
+ if (varKey.equals(inputSchema.getVariable(k))) {
+ found = true;
+ break;
+ } else {
+ found = false;
+ }
+ }
+ }
+ if (!found) {
+ otherKeys.add(inputSchema.getVariable(k));
+ }
+ }
+ }
+
+ // For tokenization, sorting and loading.
+ // One token (+ optional partitioning field) + primary keys + secondary keys + other variables
+ // secondary keys and other variables will be just passed to the IndexInsertDelete Operator.
+ int numTokenKeyPairFields = (!isPartitioned) ? 1 + numKeys : 2 + numKeys;
+
+ // generate field permutations for the input
+ int[] fieldPermutation = new int[numKeys];
+
+ int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
+ int i = 0;
+ int j = 0;
+ for (LogicalVariable varKey : primaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ modificationCallbackPrimaryKeyFields[j] = i;
+ i++;
+ j++;
+ }
+ for (LogicalVariable varKey : otherKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ i++;
+ }
+ for (LogicalVariable varKey : secondaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ i++;
+ }
+
+ Dataset dataset = findDataset(dataverseName, datasetName);
+ if (dataset == null) {
+ throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+ }
+ String itemTypeName = dataset.getItemTypeName();
+ IAType itemType;
+ try {
+ itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
+ .getDatatype();
+
+ if (itemType.getTypeTag() != ATypeTag.RECORD) {
+ throw new AlgebricksException("Only record types can be tokenized.");
+ }
+
+ ARecordType recType = (ARecordType) itemType;
+
+ // Index parameters.
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+
+ List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+
+ int numTokenFields = (!isPartitioned) ? secondaryKeys.size() : secondaryKeys.size() + 1;
+ ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
+ ITypeTraits[] invListsTypeTraits = new ITypeTraits[primaryKeys.size()];
+
+ // Find the key type of the secondary key. If it's a derived type, return the derived type.
+ // e.g. UNORDERED LIST -> return UNORDERED LIST type
+ IAType secondaryKeyType = null;
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(0).toString(),
+ recType);
+ secondaryKeyType = keyPairType.first;
+ List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+ i = 0;
+ for (String partitioningKey : partitioningKeys) {
+ IAType keyType = recType.getFieldType(partitioningKey);
+ invListsTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+ ++i;
+ }
+
+ tokenTypeTraits[0] = NonTaggedFormatUtil.getTokenTypeTrait(secondaryKeyType);
+ if (isPartitioned) {
+ // The partitioning field is hardcoded to be a short *without*
+ // an Asterix type tag.
+ tokenTypeTraits[1] = ShortPointable.TYPE_TRAITS;
+ }
+
+ IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
+ secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
+
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
+ dataverseName, datasetName, indexName);
+
+ // Generate Output Record format
+ ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
+ ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
+ ISerializerDeserializerProvider serdeProvider = FormatUtils.getDefaultFormat().getSerdeProvider();
+
+ // The order of the output record: propagated variables (including PK and SK), token, and number of token.
+ // #1. propagate all input variables
+ for (int k = 0; k < recordDesc.getFieldCount(); k++) {
+ tokenKeyPairFields[k] = recordDesc.getFields()[k];
+ tokenKeyPairTypeTraits[k] = recordDesc.getTypeTraits()[k];
+ }
+ int tokenOffset = recordDesc.getFieldCount();
+
+ // #2. Specify the token type
+ tokenKeyPairFields[tokenOffset] = serdeProvider.getSerializerDeserializer(secondaryKeyType);
+ tokenKeyPairTypeTraits[tokenOffset] = tokenTypeTraits[0];
+ tokenOffset++;
+
+ // #3. Specify the length-partitioning key: number of token
+ if (isPartitioned) {
+ tokenKeyPairFields[tokenOffset] = ShortSerializerDeserializer.INSTANCE;
+ tokenKeyPairTypeTraits[tokenOffset] = tokenTypeTraits[1];
+ }
+
+ RecordDescriptor tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields, tokenKeyPairTypeTraits);
+ IOperatorDescriptor tokenizerOp;
+
+ // Keys to be tokenized : SK
+ int docField = fieldPermutation[fieldPermutation.length - 1];
+
+ // Keys to be propagated
+ int[] keyFields = new int[numKeys];
+ for (int k = 0; k < keyFields.length; k++) {
+ keyFields[k] = k;
+ }
+
+ tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
+ keyFields, isPartitioned, true);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(tokenizerOp, splitsAndConstraint.second);
+
+ } catch (MetadataException e) {
+ throw new AlgebricksException(e);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
}
@Override
@@ -1145,7 +1448,8 @@
ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
throws AlgebricksException {
return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas,
- typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec);
+ typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc, context, spec,
+ false);
}
private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -1165,8 +1469,8 @@
String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
- RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp)
- throws AlgebricksException {
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
+ boolean bulkload) throws AlgebricksException {
Dataset dataset = findDataset(dataverseName, datasetName);
if (dataset == null) {
@@ -1264,18 +1568,33 @@
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
- AsterixLSMTreeInsertDeleteOperatorDescriptor btreeBulkLoad = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(),
- appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
- comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
- new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
- compactionInfo.first, compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
- dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
- .getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
- btreeFields, filterFields), filterFactory, modificationCallbackFactory, false, indexName);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
- splitsAndConstraint.second);
+ IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
+ datasetId), compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits, filterCmpFactories,
+ btreeFields, filterFields);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ comparatorFactories, bloomFilterKeyFields, fieldPermutation,
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh);
+ } else {
+ op = new AsterixLSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields,
+ fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+ new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first,
+ compactionInfo.second, new SecondaryIndexOperationTrackerProvider(
+ dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeIOOperationCallbackFactory.INSTANCE,
+ storageProperties.getBloomFilterFalsePositiveRate(), false, filterTypeTraits,
+ filterCmpFactories, btreeFields, filterFields), filterFactory,
+ modificationCallbackFactory, false, indexName);
+ }
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
} catch (MetadataException e) {
throw new AlgebricksException(e);
} catch (IOException e) {
@@ -1288,13 +1607,25 @@
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
- IndexType indexType) throws AlgebricksException {
+ IndexType indexType, boolean bulkload) throws AlgebricksException {
+
+ // Check the index is length-partitioned or not.
+ boolean isPartitioned;
+ if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
+ isPartitioned = true;
+ } else {
+ isPartitioned = false;
+ }
// Sanity checks.
if (primaryKeys.size() > 1) {
throw new AlgebricksException("Cannot create inverted index on dataset with composite primary key.");
}
- if (secondaryKeys.size() > 1) {
+ // The size of secondaryKeys can be two if it receives input from its TokenizeOperator- [token, number of token]
+ if (secondaryKeys.size() > 1 && !isPartitioned) {
+ throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
+ } else if (secondaryKeys.size() > 2 && isPartitioned) {
throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
}
@@ -1303,13 +1634,20 @@
throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
}
+ // For tokenization, sorting and loading.
+ // One token (+ optional partitioning field) + primary keys: [token, number of token, PK]
int numKeys = primaryKeys.size() + secondaryKeys.size();
+ int numTokenKeyPairFields = (!isPartitioned) ? 1 + primaryKeys.size() : 2 + primaryKeys.size();
int numFilterFields = DatasetUtils.getFilterField(dataset) == null ? 0 : 1;
+
// generate field permutations
int[] fieldPermutation = new int[numKeys + numFilterFields];
int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
int i = 0;
int j = 0;
+
+ // If the index is partitioned: [token, number of token]
+ // Otherwise: [token]
for (LogicalVariable varKey : secondaryKeys) {
int idx = propagatedSchema.findVariable(varKey);
fieldPermutation[i] = idx;
@@ -1327,14 +1665,6 @@
fieldPermutation[numKeys] = idx;
}
- boolean isPartitioned;
- if (indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
- isPartitioned = true;
- } else {
- isPartitioned = false;
- }
-
String itemTypeName = dataset.getItemTypeName();
IAType itemType;
try {
@@ -1353,7 +1683,15 @@
List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
- int numTokenFields = (!isPartitioned) ? secondaryKeys.size() : secondaryKeys.size() + 1;
+ int numTokenFields = 0;
+
+ // SecondaryKeys.size() can be two if it comes from the bulkload.
+ // In this case, [token, number of token] are the secondaryKeys.
+ if (!isPartitioned || secondaryKeys.size() > 1)
+ numTokenFields = secondaryKeys.size();
+ else if (isPartitioned && secondaryKeys.size() == 1)
+ numTokenFields = secondaryKeys.size() + 1;
+
ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
ITypeTraits[] invListsTypeTraits = new ITypeTraits[primaryKeys.size()];
IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
@@ -1361,11 +1699,10 @@
dataset, recType, context.getBinaryComparatorFactoryProvider());
IAType secondaryKeyType = null;
- for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(i)
- .toString(), recType);
- secondaryKeyType = keyPairType.first;
- }
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(0).toString(),
+ recType);
+ secondaryKeyType = keyPairType.first;
+
List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
i = 0;
for (String partitioningKey : partitioningKeys) {
@@ -1402,8 +1739,8 @@
}
filterFieldsForNonBulkLoadOps = new int[numFilterFields];
- filterFieldsForNonBulkLoadOps[0] = numKeys;
- invertedIndexFieldsForNonBulkLoadOps = new int[numKeys];
+ filterFieldsForNonBulkLoadOps[0] = numTokenKeyPairFields;
+ invertedIndexFieldsForNonBulkLoadOps = new int[numTokenKeyPairFields];
for (int k = 0; k < invertedIndexFieldsForNonBulkLoadOps.length; k++) {
invertedIndexFieldsForNonBulkLoadOps[k] = k;
}
@@ -1435,21 +1772,29 @@
invertedIndexFieldsForNonBulkLoadOps);
} else {
indexDataFlowFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
- new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
+ compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
storageProperties.getBloomFilterFalsePositiveRate(), invertedIndexFields, filterTypeTraits,
filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
invertedIndexFieldsForNonBulkLoadOps);
}
- AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor insertDeleteOp = new AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
- appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
- invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
- indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
- splitsAndConstraint.second);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new LSMInvertedIndexBulkLoadOperatorDescriptor(spec, recordDesc, fieldPermutation, false,
+ numElementsHint, false, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
+ appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, invListComparatorFactories, tokenizerFactory, indexDataFlowFactory);
+ } else {
+ op = new AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor(spec, recordDesc,
+ appContext.getStorageManagerInterface(), splitsAndConstraint.first,
+ appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+ invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
+ indexDataFlowFactory, filterFactory, modificationCallbackFactory, indexName);
+ }
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
} catch (MetadataException e) {
throw new AlgebricksException(e);
} catch (IOException e) {
@@ -1461,8 +1806,8 @@
String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, AsterixTupleFilterFactory filterFactory,
- RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp)
- throws AlgebricksException {
+ RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, IndexOperation indexOp,
+ boolean bulkload) throws AlgebricksException {
try {
Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
String itemTypeName = dataset.getItemTypeName();
@@ -1555,19 +1900,37 @@
Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils.getMergePolicyFactory(
dataset, mdTxnCtx);
- AsterixLSMTreeInsertDeleteOperatorDescriptor rtreeUpdate = new AsterixLSMTreeInsertDeleteOperatorDescriptor(
- spec, recordDesc, appContext.getStorageManagerInterface(),
- appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
- comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
- valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
- new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), compactionInfo.first,
- compactionInfo.second, new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
- comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate(),
- rtreeFields, btreeFields, filterTypeTraits, filterCmpFactories, filterFields),
- filterFactory, modificationCallbackFactory, false, indexName);
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
+ IIndexDataflowHelperFactory idfh = new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+ RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
+ dataset.getDatasetId()), compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+ proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length),
+ storageProperties.getBloomFilterFalsePositiveRate(), rtreeFields, btreeFields, filterTypeTraits,
+ filterCmpFactories, filterFields);
+ IOperatorDescriptor op;
+ if (bulkload) {
+ long numElementsHint = getCardinalityPerPartitionHint(dataset);
+ op = new TreeIndexBulkLoadOperatorDescriptor(spec, recordDesc, appContext.getStorageManagerInterface(),
+ appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+ primaryComparatorFactories, btreeFields, fieldPermutation,
+ GlobalConfig.DEFAULT_TREE_FILL_FACTOR, false, numElementsHint, false, idfh);
+ } else {
+ op = new AsterixLSMTreeInsertDeleteOperatorDescriptor(spec, recordDesc,
+ appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+ splitsAndConstraint.first, typeTraits, comparatorFactories, null, fieldPermutation, indexOp,
+ new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+ primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset
+ .getDatasetId()), compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(
+ nestedKeyType.getTypeTag(), comparatorFactories.length), storageProperties
+ .getBloomFilterFalsePositiveRate(), rtreeFields, btreeFields, filterTypeTraits,
+ filterCmpFactories, filterFields), filterFactory,
+ modificationCallbackFactory, false, indexName);
+ }
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(op, splitsAndConstraint.second);
} catch (MetadataException | IOException e) {
throw new AlgebricksException(e);
}
@@ -1599,7 +1962,7 @@
* Calculate an estimate size of the bloom filter. Note that this is an
* estimation which assumes that the data is going to be uniformly
* distributed across all partitions.
- *
+ *
* @param dataset
* @return Number of elements that will be used to create a bloom filter per
* dataset per partition
@@ -1822,7 +2185,7 @@
/**
* Add HDFS scheduler and the cluster location constraint into the scheduler
- *
+ *
* @param properties
* the original dataset properties
* @return a new map containing the original dataset properties and the
@@ -1838,7 +2201,7 @@
/**
* Adapt the original properties to a string-object map
- *
+ *
* @param properties
* the original properties
* @return the new stirng-object map
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
index cb1124f..3b5944d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/DatasetDataSource.java
@@ -10,7 +10,9 @@
import edu.uci.ics.asterix.om.types.ARecordType;
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.LogicalVariable;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
public class DatasetDataSource extends AqlDataSource {
@@ -19,7 +21,7 @@
public DatasetDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
AqlDataSourceType datasourceType) throws AlgebricksException {
- super(id, datasourceDataverse, datasourceName, itemType, datasourceType);
+ super(id, datasourceDataverse, datasourceName, datasourceType);
MetadataTransactionContext ctx = null;
try {
ctx = MetadataManager.INSTANCE.beginTransaction();
@@ -95,4 +97,10 @@
return domain;
}
+ @Override
+ public void computeLocalStructuralProperties(List<ILocalStructuralProperty> localProps,
+ List<LogicalVariable> variables) {
+ // do nothing
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
index f729683..695ae31 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FeedDataSource.java
@@ -3,9 +3,9 @@
* 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.
@@ -29,7 +29,7 @@
public FeedDataSource(AqlSourceId id, FeedConnectionId feedId, IAType itemType, AqlDataSourceType dataSourceType)
throws AlgebricksException {
- super(id, feedId.getDataverse(), feedId.getFeedName(), itemType, dataSourceType);
+ super(id, feedId.getDataverse(), feedId.getFeedName(), dataSourceType);
this.feedConnectionId = feedId;
feed = null;
MetadataTransactionContext ctx = null;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java
new file mode 100644
index 0000000..5d30aa5
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java
@@ -0,0 +1,122 @@
+package edu.uci.ics.asterix.metadata.declared;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+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.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public class FieldExtractingAdapter implements IDatasourceAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ private final RecordDescriptor inRecDesc;
+
+ private final RecordDescriptor outRecDesc;
+
+ private final IDatasourceAdapter wrappedAdapter;
+
+ private final FieldExtractingPushRuntime fefw;
+
+ public FieldExtractingAdapter(IHyracksTaskContext ctx, RecordDescriptor inRecDesc, RecordDescriptor outRecDesc,
+ int[] extractFields, ARecordType rType, IDatasourceAdapter wrappedAdapter) {
+ this.inRecDesc = inRecDesc;
+ this.outRecDesc = outRecDesc;
+ this.wrappedAdapter = wrappedAdapter;
+ fefw = new FieldExtractingPushRuntime(ctx, extractFields, rType);
+ }
+
+ @Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ fefw.setInputRecordDescriptor(0, inRecDesc);
+ fefw.setFrameWriter(0, writer, outRecDesc);
+ fefw.open();
+ try {
+ wrappedAdapter.start(partition, fefw);
+ } catch (Throwable t) {
+ fefw.fail();
+ throw t;
+ } finally {
+ fefw.close();
+ }
+ }
+
+ private static class FieldExtractingPushRuntime extends AbstractOneInputOneOutputOneFramePushRuntime {
+
+ private final IHyracksTaskContext ctx;
+
+ private final int[] extractFields;
+
+ private final ARecordType rType;
+
+ private final int nullBitmapSize;
+
+ private final ArrayTupleBuilder tb;
+
+ public FieldExtractingPushRuntime(IHyracksTaskContext ctx, int[] extractFields, ARecordType rType) {
+ this.ctx = ctx;
+ this.extractFields = extractFields;
+ this.rType = rType;
+ nullBitmapSize = ARecordType.computeNullBitmapSize(rType);
+ tb = new ArrayTupleBuilder(extractFields.length + 1);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ initAccessAppendRef(ctx);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ tAccess.reset(buffer);
+ for (int i = 0; i < tAccess.getTupleCount(); ++i) {
+ tb.reset();
+ tRef.reset(tAccess, i);
+ byte[] record = tRef.getFieldData(0);
+ int recStart = tRef.getFieldStart(0);
+ for (int f = 0; f < extractFields.length; ++f) {
+ try {
+ int fOffset = ARecordSerializerDeserializer.getFieldOffsetById(record, recStart,
+ extractFields[f], nullBitmapSize, rType.isOpen());
+ if (fOffset == 0) {
+ tb.getDataOutput().write(ATypeTag.NULL.serialize());
+ } else {
+ IAType fType = rType.getFieldTypes()[extractFields[f]];
+ int fLen;
+ try {
+ fLen = NonTaggedFormatUtil.getFieldValueLength(record, recStart + fOffset,
+ fType.getTypeTag(), false);
+ } catch (AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ tb.getDataOutput().write(fType.getTypeTag().serialize());
+ tb.getDataOutput().write(record, recStart + fOffset, fLen);
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ tb.addFieldEndOffset();
+ }
+ tb.addField(record, recStart, tRef.getFieldLength(0));
+ appendToFrameFromTupleBuilder(tb);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ flushIfNotFailed();
+ }
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java
new file mode 100644
index 0000000..ee615c2
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java
@@ -0,0 +1,59 @@
+package edu.uci.ics.asterix.metadata.declared;
+
+import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class FieldExtractingAdapterFactory implements IAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final IAdapterFactory wrappedAdapterFactory;
+
+ private final RecordDescriptor inRecDesc;
+
+ private final RecordDescriptor outRecDesc;
+
+ private final int[] extractFields;
+
+ private final ARecordType rType;
+
+ public FieldExtractingAdapterFactory(IAdapterFactory wrappedAdapterFactory, RecordDescriptor inRecDesc,
+ RecordDescriptor outRecDesc, int[] extractFields, ARecordType rType) {
+ this.wrappedAdapterFactory = wrappedAdapterFactory;
+ this.inRecDesc = inRecDesc;
+ this.outRecDesc = outRecDesc;
+ this.extractFields = extractFields;
+ this.rType = rType;
+ }
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return wrappedAdapterFactory.getSupportedOperations();
+ }
+
+ @Override
+ public String getName() {
+ return "FieldExtractingAdapter[ " + wrappedAdapterFactory.getName() + " ]";
+ }
+
+ @Override
+ public AdapterType getAdapterType() {
+ return wrappedAdapterFactory.getAdapterType();
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return wrappedAdapterFactory.getPartitionConstraint();
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+ IDatasourceAdapter wrappedAdapter = wrappedAdapterFactory.createAdapter(ctx, partition);
+ return new FieldExtractingAdapter(ctx, inRecDesc, outRecDesc, extractFields, rType, wrappedAdapter);
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
index e13f47a..3617bba 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
@@ -3,9 +3,9 @@
* 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.
@@ -51,6 +51,7 @@
_builtinTypeMap.put("rectangle", BuiltinType.ARECTANGLE);
_builtinTypeMap.put("null", BuiltinType.ANULL);
_builtinTypeMap.put("uuid", BuiltinType.AUUID);
+ _builtinTypeMap.put("shortwithouttypeinfo", BuiltinType.SHORTWITHOUTTYPEINFO);
}
public static Map<String, BuiltinType> getBuiltinTypes() {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
index 62893e2..f834a24 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -47,7 +47,7 @@
dataParser = new ADMDataParser();
break;
case DELIMITED_DATA:
- dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+ dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter, false, -1, null);
break;
}
return new ConditionalPushTupleParser(ctx, recordType, dataParser, configuration);
@@ -99,7 +99,7 @@
public ConditionalPushTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
Map<String, String> configuration) throws HyracksDataException {
- super(ctx, recType);
+ super(ctx, recType, false, -1, null);
this.dataParser = dataParser;
String propValue = (String) configuration.get(BATCH_SIZE);
batchSize = propValue != null ? Integer.parseInt(propValue) : Integer.MAX_VALUE;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
index 1bcb509..52d6abb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
@@ -3,9 +3,9 @@
* 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.
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
+import java.nio.ByteBuffer;
+
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -21,7 +23,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
/*
* A single activity operator that provides the functionality of scanning data using an
@@ -35,7 +37,7 @@
public ExternalDataScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
IAdapterFactory dataSourceAdapterFactory) {
- super(spec, 0, 1);
+ super(spec, 1, 1);
recordDescriptors[0] = rDesc;
this.adapterFactory = dataSourceAdapterFactory;
}
@@ -45,21 +47,35 @@
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
- return new AbstractUnaryOutputSourceOperatorNodePushable() {
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+
@Override
- public void initialize() throws HyracksDataException {
+ public void open() throws HyracksDataException {
writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
IDatasourceAdapter adapter = null;
try {
adapter = adapterFactory.createAdapter(ctx, partition);
adapter.start(partition, writer);
} catch (Exception e) {
throw new HyracksDataException("exception during reading from external data source", e);
- } finally {
- writer.close();
}
+
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ writer.close();
}
};
+
}
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
index 7c25be8..d846003 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedIntakeOperatorDescriptor.java
@@ -3,9 +3,9 @@
* 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.
@@ -174,7 +174,7 @@
+ IGenericAdapterFactory.KEY_TYPE_NAME + " configuration parameter");
}
((IGenericAdapterFactory) adapterFactory).configure(adapterConfiguration,
- (ARecordType) adapterOutputType);
+ (ARecordType) adapterOutputType, false, null);
((IGenericAdapterFactory) adapterFactory).createAdapter(ctx, partition);
}
break;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
index c1f15ba..df23288 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedUtil.java
@@ -3,9 +3,9 @@
* 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.
@@ -299,7 +299,7 @@
throw new Exception("no datatype \"" + outputTypeName + "\" in dataverse \"" + feed.getDataverseName() + "\"");
}
adapterOutputType = (ARecordType) datatype.getDatatype();
- ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType);
+ ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType, false, null);
break;
default:
throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
index 47aa8d9..089b01b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.java
@@ -25,7 +25,7 @@
public static final String KEY_TYPE_NAME = "type-name";
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception;
public void setFiles(List<ExternalFile> files) throws AlgebricksException;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
index d3f063c..5ca940a 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
@@ -3,9 +3,9 @@
* 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.
@@ -142,6 +142,10 @@
AUUIDPrinter.INSTANCE.print(b, s, l, ps);
break;
}
+ case SHORTWITHOUTTYPEINFO: {
+ ShortWithoutTypeInfoPrinter.INSTANCE.print(b, s, l, ps);
+ break;
+ }
default: {
throw new NotImplementedException("No printer for type " + typeTag);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinter.java
new file mode 100644
index 0000000..58398a6
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinter.java
@@ -0,0 +1,43 @@
+/*
+ * 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.dataflow.data.nontagged.printers;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+
+public class ShortWithoutTypeInfoPrinter implements IPrinter {
+
+ public static final ShortWithoutTypeInfoPrinter INSTANCE = new ShortWithoutTypeInfoPrinter();
+
+ @Override
+ public void init() {
+
+ }
+
+ @Override
+ public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+ short d = ShortSerializerDeserializer.getShort(b, s);
+ try {
+ WriteValueTools.writeInt((int)d, ps);
+ } catch (IOException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinterFactory.java
new file mode 100644
index 0000000..7746889
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ShortWithoutTypeInfoPrinterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.dataflow.data.nontagged.printers;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class ShortWithoutTypeInfoPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final ShortWithoutTypeInfoPrinterFactory INSTANCE = new ShortWithoutTypeInfoPrinterFactory();
+
+ @Override
+ public IPrinter createPrinter() {
+ return ShortWithoutTypeInfoPrinter.INSTANCE;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
index a450f27..463c228 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ARecordSerializerDeserializer.java
@@ -3,9 +3,9 @@
* 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.
@@ -212,47 +212,54 @@
return AInt32SerializerDeserializer.getInt(serRecord, offset);
}
- public static final int getFieldOffsetById(byte[] serRecord, int fieldId, int nullBitmapSize, boolean isOpen) {
-
+ public static final int getFieldOffsetById(byte[] serRecord, int offset, int fieldId, int nullBitmapSize,
+ boolean isOpen) {
if (isOpen) {
- if (serRecord[0] == ATypeTag.RECORD.serialize()) {
+ if (serRecord[0 + offset] == ATypeTag.RECORD.serialize()) {
// 5 is the index of the byte that determines whether the record
// is expanded or not, i.e. it has an open part.
- if (serRecord[5] == 1) { // true
+ if (serRecord[5 + offset] == 1) { // true
if (nullBitmapSize > 0) {
// 14 = tag (1) + record Size (4) + isExpanded (1) +
// offset of openPart (4) + number of closed fields (4)
- if ((serRecord[14 + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
+ if ((serRecord[14 + offset + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
// the field value is null
return 0;
}
- return AInt32SerializerDeserializer.getInt(serRecord, (int) (14 + nullBitmapSize + (4 * fieldId)));
+ return AInt32SerializerDeserializer.getInt(serRecord,
+ (int) (14 + offset + nullBitmapSize + (4 * fieldId)));
} else {
if (nullBitmapSize > 0) {
// 9 = tag (1) + record Size (4) + isExpanded (1) +
// number of closed fields (4)
- if ((serRecord[10 + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
+ if ((serRecord[10 + offset + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
// the field value is null
return 0;
}
- return AInt32SerializerDeserializer.getInt(serRecord, (int) (10 + nullBitmapSize + (4 * fieldId)));
+ return AInt32SerializerDeserializer.getInt(serRecord,
+ (int) (10 + offset + nullBitmapSize + (4 * fieldId)));
}
} else
return -1;
} else {
- if (serRecord[0] == ATypeTag.RECORD.serialize()) {
+ if (serRecord[0 + offset] == ATypeTag.RECORD.serialize()) {
if (nullBitmapSize > 0)
// 9 = tag (1) + record Size (4) + number of closed fields
// (4)
- if ((serRecord[9 + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
+ if ((serRecord[9 + offset + fieldId / 8] & (1 << (7 - (fieldId % 8)))) == 0)
// the field value is null
return 0;
- return AInt32SerializerDeserializer.getInt(serRecord, (int) (9 + nullBitmapSize + (4 * fieldId)));
+ return AInt32SerializerDeserializer.getInt(serRecord,
+ (int) (9 + offset + nullBitmapSize + (4 * fieldId)));
} else
return -1;
}
}
+ public static final int getFieldOffsetById(byte[] serRecord, int fieldId, int nullBitmapSize, boolean isOpen) {
+ return getFieldOffsetById(serRecord, 0, fieldId, nullBitmapSize, isOpen);
+ }
+
public static final int getFieldOffsetByName(byte[] serRecord, byte[] fieldName) {
int openPartOffset = 0;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUUIDStringSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUUIDStringSerializerDeserializer.java
new file mode 100644
index 0000000..4a68e51
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AUUIDStringSerializerDeserializer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.base.AUUID;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+
+public class AUUIDStringSerializerDeserializer implements ISerializerDeserializer<AUUID> {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AUUIDStringSerializerDeserializer INSTANCE = new AUUIDStringSerializerDeserializer();
+
+ private AUUIDStringSerializerDeserializer() {
+ }
+
+ // Conversion from the String format (xxxxxxxx-xxxx-xxxx-xxxxxxxxxxxx) to an AUUID instance correspondence
+ @Override
+ public AUUID deserialize(DataInput in) throws HyracksDataException {
+ try {
+ return AUUID.fromString(new String(UTF8StringSerializerDeserializer.INSTANCE.deserialize(in)));
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ // Conversion from an AUUID instance to the String format (xxxxxxxx-xxxx-xxxx-xxxxxxxxxxxx) correspondence
+ @Override
+ public void serialize(AUUID instance, DataOutput out) throws HyracksDataException {
+ try {
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(instance.toStringLiteralOnly(), out);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
index 688f7b3..22ebd99 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
@@ -3,9 +3,9 @@
* 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.
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.formats.nontagged;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUUIDPrinterFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ShortWithoutTypeInfoPrinterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ABooleanPrinterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ACirclePrinterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADatePrinterFactory;
@@ -124,6 +125,8 @@
case UUID: {
return AUUIDPrinterFactory.INSTANCE;
}
+ case SHORTWITHOUTTYPEINFO:
+ return ShortWithoutTypeInfoPrinterFactory.INSTANCE;
}
}
return AObjectPrinterFactory.INSTANCE;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPrinterFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPrinterFactoryProvider.java
index 6774f7d..a306608 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPrinterFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlPrinterFactoryProvider.java
@@ -3,9 +3,9 @@
* 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.
@@ -41,6 +41,7 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUUIDPrinterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUnionPrinterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUnorderedlistPrinterFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ShortWithoutTypeInfoPrinterFactory;
import edu.uci.ics.asterix.om.types.AOrderedListType;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.AUnionType;
@@ -121,6 +122,8 @@
case UUID: {
return AUUIDPrinterFactory.INSTANCE;
}
+ case SHORTWITHOUTTYPEINFO:
+ return ShortWithoutTypeInfoPrinterFactory.INSTANCE;
}
}
return AObjectPrinterFactory.INSTANCE;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
index 0b5e4bc..12de85c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
@@ -3,9 +3,9 @@
* 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.
@@ -44,6 +44,7 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUUIDSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUUIDStringSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
@@ -57,6 +58,7 @@
import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
public class AqlSerializerDeserializerProvider implements ISerializerDeserializerProvider, Serializable {
@@ -165,6 +167,12 @@
case UUID: {
return AUUIDSerializerDeserializer.INSTANCE;
}
+ case UUID_STRING: {
+ return AUUIDStringSerializerDeserializer.INSTANCE;
+ }
+ case SHORTWITHOUTTYPEINFO: {
+ return ShortSerializerDeserializer.INSTANCE;
+ }
default: {
throw new NotImplementedException("No serializer/deserializer implemented for type "
+ aqlType.getTypeTag() + " .");
@@ -181,7 +189,7 @@
@Override
public IAObject deserialize(DataInput in) throws HyracksDataException {
try {
- //deserialize the tag to move the input cursor forward
+ //deserialize the tag to move the input cursor forward
SerializerDeserializerUtil.deserializeTag(in);
} catch (IOException e) {
throw new HyracksDataException(e);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUUID.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUUID.java
index 88c5875..23241e2 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUUID.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableUUID.java
@@ -16,4 +16,24 @@
msb = uuidBits[0];
lsb = uuidBits[1];
}
+
+ // Since AUUID is a wrapper of java.util.uuid,
+ // we can use the same method that creates a UUID from a String.
+ public void fromStringToAMuatbleUUID(String value) {
+ String[] components = value.split("-");
+ if (components.length != 5)
+ throw new IllegalArgumentException("Invalid UUID string: " + value);
+ for (int i = 0; i < 5; i++)
+ components[i] = "0x" + components[i];
+
+ msb = Long.decode(components[0]).longValue();
+ msb <<= 16;
+ msb |= Long.decode(components[1]).longValue();
+ msb <<= 16;
+ msb |= Long.decode(components[2]).longValue();
+
+ lsb = Long.decode(components[3]).longValue();
+ lsb <<= 48;
+ lsb |= Long.decode(components[4]).longValue();
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUUID.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUUID.java
index d8ada69..69dac5d 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUUID.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AUUID.java
@@ -46,6 +46,27 @@
return new AUUID(bits[0], bits[1]);
}
+ public void generateNextRandomUUID() {
+ byte[] randomBytes = new byte[16];
+ Holder.srnd.nextBytes(randomBytes);
+ uuidBitsFromBytes(randomBytes);
+ }
+
+ protected void uuidBitsFromBytes(byte[] randomBytes) {
+ this.msb = 0;
+ this.lsb = 0;
+ randomBytes[6] &= 0x0f; /* clear version */
+ randomBytes[6] |= 0x40; /* set to version 4 */
+ randomBytes[8] &= 0x3f; /* clear variant */
+ randomBytes[8] |= 0x80; /* set to IETF variant */
+ for (int i = 0; i < 8; ++i) {
+ this.msb = (this.msb << 8) | (randomBytes[i] & 0xff);
+ }
+ for (int i = 8; i < 16; ++i) {
+ this.lsb = (this.lsb << 8) | (randomBytes[i] & 0xff);
+ }
+ }
+
protected static void uuidBitsFromBytes(long[] bits, byte[] randomBytes) {
bits[0] = 0;
bits[1] = 0;
@@ -100,6 +121,33 @@
+ digits(lsb >> 48, 4) + "-" + digits(lsb, 12)) + "}";
}
+ public String toStringLiteralOnly() {
+ return digits(msb >> 32, 8) + "-" + digits(msb >> 16, 4) + "-" + digits(msb, 4) + "-" + digits(lsb >> 48, 4)
+ + "-" + digits(lsb, 12);
+ }
+
+ // Since AUUID is a wrapper of java.util.uuid,
+ // we can use the same method that creates a UUID from a String.
+ public static AUUID fromString(String name) {
+ String[] components = name.split("-");
+ if (components.length != 5)
+ throw new IllegalArgumentException("Invalid UUID string: " + name);
+ for (int i = 0; i < 5; i++)
+ components[i] = "0x" + components[i];
+
+ long msb = Long.decode(components[0]).longValue();
+ msb <<= 16;
+ msb |= Long.decode(components[1]).longValue();
+ msb <<= 16;
+ msb |= Long.decode(components[2]).longValue();
+
+ long lsb = Long.decode(components[3]).longValue();
+ lsb <<= 48;
+ lsb |= Long.decode(components[4]).longValue();
+
+ return new AUUID(msb, lsb);
+ }
+
private static String digits(long val, int digits) {
long hi = 1L << (digits * 4);
return Long.toHexString(hi | (val & (hi - 1))).substring(1);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ShortWithoutTypeInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ShortWithoutTypeInfo.java
new file mode 100644
index 0000000..8957263
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ShortWithoutTypeInfo.java
@@ -0,0 +1,97 @@
+/*
+ * 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.om.base;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.visitors.IOMVisitor;
+
+public class ShortWithoutTypeInfo implements IAObject {
+
+ protected short value;
+
+ public ShortWithoutTypeInfo(short value) {
+ super();
+ this.value = value;
+ }
+
+ public ShortWithoutTypeInfo(byte[] bytes, int offset, int length) {
+ value = valueFromBytes(bytes, offset, length);
+ }
+
+ public Short getShortValue() {
+ return value;
+ }
+
+ @Override
+ public IAType getType() {
+ return BuiltinType.SHORTWITHOUTTYPEINFO;
+ }
+
+ @Override
+ public String toString() {
+ return "ShortWithoutTypeInfo: {" + value + "}";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ShortWithoutTypeInfo)) {
+ return false;
+ } else {
+ return value == (((ShortWithoutTypeInfo) obj).getShortValue());
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return value;
+ }
+
+ private static short valueFromBytes(byte[] bytes, int offset, int length) {
+ return (short) (((bytes[offset] & 0xff) << 8) + ((bytes[offset + 1] & 0xff)));
+ }
+
+ public byte[] toBytes() {
+ return new byte[] { (byte) ((value >>> 8) & 0xff), (byte) ((value >>> 0) & 0xff)};
+ }
+
+ @Override
+ public void accept(IOMVisitor visitor) throws AsterixException {
+ visitor.visitShortWithoutTypeInfo(this);
+ }
+
+ @Override
+ public boolean deepEqual(IAObject obj) {
+ return equals(obj);
+ }
+
+ @Override
+ public int hash() {
+ return hashCode();
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("ShortWithoutTypeInfo", value);
+
+ return json;
+ }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
index 38e9a11..15de81c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
@@ -3,9 +3,9 @@
* 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.
@@ -43,6 +43,7 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ATimePrinter;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUUIDPrinter;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AYearMonthDurationPrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ShortWithoutTypeInfoPrinter;
import edu.uci.ics.asterix.om.pointables.AFlatValuePointable;
import edu.uci.ics.asterix.om.pointables.AListPointable;
import edu.uci.ics.asterix.om.pointables.ARecordPointable;
@@ -193,6 +194,10 @@
AUUIDPrinter.INSTANCE.print(b, s, l, ps);
break;
}
+ case SHORTWITHOUTTYPEINFO: {
+ ShortWithoutTypeInfoPrinter.INSTANCE.print(b, s, l, ps);
+ break;
+ }
default: {
throw new NotImplementedException("No printer for type " + typeTag);
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
index 4b3584b..24e8ba5 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
@@ -21,6 +21,7 @@
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AUUIDPrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ShortWithoutTypeInfoPrinter;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ABooleanPrinter;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ACirclePrinter;
import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADatePrinter;
@@ -188,6 +189,10 @@
AUUIDPrinter.INSTANCE.print(b, s, l, ps);
break;
}
+ case SHORTWITHOUTTYPEINFO: {
+ ShortWithoutTypeInfoPrinter.INSTANCE.print(b, s, l, ps);
+ break;
+ }
default: {
throw new NotImplementedException("No printer for type " + typeTag);
}
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 d444f31..493709e 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
@@ -3,9 +3,9 @@
* 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.
@@ -30,6 +30,7 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.asterix.om.visitors.IOMVisitor;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -126,7 +127,7 @@
/**
* Returns the position of the field in the closed schema or -1 if the field does not exist.
- *
+ *
* @param bytes
* the serialized bytes of the field name
* @param start
@@ -204,7 +205,7 @@
/**
* Returns the position of the field in the closed schema or -1 if the field does not exist.
- *
+ *
* @param fieldName
* the name of the field whose position is sought
* @return the position of the field in the closed schema or -1 if the field does not exist.
@@ -218,7 +219,7 @@
/**
* Returns the field type of the field name if it exists, otherwise null.
- *
+ *
* @param fieldName
* the fieldName whose type is sought
* @return the field type of the field name if it exists, otherwise null
@@ -235,7 +236,7 @@
/**
* Returns true or false indicating whether or not a field is closed.
- *
+ *
* @param fieldName
* the name of the field to check
* @return true if fieldName is a closed field, otherwise false
@@ -247,7 +248,7 @@
/**
* Validates the partitioning expression that will be used to partition a dataset.
- *
+ *
* @param partitioningExprs
* a list of partitioning expressions that will be validated
* @throws AlgebricksException
@@ -261,7 +262,7 @@
}
String fieldName = partitioningExprs.get(0);
- IAType fieldType = getPartitioningExpressionType(fieldName);
+ IAType fieldType = getPartitioningExpressionType(fieldName, autogenerated);
ATypeTag pkTypeTag = fieldType.getTypeTag();
if (pkTypeTag != ATypeTag.UUID) {
throw new AsterixException("Cannot autogenerate a primary key for type " + pkTypeTag
@@ -269,7 +270,7 @@
}
} else {
for (String fieldName : partitioningExprs) {
- IAType fieldType = getPartitioningExpressionType(fieldName);
+ IAType fieldType = getPartitioningExpressionType(fieldName, autogenerated);
switch (fieldType.getTypeTag()) {
case INT8:
case INT16:
@@ -295,17 +296,22 @@
}
}
- private IAType getPartitioningExpressionType(String fieldName) throws AsterixException, IOException {
+ private IAType getPartitioningExpressionType(String fieldName, boolean autogenerated) throws AsterixException,
+ IOException {
IAType fieldType = getFieldType(fieldName);
if (fieldType == null) {
- throw new AsterixException("Primary key field \"" + fieldName + "\" could not be found.");
+ if (autogenerated) {
+ throw new AsterixException("Primary key field: " + fieldName + " should be defined in the type that the dataset is using.");
+ } else {
+ throw new AsterixException("Primary key field: " + fieldName + " could not be found.");
+ }
}
return fieldType;
}
/**
* Validates the key fields that will be used as keys of an index.
- *
+ *
* @param keyFieldNames
* a list of key fields that will be validated
* @param indexType
@@ -411,7 +417,7 @@
/**
* 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
@@ -512,4 +518,8 @@
type.put("fields", fields);
return type;
}
+
+ public static int computeNullBitmapSize(ARecordType rt) {
+ return NonTaggedFormatUtil.hasNullableField(rt) ? (int) Math.ceil(rt.getFieldNames().length / 8.0) : 0;
+ }
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
index ffbbd64..360fa7f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
@@ -3,9 +3,9 @@
* 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.
@@ -21,7 +21,7 @@
/**
* There is a unique tag for each primitive type and for each kind of
* non-primitive type in the object model.
- *
+ *
* @author Nicola
*/
public enum ATypeTag implements IEnumSerializer {
@@ -62,7 +62,9 @@
SYSTEM_NULL(35),
YEARMONTHDURATION(36),
DAYTIMEDURATION(37),
- UUID(38);
+ UUID(38),
+ UUID_STRING(39),
+ SHORTWITHOUTTYPEINFO(40);
private byte value;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
index 97fb729..ca13f39 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
@@ -3,9 +3,9 @@
* 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.
@@ -738,6 +738,33 @@
}
};
+ // AUUID_STRING is used when converting between the string representation of
+ // UUID and corresponding a UUID instance
+ public static final BuiltinType AUUID_STRING = new LowerCaseConstructorType() {
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.UUID_STRING;
+ }
+
+ @Override
+ public String getDisplayName() {
+ return "UUID_STRING";
+ }
+
+ @Override
+ public String getTypeName() {
+ return "uuid_string";
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", getDisplayName());
+ return type;
+ }
+ };
+
public static final IAType ANY = new BuiltinType() {
private static final long serialVersionUID = 1L;
@@ -770,6 +797,38 @@
}
};
+ public static final BuiltinType SHORTWITHOUTTYPEINFO = new BuiltinType() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.SHORTWITHOUTTYPEINFO;
+ }
+
+ @Override
+ public String getTypeName() {
+ return "shortwithouttypeinfo";
+ }
+
+ @Override
+ public String getDisplayName() {
+ return "SHORTWITHOUTTYPEINFO";
+ }
+
+ @Override
+ public String getConstructor() {
+ return null;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject type = new JSONObject();
+ type.put("type", "SHORTWITHOUTTYPEINFO");
+ return type;
+ }
+ };
+
public abstract String getConstructor();
@Override
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
index e3d87d1..b16fddb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
@@ -3,9 +3,9 @@
* 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.
@@ -25,11 +25,9 @@
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.AUnionType;
-import edu.uci.ics.asterix.om.types.AUnorderedListType;
import edu.uci.ics.asterix.om.types.AbstractCollectionType;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.asterix.om.types.EnumDeserializer;
@@ -207,39 +205,34 @@
}
}
- public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
+ public static IAType getTokenType(IAType keyType)
+ throws AlgebricksException {
IAType type = keyType;
ATypeTag typeTag = keyType.getTypeTag();
// Extract item type from list.
- if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
+ if (typeTag == ATypeTag.UNORDEREDLIST
+ || typeTag == ATypeTag.ORDEREDLIST) {
AbstractCollectionType listType = (AbstractCollectionType) keyType;
if (!listType.isTyped()) {
- throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+ throw new AlgebricksException(
+ "Cannot build an inverted index on untyped lists.)");
}
type = listType.getItemType();
}
- // Ignore case for string types.
- return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
+ return type;
}
- public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
- IAType type = keyType;
- ATypeTag typeTag = keyType.getTypeTag();
- // Extract item type from list.
- if (typeTag == ATypeTag.UNORDEREDLIST) {
- AUnorderedListType ulistType = (AUnorderedListType) keyType;
- if (!ulistType.isTyped()) {
- throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
- }
- type = ulistType.getItemType();
- }
- if (typeTag == ATypeTag.ORDEREDLIST) {
- AOrderedListType olistType = (AOrderedListType) keyType;
- if (!olistType.isTyped()) {
- throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
- }
- type = olistType.getItemType();
- }
+ public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(
+ IAType keyType) throws AlgebricksException {
+ IAType type = getTokenType(keyType);
+ // Ignore case for string types.
+ return AqlBinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(type, true, true);
+ }
+
+ public static ITypeTraits getTokenTypeTrait(IAType keyType)
+ throws AlgebricksException {
+ IAType type = getTokenType(keyType);
return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
}
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
index 8952651..15ca909 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
@@ -3,9 +3,9 @@
* 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.
@@ -43,6 +43,7 @@
import edu.uci.ics.asterix.om.base.AUUID;
import edu.uci.ics.asterix.om.base.AUnorderedList;
import edu.uci.ics.asterix.om.base.AYearMonthDuration;
+import edu.uci.ics.asterix.om.base.ShortWithoutTypeInfo;
import edu.uci.ics.asterix.om.types.IAType;
public interface IOMVisitor {
@@ -103,4 +104,7 @@
public void visitARectangle(ARectangle obj) throws AsterixException;
public void visitAUUID(AUUID obj) throws AsterixException;
+
+ public void visitShortWithoutTypeInfo(ShortWithoutTypeInfo obj) throws AsterixException;
+
}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
index 34b9f83..073fb0f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
@@ -3,9 +3,9 @@
* 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.
@@ -46,6 +46,7 @@
import edu.uci.ics.asterix.om.base.AYearMonthDuration;
import edu.uci.ics.asterix.om.base.IACursor;
import edu.uci.ics.asterix.om.base.IAObject;
+import edu.uci.ics.asterix.om.base.ShortWithoutTypeInfo;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -134,6 +135,11 @@
}
@Override
+ public void visitShortWithoutTypeInfo(ShortWithoutTypeInfo obj) throws AsterixException {
+ buffer.append(obj.getShortValue());
+ }
+
+ @Override
public void visitAInt8(AInt8 obj) throws AsterixException {
buffer.append(obj.getByteValue());
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FieldAccessByIndexEvalFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FieldAccessByIndexEvalFactory.java
index 20bd366..207f3af 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FieldAccessByIndexEvalFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FieldAccessByIndexEvalFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -54,10 +54,7 @@
this.recordEvalFactory = recordEvalFactory;
this.fieldIndexEvalFactory = fieldIndexEvalFactory;
this.recordType = recordType;
- if (NonTaggedFormatUtil.hasNullableField(recordType))
- this.nullBitmapSize = (int) Math.ceil(recordType.getFieldNames().length / 8.0);
- else
- this.nullBitmapSize = 0;
+ this.nullBitmapSize = ARecordType.computeNullBitmapSize(recordType);
}
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 ed0a3f3..b477545 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
@@ -3,9 +3,9 @@
* 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.
@@ -946,9 +946,9 @@
}
fieldParserFactories[i] = vpf;
}
- return new NtDelimitedDataTupleParserFactory(recType, fieldParserFactories, delimiter);
+ return new NtDelimitedDataTupleParserFactory(recType, fieldParserFactories, delimiter, false, -1, null);
} else {
- return new AdmSchemafullRecordParserFactory(recType);
+ return new AdmSchemafullRecordParserFactory(recType, false, -1, null);
}
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index 1c06604..0cd3098 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayDeque;
+import java.util.ArrayList;
import java.util.BitSet;
import java.util.List;
import java.util.Queue;
@@ -31,7 +32,9 @@
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.AMutableUUID;
import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AUUID;
import edu.uci.ics.asterix.om.types.AOrderedListType;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -45,6 +48,7 @@
import edu.uci.ics.asterix.runtime.operators.file.adm.AdmLexerException;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
/**
* Parser for ADM formatted data.
@@ -66,6 +70,11 @@
private String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
private String mismatchErrorMessage2 = " got a value of type ";
+ private boolean isPKAutoGenerated;
+ private int primaryKeyPosition;
+ private ARecordType origRecTypeForAutoGeneratedPK;
+ private byte AUUIDTag = ATypeTag.UUID.serialize();
+
static class ParseException extends AsterixException {
private static final long serialVersionUID = 1L;
private String filename;
@@ -112,11 +121,16 @@
}
public ADMDataParser() {
- this(null);
+ this(null, false, -1, null);
}
- public ADMDataParser(String filename) {
+ // Constructor for dealing with auto-generated PK
+ public ADMDataParser(String filename, boolean isPKAutoGenerated,
+ int primaryKeyPosition, ARecordType origRecTypeForAutoGeneratedPK) {
this.filename = filename;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyPosition;
+ this.origRecTypeForAutoGeneratedPK = origRecTypeForAutoGeneratedPK;
}
@Override
@@ -135,7 +149,12 @@
@Override
public void initialize(InputStream in, ARecordType recordType, boolean datasetRec) throws AsterixException {
- this.recordType = recordType;
+ // Use original record type variable - origRecTypeForAutoGeneratedPK if we have auto-generated PK.
+ // The recordType variable does not contain field information about auto-generated PK.
+ if (!isPKAutoGenerated)
+ this.recordType = recordType;
+ else
+ this.recordType = origRecTypeForAutoGeneratedPK;
this.datasetRec = datasetRec;
try {
admLexer = new AdmLexer(new java.io.InputStreamReader(in));
@@ -242,6 +261,12 @@
admLexer.getLastTokenImage().length() - 1);
aString.setValue(admLexer.containsEscapes() ? replaceEscapes(tokenImage) : tokenImage);
stringSerde.serialize(aString, out);
+ } else if (checkType(ATypeTag.UUID, objectType)) {
+ // Dealing with UUID type that is represented by a string
+ final String tokenImage = admLexer.getLastTokenImage().substring(1,
+ admLexer.getLastTokenImage().length() - 1);
+ aUUID.fromStringToAMuatbleUUID(tokenImage);
+ uuidSerde.serialize(aUUID, out);
} else
throw new ParseException(mismatchErrorMessage + objectType.getTypeName());
break;
@@ -365,6 +390,10 @@
throw new ParseException(mismatchErrorMessage + objectType.getTypeTag());
break;
}
+ case AdmLexer.TOKEN_UUID_CONS: {
+ parseConstructor(ATypeTag.UUID, objectType, out);
+ break;
+ }
case AdmLexer.TOKEN_EOF: {
break;
}
@@ -585,9 +614,28 @@
} while (inRecord);
if (recType != null) {
- nullableFieldId = checkNullConstraints(recType, nulls);
+ nullableFieldId = checkNullConstraints(recType, nulls, 0);
+ boolean insertedAutoGeneratedPK = false;
+
+ // This is a typical situation for a dataset with an auto-generated PK
+ // where nullableFieldId equals primaryKey field.
+ // In this case, we create a random ID and assign it as a PK
+ if (isPKAutoGenerated && nullableFieldId == primaryKeyPosition) {
+ fieldValueBuffer.reset();
+ aUUID.nextUUID();
+ fieldValueBuffer.getDataOutput().writeByte(AUUIDTag);
+ Integer64SerializerDeserializer.INSTANCE.serialize(aUUID.getMostSignificantBits(), fieldValueBuffer.getDataOutput());
+ Integer64SerializerDeserializer.INSTANCE.serialize(aUUID.getLeastSignificantBits(), fieldValueBuffer.getDataOutput());
+ recBuilder.addField(primaryKeyPosition, fieldValueBuffer);
+ insertedAutoGeneratedPK = true;
+ nulls.set(nullableFieldId);
+ // Check from the primary key field position to make sure no other field is missing
+ nullableFieldId = checkNullConstraints(recType, nulls, nullableFieldId);
+ }
if (nullableFieldId != -1)
- throw new ParseException("Field " + recType.getFieldNames()[nullableFieldId] + " can not be null");
+ throw new ParseException("Field: " + recType.getFieldNames()[nullableFieldId] + " can not be null");
+ if (isPKAutoGenerated && !insertedAutoGeneratedPK)
+ throw new ParseException("Auto-generated PK Field: " + recType.getFieldNames()[primaryKeyPosition] + " should not exist in the ADM file.");
}
recBuilder.write(out, true);
returnRecordBuilder(recBuilder);
@@ -595,9 +643,9 @@
returnTempBuffer(fieldValueBuffer);
}
- private int checkNullConstraints(ARecordType recType, BitSet nulls) {
+ private int checkNullConstraints(ARecordType recType, BitSet nulls, int startingPosition) {
boolean isNull = false;
- for (int i = 0; i < recType.getFieldTypes().length; i++)
+ for (int i = startingPosition; i < recType.getFieldTypes().length; i++)
if (nulls.get(i) == false) {
IAType type = recType.getFieldTypes()[i];
if (type.getTypeTag() != ATypeTag.NULL && type.getTypeTag() != ATypeTag.UNION)
@@ -892,6 +940,10 @@
case POLYGON:
APolygonSerializerDeserializer.parse(unquoted, out);
return true;
+ case UUID:
+ aUUID.fromStringToAMuatbleUUID(unquoted);
+ uuidSerde.serialize(aUUID, out);
+ return true;
default:
return false;
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractDataParser.java
index 9820a3a..e7ae93d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractDataParser.java
@@ -47,6 +47,7 @@
import edu.uci.ics.asterix.om.base.AMutableRectangle;
import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.AMutableTime;
+import edu.uci.ics.asterix.om.base.AMutableUUID;
import edu.uci.ics.asterix.om.base.AMutableYearMonthDuration;
import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.base.APoint;
@@ -54,6 +55,7 @@
import edu.uci.ics.asterix.om.base.ARectangle;
import edu.uci.ics.asterix.om.base.AString;
import edu.uci.ics.asterix.om.base.ATime;
+import edu.uci.ics.asterix.om.base.AUUID;
import edu.uci.ics.asterix.om.base.AYearMonthDuration;
import edu.uci.ics.asterix.om.base.temporal.ADateParserFactory;
import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
@@ -62,8 +64,10 @@
import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
import edu.uci.ics.asterix.om.types.BuiltinType;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
/**
* Base class for data parsers. Includes the common set of definitions for
@@ -79,6 +83,7 @@
protected AMutableFloat aFloat = new AMutableFloat(0);
protected AMutableString aString = new AMutableString("");
protected AMutableString aStringFieldName = new AMutableString("");
+ protected AMutableUUID aUUID = new AMutableUUID(0, 0);
// For temporal and spatial data types
protected AMutableTime aTime = new AMutableTime(0);
protected AMutableDateTime aDateTime = new AMutableDateTime(0L);
@@ -121,6 +126,11 @@
@SuppressWarnings("unchecked")
protected ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.ANULL);
+ // For UUID, we assume that the format is the string representation of UUID
+ // (xxxxxxxx-xxxx-xxxx-xxxxxxxxxxxx) when parsing the data.
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<AUUID> uuidSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AUUID_STRING);
// To avoid race conditions, the serdes for temporal and spatial data types needs to be one per parser
@SuppressWarnings("unchecked")
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 13305b1..e535cf4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -47,12 +47,19 @@
protected final ARecordType recType;
protected final IHyracksTaskContext ctx;
protected String filename;
+ protected boolean isPKAutoGenerated;
+ protected int primaryKeyPosition;
+ protected final ARecordType origRecTypeForAutoGeneratedPK;
- public AbstractTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
+ public AbstractTupleParser(IHyracksTaskContext ctx, ARecordType recType,
+ boolean isPKAutoGenerated, int primaryKeyPosition, ARecordType origRecTypeForAutoGeneratedPK) throws HyracksDataException {
appender = new FrameTupleAppender(ctx.getFrameSize());
frame = ctx.allocateFrame();
this.recType = recType;
this.ctx = ctx;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyPosition;
+ this.origRecTypeForAutoGeneratedPK = origRecTypeForAutoGeneratedPK;
}
public void setFilename(String filename) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java
index 5c84fe4..149fd0a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -30,13 +30,22 @@
protected ARecordType recType;
- public AdmSchemafullRecordParserFactory(ARecordType recType) {
+ // To deal with an auto-generated PK field
+ protected boolean isPKAutoGenerated;
+ protected int primaryKeyPosition;
+ protected ARecordType origRecordTypeForAutoGeneratedPK;
+
+ public AdmSchemafullRecordParserFactory(ARecordType recType, boolean isPKAutoGenerated, int primaryKeyPosition,
+ ARecordType origRecordTypeForAutoGeneratedPK) {
this.recType = recType;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyPosition;
+ this.origRecordTypeForAutoGeneratedPK = origRecordTypeForAutoGeneratedPK;
}
@Override
public ITupleParser createTupleParser(final IHyracksTaskContext ctx) throws HyracksDataException {
- return new AdmTupleParser(ctx, recType);
+ return new AdmTupleParser(ctx, recType, isPKAutoGenerated, primaryKeyPosition, origRecordTypeForAutoGeneratedPK);
}
}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
index 8aab2db..074a2ca 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
@@ -24,13 +24,14 @@
*/
public class AdmTupleParser extends AbstractTupleParser {
- public AdmTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
- super(ctx, recType);
+ public AdmTupleParser(IHyracksTaskContext ctx, ARecordType recType, boolean isPKAutoGenerated,
+ int primaryKeyPosition, ARecordType origRecordTypeForAutoGeneratedPK) throws HyracksDataException {
+ super(ctx, recType, isPKAutoGenerated, primaryKeyPosition, origRecordTypeForAutoGeneratedPK);
}
@Override
public IDataParser getDataParser() {
- return new ADMDataParser(filename);
+ return new ADMDataParser(filename, isPKAutoGenerated, primaryKeyPosition, origRecTypeForAutoGeneratedPK);
}
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 23f7aab..f516075 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -3,9 +3,9 @@
* 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.
@@ -33,6 +33,7 @@
import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParser;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -50,19 +51,35 @@
private byte[] fieldTypeTags;
private int[] fldIds;
private ArrayBackedValueStorage[] nameBuffers;
+ private byte AUUIDTag = ATypeTag.UUID.serialize();
+
+ // Variables used to set a UUID for the auto-generated PK field
+ private boolean isPKAutoGenerated;
+ private int primaryKeyPosition;
+ private final ARecordType origRecordTypeForAutoGeneratedPK;
private boolean areAllNullFields;
- public DelimitedDataParser(ARecordType recordType, IValueParserFactory[] valueParserFactories, char fieldDelimter) {
+ public DelimitedDataParser(ARecordType recordType, IValueParserFactory[] valueParserFactories, char fieldDelimter,
+ boolean isPKAutoGenerated, int primaryKeyPosition, ARecordType origRecordTypeForAutoGeneratedPK) {
this.recordType = recordType;
this.valueParserFactories = valueParserFactories;
this.fieldDelimiter = fieldDelimter;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyPosition;
+ this.origRecordTypeForAutoGeneratedPK = origRecordTypeForAutoGeneratedPK;
}
@Override
public void initialize(InputStream in, ARecordType recordType, boolean datasetRec) throws AsterixException,
IOException {
+ ARecordType recordTypeToApply = null;
+ if (isPKAutoGenerated)
+ recordTypeToApply = origRecordTypeForAutoGeneratedPK;
+ else
+ recordTypeToApply = recordType;
+
valueParsers = new IValueParser[valueParserFactories.length];
for (int i = 0; i < valueParserFactories.length; ++i) {
valueParsers[i] = valueParserFactories[i].createValueParser();
@@ -70,8 +87,12 @@
fieldValueBuffer = new ArrayBackedValueStorage();
fieldValueBufferOutput = fieldValueBuffer.getDataOutput();
+
+ // If PK is auto-generated, then we need to use the recordType that
+ // includes PK,
+ // since recordType variable does not include PK field.
recBuilder = new RecordBuilder();
- recBuilder.reset(recordType);
+ recBuilder.reset(recordTypeToApply);
recBuilder.init();
int n = recordType.getFieldNames().length;
@@ -104,9 +125,16 @@
@Override
public boolean parse(DataOutput out) throws AsterixException, IOException {
while (cursor.nextRecord()) {
- recBuilder.reset(recordType);
+ if (isPKAutoGenerated)
+ recBuilder.reset(origRecordTypeForAutoGeneratedPK);
+ else
+ recBuilder.reset(recordType);
+
recBuilder.init();
areAllNullFields = true;
+
+ int fieldCount = 0;
+
for (int i = 0; i < valueParsers.length; ++i) {
if (!cursor.nextField()) {
break;
@@ -115,11 +143,14 @@
if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
&& recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
- // if the field is empty and the type is optional, insert NULL
- // note that string type can also process empty field as an empty string
+ // if the field is empty and the type is optional, insert
+ // NULL
+ // note that string type can also process empty field as an
+ // empty string
if (recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.UNION
|| !NonTaggedFormatUtil.isOptionalField((AUnionType) recordType.getFieldTypes()[i])) {
- throw new AsterixException("Field " + i + " is not an optional type so it cannot accept null value. ");
+ throw new AsterixException("Field " + i
+ + " is not an optional type so it cannot accept null value. ");
}
fieldValueBufferOutput.writeByte(ATypeTag.NULL.serialize());
ANullSerializerDeserializer.INSTANCE.serialize(ANull.NULL, out);
@@ -135,7 +166,35 @@
recBuilder.addField(fldIds[i], fieldValueBuffer);
}
+ fieldCount++;
+
}
+
+ // Should not have any more fields now
+ if (cursor.nextField()) {
+ fieldCount++;
+ }
+
+ // Parsed all fields except an auto-generated PK at this point
+ // Create a new UUID and assign it as a PK.
+ if (isPKAutoGenerated && fieldCount == origRecordTypeForAutoGeneratedPK.getFieldTypes().length - 1) {
+ fieldValueBuffer.reset();
+ aUUID.nextUUID();
+ fieldValueBufferOutput.writeByte(AUUIDTag);
+ Integer64SerializerDeserializer.INSTANCE.serialize(aUUID.getMostSignificantBits(),
+ fieldValueBufferOutput);
+ Integer64SerializerDeserializer.INSTANCE.serialize(aUUID.getLeastSignificantBits(),
+ fieldValueBufferOutput);
+ recBuilder.addField(primaryKeyPosition, fieldValueBuffer);
+ areAllNullFields = false;
+ }
+ // If we have all fields in the file including auto-generated PK,
+ // throw an exception
+ else if (isPKAutoGenerated && fieldCount >= origRecordTypeForAutoGeneratedPK.getFieldTypes().length) {
+ throw new AsterixException(
+ "Check number of fields. Auto-generated PK field should not exist in the input data.");
+ }
+
if (!areAllNullFields) {
recBuilder.write(out, true);
return true;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java
index 9f5e6e9..be6c42a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java
@@ -3,9 +3,9 @@
* 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.
@@ -28,9 +28,11 @@
private final DelimitedDataParser dataParser;
public DelimitedDataTupleParser(IHyracksTaskContext ctx, ARecordType recType,
- IValueParserFactory[] valueParserFactories, char fieldDelimter) throws HyracksDataException {
- super(ctx, recType);
- dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter);
+ IValueParserFactory[] valueParserFactories, char fieldDelimter, boolean isPKAutoGenerated,
+ int primaryKeyPosition, ARecordType origRecordTypeForAutoGeneratedPK) throws HyracksDataException {
+ super(ctx, recType, isPKAutoGenerated, primaryKeyPosition, origRecordTypeForAutoGeneratedPK);
+ dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter, isPKAutoGenerated,
+ primaryKeyPosition, origRecordTypeForAutoGeneratedPK);
}
@Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java
index 6c56094..f88d39a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -30,17 +30,26 @@
protected ARecordType recordType;
protected IValueParserFactory[] valueParserFactories;
protected char fieldDelimiter;
+ // To deal with an auto-generated PK
+ protected boolean isPKAutoGenerated;
+ protected int primaryKeyPosition;
+ protected ARecordType origRecordTypeForAutoGeneratedPK;
public NtDelimitedDataTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
- char fieldDelimiter) {
+ char fieldDelimiter, boolean isPKAutoGenerated,
+ int primaryKeyposition, ARecordType origRecordTypeForAutoGeneratedPK) {
this.recordType = recordType;
this.valueParserFactories = valueParserFactories;
this.fieldDelimiter = fieldDelimiter;
+ this.isPKAutoGenerated = isPKAutoGenerated;
+ this.primaryKeyPosition = primaryKeyposition;
+ this.origRecordTypeForAutoGeneratedPK = origRecordTypeForAutoGeneratedPK;
}
@Override
public ITupleParser createTupleParser(final IHyracksTaskContext ctx) throws HyracksDataException {
- return new DelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter);
+ return new DelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter,
+ isPKAutoGenerated, primaryKeyPosition, origRecordTypeForAutoGeneratedPK);
}
}
diff --git a/asterix-runtime/src/main/resources/adm.grammar b/asterix-runtime/src/main/resources/adm.grammar
index 06614d0..b9db2e7 100644
--- a/asterix-runtime/src/main/resources/adm.grammar
+++ b/asterix-runtime/src/main/resources/adm.grammar
@@ -30,7 +30,8 @@
INTERVAL_DATE_CONS = string(interval-date)
INTERVAL_DATETIME_CONS = string(interval-datetime)
YEAR_MONTH_DURATION_CONS = string(year-month-duration)
-DAY_TIME_DURATION_CONS = string(day-time-duration)
+DAY_TIME_DURATION_CONS = string(day-time-duration)
+UUID_CONS = string(uuid)
NULL_LITERAL = string(null)
TRUE_LITERAL = string(true)
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
index 1f5de77..3593116 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/GenericSocketFeedAdapterFactory.java
@@ -78,10 +78,10 @@
}
@Override
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
this.configuration = configuration;
outputType = (ARecordType) outputType;
- this.configureFormat(outputType);
+ this.configureFormat(outputType, false, -1, null);
this.configureSockets(configuration);
}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index 8f169f2..60cf924 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -3,9 +3,9 @@
* 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.
@@ -38,7 +38,7 @@
public RateControlledFileSystemBasedAdapter(ARecordType atype, Map<String, String> configuration,
FileSystemBasedAdapter coreAdapter, String format, ITupleParserFactory parserFactory,
IHyracksTaskContext ctx) throws Exception {
- super(parserFactory, atype, ctx);
+ super(parserFactory, atype, ctx, false, -1);
this.coreAdapter = coreAdapter;
}
@@ -51,7 +51,7 @@
public void stop() {
((RateControlledTupleParser) tupleParser).stop();
}
-
+
@Override
public DataExchangeMode getDataExchangeMode() {
return DataExchangeMode.PULL;
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index d3b2120..0306a80 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -102,7 +102,7 @@
}
@Override
- public void configure(Map<String, String> configuration, ARecordType recordType) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType recordType, boolean isPKAutoGenerated, List<String> primaryKeys) throws Exception {
this.configuration = configuration;
checkRequiredArgs(configuration);
String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
@@ -116,7 +116,7 @@
}
format = configuration.get(KEY_FORMAT);
adapterFactory = (IGenericAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
- adapterFactory.configure(configuration, recordType);
+ adapterFactory.configure(configuration, recordType, isPKAutoGenerated, primaryKeys);
atype = (ARecordType) recordType;
configureFormat();
@@ -205,7 +205,7 @@
dataParser = new ADMDataParser();
break;
case DELIMITED_DATA:
- dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter);
+ dataParser = new DelimitedDataParser(recordType, valueParserFactories, delimiter, false, -1, null);
break;
}
return new RateControlledTupleParser(ctx, recordType, dataParser, configuration);
@@ -224,7 +224,7 @@
public RateControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
Map<String, String> configuration) throws HyracksDataException {
- super(ctx, recType);
+ super(ctx, recType, false, -1, null);
this.dataParser = dataParser;
String propValue = configuration.get(INTER_TUPLE_INTERVAL);
if (propValue != null) {
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
index f21a740..0b39f0c 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
@@ -3,9 +3,9 @@
* 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.
@@ -103,6 +103,6 @@
}
fileSplits = fileSplitsValue.trim().split(",");
genericSocketAdapterFactory = new GenericSocketFeedAdapterFactory();
- genericSocketAdapterFactory.configure(configuration, outputType);
+ genericSocketAdapterFactory.configure(configuration, outputType, false, null);
}
}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
index 8f24dec..4ec4650 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/TwitterFirehoseFeedAdapterFactory.java
@@ -47,7 +47,7 @@
private static final String KEY_INGESTION_CARDINALITY = "ingestion-cardinality";
/*
- * The absolute locations where ingestion operator instances will be places.
+ * The absolute locations where ingestion operator instances will be places.
*/
private static final String KEY_INGESTION_LOCATIONS = "ingestion-location";
@@ -72,7 +72,7 @@
public void configure(Map<String, String> configuration) throws Exception {
configuration.put(KEY_FORMAT, FORMAT_ADM);
this.configuration = configuration;
- this.configureFormat(initOutputType());
+ this.configureFormat(initOutputType(), false, -1, null);
}
@Override